Browse Source

KAFKA-15428: Cluster-wide dynamic log adjustments for Connect (#14538)

Reviewers: Greg Harris <greg.harris@aiven.io>, Yang Yang <yayang@uber.com>, Yash Mayya <yash.mayya@gmail.com>
pull/11052/merge
Chris Egerton 11 months ago committed by GitHub
parent
commit
091eb9b349
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      checkstyle/import-control.xml
  2. 2
      checkstyle/suppressions.xml
  3. 4
      connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java
  4. 2
      connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java
  5. 32
      connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
  6. 34
      connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java
  7. 215
      connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java
  8. 12
      connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
  9. 2
      connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java
  10. 68
      connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/LoggerLevel.java
  11. 164
      connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResource.java
  12. 21
      connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java
  13. 15
      connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java
  14. 56
      connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java
  15. 5
      connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java
  16. 2
      connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java
  17. 2
      connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorRestartApiIntegrationTest.java
  18. 8
      connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java
  19. 205
      connect/runtime/src/test/java/org/apache/kafka/connect/integration/StandaloneWorkerIntegrationTest.java
  20. 95
      connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java
  21. 246
      connect/runtime/src/test/java/org/apache/kafka/connect/runtime/LoggersTest.java
  22. 27
      connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java
  23. 238
      connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResourceTest.java
  24. 3
      connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java
  25. 65
      connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java
  26. 618
      connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/ConnectAssertions.java
  27. 956
      connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java
  28. 84
      connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectBuilder.java
  29. 918
      connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java
  30. 599
      connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java
  31. 140
      connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectStandalone.java
  32. 7
      gradle/spotbugs-exclude.xml
  33. 15
      tests/kafkatest/services/connect.py
  34. 217
      tests/kafkatest/tests/connect/connect_distributed_test.py

4
checkstyle/import-control.xml

@ -496,6 +496,7 @@ @@ -496,6 +496,7 @@
<allow pkg="org.reflections.util"/>
<allow pkg="javax.crypto"/>
<allow pkg="org.eclipse.jetty.util" />
<allow pkg="org.apache.log4j" />
<subpackage name="rest">
<allow pkg="org.eclipse.jetty" />
@ -505,9 +506,6 @@ @@ -505,9 +506,6 @@
<allow pkg="com.fasterxml.jackson" />
<allow pkg="org.apache.http"/>
<allow pkg="io.swagger.v3.oas.annotations"/>
<subpackage name="resources">
<allow pkg="org.apache.log4j" />
</subpackage>
</subpackage>
<subpackage name="isolation">

2
checkstyle/suppressions.xml

@ -139,6 +139,8 @@ @@ -139,6 +139,8 @@
files="Worker(SinkTask|SourceTask|Coordinator).java"/>
<suppress checks="ParameterNumber"
files="(ConfigKeyInfo|DistributedHerder).java"/>
<suppress checks="DefaultComesLast"
files="LoggingResource.java" />
<suppress checks="ClassDataAbstractionCoupling"
files="(RestServer|AbstractHerder|DistributedHerder|Worker).java"/>

4
connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java

@ -204,7 +204,7 @@ public class MirrorConnectorsIntegrationBaseTest { @@ -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 { @@ -214,7 +214,7 @@ public class MirrorConnectorsIntegrationBaseTest {
.brokerProps(backupBrokerProps)
.workerProps(backupWorkerProps)
.maskExitProcedures(false)
.clientConfigs(additionalBackupClusterClientsConfigs)
.clientProps(additionalBackupClusterClientsConfigs)
.build();
primary.start();

2
connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java

@ -55,7 +55,7 @@ import java.util.Map; @@ -55,7 +55,7 @@ import java.util.Map;
public class ConnectStandalone extends AbstractConnectCli<StandaloneConfig> {
private static final Logger log = LoggerFactory.getLogger(ConnectStandalone.class);
protected ConnectStandalone(String... args) {
public ConnectStandalone(String... args) {
super(args);
}

32
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java

@ -26,6 +26,7 @@ import org.apache.kafka.common.config.ConfigDef.ConfigKey; @@ -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; @@ -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; @@ -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; @@ -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 @@ -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<String, Connector> tempConnectors = new ConcurrentHashMap<>();
@ -128,7 +133,8 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con @@ -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 @@ -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 @@ -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<String, ?>, Map<String, ?>> offsets, Callback<Message> cb);
@Override
public LoggerLevel loggerLevel(String logger) {
return loggers.level(logger);
}
@Override
public Map<String, LoggerLevel> allLoggerLevels() {
return loggers.allLevels();
}
@Override
public List<String> 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);
}
}

34
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; @@ -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 { @@ -318,6 +319,39 @@ public interface Herder {
*/
void resetConnectorOffsets(String connName, Callback<Message> 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<String, LoggerLevel> 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<String> 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

215
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java

@ -0,0 +1,215 @@ @@ -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.
* <p>
* 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<String, Long> 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<org.apache.log4j.Logger> 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<String, LoggerLevel> allLevels() {
Map<String, LoggerLevel> result = new TreeMap<>();
Enumeration<org.apache.log4j.Logger> 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<String> 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<org.apache.log4j.Logger> childLoggers = loggers(namespace);
List<String> 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<org.apache.log4j.Logger> loggers(String namespace) {
Objects.requireNonNull(namespace, "Logging namespace may not be null");
if (ROOT_LOGGER_NAME.equalsIgnoreCase(namespace)) {
List<org.apache.log4j.Logger> result = Collections.list(currentLoggers());
result.add(rootLogger());
return result;
}
List<org.apache.log4j.Logger> result = new ArrayList<>();
org.apache.log4j.Logger ancestorLogger = lookupLogger(namespace);
Enumeration<org.apache.log4j.Logger> 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<org.apache.log4j.Logger> 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);
}
}

12
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java

@ -279,7 +279,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable { @@ -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 { @@ -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 { @@ -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<DistributedHerderRequest> {

2
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestServer.java

@ -57,7 +57,7 @@ public class ConnectRestServer extends RestServer { @@ -57,7 +57,7 @@ public class ConnectRestServer extends RestServer {
@Override
protected Collection<ConnectResource> adminResources() {
return Arrays.asList(
new LoggingResource()
new LoggingResource(herder)
);
}

68
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/LoggerLevel.java

@ -0,0 +1,68 @@ @@ -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
+ '}';
}
}

164
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResource.java

@ -17,28 +17,28 @@ @@ -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; @@ -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 { @@ -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<String, Map<String, String>> loggers = new TreeMap<>();
Enumeration<Logger> 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 { @@ -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<Logger> 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 { @@ -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<String, String> 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<String, String> 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<Logger> childLoggers;
if (ROOT_LOGGER_NAME.equalsIgnoreCase(namedLogger)) {
childLoggers = Collections.list(currentLoggers());
childLoggers.add(rootLogger());
} else {
childLoggers = new ArrayList<>();
Logger ancestorLogger = lookupLogger(namedLogger);
Enumeration<Logger> 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<String> 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<Logger> 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<String, String> 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<String> 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<String, String> levelToMap(Logger logger) {
return Collections.singletonMap("level", String.valueOf(logger.getLevel()));
}
}

21
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java

@ -17,6 +17,7 @@ @@ -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 { @@ -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 { @@ -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 { @@ -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<TargetState> onStart) {
Map<String, String> connConfigs = configState.connectorConfig(connName);
TargetState targetState = configState.targetState(connName);
@ -544,6 +554,11 @@ public class StandaloneHerder extends AbstractHerder { @@ -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 {

15
connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java

@ -122,6 +122,14 @@ public interface ConfigBackingStore { @@ -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 { @@ -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);
}
}

56
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java

@ -271,6 +271,14 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme @@ -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 @@ -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<String, byte[]> setupAndCreateKafkaBasedLog(String topic, final WorkerConfig config) {
String clusterId = config.kafkaClusterId();
@ -901,6 +923,9 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme @@ -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 @@ -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<String, Object> valueAsMap = (Map<String, Object>) 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;

5
connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java

@ -169,6 +169,11 @@ public class MemoryConfigBackingStore implements ConfigBackingStore { @@ -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;

2
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 @@ -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;

2
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 @@ -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;

8
connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExactlyOnceSourceIntegrationTest.java

@ -48,7 +48,7 @@ import org.apache.kafka.connect.source.SourceRecord; @@ -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 { @@ -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 { @@ -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 { @@ -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
)
);

205
connect/runtime/src/test/java/org/apache/kafka/connect/integration/StandaloneWorkerIntegrationTest.java

@ -0,0 +1,205 @@ @@ -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<String, LoggerLevel> initialLevels = connect.allLogLevels();
assertFalse("Connect REST API did not list any known loggers", initialLevels.isEmpty());
Map<String, LoggerLevel> 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<String, LoggerLevel> 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<String, LoggerLevel> testSetLoggingLevel(
String namespace,
String level,
String scope,
Map<String, LoggerLevel> initialLevels
) {
long requestTime = System.currentTimeMillis();
List<String> 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<String> 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<String, LoggerLevel> newLevels = connect.allLogLevels();
Map<String, LoggerLevel> 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<String> 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<String, LoggerLevel> 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<String, ?> entry, String namespace) {
return entry.getKey().startsWith(namespace);
}
private static boolean isModified(Map.Entry<?, LoggerLevel> entry) {
return lastModified(entry) != null;
}
private static Long lastModified(Map.Entry<?, LoggerLevel> entry) {
return entry.getValue().lastModified();
}
private static String level(Map.Entry<?, LoggerLevel> entry) {
return entry.getValue().level();
}
}

95
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java

@ -24,6 +24,7 @@ import org.apache.kafka.common.config.ConfigValue; @@ -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 { @@ -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 { @@ -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 { @@ -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 { @@ -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 { @@ -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 { @@ -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 { @@ -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 { @@ -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 { @@ -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<TaskStatus> taskStatusArgumentCaptor = ArgumentCaptor.forClass(TaskStatus.class);
doNothing().when(statusStore).putSafe(taskStatusArgumentCaptor.capture());
@ -358,9 +341,7 @@ public class AbstractHerderTest { @@ -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 { @@ -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 { @@ -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 { @@ -1051,9 +1028,7 @@ public class AbstractHerderTest {
Function<T, ConfigDef> pluginConfig,
Optional<ConfigDef> 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 { @@ -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 { @@ -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 { @@ -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 { @@ -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<ConnectorOffsets> 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 { @@ -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<ConnectorOffsets> cb = new FutureCallback<>();
@ -1210,9 +1171,7 @@ public class AbstractHerderTest { @@ -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 { @@ -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);

246
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/LoggersTest.java

@ -0,0 +1,246 @@ @@ -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<String, LoggerLevel> expectedLevels = Collections.singletonMap(
"b",
new LoggerLevel(Level.INFO.toString(), null)
);
Map<String, LoggerLevel> 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<String> 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<String> 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<String, LoggerLevel> 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<String, LoggerLevel> 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<String, Logger> 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<Logger> currentLoggers() {
return new Vector<>(currentLoggers.values()).elements();
}
@Override
Logger rootLogger() {
return rootLogger;
}
}
private Logger logger(String name) {
return new Logger(name) { };
}
}

27
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/ConnectRestServerTest.java

@ -34,6 +34,7 @@ import org.apache.kafka.common.utils.LogCaptureAppender; @@ -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; @@ -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 { @@ -247,15 +247,18 @@ public class ConnectRestServerTest {
}
@Test
public void testLoggersEndpointWithDefaults() throws IOException {
public void testLoggerEndpointWithDefaults() throws IOException {
Map<String, String> 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 { @@ -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<String, Map<String, ?>> loggers = mapper.readValue(responseStr, new TypeReference<Map<String, Map<String, ?>>>() {
});
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<String, Object> expectedLogger = new HashMap<>();
expectedLogger.put("level", loggingLevel);
expectedLogger.put("last_modified", lastModified);
Map<String, Map<String, Object>> expectedLoggers = Collections.singletonMap(logger, expectedLogger);
Map<String, Map<String, Object>> actualLoggers = mapper.readValue(responseStr, new TypeReference<Map<String, Map<String, Object>>>() { });
assertEquals(expectedLoggers, actualLoggers);
}
@Test

238
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/LoggingResourceTest.java

@ -16,176 +16,150 @@ @@ -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<String, Map<String, String>> loggers = (Map<String, Map<String, String>>) 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<String, String> level = (Map<String, String>) 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<String> modified = (List<String>) 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<String> modified = (List<String>) 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<String> expectedLoggers = Arrays.asList(
"org.apache.kafka.connect",
"org.apache.kafka.connect.runtime.distributed.DistributedHerder"
);
when(herder.setWorkerLoggerLevel(logger, level)).thenReturn(expectedLoggers);
List<String> 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<String>) 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<Logger> 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);
}
}
}

3
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java

@ -19,6 +19,7 @@ package org.apache.kafka.connect.runtime.standalone; @@ -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 { @@ -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);

65
connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java

@ -1461,6 +1461,71 @@ public class KafkaConfigBackingStoreTest { @@ -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<ConsumerRecord<String, byte[]>> 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<byte[], Struct> 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<String, byte[]> 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();

618
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/ConnectAssertions.java

@ -0,0 +1,618 @@ @@ -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<Boolean> checkWorkersUp(int numWorkers, BiFunction<Integer, Integer, Boolean> 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<Boolean> checkBrokersUp(int numBrokers, BiFunction<Integer, Integer, Boolean> 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<String> topicNameSet = new HashSet<>(Arrays.asList(topicNames));
AtomicReference<Set<String>> 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<String> topicNameSet = new HashSet<>(Arrays.asList(topicNames));
AtomicReference<Set<String>> missingTopics = new AtomicReference<>(topicNameSet);
waitForCondition(
() -> checkTopicsExist(topicNameSet, (actual, expected) -> {
Set<String> 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<Boolean> checkTopicsExist(Set<String> topicNames, BiFunction<Set<String>, Set<String>, Boolean> comp) {
try {
Map<String, Optional<TopicDescription>> topics = connect.kafka().describeTopics(topicNames);
Set<String> 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<Boolean> checkTopicSettings(String topicName, int replicas, int partitions) {
try {
Map<String, Optional<TopicDescription>> 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<String, String> 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<String, String> 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<Boolean> checkValidationErrors(String connectorClass, Map<String, String> connConfig,
int numErrors, BiFunction<Integer, Integer, Boolean> 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 <strong><em>GET /connectors/{connector}/status</em></strong> 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<Boolean> checkConnectorState(
String connectorName,
AbstractStatus.State connectorState,
int numTasks,
AbstractStatus.State tasksState,
BiFunction<Integer, Integer, Boolean> 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<Boolean> checkConnectorState(
String connectorName,
AbstractStatus.State connectorState,
int numTasks,
int numTasksInTasksState,
AbstractStatus.State tasksState,
BiFunction<Integer, Integer, Boolean> 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<String> 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<Boolean> checkConnectorActiveTopics(String connectorName, Collection<String> 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();
}
}
}

956
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnect.java

@ -0,0 +1,956 @@ @@ -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<String, String> 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<WorkerHandle> 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<String, String> 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<String, String> 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<String, String> 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<String> 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<String, Map<String, List<String>>> activeTopics = mapper
.readerFor(new TypeReference<Map<String, Map<String, List<String>>>>() { })
.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 <code>GET /connectors/{connector}</code> 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<TaskInfo> 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<List<TaskInfo>>() { });
}
} 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 <strong><em>GET /connectors/{connector}/offsets</em></strong> 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 <strong><em>PATCH /connectors/{connector}/offsets</em></strong>
* 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<String, ?> partition, Map<String, ?> offset) {
return alterConnectorOffsets(
connectorName,
new ConnectorOffsets(Collections.singletonList(new ConnectorOffset(partition, offset)))
);
}
/**
* Alter the offset for a sink connector's topic partition via the <strong><em>PATCH /connectors/{connector}/offsets</em></strong>
* 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 <strong><em>PATCH /connectors/{connector}/offsets</em></strong> 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 <strong><em>DELETE /connectors/{connector}/offsets</em></strong> 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<String, LoggerLevel> 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<Map<String, LoggerLevel>>() { })
.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<String> 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<List<String>>() { })
.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<String> 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<String, String> 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<String, String> 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<String, String> 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<WorkerHandle> 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;
}
}

84
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectBuilder.java

@ -0,0 +1,84 @@ @@ -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<C extends EmbeddedConnect, B extends EmbeddedConnectBuilder<C, B>> {
private Map<String, String> workerProps = new HashMap<>();
private int numBrokers = EmbeddedConnect.DEFAULT_NUM_BROKERS;
private Properties brokerProps = new Properties();
private boolean maskExitProcedures = true;
private final Map<String, String> clientProps = new HashMap<>();
protected abstract C build(
int numBrokers,
Properties brokerProps,
boolean maskExitProcedures,
Map<String, String> clientProps,
Map<String, String> workerProps
);
public B workerProps(Map<String, String> 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<String, String> 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;
}
}

918
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java

File diff suppressed because it is too large Load Diff

599
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java

@ -16,603 +16,14 @@ @@ -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<Boolean> checkWorkersUp(int numWorkers, BiFunction<Integer, Integer, Boolean> 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<Boolean> checkBrokersUp(int numBrokers, BiFunction<Integer, Integer, Boolean> 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<String> topicNameSet = new HashSet<>(Arrays.asList(topicNames));
AtomicReference<Set<String>> 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<String> topicNameSet = new HashSet<>(Arrays.asList(topicNames));
AtomicReference<Set<String>> missingTopics = new AtomicReference<>(topicNameSet);
waitForCondition(
() -> checkTopicsExist(topicNameSet, (actual, expected) -> {
Set<String> 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<Boolean> checkTopicsExist(Set<String> topicNames, BiFunction<Set<String>, Set<String>, Boolean> comp) {
try {
Map<String, Optional<TopicDescription>> topics = connect.kafka().describeTopics(topicNames);
Set<String> 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<Boolean> checkTopicSettings(String topicName, int replicas, int partitions) {
try {
Map<String, Optional<TopicDescription>> 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<String, String> 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<String, String> 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<Boolean> checkValidationErrors(String connectorClass, Map<String, String> connConfig,
int numErrors, BiFunction<Integer, Integer, Boolean> 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 <strong><em>GET /connectors/{connector}/status</em></strong> 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<Boolean> checkConnectorState(
String connectorName,
AbstractStatus.State connectorState,
int numTasks,
AbstractStatus.State tasksState,
BiFunction<Integer, Integer, Boolean> 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<Boolean> checkConnectorState(
String connectorName,
AbstractStatus.State connectorState,
int numTasks,
int numTasksInTasksState,
AbstractStatus.State tasksState,
BiFunction<Integer, Integer, Boolean> 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<String> 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<Boolean> checkConnectorActiveTopics(String connectorName, Collection<String> 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();
}
}
}

140
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectStandalone.java

@ -0,0 +1,140 @@ @@ -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<String, String> workerProps;
private final String offsetsFile;
private WorkerHandle connectWorker;
private EmbeddedConnectStandalone(
int numBrokers,
Properties brokerProps,
boolean maskExitProcedures,
Map<String, String> clientProps,
Map<String, String> 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<WorkerHandle> workers() {
return connectWorker != null
? Collections.singleton(connectWorker)
: Collections.emptySet();
}
public static class Builder extends EmbeddedConnectBuilder<EmbeddedConnectStandalone, Builder> {
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<String, String> clientProps,
Map<String, String> 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);
}
}
}
}

7
gradle/spotbugs-exclude.xml

@ -291,6 +291,13 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read @@ -291,6 +291,13 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read
<Bug pattern="EQ_CHECK_FOR_OPERAND_NOT_COMPATIBLE_WITH_THIS"/>
</Match>
<Match>
<!-- Suppress aggressive and unnecessary warnings about switch fallthrough -->
<Class name="org.apache.kafka.connect.runtime.rest.resources.LoggingResource" />
<Method name="setLevel" />
<Bug pattern="SF_SWITCH_FALLTHROUGH" />
</Match>
<Match>
<!-- Suppress some minor warnings about machine-generated code for benchmarking. -->
<Package name="~org\.apache\.kafka\.jmh\..*\.jmh_generated"/>

15
tests/kafkatest/services/connect.py

@ -241,6 +241,21 @@ class ConnectServiceBase(KafkaPathResolverMixin, Service): @@ -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)

217
tests/kafkatest/tests/connect/connect_distributed_test.py

@ -81,7 +81,13 @@ class ConnectDistributedTest(Test): @@ -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): @@ -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): @@ -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): @@ -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): @@ -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): @@ -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)

Loading…
Cancel
Save