mirror of https://github.com/apache/kafka.git
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>
This commit is contained in:
parent
c81a725219
commit
091eb9b349
|
@ -496,6 +496,7 @@
|
||||||
<allow pkg="org.reflections.util"/>
|
<allow pkg="org.reflections.util"/>
|
||||||
<allow pkg="javax.crypto"/>
|
<allow pkg="javax.crypto"/>
|
||||||
<allow pkg="org.eclipse.jetty.util" />
|
<allow pkg="org.eclipse.jetty.util" />
|
||||||
|
<allow pkg="org.apache.log4j" />
|
||||||
|
|
||||||
<subpackage name="rest">
|
<subpackage name="rest">
|
||||||
<allow pkg="org.eclipse.jetty" />
|
<allow pkg="org.eclipse.jetty" />
|
||||||
|
@ -505,9 +506,6 @@
|
||||||
<allow pkg="com.fasterxml.jackson" />
|
<allow pkg="com.fasterxml.jackson" />
|
||||||
<allow pkg="org.apache.http"/>
|
<allow pkg="org.apache.http"/>
|
||||||
<allow pkg="io.swagger.v3.oas.annotations"/>
|
<allow pkg="io.swagger.v3.oas.annotations"/>
|
||||||
<subpackage name="resources">
|
|
||||||
<allow pkg="org.apache.log4j" />
|
|
||||||
</subpackage>
|
|
||||||
</subpackage>
|
</subpackage>
|
||||||
|
|
||||||
<subpackage name="isolation">
|
<subpackage name="isolation">
|
||||||
|
|
|
@ -139,6 +139,8 @@
|
||||||
files="Worker(SinkTask|SourceTask|Coordinator).java"/>
|
files="Worker(SinkTask|SourceTask|Coordinator).java"/>
|
||||||
<suppress checks="ParameterNumber"
|
<suppress checks="ParameterNumber"
|
||||||
files="(ConfigKeyInfo|DistributedHerder).java"/>
|
files="(ConfigKeyInfo|DistributedHerder).java"/>
|
||||||
|
<suppress checks="DefaultComesLast"
|
||||||
|
files="LoggingResource.java" />
|
||||||
|
|
||||||
<suppress checks="ClassDataAbstractionCoupling"
|
<suppress checks="ClassDataAbstractionCoupling"
|
||||||
files="(RestServer|AbstractHerder|DistributedHerder|Worker).java"/>
|
files="(RestServer|AbstractHerder|DistributedHerder|Worker).java"/>
|
||||||
|
|
|
@ -204,7 +204,7 @@ public class MirrorConnectorsIntegrationBaseTest {
|
||||||
.brokerProps(primaryBrokerProps)
|
.brokerProps(primaryBrokerProps)
|
||||||
.workerProps(primaryWorkerProps)
|
.workerProps(primaryWorkerProps)
|
||||||
.maskExitProcedures(false)
|
.maskExitProcedures(false)
|
||||||
.clientConfigs(additionalPrimaryClusterClientsConfigs)
|
.clientProps(additionalPrimaryClusterClientsConfigs)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
backup = new EmbeddedConnectCluster.Builder()
|
backup = new EmbeddedConnectCluster.Builder()
|
||||||
|
@ -214,7 +214,7 @@ public class MirrorConnectorsIntegrationBaseTest {
|
||||||
.brokerProps(backupBrokerProps)
|
.brokerProps(backupBrokerProps)
|
||||||
.workerProps(backupWorkerProps)
|
.workerProps(backupWorkerProps)
|
||||||
.maskExitProcedures(false)
|
.maskExitProcedures(false)
|
||||||
.clientConfigs(additionalBackupClusterClientsConfigs)
|
.clientProps(additionalBackupClusterClientsConfigs)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
primary.start();
|
primary.start();
|
||||||
|
|
|
@ -55,7 +55,7 @@ import java.util.Map;
|
||||||
public class ConnectStandalone extends AbstractConnectCli<StandaloneConfig> {
|
public class ConnectStandalone extends AbstractConnectCli<StandaloneConfig> {
|
||||||
private static final Logger log = LoggerFactory.getLogger(ConnectStandalone.class);
|
private static final Logger log = LoggerFactory.getLogger(ConnectStandalone.class);
|
||||||
|
|
||||||
protected ConnectStandalone(String... args) {
|
public ConnectStandalone(String... args) {
|
||||||
super(args);
|
super(args);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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.ConfigDef.Type;
|
||||||
import org.apache.kafka.common.config.ConfigTransformer;
|
import org.apache.kafka.common.config.ConfigTransformer;
|
||||||
import org.apache.kafka.common.config.ConfigValue;
|
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.common.utils.Utils;
|
||||||
import org.apache.kafka.connect.connector.Connector;
|
import org.apache.kafka.connect.connector.Connector;
|
||||||
import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy;
|
import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy;
|
||||||
|
@ -43,6 +44,7 @@ import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo;
|
||||||
import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
|
import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
|
||||||
import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
|
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.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.entities.Message;
|
||||||
import org.apache.kafka.connect.runtime.rest.errors.BadRequestException;
|
import org.apache.kafka.connect.runtime.rest.errors.BadRequestException;
|
||||||
import org.apache.kafka.connect.sink.SinkConnector;
|
import org.apache.kafka.connect.sink.SinkConnector;
|
||||||
|
@ -56,6 +58,7 @@ import org.apache.kafka.connect.transforms.Transformation;
|
||||||
import org.apache.kafka.connect.transforms.predicates.Predicate;
|
import org.apache.kafka.connect.transforms.predicates.Predicate;
|
||||||
import org.apache.kafka.connect.util.Callback;
|
import org.apache.kafka.connect.util.Callback;
|
||||||
import org.apache.kafka.connect.util.ConnectorTaskId;
|
import org.apache.kafka.connect.util.ConnectorTaskId;
|
||||||
|
import org.apache.log4j.Level;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
@ -72,6 +75,7 @@ import java.util.LinkedHashMap;
|
||||||
import java.util.LinkedHashSet;
|
import java.util.LinkedHashSet;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Locale;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -120,6 +124,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
|
||||||
private final ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy;
|
private final ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy;
|
||||||
protected volatile boolean running = false;
|
protected volatile boolean running = false;
|
||||||
private final ExecutorService connectorExecutor;
|
private final ExecutorService connectorExecutor;
|
||||||
|
protected final Loggers loggers;
|
||||||
|
|
||||||
private final ConcurrentMap<String, Connector> tempConnectors = new ConcurrentHashMap<>();
|
private final ConcurrentMap<String, Connector> tempConnectors = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
|
@ -128,7 +133,8 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
|
||||||
String kafkaClusterId,
|
String kafkaClusterId,
|
||||||
StatusBackingStore statusBackingStore,
|
StatusBackingStore statusBackingStore,
|
||||||
ConfigBackingStore configBackingStore,
|
ConfigBackingStore configBackingStore,
|
||||||
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) {
|
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
|
||||||
|
Time time) {
|
||||||
this.worker = worker;
|
this.worker = worker;
|
||||||
this.worker.herder = this;
|
this.worker.herder = this;
|
||||||
this.workerId = workerId;
|
this.workerId = workerId;
|
||||||
|
@ -137,6 +143,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
|
||||||
this.configBackingStore = configBackingStore;
|
this.configBackingStore = configBackingStore;
|
||||||
this.connectorClientConfigOverridePolicy = connectorClientConfigOverridePolicy;
|
this.connectorClientConfigOverridePolicy = connectorClientConfigOverridePolicy;
|
||||||
this.connectorExecutor = Executors.newCachedThreadPool();
|
this.connectorExecutor = Executors.newCachedThreadPool();
|
||||||
|
this.loggers = new Loggers(time);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -917,4 +924,27 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
|
||||||
* @param cb callback to invoke upon completion
|
* @param cb callback to invoke upon completion
|
||||||
*/
|
*/
|
||||||
protected abstract void modifyConnectorOffsets(String connName, Map<Map<String, ?>, Map<String, ?>> offsets, Callback<Message> cb);
|
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);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.ConnectorInfo;
|
||||||
import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
|
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.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.Message;
|
||||||
import org.apache.kafka.connect.runtime.rest.entities.TaskInfo;
|
import org.apache.kafka.connect.runtime.rest.entities.TaskInfo;
|
||||||
import org.apache.kafka.connect.storage.StatusBackingStore;
|
import org.apache.kafka.connect.storage.StatusBackingStore;
|
||||||
|
@ -318,6 +319,39 @@ public interface Herder {
|
||||||
*/
|
*/
|
||||||
void resetConnectorOffsets(String connName, Callback<Message> cb);
|
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 {
|
enum ConfigReloadAction {
|
||||||
NONE,
|
NONE,
|
||||||
RESTART
|
RESTART
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -279,7 +279,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
ExecutorService forwardRequestExecutor,
|
ExecutorService forwardRequestExecutor,
|
||||||
// https://github.com/mockito/mockito/issues/2601 explains why we can't use varargs here
|
// https://github.com/mockito/mockito/issues/2601 explains why we can't use varargs here
|
||||||
AutoCloseable[] uponShutdown) {
|
AutoCloseable[] uponShutdown) {
|
||||||
super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy);
|
super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy, time);
|
||||||
|
|
||||||
this.time = time;
|
this.time = time;
|
||||||
this.herderMetrics = new HerderMetrics(metrics);
|
this.herderMetrics = new HerderMetrics(metrics);
|
||||||
|
@ -1612,6 +1612,11 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
return true;
|
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
|
// Should only be called from work thread, so synchronization should not be needed
|
||||||
private boolean isLeader() {
|
private boolean isLeader() {
|
||||||
return assignment != null && member.memberId().equals(assignment.leader());
|
return assignment != null && member.memberId().equals(assignment.leader());
|
||||||
|
@ -2376,6 +2381,11 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
}
|
}
|
||||||
member.wakeup();
|
member.wakeup();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onLoggingLevelUpdate(String namespace, String level) {
|
||||||
|
setWorkerLoggerLevel(namespace, level);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class DistributedHerderRequest implements HerderRequest, Comparable<DistributedHerderRequest> {
|
class DistributedHerderRequest implements HerderRequest, Comparable<DistributedHerderRequest> {
|
||||||
|
|
|
@ -57,7 +57,7 @@ public class ConnectRestServer extends RestServer {
|
||||||
@Override
|
@Override
|
||||||
protected Collection<ConnectResource> adminResources() {
|
protected Collection<ConnectResource> adminResources() {
|
||||||
return Arrays.asList(
|
return Arrays.asList(
|
||||||
new LoggingResource()
|
new LoggingResource(herder)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
+ '}';
|
||||||
|
}
|
||||||
|
}
|
|
@ -17,28 +17,28 @@
|
||||||
package org.apache.kafka.connect.runtime.rest.resources;
|
package org.apache.kafka.connect.runtime.rest.resources;
|
||||||
|
|
||||||
import io.swagger.v3.oas.annotations.Operation;
|
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.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.kafka.connect.runtime.rest.errors.BadRequestException;
|
||||||
import org.apache.log4j.Level;
|
import org.apache.log4j.Level;
|
||||||
import org.apache.log4j.LogManager;
|
import org.slf4j.LoggerFactory;
|
||||||
import org.apache.log4j.Logger;
|
|
||||||
|
|
||||||
import javax.ws.rs.Consumes;
|
import javax.ws.rs.Consumes;
|
||||||
|
import javax.ws.rs.DefaultValue;
|
||||||
import javax.ws.rs.GET;
|
import javax.ws.rs.GET;
|
||||||
import javax.ws.rs.PUT;
|
import javax.ws.rs.PUT;
|
||||||
import javax.ws.rs.Path;
|
import javax.ws.rs.Path;
|
||||||
import javax.ws.rs.PathParam;
|
import javax.ws.rs.PathParam;
|
||||||
import javax.ws.rs.Produces;
|
import javax.ws.rs.Produces;
|
||||||
|
import javax.ws.rs.QueryParam;
|
||||||
import javax.ws.rs.core.MediaType;
|
import javax.ws.rs.core.MediaType;
|
||||||
import javax.ws.rs.core.Response;
|
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.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.TreeMap;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A set of endpoints to adjust the log levels of runtime loggers.
|
* A set of endpoints to adjust the log levels of runtime loggers.
|
||||||
|
@ -48,10 +48,16 @@ import java.util.TreeMap;
|
||||||
@Consumes(MediaType.APPLICATION_JSON)
|
@Consumes(MediaType.APPLICATION_JSON)
|
||||||
public class LoggingResource implements ConnectResource {
|
public class LoggingResource implements ConnectResource {
|
||||||
|
|
||||||
/**
|
private static final org.slf4j.Logger log = LoggerFactory.getLogger(LoggingResource.class);
|
||||||
* Log4j uses "root" (case-insensitive) as name of the root logger.
|
|
||||||
*/
|
private static final String WORKER_SCOPE = "worker";
|
||||||
private static final String ROOT_LOGGER_NAME = "root";
|
private static final String CLUSTER_SCOPE = "cluster";
|
||||||
|
|
||||||
|
private final Herder herder;
|
||||||
|
|
||||||
|
public LoggingResource(Herder herder) {
|
||||||
|
this.herder = herder;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void requestTimeout(long requestTimeoutMs) {
|
public void requestTimeout(long requestTimeoutMs) {
|
||||||
|
@ -67,19 +73,7 @@ public class LoggingResource implements ConnectResource {
|
||||||
@Path("/")
|
@Path("/")
|
||||||
@Operation(summary = "List the current loggers that have their levels explicitly set and their log levels")
|
@Operation(summary = "List the current loggers that have their levels explicitly set and their log levels")
|
||||||
public Response listLoggers() {
|
public Response listLoggers() {
|
||||||
Map<String, Map<String, String>> loggers = new TreeMap<>();
|
return Response.ok(herder.allLoggerLevels()).build();
|
||||||
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();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -94,34 +88,17 @@ public class LoggingResource implements ConnectResource {
|
||||||
public Response getLogger(final @PathParam("logger") String namedLogger) {
|
public Response getLogger(final @PathParam("logger") String namedLogger) {
|
||||||
Objects.requireNonNull(namedLogger, "require non-null name");
|
Objects.requireNonNull(namedLogger, "require non-null name");
|
||||||
|
|
||||||
Logger logger = null;
|
LoggerLevel loggerLevel = herder.loggerLevel(namedLogger);
|
||||||
if (ROOT_LOGGER_NAME.equalsIgnoreCase(namedLogger)) {
|
if (loggerLevel == null)
|
||||||
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) {
|
|
||||||
throw new NotFoundException("Logger " + namedLogger + " not found.");
|
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.
|
* 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:
|
* @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
|
* DEBUG, ERROR, FATAL, INFO, TRACE, WARN
|
||||||
* @return names of loggers whose levels were modified
|
* @return names of loggers whose levels were modified
|
||||||
|
@ -129,87 +106,36 @@ public class LoggingResource implements ConnectResource {
|
||||||
@PUT
|
@PUT
|
||||||
@Path("/{logger}")
|
@Path("/{logger}")
|
||||||
@Operation(summary = "Set the log level for the specified logger")
|
@Operation(summary = "Set the log level for the specified logger")
|
||||||
public Response setLevel(final @PathParam("logger") String namedLogger,
|
@SuppressWarnings("fallthrough")
|
||||||
final Map<String, String> levelMap) {
|
public Response setLevel(final @PathParam("logger") String namespace,
|
||||||
String desiredLevelStr = levelMap.get("level");
|
final Map<String, String> levelMap,
|
||||||
if (desiredLevelStr == null) {
|
@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;
|
||||||
|
}
|
||||||
|
|
||||||
|
String levelString = levelMap.get("level");
|
||||||
|
if (levelString == null) {
|
||||||
throw new BadRequestException("Desired 'level' parameter was not specified in request.");
|
throw new BadRequestException("Desired 'level' parameter was not specified in request.");
|
||||||
}
|
}
|
||||||
|
|
||||||
Level level = Level.toLevel(desiredLevelStr.toUpperCase(Locale.ROOT), null);
|
// Make sure that this is a valid level
|
||||||
|
Level level = Level.toLevel(levelString.toUpperCase(Locale.ROOT), null);
|
||||||
if (level == null) {
|
if (level == null) {
|
||||||
throw new NotFoundException("invalid log level '" + desiredLevelStr + "'.");
|
throw new NotFoundException("invalid log level '" + levelString + "'.");
|
||||||
}
|
}
|
||||||
|
|
||||||
List<Logger> childLoggers;
|
switch (scope.toLowerCase(Locale.ROOT)) {
|
||||||
if (ROOT_LOGGER_NAME.equalsIgnoreCase(namedLogger)) {
|
default:
|
||||||
childLoggers = Collections.list(currentLoggers());
|
log.warn("Received invalid scope '{}' in request to adjust logging level; will default to {}", scope, WORKER_SCOPE);
|
||||||
childLoggers.add(rootLogger());
|
case WORKER_SCOPE:
|
||||||
} else {
|
List<String> affectedLoggers = herder.setWorkerLoggerLevel(namespace, levelString);
|
||||||
childLoggers = new ArrayList<>();
|
return Response.ok(affectedLoggers).build();
|
||||||
Logger ancestorLogger = lookupLogger(namedLogger);
|
case CLUSTER_SCOPE:
|
||||||
Enumeration<Logger> en = currentLoggers();
|
herder.setClusterLoggerLevel(namespace, levelString);
|
||||||
boolean present = false;
|
return Response.noContent().build();
|
||||||
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);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
List<String> modifiedLoggerNames = new ArrayList<>();
|
|
||||||
for (Logger logger: childLoggers) {
|
|
||||||
logger.setLevel(level);
|
|
||||||
modifiedLoggerNames.add(logger.getName());
|
|
||||||
}
|
|
||||||
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();
|
|
||||||
}
|
|
||||||
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()));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.kafka.connect.runtime.standalone;
|
package org.apache.kafka.connect.runtime.standalone;
|
||||||
|
|
||||||
import org.apache.kafka.common.utils.ThreadUtils;
|
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.connector.policy.ConnectorClientConfigOverridePolicy;
|
||||||
import org.apache.kafka.connect.errors.AlreadyExistsException;
|
import org.apache.kafka.connect.errors.AlreadyExistsException;
|
||||||
import org.apache.kafka.connect.errors.ConnectException;
|
import org.apache.kafka.connect.errors.ConnectException;
|
||||||
|
@ -82,7 +83,9 @@ public class StandaloneHerder extends AbstractHerder {
|
||||||
kafkaClusterId,
|
kafkaClusterId,
|
||||||
new MemoryStatusBackingStore(),
|
new MemoryStatusBackingStore(),
|
||||||
new MemoryConfigBackingStore(worker.configTransformer()),
|
new MemoryConfigBackingStore(worker.configTransformer()),
|
||||||
connectorClientConfigOverridePolicy);
|
connectorClientConfigOverridePolicy,
|
||||||
|
Time.SYSTEM
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
// visible for testing
|
// visible for testing
|
||||||
|
@ -91,8 +94,9 @@ public class StandaloneHerder extends AbstractHerder {
|
||||||
String kafkaClusterId,
|
String kafkaClusterId,
|
||||||
StatusBackingStore statusBackingStore,
|
StatusBackingStore statusBackingStore,
|
||||||
MemoryConfigBackingStore configBackingStore,
|
MemoryConfigBackingStore configBackingStore,
|
||||||
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) {
|
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
|
||||||
super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy);
|
Time time) {
|
||||||
|
super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy, time);
|
||||||
this.configState = ClusterConfigState.EMPTY;
|
this.configState = ClusterConfigState.EMPTY;
|
||||||
this.requestExecutorService = Executors.newSingleThreadScheduledExecutor();
|
this.requestExecutorService = Executors.newSingleThreadScheduledExecutor();
|
||||||
configBackingStore.setUpdateListener(new ConfigUpdateListener());
|
configBackingStore.setUpdateListener(new ConfigUpdateListener());
|
||||||
|
@ -404,6 +408,12 @@ public class StandaloneHerder extends AbstractHerder {
|
||||||
return true;
|
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) {
|
private void startConnector(String connName, Callback<TargetState> onStart) {
|
||||||
Map<String, String> connConfigs = configState.connectorConfig(connName);
|
Map<String, String> connConfigs = configState.connectorConfig(connName);
|
||||||
TargetState targetState = configState.targetState(connName);
|
TargetState targetState = configState.targetState(connName);
|
||||||
|
@ -544,6 +554,11 @@ public class StandaloneHerder extends AbstractHerder {
|
||||||
public void onRestartRequest(RestartRequest restartRequest) {
|
public void onRestartRequest(RestartRequest restartRequest) {
|
||||||
// no-op
|
// no-op
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onLoggingLevelUpdate(String namespace, String level) {
|
||||||
|
// no-op
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static class StandaloneHerderRequest implements HerderRequest {
|
static class StandaloneHerderRequest implements HerderRequest {
|
||||||
|
|
|
@ -122,6 +122,14 @@ public interface ConfigBackingStore {
|
||||||
default void claimWritePrivileges() {
|
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.
|
* Set an update listener to get notifications when there are new records written to the backing store.
|
||||||
* @param listener non-null listener
|
* @param listener non-null listener
|
||||||
|
@ -164,6 +172,13 @@ public interface ConfigBackingStore {
|
||||||
* @param restartRequest the {@link RestartRequest restart request}
|
* @param restartRequest the {@link RestartRequest restart request}
|
||||||
*/
|
*/
|
||||||
void onRestartRequest(RestartRequest restartRequest);
|
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);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -271,6 +271,14 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme
|
||||||
.field(ONLY_FAILED_FIELD_NAME, Schema.BOOLEAN_SCHEMA)
|
.field(ONLY_FAILED_FIELD_NAME, Schema.BOOLEAN_SCHEMA)
|
||||||
.build();
|
.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
|
// Visible for testing
|
||||||
static final long READ_WRITE_TOTAL_TIMEOUT_MS = 30000;
|
static final long READ_WRITE_TOTAL_TIMEOUT_MS = 30000;
|
||||||
|
|
||||||
|
@ -732,6 +740,20 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void putLoggerLevel(String namespace, String level) {
|
||||||
|
log.debug("Writing level {} for logging namespace {} to Kafka", level, namespace);
|
||||||
|
Struct value = new Struct(LOGGER_LEVEL_V0);
|
||||||
|
value.put("level", level);
|
||||||
|
byte[] serializedValue = converter.fromConnectData(topic, value.schema(), value);
|
||||||
|
try {
|
||||||
|
configLog.sendWithReceipt(LOGGER_CLUSTER_KEY(namespace), serializedValue).get(READ_WRITE_TOTAL_TIMEOUT_MS, TimeUnit.MILLISECONDS);
|
||||||
|
} catch (InterruptedException | ExecutionException | TimeoutException e) {
|
||||||
|
log.error("Failed to write logger level to Kafka", e);
|
||||||
|
throw new ConnectException("Error writing logger level to Kafka", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// package private for testing
|
// package private for testing
|
||||||
KafkaBasedLog<String, byte[]> setupAndCreateKafkaBasedLog(String topic, final WorkerConfig config) {
|
KafkaBasedLog<String, byte[]> setupAndCreateKafkaBasedLog(String topic, final WorkerConfig config) {
|
||||||
String clusterId = config.kafkaClusterId();
|
String clusterId = config.kafkaClusterId();
|
||||||
|
@ -901,6 +923,9 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme
|
||||||
processTaskCountRecord(connectorName, value);
|
processTaskCountRecord(connectorName, value);
|
||||||
} else if (record.key().equals(SESSION_KEY_KEY)) {
|
} else if (record.key().equals(SESSION_KEY_KEY)) {
|
||||||
processSessionKeyRecord(value);
|
processSessionKeyRecord(value);
|
||||||
|
} else if (record.key().startsWith(LOGGER_CLUSTER_PREFIX)) {
|
||||||
|
String loggingNamespace = record.key().substring(LOGGER_CLUSTER_PREFIX.length());
|
||||||
|
processLoggerLevelRecord(loggingNamespace, value);
|
||||||
} else {
|
} else {
|
||||||
log.error("Discarding config update record with invalid key: {}", record.key());
|
log.error("Discarding config update record with invalid key: {}", record.key());
|
||||||
}
|
}
|
||||||
|
@ -1185,6 +1210,37 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme
|
||||||
updateListener.onSessionKeyUpdate(KafkaConfigBackingStore.this.sessionKey);
|
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) {
|
private ConnectorTaskId parseTaskId(String key) {
|
||||||
String[] parts = key.split("-");
|
String[] parts = key.split("-");
|
||||||
if (parts.length < 3) return null;
|
if (parts.length < 3) return null;
|
||||||
|
|
|
@ -169,6 +169,11 @@ public class MemoryConfigBackingStore implements ConfigBackingStore {
|
||||||
// no-op
|
// no-op
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void putLoggerLevel(String namespace, String level) {
|
||||||
|
// no-op
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void setUpdateListener(UpdateListener listener) {
|
public synchronized void setUpdateListener(UpdateListener listener) {
|
||||||
this.updateListener = listener;
|
this.updateListener = listener;
|
||||||
|
|
|
@ -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.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.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_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.CoreMatchers.containsString;
|
||||||
import static org.hamcrest.MatcherAssert.assertThat;
|
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.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
|
@ -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.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.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_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.assertEquals;
|
||||||
import static org.junit.Assert.assertNotEquals;
|
import static org.junit.Assert.assertNotEquals;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
|
@ -48,7 +48,7 @@ import org.apache.kafka.connect.source.SourceRecord;
|
||||||
import org.apache.kafka.connect.source.SourceTask;
|
import org.apache.kafka.connect.source.SourceTask;
|
||||||
import org.apache.kafka.connect.storage.StringConverter;
|
import org.apache.kafka.connect.storage.StringConverter;
|
||||||
import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster;
|
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.connect.util.clusters.EmbeddedKafkaCluster;
|
||||||
import org.apache.kafka.test.IntegrationTest;
|
import org.apache.kafka.test.IntegrationTest;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
@ -650,7 +650,7 @@ public class ExactlyOnceSourceIntegrationTest {
|
||||||
final String globalOffsetsTopic = "connect-worker-offsets-topic";
|
final String globalOffsetsTopic = "connect-worker-offsets-topic";
|
||||||
workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, globalOffsetsTopic);
|
workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, globalOffsetsTopic);
|
||||||
|
|
||||||
connectBuilder.clientConfigs(superUserClientConfig);
|
connectBuilder.clientProps(superUserClientConfig);
|
||||||
|
|
||||||
startConnect();
|
startConnect();
|
||||||
|
|
||||||
|
@ -1095,7 +1095,7 @@ public class ExactlyOnceSourceIntegrationTest {
|
||||||
private void assertConnectorStarted(StartAndStopLatch connectorStart) throws InterruptedException {
|
private void assertConnectorStarted(StartAndStopLatch connectorStart) throws InterruptedException {
|
||||||
assertTrue("Connector and tasks did not finish startup in time",
|
assertTrue("Connector and tasks did not finish startup in time",
|
||||||
connectorStart.await(
|
connectorStart.await(
|
||||||
EmbeddedConnectClusterAssertions.CONNECTOR_SETUP_DURATION_MS,
|
ConnectAssertions.CONNECTOR_SETUP_DURATION_MS,
|
||||||
TimeUnit.MILLISECONDS
|
TimeUnit.MILLISECONDS
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
@ -1105,7 +1105,7 @@ public class ExactlyOnceSourceIntegrationTest {
|
||||||
assertTrue(
|
assertTrue(
|
||||||
"Connector and tasks did not finish shutdown in time",
|
"Connector and tasks did not finish shutdown in time",
|
||||||
connectorStop.await(
|
connectorStop.await(
|
||||||
EmbeddedConnectClusterAssertions.CONNECTOR_SHUTDOWN_DURATION_MS,
|
ConnectAssertions.CONNECTOR_SHUTDOWN_DURATION_MS,
|
||||||
TimeUnit.MILLISECONDS
|
TimeUnit.MILLISECONDS
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
|
@ -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();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -24,6 +24,7 @@ import org.apache.kafka.common.config.ConfigValue;
|
||||||
import org.apache.kafka.common.config.SaslConfigs;
|
import org.apache.kafka.common.config.SaslConfigs;
|
||||||
import org.apache.kafka.common.config.provider.DirectoryConfigProvider;
|
import org.apache.kafka.common.config.provider.DirectoryConfigProvider;
|
||||||
import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredLoginCallbackHandler;
|
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.Connector;
|
||||||
import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy;
|
import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy;
|
||||||
import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy;
|
import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy;
|
||||||
|
@ -176,9 +177,7 @@ public class AbstractHerderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testConnectors() {
|
public void testConnectors() {
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(configStore.snapshot()).thenReturn(SNAPSHOT);
|
when(configStore.snapshot()).thenReturn(SNAPSHOT);
|
||||||
assertEquals(Collections.singleton(CONN1), new HashSet<>(herder.connectors()));
|
assertEquals(Collections.singleton(CONN1), new HashSet<>(herder.connectors()));
|
||||||
|
@ -188,9 +187,7 @@ public class AbstractHerderTest {
|
||||||
public void testConnectorClientConfigOverridePolicyClose() {
|
public void testConnectorClientConfigOverridePolicyClose() {
|
||||||
SampleConnectorClientConfigOverridePolicy noneConnectorClientConfigOverridePolicy = new SampleConnectorClientConfigOverridePolicy();
|
SampleConnectorClientConfigOverridePolicy noneConnectorClientConfigOverridePolicy = new SampleConnectorClientConfigOverridePolicy();
|
||||||
|
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder(noneConnectorClientConfigOverridePolicy);
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
herder.stopServices();
|
herder.stopServices();
|
||||||
assertTrue(noneConnectorClientConfigOverridePolicy.isClosed());
|
assertTrue(noneConnectorClientConfigOverridePolicy.isClosed());
|
||||||
|
@ -200,9 +197,7 @@ public class AbstractHerderTest {
|
||||||
public void testConnectorStatus() {
|
public void testConnectorStatus() {
|
||||||
ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0);
|
ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0);
|
||||||
|
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(plugins.newConnector(anyString())).thenReturn(new SampleSourceConnector());
|
when(plugins.newConnector(anyString())).thenReturn(new SampleSourceConnector());
|
||||||
when(herder.plugins()).thenReturn(plugins);
|
when(herder.plugins()).thenReturn(plugins);
|
||||||
|
@ -236,9 +231,7 @@ public class AbstractHerderTest {
|
||||||
public void testConnectorStatusMissingPlugin() {
|
public void testConnectorStatusMissingPlugin() {
|
||||||
ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0);
|
ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0);
|
||||||
|
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(plugins.newConnector(anyString())).thenThrow(new ConnectException("Unable to find class"));
|
when(plugins.newConnector(anyString())).thenThrow(new ConnectException("Unable to find class"));
|
||||||
when(herder.plugins()).thenReturn(plugins);
|
when(herder.plugins()).thenReturn(plugins);
|
||||||
|
@ -269,9 +262,7 @@ public class AbstractHerderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testConnectorInfo() {
|
public void testConnectorInfo() {
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(plugins.newConnector(anyString())).thenReturn(new SampleSourceConnector());
|
when(plugins.newConnector(anyString())).thenReturn(new SampleSourceConnector());
|
||||||
when(herder.plugins()).thenReturn(plugins);
|
when(herder.plugins()).thenReturn(plugins);
|
||||||
|
@ -288,9 +279,7 @@ public class AbstractHerderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testPauseConnector() {
|
public void testPauseConnector() {
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(configStore.contains(CONN1)).thenReturn(true);
|
when(configStore.contains(CONN1)).thenReturn(true);
|
||||||
|
|
||||||
|
@ -301,9 +290,7 @@ public class AbstractHerderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testResumeConnector() {
|
public void testResumeConnector() {
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(configStore.contains(CONN1)).thenReturn(true);
|
when(configStore.contains(CONN1)).thenReturn(true);
|
||||||
|
|
||||||
|
@ -314,9 +301,7 @@ public class AbstractHerderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testConnectorInfoMissingPlugin() {
|
public void testConnectorInfoMissingPlugin() {
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(plugins.newConnector(anyString())).thenThrow(new ConnectException("No class found"));
|
when(plugins.newConnector(anyString())).thenThrow(new ConnectException("No class found"));
|
||||||
when(herder.plugins()).thenReturn(plugins);
|
when(herder.plugins()).thenReturn(plugins);
|
||||||
|
@ -336,9 +321,7 @@ public class AbstractHerderTest {
|
||||||
ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0);
|
ConnectorTaskId taskId = new ConnectorTaskId(connectorName, 0);
|
||||||
String workerId = "workerId";
|
String workerId = "workerId";
|
||||||
|
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
final ArgumentCaptor<TaskStatus> taskStatusArgumentCaptor = ArgumentCaptor.forClass(TaskStatus.class);
|
final ArgumentCaptor<TaskStatus> taskStatusArgumentCaptor = ArgumentCaptor.forClass(TaskStatus.class);
|
||||||
doNothing().when(statusStore).putSafe(taskStatusArgumentCaptor.capture());
|
doNothing().when(statusStore).putSafe(taskStatusArgumentCaptor.capture());
|
||||||
|
@ -358,9 +341,7 @@ public class AbstractHerderTest {
|
||||||
public void testBuildRestartPlanForUnknownConnector() {
|
public void testBuildRestartPlanForUnknownConnector() {
|
||||||
String connectorName = "UnknownConnector";
|
String connectorName = "UnknownConnector";
|
||||||
RestartRequest restartRequest = new RestartRequest(connectorName, false, true);
|
RestartRequest restartRequest = new RestartRequest(connectorName, false, true);
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(statusStore.get(connectorName)).thenReturn(null);
|
when(statusStore.get(connectorName)).thenReturn(null);
|
||||||
|
|
||||||
|
@ -415,9 +396,7 @@ public class AbstractHerderTest {
|
||||||
taskStatuses.add(new TaskStatus(taskId1, AbstractStatus.State.RUNNING, workerId, generation));
|
taskStatuses.add(new TaskStatus(taskId1, AbstractStatus.State.RUNNING, workerId, generation));
|
||||||
taskStatuses.add(new TaskStatus(taskId2, AbstractStatus.State.FAILED, workerId, generation));
|
taskStatuses.add(new TaskStatus(taskId2, AbstractStatus.State.FAILED, workerId, generation));
|
||||||
|
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(herder.rawConfig(connectorName)).thenReturn(null);
|
when(herder.rawConfig(connectorName)).thenReturn(null);
|
||||||
|
|
||||||
|
@ -447,9 +426,7 @@ public class AbstractHerderTest {
|
||||||
taskStatuses.add(new TaskStatus(taskId1, AbstractStatus.State.RUNNING, workerId, generation));
|
taskStatuses.add(new TaskStatus(taskId1, AbstractStatus.State.RUNNING, workerId, generation));
|
||||||
taskStatuses.add(new TaskStatus(taskId2, AbstractStatus.State.FAILED, workerId, generation));
|
taskStatuses.add(new TaskStatus(taskId2, AbstractStatus.State.FAILED, workerId, generation));
|
||||||
|
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(herder.rawConfig(connectorName)).thenReturn(null);
|
when(herder.rawConfig(connectorName)).thenReturn(null);
|
||||||
|
|
||||||
|
@ -1051,9 +1028,7 @@ public class AbstractHerderTest {
|
||||||
Function<T, ConfigDef> pluginConfig,
|
Function<T, ConfigDef> pluginConfig,
|
||||||
Optional<ConfigDef> baseConfig
|
Optional<ConfigDef> baseConfig
|
||||||
) throws ClassNotFoundException {
|
) throws ClassNotFoundException {
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
when(plugins.pluginClass(pluginName)).then(invocation -> newPluginInstance.get().getClass());
|
when(plugins.pluginClass(pluginName)).then(invocation -> newPluginInstance.get().getClass());
|
||||||
when(plugins.newPlugin(anyString())).then(invocation -> newPluginInstance.get());
|
when(plugins.newPlugin(anyString())).then(invocation -> newPluginInstance.get());
|
||||||
|
@ -1073,9 +1048,7 @@ public class AbstractHerderTest {
|
||||||
@Test(expected = NotFoundException.class)
|
@Test(expected = NotFoundException.class)
|
||||||
public void testGetConnectorConfigDefWithBadName() throws Exception {
|
public void testGetConnectorConfigDefWithBadName() throws Exception {
|
||||||
String connName = "AnotherPlugin";
|
String connName = "AnotherPlugin";
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
when(worker.getPlugins()).thenReturn(plugins);
|
when(worker.getPlugins()).thenReturn(plugins);
|
||||||
when(plugins.pluginClass(anyString())).thenThrow(new ClassNotFoundException());
|
when(plugins.pluginClass(anyString())).thenThrow(new ClassNotFoundException());
|
||||||
herder.connectorPluginConfig(connName);
|
herder.connectorPluginConfig(connName);
|
||||||
|
@ -1085,9 +1058,7 @@ public class AbstractHerderTest {
|
||||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||||
public void testGetConnectorConfigDefWithInvalidPluginType() throws Exception {
|
public void testGetConnectorConfigDefWithInvalidPluginType() throws Exception {
|
||||||
String connName = "AnotherPlugin";
|
String connName = "AnotherPlugin";
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
when(worker.getPlugins()).thenReturn(plugins);
|
when(worker.getPlugins()).thenReturn(plugins);
|
||||||
when(plugins.pluginClass(anyString())).thenReturn((Class) Object.class);
|
when(plugins.pluginClass(anyString())).thenReturn((Class) Object.class);
|
||||||
when(plugins.newPlugin(anyString())).thenReturn(new DirectoryConfigProvider());
|
when(plugins.newPlugin(anyString())).thenReturn(new DirectoryConfigProvider());
|
||||||
|
@ -1097,9 +1068,7 @@ public class AbstractHerderTest {
|
||||||
@Test
|
@Test
|
||||||
public void testGetConnectorTypeWithMissingPlugin() {
|
public void testGetConnectorTypeWithMissingPlugin() {
|
||||||
String connName = "AnotherPlugin";
|
String connName = "AnotherPlugin";
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
when(worker.getPlugins()).thenReturn(plugins);
|
when(worker.getPlugins()).thenReturn(plugins);
|
||||||
when(plugins.newConnector(anyString())).thenThrow(new ConnectException("No class found"));
|
when(plugins.newConnector(anyString())).thenThrow(new ConnectException("No class found"));
|
||||||
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Collections.singletonMap(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connName)));
|
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Collections.singletonMap(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connName)));
|
||||||
|
@ -1107,26 +1076,20 @@ public class AbstractHerderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGetConnectorTypeWithNullConfig() {
|
public void testGetConnectorTypeWithNullConfig() {
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(null));
|
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(null));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGetConnectorTypeWithEmptyConfig() {
|
public void testGetConnectorTypeWithEmptyConfig() {
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Collections.emptyMap()));
|
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Collections.emptyMap()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testConnectorOffsetsConnectorNotFound() {
|
public void testConnectorOffsetsConnectorNotFound() {
|
||||||
when(configStore.snapshot()).thenReturn(SNAPSHOT);
|
when(configStore.snapshot()).thenReturn(SNAPSHOT);
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
FutureCallback<ConnectorOffsets> cb = new FutureCallback<>();
|
FutureCallback<ConnectorOffsets> cb = new FutureCallback<>();
|
||||||
herder.connectorOffsets("unknown-connector", cb);
|
herder.connectorOffsets("unknown-connector", cb);
|
||||||
ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS));
|
ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS));
|
||||||
|
@ -1145,9 +1108,7 @@ public class AbstractHerderTest {
|
||||||
workerCallback.getValue().onCompletion(null, offsets);
|
workerCallback.getValue().onCompletion(null, offsets);
|
||||||
return null;
|
return null;
|
||||||
}).when(worker).connectorOffsets(eq(CONN1), eq(CONN1_CONFIG), workerCallback.capture());
|
}).when(worker).connectorOffsets(eq(CONN1), eq(CONN1_CONFIG), workerCallback.capture());
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder();
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, noneConnectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
when(configStore.snapshot()).thenReturn(SNAPSHOT);
|
when(configStore.snapshot()).thenReturn(SNAPSHOT);
|
||||||
|
|
||||||
FutureCallback<ConnectorOffsets> cb = new FutureCallback<>();
|
FutureCallback<ConnectorOffsets> cb = new FutureCallback<>();
|
||||||
|
@ -1210,9 +1171,7 @@ public class AbstractHerderTest {
|
||||||
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
|
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
|
||||||
int countOfCallingNewConnector) {
|
int countOfCallingNewConnector) {
|
||||||
|
|
||||||
AbstractHerder herder = mock(AbstractHerder.class, withSettings()
|
AbstractHerder herder = testHerder(connectorClientConfigOverridePolicy);
|
||||||
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, connectorClientConfigOverridePolicy)
|
|
||||||
.defaultAnswer(CALLS_REAL_METHODS));
|
|
||||||
|
|
||||||
// Call to validateConnectorConfig
|
// Call to validateConnectorConfig
|
||||||
when(worker.configTransformer()).thenReturn(transformer);
|
when(worker.configTransformer()).thenReturn(transformer);
|
||||||
|
@ -1232,6 +1191,16 @@ public class AbstractHerderTest {
|
||||||
return herder;
|
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) {
|
private void mockValidationIsolation(String connectorClass, Connector connector) {
|
||||||
when(plugins.newConnector(connectorClass)).thenReturn(connector);
|
when(plugins.newConnector(connectorClass)).thenReturn(connector);
|
||||||
when(plugins.connectorLoader(connectorClass)).thenReturn(classLoader);
|
when(plugins.connectorLoader(connectorClass)).thenReturn(classLoader);
|
||||||
|
|
|
@ -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) { };
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -34,6 +34,7 @@ import org.apache.kafka.common.utils.LogCaptureAppender;
|
||||||
import org.apache.kafka.connect.rest.ConnectRestExtension;
|
import org.apache.kafka.connect.rest.ConnectRestExtension;
|
||||||
import org.apache.kafka.connect.runtime.Herder;
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
@ -56,7 +57,6 @@ import java.util.Map;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertNotEquals;
|
import static org.junit.Assert.assertNotEquals;
|
||||||
import static org.junit.Assert.assertNotNull;
|
|
||||||
import static org.junit.Assert.assertNull;
|
import static org.junit.Assert.assertNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
|
@ -247,15 +247,18 @@ public class ConnectRestServerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testLoggersEndpointWithDefaults() throws IOException {
|
public void testLoggerEndpointWithDefaults() throws IOException {
|
||||||
Map<String, String> configMap = new HashMap<>(baseServerProps());
|
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(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
||||||
doReturn(plugins).when(herder).plugins();
|
doReturn(plugins).when(herder).plugins();
|
||||||
expectEmptyRestExtensions();
|
expectEmptyRestExtensions();
|
||||||
|
doReturn(Collections.emptyList()).when(herder).setWorkerLoggerLevel(logger, loggingLevel);
|
||||||
// create some loggers in the process
|
doReturn(Collections.singletonMap(logger, new LoggerLevel(loggingLevel, lastModified))).when(herder).allLoggerLevels();
|
||||||
LoggerFactory.getLogger("a.b.c.s.W");
|
|
||||||
|
|
||||||
server = new ConnectRestServer(null, null, configMap);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
server.initializeServer();
|
server.initializeServer();
|
||||||
|
@ -265,14 +268,16 @@ public class ConnectRestServerTest {
|
||||||
|
|
||||||
URI serverUrl = server.advertisedUrl();
|
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");
|
String responseStr = executeGet(serverUrl, "/admin/loggers");
|
||||||
Map<String, Map<String, ?>> loggers = mapper.readValue(responseStr, new TypeReference<Map<String, Map<String, ?>>>() {
|
|
||||||
});
|
Map<String, Object> expectedLogger = new HashMap<>();
|
||||||
assertNotNull("expected non null response for /admin/loggers" + prettyPrint(loggers), loggers);
|
expectedLogger.put("level", loggingLevel);
|
||||||
assertTrue("expect at least 1 logger. instead found " + prettyPrint(loggers), loggers.size() >= 1);
|
expectedLogger.put("last_modified", lastModified);
|
||||||
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, 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
|
@Test
|
||||||
|
|
|
@ -16,176 +16,150 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.connect.runtime.rest.resources;
|
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.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.kafka.connect.runtime.rest.errors.BadRequestException;
|
||||||
import org.apache.log4j.Hierarchy;
|
import org.junit.Before;
|
||||||
import org.apache.log4j.Level;
|
|
||||||
import org.apache.log4j.Logger;
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
|
import org.mockito.Mock;
|
||||||
import org.mockito.junit.MockitoJUnitRunner;
|
import org.mockito.junit.MockitoJUnitRunner;
|
||||||
|
import org.slf4j.event.Level;
|
||||||
|
|
||||||
|
import javax.ws.rs.core.Response;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Enumeration;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Vector;
|
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertNull;
|
import static org.junit.Assert.assertNull;
|
||||||
import static org.junit.Assert.assertThrows;
|
import static org.junit.Assert.assertThrows;
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
import static org.mockito.Mockito.verify;
|
||||||
import static org.mockito.Mockito.mock;
|
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
@RunWith(MockitoJUnitRunner.StrictStubs.class)
|
@RunWith(MockitoJUnitRunner.StrictStubs.class)
|
||||||
public class LoggingResourceTest {
|
public class LoggingResourceTest {
|
||||||
|
|
||||||
@Test
|
private LoggingResource loggingResource;
|
||||||
public void getLoggersIgnoresNullLevelsTest() {
|
|
||||||
LoggingResource loggingResource = mock(LoggingResource.class);
|
@Mock
|
||||||
Logger root = new Logger("root") {
|
private Herder herder;
|
||||||
};
|
|
||||||
Logger a = new Logger("a") {
|
@Before
|
||||||
};
|
public void setup() {
|
||||||
a.setLevel(null);
|
loggingResource = new LoggingResource(herder);
|
||||||
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"));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void getLoggerFallsbackToEffectiveLogLevelTest() {
|
public void testGetLevelNotFound() {
|
||||||
LoggingResource loggingResource = mock(LoggingResource.class);
|
final String logger = "org.apache.rostropovich";
|
||||||
Logger root = new Logger("root") {
|
when(herder.loggerLevel(logger)).thenReturn(null);
|
||||||
};
|
assertThrows(
|
||||||
root.setLevel(Level.ERROR);
|
NotFoundException.class,
|
||||||
Hierarchy hierarchy = new Hierarchy(root);
|
() -> loggingResource.getLogger(logger)
|
||||||
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"));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void getUnknownLoggerTest() {
|
public void testGetLevel() {
|
||||||
LoggingResource loggingResource = mock(LoggingResource.class);
|
final String logger = "org.apache.kafka.producer";
|
||||||
Logger root = new Logger("root") {
|
final LoggerLevel expectedLevel = new LoggerLevel(Level.WARN.toString(), 976L);
|
||||||
};
|
when(herder.loggerLevel(logger)).thenReturn(expectedLevel);
|
||||||
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"));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
Response response = loggingResource.getLogger(logger);
|
||||||
public void setLevelTest() {
|
assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
||||||
LoggingResource loggingResource = mock(LoggingResource.class);
|
LoggerLevel actualLevel = (LoggerLevel) response.getEntity();
|
||||||
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);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
assertEquals(
|
||||||
public void setRootLevelTest() {
|
expectedLevel,
|
||||||
LoggingResource loggingResource = mock(LoggingResource.class);
|
actualLevel
|
||||||
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);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void setLevelWithEmptyArgTest() {
|
public void setLevelWithEmptyArgTest() {
|
||||||
LoggingResource loggingResource = mock(LoggingResource.class);
|
for (String scope : Arrays.asList("worker", "cluster", "N/A", null)) {
|
||||||
Logger root = new Logger("root") {
|
assertThrows(
|
||||||
};
|
BadRequestException.class,
|
||||||
root.setLevel(Level.ERROR);
|
() -> loggingResource.setLevel(
|
||||||
Hierarchy hierarchy = new Hierarchy(root);
|
"@root",
|
||||||
Logger a = hierarchy.getLogger("a");
|
Collections.emptyMap(),
|
||||||
a.setLevel(null);
|
scope
|
||||||
Logger b = hierarchy.getLogger("b");
|
)
|
||||||
b.setLevel(Level.INFO);
|
);
|
||||||
when(loggingResource.setLevel(any(), any())).thenCallRealMethod();
|
}
|
||||||
assertThrows(BadRequestException.class, () -> loggingResource.setLevel("@root", Collections.emptyMap()));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void setLevelWithInvalidArgTest() {
|
public void setLevelWithInvalidArgTest() {
|
||||||
LoggingResource loggingResource = mock(LoggingResource.class);
|
for (String scope : Arrays.asList("worker", "cluster", "N/A", null)) {
|
||||||
Logger root = new Logger("root") {
|
assertThrows(
|
||||||
};
|
NotFoundException.class,
|
||||||
root.setLevel(Level.ERROR);
|
() -> loggingResource.setLevel(
|
||||||
Hierarchy hierarchy = new Hierarchy(root);
|
"@root",
|
||||||
Logger a = hierarchy.getLogger("a");
|
Collections.singletonMap("level", "HIGH"),
|
||||||
a.setLevel(null);
|
scope
|
||||||
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")));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private Enumeration<Logger> loggers(Logger... loggers) {
|
@Test
|
||||||
return new Vector<>(Arrays.asList(loggers)).elements();
|
public void testSetLevelDefaultScope() {
|
||||||
|
testSetLevelWorkerScope(null, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSetLevelInvalidScope() {
|
||||||
|
testSetLevelWorkerScope("kip-976", true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
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);
|
||||||
|
}
|
||||||
|
|
||||||
|
@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);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -19,6 +19,7 @@ package org.apache.kafka.connect.runtime.standalone;
|
||||||
import org.apache.kafka.common.config.Config;
|
import org.apache.kafka.common.config.Config;
|
||||||
import org.apache.kafka.common.config.ConfigDef;
|
import org.apache.kafka.common.config.ConfigDef;
|
||||||
import org.apache.kafka.common.config.ConfigValue;
|
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.Connector;
|
||||||
import org.apache.kafka.connect.connector.Task;
|
import org.apache.kafka.connect.connector.Task;
|
||||||
import org.apache.kafka.connect.errors.AlreadyExistsException;
|
import org.apache.kafka.connect.errors.AlreadyExistsException;
|
||||||
|
@ -130,7 +131,7 @@ public class StandaloneHerderTest {
|
||||||
worker = PowerMock.createMock(Worker.class);
|
worker = PowerMock.createMock(Worker.class);
|
||||||
String[] methodNames = new String[]{"connectorType", "buildRestartPlan", "recordRestarting"};
|
String[] methodNames = new String[]{"connectorType", "buildRestartPlan", "recordRestarting"};
|
||||||
herder = PowerMock.createPartialMock(StandaloneHerder.class, methodNames,
|
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<>();
|
createCallback = new FutureCallback<>();
|
||||||
plugins = PowerMock.createMock(Plugins.class);
|
plugins = PowerMock.createMock(Plugins.class);
|
||||||
pluginLoader = PowerMock.createMock(PluginClassLoader.class);
|
pluginLoader = PowerMock.createMock(PluginClassLoader.class);
|
||||||
|
|
|
@ -1461,6 +1461,71 @@ public class KafkaConfigBackingStoreTest {
|
||||||
PowerMock.verifyAll();
|
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
|
@Test
|
||||||
public void testExceptionOnStartWhenConfigTopicHasMultiplePartitions() throws Exception {
|
public void testExceptionOnStartWhenConfigTopicHasMultiplePartitions() throws Exception {
|
||||||
expectConfigure();
|
expectConfigure();
|
||||||
|
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
File diff suppressed because it is too large
Load Diff
|
@ -16,603 +16,14 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.connect.util.clusters;
|
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 {
|
@Deprecated
|
||||||
|
public class EmbeddedConnectClusterAssertions extends ConnectAssertions {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectClusterAssertions.class);
|
EmbeddedConnectClusterAssertions(EmbeddedConnect connect) {
|
||||||
public static final long WORKER_SETUP_DURATION_MS = TimeUnit.MINUTES.toMillis(5);
|
super(connect);
|
||||||
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());
|
|
||||||
}
|
|
||||||
|
|
||||||
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 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();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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"/>
|
<Bug pattern="EQ_CHECK_FOR_OPERAND_NOT_COMPATIBLE_WITH_THIS"/>
|
||||||
</Match>
|
</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>
|
<Match>
|
||||||
<!-- Suppress some minor warnings about machine-generated code for benchmarking. -->
|
<!-- Suppress some minor warnings about machine-generated code for benchmarking. -->
|
||||||
<Package name="~org\.apache\.kafka\.jmh\..*\.jmh_generated"/>
|
<Package name="~org\.apache\.kafka\.jmh\..*\.jmh_generated"/>
|
||||||
|
|
|
@ -241,6 +241,21 @@ class ConnectServiceBase(KafkaPathResolverMixin, Service):
|
||||||
def validate_config(self, connector_type, validate_request, node=None):
|
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")
|
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"):
|
def _rest(self, path, body=None, node=None, method="GET"):
|
||||||
if node is None:
|
if node is None:
|
||||||
node = random.choice(self.nodes)
|
node = random.choice(self.nodes)
|
||||||
|
|
|
@ -81,7 +81,13 @@ class ConnectDistributedTest(Test):
|
||||||
self.value_converter = "org.apache.kafka.connect.json.JsonConverter"
|
self.value_converter = "org.apache.kafka.connect.json.JsonConverter"
|
||||||
self.schemas = True
|
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,
|
self.kafka = KafkaService(self.test_context, self.num_brokers, self.zk,
|
||||||
security_protocol=security_protocol, interbroker_security_protocol=security_protocol,
|
security_protocol=security_protocol, interbroker_security_protocol=security_protocol,
|
||||||
topics=self.topics, version=broker_version,
|
topics=self.topics, version=broker_version,
|
||||||
|
@ -94,7 +100,7 @@ class ConnectDistributedTest(Test):
|
||||||
for node in self.kafka.nodes:
|
for node in self.kafka.nodes:
|
||||||
node.config[config_property.MESSAGE_TIMESTAMP_TYPE] = timestamp_type
|
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)
|
include_filestream_connectors=include_filestream_connectors)
|
||||||
self.cc.log_level = "DEBUG"
|
self.cc.log_level = "DEBUG"
|
||||||
|
|
||||||
|
@ -375,6 +381,184 @@ class ConnectDistributedTest(Test):
|
||||||
wait_until(lambda: self.is_paused(self.source, node), timeout_sec=120,
|
wait_until(lambda: self.is_paused(self.source, node), timeout_sec=120,
|
||||||
err_msg="Failed to see connector startup in PAUSED state")
|
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)
|
@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)
|
@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):
|
def test_file_source_and_sink(self, security_protocol, exactly_once_source, connect_protocol, metadata_quorum):
|
||||||
|
@ -434,15 +618,7 @@ class ConnectDistributedTest(Test):
|
||||||
# Don't want to restart worker nodes in the same order every time
|
# Don't want to restart worker nodes in the same order every time
|
||||||
shuffled_nodes = self.cc.nodes[start:] + self.cc.nodes[:start]
|
shuffled_nodes = self.cc.nodes[start:] + self.cc.nodes[:start]
|
||||||
for node in shuffled_nodes:
|
for node in shuffled_nodes:
|
||||||
started = time.time()
|
self._restart_worker(node, clean=clean)
|
||||||
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)
|
|
||||||
|
|
||||||
# Give additional time for the consumer groups to recover. Even if it is not a hard bounce, there are
|
# 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
|
# 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).
|
# (e.g. if the client shuts down before it has received the memberId from its initial JoinGroup).
|
||||||
|
@ -560,14 +736,7 @@ class ConnectDistributedTest(Test):
|
||||||
# Don't want to restart worker nodes in the same order every time
|
# Don't want to restart worker nodes in the same order every time
|
||||||
shuffled_nodes = self.cc.nodes[start:] + self.cc.nodes[:start]
|
shuffled_nodes = self.cc.nodes[start:] + self.cc.nodes[:start]
|
||||||
for node in shuffled_nodes:
|
for node in shuffled_nodes:
|
||||||
started = time.time()
|
self._restart_worker(node, clean=clean)
|
||||||
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)
|
|
||||||
|
|
||||||
if i < 2:
|
if i < 2:
|
||||||
# Give additional time for the worker group to recover. Even if it is not a hard bounce, there are
|
# Give additional time for the worker group to recover. Even if it is not a hard bounce, there are
|
||||||
|
@ -775,3 +944,13 @@ class ConnectDistributedTest(Test):
|
||||||
return list(node.account.ssh_capture("cat " + file))
|
return list(node.account.ssh_capture("cat " + file))
|
||||||
except RemoteCommandError:
|
except RemoteCommandError:
|
||||||
return []
|
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…
Reference in New Issue