From 091eb9b349ea2a383affd3089500ed855c5e4e17 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Fri, 20 Oct 2023 09:52:37 -0400 Subject: [PATCH] KAFKA-15428: Cluster-wide dynamic log adjustments for Connect (#14538) Reviewers: Greg Harris , Yang Yang , Yash Mayya --- checkstyle/import-control.xml | 4 +- checkstyle/suppressions.xml | 2 + .../MirrorConnectorsIntegrationBaseTest.java | 4 +- .../kafka/connect/cli/ConnectStandalone.java | 2 +- .../kafka/connect/runtime/AbstractHerder.java | 32 +- .../apache/kafka/connect/runtime/Herder.java | 34 + .../apache/kafka/connect/runtime/Loggers.java | 215 ++++ .../distributed/DistributedHerder.java | 12 +- .../runtime/rest/ConnectRestServer.java | 2 +- .../runtime/rest/entities/LoggerLevel.java | 68 ++ .../rest/resources/LoggingResource.java | 164 +-- .../runtime/standalone/StandaloneHerder.java | 21 +- .../connect/storage/ConfigBackingStore.java | 15 + .../storage/KafkaConfigBackingStore.java | 56 + .../storage/MemoryConfigBackingStore.java | 5 + .../ConnectWorkerIntegrationTest.java | 2 +- .../ConnectorRestartApiIntegrationTest.java | 2 +- .../ExactlyOnceSourceIntegrationTest.java | 8 +- .../StandaloneWorkerIntegrationTest.java | 205 ++++ .../connect/runtime/AbstractHerderTest.java | 95 +- .../kafka/connect/runtime/LoggersTest.java | 246 +++++ .../runtime/rest/ConnectRestServerTest.java | 27 +- .../rest/resources/LoggingResourceTest.java | 238 ++--- .../standalone/StandaloneHerderTest.java | 3 +- .../storage/KafkaConfigBackingStoreTest.java | 65 ++ .../util/clusters/ConnectAssertions.java | 618 +++++++++++ .../util/clusters/EmbeddedConnect.java | 956 ++++++++++++++++++ .../util/clusters/EmbeddedConnectBuilder.java | 84 ++ .../util/clusters/EmbeddedConnectCluster.java | 918 +---------------- .../EmbeddedConnectClusterAssertions.java | 599 +---------- .../clusters/EmbeddedConnectStandalone.java | 140 +++ gradle/spotbugs-exclude.xml | 7 + tests/kafkatest/services/connect.py | 15 + .../tests/connect/connect_distributed_test.py | 217 +++- 34 files changed, 3254 insertions(+), 1827 deletions(-) create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/LoggerLevel.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/StandaloneWorkerIntegrationTest.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/runtime/LoggersTest.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/ConnectAssertions.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectBuilder.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectStandalone.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index ea41f587c92..46774912418 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -496,6 +496,7 @@ + @@ -505,9 +506,6 @@ - - - diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index dfbe0286322..bd54231ffe5 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -139,6 +139,8 @@ files="Worker(SinkTask|SourceTask|Coordinator).java"/> + diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index 6ea080a59a3..0f3a189b05e 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -204,7 +204,7 @@ public class MirrorConnectorsIntegrationBaseTest { .brokerProps(primaryBrokerProps) .workerProps(primaryWorkerProps) .maskExitProcedures(false) - .clientConfigs(additionalPrimaryClusterClientsConfigs) + .clientProps(additionalPrimaryClusterClientsConfigs) .build(); backup = new EmbeddedConnectCluster.Builder() @@ -214,7 +214,7 @@ public class MirrorConnectorsIntegrationBaseTest { .brokerProps(backupBrokerProps) .workerProps(backupWorkerProps) .maskExitProcedures(false) - .clientConfigs(additionalBackupClusterClientsConfigs) + .clientProps(additionalBackupClusterClientsConfigs) .build(); primary.start(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java index e8a3dae46f3..c1977de3df9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java @@ -55,7 +55,7 @@ import java.util.Map; public class ConnectStandalone extends AbstractConnectCli { private static final Logger log = LoggerFactory.getLogger(ConnectStandalone.class); - protected ConnectStandalone(String... args) { + public ConnectStandalone(String... args) { super(args); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index 68027372662..662c269c7c9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.config.ConfigDef.ConfigKey; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigTransformer; import org.apache.kafka.common.config.ConfigValue; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; @@ -43,6 +44,7 @@ import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; 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.ConnectorType; +import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel; import org.apache.kafka.connect.runtime.rest.entities.Message; import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; import org.apache.kafka.connect.sink.SinkConnector; @@ -56,6 +58,7 @@ import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.transforms.predicates.Predicate; import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.log4j.Level; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,6 +75,7 @@ import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.LinkedList; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -120,6 +124,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con private final ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy; protected volatile boolean running = false; private final ExecutorService connectorExecutor; + protected final Loggers loggers; private final ConcurrentMap tempConnectors = new ConcurrentHashMap<>(); @@ -128,7 +133,8 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con String kafkaClusterId, StatusBackingStore statusBackingStore, ConfigBackingStore configBackingStore, - ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) { + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + Time time) { this.worker = worker; this.worker.herder = this; this.workerId = workerId; @@ -137,6 +143,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con this.configBackingStore = configBackingStore; this.connectorClientConfigOverridePolicy = connectorClientConfigOverridePolicy; this.connectorExecutor = Executors.newCachedThreadPool(); + this.loggers = new Loggers(time); } @Override @@ -917,4 +924,27 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con * @param cb callback to invoke upon completion */ protected abstract void modifyConnectorOffsets(String connName, Map, Map> offsets, Callback cb); + + @Override + public LoggerLevel loggerLevel(String logger) { + return loggers.level(logger); + } + + @Override + public Map allLoggerLevels() { + return loggers.allLevels(); + } + + @Override + public List setWorkerLoggerLevel(String namespace, String desiredLevelStr) { + Level level = Level.toLevel(desiredLevelStr.toUpperCase(Locale.ROOT), null); + + if (level == null) { + log.warn("Ignoring request to set invalid level '{}' for namespace {}", desiredLevelStr, namespace); + return Collections.emptyList(); + } + + return loggers.setLevel(namespace, level); + } + } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java index a6f9adfced2..566a5c4c096 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java @@ -25,6 +25,7 @@ import org.apache.kafka.connect.runtime.rest.entities.ConfigKeyInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; 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.LoggerLevel; import org.apache.kafka.connect.runtime.rest.entities.Message; import org.apache.kafka.connect.runtime.rest.entities.TaskInfo; import org.apache.kafka.connect.storage.StatusBackingStore; @@ -318,6 +319,39 @@ public interface Herder { */ void resetConnectorOffsets(String connName, Callback cb); + /** + * Get the level for a logger. + * @param logger the name of the logger to retrieve the level for; may not be null + * @return the level for the logger, or null if no logger with the given name exists + */ + LoggerLevel loggerLevel(String logger); + + /** + * Get the levels for all known loggers. + * @return a map of logger name to {@link LoggerLevel}; may be empty, but never null + */ + Map allLoggerLevels(); + + /** + * Set the level for a logging namespace (i.e., a specific logger and all of its children) on this + * worker. Changes should only last over the lifetime of the worker, and should be wiped if/when + * the worker is restarted. + * @param namespace the logging namespace to alter; may not be null + * @param level the new level to set for the namespace; may not be null + * @return all loggers that were affected by this action; may be empty (including if the specified + * level is not a valid logging level), but never null + */ + List setWorkerLoggerLevel(String namespace, String level); + + /** + * Set the level for a logging namespace (i.e., a specific logger and all of its children) for all workers + * in the cluster. Changes should only last over the lifetime of workers, and should be wiped if/when + * workers are restarted. + * @param namespace the logging namespace to alter; may not be null + * @param level the new level to set for the namespace; may not be null + */ + void setClusterLoggerLevel(String namespace, String level); + enum ConfigReloadAction { NONE, RESTART diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java new file mode 100644 index 00000000000..b0d58871c38 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java @@ -0,0 +1,215 @@ +/* + * 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 org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +/** + * Manages logging levels on a single worker. Supports dynamic adjustment and querying + * of logging levels. + *

+ * This class is thread-safe; concurrent calls to all of its public methods from any number + * of threads are permitted. + */ +public class Loggers { + + private static final Logger log = LoggerFactory.getLogger(Loggers.class); + + /** + * Log4j uses "root" (case-insensitive) as name of the root logger. + */ + private static final String ROOT_LOGGER_NAME = "root"; + + private final Time time; + private final Map lastModifiedTimes; + + public Loggers(Time time) { + this.time = time; + this.lastModifiedTimes = new HashMap<>(); + } + + /** + * Retrieve the current level for a single logger. + * @param logger the name of the logger to retrieve the level for; may not be null + * @return the current level (falling back on the effective level if necessary) of the logger, + * or null if no logger with the specified name exists + */ + public synchronized LoggerLevel level(String logger) { + Objects.requireNonNull(logger, "Logger may not be null"); + + org.apache.log4j.Logger foundLogger = null; + if (ROOT_LOGGER_NAME.equalsIgnoreCase(logger)) { + foundLogger = rootLogger(); + } else { + Enumeration en = currentLoggers(); + // search within existing loggers for the given name. + // using LogManger.getLogger() will create a logger if it doesn't exist + // (potential leak since these don't get cleaned up). + while (en.hasMoreElements()) { + org.apache.log4j.Logger l = en.nextElement(); + if (logger.equals(l.getName())) { + foundLogger = l; + break; + } + } + } + + if (foundLogger == null) { + log.warn("Unable to find level for logger {}", logger); + return null; + } + + return loggerLevel(foundLogger); + } + + /** + * Retrieve the current levels of all known loggers + * @return the levels of all known loggers; may be empty, but never null + */ + public synchronized Map allLevels() { + Map result = new TreeMap<>(); + + Enumeration enumeration = currentLoggers(); + Collections.list(enumeration) + .stream() + .filter(logger -> logger.getLevel() != null) + .forEach(logger -> result.put(logger.getName(), loggerLevel(logger))); + + org.apache.log4j.Logger root = rootLogger(); + if (root.getLevel() != null) { + result.put(ROOT_LOGGER_NAME, loggerLevel(root)); + } + + return result; + } + + /** + * Set the level for the specified logger and all of its children + * @param namespace the name of the logger to adjust along with its children; may not be null + * @param level the level to set for the logger and its children; may not be null + * @return all loggers that were affected by this action, sorted by their natural ordering; + * may be empty, but never null + */ + public synchronized List setLevel(String namespace, Level level) { + Objects.requireNonNull(namespace, "Logging namespace may not be null"); + Objects.requireNonNull(level, "Level may not be null"); + + log.info("Setting level of namespace {} and children to {}", namespace, level); + List childLoggers = loggers(namespace); + + List result = new ArrayList<>(); + for (org.apache.log4j.Logger logger: childLoggers) { + setLevel(logger, level); + result.add(logger.getName()); + } + Collections.sort(result); + + return result; + } + + /** + * Retrieve all known loggers within a given namespace, creating an ancestor logger for that + * namespace if one does not already exist + * @param namespace the namespace that the loggers should fall under; may not be null + * @return all loggers that fall under the given namespace; never null, and will always contain + * at least one logger (the ancestor logger for the namespace) + */ + private synchronized List loggers(String namespace) { + Objects.requireNonNull(namespace, "Logging namespace may not be null"); + + if (ROOT_LOGGER_NAME.equalsIgnoreCase(namespace)) { + List result = Collections.list(currentLoggers()); + result.add(rootLogger()); + return result; + } + + List result = new ArrayList<>(); + org.apache.log4j.Logger ancestorLogger = lookupLogger(namespace); + Enumeration en = currentLoggers(); + boolean present = false; + while (en.hasMoreElements()) { + org.apache.log4j.Logger current = en.nextElement(); + if (current.getName().startsWith(namespace)) { + result.add(current); + } + if (namespace.equals(current.getName())) { + present = true; + } + } + + if (!present) { + result.add(ancestorLogger); + } + + return result; + } + + // visible for testing + org.apache.log4j.Logger lookupLogger(String logger) { + return LogManager.getLogger(logger); + } + + @SuppressWarnings("unchecked") + // visible for testing + Enumeration currentLoggers() { + return LogManager.getCurrentLoggers(); + } + + // visible for testing + org.apache.log4j.Logger rootLogger() { + return LogManager.getRootLogger(); + } + + private void setLevel(org.apache.log4j.Logger logger, Level level) { + Level currentLevel = logger.getLevel(); + if (currentLevel == null) + currentLevel = logger.getEffectiveLevel(); + + if (level.equals(currentLevel)) { + log.debug("Skipping update for logger {} since its level is already {}", logger.getName(), level); + return; + } + + log.debug("Setting level of logger {} (excluding children) to {}", logger.getName(), level); + logger.setLevel(level); + lastModifiedTimes.put(logger.getName(), time.milliseconds()); + } + + private LoggerLevel loggerLevel(org.apache.log4j.Logger logger) { + Level level = logger.getLevel(); + if (level == null) + level = logger.getEffectiveLevel(); + + Long lastModified = lastModifiedTimes.get(logger.getName()); + return new LoggerLevel(Objects.toString(level), lastModified); + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 94d19207cdc..48b5fad3423 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -279,7 +279,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable { ExecutorService forwardRequestExecutor, // https://github.com/mockito/mockito/issues/2601 explains why we can't use varargs here AutoCloseable[] uponShutdown) { - super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy); + super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy, time); this.time = time; this.herderMetrics = new HerderMetrics(metrics); @@ -1612,6 +1612,11 @@ public class DistributedHerder extends AbstractHerder implements Runnable { return true; } + @Override + public void setClusterLoggerLevel(String namespace, String level) { + configBackingStore.putLoggerLevel(namespace, level); + } + // Should only be called from work thread, so synchronization should not be needed private boolean isLeader() { return assignment != null && member.memberId().equals(assignment.leader()); @@ -2376,6 +2381,11 @@ public class DistributedHerder extends AbstractHerder implements Runnable { } member.wakeup(); } + + @Override + public void onLoggingLevelUpdate(String namespace, String level) { + setWorkerLoggerLevel(namespace, level); + } } class DistributedHerderRequest implements HerderRequest, Comparable { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java index ca9eb731c0c..6cef19c22b8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java @@ -57,7 +57,7 @@ public class ConnectRestServer extends RestServer { @Override protected Collection adminResources() { return Arrays.asList( - new LoggingResource() + new LoggingResource(herder) ); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/LoggerLevel.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/LoggerLevel.java new file mode 100644 index 00000000000..4a9a6be32e5 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/LoggerLevel.java @@ -0,0 +1,68 @@ +/* + * 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.rest.entities; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public class LoggerLevel { + + private final String level; + private final Long lastModified; + + public LoggerLevel( + @JsonProperty("level") String level, + @JsonProperty("last_modified") Long lastModified + ) { + this.level = Objects.requireNonNull(level, "level may not be null"); + this.lastModified = lastModified; + } + + @JsonProperty + public String level() { + return level; + } + + @JsonProperty("last_modified") + public Long lastModified() { + return lastModified; + } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + LoggerLevel that = (LoggerLevel) o; + return level.equals(that.level) && Objects.equals(lastModified, that.lastModified); + } + + @Override + public int hashCode() { + return Objects.hash(level, lastModified); + } + + @Override + public String toString() { + return "LoggerLevel{" + + "level='" + level + '\'' + + ", lastModified=" + lastModified + + '}'; + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResource.java index 812cf696563..b215fe72adb 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResource.java @@ -17,28 +17,28 @@ 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.errors.NotFoundException; +import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel; import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; import org.apache.log4j.Level; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; +import org.slf4j.LoggerFactory; import javax.ws.rs.Consumes; +import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; import javax.ws.rs.PUT; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Enumeration; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Objects; -import java.util.TreeMap; /** * A set of endpoints to adjust the log levels of runtime loggers. @@ -48,10 +48,16 @@ import java.util.TreeMap; @Consumes(MediaType.APPLICATION_JSON) public class LoggingResource implements ConnectResource { - /** - * Log4j uses "root" (case-insensitive) as name of the root logger. - */ - private static final String ROOT_LOGGER_NAME = "root"; + private static final org.slf4j.Logger log = LoggerFactory.getLogger(LoggingResource.class); + + private static final String WORKER_SCOPE = "worker"; + private static final String CLUSTER_SCOPE = "cluster"; + + private final Herder herder; + + public LoggingResource(Herder herder) { + this.herder = herder; + } @Override public void requestTimeout(long requestTimeoutMs) { @@ -67,19 +73,7 @@ public class LoggingResource implements ConnectResource { @Path("/") @Operation(summary = "List the current loggers that have their levels explicitly set and their log levels") public Response listLoggers() { - Map> loggers = new TreeMap<>(); - Enumeration enumeration = currentLoggers(); - Collections.list(enumeration) - .stream() - .filter(logger -> logger.getLevel() != null) - .forEach(logger -> loggers.put(logger.getName(), levelToMap(logger))); - - Logger root = rootLogger(); - if (root.getLevel() != null) { - loggers.put(ROOT_LOGGER_NAME, levelToMap(root)); - } - - return Response.ok(loggers).build(); + return Response.ok(herder.allLoggerLevels()).build(); } /** @@ -94,34 +88,17 @@ public class LoggingResource implements ConnectResource { public Response getLogger(final @PathParam("logger") String namedLogger) { Objects.requireNonNull(namedLogger, "require non-null name"); - Logger logger = null; - if (ROOT_LOGGER_NAME.equalsIgnoreCase(namedLogger)) { - logger = rootLogger(); - } else { - Enumeration en = currentLoggers(); - // search within existing loggers for the given name. - // using LogManger.getLogger() will create a logger if it doesn't exist - // (potential leak since these don't get cleaned up). - while (en.hasMoreElements()) { - Logger l = en.nextElement(); - if (namedLogger.equals(l.getName())) { - logger = l; - break; - } - } - } - if (logger == null) { + LoggerLevel loggerLevel = herder.loggerLevel(namedLogger); + if (loggerLevel == null) throw new NotFoundException("Logger " + namedLogger + " not found."); - } else { - return Response.ok(effectiveLevelToMap(logger)).build(); - } - } + return Response.ok(loggerLevel).build(); + } /** * Adjust level of a named logger. If the name corresponds to an ancestor, then the log level is applied to all child loggers. * - * @param namedLogger name of the logger + * @param namespace name of the logger * @param levelMap a map that is expected to contain one key 'level', and a value that is one of the log4j levels: * DEBUG, ERROR, FATAL, INFO, TRACE, WARN * @return names of loggers whose levels were modified @@ -129,87 +106,36 @@ public class LoggingResource implements ConnectResource { @PUT @Path("/{logger}") @Operation(summary = "Set the log level for the specified logger") - public Response setLevel(final @PathParam("logger") String namedLogger, - final Map levelMap) { - String desiredLevelStr = levelMap.get("level"); - if (desiredLevelStr == null) { - throw new BadRequestException("Desired 'level' parameter was not specified in request."); + @SuppressWarnings("fallthrough") + public Response setLevel(final @PathParam("logger") String namespace, + final Map levelMap, + @DefaultValue("worker") @QueryParam("scope") @Parameter(description = "The scope for the logging modification (single-worker, cluster-wide, etc.)") String scope) { + if (scope == null) { + log.warn("Received null scope in request to adjust logging level; will default to {}", WORKER_SCOPE); + scope = WORKER_SCOPE; } - Level level = Level.toLevel(desiredLevelStr.toUpperCase(Locale.ROOT), null); - if (level == null) { - throw new NotFoundException("invalid log level '" + desiredLevelStr + "'."); - } - - List childLoggers; - if (ROOT_LOGGER_NAME.equalsIgnoreCase(namedLogger)) { - childLoggers = Collections.list(currentLoggers()); - childLoggers.add(rootLogger()); - } else { - childLoggers = new ArrayList<>(); - Logger ancestorLogger = lookupLogger(namedLogger); - Enumeration en = currentLoggers(); - boolean present = false; - while (en.hasMoreElements()) { - Logger current = en.nextElement(); - if (current.getName().startsWith(namedLogger)) { - childLoggers.add(current); - } - if (namedLogger.equals(current.getName())) { - present = true; - } - } - if (!present) { - childLoggers.add(ancestorLogger); - } + String levelString = levelMap.get("level"); + if (levelString == null) { + throw new BadRequestException("Desired 'level' parameter was not specified in request."); } - List modifiedLoggerNames = new ArrayList<>(); - for (Logger logger: childLoggers) { - logger.setLevel(level); - modifiedLoggerNames.add(logger.getName()); + // Make sure that this is a valid level + Level level = Level.toLevel(levelString.toUpperCase(Locale.ROOT), null); + if (level == null) { + throw new NotFoundException("invalid log level '" + levelString + "'."); } - Collections.sort(modifiedLoggerNames); - return Response.ok(modifiedLoggerNames).build(); - } - - protected Logger lookupLogger(String namedLogger) { - return LogManager.getLogger(namedLogger); - } - - @SuppressWarnings("unchecked") - protected Enumeration currentLoggers() { - return LogManager.getCurrentLoggers(); - } - - protected Logger rootLogger() { - return LogManager.getRootLogger(); - } - - /** - * - * Map representation of a logger's effective log level. - * - * @param logger a non-null log4j logger - * @return a singleton map whose key is level and the value is the string representation of the logger's effective log level. - */ - private static Map effectiveLevelToMap(Logger logger) { - Level level = logger.getLevel(); - if (level == null) { - level = logger.getEffectiveLevel(); + switch (scope.toLowerCase(Locale.ROOT)) { + default: + log.warn("Received invalid scope '{}' in request to adjust logging level; will default to {}", scope, WORKER_SCOPE); + case WORKER_SCOPE: + List affectedLoggers = herder.setWorkerLoggerLevel(namespace, levelString); + return Response.ok(affectedLoggers).build(); + case CLUSTER_SCOPE: + herder.setClusterLoggerLevel(namespace, levelString); + return Response.noContent().build(); } - return Collections.singletonMap("level", String.valueOf(level)); } - /** - * - * Map representation of a logger's log level. - * - * @param logger a non-null log4j logger - * @return a singleton map whose key is level and the value is the string representation of the logger's log level. - */ - private static Map levelToMap(Logger logger) { - return Collections.singletonMap("level", String.valueOf(logger.getLevel())); - } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index babb157f772..0da89b2f668 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.runtime.standalone; import org.apache.kafka.common.utils.ThreadUtils; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.errors.AlreadyExistsException; import org.apache.kafka.connect.errors.ConnectException; @@ -82,7 +83,9 @@ public class StandaloneHerder extends AbstractHerder { kafkaClusterId, new MemoryStatusBackingStore(), new MemoryConfigBackingStore(worker.configTransformer()), - connectorClientConfigOverridePolicy); + connectorClientConfigOverridePolicy, + Time.SYSTEM + ); } // visible for testing @@ -91,8 +94,9 @@ public class StandaloneHerder extends AbstractHerder { String kafkaClusterId, StatusBackingStore statusBackingStore, MemoryConfigBackingStore configBackingStore, - ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) { - super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy); + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, + Time time) { + super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy, time); this.configState = ClusterConfigState.EMPTY; this.requestExecutorService = Executors.newSingleThreadScheduledExecutor(); configBackingStore.setUpdateListener(new ConfigUpdateListener()); @@ -404,6 +408,12 @@ public class StandaloneHerder extends AbstractHerder { return true; } + @Override + public void setClusterLoggerLevel(String namespace, String level) { + // In standalone mode, this single worker is the entire cluster + setWorkerLoggerLevel(namespace, level); + } + private void startConnector(String connName, Callback onStart) { Map connConfigs = configState.connectorConfig(connName); TargetState targetState = configState.targetState(connName); @@ -544,6 +554,11 @@ public class StandaloneHerder extends AbstractHerder { public void onRestartRequest(RestartRequest restartRequest) { // no-op } + + @Override + public void onLoggingLevelUpdate(String namespace, String level) { + // no-op + } } static class StandaloneHerderRequest implements HerderRequest { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java index e867087593a..c869c545f80 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java @@ -122,6 +122,14 @@ public interface ConfigBackingStore { default void claimWritePrivileges() { } + /** + * Emit a new level for the specified logging namespace (and all of its children). This level should + * be applied by all workers currently in the cluster, but not to workers that join after it is stored. + * @param namespace the namespace to adjust; may not be null + * @param level the new level for the namespace; may not be null + */ + void putLoggerLevel(String namespace, String level); + /** * Set an update listener to get notifications when there are new records written to the backing store. * @param listener non-null listener @@ -164,6 +172,13 @@ public interface ConfigBackingStore { * @param restartRequest the {@link RestartRequest restart request} */ void onRestartRequest(RestartRequest restartRequest); + + /** + * Invoked when a dynamic log level adjustment has been read + * @param namespace the namespace to adjust; never null + * @param level the level to set the namespace to; never null + */ + void onLoggingLevelUpdate(String namespace, String level); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java index ad8608b76cd..a95c8249942 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java @@ -271,6 +271,14 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme .field(ONLY_FAILED_FIELD_NAME, Schema.BOOLEAN_SCHEMA) .build(); + public static final String LOGGER_CLUSTER_PREFIX = "logger-cluster-"; + public static String LOGGER_CLUSTER_KEY(String namespace) { + return LOGGER_CLUSTER_PREFIX + namespace; + } + public static final Schema LOGGER_LEVEL_V0 = SchemaBuilder.struct() + .field("level", Schema.STRING_SCHEMA) + .build(); + // Visible for testing static final long READ_WRITE_TOTAL_TIMEOUT_MS = 30000; @@ -732,6 +740,20 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme } } + @Override + public void putLoggerLevel(String namespace, String level) { + log.debug("Writing level {} for logging namespace {} to Kafka", level, namespace); + Struct value = new Struct(LOGGER_LEVEL_V0); + value.put("level", level); + byte[] serializedValue = converter.fromConnectData(topic, value.schema(), value); + try { + configLog.sendWithReceipt(LOGGER_CLUSTER_KEY(namespace), serializedValue).get(READ_WRITE_TOTAL_TIMEOUT_MS, TimeUnit.MILLISECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + log.error("Failed to write logger level to Kafka", e); + throw new ConnectException("Error writing logger level to Kafka", e); + } + } + // package private for testing KafkaBasedLog setupAndCreateKafkaBasedLog(String topic, final WorkerConfig config) { String clusterId = config.kafkaClusterId(); @@ -901,6 +923,9 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme processTaskCountRecord(connectorName, value); } else if (record.key().equals(SESSION_KEY_KEY)) { processSessionKeyRecord(value); + } else if (record.key().startsWith(LOGGER_CLUSTER_PREFIX)) { + String loggingNamespace = record.key().substring(LOGGER_CLUSTER_PREFIX.length()); + processLoggerLevelRecord(loggingNamespace, value); } else { log.error("Discarding config update record with invalid key: {}", record.key()); } @@ -1185,6 +1210,37 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme updateListener.onSessionKeyUpdate(KafkaConfigBackingStore.this.sessionKey); } + private void processLoggerLevelRecord(String namespace, SchemaAndValue value) { + if (value.value() == null) { + log.error("Ignoring logging level for namespace {} because it is unexpectedly null", namespace); + return; + } + if (!(value.value() instanceof Map)) { + log.error("Ignoring logging level for namespace {} because the value is not a Map but is {}", namespace, className(value.value())); + return; + } + + @SuppressWarnings("unchecked") + Map valueAsMap = (Map) value.value(); + + Object level = valueAsMap.get("level"); + if (!(level instanceof String)) { + log.error("Invalid data for logging level key 'level' field with namespace {}; should be a String but it is {}", namespace, className(level)); + return; + } + + if (started) { + updateListener.onLoggingLevelUpdate(namespace, (String) level); + } else { + // TRACE level since there may be many of these records in the config topic + log.trace( + "Ignoring old logging level {} for namespace {} that was writen to the config topic before this worker completed startup", + level, + namespace + ); + } + } + private ConnectorTaskId parseTaskId(String key) { String[] parts = key.split("-"); if (parts.length < 3) return null; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java index dcdfd71296b..52c360c3b33 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java @@ -169,6 +169,11 @@ public class MemoryConfigBackingStore implements ConfigBackingStore { // no-op } + @Override + public void putLoggerLevel(String namespace, String level) { + // no-op + } + @Override public synchronized void setUpdateListener(UpdateListener listener) { this.updateListener = listener; 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 42c3831faf4..cd9b9c05171 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 @@ -54,9 +54,9 @@ import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CO import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; 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.apache.kafka.connect.util.clusters.ConnectAssertions.CONNECTOR_SETUP_DURATION_MS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorRestartApiIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorRestartApiIntegrationTest.java index e399eee2ae9..26b4eb11417 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorRestartApiIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorRestartApiIntegrationTest.java @@ -53,7 +53,7 @@ import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CO import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; 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.apache.kafka.connect.util.clusters.ConnectAssertions.CONNECTOR_SETUP_DURATION_MS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java index 26b2d7cba16..4eac236810a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java @@ -48,7 +48,7 @@ import org.apache.kafka.connect.source.SourceRecord; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; -import org.apache.kafka.connect.util.clusters.EmbeddedConnectClusterAssertions; +import org.apache.kafka.connect.util.clusters.ConnectAssertions; import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; import org.apache.kafka.test.IntegrationTest; import org.junit.After; @@ -650,7 +650,7 @@ public class ExactlyOnceSourceIntegrationTest { final String globalOffsetsTopic = "connect-worker-offsets-topic"; workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, globalOffsetsTopic); - connectBuilder.clientConfigs(superUserClientConfig); + connectBuilder.clientProps(superUserClientConfig); startConnect(); @@ -1095,7 +1095,7 @@ public class ExactlyOnceSourceIntegrationTest { private void assertConnectorStarted(StartAndStopLatch connectorStart) throws InterruptedException { assertTrue("Connector and tasks did not finish startup in time", connectorStart.await( - EmbeddedConnectClusterAssertions.CONNECTOR_SETUP_DURATION_MS, + ConnectAssertions.CONNECTOR_SETUP_DURATION_MS, TimeUnit.MILLISECONDS ) ); @@ -1105,7 +1105,7 @@ public class ExactlyOnceSourceIntegrationTest { assertTrue( "Connector and tasks did not finish shutdown in time", connectorStop.await( - EmbeddedConnectClusterAssertions.CONNECTOR_SHUTDOWN_DURATION_MS, + ConnectAssertions.CONNECTOR_SHUTDOWN_DURATION_MS, TimeUnit.MILLISECONDS ) ); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/StandaloneWorkerIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/StandaloneWorkerIntegrationTest.java new file mode 100644 index 00000000000..ea938f9a4f6 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/StandaloneWorkerIntegrationTest.java @@ -0,0 +1,205 @@ +/* + * 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.integration; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectStandalone; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +@Category(IntegrationTest.class) +public class StandaloneWorkerIntegrationTest { + + private EmbeddedConnectStandalone connect; + + @Before + public void setup() { + connect = new EmbeddedConnectStandalone.Builder() + .build(); + connect.start(); + } + + @After + public void cleanup() { + connect.stop(); + } + + @Test + public void testDynamicLogging() { + Map initialLevels = connect.allLogLevels(); + assertFalse("Connect REST API did not list any known loggers", initialLevels.isEmpty()); + Map invalidModifiedLoggers = Utils.filterMap( + initialLevels, + StandaloneWorkerIntegrationTest::isModified + ); + assertEquals( + "No loggers should have a non-null last-modified timestamp", + Collections.emptyMap(), + invalidModifiedLoggers + ); + + // Tests with no scope + // The current level may match the first level we set the namespace to, + // so we issue a preliminary request with a different level to guarantee that a + // change takes place and that the last modified timestamp should be non-null + final String namespace1 = "org.apache.kafka.connect"; + final String level1 = "DEBUG"; + connect.setLogLevel(namespace1, "ERROR", null); + Map currentLevels = testSetLoggingLevel(namespace1, level1, null, initialLevels); + + // Tests with scope=worker + final String namespace2 = "org.apache.kafka.clients"; + final String level2 = "INFO"; + connect.setLogLevel(namespace2, "WARN", "worker"); + currentLevels = testSetLoggingLevel(namespace2, level2, "worker", currentLevels); + + LoggerLevel priorLoggerLevel = connect.getLogLevel(namespace2); + connect.setLogLevel(namespace2, level2, "worker"); + LoggerLevel currentLoggerLevel = connect.getLogLevel(namespace2); + assertEquals( + "Log level and last-modified timestamp should not be affected by consecutive identical requests", + priorLoggerLevel, + currentLoggerLevel + ); + + // Tests with scope=cluster + final String namespace3 = "org.apache.kafka.streams"; + final String level3 = "TRACE"; + connect.setLogLevel(namespace3, "DEBUG", "cluster"); + testSetLoggingLevel(namespace3, level3, "cluster", currentLevels); + } + + private Map testSetLoggingLevel( + String namespace, + String level, + String scope, + Map initialLevels + ) { + long requestTime = System.currentTimeMillis(); + List affectedLoggers = connect.setLogLevel(namespace, level, scope); + if ("cluster".equals(scope)) { + assertNull( + "Modifying log levels with scope=cluster should result in an empty response", + affectedLoggers + ); + } else { + assertTrue(affectedLoggers.contains(namespace)); + List invalidAffectedLoggers = affectedLoggers.stream() + .filter(l -> !l.startsWith(namespace)) + .collect(Collectors.toList()); + assertEquals( + "No loggers outside the namespace '" + namespace + + "' should have been included in the response for a request to modify that namespace", + Collections.emptyList(), + invalidAffectedLoggers + ); + } + + // Verify the information for this single logger + + LoggerLevel loggerLevel = connect.getLogLevel(namespace); + assertNotNull(loggerLevel); + assertEquals(level, loggerLevel.level()); + assertNotNull(loggerLevel.lastModified()); + assertTrue( + "Last-modified timestamp for logger level is " + loggerLevel.lastModified() + + ", which is before " + requestTime + ", the most-recent time the level was adjusted", + loggerLevel.lastModified() >= requestTime + ); + + // Verify information for all listed loggers + + Map newLevels = connect.allLogLevels(); + + Map invalidAffectedLoggerLevels = Utils.filterMap( + newLevels, + e -> hasNamespace(e, namespace) + && (!level(e).equals(level) + || !isModified(e) + || lastModified(e) < requestTime + ) + ); + assertEquals( + "At least one logger in the affected namespace '" + namespace + + "' does not have the expected level of '" + level + + "', has a null last-modified timestamp, or has a last-modified timestamp " + + "that is less recent than " + requestTime + + ", which is when the namespace was last adjusted", + Collections.emptyMap(), + invalidAffectedLoggerLevels + ); + + Set droppedLoggers = Utils.diff(HashSet::new, initialLevels.keySet(), newLevels.keySet()); + assertEquals( + "At least one logger was present in the listing of all loggers " + + "before the logging level for namespace '" + namespace + + "' was set to '" + level + + "' that is no longer present", + Collections.emptySet(), + droppedLoggers + ); + + Map invalidUnaffectedLoggerLevels = Utils.filterMap( + newLevels, + e -> !hasNamespace(e, namespace) && !e.getValue().equals(initialLevels.get(e.getKey())) + ); + assertEquals( + "At least one logger outside of the affected namespace '" + namespace + + "' has a different logging level or last-modified timestamp than it did " + + "before the namespace was set to level '" + level + + "'; none of these loggers should have been affected", + Collections.emptyMap(), + invalidUnaffectedLoggerLevels + ); + + return newLevels; + } + + private static boolean hasNamespace(Map.Entry entry, String namespace) { + return entry.getKey().startsWith(namespace); + } + + private static boolean isModified(Map.Entry entry) { + return lastModified(entry) != null; + } + + private static Long lastModified(Map.Entry entry) { + return entry.getValue().lastModified(); + } + + private static String level(Map.Entry entry) { + return entry.getValue().level(); + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index 6b06c43ce3d..eb86920adff 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.config.provider.DirectoryConfigProvider; import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredLoginCallbackHandler; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; @@ -176,9 +177,7 @@ public class AbstractHerderTest { @Test public void testConnectors() { - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(configStore.snapshot()).thenReturn(SNAPSHOT); assertEquals(Collections.singleton(CONN1), new HashSet<>(herder.connectors())); @@ -188,9 +187,7 @@ public class AbstractHerderTest { public void testConnectorClientConfigOverridePolicyClose() { SampleConnectorClientConfigOverridePolicy noneConnectorClientConfigOverridePolicy = new SampleConnectorClientConfigOverridePolicy(); - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(noneConnectorClientConfigOverridePolicy); herder.stopServices(); assertTrue(noneConnectorClientConfigOverridePolicy.isClosed()); @@ -200,9 +197,7 @@ public class AbstractHerderTest { public void testConnectorStatus() { ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0); - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(plugins.newConnector(anyString())).thenReturn(new SampleSourceConnector()); when(herder.plugins()).thenReturn(plugins); @@ -236,9 +231,7 @@ public class AbstractHerderTest { public void testConnectorStatusMissingPlugin() { ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0); - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(plugins.newConnector(anyString())).thenThrow(new ConnectException("Unable to find class")); when(herder.plugins()).thenReturn(plugins); @@ -269,9 +262,7 @@ public class AbstractHerderTest { @Test public void testConnectorInfo() { - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(plugins.newConnector(anyString())).thenReturn(new SampleSourceConnector()); when(herder.plugins()).thenReturn(plugins); @@ -288,9 +279,7 @@ public class AbstractHerderTest { @Test public void testPauseConnector() { - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(configStore.contains(CONN1)).thenReturn(true); @@ -301,9 +290,7 @@ public class AbstractHerderTest { @Test public void testResumeConnector() { - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(configStore.contains(CONN1)).thenReturn(true); @@ -314,9 +301,7 @@ public class AbstractHerderTest { @Test public void testConnectorInfoMissingPlugin() { - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(plugins.newConnector(anyString())).thenThrow(new ConnectException("No class found")); when(herder.plugins()).thenReturn(plugins); @@ -336,9 +321,7 @@ public class AbstractHerderTest { ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0); String workerId = "workerId"; - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); final ArgumentCaptor taskStatusArgumentCaptor = ArgumentCaptor.forClass(TaskStatus.class); doNothing().when(statusStore).putSafe(taskStatusArgumentCaptor.capture()); @@ -358,9 +341,7 @@ public class AbstractHerderTest { public void testBuildRestartPlanForUnknownConnector() { String connectorName = "UnknownConnector"; RestartRequest restartRequest = new RestartRequest(connectorName, false, true); - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(statusStore.get(connectorName)).thenReturn(null); @@ -415,9 +396,7 @@ public class AbstractHerderTest { taskStatuses.add(new TaskStatus(taskId1, AbstractStatus.State.RUNNING, workerId, generation)); taskStatuses.add(new TaskStatus(taskId2, AbstractStatus.State.FAILED, workerId, generation)); - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(herder.rawConfig(connectorName)).thenReturn(null); @@ -447,9 +426,7 @@ public class AbstractHerderTest { taskStatuses.add(new TaskStatus(taskId1, AbstractStatus.State.RUNNING, workerId, generation)); taskStatuses.add(new TaskStatus(taskId2, AbstractStatus.State.FAILED, workerId, generation)); - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(herder.rawConfig(connectorName)).thenReturn(null); @@ -1051,9 +1028,7 @@ public class AbstractHerderTest { Function pluginConfig, Optional baseConfig ) throws ClassNotFoundException { - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(plugins.pluginClass(pluginName)).then(invocation -> newPluginInstance.get().getClass()); when(plugins.newPlugin(anyString())).then(invocation -> newPluginInstance.get()); @@ -1073,9 +1048,7 @@ public class AbstractHerderTest { @Test(expected = NotFoundException.class) public void testGetConnectorConfigDefWithBadName() throws Exception { String connName = "AnotherPlugin"; - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(worker.getPlugins()).thenReturn(plugins); when(plugins.pluginClass(anyString())).thenThrow(new ClassNotFoundException()); herder.connectorPluginConfig(connName); @@ -1085,9 +1058,7 @@ public class AbstractHerderTest { @SuppressWarnings({"rawtypes", "unchecked"}) public void testGetConnectorConfigDefWithInvalidPluginType() throws Exception { String connName = "AnotherPlugin"; - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(worker.getPlugins()).thenReturn(plugins); when(plugins.pluginClass(anyString())).thenReturn((Class) Object.class); when(plugins.newPlugin(anyString())).thenReturn(new DirectoryConfigProvider()); @@ -1097,9 +1068,7 @@ public class AbstractHerderTest { @Test public void testGetConnectorTypeWithMissingPlugin() { String connName = "AnotherPlugin"; - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(worker.getPlugins()).thenReturn(plugins); when(plugins.newConnector(anyString())).thenThrow(new ConnectException("No class found")); assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Collections.singletonMap(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connName))); @@ -1107,26 +1076,20 @@ public class AbstractHerderTest { @Test public void testGetConnectorTypeWithNullConfig() { - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); assertEquals(ConnectorType.UNKNOWN, herder.connectorType(null)); } @Test public void testGetConnectorTypeWithEmptyConfig() { - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Collections.emptyMap())); } @Test public void testConnectorOffsetsConnectorNotFound() { when(configStore.snapshot()).thenReturn(SNAPSHOT); - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); FutureCallback cb = new FutureCallback<>(); herder.connectorOffsets("unknown-connector", cb); ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS)); @@ -1145,9 +1108,7 @@ public class AbstractHerderTest { workerCallback.getValue().onCompletion(null, offsets); return null; }).when(worker).connectorOffsets(eq(CONN1), eq(CONN1_CONFIG), workerCallback.capture()); - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(); when(configStore.snapshot()).thenReturn(SNAPSHOT); FutureCallback cb = new FutureCallback<>(); @@ -1210,9 +1171,7 @@ public class AbstractHerderTest { ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy, int countOfCallingNewConnector) { - AbstractHerder herder = mock(AbstractHerder.class, withSettings() - .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, connectorClientConfigOverridePolicy) - .defaultAnswer(CALLS_REAL_METHODS)); + AbstractHerder herder = testHerder(connectorClientConfigOverridePolicy); // Call to validateConnectorConfig when(worker.configTransformer()).thenReturn(transformer); @@ -1232,6 +1191,16 @@ public class AbstractHerderTest { return herder; } + private AbstractHerder testHerder() { + return testHerder(noneConnectorClientConfigOverridePolicy); + } + + private AbstractHerder testHerder(ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) { + return mock(AbstractHerder.class, withSettings() + .useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, connectorClientConfigOverridePolicy, Time.SYSTEM) + .defaultAnswer(CALLS_REAL_METHODS)); + } + private void mockValidationIsolation(String connectorClass, Connector connector) { when(plugins.newConnector(connectorClass)).thenReturn(connector); when(plugins.connectorLoader(connectorClass)).thenReturn(classLoader); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/LoggersTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/LoggersTest.java new file mode 100644 index 00000000000..184724ef255 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/LoggersTest.java @@ -0,0 +1,246 @@ +/* + * 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 org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel; +import org.apache.log4j.Hierarchy; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Vector; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; + +@RunWith(MockitoJUnitRunner.StrictStubs.class) +public class LoggersTest { + + private static final long INITIAL_TIME = 1696951712135L; + private Time time; + + @Before + public void setup() { + time = new MockTime(0, INITIAL_TIME, 0); + } + + @Test + public void testGetLoggersIgnoresNullLevels() { + Logger root = logger("root"); + + Logger a = logger("a"); + a.setLevel(null); + Logger b = logger("b"); + b.setLevel(Level.INFO); + + Loggers loggers = new TestLoggers(root, a, b); + + Map expectedLevels = Collections.singletonMap( + "b", + new LoggerLevel(Level.INFO.toString(), null) + ); + Map actualLevels = loggers.allLevels(); + assertEquals(expectedLevels, actualLevels); + } + + @Test + public void testGetLoggerFallsBackToEffectiveLogLevel() { + Logger root = logger("root"); + root.setLevel(Level.ERROR); + + Hierarchy hierarchy = new Hierarchy(root); + Logger a = hierarchy.getLogger("a"); + a.setLevel(null); + Logger b = hierarchy.getLogger("b"); + b.setLevel(Level.INFO); + + Loggers loggers = new TestLoggers(root, a, b); + + LoggerLevel expectedLevel = new LoggerLevel(Level.ERROR.toString(), null); + LoggerLevel actualLevel = loggers.level("a"); + assertEquals(expectedLevel, actualLevel); + } + + @Test + public void testGetUnknownLogger() { + Logger root = logger("root"); + root.setLevel(Level.ERROR); + + Hierarchy hierarchy = new Hierarchy(root); + Logger a = hierarchy.getLogger("a"); + a.setLevel(null); + Logger b = hierarchy.getLogger("b"); + b.setLevel(Level.INFO); + + Loggers loggers = new TestLoggers(root, a, b); + + LoggerLevel level = loggers.level("c"); + assertNull(level); + } + + @Test + public void testSetLevel() { + Logger root = logger("root"); + root.setLevel(Level.ERROR); + + Logger x = logger("a.b.c.p.X"); + Logger y = logger("a.b.c.p.Y"); + Logger z = logger("a.b.c.p.Z"); + Logger w = logger("a.b.c.s.W"); + x.setLevel(Level.INFO); + y.setLevel(Level.INFO); + z.setLevel(Level.INFO); + w.setLevel(Level.INFO); + + // We don't explicitly register a logger for a.b.c.p, so it won't appear in the list of current loggers; + // one should be created by the Loggers instance when we set the level + TestLoggers loggers = new TestLoggers(root, x, y, z, w); + + List modified = loggers.setLevel("a.b.c.p", Level.DEBUG); + assertEquals(Arrays.asList("a.b.c.p", "a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z"), modified); + assertEquals(Level.DEBUG.toString(), loggers.level("a.b.c.p").level()); + assertEquals(Level.DEBUG, x.getLevel()); + assertEquals(Level.DEBUG, y.getLevel()); + assertEquals(Level.DEBUG, z.getLevel()); + + LoggerLevel expectedLevel = new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME); + LoggerLevel actualLevel = loggers.level("a.b.c.p"); + assertEquals(expectedLevel, actualLevel); + + // Sleep a little and adjust the level of a leaf logger + time.sleep(10); + loggers.setLevel("a.b.c.p.X", Level.ERROR); + expectedLevel = new LoggerLevel(Level.ERROR.toString(), INITIAL_TIME + 10); + actualLevel = loggers.level("a.b.c.p.X"); + assertEquals(expectedLevel, actualLevel); + + // Make sure that the direct parent logger and a sibling logger remain unaffected + expectedLevel = new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME); + actualLevel = loggers.level("a.b.c.p"); + assertEquals(expectedLevel, actualLevel); + + expectedLevel = new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME); + actualLevel = loggers.level("a.b.c.p.Y"); + assertEquals(expectedLevel, actualLevel); + + // Set the same level again, and verify that the last modified time hasn't been altered + time.sleep(10); + loggers.setLevel("a.b.c.p.X", Level.ERROR); + expectedLevel = new LoggerLevel(Level.ERROR.toString(), INITIAL_TIME + 10); + actualLevel = loggers.level("a.b.c.p.X"); + assertEquals(expectedLevel, actualLevel); + } + + @Test + public void testSetRootLevel() { + Logger root = logger("root"); + root.setLevel(Level.ERROR); + + Logger p = logger("a.b.c.p"); + Logger x = logger("a.b.c.p.X"); + Logger y = logger("a.b.c.p.Y"); + Logger z = logger("a.b.c.p.Z"); + Logger w = logger("a.b.c.s.W"); + x.setLevel(Level.INFO); + y.setLevel(Level.INFO); + z.setLevel(Level.INFO); + w.setLevel(Level.INFO); + + Loggers loggers = new TestLoggers(root, x, y, z, w); + + List modified = loggers.setLevel("root", Level.DEBUG); + assertEquals(Arrays.asList("a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z", "a.b.c.s.W", "root"), modified); + + assertNull(p.getLevel()); + + assertEquals(root.getLevel(), Level.DEBUG); + + assertEquals(w.getLevel(), Level.DEBUG); + assertEquals(x.getLevel(), Level.DEBUG); + assertEquals(y.getLevel(), Level.DEBUG); + assertEquals(z.getLevel(), Level.DEBUG); + + Map expectedLevels = new HashMap<>(); + expectedLevels.put("root", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME)); + expectedLevels.put("a.b.c.p.X", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME)); + expectedLevels.put("a.b.c.p.Y", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME)); + expectedLevels.put("a.b.c.p.Z", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME)); + expectedLevels.put("a.b.c.s.W", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME)); + + Map actualLevels = loggers.allLevels(); + assertEquals(expectedLevels, actualLevels); + } + + @Test + public void testSetLevelNullArguments() { + Logger root = logger("root"); + Loggers loggers = new TestLoggers(root); + assertThrows(NullPointerException.class, () -> loggers.setLevel(null, Level.INFO)); + assertThrows(NullPointerException.class, () -> loggers.setLevel("root", null)); + } + + private class TestLoggers extends Loggers { + + private final Logger rootLogger; + private final Map currentLoggers; + + public TestLoggers(Logger rootLogger, Logger... knownLoggers) { + super(time); + this.rootLogger = rootLogger; + this.currentLoggers = new HashMap<>(Stream.of(knownLoggers) + .collect(Collectors.toMap( + Logger::getName, + Function.identity() + ))); + } + + @Override + Logger lookupLogger(String logger) { + return currentLoggers.computeIfAbsent(logger, l -> new Logger(logger) { }); + } + + @Override + Enumeration currentLoggers() { + return new Vector<>(currentLoggers.values()).elements(); + } + + @Override + Logger rootLogger() { + return rootLogger; + } + } + + private Logger logger(String name) { + return new Logger(name) { }; + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java index 2b78900bc25..f2978678bbf 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java @@ -34,6 +34,7 @@ import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.connect.rest.ConnectRestExtension; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -56,7 +57,6 @@ import java.util.Map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -247,15 +247,18 @@ public class ConnectRestServerTest { } @Test - public void testLoggersEndpointWithDefaults() throws IOException { + public void testLoggerEndpointWithDefaults() throws IOException { Map configMap = new HashMap<>(baseServerProps()); + final String logger = "a.b.c.s.W"; + final String loggingLevel = "INFO"; + final long lastModified = 789052637671L; + doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId(); doReturn(plugins).when(herder).plugins(); expectEmptyRestExtensions(); - - // create some loggers in the process - LoggerFactory.getLogger("a.b.c.s.W"); + doReturn(Collections.emptyList()).when(herder).setWorkerLoggerLevel(logger, loggingLevel); + doReturn(Collections.singletonMap(logger, new LoggerLevel(loggingLevel, lastModified))).when(herder).allLoggerLevels(); server = new ConnectRestServer(null, null, configMap); server.initializeServer(); @@ -265,14 +268,16 @@ public class ConnectRestServerTest { URI serverUrl = server.advertisedUrl(); - executePut(serverUrl, "/admin/loggers/a.b.c.s.W", "{\"level\": \"INFO\"}"); + executePut(serverUrl, "/admin/loggers/" + logger, "{\"level\": \"" + loggingLevel + "\"}"); String responseStr = executeGet(serverUrl, "/admin/loggers"); - Map> loggers = mapper.readValue(responseStr, new TypeReference>>() { - }); - assertNotNull("expected non null response for /admin/loggers" + prettyPrint(loggers), loggers); - assertTrue("expect at least 1 logger. instead found " + prettyPrint(loggers), loggers.size() >= 1); - assertEquals("expected to find logger a.b.c.s.W set to INFO level", loggers.get("a.b.c.s.W").get("level"), "INFO"); + + Map expectedLogger = new HashMap<>(); + expectedLogger.put("level", loggingLevel); + expectedLogger.put("last_modified", lastModified); + Map> expectedLoggers = Collections.singletonMap(logger, expectedLogger); + Map> actualLoggers = mapper.readValue(responseStr, new TypeReference>>() { }); + assertEquals(expectedLoggers, actualLoggers); } @Test diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResourceTest.java index 84da433a894..3b3df993511 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResourceTest.java @@ -16,176 +16,150 @@ */ package org.apache.kafka.connect.runtime.rest.resources; +import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.connect.errors.NotFoundException; +import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel; import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; -import org.apache.log4j.Hierarchy; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; +import org.slf4j.event.Level; +import javax.ws.rs.core.Response; import java.util.Arrays; import java.util.Collections; -import java.util.Enumeration; import java.util.List; -import java.util.Map; -import java.util.Vector; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@SuppressWarnings("unchecked") @RunWith(MockitoJUnitRunner.StrictStubs.class) public class LoggingResourceTest { + private LoggingResource loggingResource; + + @Mock + private Herder herder; + + @Before + public void setup() { + loggingResource = new LoggingResource(herder); + } + @Test - public void getLoggersIgnoresNullLevelsTest() { - LoggingResource loggingResource = mock(LoggingResource.class); - Logger root = new Logger("root") { - }; - Logger a = new Logger("a") { - }; - a.setLevel(null); - Logger b = new Logger("b") { - }; - b.setLevel(Level.INFO); - when(loggingResource.currentLoggers()).thenReturn(loggers(a, b)); - when(loggingResource.rootLogger()).thenReturn(root); - when(loggingResource.listLoggers()).thenCallRealMethod(); - Map> loggers = (Map>) loggingResource.listLoggers().getEntity(); - assertEquals(1, loggers.size()); - assertEquals("INFO", loggers.get("b").get("level")); + public void testGetLevelNotFound() { + final String logger = "org.apache.rostropovich"; + when(herder.loggerLevel(logger)).thenReturn(null); + assertThrows( + NotFoundException.class, + () -> loggingResource.getLogger(logger) + ); } @Test - public void getLoggerFallsbackToEffectiveLogLevelTest() { - LoggingResource loggingResource = mock(LoggingResource.class); - Logger root = new Logger("root") { - }; - root.setLevel(Level.ERROR); - Hierarchy hierarchy = new Hierarchy(root); - Logger a = hierarchy.getLogger("a"); - a.setLevel(null); - Logger b = hierarchy.getLogger("b"); - b.setLevel(Level.INFO); - when(loggingResource.currentLoggers()).thenReturn(loggers(a, b)); - when(loggingResource.getLogger(any())).thenCallRealMethod(); - Map level = (Map) loggingResource.getLogger("a").getEntity(); - assertEquals(1, level.size()); - assertEquals("ERROR", level.get("level")); + public void testGetLevel() { + final String logger = "org.apache.kafka.producer"; + final LoggerLevel expectedLevel = new LoggerLevel(Level.WARN.toString(), 976L); + when(herder.loggerLevel(logger)).thenReturn(expectedLevel); + + Response response = loggingResource.getLogger(logger); + assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + LoggerLevel actualLevel = (LoggerLevel) response.getEntity(); + + assertEquals( + expectedLevel, + actualLevel + ); } @Test - public void getUnknownLoggerTest() { - LoggingResource loggingResource = mock(LoggingResource.class); - Logger root = new Logger("root") { - }; - root.setLevel(Level.ERROR); - Hierarchy hierarchy = new Hierarchy(root); - Logger a = hierarchy.getLogger("a"); - a.setLevel(null); - Logger b = hierarchy.getLogger("b"); - b.setLevel(Level.INFO); - when(loggingResource.currentLoggers()).thenReturn(loggers(a, b)); - when(loggingResource.getLogger(any())).thenCallRealMethod(); - assertThrows(NotFoundException.class, () -> loggingResource.getLogger("c")); + public void setLevelWithEmptyArgTest() { + for (String scope : Arrays.asList("worker", "cluster", "N/A", null)) { + assertThrows( + BadRequestException.class, + () -> loggingResource.setLevel( + "@root", + Collections.emptyMap(), + scope + ) + ); + } } @Test - public void setLevelTest() { - LoggingResource loggingResource = mock(LoggingResource.class); - Logger root = new Logger("root") { - }; - root.setLevel(Level.ERROR); - Hierarchy hierarchy = new Hierarchy(root); - Logger p = hierarchy.getLogger("a.b.c.p"); - Logger x = hierarchy.getLogger("a.b.c.p.X"); - Logger y = hierarchy.getLogger("a.b.c.p.Y"); - Logger z = hierarchy.getLogger("a.b.c.p.Z"); - Logger w = hierarchy.getLogger("a.b.c.s.W"); - x.setLevel(Level.INFO); - y.setLevel(Level.INFO); - z.setLevel(Level.INFO); - w.setLevel(Level.INFO); - when(loggingResource.currentLoggers()).thenReturn(loggers(x, y, z, w)); - when(loggingResource.lookupLogger("a.b.c.p")).thenReturn(p); - when(loggingResource.setLevel(any(), any())).thenCallRealMethod(); - List modified = (List) loggingResource.setLevel("a.b.c.p", Collections.singletonMap("level", "DEBUG")).getEntity(); - assertEquals(4, modified.size()); - assertEquals(Arrays.asList("a.b.c.p", "a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z"), modified); - assertEquals(p.getLevel(), Level.DEBUG); - assertEquals(x.getLevel(), Level.DEBUG); - assertEquals(y.getLevel(), Level.DEBUG); - assertEquals(z.getLevel(), Level.DEBUG); + public void setLevelWithInvalidArgTest() { + for (String scope : Arrays.asList("worker", "cluster", "N/A", null)) { + assertThrows( + NotFoundException.class, + () -> loggingResource.setLevel( + "@root", + Collections.singletonMap("level", "HIGH"), + scope + ) + ); + } } @Test - public void setRootLevelTest() { - LoggingResource loggingResource = mock(LoggingResource.class); - Logger root = new Logger("root") { - }; - root.setLevel(Level.ERROR); - Hierarchy hierarchy = new Hierarchy(root); - Logger p = hierarchy.getLogger("a.b.c.p"); - Logger x = hierarchy.getLogger("a.b.c.p.X"); - Logger y = hierarchy.getLogger("a.b.c.p.Y"); - Logger z = hierarchy.getLogger("a.b.c.p.Z"); - Logger w = hierarchy.getLogger("a.b.c.s.W"); - x.setLevel(Level.INFO); - y.setLevel(Level.INFO); - z.setLevel(Level.INFO); - w.setLevel(Level.INFO); - when(loggingResource.currentLoggers()).thenReturn(loggers(x, y, z, w)); - when(loggingResource.rootLogger()).thenReturn(root); - when(loggingResource.setLevel(any(), any())).thenCallRealMethod(); - List modified = (List) loggingResource.setLevel("root", Collections.singletonMap("level", "DEBUG")).getEntity(); - assertEquals(5, modified.size()); - assertEquals(Arrays.asList("a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z", "a.b.c.s.W", "root"), modified); - assertNull(p.getLevel()); - assertEquals(root.getLevel(), Level.DEBUG); - assertEquals(w.getLevel(), Level.DEBUG); - assertEquals(x.getLevel(), Level.DEBUG); - assertEquals(y.getLevel(), Level.DEBUG); - assertEquals(z.getLevel(), Level.DEBUG); + public void testSetLevelDefaultScope() { + testSetLevelWorkerScope(null, true); } @Test - public void setLevelWithEmptyArgTest() { - LoggingResource loggingResource = mock(LoggingResource.class); - Logger root = new Logger("root") { - }; - root.setLevel(Level.ERROR); - Hierarchy hierarchy = new Hierarchy(root); - Logger a = hierarchy.getLogger("a"); - a.setLevel(null); - Logger b = hierarchy.getLogger("b"); - b.setLevel(Level.INFO); - when(loggingResource.setLevel(any(), any())).thenCallRealMethod(); - assertThrows(BadRequestException.class, () -> loggingResource.setLevel("@root", Collections.emptyMap())); + public void testSetLevelInvalidScope() { + testSetLevelWorkerScope("kip-976", true); } @Test - public void setLevelWithInvalidArgTest() { - LoggingResource loggingResource = mock(LoggingResource.class); - Logger root = new Logger("root") { - }; - root.setLevel(Level.ERROR); - Hierarchy hierarchy = new Hierarchy(root); - Logger a = hierarchy.getLogger("a"); - a.setLevel(null); - Logger b = hierarchy.getLogger("b"); - b.setLevel(Level.INFO); - when(loggingResource.setLevel(any(), any())).thenCallRealMethod(); - assertThrows(NotFoundException.class, () -> loggingResource.setLevel("@root", Collections.singletonMap("level", "HIGH"))); + public void testSetLevelWorkerScope() { + testSetLevelWorkerScope("worker", false); + } + + @SuppressWarnings("unchecked") + private void testSetLevelWorkerScope(String scope, boolean expectWarning) { + final String logger = "org.apache.kafka.connect"; + final String level = "TRACE"; + final List expectedLoggers = Arrays.asList( + "org.apache.kafka.connect", + "org.apache.kafka.connect.runtime.distributed.DistributedHerder" + ); + when(herder.setWorkerLoggerLevel(logger, level)).thenReturn(expectedLoggers); + + List actualLoggers; + try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(LoggingResource.class)) { + Response response = loggingResource.setLevel(logger, Collections.singletonMap("level", level), scope); + assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + actualLoggers = (List) response.getEntity(); + long warningMessages = logCaptureAppender.getEvents().stream() + .filter(e -> "WARN".equals(e.getLevel())) + .count(); + if (expectWarning) { + assertEquals(1, warningMessages); + } else { + assertEquals(0, warningMessages); + } + } + + assertEquals(expectedLoggers, actualLoggers); } - private Enumeration loggers(Logger... loggers) { - return new Vector<>(Arrays.asList(loggers)).elements(); + @Test + public void testSetLevelClusterScope() { + final String logger = "org.apache.kafka.connect"; + final String level = "TRACE"; + + Response response = loggingResource.setLevel(logger, Collections.singletonMap("level", level), "cluster"); + + assertEquals(Response.Status.NO_CONTENT.getStatusCode(), response.getStatus()); + assertNull(response.getEntity()); + + verify(herder).setClusterLoggerLevel(logger, level); } -} +} \ No newline at end of file diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index 500ce58b8c6..d9c64f5e36c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -19,6 +19,7 @@ package org.apache.kafka.connect.runtime.standalone; import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigValue; +import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.errors.AlreadyExistsException; @@ -130,7 +131,7 @@ public class StandaloneHerderTest { worker = PowerMock.createMock(Worker.class); String[] methodNames = new String[]{"connectorType", "buildRestartPlan", "recordRestarting"}; herder = PowerMock.createPartialMock(StandaloneHerder.class, methodNames, - worker, WORKER_ID, KAFKA_CLUSTER_ID, statusBackingStore, new MemoryConfigBackingStore(transformer), noneConnectorClientConfigOverridePolicy); + worker, WORKER_ID, KAFKA_CLUSTER_ID, statusBackingStore, new MemoryConfigBackingStore(transformer), noneConnectorClientConfigOverridePolicy, new MockTime()); createCallback = new FutureCallback<>(); plugins = PowerMock.createMock(Plugins.class); pluginLoader = PowerMock.createMock(PluginClassLoader.class); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java index c3d4e7728d0..59420e8faf0 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java @@ -1461,6 +1461,71 @@ public class KafkaConfigBackingStoreTest { PowerMock.verifyAll(); } + @Test + public void testPutLogLevel() throws Exception { + final String logger1 = "org.apache.zookeeper"; + final String logger2 = "org.apache.cassandra"; + final String logger3 = "org.apache.kafka.clients"; + final String logger4 = "org.apache.kafka.connect"; + final String level1 = "ERROR"; + final String level3 = "WARN"; + final String level4 = "DEBUG"; + + final Struct existingLogLevel = new Struct(KafkaConfigBackingStore.LOGGER_LEVEL_V0) + .put("level", level1); + + // Pre-populate the config topic with a couple of logger level records; these should be ignored (i.e., + // not reported to the update listener) + List> existingRecords = Arrays.asList( + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, "logger-cluster-" + logger1, + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty() + ), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, "logger-cluster-" + logger2, + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty() + ) + ); + LinkedHashMap deserialized = new LinkedHashMap<>(); + deserialized.put(CONFIGS_SERIALIZED.get(0), existingLogLevel); + // Make sure we gracefully handle tombstones + deserialized.put(CONFIGS_SERIALIZED.get(1), null); + logOffset = 2; + + expectConfigure(); + expectStart(existingRecords, deserialized); + expectPartitionCount(1); + expectStop(); + + expectConvertWriteRead( + "logger-cluster-" + logger3, KafkaConfigBackingStore.LOGGER_LEVEL_V0, CONFIGS_SERIALIZED.get(2), + "level", level3); + expectConvertWriteRead( + "logger-cluster-" + logger4, KafkaConfigBackingStore.LOGGER_LEVEL_V0, CONFIGS_SERIALIZED.get(3), + "level", level4); + + LinkedHashMap newRecords = new LinkedHashMap<>(); + newRecords.put("logger-cluster-" + logger3, CONFIGS_SERIALIZED.get(2)); + newRecords.put("logger-cluster-" + logger4, CONFIGS_SERIALIZED.get(3)); + expectReadToEnd(newRecords); + + configUpdateListener.onLoggingLevelUpdate(logger3, level3); + EasyMock.expectLastCall(); + configUpdateListener.onLoggingLevelUpdate(logger4, level4); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); + configStorage.start(); + + configStorage.putLoggerLevel(logger3, level3); + configStorage.putLoggerLevel(logger4, level4); + configStorage.refresh(0, TimeUnit.SECONDS); + + configStorage.stop(); + + PowerMock.verifyAll(); + } + @Test public void testExceptionOnStartWhenConfigTopicHasMultiplePartitions() throws Exception { expectConfigure(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/ConnectAssertions.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/ConnectAssertions.java new file mode 100644 index 00000000000..1d1b042bc8f --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/ConnectAssertions.java @@ -0,0 +1,618 @@ +/* + * 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.util.clusters; + +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.connect.runtime.AbstractStatus; +import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.ws.rs.core.Response; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +import static org.apache.kafka.test.TestUtils.waitForCondition; + +/** + * A set of common assertions that can be applied to a Connect cluster during integration testing + */ +public class ConnectAssertions { + + private static final Logger log = LoggerFactory.getLogger(ConnectAssertions.class); + public static final long WORKER_SETUP_DURATION_MS = TimeUnit.MINUTES.toMillis(5); + public static final long VALIDATION_DURATION_MS = TimeUnit.SECONDS.toMillis(30); + public static final long CONNECTOR_SETUP_DURATION_MS = TimeUnit.MINUTES.toMillis(2); + // Creating a connector requires two rounds of rebalance; destroying one only requires one + // Assume it'll take ~half the time to destroy a connector as it does to create one + public static final long CONNECTOR_SHUTDOWN_DURATION_MS = TimeUnit.MINUTES.toMillis(1); + private static final long CONNECT_INTERNAL_TOPIC_UPDATES_DURATION_MS = TimeUnit.SECONDS.toMillis(60); + + private final EmbeddedConnect connect; + + ConnectAssertions(EmbeddedConnect connect) { + this.connect = connect; + } + + /** + * Assert that at least the requested number of workers are up and running. + * + * @param numWorkers the number of online workers + */ + public void assertAtLeastNumWorkersAreUp(int numWorkers, String detailMessage) throws InterruptedException { + try { + waitForCondition( + () -> checkWorkersUp(numWorkers, (actual, expected) -> actual >= expected).orElse(false), + WORKER_SETUP_DURATION_MS, + "Didn't meet the minimum requested number of online workers: " + numWorkers); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Assert that at least the requested number of workers are up and running. + * + * @param numWorkers the number of online workers + */ + public void assertExactlyNumWorkersAreUp(int numWorkers, String detailMessage) throws InterruptedException { + try { + waitForCondition( + () -> checkWorkersUp(numWorkers, (actual, expected) -> actual == expected).orElse(false), + WORKER_SETUP_DURATION_MS, + "Didn't meet the exact requested number of online workers: " + numWorkers); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Confirm that the requested number of workers are up and running. + * + * @param numWorkers the number of online workers + * @return true if at least {@code numWorkers} are up; false otherwise + */ + protected Optional checkWorkersUp(int numWorkers, BiFunction comp) { + try { + int numUp = connect.activeWorkers().size(); + return Optional.of(comp.apply(numUp, numWorkers)); + } catch (Exception e) { + log.error("Could not check active workers.", e); + return Optional.empty(); + } + } + + /** + * Assert that at least the requested number of workers are up and running. + * + * @param numBrokers the number of online brokers + */ + public void assertExactlyNumBrokersAreUp(int numBrokers, String detailMessage) throws InterruptedException { + try { + waitForCondition( + () -> checkBrokersUp(numBrokers, (actual, expected) -> actual == expected).orElse(false), + WORKER_SETUP_DURATION_MS, + "Didn't meet the exact requested number of online brokers: " + numBrokers); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Confirm that the requested number of brokers are up and running. + * + * @param numBrokers the number of online brokers + * @return true if at least {@code numBrokers} are up; false otherwise + */ + protected Optional checkBrokersUp(int numBrokers, BiFunction comp) { + try { + int numRunning = connect.kafka().runningBrokers().size(); + return Optional.of(comp.apply(numRunning, numBrokers)); + } catch (Exception e) { + log.error("Could not check running brokers.", e); + return Optional.empty(); + } + } + + /** + * Assert that the topics with the specified names do not exist. + * + * @param topicNames the names of the topics that are expected to not exist + */ + public void assertTopicsDoNotExist(String... topicNames) throws InterruptedException { + Set topicNameSet = new HashSet<>(Arrays.asList(topicNames)); + AtomicReference> existingTopics = new AtomicReference<>(topicNameSet); + waitForCondition( + () -> checkTopicsExist(topicNameSet, (actual, expected) -> { + existingTopics.set(actual); + return actual.isEmpty(); + }).orElse(false), + CONNECTOR_SETUP_DURATION_MS, + "Unexpectedly found topics " + existingTopics.get()); + } + + /** + * Assert that the topics with the specified names do exist. + * + * @param topicNames the names of the topics that are expected to exist + */ + public void assertTopicsExist(String... topicNames) throws InterruptedException { + Set topicNameSet = new HashSet<>(Arrays.asList(topicNames)); + AtomicReference> missingTopics = new AtomicReference<>(topicNameSet); + waitForCondition( + () -> checkTopicsExist(topicNameSet, (actual, expected) -> { + Set missing = new HashSet<>(expected); + missing.removeAll(actual); + missingTopics.set(missing); + return missing.isEmpty(); + }).orElse(false), + CONNECTOR_SETUP_DURATION_MS, + "Didn't find the topics " + missingTopics.get()); + } + + protected Optional checkTopicsExist(Set topicNames, BiFunction, Set, Boolean> comp) { + try { + Map> topics = connect.kafka().describeTopics(topicNames); + Set actualExistingTopics = topics.entrySet() + .stream() + .filter(e -> e.getValue().isPresent()) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + return Optional.of(comp.apply(actualExistingTopics, topicNames)); + } catch (Exception e) { + log.error("Failed to describe the topic(s): {}.", topicNames, e); + return Optional.empty(); + } + } + + /** + * Assert that the named topic is configured to have the specified replication factor and + * number of partitions. + * + * @param topicName the name of the topic that is expected to exist + * @param replicas the replication factor + * @param partitions the number of partitions + * @param detailMessage the assertion message + */ + public void assertTopicSettings(String topicName, int replicas, int partitions, String detailMessage) + throws InterruptedException { + try { + waitForCondition( + () -> checkTopicSettings( + topicName, + replicas, + partitions + ).orElse(false), + VALIDATION_DURATION_MS, + "Topic " + topicName + " does not exist or does not have exactly " + + partitions + " partitions or at least " + + replicas + " per partition"); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + protected Optional checkTopicSettings(String topicName, int replicas, int partitions) { + try { + Map> topics = connect.kafka().describeTopics(topicName); + TopicDescription topicDesc = topics.get(topicName).orElse(null); + boolean result = topicDesc != null + && topicDesc.name().equals(topicName) + && topicDesc.partitions().size() == partitions + && topicDesc.partitions().stream().allMatch(p -> p.replicas().size() >= replicas); + return Optional.of(result); + } catch (Exception e) { + log.error("Failed to describe the topic: {}.", topicName, e); + return Optional.empty(); + } + } + + /** + * Assert that the required number of errors are produced by a connector config validation. + * + * @param connectorClass the class of the connector to validate + * @param connConfig the intended configuration + * @param numErrors the number of errors expected + * @param detailMessage the assertion message + */ + public void assertExactlyNumErrorsOnConnectorConfigValidation(String connectorClass, Map connConfig, + int numErrors, String detailMessage) throws InterruptedException { + assertExactlyNumErrorsOnConnectorConfigValidation(connectorClass, connConfig, numErrors, detailMessage, VALIDATION_DURATION_MS); + } + + /** + * Assert that the required number of errors are produced by a connector config validation. + * + * @param connectorClass the class of the connector to validate + * @param connConfig the intended configuration + * @param numErrors the number of errors expected + * @param detailMessage the assertion message + * @param timeout how long to retry for before throwing an exception + * + * @throws AssertionError if the exact number of errors is not produced during config + * validation before the timeout expires + */ + public void assertExactlyNumErrorsOnConnectorConfigValidation(String connectorClass, Map connConfig, + int numErrors, String detailMessage, long timeout) throws InterruptedException { + try { + waitForCondition( + () -> checkValidationErrors( + connectorClass, + connConfig, + numErrors, + (actual, expected) -> actual == expected + ).orElse(false), + timeout, + "Didn't meet the exact requested number of validation errors: " + numErrors); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Confirm that the requested number of errors are produced by {@link EmbeddedConnect#validateConnectorConfig}. + * + * @param connectorClass the class of the connector to validate + * @param connConfig the intended configuration + * @param numErrors the number of errors expected + * @return true if exactly {@code numErrors} are produced by the validation; false otherwise + */ + protected Optional checkValidationErrors(String connectorClass, Map connConfig, + int numErrors, BiFunction comp) { + try { + int numErrorsProduced = connect.validateConnectorConfig(connectorClass, connConfig).errorCount(); + return Optional.of(comp.apply(numErrorsProduced, numErrors)); + } catch (Exception e) { + log.error("Could not check config validation error count.", e); + return Optional.empty(); + } + } + + /** + * Assert that a connector is running with at least the given number of tasks all in running state + * + * @param connectorName the connector name + * @param numTasks the number of tasks + * @param detailMessage + * @throws InterruptedException + */ + public void assertConnectorAndAtLeastNumTasksAreRunning(String connectorName, int numTasks, String detailMessage) + throws InterruptedException { + try { + waitForCondition( + () -> checkConnectorState( + connectorName, + AbstractStatus.State.RUNNING, + numTasks, + AbstractStatus.State.RUNNING, + (actual, expected) -> actual >= expected + ).orElse(false), + CONNECTOR_SETUP_DURATION_MS, + "The connector or at least " + numTasks + " of tasks are not running."); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Assert that a connector is running, that it has a specific number of tasks, and that all of + * its tasks are in the RUNNING state. + * + * @param connectorName the connector name + * @param numTasks the number of tasks + * @param detailMessage the assertion message + * @throws InterruptedException + */ + public void assertConnectorAndExactlyNumTasksAreRunning(String connectorName, int numTasks, String detailMessage) + throws InterruptedException { + try { + waitForCondition( + () -> checkConnectorState( + connectorName, + AbstractStatus.State.RUNNING, + numTasks, + AbstractStatus.State.RUNNING, + (actual, expected) -> actual == expected + ).orElse(false), + CONNECTOR_SETUP_DURATION_MS, + "The connector or exactly " + numTasks + " tasks are not running."); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Assert that a connector is paused, that it has a specific number of tasks, and that all of + * its tasks are in the PAUSED state. + * + * @param connectorName the connector name + * @param numTasks the number of tasks + * @param detailMessage the assertion message + * @throws InterruptedException + */ + public void assertConnectorAndExactlyNumTasksArePaused(String connectorName, int numTasks, String detailMessage) + throws InterruptedException { + try { + waitForCondition( + () -> checkConnectorState( + connectorName, + AbstractStatus.State.PAUSED, + numTasks, + AbstractStatus.State.PAUSED, + Integer::equals + ).orElse(false), + CONNECTOR_SHUTDOWN_DURATION_MS, + "The connector or exactly " + numTasks + " tasks are not paused."); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Assert that a connector is running, that it has a specific number of tasks, and that all of + * its tasks are in the FAILED state. + * + * @param connectorName the connector name + * @param numTasks the number of tasks + * @param detailMessage the assertion message + * @throws InterruptedException + */ + public void assertConnectorIsRunningAndTasksHaveFailed(String connectorName, int numTasks, String detailMessage) + throws InterruptedException { + try { + waitForCondition( + () -> checkConnectorState( + connectorName, + AbstractStatus.State.RUNNING, + numTasks, + AbstractStatus.State.FAILED, + (actual, expected) -> actual >= expected + ).orElse(false), + CONNECTOR_SETUP_DURATION_MS, + "Either the connector is not running or not all the " + numTasks + " tasks have failed."); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Assert that a connector is running, that it has a specific number of tasks out of that numFailedTasks are in the FAILED state. + * + * @param connectorName the connector name + * @param numTasks the number of tasks + * @param numFailedTasks the number of failed tasks + * @param detailMessage the assertion message + * @throws InterruptedException + */ + public void assertConnectorIsRunningAndNumTasksHaveFailed(String connectorName, int numTasks, int numFailedTasks, String detailMessage) + throws InterruptedException { + try { + waitForCondition( + () -> checkConnectorState( + connectorName, + AbstractStatus.State.RUNNING, + numTasks, + numFailedTasks, + AbstractStatus.State.FAILED, + (actual, expected) -> actual >= expected + ).orElse(false), + CONNECTOR_SETUP_DURATION_MS, + "Either the connector is not running or not all the " + numTasks + " tasks have failed."); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Assert that a connector is in FAILED state, that it has a specific number of tasks, and that all of + * its tasks are in the FAILED state. + * + * @param connectorName the connector name + * @param numTasks the number of tasks + * @param detailMessage the assertion message + * @throws InterruptedException + */ + public void assertConnectorIsFailedAndTasksHaveFailed(String connectorName, int numTasks, String detailMessage) + throws InterruptedException { + try { + waitForCondition( + () -> checkConnectorState( + connectorName, + AbstractStatus.State.FAILED, + numTasks, + AbstractStatus.State.FAILED, + (actual, expected) -> actual >= expected + ).orElse(false), + CONNECTOR_SETUP_DURATION_MS, + "Either the connector is running or not all the " + numTasks + " tasks have failed."); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Assert that a connector does not exist. This can be used to verify that a connector has been successfully deleted. + * + * @param connectorName the connector name + * @param detailMessage the assertion message + * @throws InterruptedException + */ + public void assertConnectorDoesNotExist(String connectorName, String detailMessage) + throws InterruptedException { + try { + waitForCondition( + () -> checkConnectorDoesNotExist(connectorName), + CONNECTOR_SETUP_DURATION_MS, + "The connector should not exist."); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Check whether a connector exists by querying the GET /connectors/{connector}/status endpoint + * + * @param connectorName the connector name + * @return true if the connector does not exist; false otherwise + */ + protected boolean checkConnectorDoesNotExist(String connectorName) { + try { + connect.connectorStatus(connectorName); + } catch (ConnectRestException e) { + return e.statusCode() == Response.Status.NOT_FOUND.getStatusCode(); + } catch (Exception e) { + log.error("Could not check connector state info.", e); + return false; + } + return false; + } + + /** + * Assert that a connector is in the stopped state and has no tasks. + * + * @param connectorName the connector name + * @param detailMessage the assertion message + * @throws InterruptedException + */ + public void assertConnectorIsStopped(String connectorName, String detailMessage) + throws InterruptedException { + try { + waitForCondition( + () -> checkConnectorState( + connectorName, + AbstractStatus.State.STOPPED, + 0, + null, + Integer::equals + ).orElse(false), + CONNECTOR_SHUTDOWN_DURATION_MS, + "At least the connector or one of its tasks is still running"); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Check whether the given connector state matches the current state of the connector and + * whether it has at least the given number of tasks, with all the tasks matching the given + * task state. + * @param connectorName the connector + * @param connectorState + * @param numTasks the expected number of tasks + * @param tasksState + * @return true if the connector and tasks are in RUNNING state; false otherwise + */ + protected Optional checkConnectorState( + String connectorName, + AbstractStatus.State connectorState, + int numTasks, + AbstractStatus.State tasksState, + BiFunction comp + ) { + try { + ConnectorStateInfo info = connect.connectorStatus(connectorName); + boolean result = info != null + && comp.apply(info.tasks().size(), numTasks) + && info.connector().state().equals(connectorState.toString()) + && info.tasks().stream().allMatch(s -> s.state().equals(tasksState.toString())); + return Optional.of(result); + } catch (Exception e) { + log.error("Could not check connector state info.", e); + return Optional.empty(); + } + } + + /** + * Check whether the given connector state matches the current state of the connector and + * whether it has at least the given number of tasks, with numTasksInTasksState matching the given + * task state. + * @param connectorName the connector + * @param connectorState + * @param numTasks the expected number of tasks + * @param tasksState + * @return true if the connector and tasks are in RUNNING state; false otherwise + */ + protected Optional checkConnectorState( + String connectorName, + AbstractStatus.State connectorState, + int numTasks, + int numTasksInTasksState, + AbstractStatus.State tasksState, + BiFunction comp + ) { + try { + ConnectorStateInfo info = connect.connectorStatus(connectorName); + boolean result = info != null + && comp.apply(info.tasks().size(), numTasks) + && info.connector().state().equals(connectorState.toString()) + && info.tasks().stream().filter(s -> s.state().equals(tasksState.toString())).count() == numTasksInTasksState; + return Optional.of(result); + } catch (Exception e) { + log.error("Could not check connector state info.", e); + return Optional.empty(); + } + } + /** + * Assert that a connector's set of active topics matches the given collection of topic names. + * + * @param connectorName the connector name + * @param topics a collection of topics to compare against + * @param detailMessage the assertion message + * @throws InterruptedException + */ + public void assertConnectorActiveTopics(String connectorName, Collection topics, String detailMessage) throws InterruptedException { + try { + waitForCondition( + () -> checkConnectorActiveTopics(connectorName, topics).orElse(false), + CONNECT_INTERNAL_TOPIC_UPDATES_DURATION_MS, + "Connector active topics don't match the expected collection"); + } catch (AssertionError e) { + throw new AssertionError(detailMessage, e); + } + } + + /** + * Check whether a connector's set of active topics matches the given collection of topic names. + * + * @param connectorName the connector name + * @param topics a collection of topics to compare against + * @return true if the connector's active topics matches the given collection; false otherwise + */ + protected Optional checkConnectorActiveTopics(String connectorName, Collection topics) { + try { + ActiveTopicsInfo info = connect.connectorTopics(connectorName); + boolean result = info != null + && topics.size() == info.topics().size() + && topics.containsAll(info.topics()); + log.debug("Found connector {} using topics: {}", connectorName, info.topics()); + return Optional.of(result); + } catch (Exception e) { + log.error("Could not check connector {} state info.", connectorName, e); + return Optional.empty(); + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java new file mode 100644 index 00000000000..3b8c504e60e --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java @@ -0,0 +1,956 @@ +/* + * 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.util.clusters; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffset; +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.LoggerLevel; +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; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.client.api.Request; +import org.eclipse.jetty.client.util.StringContentProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.ws.rs.core.Response; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +abstract class EmbeddedConnect { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedConnect.class); + + public static final int DEFAULT_NUM_BROKERS = 1; + + protected final int numBrokers; + + private final EmbeddedKafkaCluster kafkaCluster; + private final boolean maskExitProcedures; + private final HttpClient httpClient; + private final ConnectAssertions assertions; + private final ClassLoader originalClassLoader; + + protected EmbeddedConnect( + int numBrokers, + Properties brokerProps, + boolean maskExitProcedures, + Map clientProps + ) { + this.numBrokers = numBrokers; + this.kafkaCluster = new EmbeddedKafkaCluster(numBrokers, brokerProps, clientProps); + this.maskExitProcedures = maskExitProcedures; + this.httpClient = new HttpClient(); + this.assertions = new ConnectAssertions(this); + // we should keep the original class loader and set it back after connector stopped since the connector will change the class loader, + // and then, the Mockito will use the unexpected class loader to generate the wrong proxy instance, which makes mock failed + this.originalClassLoader = Thread.currentThread().getContextClassLoader(); + } + + /** + * @return the set of all {@link WorkerHandle workers}, running or stopped, in the cluster; + * may be empty, but never null + */ + protected abstract Set workers(); + + /** + * Start (or restart) the {@link WorkerHandle workers} in the cluster. + */ + public abstract void startConnect(); + + /** + * A more graceful way to handle abnormal exit of services in integration tests. + */ + public Exit.Procedure exitProcedure = (code, message) -> { + if (code != 0) { + String exitMessage = "Abrupt service exit with code " + code + " and message " + message; + log.warn(exitMessage); + throw new UngracefulShutdownException(exitMessage); + } + }; + + /** + * A more graceful way to handle abnormal halt of services in integration tests. + */ + public Exit.Procedure haltProcedure = (code, message) -> { + if (code != 0) { + String haltMessage = "Abrupt service halt with code " + code + " and message " + message; + log.warn(haltMessage); + throw new UngracefulShutdownException(haltMessage); + } + }; + + /** + * Start the connect cluster and the embedded Kafka and Zookeeper cluster. + */ + public void start() { + if (maskExitProcedures) { + Exit.setExitProcedure(exitProcedure); + Exit.setHaltProcedure(haltProcedure); + } + kafkaCluster.start(); + startConnect(); + try { + httpClient.start(); + } catch (Exception e) { + throw new ConnectException("Failed to start HTTP client", e); + } + } + + /** + * Stop the connect cluster and the embedded Kafka and Zookeeper cluster. + * Clean up any temp directories created locally. + * + * @throws RuntimeException if Kafka brokers fail to stop + */ + public void stop() { + Utils.closeQuietly(httpClient::stop, "HTTP client for embedded Connect cluster"); + workers().forEach(this::stopWorker); + try { + kafkaCluster.stop(); + } catch (UngracefulShutdownException e) { + log.warn("Kafka did not shutdown gracefully"); + } catch (Exception e) { + log.error("Could not stop kafka", e); + throw new RuntimeException("Could not stop brokers", e); + } finally { + if (maskExitProcedures) { + Exit.resetExitProcedure(); + Exit.resetHaltProcedure(); + } + Plugins.compareAndSwapLoaders(originalClassLoader); + } + } + + protected void stopWorker(WorkerHandle worker) { + try { + log.info("Stopping worker {}", worker); + worker.stop(); + } catch (UngracefulShutdownException e) { + log.warn("Worker {} did not shutdown gracefully", worker); + } catch (Exception e) { + log.error("Could not stop connect", e); + throw new RuntimeException("Could not stop worker", e); + } + } + + /** + * Set a new timeout for REST requests to each worker in the cluster. Useful if a request + * is expected to block, since the time spent awaiting that request can be reduced + * and test runtime bloat can be avoided. + * @param requestTimeoutMs the new timeout in milliseconds; must be positive + */ + public void requestTimeout(long requestTimeoutMs) { + workers().forEach(worker -> worker.requestTimeout(requestTimeoutMs)); + } + + /** + * Configure a connector. If the connector does not already exist, a new one will be created and + * the given configuration will be applied to it. + * + * @param connName the name of the connector + * @param connConfig the intended configuration + * @throws ConnectRestException if the REST api returns error status + * @throws ConnectException if the configuration fails to be serialized or if the request could not be sent + */ + public String configureConnector(String connName, Map connConfig) { + String url = endpointForResource(String.format("connectors/%s/config", connName)); + return putConnectorConfig(url, connConfig); + } + + /** + * Validate a given connector configuration. If the configuration validates or + * has a configuration error, an instance of {@link ConfigInfos} is returned. If the validation fails + * an exception is thrown. + * + * @param connClassName the name of the connector class + * @param connConfig the intended configuration + * @throws ConnectRestException if the REST api returns error status + * @throws ConnectException if the configuration fails to serialize/deserialize or if the request failed to send + */ + public ConfigInfos validateConnectorConfig(String connClassName, Map connConfig) { + String url = endpointForResource(String.format("connector-plugins/%s/config/validate", connClassName)); + String response = putConnectorConfig(url, connConfig); + ConfigInfos configInfos; + try { + configInfos = new ObjectMapper().readValue(response, ConfigInfos.class); + } catch (IOException e) { + throw new ConnectException("Unable deserialize response into a ConfigInfos object"); + } + return configInfos; + } + + /** + * Execute a PUT request with the given connector configuration on the given URL endpoint. + * + * @param url the full URL of the endpoint that corresponds to the given REST resource + * @param connConfig the intended configuration + * @throws ConnectRestException if the REST api returns error status + * @throws ConnectException if the configuration fails to be serialized or if the request could not be sent + */ + protected String putConnectorConfig(String url, Map connConfig) { + ObjectMapper mapper = new ObjectMapper(); + String content; + try { + content = mapper.writeValueAsString(connConfig); + } catch (IOException e) { + throw new ConnectException("Could not serialize connector configuration and execute PUT request"); + } + Response response = requestPut(url, content); + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + return responseToString(response); + } + throw new ConnectRestException(response.getStatus(), + "Could not execute PUT request. Error response: " + responseToString(response)); + } + + /** + * Delete an existing connector. + * + * @param connName name of the connector to be deleted + * @throws ConnectRestException if the REST API returns error status + * @throws ConnectException for any other error. + */ + public void deleteConnector(String connName) { + String url = endpointForResource(String.format("connectors/%s", connName)); + Response response = requestDelete(url); + if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { + throw new ConnectRestException(response.getStatus(), + "Could not execute DELETE request. Error response: " + responseToString(response)); + } + } + + /** + * Stop an existing connector. + * + * @param connName name of the connector to be paused + * @throws ConnectRestException if the REST API returns error status + * @throws ConnectException for any other error. + */ + public void stopConnector(String connName) { + String url = endpointForResource(String.format("connectors/%s/stop", connName)); + Response response = requestPut(url, ""); + if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { + throw new ConnectRestException(response.getStatus(), + "Could not execute PUT request. Error response: " + responseToString(response)); + } + } + + /** + * Pause an existing connector. + * + * @param connName name of the connector to be paused + * @throws ConnectRestException if the REST API returns error status + * @throws ConnectException for any other error. + */ + public void pauseConnector(String connName) { + String url = endpointForResource(String.format("connectors/%s/pause", connName)); + Response response = requestPut(url, ""); + if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { + throw new ConnectRestException(response.getStatus(), + "Could not execute PUT request. Error response: " + responseToString(response)); + } + } + + /** + * Resume an existing connector. + * + * @param connName name of the connector to be resumed + * @throws ConnectRestException if the REST API returns error status + * @throws ConnectException for any other error. + */ + public void resumeConnector(String connName) { + String url = endpointForResource(String.format("connectors/%s/resume", connName)); + Response response = requestPut(url, ""); + if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { + throw new ConnectRestException(response.getStatus(), + "Could not execute PUT request. Error response: " + responseToString(response)); + } + } + + /** + * Restart an existing connector. + * + * @param connName name of the connector to be restarted + * @throws ConnectRestException if the REST API returns error status + * @throws ConnectException for any other error. + */ + public void restartConnector(String connName) { + String url = endpointForResource(String.format("connectors/%s/restart", connName)); + Response response = requestPost(url, "", Collections.emptyMap()); + if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { + throw new ConnectRestException(response.getStatus(), + "Could not execute POST request. Error response: " + responseToString(response)); + } + } + + /** + * Restart an existing connector and its tasks. + * + * @param connName name of the connector to be restarted + * @param onlyFailed true if only failed instances should be restarted + * @param includeTasks true if tasks should be restarted, or false if only the connector should be restarted + * @param onlyCallOnEmptyWorker true if the REST API call should be called on a worker not running this connector or its tasks + * @throws ConnectRestException if the REST API returns error status + * @throws ConnectException for any other error. + */ + public ConnectorStateInfo restartConnectorAndTasks(String connName, boolean onlyFailed, boolean includeTasks, boolean onlyCallOnEmptyWorker) { + ObjectMapper mapper = new ObjectMapper(); + String restartPath = String.format("connectors/%s/restart?onlyFailed=" + onlyFailed + "&includeTasks=" + includeTasks, connName); + String restartEndpoint; + if (onlyCallOnEmptyWorker) { + restartEndpoint = endpointForResourceNotRunningConnector(restartPath, connName); + } else { + restartEndpoint = endpointForResource(restartPath); + } + Response response = requestPost(restartEndpoint, "", Collections.emptyMap()); + try { + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + //only the 202 stauts returns a body + if (response.getStatus() == Response.Status.ACCEPTED.getStatusCode()) { + return mapper.readerFor(ConnectorStateInfo.class) + .readValue(responseToString(response)); + } + } + return null; + } catch (IOException e) { + log.error("Could not read connector state from response: {}", + responseToString(response), e); + throw new ConnectException("Could not not parse connector state", e); + } + } + /** + * Get the connector names of the connectors currently running on this cluster. + * + * @return the list of connector names + * @throws ConnectRestException if the HTTP request to the REST API failed with a valid status code. + * @throws ConnectException for any other error. + */ + public Collection connectors() { + ObjectMapper mapper = new ObjectMapper(); + String url = endpointForResource("connectors"); + Response response = requestGet(url); + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + try { + return mapper.readerFor(Collection.class).readValue(responseToString(response)); + } catch (IOException e) { + log.error("Could not parse connector list from response: {}", + responseToString(response), e + ); + throw new ConnectException("Could not not parse connector list", e); + } + } + throw new ConnectRestException(response.getStatus(), + "Could not read connector list. Error response: " + responseToString(response)); + } + + /** + * Get the status for a connector running in this cluster. + * + * @param connectorName name of the connector + * @return an instance of {@link ConnectorStateInfo} populated with state information of the connector and its tasks. + * @throws ConnectRestException if the HTTP request to the REST API failed with a valid status code. + * @throws ConnectException for any other error. + */ + public ConnectorStateInfo connectorStatus(String connectorName) { + ObjectMapper mapper = new ObjectMapper(); + String url = endpointForResource(String.format("connectors/%s/status", connectorName)); + Response response = requestGet(url); + try { + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + return mapper.readerFor(ConnectorStateInfo.class) + .readValue(responseToString(response)); + } + } catch (IOException e) { + log.error("Could not read connector state from response: {}", + responseToString(response), e); + throw new ConnectException("Could not not parse connector state", e); + } + throw new ConnectRestException(response.getStatus(), + "Could not read connector state. Error response: " + responseToString(response)); + } + + /** + * Get the active topics of a connector running in this cluster. + * + * @param connectorName name of the connector + * @return an instance of {@link ConnectorStateInfo} populated with state information of the connector and its tasks. + * @throws ConnectRestException if the HTTP request to the REST API failed with a valid status code. + * @throws ConnectException for any other error. + */ + public ActiveTopicsInfo connectorTopics(String connectorName) { + ObjectMapper mapper = new ObjectMapper(); + String url = endpointForResource(String.format("connectors/%s/topics", connectorName)); + Response response = requestGet(url); + try { + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + Map>> activeTopics = mapper + .readerFor(new TypeReference>>>() { }) + .readValue(responseToString(response)); + return new ActiveTopicsInfo(connectorName, + activeTopics.get(connectorName).getOrDefault("topics", Collections.emptyList())); + } + } catch (IOException e) { + log.error("Could not read connector state from response: {}", + responseToString(response), e); + throw new ConnectException("Could not not parse connector state", e); + } + throw new ConnectRestException(response.getStatus(), + "Could not read connector state. Error response: " + responseToString(response)); + } + + /** + * Get the info of a connector running in this cluster (retrieved via the GET /connectors/{connector} endpoint). + + * @param connectorName name of the connector + * @return an instance of {@link ConnectorInfo} populated with state information of the connector and its tasks. + */ + public ConnectorInfo connectorInfo(String connectorName) { + ObjectMapper mapper = new ObjectMapper(); + String url = endpointForResource(String.format("connectors/%s", connectorName)); + Response response = requestGet(url); + try { + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + return mapper.readValue(responseToString(response), ConnectorInfo.class); + } + } catch (IOException e) { + log.error("Could not read connector info from response: {}", + responseToString(response), e); + throw new ConnectException("Could not not parse connector info", e); + } + throw new ConnectRestException(response.getStatus(), + "Could not read connector info. Error response: " + responseToString(response)); + } + + /** + * Get the task configs of a connector running in this cluster. + * + * @param connectorName name of the connector + * @return a list of task configurations for the connector + */ + public List taskConfigs(String connectorName) { + ObjectMapper mapper = new ObjectMapper(); + 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>() { }); + } + } catch (IOException e) { + log.error("Could not read task configs from response: {}", + responseToString(response), e); + throw new ConnectException("Could not not parse task configs", e); + } + throw new ConnectRestException(response.getStatus(), + "Could not read task configs. Error response: " + responseToString(response)); + } + + /** + * Reset the set of active topics of a connector running in this cluster. + * + * @param connectorName name of the connector + * @throws ConnectRestException if the HTTP request to the REST API failed with a valid status code. + * @throws ConnectException for any other error. + */ + public void resetConnectorTopics(String connectorName) { + String url = endpointForResource(String.format("connectors/%s/topics/reset", connectorName)); + Response response = requestPut(url, null); + if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { + throw new ConnectRestException(response.getStatus(), + "Resetting active topics for connector " + connectorName + " failed. " + + "Error response: " + responseToString(response)); + } + } + + /** + * Get the offsets for a connector via the GET /connectors/{connector}/offsets endpoint + * + * @param connectorName name of the connector whose offsets are to be retrieved + * @return the connector's offsets + */ + public ConnectorOffsets connectorOffsets(String connectorName) { + String url = endpointForResource(String.format("connectors/%s/offsets", connectorName)); + Response response = requestGet(url); + ObjectMapper mapper = new ObjectMapper(); + + try { + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + return mapper.readerFor(ConnectorOffsets.class).readValue(responseToString(response)); + } + } catch (IOException e) { + throw new ConnectException("Could not not parse connector offsets", e); + } + throw new ConnectRestException(response.getStatus(), + "Could not fetch connector offsets. Error response: " + responseToString(response)); + } + + /** + * Alter the offset for a source connector's partition via the PATCH /connectors/{connector}/offsets + * endpoint + * + * @param connectorName name of the source connector whose offset is to be altered + * @param partition the source partition for which the offset is to be altered + * @param offset the source offset to be written + * + * @return the API response as a {@link java.lang.String} + */ + public String alterSourceConnectorOffset(String connectorName, Map partition, Map offset) { + return alterConnectorOffsets( + connectorName, + new ConnectorOffsets(Collections.singletonList(new ConnectorOffset(partition, offset))) + ); + } + + /** + * Alter the offset for a sink connector's topic partition via the PATCH /connectors/{connector}/offsets + * endpoint + * + * @param connectorName name of the sink connector whose offset is to be altered + * @param topicPartition the topic partition for which the offset is to be altered + * @param offset the offset to be written + * + * @return the API response as a {@link java.lang.String} + */ + public String alterSinkConnectorOffset(String connectorName, TopicPartition topicPartition, Long offset) { + return alterConnectorOffsets( + connectorName, + SinkUtils.consumerGroupOffsetsToConnectorOffsets(Collections.singletonMap(topicPartition, new OffsetAndMetadata(offset))) + ); + } + + /** + * Alter a connector's offsets via the PATCH /connectors/{connector}/offsets endpoint + * + * @param connectorName name of the connector whose offsets are to be altered + * @param offsets offsets to alter + * + * @return the API response as a {@link java.lang.String} + */ + public String alterConnectorOffsets(String connectorName, ConnectorOffsets offsets) { + String url = endpointForResource(String.format("connectors/%s/offsets", connectorName)); + ObjectMapper mapper = new ObjectMapper(); + String content; + try { + content = mapper.writeValueAsString(offsets); + } catch (IOException e) { + throw new ConnectException("Could not serialize connector offsets and execute PATCH request"); + } + + Response response = requestPatch(url, content); + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + return responseToString(response); + } else { + throw new ConnectRestException(response.getStatus(), + "Could not alter connector offsets. Error response: " + responseToString(response)); + } + } + + /** + * Reset a connector's offsets via the DELETE /connectors/{connector}/offsets endpoint + * + * @param connectorName name of the connector whose offsets are to be reset + */ + public String resetConnectorOffsets(String connectorName) { + String url = endpointForResource(String.format("connectors/%s/offsets", connectorName)); + Response response = requestDelete(url); + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + return responseToString(response); + } else { + throw new ConnectRestException(response.getStatus(), + "Could not reset connector offsets. Error response: " + responseToString(response)); + } + } + + /** + * Get the {@link LoggerLevel level} for a specific logger + * @param logger the name of the logger + * @return the level for the logger, as reported by the Connect REST API + */ + public LoggerLevel getLogLevel(String logger) { + String resource = "admin/loggers/" + logger; + String url = adminEndpoint(resource); + Response response = requestGet(url); + + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper.readerFor(LoggerLevel.class).readValue(responseToString(response)); + } catch (IOException e) { + log.error("Could not read logger level from response: {}", + responseToString(response), e); + throw new ConnectException("Could not not parse logger level", e); + } + } else { + throw new ConnectRestException( + response.getStatus(), + "Could not read log level. Error response: " + responseToString(response) + ); + } + } + + /** + * Get the {@link LoggerLevel levels} for all known loggers + * @return the levels of all known loggers, as reported by the Connect REST API + */ + public Map allLogLevels() { + String resource = "admin/loggers"; + String url = adminEndpoint(resource); + Response response = requestGet(url); + + if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper + .readerFor(new TypeReference>() { }) + .readValue(responseToString(response)); + } catch (IOException e) { + log.error("Could not read logger levels from response: {}", + responseToString(response), e); + throw new ConnectException("Could not not parse logger levels", e); + } + } else { + throw new ConnectRestException( + response.getStatus(), + "Could not read log levels. Error response: " + responseToString(response) + ); + } + } + + /** + * Adjust the level of a logging namespace. + * @param namespace the namespace to adjust; may not be null + * @param level the level to set the namespace to; may not be null + * @param scope the scope of the operation; may be null + * @return the list of affected loggers, as reported by the Connect REST API; + * may be null if no body was included in the response + */ + public List setLogLevel(String namespace, String level, String scope) { + String resource = "admin/loggers/" + namespace; + if (scope != null) + resource += "?scope=" + scope; + String url = adminEndpoint(resource); + String body = "{\"level\": \"" + level + "\"}"; + Response response = requestPut(url, body); + + if (response.getStatus() == Response.Status.NO_CONTENT.getStatusCode()) { + if (response.getEntity() != null && !response.getEntity().equals("")) { + // Don't use JUnit assertNull here because this library is used by both + // Connect runtime tests and MirrorMaker 2 tests, which use different + // versions of JUnit + throw new AssertionError( + "Response with 204 status contained non-null entity: '" + + response.getEntity() + "'" + ); + } + return null; + } else if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper + .readerFor(new TypeReference>() { }) + .readValue(responseToString(response)); + } catch (IOException e) { + log.error("Could not read loggers from response: {}", + responseToString(response), e); + throw new ConnectException("Could not not parse loggers", e); + } + } else { + throw new ConnectRestException( + response.getStatus(), + "Could not set log level. Error response: " + responseToString(response) + ); + } + } + + /** + * Get the full URL of the admin endpoint that corresponds to the given REST resource + * + * @param resource the resource under the worker's admin endpoint + * @return the admin endpoint URL + * @throws ConnectException if no admin REST endpoint is available + */ + public String adminEndpoint(String resource) { + String url = workers().stream() + .map(WorkerHandle::adminUrl) + .filter(Objects::nonNull) + .findFirst() + .orElseThrow(() -> new ConnectException("Admin endpoint is disabled.")) + .toString(); + return url + resource; + } + + /** + * Get the full URL of the endpoint that corresponds to the given REST resource + * + * @param resource the resource under the worker's admin endpoint + * @return the admin endpoint URL + * @throws ConnectException if no REST endpoint is available + */ + public String endpointForResource(String resource) { + String url = workers().stream() + .map(WorkerHandle::url) + .filter(Objects::nonNull) + .findFirst() + .orElseThrow(() -> new ConnectException("Connect workers have not been provisioned")) + .toString(); + 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 endpointForResourceNotRunningConnector(String resource, String connectorName) { + ConnectorStateInfo info = connectorStatus(connectorName); + Set activeWorkerUrls = new HashSet<>(); + activeWorkerUrls.add(String.format("http://%s/", info.connector().workerId())); + info.tasks().forEach(t -> activeWorkerUrls.add(String.format("http://%s/", t.workerId()))); + String url = workers().stream() + .map(WorkerHandle::url) + .filter(Objects::nonNull) + .filter(workerUrl -> !activeWorkerUrls.contains(workerUrl.toString())) + .findFirst() + .orElseThrow(() -> new ConnectException( + String.format("Connect workers have not been provisioned or no free worker found that is not running this connector(%s) or its tasks", connectorName))) + .toString(); + return url + resource; + } + + /** + * Return the handle to the Kafka cluster this Connect cluster connects to. + * + * @return the Kafka cluster handle + */ + public EmbeddedKafkaCluster kafka() { + return kafkaCluster; + } + + /** + * Execute a GET request on the given URL. + * + * @param url the HTTP endpoint + * @return the response to the GET request + * @throws ConnectException if execution of the GET request fails + * @deprecated Use {@link #requestGet(String)} instead. + */ + @Deprecated + public String executeGet(String url) { + return responseToString(requestGet(url)); + } + + /** + * Execute a GET request on the given URL. + * + * @param url the HTTP endpoint + * @return the response to the GET request + * @throws ConnectException if execution of the GET request fails + */ + public Response requestGet(String url) { + return requestHttpMethod(url, null, Collections.emptyMap(), "GET"); + } + + /** + * Execute a PUT request on the given URL. + * + * @param url the HTTP endpoint + * @param body the payload of the PUT request + * @return the response to the PUT request + * @throws ConnectException if execution of the PUT request fails + * @deprecated Use {@link #requestPut(String, String)} instead. + */ + @Deprecated + public int executePut(String url, String body) { + return requestPut(url, body).getStatus(); + } + + /** + * Execute a PUT request on the given URL. + * + * @param url the HTTP endpoint + * @param body the payload of the PUT request + * @return the response to the PUT request + * @throws ConnectException if execution of the PUT request fails + */ + public Response requestPut(String url, String body) { + return requestHttpMethod(url, body, Collections.emptyMap(), "PUT"); + } + + /** + * Execute a POST request on the given URL. + * + * @param url the HTTP endpoint + * @param body the payload of the POST request + * @param headers a map that stores the POST request headers + * @return the response to the POST request + * @throws ConnectException if execution of the POST request fails + * @deprecated Use {@link #requestPost(String, String, java.util.Map)} instead. + */ + @Deprecated + public int executePost(String url, String body, Map headers) { + return requestPost(url, body, headers).getStatus(); + } + + /** + * Execute a POST request on the given URL. + * + * @param url the HTTP endpoint + * @param body the payload of the POST request + * @param headers a map that stores the POST request headers + * @return the response to the POST request + * @throws ConnectException if execution of the POST request fails + */ + public Response requestPost(String url, String body, Map headers) { + return requestHttpMethod(url, body, headers, "POST"); + } + + /** + * Execute a PATCH request on the given URL. + * + * @param url the HTTP endpoint + * @param body the payload of the PATCH request + * @return the response to the PATCH request + * @throws ConnectException if execution of the PATCH request fails + */ + public Response requestPatch(String url, String body) { + return requestHttpMethod(url, body, Collections.emptyMap(), "PATCH"); + } + + /** + * Execute a DELETE request on the given URL. + * + * @param url the HTTP endpoint + * @return the response to the DELETE request + * @throws ConnectException if execution of the DELETE request fails + * @deprecated Use {@link #requestDelete(String)} instead. + */ + @Deprecated + public int executeDelete(String url) { + return requestDelete(url).getStatus(); + } + + /** + * Execute a DELETE request on the given URL. + * + * @param url the HTTP endpoint + * @return the response to the DELETE request + * @throws ConnectException if execution of the DELETE request fails + */ + public Response requestDelete(String url) { + return requestHttpMethod(url, null, Collections.emptyMap(), "DELETE"); + } + + /** + * A general method that executes an HTTP request on a given URL. + * + * @param url the HTTP endpoint + * @param body the payload of the request; null if there isn't one + * @param headers a map that stores the request headers; empty if there are no headers + * @param httpMethod the name of the HTTP method to execute + * @return the response to the HTTP request + * @throws ConnectException if execution of the HTTP method fails + */ + protected Response requestHttpMethod(String url, String body, Map headers, + String httpMethod) { + log.debug("Executing {} request to URL={}." + (body != null ? " Payload={}" : ""), + httpMethod, url, body); + + try { + Request req = httpClient.newRequest(url); + req.method(httpMethod); + if (body != null) { + headers.forEach(req::header); + req.content(new StringContentProvider(body), "application/json"); + } + + ContentResponse res = req.send(); + log.info("{} response for URL={} is {}", + httpMethod, url, res.getContentAsString().isEmpty() ? "empty" : res.getContentAsString()); + return Response.status(Response.Status.fromStatusCode(res.getStatus())) + .entity(res.getContentAsString()) + .build(); + } catch (Exception e) { + log.error("Could not execute " + httpMethod + " request to " + url, e); + throw new ConnectException(e); + } + } + + private String responseToString(Response response) { + return response == null ? "empty" : (String) response.getEntity(); + } + + /** + * Get the workers that are up and running. + * + * @return the list of handles of the online workers + */ + public Set activeWorkers() { + ObjectMapper mapper = new ObjectMapper(); + return workers().stream() + .filter(w -> { + try { + mapper.readerFor(ServerInfo.class) + .readValue(responseToString(requestGet(w.url().toString()))); + return true; + } catch (ConnectException | IOException e) { + // Worker failed to respond. Consider it's offline + return false; + } + }) + .collect(Collectors.toSet()); + } + + + /** + * Return the available assertions for this Connect cluster + * + * @return the assertions object + */ + public ConnectAssertions assertions() { + return assertions; + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectBuilder.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectBuilder.java new file mode 100644 index 00000000000..645f330bfa3 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectBuilder.java @@ -0,0 +1,84 @@ +/* + * 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.util.clusters; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +abstract class EmbeddedConnectBuilder> { + private Map workerProps = new HashMap<>(); + private int numBrokers = EmbeddedConnect.DEFAULT_NUM_BROKERS; + private Properties brokerProps = new Properties(); + private boolean maskExitProcedures = true; + private final Map clientProps = new HashMap<>(); + + protected abstract C build( + int numBrokers, + Properties brokerProps, + boolean maskExitProcedures, + Map clientProps, + Map workerProps + ); + + public B workerProps(Map workerProps) { + this.workerProps = workerProps; + return self(); + } + + public B numBrokers(int numBrokers) { + this.numBrokers = numBrokers; + return self(); + } + + public B brokerProps(Properties brokerProps) { + this.brokerProps = brokerProps; + return self(); + } + + public B clientProps(Map clientProps) { + this.clientProps.putAll(clientProps); + return self(); + } + + /** + * In the event of ungraceful shutdown, embedded clusters call exit or halt with non-zero + * exit statuses. Exiting with a non-zero status forces a test to fail and is hard to + * handle. Because graceful exit is usually not required during a test and because + * depending on such an exit increases flakiness, this setting allows masking + * exit and halt procedures by using a runtime exception instead. Customization of the + * exit and halt procedures is possible through {@code exitProcedure} and {@code + * haltProcedure} respectively. + * + * @param mask if false, exit and halt procedures remain unchanged; true is the default. + * @return the builder for this cluster + */ + public B maskExitProcedures(boolean mask) { + this.maskExitProcedures = mask; + return self(); + } + + public C build() { + return build(numBrokers, brokerProps, maskExitProcedures, clientProps, workerProps); + } + + @SuppressWarnings("unchecked") + protected B self() { + return (B) this; + } + +} 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 9e3edcd8f24..00156cceeec 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 @@ -16,47 +16,16 @@ */ package org.apache.kafka.connect.util.clusters; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Exit; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.runtime.isolation.Plugins; -import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo; -import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; -import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; -import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffset; -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; -import org.eclipse.jetty.client.api.ContentResponse; -import org.eclipse.jetty.client.api.Request; -import org.eclipse.jetty.client.util.StringContentProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.ws.rs.core.Response; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashSet; -import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Properties; import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; @@ -72,115 +41,42 @@ import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.STA import static org.apache.kafka.connect.runtime.rest.RestServerConfig.LISTENERS_CONFIG; /** - * Start an embedded connect worker. Internally, this class will spin up a Kafka and Zk cluster, setup any tmp - * directories and clean up them on them. Methods on the same {@code EmbeddedConnectCluster} are + * Start an embedded connect cluster. Internally, this class will spin up a Kafka and Zk cluster, set up any tmp + * directories, and clean them up on exit. Methods on the same {@code EmbeddedConnectCluster} are * not guaranteed to be thread-safe. */ -public class EmbeddedConnectCluster { +public class EmbeddedConnectCluster extends EmbeddedConnect { private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectCluster.class); - public static final int DEFAULT_NUM_BROKERS = 1; public static final int DEFAULT_NUM_WORKERS = 1; - private static final Properties DEFAULT_BROKER_CONFIG = new Properties(); private static final String REST_HOST_NAME = "localhost"; private static final String DEFAULT_WORKER_NAME_PREFIX = "connect-worker-"; private final Set connectCluster; - private final EmbeddedKafkaCluster kafkaCluster; - private final HttpClient httpClient; private final Map workerProps; private final String connectClusterName; - private final int numBrokers; private final int numInitialWorkers; - private final boolean maskExitProcedures; private final String workerNamePrefix; private final AtomicInteger nextWorkerId = new AtomicInteger(0); - private final EmbeddedConnectClusterAssertions assertions; - // we should keep the original class loader and set it back after connector stopped since the connector will change the class loader, - // and then, the Mockito will use the unexpected class loader to generate the wrong proxy instance, which makes mock failed - private final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader(); - private EmbeddedConnectCluster(String name, Map workerProps, int numWorkers, - int numBrokers, Properties brokerProps, - boolean maskExitProcedures, - Map additionalKafkaClusterClientConfigs) { + private EmbeddedConnectCluster( + int numBrokers, + Properties brokerProps, + boolean maskExitProcedures, + Map clientProps, + Map workerProps, + String name, + int numWorkers + ) { + super(numBrokers, brokerProps, maskExitProcedures, clientProps); this.workerProps = workerProps; this.connectClusterName = name; - this.numBrokers = numBrokers; - this.kafkaCluster = new EmbeddedKafkaCluster(numBrokers, brokerProps, additionalKafkaClusterClientConfigs); this.connectCluster = new LinkedHashSet<>(); - this.httpClient = new HttpClient(); this.numInitialWorkers = numWorkers; - this.maskExitProcedures = maskExitProcedures; // leaving non-configurable for now this.workerNamePrefix = DEFAULT_WORKER_NAME_PREFIX; - this.assertions = new EmbeddedConnectClusterAssertions(this); - } - - /** - * A more graceful way to handle abnormal exit of services in integration tests. - */ - public Exit.Procedure exitProcedure = (code, message) -> { - if (code != 0) { - String exitMessage = "Abrupt service exit with code " + code + " and message " + message; - log.warn(exitMessage); - throw new UngracefulShutdownException(exitMessage); - } - }; - - /** - * A more graceful way to handle abnormal halt of services in integration tests. - */ - public Exit.Procedure haltProcedure = (code, message) -> { - if (code != 0) { - String haltMessage = "Abrupt service halt with code " + code + " and message " + message; - log.warn(haltMessage); - throw new UngracefulShutdownException(haltMessage); - } - }; - - /** - * Start the connect cluster and the embedded Kafka and Zookeeper cluster. - */ - public void start() { - if (maskExitProcedures) { - Exit.setExitProcedure(exitProcedure); - Exit.setHaltProcedure(haltProcedure); - } - kafkaCluster.start(); - startConnect(); - try { - httpClient.start(); - } catch (Exception e) { - throw new ConnectException("Failed to start HTTP client", e); - } - } - - /** - * Stop the connect cluster and the embedded Kafka and Zookeeper cluster. - * Clean up any temp directories created locally. - * - * @throws RuntimeException if Kafka brokers fail to stop - */ - public void stop() { - Utils.closeQuietly(httpClient::stop, "HTTP client for embedded Connect cluster"); - connectCluster.forEach(this::stopWorker); - try { - kafkaCluster.stop(); - } catch (UngracefulShutdownException e) { - log.warn("Kafka did not shutdown gracefully"); - } catch (Exception e) { - log.error("Could not stop kafka", e); - throw new RuntimeException("Could not stop brokers", e); - } finally { - if (maskExitProcedures) { - Exit.resetExitProcedure(); - Exit.resetHaltProcedure(); - } - Plugins.compareAndSwapLoaders(originalClassLoader); - } } /** @@ -225,28 +121,6 @@ public class EmbeddedConnectCluster { connectCluster.remove(worker); } - private void stopWorker(WorkerHandle worker) { - try { - log.info("Stopping worker {}", worker); - worker.stop(); - } catch (UngracefulShutdownException e) { - log.warn("Worker {} did not shutdown gracefully", worker); - } catch (Exception e) { - log.error("Could not stop connect", e); - throw new RuntimeException("Could not stop worker", e); - } - } - - /** - * Set a new timeout for REST requests to each worker in the cluster. Useful if a request - * is expected to block, since the time spent awaiting that request can be reduced - * and test runtime bloat can be avoided. - * @param requestTimeoutMs the new timeout in milliseconds; must be positive - */ - public void requestTimeout(long requestTimeoutMs) { - connectCluster.forEach(worker -> worker.requestTimeout(requestTimeoutMs)); - } - /** * Determine whether the Connect cluster has any workers running. * @@ -265,6 +139,7 @@ public class EmbeddedConnectCluster { return workers().stream().allMatch(WorkerHandle::isRunning); } + @Override public void startConnect() { log.info("Starting Connect cluster '{}' with {} workers", connectClusterName, numInitialWorkers); @@ -273,16 +148,16 @@ public class EmbeddedConnectCluster { workerProps.put(LISTENERS_CONFIG, "HTTP://" + REST_HOST_NAME + ":0"); String internalTopicsReplFactor = String.valueOf(numBrokers); - putIfAbsent(workerProps, GROUP_ID_CONFIG, "connect-integration-test-" + connectClusterName); - putIfAbsent(workerProps, OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + connectClusterName); - putIfAbsent(workerProps, OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); - putIfAbsent(workerProps, CONFIG_TOPIC_CONFIG, "connect-config-topic-" + connectClusterName); - putIfAbsent(workerProps, CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); - putIfAbsent(workerProps, STATUS_STORAGE_TOPIC_CONFIG, "connect-status-topic-" + connectClusterName); - putIfAbsent(workerProps, STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); - putIfAbsent(workerProps, KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); - putIfAbsent(workerProps, VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); - putIfAbsent(workerProps, PLUGIN_DISCOVERY_CONFIG, "hybrid_fail"); + workerProps.putIfAbsent(GROUP_ID_CONFIG, "connect-integration-test-" + connectClusterName); + workerProps.putIfAbsent(OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + connectClusterName); + workerProps.putIfAbsent(OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); + workerProps.putIfAbsent(CONFIG_TOPIC_CONFIG, "connect-config-topic-" + connectClusterName); + workerProps.putIfAbsent(CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); + workerProps.putIfAbsent(STATUS_STORAGE_TOPIC_CONFIG, "connect-status-topic-" + connectClusterName); + workerProps.putIfAbsent(STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); + workerProps.putIfAbsent(KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + workerProps.putIfAbsent(VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + workerProps.putIfAbsent(PLUGIN_DISCOVERY_CONFIG, "hybrid_fail"); for (int i = 0; i < numInitialWorkers; i++) { addWorker(); @@ -302,27 +177,6 @@ public class EmbeddedConnectCluster { return connectClusterName; } - /** - * Get the workers that are up and running. - * - * @return the list of handles of the online workers - */ - public Set activeWorkers() { - ObjectMapper mapper = new ObjectMapper(); - return connectCluster.stream() - .filter(w -> { - try { - mapper.readerFor(ServerInfo.class) - .readValue(responseToString(requestGet(w.url().toString()))); - return true; - } catch (ConnectException | IOException e) { - // Worker failed to respond. Consider it's offline - return false; - } - }) - .collect(Collectors.toSet()); - } - /** * Get the provisioned workers. * @@ -332,722 +186,46 @@ public class EmbeddedConnectCluster { return new LinkedHashSet<>(connectCluster); } - /** - * Configure a connector. If the connector does not already exist, a new one will be created and - * the given configuration will be applied to it. - * - * @param connName the name of the connector - * @param connConfig the intended configuration - * @throws ConnectRestException if the REST api returns error status - * @throws ConnectException if the configuration fails to be serialized or if the request could not be sent - */ - public String configureConnector(String connName, Map connConfig) { - String url = endpointForResource(String.format("connectors/%s/config", connName)); - return putConnectorConfig(url, connConfig); - } - - /** - * Validate a given connector configuration. If the configuration validates or - * has a configuration error, an instance of {@link ConfigInfos} is returned. If the validation fails - * an exception is thrown. - * - * @param connClassName the name of the connector class - * @param connConfig the intended configuration - * @throws ConnectRestException if the REST api returns error status - * @throws ConnectException if the configuration fails to serialize/deserialize or if the request failed to send - */ - public ConfigInfos validateConnectorConfig(String connClassName, Map connConfig) { - String url = endpointForResource(String.format("connector-plugins/%s/config/validate", connClassName)); - String response = putConnectorConfig(url, connConfig); - ConfigInfos configInfos; - try { - configInfos = new ObjectMapper().readValue(response, ConfigInfos.class); - } catch (IOException e) { - throw new ConnectException("Unable deserialize response into a ConfigInfos object"); - } - return configInfos; - } - - /** - * Execute a PUT request with the given connector configuration on the given URL endpoint. - * - * @param url the full URL of the endpoint that corresponds to the given REST resource - * @param connConfig the intended configuration - * @throws ConnectRestException if the REST api returns error status - * @throws ConnectException if the configuration fails to be serialized or if the request could not be sent - */ - protected String putConnectorConfig(String url, Map connConfig) { - ObjectMapper mapper = new ObjectMapper(); - String content; - try { - content = mapper.writeValueAsString(connConfig); - } catch (IOException e) { - throw new ConnectException("Could not serialize connector configuration and execute PUT request"); - } - Response response = requestPut(url, content); - if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { - return responseToString(response); - } - throw new ConnectRestException(response.getStatus(), - "Could not execute PUT request. Error response: " + responseToString(response)); - } - - /** - * Delete an existing connector. - * - * @param connName name of the connector to be deleted - * @throws ConnectRestException if the REST API returns error status - * @throws ConnectException for any other error. - */ - public void deleteConnector(String connName) { - String url = endpointForResource(String.format("connectors/%s", connName)); - Response response = requestDelete(url); - if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { - throw new ConnectRestException(response.getStatus(), - "Could not execute DELETE request. Error response: " + responseToString(response)); - } - } - - /** - * Stop an existing connector. - * - * @param connName name of the connector to be paused - * @throws ConnectRestException if the REST API returns error status - * @throws ConnectException for any other error. - */ - public void stopConnector(String connName) { - String url = endpointForResource(String.format("connectors/%s/stop", connName)); - Response response = requestPut(url, ""); - if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { - throw new ConnectRestException(response.getStatus(), - "Could not execute PUT request. Error response: " + responseToString(response)); - } - } - - /** - * Pause an existing connector. - * - * @param connName name of the connector to be paused - * @throws ConnectRestException if the REST API returns error status - * @throws ConnectException for any other error. - */ - public void pauseConnector(String connName) { - String url = endpointForResource(String.format("connectors/%s/pause", connName)); - Response response = requestPut(url, ""); - if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { - throw new ConnectRestException(response.getStatus(), - "Could not execute PUT request. Error response: " + responseToString(response)); - } - } - - /** - * Resume an existing connector. - * - * @param connName name of the connector to be resumed - * @throws ConnectRestException if the REST API returns error status - * @throws ConnectException for any other error. - */ - public void resumeConnector(String connName) { - String url = endpointForResource(String.format("connectors/%s/resume", connName)); - Response response = requestPut(url, ""); - if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { - throw new ConnectRestException(response.getStatus(), - "Could not execute PUT request. Error response: " + responseToString(response)); - } - } - - /** - * Restart an existing connector. - * - * @param connName name of the connector to be restarted - * @throws ConnectRestException if the REST API returns error status - * @throws ConnectException for any other error. - */ - public void restartConnector(String connName) { - String url = endpointForResource(String.format("connectors/%s/restart", connName)); - Response response = requestPost(url, "", Collections.emptyMap()); - if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { - throw new ConnectRestException(response.getStatus(), - "Could not execute POST request. Error response: " + responseToString(response)); - } - } - - /** - * Restart an existing connector and its tasks. - * - * @param connName name of the connector to be restarted - * @param onlyFailed true if only failed instances should be restarted - * @param includeTasks true if tasks should be restarted, or false if only the connector should be restarted - * @param onlyCallOnEmptyWorker true if the REST API call should be called on a worker not running this connector or its tasks - * @throws ConnectRestException if the REST API returns error status - * @throws ConnectException for any other error. - */ - public ConnectorStateInfo restartConnectorAndTasks(String connName, boolean onlyFailed, boolean includeTasks, boolean onlyCallOnEmptyWorker) { - ObjectMapper mapper = new ObjectMapper(); - String restartPath = String.format("connectors/%s/restart?onlyFailed=" + onlyFailed + "&includeTasks=" + includeTasks, connName); - String restartEndpoint; - if (onlyCallOnEmptyWorker) { - restartEndpoint = endpointForResourceNotRunningConnector(restartPath, connName); - } else { - restartEndpoint = endpointForResource(restartPath); - } - Response response = requestPost(restartEndpoint, "", Collections.emptyMap()); - try { - if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { - //only the 202 stauts returns a body - if (response.getStatus() == Response.Status.ACCEPTED.getStatusCode()) { - return mapper.readerFor(ConnectorStateInfo.class) - .readValue(responseToString(response)); - } - } - return null; - } catch (IOException e) { - log.error("Could not read connector state from response: {}", - responseToString(response), e); - throw new ConnectException("Could not not parse connector state", e); - } - } - /** - * Get the connector names of the connectors currently running on this cluster. - * - * @return the list of connector names - * @throws ConnectRestException if the HTTP request to the REST API failed with a valid status code. - * @throws ConnectException for any other error. - */ - public Collection connectors() { - ObjectMapper mapper = new ObjectMapper(); - String url = endpointForResource("connectors"); - Response response = requestGet(url); - if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { - try { - return mapper.readerFor(Collection.class).readValue(responseToString(response)); - } catch (IOException e) { - log.error("Could not parse connector list from response: {}", - responseToString(response), e - ); - throw new ConnectException("Could not not parse connector list", e); - } - } - throw new ConnectRestException(response.getStatus(), - "Could not read connector list. Error response: " + responseToString(response)); - } - - /** - * Get the status for a connector running in this cluster. - * - * @param connectorName name of the connector - * @return an instance of {@link ConnectorStateInfo} populated with state information of the connector and its tasks. - * @throws ConnectRestException if the HTTP request to the REST API failed with a valid status code. - * @throws ConnectException for any other error. - */ - public ConnectorStateInfo connectorStatus(String connectorName) { - ObjectMapper mapper = new ObjectMapper(); - String url = endpointForResource(String.format("connectors/%s/status", connectorName)); - Response response = requestGet(url); - try { - if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { - return mapper.readerFor(ConnectorStateInfo.class) - .readValue(responseToString(response)); - } - } catch (IOException e) { - log.error("Could not read connector state from response: {}", - responseToString(response), e); - throw new ConnectException("Could not not parse connector state", e); - } - throw new ConnectRestException(response.getStatus(), - "Could not read connector state. Error response: " + responseToString(response)); - } - - /** - * Get the active topics of a connector running in this cluster. - * - * @param connectorName name of the connector - * @return an instance of {@link ConnectorStateInfo} populated with state information of the connector and its tasks. - * @throws ConnectRestException if the HTTP request to the REST API failed with a valid status code. - * @throws ConnectException for any other error. - */ - public ActiveTopicsInfo connectorTopics(String connectorName) { - ObjectMapper mapper = new ObjectMapper(); - String url = endpointForResource(String.format("connectors/%s/topics", connectorName)); - Response response = requestGet(url); - try { - if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { - Map>> activeTopics = mapper - .readerFor(new TypeReference>>>() { }) - .readValue(responseToString(response)); - return new ActiveTopicsInfo(connectorName, - activeTopics.get(connectorName).getOrDefault("topics", Collections.emptyList())); - } - } catch (IOException e) { - log.error("Could not read connector state from response: {}", - responseToString(response), e); - throw new ConnectException("Could not not parse connector state", e); - } - throw new ConnectRestException(response.getStatus(), - "Could not read connector state. Error response: " + responseToString(response)); - } - - /** - * Get the info of a connector running in this cluster (retrieved via the GET /connectors/{connector} endpoint). - - * @param connectorName name of the connector - * @return an instance of {@link ConnectorInfo} populated with state information of the connector and its tasks. - */ - public ConnectorInfo connectorInfo(String connectorName) { - ObjectMapper mapper = new ObjectMapper(); - String url = endpointForResource(String.format("connectors/%s", connectorName)); - Response response = requestGet(url); - try { - if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { - return mapper.readValue(responseToString(response), ConnectorInfo.class); - } - } catch (IOException e) { - log.error("Could not read connector info from response: {}", - responseToString(response), e); - throw new ConnectException("Could not not parse connector info", e); - } - throw new ConnectRestException(response.getStatus(), - "Could not read connector info. Error response: " + responseToString(response)); - } - - /** - * Get the task configs of a connector running in this cluster. - * - * @param connectorName name of the connector - * @return a list of task configurations for the connector - */ - public List taskConfigs(String connectorName) { - ObjectMapper mapper = new ObjectMapper(); - 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>() { }); - } - } catch (IOException e) { - log.error("Could not read task configs from response: {}", - responseToString(response), e); - throw new ConnectException("Could not not parse task configs", e); - } - throw new ConnectRestException(response.getStatus(), - "Could not read task configs. Error response: " + responseToString(response)); - } - - /** - * Reset the set of active topics of a connector running in this cluster. - * - * @param connectorName name of the connector - * @throws ConnectRestException if the HTTP request to the REST API failed with a valid status code. - * @throws ConnectException for any other error. - */ - public void resetConnectorTopics(String connectorName) { - String url = endpointForResource(String.format("connectors/%s/topics/reset", connectorName)); - Response response = requestPut(url, null); - if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { - throw new ConnectRestException(response.getStatus(), - "Resetting active topics for connector " + connectorName + " failed. " - + "Error response: " + responseToString(response)); - } - } - - /** - * Get the offsets for a connector via the GET /connectors/{connector}/offsets endpoint - * - * @param connectorName name of the connector whose offsets are to be retrieved - * @return the connector's offsets - */ - public ConnectorOffsets connectorOffsets(String connectorName) { - String url = endpointForResource(String.format("connectors/%s/offsets", connectorName)); - Response response = requestGet(url); - ObjectMapper mapper = new ObjectMapper(); - - try { - if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { - return mapper.readerFor(ConnectorOffsets.class).readValue(responseToString(response)); - } - } catch (IOException e) { - throw new ConnectException("Could not not parse connector offsets", e); - } - throw new ConnectRestException(response.getStatus(), - "Could not fetch connector offsets. Error response: " + responseToString(response)); - } - - /** - * Alter the offset for a source connector's partition via the PATCH /connectors/{connector}/offsets - * endpoint - * - * @param connectorName name of the source connector whose offset is to be altered - * @param partition the source partition for which the offset is to be altered - * @param offset the source offset to be written - * - * @return the API response as a {@link java.lang.String} - */ - public String alterSourceConnectorOffset(String connectorName, Map partition, Map offset) { - return alterConnectorOffsets( - connectorName, - new ConnectorOffsets(Collections.singletonList(new ConnectorOffset(partition, offset))) - ); - } - - /** - * Alter the offset for a sink connector's topic partition via the PATCH /connectors/{connector}/offsets - * endpoint - * - * @param connectorName name of the sink connector whose offset is to be altered - * @param topicPartition the topic partition for which the offset is to be altered - * @param offset the offset to be written - * - * @return the API response as a {@link java.lang.String} - */ - public String alterSinkConnectorOffset(String connectorName, TopicPartition topicPartition, Long offset) { - return alterConnectorOffsets( - connectorName, - SinkUtils.consumerGroupOffsetsToConnectorOffsets(Collections.singletonMap(topicPartition, new OffsetAndMetadata(offset))) - ); - } - - /** - * Alter a connector's offsets via the PATCH /connectors/{connector}/offsets endpoint - * - * @param connectorName name of the connector whose offsets are to be altered - * @param offsets offsets to alter - * - * @return the API response as a {@link java.lang.String} - */ - public String alterConnectorOffsets(String connectorName, ConnectorOffsets offsets) { - String url = endpointForResource(String.format("connectors/%s/offsets", connectorName)); - ObjectMapper mapper = new ObjectMapper(); - String content; - try { - content = mapper.writeValueAsString(offsets); - } catch (IOException e) { - throw new ConnectException("Could not serialize connector offsets and execute PATCH request"); - } - - Response response = requestPatch(url, content); - if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { - return responseToString(response); - } else { - throw new ConnectRestException(response.getStatus(), - "Could not alter connector offsets. Error response: " + responseToString(response)); - } - } - - /** - * Reset a connector's offsets via the DELETE /connectors/{connector}/offsets endpoint - * - * @param connectorName name of the connector whose offsets are to be reset - */ - public String resetConnectorOffsets(String connectorName) { - String url = endpointForResource(String.format("connectors/%s/offsets", connectorName)); - Response response = requestDelete(url); - if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { - return responseToString(response); - } else { - throw new ConnectRestException(response.getStatus(), - "Could not reset connector offsets. Error response: " + responseToString(response)); - } - } - - /** - * Get the full URL of the admin endpoint that corresponds to the given REST resource - * - * @param resource the resource under the worker's admin endpoint - * @return the admin endpoint URL - * @throws ConnectException if no admin REST endpoint is available - */ - public String adminEndpoint(String resource) { - String url = connectCluster.stream() - .map(WorkerHandle::adminUrl) - .filter(Objects::nonNull) - .findFirst() - .orElseThrow(() -> new ConnectException("Admin endpoint is disabled.")) - .toString(); - return url + resource; - } - - /** - * Get the full URL of the endpoint that corresponds to the given REST resource - * - * @param resource the resource under the worker's admin endpoint - * @return the admin endpoint URL - * @throws ConnectException if no REST endpoint is available - */ - public String endpointForResource(String resource) { - String url = connectCluster.stream() - .map(WorkerHandle::url) - .filter(Objects::nonNull) - .findFirst() - .orElseThrow(() -> new ConnectException("Connect workers have not been provisioned")) - .toString(); - 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 endpointForResourceNotRunningConnector(String resource, String connectorName) { - ConnectorStateInfo info = connectorStatus(connectorName); - Set activeWorkerUrls = new HashSet<>(); - activeWorkerUrls.add(String.format("http://%s/", info.connector().workerId())); - info.tasks().forEach(t -> activeWorkerUrls.add(String.format("http://%s/", t.workerId()))); - String url = connectCluster.stream() - .map(WorkerHandle::url) - .filter(Objects::nonNull) - .filter(workerUrl -> !activeWorkerUrls.contains(workerUrl.toString())) - .findFirst() - .orElseThrow(() -> new ConnectException( - String.format("Connect workers have not been provisioned or no free worker found that is not running this connector(%s) or its tasks", connectorName))) - .toString(); - return url + resource; - } - - private static void putIfAbsent(Map props, String propertyKey, String propertyValue) { - if (!props.containsKey(propertyKey)) { - props.put(propertyKey, propertyValue); - } - } - - /** - * Return the handle to the Kafka cluster this Connect cluster connects to. - * - * @return the Kafka cluster handle - */ - public EmbeddedKafkaCluster kafka() { - return kafkaCluster; - } - - /** - * Execute a GET request on the given URL. - * - * @param url the HTTP endpoint - * @return the response to the GET request - * @throws ConnectException if execution of the GET request fails - * @deprecated Use {@link #requestGet(String)} instead. - */ - @Deprecated - public String executeGet(String url) { - return responseToString(requestGet(url)); - } - - /** - * Execute a GET request on the given URL. - * - * @param url the HTTP endpoint - * @return the response to the GET request - * @throws ConnectException if execution of the GET request fails - */ - public Response requestGet(String url) { - return requestHttpMethod(url, null, Collections.emptyMap(), "GET"); - } - - /** - * Execute a PUT request on the given URL. - * - * @param url the HTTP endpoint - * @param body the payload of the PUT request - * @return the response to the PUT request - * @throws ConnectException if execution of the PUT request fails - * @deprecated Use {@link #requestPut(String, String)} instead. - */ - @Deprecated - public int executePut(String url, String body) { - return requestPut(url, body).getStatus(); - } - - /** - * Execute a PUT request on the given URL. - * - * @param url the HTTP endpoint - * @param body the payload of the PUT request - * @return the response to the PUT request - * @throws ConnectException if execution of the PUT request fails - */ - public Response requestPut(String url, String body) { - return requestHttpMethod(url, body, Collections.emptyMap(), "PUT"); - } - - /** - * Execute a POST request on the given URL. - * - * @param url the HTTP endpoint - * @param body the payload of the POST request - * @param headers a map that stores the POST request headers - * @return the response to the POST request - * @throws ConnectException if execution of the POST request fails - * @deprecated Use {@link #requestPost(String, String, java.util.Map)} instead. - */ - @Deprecated - public int executePost(String url, String body, Map headers) { - return requestPost(url, body, headers).getStatus(); - } - - /** - * Execute a POST request on the given URL. - * - * @param url the HTTP endpoint - * @param body the payload of the POST request - * @param headers a map that stores the POST request headers - * @return the response to the POST request - * @throws ConnectException if execution of the POST request fails - */ - public Response requestPost(String url, String body, Map headers) { - return requestHttpMethod(url, body, headers, "POST"); - } - - /** - * Execute a PATCH request on the given URL. - * - * @param url the HTTP endpoint - * @param body the payload of the PATCH request - * @return the response to the PATCH request - * @throws ConnectException if execution of the PATCH request fails - */ - public Response requestPatch(String url, String body) { - return requestHttpMethod(url, body, Collections.emptyMap(), "PATCH"); - } - - /** - * Execute a DELETE request on the given URL. - * - * @param url the HTTP endpoint - * @return the response to the DELETE request - * @throws ConnectException if execution of the DELETE request fails - * @deprecated Use {@link #requestDelete(String)} instead. - */ - @Deprecated - public int executeDelete(String url) { - return requestDelete(url).getStatus(); - } - - /** - * Execute a DELETE request on the given URL. - * - * @param url the HTTP endpoint - * @return the response to the DELETE request - * @throws ConnectException if execution of the DELETE request fails - */ - public Response requestDelete(String url) { - return requestHttpMethod(url, null, Collections.emptyMap(), "DELETE"); - } - - /** - * A general method that executes an HTTP request on a given URL. - * - * @param url the HTTP endpoint - * @param body the payload of the request; null if there isn't one - * @param headers a map that stores the request headers; empty if there are no headers - * @param httpMethod the name of the HTTP method to execute - * @return the response to the HTTP request - * @throws ConnectException if execution of the HTTP method fails - */ - protected Response requestHttpMethod(String url, String body, Map headers, - String httpMethod) { - log.debug("Executing {} request to URL={}." + (body != null ? " Payload={}" : ""), - httpMethod, url, body); - - try { - Request req = httpClient.newRequest(url); - req.method(httpMethod); - if (body != null) { - headers.forEach(req::header); - req.content(new StringContentProvider(body), "application/json"); - } - - ContentResponse res = req.send(); - log.info("{} response for URL={} is {}", - httpMethod, url, res.getContentAsString().isEmpty() ? "empty" : res.getContentAsString()); - return Response.status(Response.Status.fromStatusCode(res.getStatus())) - .entity(res.getContentAsString()) - .build(); - } catch (Exception e) { - log.error("Could not execute " + httpMethod + " request to " + url, e); - throw new ConnectException(e); - } - } - - private String responseToString(Response response) { - return response == null ? "empty" : (String) response.getEntity(); - } - - public static class Builder { + public static class Builder extends EmbeddedConnectBuilder { private String name = UUID.randomUUID().toString(); - private Map workerProps = new HashMap<>(); private int numWorkers = DEFAULT_NUM_WORKERS; - private int numBrokers = DEFAULT_NUM_BROKERS; - private Properties brokerProps = DEFAULT_BROKER_CONFIG; - private boolean maskExitProcedures = true; - - private Map clientConfigs = new HashMap<>(); public Builder name(String name) { this.name = name; return this; } - public Builder workerProps(Map workerProps) { - this.workerProps = workerProps; - return this; - } - public Builder numWorkers(int numWorkers) { this.numWorkers = numWorkers; return this; } - public Builder numBrokers(int numBrokers) { - this.numBrokers = numBrokers; - return this; - } - - public Builder brokerProps(Properties brokerProps) { - this.brokerProps = brokerProps; - return this; - } - - public Builder clientConfigs(Map clientConfigs) { - this.clientConfigs.putAll(clientConfigs); - return this; - } - /** - * In the event of ungraceful shutdown, embedded clusters call exit or halt with non-zero - * exit statuses. Exiting with a non-zero status forces a test to fail and is hard to - * handle. Because graceful exit is usually not required during a test and because - * depending on such an exit increases flakiness, this setting allows masking - * exit and halt procedures by using a runtime exception instead. Customization of the - * exit and halt procedures is possible through {@code exitProcedure} and {@code - * haltProcedure} respectively. - * - * @param mask if false, exit and halt procedures remain unchanged; true is the default. - * @return the builder for this cluster + * @deprecated Use {@link #clientProps(Map)} instead. */ - public Builder maskExitProcedures(boolean mask) { - this.maskExitProcedures = mask; - return this; - } - - public EmbeddedConnectCluster build() { - return new EmbeddedConnectCluster(name, workerProps, numWorkers, numBrokers, - brokerProps, maskExitProcedures, clientConfigs); + @Deprecated + public Builder clientConfigs(Map clientProps) { + return clientProps(clientProps); + } + + @Override + protected EmbeddedConnectCluster build( + int numBrokers, + Properties brokerProps, + boolean maskExitProcedures, + Map clientProps, + Map workerProps + ) { + return new EmbeddedConnectCluster( + numBrokers, + brokerProps, + maskExitProcedures, + clientProps, + workerProps, + name, + numWorkers + ); } } - /** - * Return the available assertions for this Connect cluster - * - * @return the assertions object - */ - public EmbeddedConnectClusterAssertions assertions() { - return assertions; - } - } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java index d8e488f4727..e6f93f50009 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java @@ -16,603 +16,14 @@ */ package org.apache.kafka.connect.util.clusters; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.connect.runtime.AbstractStatus; -import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo; -import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; -import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.ws.rs.core.Response; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiFunction; -import java.util.stream.Collectors; - -import static org.apache.kafka.test.TestUtils.waitForCondition; - /** - * A set of common assertions that can be applied to a Connect cluster during integration testing + * @deprecated Use {@link ConnectAssertions} instead. */ -public class EmbeddedConnectClusterAssertions { - - private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectClusterAssertions.class); - public static final long WORKER_SETUP_DURATION_MS = TimeUnit.MINUTES.toMillis(5); - public static final long VALIDATION_DURATION_MS = TimeUnit.SECONDS.toMillis(30); - public static final long CONNECTOR_SETUP_DURATION_MS = TimeUnit.MINUTES.toMillis(2); - // Creating a connector requires two rounds of rebalance; destroying one only requires one - // Assume it'll take ~half the time to destroy a connector as it does to create one - public static final long CONNECTOR_SHUTDOWN_DURATION_MS = TimeUnit.MINUTES.toMillis(1); - private static final long CONNECT_INTERNAL_TOPIC_UPDATES_DURATION_MS = TimeUnit.SECONDS.toMillis(60); - - private final EmbeddedConnectCluster connect; - - EmbeddedConnectClusterAssertions(EmbeddedConnectCluster connect) { - this.connect = connect; - } - - /** - * Assert that at least the requested number of workers are up and running. - * - * @param numWorkers the number of online workers - */ - public void assertAtLeastNumWorkersAreUp(int numWorkers, String detailMessage) throws InterruptedException { - try { - waitForCondition( - () -> checkWorkersUp(numWorkers, (actual, expected) -> actual >= expected).orElse(false), - WORKER_SETUP_DURATION_MS, - "Didn't meet the minimum requested number of online workers: " + numWorkers); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Assert that at least the requested number of workers are up and running. - * - * @param numWorkers the number of online workers - */ - public void assertExactlyNumWorkersAreUp(int numWorkers, String detailMessage) throws InterruptedException { - try { - waitForCondition( - () -> checkWorkersUp(numWorkers, (actual, expected) -> actual == expected).orElse(false), - WORKER_SETUP_DURATION_MS, - "Didn't meet the exact requested number of online workers: " + numWorkers); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Confirm that the requested number of workers are up and running. - * - * @param numWorkers the number of online workers - * @return true if at least {@code numWorkers} are up; false otherwise - */ - protected Optional checkWorkersUp(int numWorkers, BiFunction comp) { - try { - int numUp = connect.activeWorkers().size(); - return Optional.of(comp.apply(numUp, numWorkers)); - } catch (Exception e) { - log.error("Could not check active workers.", e); - return Optional.empty(); - } - } - - /** - * Assert that at least the requested number of workers are up and running. - * - * @param numBrokers the number of online brokers - */ - public void assertExactlyNumBrokersAreUp(int numBrokers, String detailMessage) throws InterruptedException { - try { - waitForCondition( - () -> checkBrokersUp(numBrokers, (actual, expected) -> actual == expected).orElse(false), - WORKER_SETUP_DURATION_MS, - "Didn't meet the exact requested number of online brokers: " + numBrokers); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Confirm that the requested number of brokers are up and running. - * - * @param numBrokers the number of online brokers - * @return true if at least {@code numBrokers} are up; false otherwise - */ - protected Optional checkBrokersUp(int numBrokers, BiFunction comp) { - try { - int numRunning = connect.kafka().runningBrokers().size(); - return Optional.of(comp.apply(numRunning, numBrokers)); - } catch (Exception e) { - log.error("Could not check running brokers.", e); - return Optional.empty(); - } - } - - /** - * Assert that the topics with the specified names do not exist. - * - * @param topicNames the names of the topics that are expected to not exist - */ - public void assertTopicsDoNotExist(String... topicNames) throws InterruptedException { - Set topicNameSet = new HashSet<>(Arrays.asList(topicNames)); - AtomicReference> existingTopics = new AtomicReference<>(topicNameSet); - waitForCondition( - () -> checkTopicsExist(topicNameSet, (actual, expected) -> { - existingTopics.set(actual); - return actual.isEmpty(); - }).orElse(false), - CONNECTOR_SETUP_DURATION_MS, - "Unexpectedly found topics " + existingTopics.get()); - } - - /** - * Assert that the topics with the specified names do exist. - * - * @param topicNames the names of the topics that are expected to exist - */ - public void assertTopicsExist(String... topicNames) throws InterruptedException { - Set topicNameSet = new HashSet<>(Arrays.asList(topicNames)); - AtomicReference> missingTopics = new AtomicReference<>(topicNameSet); - waitForCondition( - () -> checkTopicsExist(topicNameSet, (actual, expected) -> { - Set missing = new HashSet<>(expected); - missing.removeAll(actual); - missingTopics.set(missing); - return missing.isEmpty(); - }).orElse(false), - CONNECTOR_SETUP_DURATION_MS, - "Didn't find the topics " + missingTopics.get()); - } +@Deprecated +public class EmbeddedConnectClusterAssertions extends ConnectAssertions { - protected Optional checkTopicsExist(Set topicNames, BiFunction, Set, Boolean> comp) { - try { - Map> topics = connect.kafka().describeTopics(topicNames); - Set actualExistingTopics = topics.entrySet() - .stream() - .filter(e -> e.getValue().isPresent()) - .map(Map.Entry::getKey) - .collect(Collectors.toSet()); - return Optional.of(comp.apply(actualExistingTopics, topicNames)); - } catch (Exception e) { - log.error("Failed to describe the topic(s): {}.", topicNames, e); - return Optional.empty(); - } + EmbeddedConnectClusterAssertions(EmbeddedConnect connect) { + super(connect); } - /** - * Assert that the named topic is configured to have the specified replication factor and - * number of partitions. - * - * @param topicName the name of the topic that is expected to exist - * @param replicas the replication factor - * @param partitions the number of partitions - * @param detailMessage the assertion message - */ - public void assertTopicSettings(String topicName, int replicas, int partitions, String detailMessage) - throws InterruptedException { - try { - waitForCondition( - () -> checkTopicSettings( - topicName, - replicas, - partitions - ).orElse(false), - VALIDATION_DURATION_MS, - "Topic " + topicName + " does not exist or does not have exactly " - + partitions + " partitions or at least " - + replicas + " per partition"); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - protected Optional checkTopicSettings(String topicName, int replicas, int partitions) { - try { - Map> topics = connect.kafka().describeTopics(topicName); - TopicDescription topicDesc = topics.get(topicName).orElse(null); - boolean result = topicDesc != null - && topicDesc.name().equals(topicName) - && topicDesc.partitions().size() == partitions - && topicDesc.partitions().stream().allMatch(p -> p.replicas().size() >= replicas); - return Optional.of(result); - } catch (Exception e) { - log.error("Failed to describe the topic: {}.", topicName, e); - return Optional.empty(); - } - } - - /** - * Assert that the required number of errors are produced by a connector config validation. - * - * @param connectorClass the class of the connector to validate - * @param connConfig the intended configuration - * @param numErrors the number of errors expected - * @param detailMessage the assertion message - */ - public void assertExactlyNumErrorsOnConnectorConfigValidation(String connectorClass, Map connConfig, - int numErrors, String detailMessage) throws InterruptedException { - assertExactlyNumErrorsOnConnectorConfigValidation(connectorClass, connConfig, numErrors, detailMessage, VALIDATION_DURATION_MS); - } - - /** - * Assert that the required number of errors are produced by a connector config validation. - * - * @param connectorClass the class of the connector to validate - * @param connConfig the intended configuration - * @param numErrors the number of errors expected - * @param detailMessage the assertion message - * @param timeout how long to retry for before throwing an exception - * - * @throws AssertionError if the exact number of errors is not produced during config - * validation before the timeout expires - */ - public void assertExactlyNumErrorsOnConnectorConfigValidation(String connectorClass, Map connConfig, - int numErrors, String detailMessage, long timeout) throws InterruptedException { - try { - waitForCondition( - () -> checkValidationErrors( - connectorClass, - connConfig, - numErrors, - (actual, expected) -> actual == expected - ).orElse(false), - timeout, - "Didn't meet the exact requested number of validation errors: " + numErrors); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Confirm that the requested number of errors are produced by {@link EmbeddedConnectCluster#validateConnectorConfig}. - * - * @param connectorClass the class of the connector to validate - * @param connConfig the intended configuration - * @param numErrors the number of errors expected - * @return true if exactly {@code numErrors} are produced by the validation; false otherwise - */ - protected Optional checkValidationErrors(String connectorClass, Map connConfig, - int numErrors, BiFunction comp) { - try { - int numErrorsProduced = connect.validateConnectorConfig(connectorClass, connConfig).errorCount(); - return Optional.of(comp.apply(numErrorsProduced, numErrors)); - } catch (Exception e) { - log.error("Could not check config validation error count.", e); - return Optional.empty(); - } - } - - /** - * Assert that a connector is running with at least the given number of tasks all in running state - * - * @param connectorName the connector name - * @param numTasks the number of tasks - * @param detailMessage - * @throws InterruptedException - */ - public void assertConnectorAndAtLeastNumTasksAreRunning(String connectorName, int numTasks, String detailMessage) - throws InterruptedException { - try { - waitForCondition( - () -> checkConnectorState( - connectorName, - AbstractStatus.State.RUNNING, - numTasks, - AbstractStatus.State.RUNNING, - (actual, expected) -> actual >= expected - ).orElse(false), - CONNECTOR_SETUP_DURATION_MS, - "The connector or at least " + numTasks + " of tasks are not running."); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Assert that a connector is running, that it has a specific number of tasks, and that all of - * its tasks are in the RUNNING state. - * - * @param connectorName the connector name - * @param numTasks the number of tasks - * @param detailMessage the assertion message - * @throws InterruptedException - */ - public void assertConnectorAndExactlyNumTasksAreRunning(String connectorName, int numTasks, String detailMessage) - throws InterruptedException { - try { - waitForCondition( - () -> checkConnectorState( - connectorName, - AbstractStatus.State.RUNNING, - numTasks, - AbstractStatus.State.RUNNING, - (actual, expected) -> actual == expected - ).orElse(false), - CONNECTOR_SETUP_DURATION_MS, - "The connector or exactly " + numTasks + " tasks are not running."); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Assert that a connector is paused, that it has a specific number of tasks, and that all of - * its tasks are in the PAUSED state. - * - * @param connectorName the connector name - * @param numTasks the number of tasks - * @param detailMessage the assertion message - * @throws InterruptedException - */ - public void assertConnectorAndExactlyNumTasksArePaused(String connectorName, int numTasks, String detailMessage) - throws InterruptedException { - try { - waitForCondition( - () -> checkConnectorState( - connectorName, - AbstractStatus.State.PAUSED, - numTasks, - AbstractStatus.State.PAUSED, - Integer::equals - ).orElse(false), - CONNECTOR_SHUTDOWN_DURATION_MS, - "The connector or exactly " + numTasks + " tasks are not paused."); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Assert that a connector is running, that it has a specific number of tasks, and that all of - * its tasks are in the FAILED state. - * - * @param connectorName the connector name - * @param numTasks the number of tasks - * @param detailMessage the assertion message - * @throws InterruptedException - */ - public void assertConnectorIsRunningAndTasksHaveFailed(String connectorName, int numTasks, String detailMessage) - throws InterruptedException { - try { - waitForCondition( - () -> checkConnectorState( - connectorName, - AbstractStatus.State.RUNNING, - numTasks, - AbstractStatus.State.FAILED, - (actual, expected) -> actual >= expected - ).orElse(false), - CONNECTOR_SETUP_DURATION_MS, - "Either the connector is not running or not all the " + numTasks + " tasks have failed."); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Assert that a connector is running, that it has a specific number of tasks out of that numFailedTasks are in the FAILED state. - * - * @param connectorName the connector name - * @param numTasks the number of tasks - * @param numFailedTasks the number of failed tasks - * @param detailMessage the assertion message - * @throws InterruptedException - */ - public void assertConnectorIsRunningAndNumTasksHaveFailed(String connectorName, int numTasks, int numFailedTasks, String detailMessage) - throws InterruptedException { - try { - waitForCondition( - () -> checkConnectorState( - connectorName, - AbstractStatus.State.RUNNING, - numTasks, - numFailedTasks, - AbstractStatus.State.FAILED, - (actual, expected) -> actual >= expected - ).orElse(false), - CONNECTOR_SETUP_DURATION_MS, - "Either the connector is not running or not all the " + numTasks + " tasks have failed."); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Assert that a connector is in FAILED state, that it has a specific number of tasks, and that all of - * its tasks are in the FAILED state. - * - * @param connectorName the connector name - * @param numTasks the number of tasks - * @param detailMessage the assertion message - * @throws InterruptedException - */ - public void assertConnectorIsFailedAndTasksHaveFailed(String connectorName, int numTasks, String detailMessage) - throws InterruptedException { - try { - waitForCondition( - () -> checkConnectorState( - connectorName, - AbstractStatus.State.FAILED, - numTasks, - AbstractStatus.State.FAILED, - (actual, expected) -> actual >= expected - ).orElse(false), - CONNECTOR_SETUP_DURATION_MS, - "Either the connector is running or not all the " + numTasks + " tasks have failed."); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Assert that a connector does not exist. This can be used to verify that a connector has been successfully deleted. - * - * @param connectorName the connector name - * @param detailMessage the assertion message - * @throws InterruptedException - */ - public void assertConnectorDoesNotExist(String connectorName, String detailMessage) - throws InterruptedException { - try { - waitForCondition( - () -> checkConnectorDoesNotExist(connectorName), - CONNECTOR_SETUP_DURATION_MS, - "The connector should not exist."); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Check whether a connector exists by querying the GET /connectors/{connector}/status endpoint - * - * @param connectorName the connector name - * @return true if the connector does not exist; false otherwise - */ - protected boolean checkConnectorDoesNotExist(String connectorName) { - try { - connect.connectorStatus(connectorName); - } catch (ConnectRestException e) { - return e.statusCode() == Response.Status.NOT_FOUND.getStatusCode(); - } catch (Exception e) { - log.error("Could not check connector state info.", e); - return false; - } - return false; - } - - /** - * Assert that a connector is in the stopped state and has no tasks. - * - * @param connectorName the connector name - * @param detailMessage the assertion message - * @throws InterruptedException - */ - public void assertConnectorIsStopped(String connectorName, String detailMessage) - throws InterruptedException { - try { - waitForCondition( - () -> checkConnectorState( - connectorName, - AbstractStatus.State.STOPPED, - 0, - null, - Integer::equals - ).orElse(false), - CONNECTOR_SHUTDOWN_DURATION_MS, - "At least the connector or one of its tasks is still running"); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Check whether the given connector state matches the current state of the connector and - * whether it has at least the given number of tasks, with all the tasks matching the given - * task state. - * @param connectorName the connector - * @param connectorState - * @param numTasks the expected number of tasks - * @param tasksState - * @return true if the connector and tasks are in RUNNING state; false otherwise - */ - protected Optional checkConnectorState( - String connectorName, - AbstractStatus.State connectorState, - int numTasks, - AbstractStatus.State tasksState, - BiFunction comp - ) { - try { - ConnectorStateInfo info = connect.connectorStatus(connectorName); - boolean result = info != null - && comp.apply(info.tasks().size(), numTasks) - && info.connector().state().equals(connectorState.toString()) - && info.tasks().stream().allMatch(s -> s.state().equals(tasksState.toString())); - return Optional.of(result); - } catch (Exception e) { - log.error("Could not check connector state info.", e); - return Optional.empty(); - } - } - - /** - * Check whether the given connector state matches the current state of the connector and - * whether it has at least the given number of tasks, with numTasksInTasksState matching the given - * task state. - * @param connectorName the connector - * @param connectorState - * @param numTasks the expected number of tasks - * @param tasksState - * @return true if the connector and tasks are in RUNNING state; false otherwise - */ - protected Optional checkConnectorState( - String connectorName, - AbstractStatus.State connectorState, - int numTasks, - int numTasksInTasksState, - AbstractStatus.State tasksState, - BiFunction comp - ) { - try { - ConnectorStateInfo info = connect.connectorStatus(connectorName); - boolean result = info != null - && comp.apply(info.tasks().size(), numTasks) - && info.connector().state().equals(connectorState.toString()) - && info.tasks().stream().filter(s -> s.state().equals(tasksState.toString())).count() == numTasksInTasksState; - return Optional.of(result); - } catch (Exception e) { - log.error("Could not check connector state info.", e); - return Optional.empty(); - } - } - /** - * Assert that a connector's set of active topics matches the given collection of topic names. - * - * @param connectorName the connector name - * @param topics a collection of topics to compare against - * @param detailMessage the assertion message - * @throws InterruptedException - */ - public void assertConnectorActiveTopics(String connectorName, Collection topics, String detailMessage) throws InterruptedException { - try { - waitForCondition( - () -> checkConnectorActiveTopics(connectorName, topics).orElse(false), - CONNECT_INTERNAL_TOPIC_UPDATES_DURATION_MS, - "Connector active topics don't match the expected collection"); - } catch (AssertionError e) { - throw new AssertionError(detailMessage, e); - } - } - - /** - * Check whether a connector's set of active topics matches the given collection of topic names. - * - * @param connectorName the connector name - * @param topics a collection of topics to compare against - * @return true if the connector's active topics matches the given collection; false otherwise - */ - protected Optional checkConnectorActiveTopics(String connectorName, Collection topics) { - try { - ActiveTopicsInfo info = connect.connectorTopics(connectorName); - boolean result = info != null - && topics.size() == info.topics().size() - && topics.containsAll(info.topics()); - log.debug("Found connector {} using topics: {}", connectorName, info.topics()); - return Optional.of(result); - } catch (Exception e) { - log.error("Could not check connector {} state info.", connectorName, e); - return Optional.empty(); - } - } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectStandalone.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectStandalone.java new file mode 100644 index 00000000000..43a68d93e8a --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectStandalone.java @@ -0,0 +1,140 @@ +/* + * 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.util.clusters; + +import org.apache.kafka.connect.cli.ConnectStandalone; +import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.test.TestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.PLUGIN_DISCOVERY_CONFIG; +import static org.apache.kafka.connect.runtime.rest.RestServerConfig.LISTENERS_CONFIG; +import static org.apache.kafka.connect.runtime.standalone.StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG; + +/** + * Start a standalone embedded connect worker. Internally, this class will spin up a Kafka and Zk cluster, + * set up any tmp directories. and clean them up on exit. Methods on the same + * {@code EmbeddedConnectStandalone} are not guaranteed to be thread-safe. + */ +public class EmbeddedConnectStandalone extends EmbeddedConnect { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectStandalone.class); + + private static final String REST_HOST_NAME = "localhost"; + + private final Map workerProps; + private final String offsetsFile; + + private WorkerHandle connectWorker; + + private EmbeddedConnectStandalone( + int numBrokers, + Properties brokerProps, + boolean maskExitProcedures, + Map clientProps, + Map workerProps, + String offsetsFile + ) { + super(numBrokers, brokerProps, maskExitProcedures, clientProps); + this.workerProps = workerProps; + this.offsetsFile = offsetsFile; + } + + @Override + public void startConnect() { + log.info("Starting standalone Connect worker"); + + workerProps.put(BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); + // use a random available port + workerProps.put(LISTENERS_CONFIG, "HTTP://" + REST_HOST_NAME + ":0"); + + workerProps.putIfAbsent(OFFSET_STORAGE_FILE_FILENAME_CONFIG, offsetsFile); + workerProps.putIfAbsent(KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + workerProps.putIfAbsent(VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + workerProps.putIfAbsent(PLUGIN_DISCOVERY_CONFIG, "hybrid_fail"); + + Connect connect = new ConnectStandalone().startConnect(workerProps); + connectWorker = new WorkerHandle("standalone", connect); + } + + @Override + public String toString() { + return String.format("EmbeddedConnectStandalone(numBrokers= %d, workerProps= %s)", + numBrokers, + workerProps); + } + + @Override + protected Set workers() { + return connectWorker != null + ? Collections.singleton(connectWorker) + : Collections.emptySet(); + } + + public static class Builder extends EmbeddedConnectBuilder { + + private String offsetsFile = null; + + public Builder offsetsFile(String offsetsFile) { + this.offsetsFile = offsetsFile; + return this; + } + + @Override + protected EmbeddedConnectStandalone build( + int numBrokers, + Properties brokerProps, + boolean maskExitProcedures, + Map clientProps, + Map workerProps + ) { + if (offsetsFile == null) + offsetsFile = tempOffsetsFile(); + + return new EmbeddedConnectStandalone( + numBrokers, + brokerProps, + maskExitProcedures, + clientProps, + workerProps, + offsetsFile + ); + } + + private String tempOffsetsFile() { + try { + return TestUtils + .tempFile("connect-standalone-offsets", null) + .getAbsolutePath(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to create temporary offsets file", e); + } + } + } + +} diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml index 9b7f4b7669c..880806c87b2 100644 --- a/gradle/spotbugs-exclude.xml +++ b/gradle/spotbugs-exclude.xml @@ -291,6 +291,13 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read + + + + + + + diff --git a/tests/kafkatest/services/connect.py b/tests/kafkatest/services/connect.py index da47b90d424..d6d5e0d2791 100644 --- a/tests/kafkatest/services/connect.py +++ b/tests/kafkatest/services/connect.py @@ -241,6 +241,21 @@ class ConnectServiceBase(KafkaPathResolverMixin, Service): def validate_config(self, connector_type, validate_request, node=None): return self._rest('/connector-plugins/' + connector_type + '/config/validate', validate_request, node=node, method="PUT") + def get_logger(self, node, logger): + return self._rest('/admin/loggers/' + logger, node=node) + + def get_all_loggers(self, node): + return self._rest('/admin/loggers', node=node) + + def set_logger(self, node, logger, level, scope=None): + set_request = { + 'level': level + } + path = '/admin/loggers/' + logger + if scope is not None: + path += '?scope=' + scope + return self._rest(path, set_request, node, "PUT") + def _rest(self, path, body=None, node=None, method="GET"): if node is None: node = random.choice(self.nodes) diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py index 591819bd8c9..6b2394fe659 100644 --- a/tests/kafkatest/tests/connect/connect_distributed_test.py +++ b/tests/kafkatest/tests/connect/connect_distributed_test.py @@ -81,7 +81,13 @@ class ConnectDistributedTest(Test): self.value_converter = "org.apache.kafka.connect.json.JsonConverter" self.schemas = True - def setup_services(self, security_protocol=SecurityConfig.PLAINTEXT, timestamp_type=None, broker_version=DEV_BRANCH, auto_create_topics=False, include_filestream_connectors=False): + def setup_services(self, + security_protocol=SecurityConfig.PLAINTEXT, + timestamp_type=None, + broker_version=DEV_BRANCH, + auto_create_topics=False, + include_filestream_connectors=False, + num_workers=3): self.kafka = KafkaService(self.test_context, self.num_brokers, self.zk, security_protocol=security_protocol, interbroker_security_protocol=security_protocol, topics=self.topics, version=broker_version, @@ -94,7 +100,7 @@ class ConnectDistributedTest(Test): for node in self.kafka.nodes: node.config[config_property.MESSAGE_TIMESTAMP_TYPE] = timestamp_type - self.cc = ConnectDistributedService(self.test_context, 3, self.kafka, [self.INPUT_FILE, self.OUTPUT_FILE], + self.cc = ConnectDistributedService(self.test_context, num_workers, self.kafka, [self.INPUT_FILE, self.OUTPUT_FILE], include_filestream_connectors=include_filestream_connectors) self.cc.log_level = "DEBUG" @@ -375,6 +381,184 @@ class ConnectDistributedTest(Test): wait_until(lambda: self.is_paused(self.source, node), timeout_sec=120, err_msg="Failed to see connector startup in PAUSED state") + @cluster(num_nodes=5) + def test_dynamic_logging(self): + """ + Test out the REST API for dynamically adjusting logging levels, on both a single-worker and cluster-wide basis. + """ + + self.setup_services(num_workers=3) + self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) + self.cc.start() + + worker = self.cc.nodes[0] + initial_loggers = self.cc.get_all_loggers(worker) + self.logger.debug("Listed all loggers via REST API: %s", str(initial_loggers)) + assert initial_loggers is not None + assert 'root' in initial_loggers + # We need root and at least one other namespace (the other namespace is checked + # later on to make sure that it hasn't changed) + assert len(initial_loggers) >= 2 + # We haven't made any modifications yet; ensure that the last-modified timestamps + # for all namespaces are null + for logger in initial_loggers.values(): + assert logger['last_modified'] is None + + # Find a non-root namespace to adjust + namespace = None + for logger in initial_loggers.keys(): + if logger != 'root': + namespace = logger + break + assert namespace is not None + + initial_level = self.cc.get_logger(worker, namespace)['level'] + # Make sure we pick a different one than what's already set for that namespace + new_level = self._different_level(initial_level) + request_time = self._set_logger(worker, namespace, new_level) + + # Verify that our adjustment was applied on the worker we issued the request to... + assert self._loggers_are_set(new_level, request_time, namespace, workers=[worker]) + # ... and that no adjustments have been applied to the other workers in the cluster + assert self._loggers_are_set(initial_level, None, namespace, workers=self.cc.nodes[1:]) + + # Force all loggers to get updated by setting the root namespace to + # two different levels + # This guarantees that their last-modified times will be updated + self._set_logger(worker, 'root', 'DEBUG', 'cluster') + new_root = 'INFO' + request_time = self._set_logger(worker, 'root', new_root, 'cluster') + self._wait_for_loggers(new_root, request_time, 'root') + + new_level = 'DEBUG' + request_time = self._set_logger(worker, namespace, new_level, 'cluster') + self._wait_for_loggers(new_level, request_time, namespace) + + prior_all_loggers = [self.cc.get_all_loggers(node) for node in self.cc.nodes] + # Set the same level twice for a namespace + self._set_logger(worker, namespace, new_level, 'cluster') + + prior_namespace = namespace + new_namespace = None + for logger, level in prior_all_loggers[0].items(): + if logger != 'root' and not logger.startswith(namespace): + new_namespace = logger + new_level = self._different_level(level['level']) + assert new_namespace is not None + + request_time = self._set_logger(worker, new_namespace, new_level, 'cluster') + self._wait_for_loggers(new_level, request_time, new_namespace) + + # Verify that the last-modified timestamp and logging level of the prior namespace + # has not changed since the second-most-recent adjustment for it (the most-recent + # adjustment used the same level and should not have had any impact on level or + # timestamp) + new_all_loggers = [self.cc.get_all_loggers(node) for node in self.cc.nodes] + assert len(prior_all_loggers) == len(new_all_loggers) + for i in range(len(prior_all_loggers)): + prior_loggers, new_loggers = prior_all_loggers[i], new_all_loggers[i] + for logger, prior_level in prior_loggers.items(): + if logger.startswith(prior_namespace): + new_level = new_loggers[logger] + assert prior_level == new_level + + # Forcibly update all loggers in the cluster to a new level, bumping their + # last-modified timestamps + new_root = 'INFO' + self._set_logger(worker, 'root', 'DEBUG', 'cluster') + root_request_time = self._set_logger(worker, 'root', new_root, 'cluster') + self._wait_for_loggers(new_root, root_request_time, 'root') + # Track the loggers reported on every node + prior_all_loggers = [self.cc.get_all_loggers(node) for node in self.cc.nodes] + + # Make a final worker-scoped logging adjustment + namespace = new_namespace + new_level = self._different_level(new_root) + request_time = self._set_logger(worker, namespace, new_level, 'worker') + assert self._loggers_are_set(new_level, request_time, namespace, workers=[worker]) + + # Make sure no changes to loggers outside the affected namespace have taken place + all_loggers = self.cc.get_all_loggers(worker) + for logger, level in all_loggers.items(): + if not logger.startswith(namespace): + assert level['level'] == new_root + assert root_request_time <= level['last_modified'] < request_time + + # Verify that the last worker-scoped request we issued had no effect on other + # workers in the cluster + new_all_loggers = [self.cc.get_all_loggers(node) for node in self.cc.nodes] + # Exclude the first node, which we've made worker-scope modifications to + # since we last adjusted the cluster-scope root level + assert prior_all_loggers[1:] == new_all_loggers[1:] + + # Restart a worker and ensure that all logging level adjustments (regardless of scope) + # have been discarded + self._restart_worker(worker) + restarted_loggers = self.cc.get_all_loggers(worker) + assert initial_loggers == restarted_loggers + + def _different_level(self, current_level): + return 'INFO' if current_level is None or current_level.upper() != 'INFO' else 'WARN' + + def _set_logger(self, worker, namespace, new_level, scope=None): + """ + Set a log level via the PUT /admin/loggers/{logger} endpoint, verify that the response + has the expected format, and then return the time at which the request was issued. + :param worker: the worker to issue the REST request to + :param namespace: the logging namespace to adjust + :param new_level: the new level for the namespace + :param scope: the scope of the logging adjustment; if None, then no scope will be specified + in the REST request + :return: the time at or directly before which the REST request was made + """ + request_time = int(time.time() * 1000) + affected_loggers = self.cc.set_logger(worker, namespace, new_level, scope) + if scope is not None and scope.lower() == 'cluster': + assert affected_loggers is None + else: + assert len(affected_loggers) >= 1 + for logger in affected_loggers: + assert logger.startswith(namespace) + return request_time + + def _loggers_are_set(self, expected_level, last_modified, namespace, workers=None): + """ + Verify that all loggers for a namespace (as returned from the GET /admin/loggers endpoint) have + an expected level and last-modified timestamp. + :param expected_level: the expected level for all loggers in the namespace + :param last_modified: the expected last modified timestamp; if None, then all loggers + are expected to have null timestamps; otherwise, all loggers are expected to have timestamps + greater than or equal to this value + :param namespace: the logging namespace to examine + :param workers: the workers to query + :return: whether the expected logging levels and last-modified timestamps are set + """ + if workers is None: + workers = self.cc.nodes + for worker in workers: + all_loggers = self.cc.get_all_loggers(worker) + self.logger.debug("Read loggers on %s from Connect REST API: %s", str(worker), str(all_loggers)) + namespaced_loggers = {k: v for k, v in all_loggers.items() if k.startswith(namespace)} + if len(namespaced_loggers) < 1: + return False + for logger in namespaced_loggers.values(): + if logger['level'] != expected_level: + return False + if last_modified is None: + # Fail fast if there's a non-null timestamp; it'll never be reset to null + assert logger['last_modified'] is None + elif logger['last_modified'] is None or logger['last_modified'] < last_modified: + return False + return True + + def _wait_for_loggers(self, level, request_time, namespace, workers=None): + wait_until( + lambda: self._loggers_are_set(level, request_time, namespace, workers), + # This should be super quick--just a write+read of the config topic, which workers are constantly polling + timeout_sec=10, + err_msg="Log level for namespace '" + namespace + "' was not adjusted in a reasonable amount of time." + ) + @cluster(num_nodes=6) @matrix(security_protocol=[SecurityConfig.PLAINTEXT, SecurityConfig.SASL_SSL], exactly_once_source=[True, False], connect_protocol=['sessioned', 'compatible', 'eager'], metadata_quorum=quorum.all_non_upgrade) def test_file_source_and_sink(self, security_protocol, exactly_once_source, connect_protocol, metadata_quorum): @@ -434,15 +618,7 @@ class ConnectDistributedTest(Test): # Don't want to restart worker nodes in the same order every time shuffled_nodes = self.cc.nodes[start:] + self.cc.nodes[:start] for node in shuffled_nodes: - started = time.time() - self.logger.info("%s bouncing Kafka Connect on %s", clean and "Clean" or "Hard", str(node.account)) - self.cc.stop_node(node, clean_shutdown=clean, await_shutdown=True) - with node.account.monitor_log(self.cc.LOG_FILE) as monitor: - self.cc.start_node(node) - monitor.wait_until("Starting connectors and tasks using config offset", timeout_sec=90, - err_msg="Kafka Connect worker didn't successfully join group and start work") - self.logger.info("Bounced Kafka Connect on %s and rejoined in %f seconds", node.account, time.time() - started) - + self._restart_worker(node, clean=clean) # Give additional time for the consumer groups to recover. Even if it is not a hard bounce, there are # some cases where a restart can cause a rebalance to take the full length of the session timeout # (e.g. if the client shuts down before it has received the memberId from its initial JoinGroup). @@ -560,14 +736,7 @@ class ConnectDistributedTest(Test): # Don't want to restart worker nodes in the same order every time shuffled_nodes = self.cc.nodes[start:] + self.cc.nodes[:start] for node in shuffled_nodes: - started = time.time() - self.logger.info("%s bouncing Kafka Connect on %s", clean and "Clean" or "Hard", str(node.account)) - self.cc.stop_node(node, clean_shutdown=clean, await_shutdown=True) - with node.account.monitor_log(self.cc.LOG_FILE) as monitor: - self.cc.start_node(node) - monitor.wait_until("Starting connectors and tasks using config offset", timeout_sec=90, - err_msg="Kafka Connect worker didn't successfully join group and start work") - self.logger.info("Bounced Kafka Connect on %s and rejoined in %f seconds", node.account, time.time() - started) + self._restart_worker(node, clean=clean) if i < 2: # Give additional time for the worker group to recover. Even if it is not a hard bounce, there are @@ -775,3 +944,13 @@ class ConnectDistributedTest(Test): return list(node.account.ssh_capture("cat " + file)) except RemoteCommandError: return [] + + def _restart_worker(self, node, clean=True): + started = time.time() + self.logger.info("%s bouncing Kafka Connect on %s", clean and "Clean" or "Hard", str(node.account)) + self.cc.stop_node(node, clean_shutdown=clean, await_shutdown=True) + with node.account.monitor_log(self.cc.LOG_FILE) as monitor: + self.cc.start_node(node) + monitor.wait_until("Starting connectors and tasks using config offset", timeout_sec=90, + err_msg="Kafka Connect worker didn't successfully join group and start work") + self.logger.info("Bounced Kafka Connect on %s and rejoined in %f seconds", node.account, time.time() - started)