mirror of https://github.com/apache/kafka.git
KAFKA-15086, KAFKA-9981: Intra-cluster communication for Mirror Maker 2 (#13137)
Reviewers: Daniel Urban <durban@cloudera.com>, Greg Harris <greg.harris@aiven.io>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
This commit is contained in:
parent
031d0ff1ba
commit
f93d5af839
11
build.gradle
11
build.gradle
|
@ -2865,6 +2865,17 @@ project(':connect:mirror') {
|
||||||
implementation libs.argparse4j
|
implementation libs.argparse4j
|
||||||
implementation libs.jacksonAnnotations
|
implementation libs.jacksonAnnotations
|
||||||
implementation libs.slf4jApi
|
implementation libs.slf4jApi
|
||||||
|
implementation libs.jacksonAnnotations
|
||||||
|
implementation libs.jacksonJaxrsJsonProvider
|
||||||
|
implementation libs.jerseyContainerServlet
|
||||||
|
implementation libs.jerseyHk2
|
||||||
|
implementation libs.jaxbApi // Jersey dependency that was available in the JDK before Java 9
|
||||||
|
implementation libs.activation // Jersey dependency that was available in the JDK before Java 9
|
||||||
|
implementation libs.jettyServer
|
||||||
|
implementation libs.jettyServlet
|
||||||
|
implementation libs.jettyServlets
|
||||||
|
implementation libs.jettyClient
|
||||||
|
implementation libs.swaggerAnnotations
|
||||||
|
|
||||||
testImplementation libs.junitJupiter
|
testImplementation libs.junitJupiter
|
||||||
testImplementation libs.mockitoCore
|
testImplementation libs.mockitoCore
|
||||||
|
|
|
@ -594,6 +594,9 @@
|
||||||
<allow pkg="org.apache.kafka.connect.integration" />
|
<allow pkg="org.apache.kafka.connect.integration" />
|
||||||
<allow pkg="org.apache.kafka.connect.mirror" />
|
<allow pkg="org.apache.kafka.connect.mirror" />
|
||||||
<allow pkg="kafka.server" />
|
<allow pkg="kafka.server" />
|
||||||
|
<subpackage name="rest">
|
||||||
|
<allow pkg="javax.ws.rs" />
|
||||||
|
</subpackage>
|
||||||
</subpackage>
|
</subpackage>
|
||||||
|
|
||||||
<subpackage name="runtime">
|
<subpackage name="runtime">
|
||||||
|
|
|
@ -138,7 +138,7 @@
|
||||||
<suppress checks="ParameterNumber"
|
<suppress checks="ParameterNumber"
|
||||||
files="Worker(SinkTask|SourceTask|Coordinator).java"/>
|
files="Worker(SinkTask|SourceTask|Coordinator).java"/>
|
||||||
<suppress checks="ParameterNumber"
|
<suppress checks="ParameterNumber"
|
||||||
files="ConfigKeyInfo.java"/>
|
files="(ConfigKeyInfo|DistributedHerder).java"/>
|
||||||
|
|
||||||
<suppress checks="ClassDataAbstractionCoupling"
|
<suppress checks="ClassDataAbstractionCoupling"
|
||||||
files="(RestServer|AbstractHerder|DistributedHerder|Worker).java"/>
|
files="(RestServer|AbstractHerder|DistributedHerder|Worker).java"/>
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.kafka.connect.mirror;
|
||||||
import org.apache.kafka.common.utils.Time;
|
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.common.utils.Exit;
|
import org.apache.kafka.common.utils.Exit;
|
||||||
|
import org.apache.kafka.connect.mirror.rest.MirrorRestServer;
|
||||||
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.Worker;
|
import org.apache.kafka.connect.runtime.Worker;
|
||||||
|
@ -26,6 +27,7 @@ import org.apache.kafka.connect.runtime.WorkerConfigTransformer;
|
||||||
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
|
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
|
||||||
import org.apache.kafka.connect.runtime.distributed.DistributedHerder;
|
import org.apache.kafka.connect.runtime.distributed.DistributedHerder;
|
||||||
import org.apache.kafka.connect.runtime.distributed.NotLeaderException;
|
import org.apache.kafka.connect.runtime.distributed.NotLeaderException;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
import org.apache.kafka.connect.storage.KafkaOffsetBackingStore;
|
import org.apache.kafka.connect.storage.KafkaOffsetBackingStore;
|
||||||
import org.apache.kafka.connect.storage.StatusBackingStore;
|
import org.apache.kafka.connect.storage.StatusBackingStore;
|
||||||
import org.apache.kafka.connect.storage.KafkaStatusBackingStore;
|
import org.apache.kafka.connect.storage.KafkaStatusBackingStore;
|
||||||
|
@ -46,6 +48,9 @@ import net.sourceforge.argparse4j.inf.ArgumentParser;
|
||||||
import net.sourceforge.argparse4j.inf.ArgumentParserException;
|
import net.sourceforge.argparse4j.inf.ArgumentParserException;
|
||||||
import net.sourceforge.argparse4j.ArgumentParsers;
|
import net.sourceforge.argparse4j.ArgumentParsers;
|
||||||
|
|
||||||
|
import java.io.UnsupportedEncodingException;
|
||||||
|
import java.net.URLEncoder;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
@ -101,11 +106,13 @@ public class MirrorMaker {
|
||||||
private CountDownLatch stopLatch;
|
private CountDownLatch stopLatch;
|
||||||
private final AtomicBoolean shutdown = new AtomicBoolean(false);
|
private final AtomicBoolean shutdown = new AtomicBoolean(false);
|
||||||
private final ShutdownHook shutdownHook;
|
private final ShutdownHook shutdownHook;
|
||||||
private final String advertisedBaseUrl;
|
private final String advertisedUrl;
|
||||||
private final Time time;
|
private final Time time;
|
||||||
private final MirrorMakerConfig config;
|
private final MirrorMakerConfig config;
|
||||||
private final Set<String> clusters;
|
private final Set<String> clusters;
|
||||||
private final Set<SourceAndTarget> herderPairs;
|
private final Set<SourceAndTarget> herderPairs;
|
||||||
|
private final MirrorRestServer internalServer;
|
||||||
|
private final RestClient restClient;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param config MM2 configuration from mm2.properties file
|
* @param config MM2 configuration from mm2.properties file
|
||||||
|
@ -117,7 +124,16 @@ public class MirrorMaker {
|
||||||
public MirrorMaker(MirrorMakerConfig config, List<String> clusters, Time time) {
|
public MirrorMaker(MirrorMakerConfig config, List<String> clusters, Time time) {
|
||||||
log.debug("Kafka MirrorMaker instance created");
|
log.debug("Kafka MirrorMaker instance created");
|
||||||
this.time = time;
|
this.time = time;
|
||||||
this.advertisedBaseUrl = "NOTUSED";
|
if (config.enableInternalRest()) {
|
||||||
|
this.restClient = new RestClient(config);
|
||||||
|
internalServer = new MirrorRestServer(config.originals(), restClient);
|
||||||
|
internalServer.initializeServer();
|
||||||
|
this.advertisedUrl = internalServer.advertisedUrl().toString();
|
||||||
|
} else {
|
||||||
|
internalServer = null;
|
||||||
|
restClient = null;
|
||||||
|
this.advertisedUrl = "NOTUSED";
|
||||||
|
}
|
||||||
this.config = config;
|
this.config = config;
|
||||||
if (clusters != null && !clusters.isEmpty()) {
|
if (clusters != null && !clusters.isEmpty()) {
|
||||||
this.clusters = new HashSet<>(clusters);
|
this.clusters = new HashSet<>(clusters);
|
||||||
|
@ -171,6 +187,10 @@ public class MirrorMaker {
|
||||||
startLatch.countDown();
|
startLatch.countDown();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if (internalServer != null) {
|
||||||
|
log.info("Initializing internal REST resources");
|
||||||
|
internalServer.initializeInternalResources(herders);
|
||||||
|
}
|
||||||
log.info("Configuring connectors...");
|
log.info("Configuring connectors...");
|
||||||
herderPairs.forEach(this::configureConnectors);
|
herderPairs.forEach(this::configureConnectors);
|
||||||
log.info("Kafka MirrorMaker started");
|
log.info("Kafka MirrorMaker started");
|
||||||
|
@ -180,6 +200,9 @@ public class MirrorMaker {
|
||||||
boolean wasShuttingDown = shutdown.getAndSet(true);
|
boolean wasShuttingDown = shutdown.getAndSet(true);
|
||||||
if (!wasShuttingDown) {
|
if (!wasShuttingDown) {
|
||||||
log.info("Kafka MirrorMaker stopping");
|
log.info("Kafka MirrorMaker stopping");
|
||||||
|
if (internalServer != null) {
|
||||||
|
Utils.closeQuietly(internalServer::stop, "Internal REST server");
|
||||||
|
}
|
||||||
for (Herder herder : herders.values()) {
|
for (Herder herder : herders.values()) {
|
||||||
try {
|
try {
|
||||||
herder.stop();
|
herder.stop();
|
||||||
|
@ -204,11 +227,13 @@ public class MirrorMaker {
|
||||||
Map<String, String> connectorProps = config.connectorBaseConfig(sourceAndTarget, connectorClass);
|
Map<String, String> connectorProps = config.connectorBaseConfig(sourceAndTarget, connectorClass);
|
||||||
herders.get(sourceAndTarget)
|
herders.get(sourceAndTarget)
|
||||||
.putConnectorConfig(connectorClass.getSimpleName(), connectorProps, true, (e, x) -> {
|
.putConnectorConfig(connectorClass.getSimpleName(), connectorProps, true, (e, x) -> {
|
||||||
if (e instanceof NotLeaderException) {
|
if (e == null) {
|
||||||
// No way to determine if the connector is a leader or not beforehand.
|
log.info("{} connector configured for {}.", connectorClass.getSimpleName(), sourceAndTarget);
|
||||||
log.info("Connector {} is a follower. Using existing configuration.", sourceAndTarget);
|
} else if (e instanceof NotLeaderException) {
|
||||||
|
// No way to determine if the herder is a leader or not beforehand.
|
||||||
|
log.info("This node is a follower for {}. Using existing connector configuration.", sourceAndTarget);
|
||||||
} else {
|
} else {
|
||||||
log.info("Connector {} configured.", sourceAndTarget, e);
|
log.error("Failed to configure {} connector for {}", connectorClass.getSimpleName(), sourceAndTarget, e);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -226,7 +251,14 @@ public class MirrorMaker {
|
||||||
private void addHerder(SourceAndTarget sourceAndTarget) {
|
private void addHerder(SourceAndTarget sourceAndTarget) {
|
||||||
log.info("creating herder for " + sourceAndTarget.toString());
|
log.info("creating herder for " + sourceAndTarget.toString());
|
||||||
Map<String, String> workerProps = config.workerConfig(sourceAndTarget);
|
Map<String, String> workerProps = config.workerConfig(sourceAndTarget);
|
||||||
String advertisedUrl = advertisedBaseUrl + "/" + sourceAndTarget.source();
|
List<String> restNamespace;
|
||||||
|
try {
|
||||||
|
String encodedSource = encodePath(sourceAndTarget.source());
|
||||||
|
String encodedTarget = encodePath(sourceAndTarget.target());
|
||||||
|
restNamespace = Arrays.asList(encodedSource, encodedTarget);
|
||||||
|
} catch (UnsupportedEncodingException e) {
|
||||||
|
throw new RuntimeException("Unable to create encoded URL paths for source and target using UTF-8", e);
|
||||||
|
}
|
||||||
String workerId = sourceAndTarget.toString();
|
String workerId = sourceAndTarget.toString();
|
||||||
Plugins plugins = new Plugins(workerProps);
|
Plugins plugins = new Plugins(workerProps);
|
||||||
plugins.compareAndSwapWithDelegatingLoader();
|
plugins.compareAndSwapWithDelegatingLoader();
|
||||||
|
@ -255,13 +287,26 @@ public class MirrorMaker {
|
||||||
// Pass the shared admin to the distributed herder as an additional AutoCloseable object that should be closed when the
|
// Pass the shared admin to the distributed herder as an additional AutoCloseable object that should be closed when the
|
||||||
// herder is stopped. MirrorMaker has multiple herders, and having the herder own the close responsibility is much easier than
|
// herder is stopped. MirrorMaker has multiple herders, and having the herder own the close responsibility is much easier than
|
||||||
// tracking the various shared admin objects in this class.
|
// tracking the various shared admin objects in this class.
|
||||||
// Do not provide a restClient to the DistributedHerder to indicate that request forwarding is disabled
|
|
||||||
Herder herder = new DistributedHerder(distributedConfig, time, worker,
|
Herder herder = new DistributedHerder(distributedConfig, time, worker,
|
||||||
kafkaClusterId, statusBackingStore, configBackingStore,
|
kafkaClusterId, statusBackingStore, configBackingStore,
|
||||||
advertisedUrl, null, clientConfigOverridePolicy, sharedAdmin);
|
advertisedUrl, restClient, clientConfigOverridePolicy,
|
||||||
|
restNamespace, sharedAdmin);
|
||||||
herders.put(sourceAndTarget, herder);
|
herders.put(sourceAndTarget, herder);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static String encodePath(String rawPath) throws UnsupportedEncodingException {
|
||||||
|
return URLEncoder.encode(rawPath, StandardCharsets.UTF_8.name())
|
||||||
|
// Java's out-of-the-box URL encoder encodes spaces (' ') as pluses ('+'),
|
||||||
|
// and pluses as '%2B'
|
||||||
|
// But Jetty doesn't decode pluses at all and leaves them as-are in decoded
|
||||||
|
// URLs
|
||||||
|
// So to get around that, we replace pluses in the encoded URL here with '%20',
|
||||||
|
// which is the encoding that Jetty expects for spaces
|
||||||
|
// Jetty will reverse this transformation when evaluating the path parameters
|
||||||
|
// and will return decoded strings with all special characters as they were.
|
||||||
|
.replaceAll("\\+", "%20");
|
||||||
|
}
|
||||||
|
|
||||||
private class ShutdownHook extends Thread {
|
private class ShutdownHook extends Thread {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
@ -300,7 +345,7 @@ public class MirrorMaker {
|
||||||
|
|
||||||
Properties props = Utils.loadProps(configFile.getPath());
|
Properties props = Utils.loadProps(configFile.getPath());
|
||||||
Map<String, String> config = Utils.propsToStringMap(props);
|
Map<String, String> config = Utils.propsToStringMap(props);
|
||||||
MirrorMaker mirrorMaker = new MirrorMaker(config, clusters, Time.SYSTEM);
|
MirrorMaker mirrorMaker = new MirrorMaker(config, clusters);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
mirrorMaker.start();
|
mirrorMaker.start();
|
||||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.kafka.clients.CommonClientConfigs;
|
||||||
import org.apache.kafka.connect.runtime.WorkerConfig;
|
import org.apache.kafka.connect.runtime.WorkerConfig;
|
||||||
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
|
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
|
||||||
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestServerConfig;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -81,11 +82,14 @@ public class MirrorMakerConfig extends AbstractConfig {
|
||||||
static final String TARGET_CLUSTER_PREFIX = "target.cluster.";
|
static final String TARGET_CLUSTER_PREFIX = "target.cluster.";
|
||||||
static final String SOURCE_PREFIX = "source.";
|
static final String SOURCE_PREFIX = "source.";
|
||||||
static final String TARGET_PREFIX = "target.";
|
static final String TARGET_PREFIX = "target.";
|
||||||
|
static final String ENABLE_INTERNAL_REST_CONFIG = "dedicated.mode.enable.internal.rest";
|
||||||
|
private static final String ENABLE_INTERNAL_REST_DOC =
|
||||||
|
"Whether to bring up an internal-only REST server that allows multi-node clusters to operate correctly.";
|
||||||
|
|
||||||
private final Plugins plugins;
|
private final Plugins plugins;
|
||||||
|
|
||||||
public MirrorMakerConfig(Map<?, ?> props) {
|
public MirrorMakerConfig(Map<?, ?> props) {
|
||||||
super(CONFIG_DEF, props, true);
|
super(config(), props, true);
|
||||||
plugins = new Plugins(originalsStrings());
|
plugins = new Plugins(originalsStrings());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -93,6 +97,10 @@ public class MirrorMakerConfig extends AbstractConfig {
|
||||||
return new HashSet<>(getList(CLUSTERS_CONFIG));
|
return new HashSet<>(getList(CLUSTERS_CONFIG));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean enableInternalRest() {
|
||||||
|
return getBoolean(ENABLE_INTERNAL_REST_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
public List<SourceAndTarget> clusterPairs() {
|
public List<SourceAndTarget> clusterPairs() {
|
||||||
List<SourceAndTarget> pairs = new ArrayList<>();
|
List<SourceAndTarget> pairs = new ArrayList<>();
|
||||||
Set<String> clusters = clusters();
|
Set<String> clusters = clusters();
|
||||||
|
@ -273,8 +281,10 @@ public class MirrorMakerConfig extends AbstractConfig {
|
||||||
return transformed;
|
return transformed;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static final ConfigDef CONFIG_DEF = new ConfigDef()
|
protected static ConfigDef config() {
|
||||||
|
ConfigDef result = new ConfigDef()
|
||||||
.define(CLUSTERS_CONFIG, Type.LIST, Importance.HIGH, CLUSTERS_DOC)
|
.define(CLUSTERS_CONFIG, Type.LIST, Importance.HIGH, CLUSTERS_DOC)
|
||||||
|
.define(ENABLE_INTERNAL_REST_CONFIG, Type.BOOLEAN, false, Importance.HIGH, ENABLE_INTERNAL_REST_DOC)
|
||||||
.define(CONFIG_PROVIDERS_CONFIG, Type.LIST, Collections.emptyList(), Importance.LOW, CONFIG_PROVIDERS_DOC)
|
.define(CONFIG_PROVIDERS_CONFIG, Type.LIST, Collections.emptyList(), Importance.LOW, CONFIG_PROVIDERS_DOC)
|
||||||
// security support
|
// security support
|
||||||
.define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG,
|
.define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG,
|
||||||
|
@ -285,6 +295,9 @@ public class MirrorMakerConfig extends AbstractConfig {
|
||||||
CommonClientConfigs.SECURITY_PROTOCOL_DOC)
|
CommonClientConfigs.SECURITY_PROTOCOL_DOC)
|
||||||
.withClientSslSupport()
|
.withClientSslSupport()
|
||||||
.withClientSaslSupport();
|
.withClientSaslSupport();
|
||||||
|
RestServerConfig.addInternalConfig(result);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
private Map<String, String> stringsWithPrefixStripped(String prefix) {
|
private Map<String, String> stringsWithPrefixStripped(String prefix) {
|
||||||
return originalsStrings().entrySet().stream()
|
return originalsStrings().entrySet().stream()
|
||||||
|
|
|
@ -0,0 +1,59 @@
|
||||||
|
/*
|
||||||
|
* 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.mirror.rest;
|
||||||
|
|
||||||
|
import org.apache.kafka.connect.mirror.SourceAndTarget;
|
||||||
|
import org.apache.kafka.connect.mirror.rest.resources.InternalMirrorResource;
|
||||||
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestServer;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestServerConfig;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.resources.ConnectResource;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class MirrorRestServer extends RestServer {
|
||||||
|
|
||||||
|
private final RestClient restClient;
|
||||||
|
private Map<SourceAndTarget, Herder> herders;
|
||||||
|
|
||||||
|
public MirrorRestServer(Map<?, ?> props, RestClient restClient) {
|
||||||
|
super(RestServerConfig.forInternal(props));
|
||||||
|
this.restClient = restClient;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void initializeInternalResources(Map<SourceAndTarget, Herder> herders) {
|
||||||
|
this.herders = herders;
|
||||||
|
super.initializeResources();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Collection<ConnectResource> regularResources() {
|
||||||
|
return Arrays.asList(
|
||||||
|
new InternalMirrorResource(herders, restClient)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Collection<ConnectResource> adminResources() {
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,65 @@
|
||||||
|
/*
|
||||||
|
* 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.mirror.rest.resources;
|
||||||
|
|
||||||
|
import org.apache.kafka.connect.mirror.SourceAndTarget;
|
||||||
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.resources.InternalClusterResource;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import javax.ws.rs.NotFoundException;
|
||||||
|
import javax.ws.rs.Path;
|
||||||
|
import javax.ws.rs.core.Context;
|
||||||
|
import javax.ws.rs.core.UriInfo;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
@Path("/{source}/{target}/connectors")
|
||||||
|
public class InternalMirrorResource extends InternalClusterResource {
|
||||||
|
|
||||||
|
@Context
|
||||||
|
private UriInfo uriInfo;
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(InternalMirrorResource.class);
|
||||||
|
|
||||||
|
private final Map<SourceAndTarget, Herder> herders;
|
||||||
|
|
||||||
|
public InternalMirrorResource(Map<SourceAndTarget, Herder> herders, RestClient restClient) {
|
||||||
|
super(restClient);
|
||||||
|
this.herders = herders;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Herder herderForRequest() {
|
||||||
|
String source = pathParam("source");
|
||||||
|
String target = pathParam("target");
|
||||||
|
Herder result = herders.get(new SourceAndTarget(source, target));
|
||||||
|
if (result == null) {
|
||||||
|
throw new NotFoundException("No replication flow found for source '" + source + "' and target '" + target + "'");
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private String pathParam(String name) {
|
||||||
|
String result = uriInfo.getPathParameters().getFirst(name);
|
||||||
|
if (result == null)
|
||||||
|
throw new NotFoundException("Could not parse " + name + " cluster from request path");
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,290 @@
|
||||||
|
/*
|
||||||
|
* 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.mirror.integration;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.admin.Admin;
|
||||||
|
import org.apache.kafka.clients.admin.NewTopic;
|
||||||
|
import org.apache.kafka.clients.consumer.Consumer;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||||
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
import org.apache.kafka.connect.mirror.MirrorMaker;
|
||||||
|
import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster;
|
||||||
|
import org.junit.jupiter.api.AfterEach;
|
||||||
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Tag;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Properties;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
|
import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG;
|
||||||
|
import static org.apache.kafka.test.TestUtils.waitForCondition;
|
||||||
|
|
||||||
|
@Tag("integration")
|
||||||
|
public class DedicatedMirrorIntegrationTest {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(DedicatedMirrorIntegrationTest.class);
|
||||||
|
|
||||||
|
private static final int TOPIC_CREATION_TIMEOUT_MS = 30_000;
|
||||||
|
private static final int TOPIC_REPLICATION_TIMEOUT_MS = 30_000;
|
||||||
|
|
||||||
|
private Map<String, EmbeddedKafkaCluster> kafkaClusters;
|
||||||
|
private Map<String, MirrorMaker> mirrorMakers;
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void setup() {
|
||||||
|
kafkaClusters = new HashMap<>();
|
||||||
|
mirrorMakers = new HashMap<>();
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterEach
|
||||||
|
public void teardown() throws Throwable {
|
||||||
|
AtomicReference<Throwable> shutdownFailure = new AtomicReference<>();
|
||||||
|
mirrorMakers.forEach((name, mirrorMaker) ->
|
||||||
|
Utils.closeQuietly(mirrorMaker::stop, "MirrorMaker worker '" + name + "'", shutdownFailure)
|
||||||
|
);
|
||||||
|
kafkaClusters.forEach((name, kafkaCluster) ->
|
||||||
|
Utils.closeQuietly(kafkaCluster::stop, "Embedded Kafka cluster '" + name + "'", shutdownFailure)
|
||||||
|
);
|
||||||
|
if (shutdownFailure.get() != null) {
|
||||||
|
throw shutdownFailure.get();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private EmbeddedKafkaCluster startKafkaCluster(String name, int numBrokers, Properties brokerProperties) {
|
||||||
|
if (kafkaClusters.containsKey(name))
|
||||||
|
throw new IllegalStateException("Cannot register multiple Kafka clusters with the same name");
|
||||||
|
|
||||||
|
EmbeddedKafkaCluster result = new EmbeddedKafkaCluster(numBrokers, brokerProperties);
|
||||||
|
kafkaClusters.put(name, result);
|
||||||
|
|
||||||
|
result.start();
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private MirrorMaker startMirrorMaker(String name, Map<String, String> mmProps) {
|
||||||
|
if (mirrorMakers.containsKey(name))
|
||||||
|
throw new IllegalStateException("Cannot register multiple MirrorMaker nodes with the same name");
|
||||||
|
|
||||||
|
MirrorMaker result = new MirrorMaker(mmProps);
|
||||||
|
mirrorMakers.put(name, result);
|
||||||
|
|
||||||
|
result.start();
|
||||||
|
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests a single-node cluster without the REST server enabled.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testSingleNodeCluster() throws Exception {
|
||||||
|
Properties brokerProps = new Properties();
|
||||||
|
EmbeddedKafkaCluster clusterA = startKafkaCluster("A", 1, brokerProps);
|
||||||
|
EmbeddedKafkaCluster clusterB = startKafkaCluster("B", 1, brokerProps);
|
||||||
|
|
||||||
|
clusterA.start();
|
||||||
|
clusterB.start();
|
||||||
|
|
||||||
|
try (Admin adminA = clusterA.createAdminClient();
|
||||||
|
Admin adminB = clusterB.createAdminClient()) {
|
||||||
|
|
||||||
|
// Cluster aliases
|
||||||
|
final String a = "A";
|
||||||
|
final String b = "B";
|
||||||
|
final String ab = a + "->" + b;
|
||||||
|
final String ba = b + "->" + a;
|
||||||
|
final String testTopicPrefix = "test-topic-";
|
||||||
|
|
||||||
|
Map<String, String> mmProps = new HashMap<String, String>() {{
|
||||||
|
put("dedicated.mode.enable.internal.rest", "false");
|
||||||
|
put("listeners", "http://localhost:0");
|
||||||
|
// Refresh topics very frequently to quickly pick up on topics that are created
|
||||||
|
// after the MM2 nodes are brought up during testing
|
||||||
|
put("refresh.topics.interval.seconds", "1");
|
||||||
|
put("clusters", String.join(", ", a, b));
|
||||||
|
put(a + ".bootstrap.servers", clusterA.bootstrapServers());
|
||||||
|
put(b + ".bootstrap.servers", clusterB.bootstrapServers());
|
||||||
|
put(ab + ".enabled", "true");
|
||||||
|
put(ab + ".topics", "^" + testTopicPrefix + ".*");
|
||||||
|
put(ba + ".enabled", "false");
|
||||||
|
put(ba + ".emit.heartbeats.enabled", "false");
|
||||||
|
put("replication.factor", "1");
|
||||||
|
put("checkpoints.topic.replication.factor", "1");
|
||||||
|
put("heartbeats.topic.replication.factor", "1");
|
||||||
|
put("offset-syncs.topic.replication.factor", "1");
|
||||||
|
put("offset.storage.replication.factor", "1");
|
||||||
|
put("status.storage.replication.factor", "1");
|
||||||
|
put("config.storage.replication.factor", "1");
|
||||||
|
}};
|
||||||
|
|
||||||
|
// Bring up a single-node cluster
|
||||||
|
startMirrorMaker("single node", mmProps);
|
||||||
|
|
||||||
|
final int numMessages = 10;
|
||||||
|
String topic = testTopicPrefix + "1";
|
||||||
|
|
||||||
|
// Create the topic on cluster A
|
||||||
|
createTopic(adminA, topic);
|
||||||
|
// and wait for MirrorMaker to create it on cluster B
|
||||||
|
awaitTopicCreation(b, adminB, a + "." + topic);
|
||||||
|
|
||||||
|
// Write data to the topic on cluster A
|
||||||
|
writeToTopic(clusterA, topic, numMessages);
|
||||||
|
// and wait for MirrorMaker to copy it to cluster B
|
||||||
|
awaitTopicContent(clusterB, b, a + "." + topic, numMessages);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that a multi-node dedicated cluster is able to dynamically detect new topics at runtime
|
||||||
|
* and reconfigure its connectors and their tasks to replicate those topics correctly.
|
||||||
|
* See <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-710%3A+Full+support+for+distributed+mode+in+dedicated+MirrorMaker+2.0+clusters">KIP-710</a>
|
||||||
|
* for more detail on the necessity for this test case.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testMultiNodeCluster() throws Exception {
|
||||||
|
Properties brokerProps = new Properties();
|
||||||
|
brokerProps.put("transaction.state.log.replication.factor", "1");
|
||||||
|
brokerProps.put("transaction.state.log.min.isr", "1");
|
||||||
|
EmbeddedKafkaCluster clusterA = startKafkaCluster("A", 1, brokerProps);
|
||||||
|
EmbeddedKafkaCluster clusterB = startKafkaCluster("B", 1, brokerProps);
|
||||||
|
|
||||||
|
clusterA.start();
|
||||||
|
clusterB.start();
|
||||||
|
|
||||||
|
try (Admin adminA = clusterA.createAdminClient();
|
||||||
|
Admin adminB = clusterB.createAdminClient()) {
|
||||||
|
|
||||||
|
// Cluster aliases
|
||||||
|
final String a = "A";
|
||||||
|
// Use a convoluted cluster name to ensure URL encoding/decoding works
|
||||||
|
final String b = "B- ._~:/?#[]@!$&'()*+;=\"<>%{}|\\^`618";
|
||||||
|
final String ab = a + "->" + b;
|
||||||
|
final String ba = b + "->" + a;
|
||||||
|
final String testTopicPrefix = "test-topic-";
|
||||||
|
|
||||||
|
Map<String, String> mmProps = new HashMap<String, String>() {{
|
||||||
|
put("dedicated.mode.enable.internal.rest", "true");
|
||||||
|
put("listeners", "http://localhost:0");
|
||||||
|
// Refresh topics very frequently to quickly pick up on topics that are created
|
||||||
|
// after the MM2 nodes are brought up during testing
|
||||||
|
put("refresh.topics.interval.seconds", "1");
|
||||||
|
put("clusters", String.join(", ", a, b));
|
||||||
|
put(a + ".bootstrap.servers", clusterA.bootstrapServers());
|
||||||
|
put(b + ".bootstrap.servers", clusterB.bootstrapServers());
|
||||||
|
// Enable exactly-once support to both validate that MirrorMaker can run with
|
||||||
|
// that feature turned on, and to force cross-worker communication before
|
||||||
|
// task startup
|
||||||
|
put(a + ".exactly.once.source.support", "enabled");
|
||||||
|
put(ab + ".enabled", "true");
|
||||||
|
put(ab + ".topics", "^" + testTopicPrefix + ".*");
|
||||||
|
// The name of the offset syncs topic will contain the name of the cluster in
|
||||||
|
// the replication flow that it is _not_ hosted on; create the offset syncs topic
|
||||||
|
// on the target cluster so that its name will contain the source cluster's name
|
||||||
|
// (since the target cluster's name contains characters that are not valid for
|
||||||
|
// use in a topic name)
|
||||||
|
put(ab + ".offset-syncs.topic.location", "target");
|
||||||
|
// Disable b -> a (and heartbeats from it) so that no topics are created that use
|
||||||
|
// the target cluster's name
|
||||||
|
put(ba + ".enabled", "false");
|
||||||
|
put(ba + ".emit.heartbeats.enabled", "false");
|
||||||
|
put("replication.factor", "1");
|
||||||
|
put("checkpoints.topic.replication.factor", "1");
|
||||||
|
put("heartbeats.topic.replication.factor", "1");
|
||||||
|
put("offset-syncs.topic.replication.factor", "1");
|
||||||
|
put("offset.storage.replication.factor", "1");
|
||||||
|
put("status.storage.replication.factor", "1");
|
||||||
|
put("config.storage.replication.factor", "1");
|
||||||
|
}};
|
||||||
|
|
||||||
|
// Bring up a three-node cluster
|
||||||
|
final int numNodes = 3;
|
||||||
|
for (int i = 0; i < numNodes; i++) {
|
||||||
|
startMirrorMaker("node " + i, mmProps);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create one topic per Kafka cluster per MirrorMaker node
|
||||||
|
final int topicsPerCluster = numNodes;
|
||||||
|
final int messagesPerTopic = 10;
|
||||||
|
for (int i = 0; i < topicsPerCluster; i++) {
|
||||||
|
String topic = testTopicPrefix + i;
|
||||||
|
|
||||||
|
// Create the topic on cluster A
|
||||||
|
createTopic(adminA, topic);
|
||||||
|
// and wait for MirrorMaker to create it on cluster B
|
||||||
|
awaitTopicCreation(b, adminB, a + "." + topic);
|
||||||
|
|
||||||
|
// Write data to the topic on cluster A
|
||||||
|
writeToTopic(clusterA, topic, messagesPerTopic);
|
||||||
|
// and wait for MirrorMaker to copy it to cluster B
|
||||||
|
awaitTopicContent(clusterB, b, a + "." + topic, messagesPerTopic);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void createTopic(Admin admin, String name) throws Exception {
|
||||||
|
admin.createTopics(Collections.singleton(new NewTopic(name, 1, (short) 1))).all().get();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void awaitTopicCreation(String clusterName, Admin admin, String topic) throws Exception {
|
||||||
|
waitForCondition(
|
||||||
|
() -> {
|
||||||
|
try {
|
||||||
|
Set<String> allTopics = admin.listTopics().names().get();
|
||||||
|
return allTopics.contains(topic);
|
||||||
|
} catch (Exception e) {
|
||||||
|
log.debug("Failed to check for existence of topic {} on cluster {}", topic, clusterName, e);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
},
|
||||||
|
TOPIC_CREATION_TIMEOUT_MS,
|
||||||
|
"topic " + topic + " was not created on cluster " + clusterName + " in time"
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void writeToTopic(EmbeddedKafkaCluster cluster, String topic, int numMessages) {
|
||||||
|
for (int i = 0; i <= numMessages; i++) {
|
||||||
|
cluster.produce(topic, Integer.toString(i));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void awaitTopicContent(EmbeddedKafkaCluster cluster, String clusterName, String topic, int numMessages) throws Exception {
|
||||||
|
try (Consumer<?, ?> consumer = cluster.createConsumer(Collections.singletonMap(AUTO_OFFSET_RESET_CONFIG, "earliest"))) {
|
||||||
|
consumer.subscribe(Collections.singleton(topic));
|
||||||
|
AtomicInteger messagesRead = new AtomicInteger(0);
|
||||||
|
waitForCondition(
|
||||||
|
() -> {
|
||||||
|
ConsumerRecords<?, ?> records = consumer.poll(Duration.ofSeconds(1));
|
||||||
|
return messagesRead.addAndGet(records.count()) >= numMessages;
|
||||||
|
},
|
||||||
|
TOPIC_REPLICATION_TIMEOUT_MS,
|
||||||
|
() -> "could not read " + numMessages + " from topic " + topic + " on cluster " + clusterName + " in time; only read " + messagesRead.get()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -25,6 +25,7 @@ import org.apache.kafka.connect.runtime.Herder;
|
||||||
import org.apache.kafka.connect.runtime.WorkerConfig;
|
import org.apache.kafka.connect.runtime.WorkerConfig;
|
||||||
import org.apache.kafka.connect.runtime.WorkerInfo;
|
import org.apache.kafka.connect.runtime.WorkerInfo;
|
||||||
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.ConnectRestServer;
|
||||||
import org.apache.kafka.connect.runtime.rest.RestClient;
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
import org.apache.kafka.connect.runtime.rest.RestServer;
|
import org.apache.kafka.connect.runtime.rest.RestServer;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -124,7 +125,7 @@ public abstract class AbstractConnectCli<T extends WorkerConfig> {
|
||||||
|
|
||||||
RestClient restClient = new RestClient(config);
|
RestClient restClient = new RestClient(config);
|
||||||
|
|
||||||
RestServer restServer = new RestServer(config, restClient);
|
ConnectRestServer restServer = new ConnectRestServer(config.rebalanceTimeout(), restClient, workerProps);
|
||||||
restServer.initializeServer();
|
restServer.initializeServer();
|
||||||
|
|
||||||
URI advertisedUrl = restServer.advertisedUrl();
|
URI advertisedUrl = restServer.advertisedUrl();
|
||||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.kafka.connect.util.SharedTopicAdmin;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -97,7 +98,8 @@ public class ConnectDistributed extends AbstractConnectCli<DistributedConfig> {
|
||||||
// herder is stopped. This is easier than having to track and own the lifecycle ourselves.
|
// herder is stopped. This is easier than having to track and own the lifecycle ourselves.
|
||||||
return new DistributedHerder(config, Time.SYSTEM, worker,
|
return new DistributedHerder(config, Time.SYSTEM, worker,
|
||||||
kafkaClusterId, statusBackingStore, configBackingStore,
|
kafkaClusterId, statusBackingStore, configBackingStore,
|
||||||
restServer.advertisedUrl().toString(), restClient, connectorClientConfigOverridePolicy, sharedAdmin);
|
restServer.advertisedUrl().toString(), restClient, connectorClientConfigOverridePolicy,
|
||||||
|
Collections.emptyList(), sharedAdmin);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.kafka.connect.runtime;
|
package org.apache.kafka.connect.runtime;
|
||||||
|
|
||||||
import org.apache.kafka.common.utils.Exit;
|
import org.apache.kafka.common.utils.Exit;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.ConnectRestServer;
|
||||||
import org.apache.kafka.connect.runtime.rest.RestServer;
|
import org.apache.kafka.connect.runtime.rest.RestServer;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -32,13 +33,13 @@ public class Connect {
|
||||||
private static final Logger log = LoggerFactory.getLogger(Connect.class);
|
private static final Logger log = LoggerFactory.getLogger(Connect.class);
|
||||||
|
|
||||||
private final Herder herder;
|
private final Herder herder;
|
||||||
private final RestServer rest;
|
private final ConnectRestServer rest;
|
||||||
private final CountDownLatch startLatch = new CountDownLatch(1);
|
private final CountDownLatch startLatch = new CountDownLatch(1);
|
||||||
private final CountDownLatch stopLatch = new CountDownLatch(1);
|
private final CountDownLatch stopLatch = new CountDownLatch(1);
|
||||||
private final AtomicBoolean shutdown = new AtomicBoolean(false);
|
private final AtomicBoolean shutdown = new AtomicBoolean(false);
|
||||||
private final ShutdownHook shutdownHook;
|
private final ShutdownHook shutdownHook;
|
||||||
|
|
||||||
public Connect(Herder herder, RestServer rest) {
|
public Connect(Herder herder, ConnectRestServer rest) {
|
||||||
log.debug("Kafka Connect instance created");
|
log.debug("Kafka Connect instance created");
|
||||||
this.herder = herder;
|
this.herder = herder;
|
||||||
this.rest = rest;
|
this.rest = rest;
|
||||||
|
|
|
@ -24,19 +24,15 @@ import org.apache.kafka.common.config.AbstractConfig;
|
||||||
import org.apache.kafka.common.config.ConfigDef;
|
import org.apache.kafka.common.config.ConfigDef;
|
||||||
import org.apache.kafka.common.config.ConfigDef.Importance;
|
import org.apache.kafka.common.config.ConfigDef.Importance;
|
||||||
import org.apache.kafka.common.config.ConfigDef.Type;
|
import org.apache.kafka.common.config.ConfigDef.Type;
|
||||||
import org.apache.kafka.common.config.ConfigException;
|
|
||||||
import org.apache.kafka.common.config.SslClientAuth;
|
|
||||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
|
||||||
import org.apache.kafka.common.metrics.Sensor;
|
import org.apache.kafka.common.metrics.Sensor;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
|
||||||
import org.apache.kafka.connect.errors.ConnectException;
|
import org.apache.kafka.connect.errors.ConnectException;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestServerConfig;
|
||||||
import org.apache.kafka.connect.storage.SimpleHeaderConverter;
|
import org.apache.kafka.connect.storage.SimpleHeaderConverter;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -44,8 +40,6 @@ import java.util.Objects;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
import org.eclipse.jetty.util.StringUtil;
|
|
||||||
|
|
||||||
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
|
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
|
||||||
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
|
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
|
||||||
import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TOPIC_CREATION_PREFIX;
|
import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TOPIC_CREATION_PREFIX;
|
||||||
|
@ -57,9 +51,6 @@ public class WorkerConfig extends AbstractConfig {
|
||||||
private static final Logger log = LoggerFactory.getLogger(WorkerConfig.class);
|
private static final Logger log = LoggerFactory.getLogger(WorkerConfig.class);
|
||||||
|
|
||||||
private static final Pattern COMMA_WITH_WHITESPACE = Pattern.compile("\\s*,\\s*");
|
private static final Pattern COMMA_WITH_WHITESPACE = Pattern.compile("\\s*,\\s*");
|
||||||
private static final Collection<String> HEADER_ACTIONS = Collections.unmodifiableList(
|
|
||||||
Arrays.asList("set", "add", "setDate", "addDate")
|
|
||||||
);
|
|
||||||
|
|
||||||
public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
|
public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
|
||||||
public static final String BOOTSTRAP_SERVERS_DOC
|
public static final String BOOTSTRAP_SERVERS_DOC
|
||||||
|
@ -119,47 +110,6 @@ public class WorkerConfig extends AbstractConfig {
|
||||||
+ "running with exactly-once support.";
|
+ "running with exactly-once support.";
|
||||||
public static final long OFFSET_COMMIT_TIMEOUT_MS_DEFAULT = 5000L;
|
public static final long OFFSET_COMMIT_TIMEOUT_MS_DEFAULT = 5000L;
|
||||||
|
|
||||||
public static final String LISTENERS_CONFIG = "listeners";
|
|
||||||
private static final String LISTENERS_DOC
|
|
||||||
= "List of comma-separated URIs the REST API will listen on. The supported protocols are HTTP and HTTPS.\n" +
|
|
||||||
" Specify hostname as 0.0.0.0 to bind to all interfaces.\n" +
|
|
||||||
" Leave hostname empty to bind to default interface.\n" +
|
|
||||||
" Examples of legal listener lists: HTTP://myhost:8083,HTTPS://myhost:8084";
|
|
||||||
static final List<String> LISTENERS_DEFAULT = Collections.singletonList("http://:8083");
|
|
||||||
|
|
||||||
public static final String REST_ADVERTISED_HOST_NAME_CONFIG = "rest.advertised.host.name";
|
|
||||||
private static final String REST_ADVERTISED_HOST_NAME_DOC
|
|
||||||
= "If this is set, this is the hostname that will be given out to other workers to connect to.";
|
|
||||||
|
|
||||||
public static final String REST_ADVERTISED_PORT_CONFIG = "rest.advertised.port";
|
|
||||||
private static final String REST_ADVERTISED_PORT_DOC
|
|
||||||
= "If this is set, this is the port that will be given out to other workers to connect to.";
|
|
||||||
|
|
||||||
public static final String REST_ADVERTISED_LISTENER_CONFIG = "rest.advertised.listener";
|
|
||||||
private static final String REST_ADVERTISED_LISTENER_DOC
|
|
||||||
= "Sets the advertised listener (HTTP or HTTPS) which will be given to other workers to use.";
|
|
||||||
|
|
||||||
public static final String ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG = "access.control.allow.origin";
|
|
||||||
protected static final String ACCESS_CONTROL_ALLOW_ORIGIN_DOC =
|
|
||||||
"Value to set the Access-Control-Allow-Origin header to for REST API requests." +
|
|
||||||
"To enable cross origin access, set this to the domain of the application that should be permitted" +
|
|
||||||
" to access the API, or '*' to allow access from any domain. The default value only allows access" +
|
|
||||||
" from the domain of the REST API.";
|
|
||||||
protected static final String ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT = "";
|
|
||||||
|
|
||||||
public static final String ACCESS_CONTROL_ALLOW_METHODS_CONFIG = "access.control.allow.methods";
|
|
||||||
protected static final String ACCESS_CONTROL_ALLOW_METHODS_DOC =
|
|
||||||
"Sets the methods supported for cross origin requests by setting the Access-Control-Allow-Methods header. "
|
|
||||||
+ "The default value of the Access-Control-Allow-Methods header allows cross origin requests for GET, POST and HEAD.";
|
|
||||||
protected static final String ACCESS_CONTROL_ALLOW_METHODS_DEFAULT = "";
|
|
||||||
|
|
||||||
public static final String ADMIN_LISTENERS_CONFIG = "admin.listeners";
|
|
||||||
protected static final String ADMIN_LISTENERS_DOC = "List of comma-separated URIs the Admin REST API will listen on." +
|
|
||||||
" The supported protocols are HTTP and HTTPS." +
|
|
||||||
" An empty or blank string will disable this feature." +
|
|
||||||
" The default behavior is to use the regular listener (specified by the 'listeners' property).";
|
|
||||||
public static final String ADMIN_LISTENERS_HTTPS_CONFIGS_PREFIX = "admin.listeners.https.";
|
|
||||||
|
|
||||||
public static final String PLUGIN_PATH_CONFIG = "plugin.path";
|
public static final String PLUGIN_PATH_CONFIG = "plugin.path";
|
||||||
protected static final String PLUGIN_PATH_DOC = "List of paths separated by commas (,) that "
|
protected static final String PLUGIN_PATH_DOC = "List of paths separated by commas (,) that "
|
||||||
+ "contain plugins (connectors, converters, transformations). The list should consist"
|
+ "contain plugins (connectors, converters, transformations). The list should consist"
|
||||||
|
@ -182,13 +132,6 @@ public class WorkerConfig extends AbstractConfig {
|
||||||
+ "<code>ConfigProvider</code> allows you to replace variable references in connector configurations, "
|
+ "<code>ConfigProvider</code> allows you to replace variable references in connector configurations, "
|
||||||
+ "such as for externalized secrets. ";
|
+ "such as for externalized secrets. ";
|
||||||
|
|
||||||
public static final String REST_EXTENSION_CLASSES_CONFIG = "rest.extension.classes";
|
|
||||||
protected static final String REST_EXTENSION_CLASSES_DOC =
|
|
||||||
"Comma-separated names of <code>ConnectRestExtension</code> classes, loaded and called "
|
|
||||||
+ "in the order specified. Implementing the interface "
|
|
||||||
+ "<code>ConnectRestExtension</code> allows you to inject into Connect's REST API user defined resources like filters. "
|
|
||||||
+ "Typically used to add custom capability like logging, security, etc. ";
|
|
||||||
|
|
||||||
public static final String CONNECTOR_CLIENT_POLICY_CLASS_CONFIG = "connector.client.config.override.policy";
|
public static final String CONNECTOR_CLIENT_POLICY_CLASS_CONFIG = "connector.client.config.override.policy";
|
||||||
public static final String CONNECTOR_CLIENT_POLICY_CLASS_DOC =
|
public static final String CONNECTOR_CLIENT_POLICY_CLASS_DOC =
|
||||||
"Class name or alias of implementation of <code>ConnectorClientConfigOverridePolicy</code>. Defines what client configurations can be "
|
"Class name or alias of implementation of <code>ConnectorClientConfigOverridePolicy</code>. Defines what client configurations can be "
|
||||||
|
@ -227,17 +170,13 @@ public class WorkerConfig extends AbstractConfig {
|
||||||
+ "to create topics automatically.";
|
+ "to create topics automatically.";
|
||||||
protected static final boolean TOPIC_CREATION_ENABLE_DEFAULT = true;
|
protected static final boolean TOPIC_CREATION_ENABLE_DEFAULT = true;
|
||||||
|
|
||||||
public static final String RESPONSE_HTTP_HEADERS_CONFIG = "response.http.headers.config";
|
|
||||||
protected static final String RESPONSE_HTTP_HEADERS_DOC = "Rules for REST API HTTP response headers";
|
|
||||||
protected static final String RESPONSE_HTTP_HEADERS_DEFAULT = "";
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a basic ConfigDef for a WorkerConfig. This includes all the common settings. Subclasses can use this to
|
* Get a basic ConfigDef for a WorkerConfig. This includes all the common settings. Subclasses can use this to
|
||||||
* bootstrap their own ConfigDef.
|
* bootstrap their own ConfigDef.
|
||||||
* @return a ConfigDef with all the common options specified
|
* @return a ConfigDef with all the common options specified
|
||||||
*/
|
*/
|
||||||
protected static ConfigDef baseConfigDef() {
|
protected static ConfigDef baseConfigDef() {
|
||||||
return new ConfigDef()
|
ConfigDef result = new ConfigDef()
|
||||||
.define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, BOOTSTRAP_SERVERS_DEFAULT,
|
.define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, BOOTSTRAP_SERVERS_DEFAULT,
|
||||||
Importance.HIGH, BOOTSTRAP_SERVERS_DOC)
|
Importance.HIGH, BOOTSTRAP_SERVERS_DOC)
|
||||||
.define(CLIENT_DNS_LOOKUP_CONFIG,
|
.define(CLIENT_DNS_LOOKUP_CONFIG,
|
||||||
|
@ -258,16 +197,6 @@ public class WorkerConfig extends AbstractConfig {
|
||||||
Importance.LOW, OFFSET_COMMIT_INTERVAL_MS_DOC)
|
Importance.LOW, OFFSET_COMMIT_INTERVAL_MS_DOC)
|
||||||
.define(OFFSET_COMMIT_TIMEOUT_MS_CONFIG, Type.LONG, OFFSET_COMMIT_TIMEOUT_MS_DEFAULT,
|
.define(OFFSET_COMMIT_TIMEOUT_MS_CONFIG, Type.LONG, OFFSET_COMMIT_TIMEOUT_MS_DEFAULT,
|
||||||
Importance.LOW, OFFSET_COMMIT_TIMEOUT_MS_DOC)
|
Importance.LOW, OFFSET_COMMIT_TIMEOUT_MS_DOC)
|
||||||
.define(LISTENERS_CONFIG, Type.LIST, LISTENERS_DEFAULT, new ListenersValidator(), Importance.LOW, LISTENERS_DOC)
|
|
||||||
.define(REST_ADVERTISED_HOST_NAME_CONFIG, Type.STRING, null, Importance.LOW, REST_ADVERTISED_HOST_NAME_DOC)
|
|
||||||
.define(REST_ADVERTISED_PORT_CONFIG, Type.INT, null, Importance.LOW, REST_ADVERTISED_PORT_DOC)
|
|
||||||
.define(REST_ADVERTISED_LISTENER_CONFIG, Type.STRING, null, Importance.LOW, REST_ADVERTISED_LISTENER_DOC)
|
|
||||||
.define(ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, Type.STRING,
|
|
||||||
ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT, Importance.LOW,
|
|
||||||
ACCESS_CONTROL_ALLOW_ORIGIN_DOC)
|
|
||||||
.define(ACCESS_CONTROL_ALLOW_METHODS_CONFIG, Type.STRING,
|
|
||||||
ACCESS_CONTROL_ALLOW_METHODS_DEFAULT, Importance.LOW,
|
|
||||||
ACCESS_CONTROL_ALLOW_METHODS_DOC)
|
|
||||||
.define(PLUGIN_PATH_CONFIG,
|
.define(PLUGIN_PATH_CONFIG,
|
||||||
Type.LIST,
|
Type.LIST,
|
||||||
null,
|
null,
|
||||||
|
@ -292,30 +221,37 @@ public class WorkerConfig extends AbstractConfig {
|
||||||
true,
|
true,
|
||||||
Importance.LOW,
|
Importance.LOW,
|
||||||
CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_DOC)
|
CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_DOC)
|
||||||
.define(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG,
|
|
||||||
ConfigDef.Type.STRING, SslClientAuth.NONE.toString(), in(Utils.enumOptions(SslClientAuth.class)), ConfigDef.Importance.LOW, BrokerSecurityConfigs.SSL_CLIENT_AUTH_DOC)
|
|
||||||
.define(HEADER_CONVERTER_CLASS_CONFIG, Type.CLASS,
|
.define(HEADER_CONVERTER_CLASS_CONFIG, Type.CLASS,
|
||||||
HEADER_CONVERTER_CLASS_DEFAULT,
|
HEADER_CONVERTER_CLASS_DEFAULT,
|
||||||
Importance.LOW, HEADER_CONVERTER_CLASS_DOC)
|
Importance.LOW, HEADER_CONVERTER_CLASS_DOC)
|
||||||
.define(CONFIG_PROVIDERS_CONFIG, Type.LIST,
|
.define(CONFIG_PROVIDERS_CONFIG, Type.LIST,
|
||||||
Collections.emptyList(),
|
Collections.emptyList(),
|
||||||
Importance.LOW, CONFIG_PROVIDERS_DOC)
|
Importance.LOW, CONFIG_PROVIDERS_DOC)
|
||||||
.define(REST_EXTENSION_CLASSES_CONFIG, Type.LIST, "",
|
|
||||||
Importance.LOW, REST_EXTENSION_CLASSES_DOC)
|
|
||||||
.define(ADMIN_LISTENERS_CONFIG, Type.LIST, null,
|
|
||||||
new AdminListenersValidator(), Importance.LOW, ADMIN_LISTENERS_DOC)
|
|
||||||
.define(CONNECTOR_CLIENT_POLICY_CLASS_CONFIG, Type.STRING, CONNECTOR_CLIENT_POLICY_CLASS_DEFAULT,
|
.define(CONNECTOR_CLIENT_POLICY_CLASS_CONFIG, Type.STRING, CONNECTOR_CLIENT_POLICY_CLASS_DEFAULT,
|
||||||
Importance.MEDIUM, CONNECTOR_CLIENT_POLICY_CLASS_DOC)
|
Importance.MEDIUM, CONNECTOR_CLIENT_POLICY_CLASS_DOC)
|
||||||
.define(TOPIC_TRACKING_ENABLE_CONFIG, Type.BOOLEAN, TOPIC_TRACKING_ENABLE_DEFAULT,
|
|
||||||
Importance.LOW, TOPIC_TRACKING_ENABLE_DOC)
|
|
||||||
.define(TOPIC_TRACKING_ALLOW_RESET_CONFIG, Type.BOOLEAN, TOPIC_TRACKING_ALLOW_RESET_DEFAULT,
|
|
||||||
Importance.LOW, TOPIC_TRACKING_ALLOW_RESET_DOC)
|
|
||||||
.define(TOPIC_CREATION_ENABLE_CONFIG, Type.BOOLEAN, TOPIC_CREATION_ENABLE_DEFAULT, Importance.LOW,
|
.define(TOPIC_CREATION_ENABLE_CONFIG, Type.BOOLEAN, TOPIC_CREATION_ENABLE_DEFAULT, Importance.LOW,
|
||||||
TOPIC_CREATION_ENABLE_DOC)
|
TOPIC_CREATION_ENABLE_DOC)
|
||||||
.define(RESPONSE_HTTP_HEADERS_CONFIG, Type.STRING, RESPONSE_HTTP_HEADERS_DEFAULT,
|
|
||||||
new ResponseHttpHeadersValidator(), Importance.LOW, RESPONSE_HTTP_HEADERS_DOC)
|
|
||||||
// security support
|
// security support
|
||||||
.withClientSslSupport();
|
.withClientSslSupport();
|
||||||
|
addTopicTrackingConfig(result);
|
||||||
|
RestServerConfig.addPublicConfig(result);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void addTopicTrackingConfig(ConfigDef configDef) {
|
||||||
|
configDef
|
||||||
|
.define(
|
||||||
|
TOPIC_TRACKING_ENABLE_CONFIG,
|
||||||
|
ConfigDef.Type.BOOLEAN,
|
||||||
|
TOPIC_TRACKING_ENABLE_DEFAULT,
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
TOPIC_TRACKING_ENABLE_DOC
|
||||||
|
).define(
|
||||||
|
TOPIC_TRACKING_ALLOW_RESET_CONFIG,
|
||||||
|
ConfigDef.Type.BOOLEAN,
|
||||||
|
TOPIC_TRACKING_ALLOW_RESET_DEFAULT,
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
TOPIC_TRACKING_ALLOW_RESET_DOC);
|
||||||
}
|
}
|
||||||
|
|
||||||
private String kafkaClusterId;
|
private String kafkaClusterId;
|
||||||
|
@ -396,7 +332,7 @@ public class WorkerConfig extends AbstractConfig {
|
||||||
return String.join(",", getList(BOOTSTRAP_SERVERS_CONFIG));
|
return String.join(",", getList(BOOTSTRAP_SERVERS_CONFIG));
|
||||||
}
|
}
|
||||||
|
|
||||||
public Integer getRebalanceTimeout() {
|
public Integer rebalanceTimeout() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -477,125 +413,4 @@ public class WorkerConfig extends AbstractConfig {
|
||||||
logPluginPathConfigProviderWarning(props);
|
logPluginPathConfigProviderWarning(props);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Visible for testing
|
|
||||||
static void validateHttpResponseHeaderConfig(String config) {
|
|
||||||
try {
|
|
||||||
// validate format
|
|
||||||
String[] configTokens = config.trim().split("\\s+", 2);
|
|
||||||
if (configTokens.length != 2) {
|
|
||||||
throw new ConfigException(String.format("Invalid format of header config '%s'. "
|
|
||||||
+ "Expected: '[action] [header name]:[header value]'", config));
|
|
||||||
}
|
|
||||||
|
|
||||||
// validate action
|
|
||||||
String method = configTokens[0].trim();
|
|
||||||
validateHeaderConfigAction(method);
|
|
||||||
|
|
||||||
// validate header name and header value pair
|
|
||||||
String header = configTokens[1];
|
|
||||||
String[] headerTokens = header.trim().split(":");
|
|
||||||
if (headerTokens.length != 2) {
|
|
||||||
throw new ConfigException(
|
|
||||||
String.format("Invalid format of header name and header value pair '%s'. "
|
|
||||||
+ "Expected: '[header name]:[header value]'", header));
|
|
||||||
}
|
|
||||||
|
|
||||||
// validate header name
|
|
||||||
String headerName = headerTokens[0].trim();
|
|
||||||
if (headerName.isEmpty() || headerName.matches(".*\\s+.*")) {
|
|
||||||
throw new ConfigException(String.format("Invalid header name '%s'. "
|
|
||||||
+ "The '[header name]' cannot contain whitespace", headerName));
|
|
||||||
}
|
|
||||||
} catch (ArrayIndexOutOfBoundsException e) {
|
|
||||||
throw new ConfigException(String.format("Invalid header config '%s'.", config), e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Visible for testing
|
|
||||||
static void validateHeaderConfigAction(String action) {
|
|
||||||
if (HEADER_ACTIONS.stream().noneMatch(action::equalsIgnoreCase)) {
|
|
||||||
throw new ConfigException(String.format("Invalid header config action: '%s'. "
|
|
||||||
+ "Expected one of %s", action, HEADER_ACTIONS));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class ListenersValidator implements ConfigDef.Validator {
|
|
||||||
@Override
|
|
||||||
public void ensureValid(String name, Object value) {
|
|
||||||
if (!(value instanceof List)) {
|
|
||||||
throw new ConfigException("Invalid value type for listeners (expected list of URLs , ex: http://localhost:8080,https://localhost:8443).");
|
|
||||||
}
|
|
||||||
|
|
||||||
List<?> items = (List<?>) value;
|
|
||||||
if (items.isEmpty()) {
|
|
||||||
throw new ConfigException("Invalid value for listeners, at least one URL is expected, ex: http://localhost:8080,https://localhost:8443.");
|
|
||||||
}
|
|
||||||
|
|
||||||
for (Object item : items) {
|
|
||||||
if (!(item instanceof String)) {
|
|
||||||
throw new ConfigException("Invalid type for listeners (expected String).");
|
|
||||||
}
|
|
||||||
if (Utils.isBlank((String) item)) {
|
|
||||||
throw new ConfigException("Empty URL found when parsing listeners list.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "List of comma-separated URLs, ex: http://localhost:8080,https://localhost:8443.";
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class AdminListenersValidator implements ConfigDef.Validator {
|
|
||||||
@Override
|
|
||||||
public void ensureValid(String name, Object value) {
|
|
||||||
if (value == null) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!(value instanceof List)) {
|
|
||||||
throw new ConfigException("Invalid value type for admin.listeners (expected list).");
|
|
||||||
}
|
|
||||||
|
|
||||||
List<?> items = (List<?>) value;
|
|
||||||
if (items.isEmpty()) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
for (Object item : items) {
|
|
||||||
if (!(item instanceof String)) {
|
|
||||||
throw new ConfigException("Invalid type for admin.listeners (expected String).");
|
|
||||||
}
|
|
||||||
if (Utils.isBlank((String) item)) {
|
|
||||||
throw new ConfigException("Empty URL found when parsing admin.listeners list.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "List of comma-separated URLs, ex: http://localhost:8080,https://localhost:8443.";
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class ResponseHttpHeadersValidator implements ConfigDef.Validator {
|
|
||||||
@Override
|
|
||||||
public void ensureValid(String name, Object value) {
|
|
||||||
String strValue = (String) value;
|
|
||||||
if (Utils.isBlank(strValue)) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
String[] configs = StringUtil.csvSplit(strValue); // handles and removed surrounding quotes
|
|
||||||
Arrays.stream(configs).forEach(WorkerConfig::validateHttpResponseHeaderConfig);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return "Comma-separated header rules, where each header rule is of the form "
|
|
||||||
+ "'[action] [header name]:[header value]' and optionally surrounded by double quotes "
|
|
||||||
+ "if any part of a header rule contains a comma";
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -509,7 +509,7 @@ public class DistributedConfig extends WorkerConfig {
|
||||||
private final ExactlyOnceSourceSupport exactlyOnceSourceSupport;
|
private final ExactlyOnceSourceSupport exactlyOnceSourceSupport;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Integer getRebalanceTimeout() {
|
public Integer rebalanceTimeout() {
|
||||||
return getInt(DistributedConfig.REBALANCE_TIMEOUT_MS_CONFIG);
|
return getInt(DistributedConfig.REBALANCE_TIMEOUT_MS_CONFIG);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -199,6 +199,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
private Set<String> connectorTargetStateChanges = new HashSet<>();
|
private Set<String> connectorTargetStateChanges = new HashSet<>();
|
||||||
// Access to this map is protected by the herder's monitor
|
// Access to this map is protected by the herder's monitor
|
||||||
private final Map<String, ZombieFencing> activeZombieFencings = new HashMap<>();
|
private final Map<String, ZombieFencing> activeZombieFencings = new HashMap<>();
|
||||||
|
private final List<String> restNamespace;
|
||||||
private boolean needsReconfigRebalance;
|
private boolean needsReconfigRebalance;
|
||||||
private volatile boolean fencedFromConfigTopic;
|
private volatile boolean fencedFromConfigTopic;
|
||||||
private volatile int generation;
|
private volatile int generation;
|
||||||
|
@ -228,9 +229,13 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
* @param kafkaClusterId the identifier of the Kafka cluster to use for internal topics; may not be null
|
* @param kafkaClusterId the identifier of the Kafka cluster to use for internal topics; may not be null
|
||||||
* @param statusBackingStore the backing store for statuses; may not be null
|
* @param statusBackingStore the backing store for statuses; may not be null
|
||||||
* @param configBackingStore the backing store for connector configurations; may not be null
|
* @param configBackingStore the backing store for connector configurations; may not be null
|
||||||
* @param restUrl the URL of this herder's REST API; may not be null
|
* @param restUrl the URL of this herder's REST API; may not be null, but may be an arbitrary placeholder
|
||||||
|
* value if this worker does not expose a REST API
|
||||||
|
* @param restClient a REST client that can be used to issue requests to other workers in the cluster; may
|
||||||
|
* be null if inter-worker communication is not enabled
|
||||||
* @param connectorClientConfigOverridePolicy the policy specifying the client configuration properties that may be overridden
|
* @param connectorClientConfigOverridePolicy the policy specifying the client configuration properties that may be overridden
|
||||||
* in connector configurations; may not be null
|
* in connector configurations; may not be null
|
||||||
|
* @param restNamespace zero or more path elements to prepend to the paths of forwarded REST requests; may be empty, but not null
|
||||||
* @param uponShutdown any {@link AutoCloseable} objects that should be closed when this herder is {@link #stop() stopped},
|
* @param uponShutdown any {@link AutoCloseable} objects that should be closed when this herder is {@link #stop() stopped},
|
||||||
* after all services and resources owned by this herder are stopped
|
* after all services and resources owned by this herder are stopped
|
||||||
*/
|
*/
|
||||||
|
@ -243,10 +248,10 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
String restUrl,
|
String restUrl,
|
||||||
RestClient restClient,
|
RestClient restClient,
|
||||||
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
|
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
|
||||||
|
List<String> restNamespace,
|
||||||
AutoCloseable... uponShutdown) {
|
AutoCloseable... uponShutdown) {
|
||||||
this(config, worker, worker.workerId(), kafkaClusterId, statusBackingStore, configBackingStore,
|
this(config, worker, worker.workerId(), kafkaClusterId, statusBackingStore, configBackingStore, null, restUrl, restClient, worker.metrics(),
|
||||||
null, restUrl, restClient, worker.metrics(),
|
time, connectorClientConfigOverridePolicy, restNamespace, uponShutdown);
|
||||||
time, connectorClientConfigOverridePolicy, uponShutdown);
|
|
||||||
configBackingStore.setUpdateListener(new ConfigUpdateListener());
|
configBackingStore.setUpdateListener(new ConfigUpdateListener());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -263,6 +268,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
ConnectMetrics metrics,
|
ConnectMetrics metrics,
|
||||||
Time time,
|
Time time,
|
||||||
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
|
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy,
|
||||||
|
List<String> restNamespace,
|
||||||
AutoCloseable... uponShutdown) {
|
AutoCloseable... uponShutdown) {
|
||||||
super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy);
|
super(worker, workerId, kafkaClusterId, statusBackingStore, configBackingStore, connectorClientConfigOverridePolicy);
|
||||||
|
|
||||||
|
@ -277,6 +283,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
this.keyGenerator = config.getInternalRequestKeyGenerator();
|
this.keyGenerator = config.getInternalRequestKeyGenerator();
|
||||||
this.restClient = restClient;
|
this.restClient = restClient;
|
||||||
this.isTopicTrackingEnabled = config.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG);
|
this.isTopicTrackingEnabled = config.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG);
|
||||||
|
this.restNamespace = Objects.requireNonNull(restNamespace);
|
||||||
this.uponShutdown = Arrays.asList(uponShutdown);
|
this.uponShutdown = Arrays.asList(uponShutdown);
|
||||||
|
|
||||||
String clientIdConfig = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG);
|
String clientIdConfig = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG);
|
||||||
|
@ -1162,11 +1169,17 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
callback.onCompletion(null, null);
|
callback.onCompletion(null, null);
|
||||||
} else if (error instanceof NotLeaderException) {
|
} else if (error instanceof NotLeaderException) {
|
||||||
if (restClient != null) {
|
if (restClient != null) {
|
||||||
String forwardedUrl = ((NotLeaderException) error).forwardUrl() + "connectors/" + id.connector() + "/fence";
|
String workerUrl = ((NotLeaderException) error).forwardUrl();
|
||||||
log.trace("Forwarding zombie fencing request for connector {} to leader at {}", id.connector(), forwardedUrl);
|
String fenceUrl = namespacedUrl(workerUrl)
|
||||||
|
.path("connectors")
|
||||||
|
.path(id.connector())
|
||||||
|
.path("fence")
|
||||||
|
.build()
|
||||||
|
.toString();
|
||||||
|
log.trace("Forwarding zombie fencing request for connector {} to leader at {}", id.connector(), fenceUrl);
|
||||||
forwardRequestExecutor.execute(() -> {
|
forwardRequestExecutor.execute(() -> {
|
||||||
try {
|
try {
|
||||||
restClient.httpRequest(forwardedUrl, "PUT", null, null, null, sessionKey, requestSignatureAlgorithm);
|
restClient.httpRequest(fenceUrl, "PUT", null, null, null, sessionKey, requestSignatureAlgorithm);
|
||||||
callback.onCompletion(null, null);
|
callback.onCompletion(null, null);
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
callback.onCompletion(t, null);
|
callback.onCompletion(t, null);
|
||||||
|
@ -1175,12 +1188,10 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
} else {
|
} else {
|
||||||
callback.onCompletion(
|
callback.onCompletion(
|
||||||
new ConnectException(
|
new ConnectException(
|
||||||
// TODO: Update this message if KIP-710 is accepted and merged
|
|
||||||
// (https://cwiki.apache.org/confluence/display/KAFKA/KIP-710%3A+Full+support+for+distributed+mode+in+dedicated+MirrorMaker+2.0+clusters)
|
|
||||||
"This worker is not able to communicate with the leader of the cluster, "
|
"This worker is not able to communicate with the leader of the cluster, "
|
||||||
+ "which is required for exactly-once source tasks. If running MirrorMaker 2 "
|
+ "which is required for exactly-once source tasks. If running MirrorMaker 2 "
|
||||||
+ "in dedicated mode, consider either disabling exactly-once support, or deploying "
|
+ "in dedicated mode, consider enabling inter-worker communication via the "
|
||||||
+ "the connectors for MirrorMaker 2 directly onto a distributed Kafka Connect cluster."
|
+ "'dedicated.mode.enable.internal.rest' property."
|
||||||
),
|
),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
@ -1936,12 +1947,10 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
writeToConfigTopicAsLeader(() -> configBackingStore.putTaskConfigs(connName, rawTaskProps));
|
writeToConfigTopicAsLeader(() -> configBackingStore.putTaskConfigs(connName, rawTaskProps));
|
||||||
cb.onCompletion(null, null);
|
cb.onCompletion(null, null);
|
||||||
} else if (restClient == null) {
|
} else if (restClient == null) {
|
||||||
// TODO: Update this message if KIP-710 is accepted and merged
|
|
||||||
// (https://cwiki.apache.org/confluence/display/KAFKA/KIP-710%3A+Full+support+for+distributed+mode+in+dedicated+MirrorMaker+2.0+clusters)
|
|
||||||
throw new NotLeaderException("This worker is not able to communicate with the leader of the cluster, "
|
throw new NotLeaderException("This worker is not able to communicate with the leader of the cluster, "
|
||||||
+ "which is required for dynamically-reconfiguring connectors. If running MirrorMaker 2 "
|
+ "which is required for dynamically-reconfiguring connectors. If running MirrorMaker 2 "
|
||||||
+ "in dedicated mode, consider deploying the connectors for MirrorMaker 2 directly onto a "
|
+ "in dedicated mode, consider enabling inter-worker communication via the "
|
||||||
+ "distributed Kafka Connect cluster.",
|
+ "'dedicated.mode.enable.internal.rest' property.",
|
||||||
leaderUrl()
|
leaderUrl()
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
|
@ -1956,7 +1965,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
"because the URL of the leader's REST interface is empty!"), null);
|
"because the URL of the leader's REST interface is empty!"), null);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
String reconfigUrl = UriBuilder.fromUri(leaderUrl)
|
String reconfigUrl = namespacedUrl(leaderUrl)
|
||||||
.path("connectors")
|
.path("connectors")
|
||||||
.path(connName)
|
.path(connName)
|
||||||
.path("tasks")
|
.path("tasks")
|
||||||
|
@ -1965,6 +1974,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
log.trace("Forwarding task configurations for connector {} to leader", connName);
|
log.trace("Forwarding task configurations for connector {} to leader", connName);
|
||||||
restClient.httpRequest(reconfigUrl, "POST", null, rawTaskProps, null, sessionKey, requestSignatureAlgorithm);
|
restClient.httpRequest(reconfigUrl, "POST", null, rawTaskProps, null, sessionKey, requestSignatureAlgorithm);
|
||||||
cb.onCompletion(null, null);
|
cb.onCompletion(null, null);
|
||||||
|
log.trace("Request to leader to reconfigure connector tasks succeeded");
|
||||||
} catch (ConnectException e) {
|
} catch (ConnectException e) {
|
||||||
log.error("Request to leader to reconfigure connector tasks failed", e);
|
log.error("Request to leader to reconfigure connector tasks failed", e);
|
||||||
cb.onCompletion(e, null);
|
cb.onCompletion(e, null);
|
||||||
|
@ -2460,6 +2470,14 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private UriBuilder namespacedUrl(String workerUrl) {
|
||||||
|
UriBuilder result = UriBuilder.fromUri(workerUrl);
|
||||||
|
for (String namespacePath : restNamespace) {
|
||||||
|
result = result.path(namespacePath);
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Represents an active zombie fencing: that is, an in-progress attempt to invoke
|
* Represents an active zombie fencing: that is, an in-progress attempt to invoke
|
||||||
* {@link Worker#fenceZombies(String, int, Map)} and then, if successful, write a new task count
|
* {@link Worker#fenceZombies(String, int, Map)} and then, if successful, write a new task count
|
||||||
|
|
|
@ -0,0 +1,69 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.resources.ConnectResource;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.resources.ConnectorPluginsResource;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.resources.ConnectorsResource;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.resources.InternalConnectResource;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.resources.LoggingResource;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.resources.RootResource;
|
||||||
|
import org.glassfish.jersey.server.ResourceConfig;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class ConnectRestServer extends RestServer {
|
||||||
|
|
||||||
|
private final RestClient restClient;
|
||||||
|
private Herder herder;
|
||||||
|
|
||||||
|
public ConnectRestServer(Integer rebalanceTimeoutMs, RestClient restClient, Map<?, ?> props) {
|
||||||
|
super(RestServerConfig.forPublic(rebalanceTimeoutMs, props));
|
||||||
|
this.restClient = restClient;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void initializeResources(Herder herder) {
|
||||||
|
this.herder = herder;
|
||||||
|
super.initializeResources();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Collection<ConnectResource> regularResources() {
|
||||||
|
return Arrays.asList(
|
||||||
|
new RootResource(herder),
|
||||||
|
new ConnectorsResource(herder, config, restClient),
|
||||||
|
new InternalConnectResource(herder, restClient),
|
||||||
|
new ConnectorPluginsResource(herder)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Collection<ConnectResource> adminResources() {
|
||||||
|
return Arrays.asList(
|
||||||
|
new LoggingResource()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configureRegularResources(ResourceConfig resourceConfig) {
|
||||||
|
registerRestExtensions(herder, resourceConfig);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,142 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
|
import org.apache.kafka.connect.runtime.distributed.RebalanceNeededException;
|
||||||
|
import org.apache.kafka.connect.runtime.distributed.RequestTargetException;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
|
||||||
|
import org.apache.kafka.connect.util.FutureCallback;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import javax.ws.rs.core.HttpHeaders;
|
||||||
|
import javax.ws.rs.core.Response;
|
||||||
|
import javax.ws.rs.core.UriBuilder;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
|
public class HerderRequestHandler {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(HerderRequestHandler.class);
|
||||||
|
|
||||||
|
private final RestClient restClient;
|
||||||
|
|
||||||
|
private long requestTimeoutMs;
|
||||||
|
|
||||||
|
public HerderRequestHandler(RestClient restClient, long requestTimeoutMs) {
|
||||||
|
this.restClient = restClient;
|
||||||
|
this.requestTimeoutMs = requestTimeoutMs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void requestTimeoutMs(long requestTimeoutMs) {
|
||||||
|
if (requestTimeoutMs < 1) {
|
||||||
|
throw new IllegalArgumentException("REST request timeout must be positive");
|
||||||
|
}
|
||||||
|
this.requestTimeoutMs = requestTimeoutMs;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wait for a FutureCallback to complete. If it succeeds, return the parsed response. If it fails, try to forward the
|
||||||
|
* request to the leader.
|
||||||
|
*/
|
||||||
|
public <T, U> T completeOrForwardRequest(FutureCallback<T> cb,
|
||||||
|
String path,
|
||||||
|
String method,
|
||||||
|
HttpHeaders headers,
|
||||||
|
Map<String, String> queryParameters,
|
||||||
|
Object body,
|
||||||
|
TypeReference<U> resultType,
|
||||||
|
Translator<T, U> translator,
|
||||||
|
Boolean forward) throws Throwable {
|
||||||
|
try {
|
||||||
|
return cb.get(requestTimeoutMs, TimeUnit.MILLISECONDS);
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
Throwable cause = e.getCause();
|
||||||
|
|
||||||
|
if (cause instanceof RequestTargetException) {
|
||||||
|
if (forward == null || forward) {
|
||||||
|
// the only time we allow recursive forwarding is when no forward flag has
|
||||||
|
// been set, which should only be seen by the first worker to handle a user request.
|
||||||
|
// this gives two total hops to resolve the request before giving up.
|
||||||
|
boolean recursiveForward = forward == null;
|
||||||
|
RequestTargetException targetException = (RequestTargetException) cause;
|
||||||
|
String forwardedUrl = targetException.forwardUrl();
|
||||||
|
if (forwardedUrl == null) {
|
||||||
|
// the target didn't know of the leader at this moment.
|
||||||
|
throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(),
|
||||||
|
"Cannot complete request momentarily due to no known leader URL, "
|
||||||
|
+ "likely because a rebalance was underway.");
|
||||||
|
}
|
||||||
|
UriBuilder uriBuilder = UriBuilder.fromUri(forwardedUrl)
|
||||||
|
.path(path)
|
||||||
|
.queryParam("forward", recursiveForward);
|
||||||
|
if (queryParameters != null) {
|
||||||
|
queryParameters.forEach(uriBuilder::queryParam);
|
||||||
|
}
|
||||||
|
String forwardUrl = uriBuilder.build().toString();
|
||||||
|
log.debug("Forwarding request {} {} {}", forwardUrl, method, body);
|
||||||
|
return translator.translate(restClient.httpRequest(forwardUrl, method, headers, body, resultType));
|
||||||
|
} else {
|
||||||
|
// we should find the right target for the query within two hops, so if
|
||||||
|
// we don't, it probably means that a rebalance has taken place.
|
||||||
|
throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(),
|
||||||
|
"Cannot complete request because of a conflicting operation (e.g. worker rebalance)");
|
||||||
|
}
|
||||||
|
} else if (cause instanceof RebalanceNeededException) {
|
||||||
|
throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(),
|
||||||
|
"Cannot complete request momentarily due to stale configuration (typically caused by a concurrent config change)");
|
||||||
|
}
|
||||||
|
|
||||||
|
throw cause;
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
// This timeout is for the operation itself. None of the timeout error codes are relevant, so internal server
|
||||||
|
// error is the best option
|
||||||
|
throw new ConnectRestException(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), "Request timed out");
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new ConnectRestException(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), "Request interrupted");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public <T, U> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method, HttpHeaders headers, Object body,
|
||||||
|
TypeReference<U> resultType, Translator<T, U> translator, Boolean forward) throws Throwable {
|
||||||
|
return completeOrForwardRequest(cb, path, method, headers, null, body, resultType, translator, forward);
|
||||||
|
}
|
||||||
|
|
||||||
|
public <T> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method, HttpHeaders headers, Object body,
|
||||||
|
TypeReference<T> resultType, Boolean forward) throws Throwable {
|
||||||
|
return completeOrForwardRequest(cb, path, method, headers, body, resultType, new IdentityTranslator<>(), forward);
|
||||||
|
}
|
||||||
|
|
||||||
|
public <T> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method, HttpHeaders headers,
|
||||||
|
Object body, Boolean forward) throws Throwable {
|
||||||
|
return completeOrForwardRequest(cb, path, method, headers, body, null, new IdentityTranslator<>(), forward);
|
||||||
|
}
|
||||||
|
|
||||||
|
public interface Translator<T, U> {
|
||||||
|
T translate(RestClient.HttpResponse<U> response);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class IdentityTranslator<T> implements Translator<T, T> {
|
||||||
|
@Override
|
||||||
|
public T translate(RestClient.HttpResponse<T> response) {
|
||||||
|
return response.body();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,8 +19,8 @@ package org.apache.kafka.connect.runtime.rest;
|
||||||
|
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import org.apache.kafka.connect.runtime.WorkerConfig;
|
|
||||||
import org.apache.kafka.connect.runtime.distributed.Crypto;
|
import org.apache.kafka.connect.runtime.distributed.Crypto;
|
||||||
|
import org.apache.kafka.common.config.AbstractConfig;
|
||||||
import org.apache.kafka.connect.runtime.rest.entities.ErrorMessage;
|
import org.apache.kafka.connect.runtime.rest.entities.ErrorMessage;
|
||||||
import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
|
import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
|
||||||
import org.apache.kafka.connect.runtime.rest.util.SSLUtils;
|
import org.apache.kafka.connect.runtime.rest.util.SSLUtils;
|
||||||
|
@ -51,9 +51,10 @@ import java.util.concurrent.TimeoutException;
|
||||||
public class RestClient {
|
public class RestClient {
|
||||||
private static final Logger log = LoggerFactory.getLogger(RestClient.class);
|
private static final Logger log = LoggerFactory.getLogger(RestClient.class);
|
||||||
private static final ObjectMapper JSON_SERDE = new ObjectMapper();
|
private static final ObjectMapper JSON_SERDE = new ObjectMapper();
|
||||||
private WorkerConfig config;
|
|
||||||
|
|
||||||
public RestClient(WorkerConfig config) {
|
private final AbstractConfig config;
|
||||||
|
|
||||||
|
public RestClient(AbstractConfig config) {
|
||||||
this.config = config;
|
this.config = config;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,15 +24,10 @@ import org.apache.kafka.connect.health.ConnectClusterDetails;
|
||||||
import org.apache.kafka.connect.rest.ConnectRestExtension;
|
import org.apache.kafka.connect.rest.ConnectRestExtension;
|
||||||
import org.apache.kafka.connect.rest.ConnectRestExtensionContext;
|
import org.apache.kafka.connect.rest.ConnectRestExtensionContext;
|
||||||
import org.apache.kafka.connect.runtime.Herder;
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
import org.apache.kafka.connect.runtime.WorkerConfig;
|
|
||||||
import org.apache.kafka.connect.runtime.health.ConnectClusterDetailsImpl;
|
import org.apache.kafka.connect.runtime.health.ConnectClusterDetailsImpl;
|
||||||
import org.apache.kafka.connect.runtime.health.ConnectClusterStateImpl;
|
import org.apache.kafka.connect.runtime.health.ConnectClusterStateImpl;
|
||||||
import org.apache.kafka.connect.runtime.rest.errors.ConnectExceptionMapper;
|
import org.apache.kafka.connect.runtime.rest.errors.ConnectExceptionMapper;
|
||||||
import org.apache.kafka.connect.runtime.rest.resources.ConnectResource;
|
import org.apache.kafka.connect.runtime.rest.resources.ConnectResource;
|
||||||
import org.apache.kafka.connect.runtime.rest.resources.ConnectorPluginsResource;
|
|
||||||
import org.apache.kafka.connect.runtime.rest.resources.ConnectorsResource;
|
|
||||||
import org.apache.kafka.connect.runtime.rest.resources.LoggingResource;
|
|
||||||
import org.apache.kafka.connect.runtime.rest.resources.RootResource;
|
|
||||||
import org.apache.kafka.connect.runtime.rest.util.SSLUtils;
|
import org.apache.kafka.connect.runtime.rest.util.SSLUtils;
|
||||||
import org.eclipse.jetty.server.Connector;
|
import org.eclipse.jetty.server.Connector;
|
||||||
import org.eclipse.jetty.server.CustomRequestLog;
|
import org.eclipse.jetty.server.CustomRequestLog;
|
||||||
|
@ -67,12 +62,10 @@ import java.util.Locale;
|
||||||
import java.util.regex.Matcher;
|
import java.util.regex.Matcher;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.ADMIN_LISTENERS_HTTPS_CONFIGS_PREFIX;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Embedded server for the REST API that provides the control plane for Kafka Connect workers.
|
* Embedded server for the REST API that provides the control plane for Kafka Connect workers.
|
||||||
*/
|
*/
|
||||||
public class RestServer {
|
public abstract class RestServer {
|
||||||
private static final Logger log = LoggerFactory.getLogger(RestServer.class);
|
private static final Logger log = LoggerFactory.getLogger(RestServer.class);
|
||||||
|
|
||||||
// Used to distinguish between Admin connectors and regular REST API connectors when binding admin handlers
|
// Used to distinguish between Admin connectors and regular REST API connectors when binding admin handlers
|
||||||
|
@ -84,9 +77,7 @@ public class RestServer {
|
||||||
private static final String PROTOCOL_HTTP = "http";
|
private static final String PROTOCOL_HTTP = "http";
|
||||||
private static final String PROTOCOL_HTTPS = "https";
|
private static final String PROTOCOL_HTTPS = "https";
|
||||||
|
|
||||||
private final WorkerConfig config;
|
protected final RestServerConfig config;
|
||||||
|
|
||||||
private final RestClient restClient;
|
|
||||||
private final ContextHandlerCollection handlers;
|
private final ContextHandlerCollection handlers;
|
||||||
private final Server jettyServer;
|
private final Server jettyServer;
|
||||||
|
|
||||||
|
@ -96,12 +87,11 @@ public class RestServer {
|
||||||
/**
|
/**
|
||||||
* Create a REST server for this herder using the specified configs.
|
* Create a REST server for this herder using the specified configs.
|
||||||
*/
|
*/
|
||||||
public RestServer(WorkerConfig config, RestClient restClient) {
|
protected RestServer(RestServerConfig config) {
|
||||||
this.config = config;
|
this.config = config;
|
||||||
this.restClient = restClient;
|
|
||||||
|
|
||||||
List<String> listeners = config.getList(WorkerConfig.LISTENERS_CONFIG);
|
List<String> listeners = config.listeners();
|
||||||
List<String> adminListeners = config.getList(WorkerConfig.ADMIN_LISTENERS_CONFIG);
|
List<String> adminListeners = config.adminListeners();
|
||||||
|
|
||||||
jettyServer = new Server();
|
jettyServer = new Server();
|
||||||
handlers = new ContextHandlerCollection();
|
handlers = new ContextHandlerCollection();
|
||||||
|
@ -161,7 +151,7 @@ public class RestServer {
|
||||||
if (PROTOCOL_HTTPS.equals(protocol)) {
|
if (PROTOCOL_HTTPS.equals(protocol)) {
|
||||||
SslContextFactory ssl;
|
SslContextFactory ssl;
|
||||||
if (isAdmin) {
|
if (isAdmin) {
|
||||||
ssl = SSLUtils.createServerSideSslContextFactory(config, ADMIN_LISTENERS_HTTPS_CONFIGS_PREFIX);
|
ssl = SSLUtils.createServerSideSslContextFactory(config, RestServerConfig.ADMIN_LISTENERS_HTTPS_CONFIGS_PREFIX);
|
||||||
} else {
|
} else {
|
||||||
ssl = SSLUtils.createServerSideSslContextFactory(config);
|
ssl = SSLUtils.createServerSideSslContextFactory(config);
|
||||||
}
|
}
|
||||||
|
@ -210,44 +200,47 @@ public class RestServer {
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info("REST server listening at " + jettyServer.getURI() + ", advertising URL " + advertisedUrl());
|
log.info("REST server listening at " + jettyServer.getURI() + ", advertising URL " + advertisedUrl());
|
||||||
log.info("REST admin endpoints at " + adminUrl());
|
URI adminUrl = adminUrl();
|
||||||
|
if (adminUrl != null)
|
||||||
|
log.info("REST admin endpoints at " + adminUrl);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void initializeResources(Herder herder) {
|
protected final void initializeResources() {
|
||||||
log.info("Initializing REST resources");
|
log.info("Initializing REST resources");
|
||||||
|
resources = new ArrayList<>();
|
||||||
|
|
||||||
ResourceConfig resourceConfig = new ResourceConfig();
|
ResourceConfig resourceConfig = new ResourceConfig();
|
||||||
resourceConfig.register(new JacksonJsonProvider());
|
resourceConfig.register(new JacksonJsonProvider());
|
||||||
|
|
||||||
this.resources = new ArrayList<>();
|
Collection<ConnectResource> regularResources = regularResources();
|
||||||
resources.add(new RootResource(herder));
|
regularResources.forEach(resourceConfig::register);
|
||||||
resources.add(new ConnectorsResource(herder, config, restClient));
|
resources.addAll(regularResources);
|
||||||
resources.add(new ConnectorPluginsResource(herder));
|
|
||||||
resources.forEach(resourceConfig::register);
|
|
||||||
|
|
||||||
resourceConfig.register(ConnectExceptionMapper.class);
|
resourceConfig.register(ConnectExceptionMapper.class);
|
||||||
resourceConfig.property(ServerProperties.WADL_FEATURE_DISABLE, true);
|
resourceConfig.property(ServerProperties.WADL_FEATURE_DISABLE, true);
|
||||||
|
|
||||||
registerRestExtensions(herder, resourceConfig);
|
configureRegularResources(resourceConfig);
|
||||||
|
|
||||||
List<String> adminListeners = config.getList(WorkerConfig.ADMIN_LISTENERS_CONFIG);
|
List<String> adminListeners = config.adminListeners();
|
||||||
ResourceConfig adminResourceConfig;
|
ResourceConfig adminResourceConfig;
|
||||||
if (adminListeners == null) {
|
if (adminListeners == null) {
|
||||||
log.info("Adding admin resources to main listener");
|
log.info("Adding admin resources to main listener");
|
||||||
adminResourceConfig = resourceConfig;
|
adminResourceConfig = resourceConfig;
|
||||||
LoggingResource loggingResource = new LoggingResource();
|
Collection<ConnectResource> adminResources = adminResources();
|
||||||
this.resources.add(loggingResource);
|
resources.addAll(adminResources);
|
||||||
adminResourceConfig.register(loggingResource);
|
adminResources.forEach(adminResourceConfig::register);
|
||||||
|
configureAdminResources(adminResourceConfig);
|
||||||
} else if (adminListeners.size() > 0) {
|
} else if (adminListeners.size() > 0) {
|
||||||
// TODO: we need to check if these listeners are same as 'listeners'
|
// TODO: we need to check if these listeners are same as 'listeners'
|
||||||
// TODO: the following code assumes that they are different
|
// TODO: the following code assumes that they are different
|
||||||
log.info("Adding admin resources to admin listener");
|
log.info("Adding admin resources to admin listener");
|
||||||
adminResourceConfig = new ResourceConfig();
|
adminResourceConfig = new ResourceConfig();
|
||||||
adminResourceConfig.register(new JacksonJsonProvider());
|
adminResourceConfig.register(new JacksonJsonProvider());
|
||||||
LoggingResource loggingResource = new LoggingResource();
|
Collection<ConnectResource> adminResources = adminResources();
|
||||||
this.resources.add(loggingResource);
|
resources.addAll(adminResources);
|
||||||
adminResourceConfig.register(loggingResource);
|
adminResources.forEach(adminResourceConfig::register);
|
||||||
adminResourceConfig.register(ConnectExceptionMapper.class);
|
adminResourceConfig.register(ConnectExceptionMapper.class);
|
||||||
|
configureAdminResources(adminResourceConfig);
|
||||||
} else {
|
} else {
|
||||||
log.info("Skipping adding admin resources");
|
log.info("Skipping adding admin resources");
|
||||||
// set up adminResource but add no handlers to it
|
// set up adminResource but add no handlers to it
|
||||||
|
@ -273,21 +266,21 @@ public class RestServer {
|
||||||
contextHandlers.add(adminContext);
|
contextHandlers.add(adminContext);
|
||||||
}
|
}
|
||||||
|
|
||||||
String allowedOrigins = config.getString(WorkerConfig.ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG);
|
String allowedOrigins = config.allowedOrigins();
|
||||||
if (!Utils.isBlank(allowedOrigins)) {
|
if (!Utils.isBlank(allowedOrigins)) {
|
||||||
FilterHolder filterHolder = new FilterHolder(new CrossOriginFilter());
|
FilterHolder filterHolder = new FilterHolder(new CrossOriginFilter());
|
||||||
filterHolder.setName("cross-origin");
|
filterHolder.setName("cross-origin");
|
||||||
filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_ORIGINS_PARAM, allowedOrigins);
|
filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_ORIGINS_PARAM, allowedOrigins);
|
||||||
String allowedMethods = config.getString(WorkerConfig.ACCESS_CONTROL_ALLOW_METHODS_CONFIG);
|
String allowedMethods = config.allowedMethods();
|
||||||
if (!Utils.isBlank(allowedMethods)) {
|
if (!Utils.isBlank(allowedMethods)) {
|
||||||
filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_METHODS_PARAM, allowedMethods);
|
filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_METHODS_PARAM, allowedMethods);
|
||||||
}
|
}
|
||||||
context.addFilter(filterHolder, "/*", EnumSet.of(DispatcherType.REQUEST));
|
context.addFilter(filterHolder, "/*", EnumSet.of(DispatcherType.REQUEST));
|
||||||
}
|
}
|
||||||
|
|
||||||
String headerConfig = config.getString(WorkerConfig.RESPONSE_HTTP_HEADERS_CONFIG);
|
String headerConfig = config.responseHeaders();
|
||||||
if (!Utils.isBlank(headerConfig)) {
|
if (!Utils.isBlank(headerConfig)) {
|
||||||
configureHttpResponseHeaderFilter(context);
|
configureHttpResponseHeaderFilter(context, headerConfig);
|
||||||
}
|
}
|
||||||
|
|
||||||
handlers.setHandlers(contextHandlers.toArray(new Handler[0]));
|
handlers.setHandlers(contextHandlers.toArray(new Handler[0]));
|
||||||
|
@ -309,6 +302,44 @@ public class RestServer {
|
||||||
log.info("REST resources initialized; server is started and ready to handle requests");
|
log.info("REST resources initialized; server is started and ready to handle requests");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the {@link ConnectResource resources} that should be registered with the
|
||||||
|
* standard (i.e., non-admin) listener for this server; may be empty, but not null
|
||||||
|
*/
|
||||||
|
protected abstract Collection<ConnectResource> regularResources();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the {@link ConnectResource resources} that should be registered with the
|
||||||
|
* admin listener for this server; may be empty, but not null
|
||||||
|
*/
|
||||||
|
protected abstract Collection<ConnectResource> adminResources();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Pluggable hook to customize the regular (i.e., non-admin) resources on this server
|
||||||
|
* after they have been instantiated and registered with the given {@link ResourceConfig}.
|
||||||
|
* This may be used to, for example, add REST extensions via {@link #registerRestExtensions(Herder, ResourceConfig)}.
|
||||||
|
* <p>
|
||||||
|
* <em>N.B.: Classes do <b>not</b> need to register the resources provided in {@link #regularResources()} with
|
||||||
|
* the {@link ResourceConfig} parameter in this method; they are automatically registered by the parent class.</em>
|
||||||
|
* @param resourceConfig the {@link ResourceConfig} that the server's regular listeners are registered with; never null
|
||||||
|
*/
|
||||||
|
protected void configureRegularResources(ResourceConfig resourceConfig) {
|
||||||
|
// No-op by default
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Pluggable hook to customize the admin resources on this server after they have been instantiated and registered
|
||||||
|
* with the given {@link ResourceConfig}. This may be used to, for example, add REST extensions via
|
||||||
|
* {@link #registerRestExtensions(Herder, ResourceConfig)}.
|
||||||
|
* <p>
|
||||||
|
* <em>N.B.: Classes do <b>not</b> need to register the resources provided in {@link #adminResources()} with
|
||||||
|
* the {@link ResourceConfig} parameter in this method; they are automatically registered by the parent class.</em>
|
||||||
|
* @param adminResourceConfig the {@link ResourceConfig} that the server's regular listeners are registered with; never null
|
||||||
|
*/
|
||||||
|
protected void configureAdminResources(ResourceConfig adminResourceConfig) {
|
||||||
|
// No-op by default
|
||||||
|
}
|
||||||
|
|
||||||
public URI serverUrl() {
|
public URI serverUrl() {
|
||||||
return jettyServer.getURI();
|
return jettyServer.getURI();
|
||||||
}
|
}
|
||||||
|
@ -346,13 +377,13 @@ public class RestServer {
|
||||||
ServerConnector serverConnector = findConnector(advertisedSecurityProtocol);
|
ServerConnector serverConnector = findConnector(advertisedSecurityProtocol);
|
||||||
builder.scheme(advertisedSecurityProtocol);
|
builder.scheme(advertisedSecurityProtocol);
|
||||||
|
|
||||||
String advertisedHostname = config.getString(WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG);
|
String advertisedHostname = config.advertisedHostName();
|
||||||
if (advertisedHostname != null && !advertisedHostname.isEmpty())
|
if (advertisedHostname != null && !advertisedHostname.isEmpty())
|
||||||
builder.host(advertisedHostname);
|
builder.host(advertisedHostname);
|
||||||
else if (serverConnector != null && serverConnector.getHost() != null && serverConnector.getHost().length() > 0)
|
else if (serverConnector != null && serverConnector.getHost() != null && serverConnector.getHost().length() > 0)
|
||||||
builder.host(serverConnector.getHost());
|
builder.host(serverConnector.getHost());
|
||||||
|
|
||||||
Integer advertisedPort = config.getInt(WorkerConfig.REST_ADVERTISED_PORT_CONFIG);
|
Integer advertisedPort = config.advertisedPort();
|
||||||
if (advertisedPort != null)
|
if (advertisedPort != null)
|
||||||
builder.port(advertisedPort);
|
builder.port(advertisedPort);
|
||||||
else if (serverConnector != null && serverConnector.getPort() > 0)
|
else if (serverConnector != null && serverConnector.getPort() > 0)
|
||||||
|
@ -376,7 +407,7 @@ public class RestServer {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (adminConnector == null) {
|
if (adminConnector == null) {
|
||||||
List<String> adminListeners = config.getList(WorkerConfig.ADMIN_LISTENERS_CONFIG);
|
List<String> adminListeners = config.adminListeners();
|
||||||
if (adminListeners == null) {
|
if (adminListeners == null) {
|
||||||
return advertisedUrl();
|
return advertisedUrl();
|
||||||
} else if (adminListeners.isEmpty()) {
|
} else if (adminListeners.isEmpty()) {
|
||||||
|
@ -399,9 +430,9 @@ public class RestServer {
|
||||||
}
|
}
|
||||||
|
|
||||||
String determineAdvertisedProtocol() {
|
String determineAdvertisedProtocol() {
|
||||||
String advertisedSecurityProtocol = config.getString(WorkerConfig.REST_ADVERTISED_LISTENER_CONFIG);
|
String advertisedSecurityProtocol = config.advertisedListener();
|
||||||
if (advertisedSecurityProtocol == null) {
|
if (advertisedSecurityProtocol == null) {
|
||||||
String listeners = (String) config.originals().get(WorkerConfig.LISTENERS_CONFIG);
|
String listeners = config.rawListeners();
|
||||||
|
|
||||||
if (listeners == null)
|
if (listeners == null)
|
||||||
return PROTOCOL_HTTP;
|
return PROTOCOL_HTTP;
|
||||||
|
@ -440,14 +471,14 @@ public class RestServer {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerRestExtensions(Herder herder, ResourceConfig resourceConfig) {
|
protected final void registerRestExtensions(Herder herder, ResourceConfig resourceConfig) {
|
||||||
connectRestExtensions = herder.plugins().newPlugins(
|
connectRestExtensions = herder.plugins().newPlugins(
|
||||||
config.getList(WorkerConfig.REST_EXTENSION_CLASSES_CONFIG),
|
config.restExtensions(),
|
||||||
config, ConnectRestExtension.class);
|
config, ConnectRestExtension.class);
|
||||||
|
|
||||||
long herderRequestTimeoutMs = ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS;
|
long herderRequestTimeoutMs = ConnectResource.DEFAULT_REST_REQUEST_TIMEOUT_MS;
|
||||||
|
|
||||||
Integer rebalanceTimeoutMs = config.getRebalanceTimeout();
|
Integer rebalanceTimeoutMs = config.rebalanceTimeoutMs();
|
||||||
|
|
||||||
if (rebalanceTimeoutMs != null) {
|
if (rebalanceTimeoutMs != null) {
|
||||||
herderRequestTimeoutMs = Math.min(herderRequestTimeoutMs, rebalanceTimeoutMs.longValue());
|
herderRequestTimeoutMs = Math.min(herderRequestTimeoutMs, rebalanceTimeoutMs.longValue());
|
||||||
|
@ -472,8 +503,7 @@ public class RestServer {
|
||||||
* Register header filter to ServletContextHandler.
|
* Register header filter to ServletContextHandler.
|
||||||
* @param context The servlet context handler
|
* @param context The servlet context handler
|
||||||
*/
|
*/
|
||||||
protected void configureHttpResponseHeaderFilter(ServletContextHandler context) {
|
protected void configureHttpResponseHeaderFilter(ServletContextHandler context, String headerConfig) {
|
||||||
String headerConfig = config.getString(WorkerConfig.RESPONSE_HTTP_HEADERS_CONFIG);
|
|
||||||
FilterHolder headerFilterHolder = new FilterHolder(HeaderFilter.class);
|
FilterHolder headerFilterHolder = new FilterHolder(HeaderFilter.class);
|
||||||
headerFilterHolder.setInitParameter("headerConfig", headerConfig);
|
headerFilterHolder.setInitParameter("headerConfig", headerConfig);
|
||||||
context.addFilter(headerFilterHolder, "/*", EnumSet.of(DispatcherType.REQUEST));
|
context.addFilter(headerFilterHolder, "/*", EnumSet.of(DispatcherType.REQUEST));
|
||||||
|
|
|
@ -0,0 +1,463 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.config.AbstractConfig;
|
||||||
|
import org.apache.kafka.common.config.ConfigDef;
|
||||||
|
import org.apache.kafka.common.config.ConfigException;
|
||||||
|
import org.apache.kafka.common.config.SslClientAuth;
|
||||||
|
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
||||||
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
import org.apache.kafka.connect.runtime.WorkerConfig;
|
||||||
|
import org.eclipse.jetty.util.StringUtil;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Defines the configuration surface for a {@link RestServer} instance, with support for both
|
||||||
|
* {@link #forInternal(Map) internal-only} and {@link #forPublic(Integer, Map) user-facing}
|
||||||
|
* servers. An internal-only server will expose only the endpoints and listeners necessary for
|
||||||
|
* intra-cluster communication; these include the task-write and zombie-fencing endpoints. A
|
||||||
|
* user-facing server will expose these endpoints and, in addition, all endpoints that are part of
|
||||||
|
* the public REST API for Kafka Connect; these include the connector creation, connector
|
||||||
|
* status, configuration validation, and logging endpoints. In addition, a user-facing server will
|
||||||
|
* instantiate any user-configured
|
||||||
|
* {@link RestServerConfig#REST_EXTENSION_CLASSES_CONFIG REST extensions}.
|
||||||
|
*/
|
||||||
|
public abstract class RestServerConfig extends AbstractConfig {
|
||||||
|
|
||||||
|
public static final String LISTENERS_CONFIG = "listeners";
|
||||||
|
private static final String LISTENERS_DOC
|
||||||
|
= "List of comma-separated URIs the REST API will listen on. The supported protocols are HTTP and HTTPS.\n" +
|
||||||
|
" Specify hostname as 0.0.0.0 to bind to all interfaces.\n" +
|
||||||
|
" Leave hostname empty to bind to default interface.\n" +
|
||||||
|
" Examples of legal listener lists: HTTP://myhost:8083,HTTPS://myhost:8084";
|
||||||
|
// Visible for testing
|
||||||
|
static final List<String> LISTENERS_DEFAULT = Collections.singletonList("http://:8083");
|
||||||
|
|
||||||
|
public static final String REST_ADVERTISED_HOST_NAME_CONFIG = "rest.advertised.host.name";
|
||||||
|
private static final String REST_ADVERTISED_HOST_NAME_DOC
|
||||||
|
= "If this is set, this is the hostname that will be given out to other workers to connect to.";
|
||||||
|
|
||||||
|
public static final String REST_ADVERTISED_PORT_CONFIG = "rest.advertised.port";
|
||||||
|
private static final String REST_ADVERTISED_PORT_DOC
|
||||||
|
= "If this is set, this is the port that will be given out to other workers to connect to.";
|
||||||
|
|
||||||
|
public static final String REST_ADVERTISED_LISTENER_CONFIG = "rest.advertised.listener";
|
||||||
|
private static final String REST_ADVERTISED_LISTENER_DOC
|
||||||
|
= "Sets the advertised listener (HTTP or HTTPS) which will be given to other workers to use.";
|
||||||
|
|
||||||
|
public static final String ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG = "access.control.allow.origin";
|
||||||
|
private static final String ACCESS_CONTROL_ALLOW_ORIGIN_DOC =
|
||||||
|
"Value to set the Access-Control-Allow-Origin header to for REST API requests." +
|
||||||
|
"To enable cross origin access, set this to the domain of the application that should be permitted" +
|
||||||
|
" to access the API, or '*' to allow access from any domain. The default value only allows access" +
|
||||||
|
" from the domain of the REST API.";
|
||||||
|
protected static final String ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT = "";
|
||||||
|
|
||||||
|
public static final String ACCESS_CONTROL_ALLOW_METHODS_CONFIG = "access.control.allow.methods";
|
||||||
|
private static final String ACCESS_CONTROL_ALLOW_METHODS_DOC =
|
||||||
|
"Sets the methods supported for cross origin requests by setting the Access-Control-Allow-Methods header. "
|
||||||
|
+ "The default value of the Access-Control-Allow-Methods header allows cross origin requests for GET, POST and HEAD.";
|
||||||
|
private static final String ACCESS_CONTROL_ALLOW_METHODS_DEFAULT = "";
|
||||||
|
|
||||||
|
public static final String ADMIN_LISTENERS_CONFIG = "admin.listeners";
|
||||||
|
private static final String ADMIN_LISTENERS_DOC = "List of comma-separated URIs the Admin REST API will listen on." +
|
||||||
|
" The supported protocols are HTTP and HTTPS." +
|
||||||
|
" An empty or blank string will disable this feature." +
|
||||||
|
" The default behavior is to use the regular listener (specified by the 'listeners' property).";
|
||||||
|
public static final String ADMIN_LISTENERS_HTTPS_CONFIGS_PREFIX = "admin.listeners.https.";
|
||||||
|
|
||||||
|
public static final String REST_EXTENSION_CLASSES_CONFIG = "rest.extension.classes";
|
||||||
|
private static final String REST_EXTENSION_CLASSES_DOC =
|
||||||
|
"Comma-separated names of <code>ConnectRestExtension</code> classes, loaded and called "
|
||||||
|
+ "in the order specified. Implementing the interface "
|
||||||
|
+ "<code>ConnectRestExtension</code> allows you to inject into Connect's REST API user defined resources like filters. "
|
||||||
|
+ "Typically used to add custom capability like logging, security, etc. ";
|
||||||
|
|
||||||
|
// Visible for testing
|
||||||
|
static final String RESPONSE_HTTP_HEADERS_CONFIG = "response.http.headers.config";
|
||||||
|
// Visible for testing
|
||||||
|
static final String RESPONSE_HTTP_HEADERS_DOC = "Rules for REST API HTTP response headers";
|
||||||
|
// Visible for testing
|
||||||
|
static final String RESPONSE_HTTP_HEADERS_DEFAULT = "";
|
||||||
|
private static final Collection<String> HEADER_ACTIONS = Collections.unmodifiableList(
|
||||||
|
Arrays.asList("set", "add", "setDate", "addDate")
|
||||||
|
);
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the listeners to use for this server, or empty if no admin endpoints should be exposed,
|
||||||
|
* or null if the admin endpoints should be exposed on the {@link #listeners() regular listeners} for
|
||||||
|
* this server
|
||||||
|
*/
|
||||||
|
public abstract List<String> adminListeners();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a list of {@link #REST_EXTENSION_CLASSES_CONFIG REST extension} classes
|
||||||
|
* to instantiate and use with the server
|
||||||
|
*/
|
||||||
|
public abstract List<String> restExtensions();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return whether {@link WorkerConfig#TOPIC_TRACKING_ENABLE_CONFIG topic tracking}
|
||||||
|
* is enabled on this worker
|
||||||
|
*/
|
||||||
|
public abstract boolean topicTrackingEnabled();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return whether {@link WorkerConfig#TOPIC_TRACKING_ALLOW_RESET_CONFIG topic tracking resets}
|
||||||
|
* are enabled on this worker
|
||||||
|
*/
|
||||||
|
public abstract boolean topicTrackingResetEnabled();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add the properties related to a user-facing server to the given {@link ConfigDef}.
|
||||||
|
* </p>
|
||||||
|
* This automatically adds the properties for intra-cluster communication; it is not necessary to
|
||||||
|
* invoke both {@link #addInternalConfig(ConfigDef)} and this method on the same {@link ConfigDef}.
|
||||||
|
* @param configDef the {@link ConfigDef} to add the properties to; may not be null
|
||||||
|
*/
|
||||||
|
public static void addPublicConfig(ConfigDef configDef) {
|
||||||
|
addInternalConfig(configDef);
|
||||||
|
configDef
|
||||||
|
.define(
|
||||||
|
REST_EXTENSION_CLASSES_CONFIG,
|
||||||
|
ConfigDef.Type.LIST,
|
||||||
|
"",
|
||||||
|
ConfigDef.Importance.LOW, REST_EXTENSION_CLASSES_DOC
|
||||||
|
).define(ADMIN_LISTENERS_CONFIG,
|
||||||
|
ConfigDef.Type.LIST,
|
||||||
|
null,
|
||||||
|
new AdminListenersValidator(),
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
ADMIN_LISTENERS_DOC);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add the properties related to an internal-only server to the given {@link ConfigDef}.
|
||||||
|
* @param configDef the {@link ConfigDef} to add the properties to; may not be null
|
||||||
|
*/
|
||||||
|
public static void addInternalConfig(ConfigDef configDef) {
|
||||||
|
configDef
|
||||||
|
.define(
|
||||||
|
LISTENERS_CONFIG,
|
||||||
|
ConfigDef.Type.LIST,
|
||||||
|
LISTENERS_DEFAULT,
|
||||||
|
new ListenersValidator(),
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
LISTENERS_DOC
|
||||||
|
).define(
|
||||||
|
REST_ADVERTISED_HOST_NAME_CONFIG,
|
||||||
|
ConfigDef.Type.STRING,
|
||||||
|
null,
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
REST_ADVERTISED_HOST_NAME_DOC
|
||||||
|
).define(
|
||||||
|
REST_ADVERTISED_PORT_CONFIG,
|
||||||
|
ConfigDef.Type.INT,
|
||||||
|
null,
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
REST_ADVERTISED_PORT_DOC
|
||||||
|
).define(
|
||||||
|
REST_ADVERTISED_LISTENER_CONFIG,
|
||||||
|
ConfigDef.Type.STRING,
|
||||||
|
null,
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
REST_ADVERTISED_LISTENER_DOC
|
||||||
|
).define(
|
||||||
|
ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG,
|
||||||
|
ConfigDef.Type.STRING,
|
||||||
|
ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT,
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
ACCESS_CONTROL_ALLOW_ORIGIN_DOC
|
||||||
|
).define(
|
||||||
|
ACCESS_CONTROL_ALLOW_METHODS_CONFIG,
|
||||||
|
ConfigDef.Type.STRING,
|
||||||
|
ACCESS_CONTROL_ALLOW_METHODS_DEFAULT,
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
ACCESS_CONTROL_ALLOW_METHODS_DOC
|
||||||
|
).define(
|
||||||
|
RESPONSE_HTTP_HEADERS_CONFIG,
|
||||||
|
ConfigDef.Type.STRING,
|
||||||
|
RESPONSE_HTTP_HEADERS_DEFAULT,
|
||||||
|
new ResponseHttpHeadersValidator(),
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
RESPONSE_HTTP_HEADERS_DOC
|
||||||
|
).define(
|
||||||
|
BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG,
|
||||||
|
ConfigDef.Type.STRING,
|
||||||
|
SslClientAuth.NONE.toString(),
|
||||||
|
in(Utils.enumOptions(SslClientAuth.class)),
|
||||||
|
ConfigDef.Importance.LOW,
|
||||||
|
BrokerSecurityConfigs.SSL_CLIENT_AUTH_DOC);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static RestServerConfig forPublic(Integer rebalanceTimeoutMs, Map<?, ?> props) {
|
||||||
|
return new PublicConfig(rebalanceTimeoutMs, props);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static RestServerConfig forInternal(Map<?, ?> props) {
|
||||||
|
return new InternalConfig(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> listeners() {
|
||||||
|
return getList(LISTENERS_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String rawListeners() {
|
||||||
|
return (String) originals().get(LISTENERS_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String allowedOrigins() {
|
||||||
|
return getString(ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String allowedMethods() {
|
||||||
|
return getString(ACCESS_CONTROL_ALLOW_METHODS_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String responseHeaders() {
|
||||||
|
return getString(RESPONSE_HTTP_HEADERS_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String advertisedListener() {
|
||||||
|
return getString(RestServerConfig.REST_ADVERTISED_LISTENER_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public String advertisedHostName() {
|
||||||
|
return getString(REST_ADVERTISED_HOST_NAME_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Integer advertisedPort() {
|
||||||
|
return getInt(REST_ADVERTISED_PORT_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Integer rebalanceTimeoutMs() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected RestServerConfig(ConfigDef configDef, Map<?, ?> props) {
|
||||||
|
super(configDef, props);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Visible for testing
|
||||||
|
static void validateHttpResponseHeaderConfig(String config) {
|
||||||
|
try {
|
||||||
|
// validate format
|
||||||
|
String[] configTokens = config.trim().split("\\s+", 2);
|
||||||
|
if (configTokens.length != 2) {
|
||||||
|
throw new ConfigException(String.format("Invalid format of header config '%s'. "
|
||||||
|
+ "Expected: '[action] [header name]:[header value]'", config));
|
||||||
|
}
|
||||||
|
|
||||||
|
// validate action
|
||||||
|
String method = configTokens[0].trim();
|
||||||
|
validateHeaderConfigAction(method);
|
||||||
|
|
||||||
|
// validate header name and header value pair
|
||||||
|
String header = configTokens[1];
|
||||||
|
String[] headerTokens = header.trim().split(":");
|
||||||
|
if (headerTokens.length != 2) {
|
||||||
|
throw new ConfigException(
|
||||||
|
String.format("Invalid format of header name and header value pair '%s'. "
|
||||||
|
+ "Expected: '[header name]:[header value]'", header));
|
||||||
|
}
|
||||||
|
|
||||||
|
// validate header name
|
||||||
|
String headerName = headerTokens[0].trim();
|
||||||
|
if (headerName.isEmpty() || headerName.matches(".*\\s+.*")) {
|
||||||
|
throw new ConfigException(String.format("Invalid header name '%s'. "
|
||||||
|
+ "The '[header name]' cannot contain whitespace", headerName));
|
||||||
|
}
|
||||||
|
} catch (ArrayIndexOutOfBoundsException e) {
|
||||||
|
throw new ConfigException(String.format("Invalid header config '%s'.", config), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Visible for testing
|
||||||
|
static void validateHeaderConfigAction(String action) {
|
||||||
|
if (HEADER_ACTIONS.stream().noneMatch(action::equalsIgnoreCase)) {
|
||||||
|
throw new ConfigException(String.format("Invalid header config action: '%s'. "
|
||||||
|
+ "Expected one of %s", action, HEADER_ACTIONS));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class ListenersValidator implements ConfigDef.Validator {
|
||||||
|
@Override
|
||||||
|
public void ensureValid(String name, Object value) {
|
||||||
|
if (!(value instanceof List)) {
|
||||||
|
throw new ConfigException("Invalid value type for listeners (expected list of URLs , ex: http://localhost:8080,https://localhost:8443).");
|
||||||
|
}
|
||||||
|
|
||||||
|
List<?> items = (List<?>) value;
|
||||||
|
if (items.isEmpty()) {
|
||||||
|
throw new ConfigException("Invalid value for listeners, at least one URL is expected, ex: http://localhost:8080,https://localhost:8443.");
|
||||||
|
}
|
||||||
|
|
||||||
|
for (Object item : items) {
|
||||||
|
if (!(item instanceof String)) {
|
||||||
|
throw new ConfigException("Invalid type for listeners (expected String).");
|
||||||
|
}
|
||||||
|
if (Utils.isBlank((String) item)) {
|
||||||
|
throw new ConfigException("Empty URL found when parsing listeners list.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "List of comma-separated URLs, ex: http://localhost:8080,https://localhost:8443.";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class AdminListenersValidator implements ConfigDef.Validator {
|
||||||
|
@Override
|
||||||
|
public void ensureValid(String name, Object value) {
|
||||||
|
if (value == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!(value instanceof List)) {
|
||||||
|
throw new ConfigException("Invalid value type for admin.listeners (expected list).");
|
||||||
|
}
|
||||||
|
|
||||||
|
List<?> items = (List<?>) value;
|
||||||
|
if (items.isEmpty()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (Object item : items) {
|
||||||
|
if (!(item instanceof String)) {
|
||||||
|
throw new ConfigException("Invalid type for admin.listeners (expected String).");
|
||||||
|
}
|
||||||
|
if (Utils.isBlank((String) item)) {
|
||||||
|
throw new ConfigException("Empty URL found when parsing admin.listeners list.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "List of comma-separated URLs, ex: http://localhost:8080,https://localhost:8443.";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class ResponseHttpHeadersValidator implements ConfigDef.Validator {
|
||||||
|
@Override
|
||||||
|
public void ensureValid(String name, Object value) {
|
||||||
|
String strValue = (String) value;
|
||||||
|
if (Utils.isBlank(strValue)) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
String[] configs = StringUtil.csvSplit(strValue); // handles and removed surrounding quotes
|
||||||
|
Arrays.stream(configs).forEach(RestServerConfig::validateHttpResponseHeaderConfig);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "Comma-separated header rules, where each header rule is of the form "
|
||||||
|
+ "'[action] [header name]:[header value]' and optionally surrounded by double quotes "
|
||||||
|
+ "if any part of a header rule contains a comma";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class InternalConfig extends RestServerConfig {
|
||||||
|
|
||||||
|
private static ConfigDef config() {
|
||||||
|
ConfigDef result = new ConfigDef().withClientSslSupport();
|
||||||
|
addInternalConfig(result);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> adminListeners() {
|
||||||
|
// Disable admin resources (such as the logging resource)
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> restExtensions() {
|
||||||
|
// Disable the use of REST extensions
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean topicTrackingEnabled() {
|
||||||
|
// Topic tracking is unnecessary if we don't expose a public REST API
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean topicTrackingResetEnabled() {
|
||||||
|
// Topic tracking is unnecessary if we don't expose a public REST API
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
public InternalConfig(Map<?, ?> props) {
|
||||||
|
super(config(), props);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class PublicConfig extends RestServerConfig {
|
||||||
|
|
||||||
|
private final Integer rebalanceTimeoutMs;
|
||||||
|
private static ConfigDef config() {
|
||||||
|
ConfigDef result = new ConfigDef().withClientSslSupport();
|
||||||
|
addPublicConfig(result);
|
||||||
|
WorkerConfig.addTopicTrackingConfig(result);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> adminListeners() {
|
||||||
|
return getList(ADMIN_LISTENERS_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> restExtensions() {
|
||||||
|
return getList(REST_EXTENSION_CLASSES_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Integer rebalanceTimeoutMs() {
|
||||||
|
return rebalanceTimeoutMs;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean topicTrackingEnabled() {
|
||||||
|
return getBoolean(WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean topicTrackingResetEnabled() {
|
||||||
|
return getBoolean(WorkerConfig.TOPIC_TRACKING_ALLOW_RESET_CONFIG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public PublicConfig(Integer rebalanceTimeoutMs, Map<?, ?> props) {
|
||||||
|
super(config(), props);
|
||||||
|
this.rebalanceTimeoutMs = rebalanceTimeoutMs;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -21,19 +21,15 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import javax.ws.rs.DefaultValue;
|
import javax.ws.rs.DefaultValue;
|
||||||
import javax.ws.rs.core.HttpHeaders;
|
import javax.ws.rs.core.HttpHeaders;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
import io.swagger.v3.oas.annotations.Operation;
|
import io.swagger.v3.oas.annotations.Operation;
|
||||||
import io.swagger.v3.oas.annotations.Parameter;
|
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.ConnectorConfig;
|
import org.apache.kafka.connect.runtime.ConnectorConfig;
|
||||||
import org.apache.kafka.connect.runtime.Herder;
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
import org.apache.kafka.connect.runtime.RestartRequest;
|
import org.apache.kafka.connect.runtime.RestartRequest;
|
||||||
import org.apache.kafka.connect.runtime.WorkerConfig;
|
import org.apache.kafka.connect.runtime.rest.HerderRequestHandler;
|
||||||
import org.apache.kafka.connect.runtime.distributed.Crypto;
|
|
||||||
import org.apache.kafka.connect.runtime.distributed.RebalanceNeededException;
|
|
||||||
import org.apache.kafka.connect.runtime.distributed.RequestTargetException;
|
|
||||||
import org.apache.kafka.connect.runtime.rest.InternalRequestSignature;
|
|
||||||
import org.apache.kafka.connect.runtime.rest.RestClient;
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestServerConfig;
|
||||||
import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo;
|
import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo;
|
||||||
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.ConnectorStateInfo;
|
import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
|
||||||
|
@ -67,43 +63,33 @@ import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ExecutionException;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
import java.util.concurrent.TimeoutException;
|
|
||||||
|
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ALLOW_RESET_CONFIG;
|
import static org.apache.kafka.connect.runtime.rest.HerderRequestHandler.Translator;
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG;
|
import static org.apache.kafka.connect.runtime.rest.HerderRequestHandler.IdentityTranslator;
|
||||||
|
|
||||||
@Path("/connectors")
|
@Path("/connectors")
|
||||||
@Produces(MediaType.APPLICATION_JSON)
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
@Consumes(MediaType.APPLICATION_JSON)
|
@Consumes(MediaType.APPLICATION_JSON)
|
||||||
public class ConnectorsResource implements ConnectResource {
|
public class ConnectorsResource implements ConnectResource {
|
||||||
private static final Logger log = LoggerFactory.getLogger(ConnectorsResource.class);
|
private static final Logger log = LoggerFactory.getLogger(ConnectorsResource.class);
|
||||||
private static final TypeReference<List<Map<String, String>>> TASK_CONFIGS_TYPE =
|
|
||||||
new TypeReference<List<Map<String, String>>>() { };
|
|
||||||
|
|
||||||
private final Herder herder;
|
private final Herder herder;
|
||||||
private final RestClient restClient;
|
private final HerderRequestHandler requestHandler;
|
||||||
private long requestTimeoutMs;
|
|
||||||
@javax.ws.rs.core.Context
|
@javax.ws.rs.core.Context
|
||||||
private ServletContext context;
|
private ServletContext context;
|
||||||
private final boolean isTopicTrackingDisabled;
|
private final boolean isTopicTrackingDisabled;
|
||||||
private final boolean isTopicTrackingResetDisabled;
|
private final boolean isTopicTrackingResetDisabled;
|
||||||
|
|
||||||
public ConnectorsResource(Herder herder, WorkerConfig config, RestClient restClient) {
|
public ConnectorsResource(Herder herder, RestServerConfig config, RestClient restClient) {
|
||||||
this.herder = herder;
|
this.herder = herder;
|
||||||
this.restClient = restClient;
|
this.requestHandler = new HerderRequestHandler(restClient, DEFAULT_REST_REQUEST_TIMEOUT_MS);
|
||||||
isTopicTrackingDisabled = !config.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG);
|
this.isTopicTrackingDisabled = !config.topicTrackingEnabled();
|
||||||
isTopicTrackingResetDisabled = !config.getBoolean(TOPIC_TRACKING_ALLOW_RESET_CONFIG);
|
this.isTopicTrackingResetDisabled = !config.topicTrackingResetEnabled();
|
||||||
this.requestTimeoutMs = DEFAULT_REST_REQUEST_TIMEOUT_MS;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void requestTimeout(long requestTimeoutMs) {
|
public void requestTimeout(long requestTimeoutMs) {
|
||||||
if (requestTimeoutMs < 1) {
|
requestHandler.requestTimeoutMs(requestTimeoutMs);
|
||||||
throw new IllegalArgumentException("REST request timeout must be positive");
|
|
||||||
}
|
|
||||||
this.requestTimeoutMs = requestTimeoutMs;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
|
@ -160,7 +146,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
|
|
||||||
FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>();
|
FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>();
|
||||||
herder.putConnectorConfig(name, configs, false, cb);
|
herder.putConnectorConfig(name, configs, false, cb);
|
||||||
Herder.Created<ConnectorInfo> info = completeOrForwardRequest(cb, "/connectors", "POST", headers, createRequest,
|
Herder.Created<ConnectorInfo> info = requestHandler.completeOrForwardRequest(cb, "/connectors", "POST", headers, createRequest,
|
||||||
new TypeReference<ConnectorInfo>() { }, new CreatedConnectorInfoTranslator(), forward);
|
new TypeReference<ConnectorInfo>() { }, new CreatedConnectorInfoTranslator(), forward);
|
||||||
|
|
||||||
URI location = UriBuilder.fromUri("/connectors").path(name).build();
|
URI location = UriBuilder.fromUri("/connectors").path(name).build();
|
||||||
|
@ -175,7 +161,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
||||||
FutureCallback<ConnectorInfo> cb = new FutureCallback<>();
|
FutureCallback<ConnectorInfo> cb = new FutureCallback<>();
|
||||||
herder.connectorInfo(connector, cb);
|
herder.connectorInfo(connector, cb);
|
||||||
return completeOrForwardRequest(cb, "/connectors/" + connector, "GET", headers, null, forward);
|
return requestHandler.completeOrForwardRequest(cb, "/connectors/" + connector, "GET", headers, null, forward);
|
||||||
}
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
|
@ -186,7 +172,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
||||||
FutureCallback<Map<String, String>> cb = new FutureCallback<>();
|
FutureCallback<Map<String, String>> cb = new FutureCallback<>();
|
||||||
herder.connectorConfig(connector, cb);
|
herder.connectorConfig(connector, cb);
|
||||||
return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", headers, null, forward);
|
return requestHandler.completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", headers, null, forward);
|
||||||
}
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
|
@ -198,7 +184,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
||||||
FutureCallback<Map<ConnectorTaskId, Map<String, String>>> cb = new FutureCallback<>();
|
FutureCallback<Map<ConnectorTaskId, Map<String, String>>> cb = new FutureCallback<>();
|
||||||
herder.tasksConfig(connector, cb);
|
herder.tasksConfig(connector, cb);
|
||||||
return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks-config", "GET", headers, null, forward);
|
return requestHandler.completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks-config", "GET", headers, null, forward);
|
||||||
}
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
|
@ -247,7 +233,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
checkAndPutConnectorConfigName(connector, connectorConfig);
|
checkAndPutConnectorConfigName(connector, connectorConfig);
|
||||||
|
|
||||||
herder.putConnectorConfig(connector, connectorConfig, true, cb);
|
herder.putConnectorConfig(connector, connectorConfig, true, cb);
|
||||||
Herder.Created<ConnectorInfo> createdInfo = completeOrForwardRequest(cb, "/connectors/" + connector + "/config",
|
Herder.Created<ConnectorInfo> createdInfo = requestHandler.completeOrForwardRequest(cb, "/connectors/" + connector + "/config",
|
||||||
"PUT", headers, connectorConfig, new TypeReference<ConnectorInfo>() { }, new CreatedConnectorInfoTranslator(), forward);
|
"PUT", headers, connectorConfig, new TypeReference<ConnectorInfo>() { }, new CreatedConnectorInfoTranslator(), forward);
|
||||||
Response.ResponseBuilder response;
|
Response.ResponseBuilder response;
|
||||||
if (createdInfo.created()) {
|
if (createdInfo.created()) {
|
||||||
|
@ -273,7 +259,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
// For backward compatibility, just restart the connector instance and return OK with no body
|
// For backward compatibility, just restart the connector instance and return OK with no body
|
||||||
FutureCallback<Void> cb = new FutureCallback<>();
|
FutureCallback<Void> cb = new FutureCallback<>();
|
||||||
herder.restartConnector(connector, cb);
|
herder.restartConnector(connector, cb);
|
||||||
completeOrForwardRequest(cb, forwardingPath, "POST", headers, null, forward);
|
requestHandler.completeOrForwardRequest(cb, forwardingPath, "POST", headers, null, forward);
|
||||||
return Response.noContent().build();
|
return Response.noContent().build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -283,7 +269,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
Map<String, String> queryParameters = new HashMap<>();
|
Map<String, String> queryParameters = new HashMap<>();
|
||||||
queryParameters.put("includeTasks", includeTasks.toString());
|
queryParameters.put("includeTasks", includeTasks.toString());
|
||||||
queryParameters.put("onlyFailed", onlyFailed.toString());
|
queryParameters.put("onlyFailed", onlyFailed.toString());
|
||||||
ConnectorStateInfo stateInfo = completeOrForwardRequest(cb, forwardingPath, "POST", headers, queryParameters, null, new TypeReference<ConnectorStateInfo>() {
|
ConnectorStateInfo stateInfo = requestHandler.completeOrForwardRequest(cb, forwardingPath, "POST", headers, queryParameters, null, new TypeReference<ConnectorStateInfo>() {
|
||||||
}, new IdentityTranslator<>(), forward);
|
}, new IdentityTranslator<>(), forward);
|
||||||
return Response.accepted().entity(stateInfo).build();
|
return Response.accepted().entity(stateInfo).build();
|
||||||
}
|
}
|
||||||
|
@ -314,33 +300,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
||||||
FutureCallback<List<TaskInfo>> cb = new FutureCallback<>();
|
FutureCallback<List<TaskInfo>> cb = new FutureCallback<>();
|
||||||
herder.taskConfigs(connector, cb);
|
herder.taskConfigs(connector, cb);
|
||||||
return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "GET", headers, null, new TypeReference<List<TaskInfo>>() {
|
return requestHandler.completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "GET", headers, null, new TypeReference<List<TaskInfo>>() { }, forward);
|
||||||
}, forward);
|
|
||||||
}
|
|
||||||
|
|
||||||
@POST
|
|
||||||
@Path("/{connector}/tasks")
|
|
||||||
@Operation(hidden = true, summary = "This operation is only for inter-worker communications")
|
|
||||||
public void putTaskConfigs(final @PathParam("connector") String connector,
|
|
||||||
final @Context HttpHeaders headers,
|
|
||||||
final @QueryParam("forward") Boolean forward,
|
|
||||||
final byte[] requestBody) throws Throwable {
|
|
||||||
List<Map<String, String>> taskConfigs = new ObjectMapper().readValue(requestBody, TASK_CONFIGS_TYPE);
|
|
||||||
FutureCallback<Void> cb = new FutureCallback<>();
|
|
||||||
herder.putTaskConfigs(connector, taskConfigs, cb, InternalRequestSignature.fromHeaders(Crypto.SYSTEM, requestBody, headers));
|
|
||||||
completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "POST", headers, taskConfigs, forward);
|
|
||||||
}
|
|
||||||
|
|
||||||
@PUT
|
|
||||||
@Path("/{connector}/fence")
|
|
||||||
@Operation(hidden = true, summary = "This operation is only for inter-worker communications")
|
|
||||||
public void fenceZombies(final @PathParam("connector") String connector,
|
|
||||||
final @Context HttpHeaders headers,
|
|
||||||
final @QueryParam("forward") Boolean forward,
|
|
||||||
final byte[] requestBody) throws Throwable {
|
|
||||||
FutureCallback<Void> cb = new FutureCallback<>();
|
|
||||||
herder.fenceZombieSourceTasks(connector, cb, InternalRequestSignature.fromHeaders(Crypto.SYSTEM, requestBody, headers));
|
|
||||||
completeOrForwardRequest(cb, "/connectors/" + connector + "/fence", "PUT", headers, requestBody, forward);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
|
@ -362,7 +322,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
FutureCallback<Void> cb = new FutureCallback<>();
|
FutureCallback<Void> cb = new FutureCallback<>();
|
||||||
ConnectorTaskId taskId = new ConnectorTaskId(connector, task);
|
ConnectorTaskId taskId = new ConnectorTaskId(connector, task);
|
||||||
herder.restartTask(taskId, cb);
|
herder.restartTask(taskId, cb);
|
||||||
completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks/" + task + "/restart", "POST", headers, null, forward);
|
requestHandler.completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks/" + task + "/restart", "POST", headers, null, forward);
|
||||||
}
|
}
|
||||||
|
|
||||||
@DELETE
|
@DELETE
|
||||||
|
@ -373,7 +333,7 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
final @Parameter(hidden = true) @QueryParam("forward") Boolean forward) throws Throwable {
|
||||||
FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>();
|
FutureCallback<Herder.Created<ConnectorInfo>> cb = new FutureCallback<>();
|
||||||
herder.deleteConnectorConfig(connector, cb);
|
herder.deleteConnectorConfig(connector, cb);
|
||||||
completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", headers, null, forward);
|
requestHandler.completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", headers, null, forward);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check whether the connector name from the url matches the one (if there is one) provided in the connectorConfig
|
// Check whether the connector name from the url matches the one (if there is one) provided in the connectorConfig
|
||||||
|
@ -388,95 +348,6 @@ public class ConnectorsResource implements ConnectResource {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Wait for a FutureCallback to complete. If it succeeds, return the parsed response. If it fails, try to forward the
|
|
||||||
// request to the leader.
|
|
||||||
private <T, U> T completeOrForwardRequest(FutureCallback<T> cb,
|
|
||||||
String path,
|
|
||||||
String method,
|
|
||||||
HttpHeaders headers,
|
|
||||||
Map<String, String> queryParameters,
|
|
||||||
Object body,
|
|
||||||
TypeReference<U> resultType,
|
|
||||||
Translator<T, U> translator,
|
|
||||||
Boolean forward) throws Throwable {
|
|
||||||
try {
|
|
||||||
return cb.get(requestTimeoutMs, TimeUnit.MILLISECONDS);
|
|
||||||
} catch (ExecutionException e) {
|
|
||||||
Throwable cause = e.getCause();
|
|
||||||
|
|
||||||
if (cause instanceof RequestTargetException) {
|
|
||||||
if (restClient == null) {
|
|
||||||
throw new ConnectRestException(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(),
|
|
||||||
"Cannot complete request as non-leader with request forwarding disabled");
|
|
||||||
} else if (forward == null || forward) {
|
|
||||||
// the only time we allow recursive forwarding is when no forward flag has
|
|
||||||
// been set, which should only be seen by the first worker to handle a user request.
|
|
||||||
// this gives two total hops to resolve the request before giving up.
|
|
||||||
boolean recursiveForward = forward == null;
|
|
||||||
RequestTargetException targetException = (RequestTargetException) cause;
|
|
||||||
String forwardedUrl = targetException.forwardUrl();
|
|
||||||
if (forwardedUrl == null) {
|
|
||||||
// the target didn't know of the leader at this moment.
|
|
||||||
throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(),
|
|
||||||
"Cannot complete request momentarily due to no known leader URL, "
|
|
||||||
+ "likely because a rebalance was underway.");
|
|
||||||
}
|
|
||||||
UriBuilder uriBuilder = UriBuilder.fromUri(forwardedUrl)
|
|
||||||
.path(path)
|
|
||||||
.queryParam("forward", recursiveForward);
|
|
||||||
if (queryParameters != null) {
|
|
||||||
queryParameters.forEach(uriBuilder::queryParam);
|
|
||||||
}
|
|
||||||
String forwardUrl = uriBuilder.build().toString();
|
|
||||||
log.debug("Forwarding request {} {} {}", forwardUrl, method, body);
|
|
||||||
return translator.translate(restClient.httpRequest(forwardUrl, method, headers, body, resultType));
|
|
||||||
} else {
|
|
||||||
// we should find the right target for the query within two hops, so if
|
|
||||||
// we don't, it probably means that a rebalance has taken place.
|
|
||||||
throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(),
|
|
||||||
"Cannot complete request because of a conflicting operation (e.g. worker rebalance)");
|
|
||||||
}
|
|
||||||
} else if (cause instanceof RebalanceNeededException) {
|
|
||||||
throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(),
|
|
||||||
"Cannot complete request momentarily due to stale configuration (typically caused by a concurrent config change)");
|
|
||||||
}
|
|
||||||
|
|
||||||
throw cause;
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
// This timeout is for the operation itself. None of the timeout error codes are relevant, so internal server
|
|
||||||
// error is the best option
|
|
||||||
throw new ConnectRestException(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), "Request timed out");
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw new ConnectRestException(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), "Request interrupted");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private <T, U> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method, HttpHeaders headers, Object body,
|
|
||||||
TypeReference<U> resultType, Translator<T, U> translator, Boolean forward) throws Throwable {
|
|
||||||
return completeOrForwardRequest(cb, path, method, headers, null, body, resultType, translator, forward);
|
|
||||||
}
|
|
||||||
|
|
||||||
private <T> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method, HttpHeaders headers, Object body,
|
|
||||||
TypeReference<T> resultType, Boolean forward) throws Throwable {
|
|
||||||
return completeOrForwardRequest(cb, path, method, headers, body, resultType, new IdentityTranslator<>(), forward);
|
|
||||||
}
|
|
||||||
|
|
||||||
private <T> T completeOrForwardRequest(FutureCallback<T> cb, String path, String method, HttpHeaders headers,
|
|
||||||
Object body, Boolean forward) throws Throwable {
|
|
||||||
return completeOrForwardRequest(cb, path, method, headers, body, null, new IdentityTranslator<>(), forward);
|
|
||||||
}
|
|
||||||
|
|
||||||
private interface Translator<T, U> {
|
|
||||||
T translate(RestClient.HttpResponse<U> response);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class IdentityTranslator<T> implements Translator<T, T> {
|
|
||||||
@Override
|
|
||||||
public T translate(RestClient.HttpResponse<T> response) {
|
|
||||||
return response.body();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class CreatedConnectorInfoTranslator implements Translator<Herder.Created<ConnectorInfo>, ConnectorInfo> {
|
private static class CreatedConnectorInfoTranslator implements Translator<Herder.Created<ConnectorInfo>, ConnectorInfo> {
|
||||||
@Override
|
@Override
|
||||||
public Herder.Created<ConnectorInfo> translate(RestClient.HttpResponse<ConnectorInfo> response) {
|
public Herder.Created<ConnectorInfo> translate(RestClient.HttpResponse<ConnectorInfo> response) {
|
||||||
|
|
|
@ -0,0 +1,115 @@
|
||||||
|
/*
|
||||||
|
* 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.resources;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import io.swagger.v3.oas.annotations.Operation;
|
||||||
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
|
import org.apache.kafka.connect.runtime.distributed.Crypto;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.HerderRequestHandler;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.InternalRequestSignature;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
|
import org.apache.kafka.connect.util.FutureCallback;
|
||||||
|
|
||||||
|
import javax.ws.rs.POST;
|
||||||
|
import javax.ws.rs.PUT;
|
||||||
|
import javax.ws.rs.Path;
|
||||||
|
import javax.ws.rs.PathParam;
|
||||||
|
import javax.ws.rs.Produces;
|
||||||
|
import javax.ws.rs.QueryParam;
|
||||||
|
import javax.ws.rs.core.Context;
|
||||||
|
import javax.ws.rs.core.HttpHeaders;
|
||||||
|
import javax.ws.rs.core.MediaType;
|
||||||
|
import javax.ws.rs.core.UriInfo;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Contains endpoints necessary for intra-cluster communication--that is, requests that
|
||||||
|
* workers will issue to each other that originate from within the cluster, as opposed to
|
||||||
|
* requests that originate from a user and are forwarded from one worker to another.
|
||||||
|
*/
|
||||||
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
|
public abstract class InternalClusterResource implements ConnectResource {
|
||||||
|
|
||||||
|
private static final TypeReference<List<Map<String, String>>> TASK_CONFIGS_TYPE =
|
||||||
|
new TypeReference<List<Map<String, String>>>() { };
|
||||||
|
|
||||||
|
private final HerderRequestHandler requestHandler;
|
||||||
|
|
||||||
|
// Visible for testing
|
||||||
|
@Context
|
||||||
|
UriInfo uriInfo;
|
||||||
|
|
||||||
|
protected InternalClusterResource(RestClient restClient) {
|
||||||
|
this.requestHandler = new HerderRequestHandler(restClient, DEFAULT_REST_REQUEST_TIMEOUT_MS);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void requestTimeout(long requestTimeoutMs) {
|
||||||
|
requestHandler.requestTimeoutMs(requestTimeoutMs);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a {@link Herder} instance that can be used to satisfy the current request; may not be null
|
||||||
|
* @throws javax.ws.rs.NotFoundException if no such herder can be provided
|
||||||
|
*/
|
||||||
|
protected abstract Herder herderForRequest();
|
||||||
|
|
||||||
|
@POST
|
||||||
|
@Path("/{connector}/tasks")
|
||||||
|
@Operation(hidden = true, summary = "This operation is only for inter-worker communications")
|
||||||
|
public void putTaskConfigs(
|
||||||
|
final @PathParam("connector") String connector,
|
||||||
|
final @Context HttpHeaders headers,
|
||||||
|
final @QueryParam("forward") Boolean forward,
|
||||||
|
final byte[] requestBody) throws Throwable {
|
||||||
|
List<Map<String, String>> taskConfigs = new ObjectMapper().readValue(requestBody, TASK_CONFIGS_TYPE);
|
||||||
|
FutureCallback<Void> cb = new FutureCallback<>();
|
||||||
|
herderForRequest().putTaskConfigs(connector, taskConfigs, cb, InternalRequestSignature.fromHeaders(Crypto.SYSTEM, requestBody, headers));
|
||||||
|
requestHandler.completeOrForwardRequest(
|
||||||
|
cb,
|
||||||
|
uriInfo.getPath(),
|
||||||
|
"POST",
|
||||||
|
headers,
|
||||||
|
taskConfigs,
|
||||||
|
forward
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@PUT
|
||||||
|
@Path("/{connector}/fence")
|
||||||
|
@Operation(hidden = true, summary = "This operation is only for inter-worker communications")
|
||||||
|
public void fenceZombies(
|
||||||
|
final @PathParam("connector") String connector,
|
||||||
|
final @Context HttpHeaders headers,
|
||||||
|
final @QueryParam("forward") Boolean forward,
|
||||||
|
final byte[] requestBody) throws Throwable {
|
||||||
|
FutureCallback<Void> cb = new FutureCallback<>();
|
||||||
|
herderForRequest().fenceZombieSourceTasks(connector, cb, InternalRequestSignature.fromHeaders(Crypto.SYSTEM, requestBody, headers));
|
||||||
|
requestHandler.completeOrForwardRequest(
|
||||||
|
cb,
|
||||||
|
uriInfo.getPath(),
|
||||||
|
"PUT",
|
||||||
|
headers,
|
||||||
|
requestBody,
|
||||||
|
forward
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,39 @@
|
||||||
|
/*
|
||||||
|
* 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.resources;
|
||||||
|
|
||||||
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
|
|
||||||
|
import javax.ws.rs.Path;
|
||||||
|
|
||||||
|
@Path("/connectors")
|
||||||
|
public class InternalConnectResource extends InternalClusterResource {
|
||||||
|
|
||||||
|
private final Herder herder;
|
||||||
|
|
||||||
|
public InternalConnectResource(Herder herder, RestClient restClient) {
|
||||||
|
super(restClient);
|
||||||
|
this.herder = herder;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Herder herderForRequest() {
|
||||||
|
return herder;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -16,10 +16,10 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.connect.runtime.rest.util;
|
package org.apache.kafka.connect.runtime.rest.util;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.config.AbstractConfig;
|
||||||
import org.apache.kafka.common.config.SslConfigs;
|
import org.apache.kafka.common.config.SslConfigs;
|
||||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
||||||
import org.apache.kafka.common.config.types.Password;
|
import org.apache.kafka.common.config.types.Password;
|
||||||
import org.apache.kafka.connect.runtime.WorkerConfig;
|
|
||||||
import org.apache.kafka.connect.runtime.rest.RestClient;
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
import org.apache.kafka.connect.runtime.rest.RestServer;
|
import org.apache.kafka.connect.runtime.rest.RestServer;
|
||||||
import org.eclipse.jetty.util.ssl.SslContextFactory;
|
import org.eclipse.jetty.util.ssl.SslContextFactory;
|
||||||
|
@ -40,7 +40,7 @@ public class SSLUtils {
|
||||||
/**
|
/**
|
||||||
* Configures SSL/TLS for HTTPS Jetty Server using configs with the given prefix
|
* Configures SSL/TLS for HTTPS Jetty Server using configs with the given prefix
|
||||||
*/
|
*/
|
||||||
public static SslContextFactory createServerSideSslContextFactory(WorkerConfig config, String prefix) {
|
public static SslContextFactory createServerSideSslContextFactory(AbstractConfig config, String prefix) {
|
||||||
Map<String, Object> sslConfigValues = config.valuesWithPrefixAllOrNothing(prefix);
|
Map<String, Object> sslConfigValues = config.valuesWithPrefixAllOrNothing(prefix);
|
||||||
|
|
||||||
final SslContextFactory.Server ssl = new SslContextFactory.Server();
|
final SslContextFactory.Server ssl = new SslContextFactory.Server();
|
||||||
|
@ -56,14 +56,14 @@ public class SSLUtils {
|
||||||
/**
|
/**
|
||||||
* Configures SSL/TLS for HTTPS Jetty Server
|
* Configures SSL/TLS for HTTPS Jetty Server
|
||||||
*/
|
*/
|
||||||
public static SslContextFactory createServerSideSslContextFactory(WorkerConfig config) {
|
public static SslContextFactory createServerSideSslContextFactory(AbstractConfig config) {
|
||||||
return createServerSideSslContextFactory(config, "listeners.https.");
|
return createServerSideSslContextFactory(config, "listeners.https.");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Configures SSL/TLS for HTTPS Jetty Client
|
* Configures SSL/TLS for HTTPS Jetty Client
|
||||||
*/
|
*/
|
||||||
public static SslContextFactory createClientSideSslContextFactory(WorkerConfig config) {
|
public static SslContextFactory createClientSideSslContextFactory(AbstractConfig config) {
|
||||||
Map<String, Object> sslConfigValues = config.valuesWithPrefixAllOrNothing("listeners.https.");
|
Map<String, Object> sslConfigValues = config.valuesWithPrefixAllOrNothing("listeners.https.");
|
||||||
|
|
||||||
final SslContextFactory.Client ssl = new SslContextFactory.Client();
|
final SslContextFactory.Client ssl = new SslContextFactory.Client();
|
||||||
|
|
|
@ -44,7 +44,7 @@ import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_C
|
||||||
import static org.apache.kafka.connect.runtime.ConnectorConfig.NAME_CONFIG;
|
import static org.apache.kafka.connect.runtime.ConnectorConfig.NAME_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
|
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG;
|
import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.REST_EXTENSION_CLASSES_CONFIG;
|
import static org.apache.kafka.connect.runtime.rest.RestServerConfig.REST_EXTENSION_CLASSES_CONFIG;
|
||||||
import static org.apache.kafka.test.TestUtils.waitForCondition;
|
import static org.apache.kafka.test.TestUtils.waitForCondition;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
|
|
@ -34,8 +34,9 @@ import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
|
||||||
import org.apache.kafka.connect.runtime.distributed.NotLeaderException;
|
import org.apache.kafka.connect.runtime.distributed.NotLeaderException;
|
||||||
import org.apache.kafka.connect.runtime.distributed.RequestTargetException;
|
import org.apache.kafka.connect.runtime.distributed.RequestTargetException;
|
||||||
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.ConnectRestServer;
|
||||||
import org.apache.kafka.connect.runtime.rest.RestClient;
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
import org.apache.kafka.connect.runtime.rest.RestServer;
|
import org.apache.kafka.connect.runtime.rest.RestServerConfig;
|
||||||
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.ConnectorType;
|
import org.apache.kafka.connect.runtime.rest.entities.ConnectorType;
|
||||||
import org.apache.kafka.connect.runtime.rest.util.SSLUtils;
|
import org.apache.kafka.connect.runtime.rest.util.SSLUtils;
|
||||||
|
@ -80,10 +81,10 @@ public class RestForwardingIntegrationTest {
|
||||||
private Map<String, Object> sslConfig;
|
private Map<String, Object> sslConfig;
|
||||||
@Mock
|
@Mock
|
||||||
private Plugins plugins;
|
private Plugins plugins;
|
||||||
private RestServer followerServer;
|
private ConnectRestServer followerServer;
|
||||||
@Mock
|
@Mock
|
||||||
private Herder followerHerder;
|
private Herder followerHerder;
|
||||||
private RestServer leaderServer;
|
private ConnectRestServer leaderServer;
|
||||||
@Mock
|
@Mock
|
||||||
private Herder leaderHerder;
|
private Herder leaderHerder;
|
||||||
|
|
||||||
|
@ -158,14 +159,14 @@ public class RestForwardingIntegrationTest {
|
||||||
|
|
||||||
// Follower worker setup
|
// Follower worker setup
|
||||||
RestClient followerClient = new RestClient(followerConfig);
|
RestClient followerClient = new RestClient(followerConfig);
|
||||||
followerServer = new RestServer(followerConfig, followerClient);
|
followerServer = new ConnectRestServer(null, followerClient, followerConfig.originals());
|
||||||
followerServer.initializeServer();
|
followerServer.initializeServer();
|
||||||
when(followerHerder.plugins()).thenReturn(plugins);
|
when(followerHerder.plugins()).thenReturn(plugins);
|
||||||
followerServer.initializeResources(followerHerder);
|
followerServer.initializeResources(followerHerder);
|
||||||
|
|
||||||
// Leader worker setup
|
// Leader worker setup
|
||||||
RestClient leaderClient = new RestClient(leaderConfig);
|
RestClient leaderClient = new RestClient(leaderConfig);
|
||||||
leaderServer = new RestServer(leaderConfig, leaderClient);
|
leaderServer = new ConnectRestServer(null, leaderClient, leaderConfig.originals());
|
||||||
leaderServer.initializeServer();
|
leaderServer.initializeServer();
|
||||||
when(leaderHerder.plugins()).thenReturn(plugins);
|
when(leaderHerder.plugins()).thenReturn(plugins);
|
||||||
leaderServer.initializeResources(leaderHerder);
|
leaderServer.initializeResources(leaderHerder);
|
||||||
|
@ -235,13 +236,13 @@ public class RestForwardingIntegrationTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (dualListener) {
|
if (dualListener) {
|
||||||
workerProps.put(WorkerConfig.LISTENERS_CONFIG, "http://localhost:0, https://localhost:0");
|
workerProps.put(RestServerConfig.LISTENERS_CONFIG, "http://localhost:0, https://localhost:0");
|
||||||
// This server is brought up with both a plaintext and an SSL listener; we use this property
|
// This server is brought up with both a plaintext and an SSL listener; we use this property
|
||||||
// to dictate which URL it advertises to other servers when a request must be forwarded to it
|
// to dictate which URL it advertises to other servers when a request must be forwarded to it
|
||||||
// and which URL we issue requests against during testing
|
// and which URL we issue requests against during testing
|
||||||
workerProps.put(WorkerConfig.REST_ADVERTISED_LISTENER_CONFIG, advertiseSSL ? "https" : "http");
|
workerProps.put(RestServerConfig.REST_ADVERTISED_LISTENER_CONFIG, advertiseSSL ? "https" : "http");
|
||||||
} else {
|
} else {
|
||||||
workerProps.put(WorkerConfig.LISTENERS_CONFIG, advertiseSSL ? "https://localhost:0" : "http://localhost:0");
|
workerProps.put(RestServerConfig.LISTENERS_CONFIG, advertiseSSL ? "https://localhost:0" : "http://localhost:0");
|
||||||
}
|
}
|
||||||
|
|
||||||
return workerProps;
|
return workerProps;
|
||||||
|
|
|
@ -19,8 +19,6 @@ package org.apache.kafka.connect.runtime;
|
||||||
import org.apache.kafka.clients.CommonClientConfigs;
|
import org.apache.kafka.clients.CommonClientConfigs;
|
||||||
import org.apache.kafka.clients.admin.MockAdminClient;
|
import org.apache.kafka.clients.admin.MockAdminClient;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.config.ConfigException;
|
|
||||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
|
||||||
import org.apache.kafka.connect.errors.ConnectException;
|
import org.apache.kafka.connect.errors.ConnectException;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
@ -33,40 +31,14 @@ import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.LISTENERS_DEFAULT;
|
|
||||||
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.assertTrue;
|
|
||||||
import static org.junit.Assert.assertThrows;
|
import static org.junit.Assert.assertThrows;
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
import static org.mockito.Mockito.mockStatic;
|
import static org.mockito.Mockito.mockStatic;
|
||||||
import static org.mockito.Mockito.times;
|
import static org.mockito.Mockito.times;
|
||||||
|
|
||||||
public class WorkerConfigTest {
|
public class WorkerConfigTest {
|
||||||
private static final List<String> VALID_HEADER_CONFIGS = Arrays.asList(
|
|
||||||
"add \t Cache-Control: no-cache, no-store, must-revalidate",
|
|
||||||
"add \r X-XSS-Protection: 1; mode=block",
|
|
||||||
"\n add Strict-Transport-Security: max-age=31536000; includeSubDomains",
|
|
||||||
"AdD Strict-Transport-Security: \r max-age=31536000; includeSubDomains",
|
|
||||||
"AdD \t Strict-Transport-Security : \n max-age=31536000; includeSubDomains",
|
|
||||||
"add X-Content-Type-Options: \r nosniff",
|
|
||||||
"Set \t X-Frame-Options: \t Deny\n ",
|
|
||||||
"seT \t X-Cache-Info: \t not cacheable\n ",
|
|
||||||
"seTDate \t Expires: \r 31540000000",
|
|
||||||
"adDdate \n Last-Modified: \t 0"
|
|
||||||
);
|
|
||||||
|
|
||||||
private static final List<String> INVALID_HEADER_CONFIGS = Arrays.asList(
|
|
||||||
"set \t",
|
|
||||||
"badaction \t X-Frame-Options:DENY",
|
|
||||||
"set add X-XSS-Protection:1",
|
|
||||||
"addX-XSS-Protection",
|
|
||||||
"X-XSS-Protection:",
|
|
||||||
"add set X-XSS-Protection: 1",
|
|
||||||
"add X-XSS-Protection:1 X-XSS-Protection:1 ",
|
|
||||||
"add X-XSS-Protection",
|
|
||||||
"set X-Frame-Options:DENY, add :no-cache, no-store, must-revalidate "
|
|
||||||
);
|
|
||||||
|
|
||||||
private static final String CLUSTER_ID = "cluster-id";
|
private static final String CLUSTER_ID = "cluster-id";
|
||||||
private MockedStatic<WorkerConfig> workerConfigMockedStatic;
|
private MockedStatic<WorkerConfig> workerConfigMockedStatic;
|
||||||
|
@ -82,105 +54,6 @@ public class WorkerConfigTest {
|
||||||
workerConfigMockedStatic.close();
|
workerConfigMockedStatic.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testListenersConfigAllowedValues() {
|
|
||||||
Map<String, String> props = baseProps();
|
|
||||||
|
|
||||||
// no value set for "listeners"
|
|
||||||
WorkerConfig config = new WorkerConfig(WorkerConfig.baseConfigDef(), props);
|
|
||||||
assertEquals(LISTENERS_DEFAULT, config.getList(WorkerConfig.LISTENERS_CONFIG));
|
|
||||||
|
|
||||||
props.put(WorkerConfig.LISTENERS_CONFIG, "http://a.b:9999");
|
|
||||||
config = new WorkerConfig(WorkerConfig.baseConfigDef(), props);
|
|
||||||
assertEquals(Arrays.asList("http://a.b:9999"), config.getList(WorkerConfig.LISTENERS_CONFIG));
|
|
||||||
|
|
||||||
props.put(WorkerConfig.LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812");
|
|
||||||
config = new WorkerConfig(WorkerConfig.baseConfigDef(), props);
|
|
||||||
assertEquals(Arrays.asList("http://a.b:9999", "https://a.b:7812"), config.getList(WorkerConfig.LISTENERS_CONFIG));
|
|
||||||
|
|
||||||
new WorkerConfig(WorkerConfig.baseConfigDef(), props);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testListenersConfigNotAllowedValues() {
|
|
||||||
Map<String, String> props = baseProps();
|
|
||||||
assertEquals(LISTENERS_DEFAULT, new WorkerConfig(WorkerConfig.baseConfigDef(), props).getList(WorkerConfig.LISTENERS_CONFIG));
|
|
||||||
|
|
||||||
props.put(WorkerConfig.LISTENERS_CONFIG, "");
|
|
||||||
ConfigException ce = assertThrows(ConfigException.class, () -> new WorkerConfig(WorkerConfig.baseConfigDef(), props));
|
|
||||||
assertTrue(ce.getMessage().contains(" listeners"));
|
|
||||||
|
|
||||||
props.put(WorkerConfig.LISTENERS_CONFIG, ",,,");
|
|
||||||
ce = assertThrows(ConfigException.class, () -> new WorkerConfig(WorkerConfig.baseConfigDef(), props));
|
|
||||||
assertTrue(ce.getMessage().contains(" listeners"));
|
|
||||||
|
|
||||||
props.put(WorkerConfig.LISTENERS_CONFIG, "http://a.b:9999,");
|
|
||||||
ce = assertThrows(ConfigException.class, () -> new WorkerConfig(WorkerConfig.baseConfigDef(), props));
|
|
||||||
assertTrue(ce.getMessage().contains(" listeners"));
|
|
||||||
|
|
||||||
props.put(WorkerConfig.LISTENERS_CONFIG, "http://a.b:9999, ,https://a.b:9999");
|
|
||||||
ce = assertThrows(ConfigException.class, () -> new WorkerConfig(WorkerConfig.baseConfigDef(), props));
|
|
||||||
assertTrue(ce.getMessage().contains(" listeners"));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testAdminListenersConfigAllowedValues() {
|
|
||||||
Map<String, String> props = baseProps();
|
|
||||||
|
|
||||||
// no value set for "admin.listeners"
|
|
||||||
WorkerConfig config = new WorkerConfig(WorkerConfig.baseConfigDef(), props);
|
|
||||||
assertNull("Default value should be null.", config.getList(WorkerConfig.ADMIN_LISTENERS_CONFIG));
|
|
||||||
|
|
||||||
props.put(WorkerConfig.ADMIN_LISTENERS_CONFIG, "");
|
|
||||||
config = new WorkerConfig(WorkerConfig.baseConfigDef(), props);
|
|
||||||
assertTrue(config.getList(WorkerConfig.ADMIN_LISTENERS_CONFIG).isEmpty());
|
|
||||||
|
|
||||||
props.put(WorkerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812");
|
|
||||||
config = new WorkerConfig(WorkerConfig.baseConfigDef(), props);
|
|
||||||
assertEquals(Arrays.asList("http://a.b:9999", "https://a.b:7812"), config.getList(WorkerConfig.ADMIN_LISTENERS_CONFIG));
|
|
||||||
|
|
||||||
new WorkerConfig(WorkerConfig.baseConfigDef(), props);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testAdminListenersNotAllowingEmptyStrings() {
|
|
||||||
Map<String, String> props = baseProps();
|
|
||||||
|
|
||||||
props.put(WorkerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999,");
|
|
||||||
ConfigException ce = assertThrows(ConfigException.class, () -> new WorkerConfig(WorkerConfig.baseConfigDef(), props));
|
|
||||||
assertTrue(ce.getMessage().contains(" admin.listeners"));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testAdminListenersNotAllowingBlankStrings() {
|
|
||||||
Map<String, String> props = baseProps();
|
|
||||||
props.put(WorkerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999, ,https://a.b:9999");
|
|
||||||
assertThrows(ConfigException.class, () -> new WorkerConfig(WorkerConfig.baseConfigDef(), props));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testInvalidHeaderConfigs() {
|
|
||||||
for (String config : INVALID_HEADER_CONFIGS) {
|
|
||||||
assertInvalidHeaderConfig(config);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testValidHeaderConfigs() {
|
|
||||||
for (String config : VALID_HEADER_CONFIGS) {
|
|
||||||
assertValidHeaderConfig(config);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testInvalidSslClientAuthConfig() {
|
|
||||||
Map<String, String> props = baseProps();
|
|
||||||
|
|
||||||
props.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "abc");
|
|
||||||
ConfigException ce = assertThrows(ConfigException.class, () -> new WorkerConfig(WorkerConfig.baseConfigDef(), props));
|
|
||||||
assertTrue(ce.getMessage().contains(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testLookupKafkaClusterId() {
|
public void testLookupKafkaClusterId() {
|
||||||
final Node broker1 = new Node(0, "dummyHost-1", 1234);
|
final Node broker1 = new Node(0, "dummyHost-1", 1234);
|
||||||
|
@ -225,14 +98,6 @@ public class WorkerConfigTest {
|
||||||
workerConfigMockedStatic.verify(() -> WorkerConfig.lookupKafkaClusterId(any(WorkerConfig.class)), times(1));
|
workerConfigMockedStatic.verify(() -> WorkerConfig.lookupKafkaClusterId(any(WorkerConfig.class)), times(1));
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertInvalidHeaderConfig(String config) {
|
|
||||||
assertThrows(ConfigException.class, () -> WorkerConfig.validateHttpResponseHeaderConfig(config));
|
|
||||||
}
|
|
||||||
|
|
||||||
private void assertValidHeaderConfig(String config) {
|
|
||||||
WorkerConfig.validateHttpResponseHeaderConfig(config);
|
|
||||||
}
|
|
||||||
|
|
||||||
private Map<String, String> baseProps() {
|
private Map<String, String> baseProps() {
|
||||||
Map<String, String> props = new HashMap<>();
|
Map<String, String> props = new HashMap<>();
|
||||||
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
||||||
|
|
|
@ -244,7 +244,7 @@ public class DistributedHerderTest {
|
||||||
new String[]{"connectorType", "updateDeletedConnectorStatus", "updateDeletedTaskStatus", "validateConnectorConfig", "buildRestartPlan", "recordRestarting"},
|
new String[]{"connectorType", "updateDeletedConnectorStatus", "updateDeletedTaskStatus", "validateConnectorConfig", "buildRestartPlan", "recordRestarting"},
|
||||||
new DistributedConfig(HERDER_CONFIG), worker, WORKER_ID, KAFKA_CLUSTER_ID,
|
new DistributedConfig(HERDER_CONFIG), worker, WORKER_ID, KAFKA_CLUSTER_ID,
|
||||||
statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, noneConnectorClientConfigOverridePolicy,
|
statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, noneConnectorClientConfigOverridePolicy,
|
||||||
new AutoCloseable[]{uponShutdown});
|
Collections.emptyList(), new AutoCloseable[]{uponShutdown});
|
||||||
|
|
||||||
configUpdateListener = herder.new ConfigUpdateListener();
|
configUpdateListener = herder.new ConfigUpdateListener();
|
||||||
rebalanceListener = herder.new RebalanceListener(time);
|
rebalanceListener = herder.new RebalanceListener(time);
|
||||||
|
@ -4020,7 +4020,7 @@ public class DistributedHerderTest {
|
||||||
new String[]{"connectorType", "updateDeletedConnectorStatus", "updateDeletedTaskStatus", "validateConnectorConfig"},
|
new String[]{"connectorType", "updateDeletedConnectorStatus", "updateDeletedTaskStatus", "validateConnectorConfig"},
|
||||||
new DistributedConfig(config), worker, WORKER_ID, KAFKA_CLUSTER_ID,
|
new DistributedConfig(config), worker, WORKER_ID, KAFKA_CLUSTER_ID,
|
||||||
statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, noneConnectorClientConfigOverridePolicy,
|
statusBackingStore, configBackingStore, member, MEMBER_URL, restClient, metrics, time, noneConnectorClientConfigOverridePolicy,
|
||||||
new AutoCloseable[0]);
|
Collections.emptyList(), new AutoCloseable[0]);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.kafka.connect.rest.ConnectRestExtensionContext;
|
||||||
import org.apache.kafka.connect.runtime.WorkerConfig;
|
import org.apache.kafka.connect.runtime.WorkerConfig;
|
||||||
import org.apache.kafka.connect.runtime.isolation.Plugins.ClassLoaderUsage;
|
import org.apache.kafka.connect.runtime.isolation.Plugins.ClassLoaderUsage;
|
||||||
import org.apache.kafka.connect.runtime.isolation.TestPlugins.TestPlugin;
|
import org.apache.kafka.connect.runtime.isolation.TestPlugins.TestPlugin;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestServerConfig;
|
||||||
import org.apache.kafka.connect.storage.Converter;
|
import org.apache.kafka.connect.storage.Converter;
|
||||||
import org.apache.kafka.connect.storage.ConverterConfig;
|
import org.apache.kafka.connect.storage.ConverterConfig;
|
||||||
import org.apache.kafka.connect.storage.ConverterType;
|
import org.apache.kafka.connect.storage.ConverterType;
|
||||||
|
@ -140,12 +141,13 @@ public class PluginsTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldInstantiateAndConfigureConnectRestExtension() {
|
public void shouldInstantiateAndConfigureConnectRestExtension() {
|
||||||
props.put(WorkerConfig.REST_EXTENSION_CLASSES_CONFIG,
|
props.clear();
|
||||||
|
props.put(RestServerConfig.REST_EXTENSION_CLASSES_CONFIG,
|
||||||
TestConnectRestExtension.class.getName());
|
TestConnectRestExtension.class.getName());
|
||||||
createConfig();
|
config = RestServerConfig.forPublic(null, props);
|
||||||
|
|
||||||
List<ConnectRestExtension> connectRestExtensions =
|
List<ConnectRestExtension> connectRestExtensions =
|
||||||
plugins.newPlugins(config.getList(WorkerConfig.REST_EXTENSION_CLASSES_CONFIG),
|
plugins.newPlugins(config.getList(RestServerConfig.REST_EXTENSION_CLASSES_CONFIG),
|
||||||
config,
|
config,
|
||||||
ConnectRestExtension.class);
|
ConnectRestExtension.class);
|
||||||
assertNotNull(connectRestExtensions);
|
assertNotNull(connectRestExtensions);
|
||||||
|
|
|
@ -29,14 +29,11 @@ import org.apache.http.entity.StringEntity;
|
||||||
import org.apache.http.impl.client.BasicResponseHandler;
|
import org.apache.http.impl.client.BasicResponseHandler;
|
||||||
import org.apache.http.impl.client.CloseableHttpClient;
|
import org.apache.http.impl.client.CloseableHttpClient;
|
||||||
import org.apache.http.impl.client.HttpClients;
|
import org.apache.http.impl.client.HttpClients;
|
||||||
import org.apache.kafka.clients.CommonClientConfigs;
|
import org.apache.kafka.common.config.AbstractConfig;
|
||||||
import org.apache.kafka.common.utils.LogCaptureAppender;
|
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.WorkerConfig;
|
|
||||||
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
|
|
||||||
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
import org.apache.kafka.connect.runtime.isolation.Plugins;
|
||||||
import org.apache.kafka.connect.runtime.standalone.StandaloneConfig;
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
@ -55,20 +52,21 @@ import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.ADMIN_LISTENERS_CONFIG;
|
|
||||||
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.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.eq;
|
||||||
import static org.mockito.Mockito.doReturn;
|
import static org.mockito.Mockito.doReturn;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
public class RestServerTest {
|
public class ConnectRestServerTest {
|
||||||
|
|
||||||
private Herder herder;
|
private Herder herder;
|
||||||
private Plugins plugins;
|
private Plugins plugins;
|
||||||
private RestServer server;
|
private ConnectRestServer server;
|
||||||
private CloseableHttpClient httpClient;
|
private CloseableHttpClient httpClient;
|
||||||
private Collection<CloseableHttpResponse> responses = new ArrayList<>();
|
private Collection<CloseableHttpResponse> responses = new ArrayList<>();
|
||||||
|
|
||||||
|
@ -94,18 +92,10 @@ public class RestServerTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<String, String> baseWorkerProps() {
|
private Map<String, String> baseServerProps() {
|
||||||
Map<String, String> workerProps = new HashMap<>();
|
Map<String, String> configMap = new HashMap<>();
|
||||||
workerProps.put(DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "status-topic");
|
configMap.put(RestServerConfig.LISTENERS_CONFIG, "HTTP://localhost:0");
|
||||||
workerProps.put(DistributedConfig.CONFIG_TOPIC_CONFIG, "config-topic");
|
return configMap;
|
||||||
workerProps.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
|
||||||
workerProps.put(DistributedConfig.GROUP_ID_CONFIG, "connect-test-group");
|
|
||||||
workerProps.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
|
|
||||||
workerProps.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
|
|
||||||
workerProps.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "connect-offsets");
|
|
||||||
workerProps.put(WorkerConfig.LISTENERS_CONFIG, "HTTP://localhost:0");
|
|
||||||
|
|
||||||
return workerProps;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -121,65 +111,60 @@ public class RestServerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testAdvertisedUri() {
|
public void testAdvertisedUri() {
|
||||||
// Advertised URI from listeners without protocol
|
// Advertised URI from listeners without protocol
|
||||||
Map<String, String> configMap = new HashMap<>(baseWorkerProps());
|
Map<String, String> configMap = new HashMap<>(baseServerProps());
|
||||||
configMap.put(WorkerConfig.LISTENERS_CONFIG, "http://localhost:8080,https://localhost:8443");
|
configMap.put(RestServerConfig.LISTENERS_CONFIG, "http://localhost:8080,https://localhost:8443");
|
||||||
DistributedConfig config = new DistributedConfig(configMap);
|
|
||||||
|
|
||||||
server = new RestServer(config, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
Assert.assertEquals("http://localhost:8080/", server.advertisedUrl().toString());
|
Assert.assertEquals("http://localhost:8080/", server.advertisedUrl().toString());
|
||||||
server.stop();
|
server.stop();
|
||||||
|
|
||||||
// Advertised URI from listeners with protocol
|
// Advertised URI from listeners with protocol
|
||||||
configMap = new HashMap<>(baseWorkerProps());
|
configMap = new HashMap<>(baseServerProps());
|
||||||
configMap.put(WorkerConfig.LISTENERS_CONFIG, "http://localhost:8080,https://localhost:8443");
|
configMap.put(RestServerConfig.LISTENERS_CONFIG, "http://localhost:8080,https://localhost:8443");
|
||||||
configMap.put(WorkerConfig.REST_ADVERTISED_LISTENER_CONFIG, "https");
|
configMap.put(RestServerConfig.REST_ADVERTISED_LISTENER_CONFIG, "https");
|
||||||
config = new DistributedConfig(configMap);
|
|
||||||
|
|
||||||
server = new RestServer(config, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
Assert.assertEquals("https://localhost:8443/", server.advertisedUrl().toString());
|
Assert.assertEquals("https://localhost:8443/", server.advertisedUrl().toString());
|
||||||
server.stop();
|
server.stop();
|
||||||
|
|
||||||
// Advertised URI from listeners with only SSL available
|
// Advertised URI from listeners with only SSL available
|
||||||
configMap = new HashMap<>(baseWorkerProps());
|
configMap = new HashMap<>(baseServerProps());
|
||||||
configMap.put(WorkerConfig.LISTENERS_CONFIG, "https://localhost:8443");
|
configMap.put(RestServerConfig.LISTENERS_CONFIG, "https://localhost:8443");
|
||||||
config = new DistributedConfig(configMap);
|
|
||||||
|
|
||||||
server = new RestServer(config, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
Assert.assertEquals("https://localhost:8443/", server.advertisedUrl().toString());
|
Assert.assertEquals("https://localhost:8443/", server.advertisedUrl().toString());
|
||||||
server.stop();
|
server.stop();
|
||||||
|
|
||||||
// Listener is overriden by advertised values
|
// Listener is overriden by advertised values
|
||||||
configMap = new HashMap<>(baseWorkerProps());
|
configMap = new HashMap<>(baseServerProps());
|
||||||
configMap.put(WorkerConfig.LISTENERS_CONFIG, "https://localhost:8443");
|
configMap.put(RestServerConfig.LISTENERS_CONFIG, "https://localhost:8443");
|
||||||
configMap.put(WorkerConfig.REST_ADVERTISED_LISTENER_CONFIG, "http");
|
configMap.put(RestServerConfig.REST_ADVERTISED_LISTENER_CONFIG, "http");
|
||||||
configMap.put(WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG, "somehost");
|
configMap.put(RestServerConfig.REST_ADVERTISED_HOST_NAME_CONFIG, "somehost");
|
||||||
configMap.put(WorkerConfig.REST_ADVERTISED_PORT_CONFIG, "10000");
|
configMap.put(RestServerConfig.REST_ADVERTISED_PORT_CONFIG, "10000");
|
||||||
config = new DistributedConfig(configMap);
|
|
||||||
|
|
||||||
server = new RestServer(config, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
Assert.assertEquals("http://somehost:10000/", server.advertisedUrl().toString());
|
Assert.assertEquals("http://somehost:10000/", server.advertisedUrl().toString());
|
||||||
server.stop();
|
server.stop();
|
||||||
|
|
||||||
// correct listener is chosen when https listener is configured before http listener and advertised listener is http
|
// correct listener is chosen when https listener is configured before http listener and advertised listener is http
|
||||||
configMap = new HashMap<>(baseWorkerProps());
|
configMap = new HashMap<>(baseServerProps());
|
||||||
configMap.put(WorkerConfig.LISTENERS_CONFIG, "https://encrypted-localhost:42069,http://plaintext-localhost:4761");
|
configMap.put(RestServerConfig.LISTENERS_CONFIG, "https://encrypted-localhost:42069,http://plaintext-localhost:4761");
|
||||||
configMap.put(WorkerConfig.REST_ADVERTISED_LISTENER_CONFIG, "http");
|
configMap.put(RestServerConfig.REST_ADVERTISED_LISTENER_CONFIG, "http");
|
||||||
config = new DistributedConfig(configMap);
|
|
||||||
server = new RestServer(config, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
Assert.assertEquals("http://plaintext-localhost:4761/", server.advertisedUrl().toString());
|
Assert.assertEquals("http://plaintext-localhost:4761/", server.advertisedUrl().toString());
|
||||||
server.stop();
|
server.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testOptionsDoesNotIncludeWadlOutput() throws IOException {
|
public void testOptionsDoesNotIncludeWadlOutput() throws IOException {
|
||||||
Map<String, String> configMap = new HashMap<>(baseWorkerProps());
|
Map<String, String> configMap = new HashMap<>(baseServerProps());
|
||||||
DistributedConfig workerConfig = new DistributedConfig(configMap);
|
|
||||||
|
|
||||||
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
||||||
doReturn(plugins).when(herder).plugins();
|
doReturn(plugins).when(herder).plugins();
|
||||||
doReturn(Collections.emptyList()).when(plugins).newPlugins(Collections.emptyList(), workerConfig, ConnectRestExtension.class);
|
expectEmptyRestExtensions();
|
||||||
|
|
||||||
server = new RestServer(workerConfig, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
server.initializeServer();
|
server.initializeServer();
|
||||||
server.initializeResources(herder);
|
server.initializeResources(herder);
|
||||||
|
|
||||||
|
@ -197,17 +182,16 @@ public class RestServerTest {
|
||||||
|
|
||||||
public void checkCORSRequest(String corsDomain, String origin, String expectedHeader, String method)
|
public void checkCORSRequest(String corsDomain, String origin, String expectedHeader, String method)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Map<String, String> workerProps = baseWorkerProps();
|
Map<String, String> configMap = baseServerProps();
|
||||||
workerProps.put(WorkerConfig.ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, corsDomain);
|
configMap.put(RestServerConfig.ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, corsDomain);
|
||||||
workerProps.put(WorkerConfig.ACCESS_CONTROL_ALLOW_METHODS_CONFIG, method);
|
configMap.put(RestServerConfig.ACCESS_CONTROL_ALLOW_METHODS_CONFIG, method);
|
||||||
WorkerConfig workerConfig = new DistributedConfig(workerProps);
|
|
||||||
|
|
||||||
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
||||||
doReturn(plugins).when(herder).plugins();
|
doReturn(plugins).when(herder).plugins();
|
||||||
doReturn(Collections.emptyList()).when(plugins).newPlugins(Collections.emptyList(), workerConfig, ConnectRestExtension.class);
|
expectEmptyRestExtensions();
|
||||||
doReturn(Arrays.asList("a", "b")).when(herder).connectors();
|
doReturn(Arrays.asList("a", "b")).when(herder).connectors();
|
||||||
|
|
||||||
server = new RestServer(workerConfig, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
server.initializeServer();
|
server.initializeServer();
|
||||||
server.initializeResources(herder);
|
server.initializeResources(herder);
|
||||||
URI serverUrl = server.advertisedUrl();
|
URI serverUrl = server.advertisedUrl();
|
||||||
|
@ -242,16 +226,15 @@ public class RestServerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testStandaloneConfig() throws IOException {
|
public void testStandaloneConfig() throws IOException {
|
||||||
Map<String, String> workerProps = baseWorkerProps();
|
Map<String, String> configMap = baseServerProps();
|
||||||
workerProps.put("offset.storage.file.filename", "/tmp");
|
configMap.put("offset.storage.file.filename", "/tmp");
|
||||||
WorkerConfig workerConfig = new StandaloneConfig(workerProps);
|
|
||||||
|
|
||||||
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
||||||
doReturn(plugins).when(herder).plugins();
|
doReturn(plugins).when(herder).plugins();
|
||||||
doReturn(Collections.emptyList()).when(plugins).newPlugins(Collections.emptyList(), workerConfig, ConnectRestExtension.class);
|
expectEmptyRestExtensions();
|
||||||
doReturn(Arrays.asList("a", "b")).when(herder).connectors();
|
doReturn(Arrays.asList("a", "b")).when(herder).connectors();
|
||||||
|
|
||||||
server = new RestServer(workerConfig, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
server.initializeServer();
|
server.initializeServer();
|
||||||
server.initializeResources(herder);
|
server.initializeResources(herder);
|
||||||
HttpRequest request = new HttpGet("/connectors");
|
HttpRequest request = new HttpGet("/connectors");
|
||||||
|
@ -262,17 +245,16 @@ public class RestServerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testLoggersEndpointWithDefaults() throws IOException {
|
public void testLoggersEndpointWithDefaults() throws IOException {
|
||||||
Map<String, String> configMap = new HashMap<>(baseWorkerProps());
|
Map<String, String> configMap = new HashMap<>(baseServerProps());
|
||||||
DistributedConfig workerConfig = new DistributedConfig(configMap);
|
|
||||||
|
|
||||||
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
||||||
doReturn(plugins).when(herder).plugins();
|
doReturn(plugins).when(herder).plugins();
|
||||||
doReturn(Collections.emptyList()).when(plugins).newPlugins(Collections.emptyList(), workerConfig, ConnectRestExtension.class);
|
expectEmptyRestExtensions();
|
||||||
|
|
||||||
// create some loggers in the process
|
// create some loggers in the process
|
||||||
LoggerFactory.getLogger("a.b.c.s.W");
|
LoggerFactory.getLogger("a.b.c.s.W");
|
||||||
|
|
||||||
server = new RestServer(workerConfig, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
server.initializeServer();
|
server.initializeServer();
|
||||||
server.initializeResources(herder);
|
server.initializeResources(herder);
|
||||||
|
|
||||||
|
@ -292,14 +274,12 @@ public class RestServerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIndependentAdminEndpoint() throws IOException {
|
public void testIndependentAdminEndpoint() throws IOException {
|
||||||
Map<String, String> configMap = new HashMap<>(baseWorkerProps());
|
Map<String, String> configMap = new HashMap<>(baseServerProps());
|
||||||
configMap.put(ADMIN_LISTENERS_CONFIG, "http://localhost:0");
|
configMap.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "http://localhost:0");
|
||||||
|
|
||||||
DistributedConfig workerConfig = new DistributedConfig(configMap);
|
|
||||||
|
|
||||||
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
||||||
doReturn(plugins).when(herder).plugins();
|
doReturn(plugins).when(herder).plugins();
|
||||||
doReturn(Collections.emptyList()).when(plugins).newPlugins(Collections.emptyList(), workerConfig, ConnectRestExtension.class);
|
expectEmptyRestExtensions();
|
||||||
|
|
||||||
// create some loggers in the process
|
// create some loggers in the process
|
||||||
LoggerFactory.getLogger("a.b.c.s.W");
|
LoggerFactory.getLogger("a.b.c.s.W");
|
||||||
|
@ -307,7 +287,7 @@ public class RestServerTest {
|
||||||
LoggerFactory.getLogger("a.b.c.p.Y");
|
LoggerFactory.getLogger("a.b.c.p.Y");
|
||||||
LoggerFactory.getLogger("a.b.c.p.Z");
|
LoggerFactory.getLogger("a.b.c.p.Z");
|
||||||
|
|
||||||
server = new RestServer(workerConfig, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
server.initializeServer();
|
server.initializeServer();
|
||||||
server.initializeResources(herder);
|
server.initializeResources(herder);
|
||||||
|
|
||||||
|
@ -322,16 +302,14 @@ public class RestServerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDisableAdminEndpoint() throws IOException {
|
public void testDisableAdminEndpoint() throws IOException {
|
||||||
Map<String, String> configMap = new HashMap<>(baseWorkerProps());
|
Map<String, String> configMap = new HashMap<>(baseServerProps());
|
||||||
configMap.put(ADMIN_LISTENERS_CONFIG, "");
|
configMap.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "");
|
||||||
|
|
||||||
DistributedConfig workerConfig = new DistributedConfig(configMap);
|
|
||||||
|
|
||||||
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
||||||
doReturn(plugins).when(herder).plugins();
|
doReturn(plugins).when(herder).plugins();
|
||||||
doReturn(Collections.emptyList()).when(plugins).newPlugins(Collections.emptyList(), workerConfig, ConnectRestExtension.class);
|
expectEmptyRestExtensions();
|
||||||
|
|
||||||
server = new RestServer(workerConfig, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
server.initializeServer();
|
server.initializeServer();
|
||||||
server.initializeResources(herder);
|
server.initializeResources(herder);
|
||||||
|
|
||||||
|
@ -344,14 +322,13 @@ public class RestServerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRequestLogs() throws IOException, InterruptedException {
|
public void testRequestLogs() throws IOException, InterruptedException {
|
||||||
Map<String, String> configMap = new HashMap<>(baseWorkerProps());
|
Map<String, String> configMap = new HashMap<>(baseServerProps());
|
||||||
DistributedConfig workerConfig = new DistributedConfig(configMap);
|
|
||||||
|
|
||||||
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
||||||
doReturn(plugins).when(herder).plugins();
|
doReturn(plugins).when(herder).plugins();
|
||||||
doReturn(Collections.emptyList()).when(plugins).newPlugins(Collections.emptyList(), workerConfig, ConnectRestExtension.class);
|
expectEmptyRestExtensions();
|
||||||
|
|
||||||
server = new RestServer(workerConfig, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
server.initializeServer();
|
server.initializeServer();
|
||||||
server.initializeResources(herder);
|
server.initializeResources(herder);
|
||||||
|
|
||||||
|
@ -388,17 +365,16 @@ public class RestServerTest {
|
||||||
|
|
||||||
private void checkCustomizedHttpResponseHeaders(String headerConfig, Map<String, String> expectedHeaders)
|
private void checkCustomizedHttpResponseHeaders(String headerConfig, Map<String, String> expectedHeaders)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Map<String, String> workerProps = baseWorkerProps();
|
Map<String, String> configMap = baseServerProps();
|
||||||
workerProps.put("offset.storage.file.filename", "/tmp");
|
configMap.put("offset.storage.file.filename", "/tmp");
|
||||||
workerProps.put(WorkerConfig.RESPONSE_HTTP_HEADERS_CONFIG, headerConfig);
|
configMap.put(RestServerConfig.RESPONSE_HTTP_HEADERS_CONFIG, headerConfig);
|
||||||
WorkerConfig workerConfig = new DistributedConfig(workerProps);
|
|
||||||
|
|
||||||
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
|
||||||
doReturn(plugins).when(herder).plugins();
|
doReturn(plugins).when(herder).plugins();
|
||||||
doReturn(Collections.emptyList()).when(plugins).newPlugins(Collections.emptyList(), workerConfig, ConnectRestExtension.class);
|
expectEmptyRestExtensions();
|
||||||
doReturn(Arrays.asList("a", "b")).when(herder).connectors();
|
doReturn(Arrays.asList("a", "b")).when(herder).connectors();
|
||||||
|
|
||||||
server = new RestServer(workerConfig, null);
|
server = new ConnectRestServer(null, null, configMap);
|
||||||
server.initializeServer();
|
server.initializeServer();
|
||||||
server.initializeResources(herder);
|
server.initializeResources(herder);
|
||||||
HttpRequest request = new HttpGet("/connectors");
|
HttpRequest request = new HttpGet("/connectors");
|
||||||
|
@ -442,4 +418,12 @@ public class RestServerTest {
|
||||||
ObjectMapper mapper = new ObjectMapper();
|
ObjectMapper mapper = new ObjectMapper();
|
||||||
return mapper.writerWithDefaultPrettyPrinter().writeValueAsString(map);
|
return mapper.writerWithDefaultPrettyPrinter().writeValueAsString(map);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void expectEmptyRestExtensions() {
|
||||||
|
doReturn(Collections.emptyList()).when(plugins).newPlugins(
|
||||||
|
eq(Collections.emptyList()),
|
||||||
|
any(AbstractConfig.class),
|
||||||
|
eq(ConnectRestExtension.class)
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -0,0 +1,167 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.config.ConfigException;
|
||||||
|
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.apache.kafka.connect.runtime.rest.RestServerConfig.LISTENERS_DEFAULT;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNull;
|
||||||
|
import static org.junit.Assert.assertThrows;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
public class RestServerConfigTest {
|
||||||
|
|
||||||
|
private static final List<String> VALID_HEADER_CONFIGS = Arrays.asList(
|
||||||
|
"add \t Cache-Control: no-cache, no-store, must-revalidate",
|
||||||
|
"add \r X-XSS-Protection: 1; mode=block",
|
||||||
|
"\n add Strict-Transport-Security: max-age=31536000; includeSubDomains",
|
||||||
|
"AdD Strict-Transport-Security: \r max-age=31536000; includeSubDomains",
|
||||||
|
"AdD \t Strict-Transport-Security : \n max-age=31536000; includeSubDomains",
|
||||||
|
"add X-Content-Type-Options: \r nosniff",
|
||||||
|
"Set \t X-Frame-Options: \t Deny\n ",
|
||||||
|
"seT \t X-Cache-Info: \t not cacheable\n ",
|
||||||
|
"seTDate \t Expires: \r 31540000000",
|
||||||
|
"adDdate \n Last-Modified: \t 0"
|
||||||
|
);
|
||||||
|
|
||||||
|
private static final List<String> INVALID_HEADER_CONFIGS = Arrays.asList(
|
||||||
|
"set \t",
|
||||||
|
"badaction \t X-Frame-Options:DENY",
|
||||||
|
"set add X-XSS-Protection:1",
|
||||||
|
"addX-XSS-Protection",
|
||||||
|
"X-XSS-Protection:",
|
||||||
|
"add set X-XSS-Protection: 1",
|
||||||
|
"add X-XSS-Protection:1 X-XSS-Protection:1 ",
|
||||||
|
"add X-XSS-Protection",
|
||||||
|
"set X-Frame-Options:DENY, add :no-cache, no-store, must-revalidate "
|
||||||
|
);
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testListenersConfigAllowedValues() {
|
||||||
|
Map<String, String> props = new HashMap<>();
|
||||||
|
|
||||||
|
// no value set for "listeners"
|
||||||
|
RestServerConfig config = RestServerConfig.forPublic(null, props);
|
||||||
|
assertEquals(LISTENERS_DEFAULT, config.listeners());
|
||||||
|
|
||||||
|
props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999");
|
||||||
|
config = RestServerConfig.forPublic(null, props);
|
||||||
|
assertEquals(Arrays.asList("http://a.b:9999"), config.listeners());
|
||||||
|
|
||||||
|
props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812");
|
||||||
|
config = RestServerConfig.forPublic(null, props);
|
||||||
|
assertEquals(Arrays.asList("http://a.b:9999", "https://a.b:7812"), config.listeners());
|
||||||
|
|
||||||
|
config = RestServerConfig.forPublic(null, props);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testListenersConfigNotAllowedValues() {
|
||||||
|
Map<String, String> props = new HashMap<>();
|
||||||
|
assertEquals(LISTENERS_DEFAULT, RestServerConfig.forPublic(null, props).listeners());
|
||||||
|
|
||||||
|
props.put(RestServerConfig.LISTENERS_CONFIG, "");
|
||||||
|
ConfigException ce = assertThrows(ConfigException.class, () -> RestServerConfig.forPublic(null, props));
|
||||||
|
assertTrue(ce.getMessage().contains(" listeners"));
|
||||||
|
|
||||||
|
props.put(RestServerConfig.LISTENERS_CONFIG, ",,,");
|
||||||
|
ce = assertThrows(ConfigException.class, () -> RestServerConfig.forPublic(null, props));
|
||||||
|
assertTrue(ce.getMessage().contains(" listeners"));
|
||||||
|
|
||||||
|
props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999,");
|
||||||
|
ce = assertThrows(ConfigException.class, () -> RestServerConfig.forPublic(null, props));
|
||||||
|
assertTrue(ce.getMessage().contains(" listeners"));
|
||||||
|
|
||||||
|
props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999, ,https://a.b:9999");
|
||||||
|
ce = assertThrows(ConfigException.class, () -> RestServerConfig.forPublic(null, props));
|
||||||
|
assertTrue(ce.getMessage().contains(" listeners"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAdminListenersConfigAllowedValues() {
|
||||||
|
Map<String, String> props = new HashMap<>();
|
||||||
|
|
||||||
|
// no value set for "admin.listeners"
|
||||||
|
RestServerConfig config = RestServerConfig.forPublic(null, props);
|
||||||
|
assertNull("Default value should be null.", config.adminListeners());
|
||||||
|
|
||||||
|
props.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "");
|
||||||
|
config = RestServerConfig.forPublic(null, props);
|
||||||
|
assertTrue(config.adminListeners().isEmpty());
|
||||||
|
|
||||||
|
props.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812");
|
||||||
|
config = RestServerConfig.forPublic(null, props);
|
||||||
|
assertEquals(Arrays.asList("http://a.b:9999", "https://a.b:7812"), config.adminListeners());
|
||||||
|
|
||||||
|
RestServerConfig.forPublic(null, props);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAdminListenersNotAllowingEmptyStrings() {
|
||||||
|
Map<String, String> props = new HashMap<>();
|
||||||
|
|
||||||
|
props.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999,");
|
||||||
|
ConfigException ce = assertThrows(ConfigException.class, () -> RestServerConfig.forPublic(null, props));
|
||||||
|
assertTrue(ce.getMessage().contains(" admin.listeners"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAdminListenersNotAllowingBlankStrings() {
|
||||||
|
Map<String, String> props = new HashMap<>();
|
||||||
|
props.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999, ,https://a.b:9999");
|
||||||
|
assertThrows(ConfigException.class, () -> RestServerConfig.forPublic(null, props));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInvalidHeaderConfigs() {
|
||||||
|
for (String config : INVALID_HEADER_CONFIGS) {
|
||||||
|
assertInvalidHeaderConfig(config);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValidHeaderConfigs() {
|
||||||
|
for (String config : VALID_HEADER_CONFIGS) {
|
||||||
|
assertValidHeaderConfig(config);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertInvalidHeaderConfig(String config) {
|
||||||
|
assertThrows(ConfigException.class, () -> RestServerConfig.validateHttpResponseHeaderConfig(config));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertValidHeaderConfig(String config) {
|
||||||
|
RestServerConfig.validateHttpResponseHeaderConfig(config);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInvalidSslClientAuthConfig() {
|
||||||
|
Map<String, String> props = new HashMap<>();
|
||||||
|
|
||||||
|
props.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "abc");
|
||||||
|
ConfigException ce = assertThrows(ConfigException.class, () -> RestServerConfig.forPublic(null, props));
|
||||||
|
assertTrue(ce.getMessage().contains(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG));
|
||||||
|
}
|
||||||
|
}
|
|
@ -16,7 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.connect.runtime.rest.resources;
|
package org.apache.kafka.connect.runtime.rest.resources;
|
||||||
|
|
||||||
import javax.crypto.Mac;
|
|
||||||
import javax.ws.rs.core.HttpHeaders;
|
import javax.ws.rs.core.HttpHeaders;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
@ -26,12 +25,11 @@ import org.apache.kafka.connect.runtime.AbstractStatus;
|
||||||
import org.apache.kafka.connect.runtime.ConnectorConfig;
|
import org.apache.kafka.connect.runtime.ConnectorConfig;
|
||||||
import org.apache.kafka.connect.runtime.Herder;
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
import org.apache.kafka.connect.runtime.RestartRequest;
|
import org.apache.kafka.connect.runtime.RestartRequest;
|
||||||
import org.apache.kafka.connect.runtime.WorkerConfig;
|
|
||||||
import org.apache.kafka.connect.runtime.distributed.NotAssignedException;
|
import org.apache.kafka.connect.runtime.distributed.NotAssignedException;
|
||||||
import org.apache.kafka.connect.runtime.distributed.NotLeaderException;
|
import org.apache.kafka.connect.runtime.distributed.NotLeaderException;
|
||||||
import org.apache.kafka.connect.runtime.distributed.RebalanceNeededException;
|
import org.apache.kafka.connect.runtime.distributed.RebalanceNeededException;
|
||||||
import org.apache.kafka.connect.runtime.rest.InternalRequestSignature;
|
|
||||||
import org.apache.kafka.connect.runtime.rest.RestClient;
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestServerConfig;
|
||||||
import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo;
|
import org.apache.kafka.connect.runtime.rest.entities.ActiveTopicsInfo;
|
||||||
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.ConnectorStateInfo;
|
import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
|
||||||
|
@ -61,7 +59,6 @@ import java.io.IOException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Base64;
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -70,8 +67,6 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ALLOW_RESET_CONFIG;
|
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG;
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertThrows;
|
import static org.junit.Assert.assertThrows;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
@ -153,15 +148,15 @@ public class ConnectorsResourceTest {
|
||||||
private ConnectorsResource connectorsResource;
|
private ConnectorsResource connectorsResource;
|
||||||
private UriInfo forward;
|
private UriInfo forward;
|
||||||
@Mock
|
@Mock
|
||||||
private WorkerConfig workerConfig;
|
|
||||||
@Mock
|
|
||||||
private RestClient restClient;
|
private RestClient restClient;
|
||||||
|
@Mock
|
||||||
|
private RestServerConfig serverConfig;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws NoSuchMethodException {
|
public void setUp() throws NoSuchMethodException {
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG)).thenReturn(true);
|
when(serverConfig.topicTrackingEnabled()).thenReturn(true);
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ALLOW_RESET_CONFIG)).thenReturn(true);
|
when(serverConfig.topicTrackingResetEnabled()).thenReturn(true);
|
||||||
connectorsResource = new ConnectorsResource(herder, workerConfig, restClient);
|
connectorsResource = new ConnectorsResource(herder, serverConfig, restClient);
|
||||||
forward = mock(UriInfo.class);
|
forward = mock(UriInfo.class);
|
||||||
MultivaluedMap<String, String> queryParams = new MultivaluedHashMap<>();
|
MultivaluedMap<String, String> queryParams = new MultivaluedHashMap<>();
|
||||||
queryParams.putSingle("forward", "true");
|
queryParams.putSingle("forward", "true");
|
||||||
|
@ -565,66 +560,6 @@ public class ConnectorsResourceTest {
|
||||||
assertThrows(NotFoundException.class, () -> connectorsResource.getTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD));
|
assertThrows(NotFoundException.class, () -> connectorsResource.getTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testPutConnectorTaskConfigsNoInternalRequestSignature() throws Throwable {
|
|
||||||
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
|
||||||
expectAndCallbackResult(cb, null).when(herder).putTaskConfigs(
|
|
||||||
eq(CONNECTOR_NAME),
|
|
||||||
eq(TASK_CONFIGS),
|
|
||||||
cb.capture(),
|
|
||||||
any()
|
|
||||||
);
|
|
||||||
|
|
||||||
connectorsResource.putTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD, serializeAsBytes(TASK_CONFIGS));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testPutConnectorTaskConfigsWithInternalRequestSignature() throws Throwable {
|
|
||||||
final String signatureAlgorithm = "HmacSHA256";
|
|
||||||
final String encodedSignature = "Kv1/OSsxzdVIwvZ4e30avyRIVrngDfhzVUm/kAZEKc4=";
|
|
||||||
|
|
||||||
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
|
||||||
final ArgumentCaptor<InternalRequestSignature> signatureCapture = ArgumentCaptor.forClass(InternalRequestSignature.class);
|
|
||||||
expectAndCallbackResult(cb, null).when(herder).putTaskConfigs(
|
|
||||||
eq(CONNECTOR_NAME),
|
|
||||||
eq(TASK_CONFIGS),
|
|
||||||
cb.capture(),
|
|
||||||
signatureCapture.capture()
|
|
||||||
);
|
|
||||||
|
|
||||||
HttpHeaders headers = mock(HttpHeaders.class);
|
|
||||||
when(headers.getHeaderString(InternalRequestSignature.SIGNATURE_ALGORITHM_HEADER))
|
|
||||||
.thenReturn(signatureAlgorithm);
|
|
||||||
when(headers.getHeaderString(InternalRequestSignature.SIGNATURE_HEADER))
|
|
||||||
.thenReturn(encodedSignature);
|
|
||||||
|
|
||||||
connectorsResource.putTaskConfigs(CONNECTOR_NAME, headers, FORWARD, serializeAsBytes(TASK_CONFIGS));
|
|
||||||
|
|
||||||
InternalRequestSignature expectedSignature = new InternalRequestSignature(
|
|
||||||
serializeAsBytes(TASK_CONFIGS),
|
|
||||||
Mac.getInstance(signatureAlgorithm),
|
|
||||||
Base64.getDecoder().decode(encodedSignature)
|
|
||||||
);
|
|
||||||
assertEquals(
|
|
||||||
expectedSignature,
|
|
||||||
signatureCapture.getValue()
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testPutConnectorTaskConfigsConnectorNotFound() {
|
|
||||||
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
|
||||||
expectAndCallbackException(cb, new NotFoundException("not found")).when(herder).putTaskConfigs(
|
|
||||||
eq(CONNECTOR_NAME),
|
|
||||||
eq(TASK_CONFIGS),
|
|
||||||
cb.capture(),
|
|
||||||
any()
|
|
||||||
);
|
|
||||||
|
|
||||||
assertThrows(NotFoundException.class, () -> connectorsResource.putTaskConfigs(CONNECTOR_NAME, NULL_HEADERS,
|
|
||||||
FORWARD, serializeAsBytes(TASK_CONFIGS)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRestartConnectorAndTasksConnectorNotFound() {
|
public void testRestartConnectorAndTasksConnectorNotFound() {
|
||||||
RestartRequest restartRequest = new RestartRequest(CONNECTOR_NAME, true, false);
|
RestartRequest restartRequest = new RestartRequest(CONNECTOR_NAME, true, false);
|
||||||
|
@ -683,55 +618,6 @@ public class ConnectorsResourceTest {
|
||||||
assertEquals(Response.Status.ACCEPTED.getStatusCode(), response.getStatus());
|
assertEquals(Response.Status.ACCEPTED.getStatusCode(), response.getStatus());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testFenceZombiesNoInternalRequestSignature() throws Throwable {
|
|
||||||
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
|
||||||
expectAndCallbackResult(cb, null)
|
|
||||||
.when(herder).fenceZombieSourceTasks(eq(CONNECTOR_NAME), cb.capture(), isNull());
|
|
||||||
|
|
||||||
connectorsResource.fenceZombies(CONNECTOR_NAME, NULL_HEADERS, FORWARD, serializeAsBytes(null));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testFenceZombiesWithInternalRequestSignature() throws Throwable {
|
|
||||||
final String signatureAlgorithm = "HmacSHA256";
|
|
||||||
final String encodedSignature = "Kv1/OSsxzdVIwvZ4e30avyRIVrngDfhzVUm/kAZEKc4=";
|
|
||||||
|
|
||||||
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
|
||||||
final ArgumentCaptor<InternalRequestSignature> signatureCapture = ArgumentCaptor.forClass(InternalRequestSignature.class);
|
|
||||||
expectAndCallbackResult(cb, null)
|
|
||||||
.when(herder).fenceZombieSourceTasks(eq(CONNECTOR_NAME), cb.capture(), signatureCapture.capture());
|
|
||||||
|
|
||||||
HttpHeaders headers = mock(HttpHeaders.class);
|
|
||||||
when(headers.getHeaderString(InternalRequestSignature.SIGNATURE_ALGORITHM_HEADER))
|
|
||||||
.thenReturn(signatureAlgorithm);
|
|
||||||
when(headers.getHeaderString(InternalRequestSignature.SIGNATURE_HEADER))
|
|
||||||
.thenReturn(encodedSignature);
|
|
||||||
|
|
||||||
connectorsResource.fenceZombies(CONNECTOR_NAME, headers, FORWARD, serializeAsBytes(null));
|
|
||||||
|
|
||||||
InternalRequestSignature expectedSignature = new InternalRequestSignature(
|
|
||||||
serializeAsBytes(null),
|
|
||||||
Mac.getInstance(signatureAlgorithm),
|
|
||||||
Base64.getDecoder().decode(encodedSignature)
|
|
||||||
);
|
|
||||||
assertEquals(
|
|
||||||
expectedSignature,
|
|
||||||
signatureCapture.getValue()
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testFenceZombiesConnectorNotFound() throws Throwable {
|
|
||||||
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
|
||||||
|
|
||||||
expectAndCallbackException(cb, new NotFoundException("not found"))
|
|
||||||
.when(herder).fenceZombieSourceTasks(eq(CONNECTOR_NAME), cb.capture(), any());
|
|
||||||
|
|
||||||
assertThrows(NotFoundException.class,
|
|
||||||
() -> connectorsResource.fenceZombies(CONNECTOR_NAME, NULL_HEADERS, FORWARD, serializeAsBytes(null)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRestartConnectorNotFound() {
|
public void testRestartConnectorNotFound() {
|
||||||
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
||||||
|
@ -806,9 +692,9 @@ public class ConnectorsResourceTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testConnectorActiveTopicsWithTopicTrackingDisabled() {
|
public void testConnectorActiveTopicsWithTopicTrackingDisabled() {
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG)).thenReturn(false);
|
when(serverConfig.topicTrackingEnabled()).thenReturn(false);
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ALLOW_RESET_CONFIG)).thenReturn(false);
|
when(serverConfig.topicTrackingResetEnabled()).thenReturn(false);
|
||||||
connectorsResource = new ConnectorsResource(herder, workerConfig, restClient);
|
connectorsResource = new ConnectorsResource(herder, serverConfig, restClient);
|
||||||
|
|
||||||
Exception e = assertThrows(ConnectRestException.class,
|
Exception e = assertThrows(ConnectRestException.class,
|
||||||
() -> connectorsResource.getConnectorActiveTopics(CONNECTOR_NAME));
|
() -> connectorsResource.getConnectorActiveTopics(CONNECTOR_NAME));
|
||||||
|
@ -817,10 +703,10 @@ public class ConnectorsResourceTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testResetConnectorActiveTopicsWithTopicTrackingDisabled() {
|
public void testResetConnectorActiveTopicsWithTopicTrackingDisabled() {
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG)).thenReturn(false);
|
when(serverConfig.topicTrackingEnabled()).thenReturn(false);
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ALLOW_RESET_CONFIG)).thenReturn(true);
|
when(serverConfig.topicTrackingResetEnabled()).thenReturn(true);
|
||||||
HttpHeaders headers = mock(HttpHeaders.class);
|
HttpHeaders headers = mock(HttpHeaders.class);
|
||||||
connectorsResource = new ConnectorsResource(herder, workerConfig, restClient);
|
connectorsResource = new ConnectorsResource(herder, serverConfig, restClient);
|
||||||
|
|
||||||
Exception e = assertThrows(ConnectRestException.class,
|
Exception e = assertThrows(ConnectRestException.class,
|
||||||
() -> connectorsResource.resetConnectorActiveTopics(CONNECTOR_NAME, headers));
|
() -> connectorsResource.resetConnectorActiveTopics(CONNECTOR_NAME, headers));
|
||||||
|
@ -829,10 +715,10 @@ public class ConnectorsResourceTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testResetConnectorActiveTopicsWithTopicTrackingEnabled() {
|
public void testResetConnectorActiveTopicsWithTopicTrackingEnabled() {
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG)).thenReturn(true);
|
when(serverConfig.topicTrackingEnabled()).thenReturn(true);
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ALLOW_RESET_CONFIG)).thenReturn(false);
|
when(serverConfig.topicTrackingResetEnabled()).thenReturn(false);
|
||||||
HttpHeaders headers = mock(HttpHeaders.class);
|
HttpHeaders headers = mock(HttpHeaders.class);
|
||||||
connectorsResource = new ConnectorsResource(herder, workerConfig, restClient);
|
connectorsResource = new ConnectorsResource(herder, serverConfig, restClient);
|
||||||
|
|
||||||
Exception e = assertThrows(ConnectRestException.class,
|
Exception e = assertThrows(ConnectRestException.class,
|
||||||
() -> connectorsResource.resetConnectorActiveTopics(CONNECTOR_NAME, headers));
|
() -> connectorsResource.resetConnectorActiveTopics(CONNECTOR_NAME, headers));
|
||||||
|
@ -841,11 +727,11 @@ public class ConnectorsResourceTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testConnectorActiveTopics() {
|
public void testConnectorActiveTopics() {
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG)).thenReturn(true);
|
when(serverConfig.topicTrackingEnabled()).thenReturn(true);
|
||||||
when(workerConfig.getBoolean(TOPIC_TRACKING_ALLOW_RESET_CONFIG)).thenReturn(true);
|
when(serverConfig.topicTrackingResetEnabled()).thenReturn(true);
|
||||||
when(herder.connectorActiveTopics(CONNECTOR_NAME))
|
when(herder.connectorActiveTopics(CONNECTOR_NAME))
|
||||||
.thenReturn(new ActiveTopicsInfo(CONNECTOR_NAME, CONNECTOR_ACTIVE_TOPICS));
|
.thenReturn(new ActiveTopicsInfo(CONNECTOR_NAME, CONNECTOR_ACTIVE_TOPICS));
|
||||||
connectorsResource = new ConnectorsResource(herder, workerConfig, restClient);
|
connectorsResource = new ConnectorsResource(herder, serverConfig, restClient);
|
||||||
|
|
||||||
Response response = connectorsResource.getConnectorActiveTopics(CONNECTOR_NAME);
|
Response response = connectorsResource.getConnectorActiveTopics(CONNECTOR_NAME);
|
||||||
assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
||||||
|
@ -858,7 +744,7 @@ public class ConnectorsResourceTest {
|
||||||
@Test
|
@Test
|
||||||
public void testResetConnectorActiveTopics() {
|
public void testResetConnectorActiveTopics() {
|
||||||
HttpHeaders headers = mock(HttpHeaders.class);
|
HttpHeaders headers = mock(HttpHeaders.class);
|
||||||
connectorsResource = new ConnectorsResource(herder, workerConfig, restClient);
|
connectorsResource = new ConnectorsResource(herder, serverConfig, restClient);
|
||||||
|
|
||||||
Response response = connectorsResource.resetConnectorActiveTopics(CONNECTOR_NAME, headers);
|
Response response = connectorsResource.resetConnectorActiveTopics(CONNECTOR_NAME, headers);
|
||||||
verify(herder).resetConnectorActiveTopics(CONNECTOR_NAME);
|
verify(herder).resetConnectorActiveTopics(CONNECTOR_NAME);
|
||||||
|
|
|
@ -0,0 +1,224 @@
|
||||||
|
/*
|
||||||
|
* 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.resources;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import org.apache.kafka.connect.errors.NotFoundException;
|
||||||
|
import org.apache.kafka.connect.runtime.Herder;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.InternalRequestSignature;
|
||||||
|
import org.apache.kafka.connect.runtime.rest.RestClient;
|
||||||
|
import org.apache.kafka.connect.util.Callback;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.mockito.ArgumentCaptor;
|
||||||
|
import org.mockito.Mock;
|
||||||
|
import org.mockito.junit.MockitoJUnitRunner;
|
||||||
|
import org.mockito.stubbing.Stubber;
|
||||||
|
|
||||||
|
import javax.crypto.Mac;
|
||||||
|
import javax.ws.rs.core.HttpHeaders;
|
||||||
|
import javax.ws.rs.core.UriInfo;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Base64;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertThrows;
|
||||||
|
import static org.mockito.Mockito.any;
|
||||||
|
import static org.mockito.Mockito.doAnswer;
|
||||||
|
import static org.mockito.Mockito.eq;
|
||||||
|
import static org.mockito.Mockito.isNull;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
|
@RunWith(MockitoJUnitRunner.StrictStubs.class)
|
||||||
|
public class InternalConnectResourceTest {
|
||||||
|
|
||||||
|
private static final Boolean FORWARD = true;
|
||||||
|
private static final String CONNECTOR_NAME = "test";
|
||||||
|
private static final HttpHeaders NULL_HEADERS = null;
|
||||||
|
private static final List<Map<String, String>> TASK_CONFIGS = new ArrayList<>();
|
||||||
|
static {
|
||||||
|
TASK_CONFIGS.add(Collections.singletonMap("config", "value"));
|
||||||
|
TASK_CONFIGS.add(Collections.singletonMap("config", "other_value"));
|
||||||
|
}
|
||||||
|
private static final String FENCE_PATH = "/connectors/" + CONNECTOR_NAME + "/fence";
|
||||||
|
private static final String TASK_CONFIGS_PATH = "/connectors/" + CONNECTOR_NAME + "/tasks";
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private UriInfo uriInfo;
|
||||||
|
@Mock
|
||||||
|
private Herder herder;
|
||||||
|
@Mock
|
||||||
|
private RestClient restClient;
|
||||||
|
|
||||||
|
private InternalConnectResource internalResource;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup() {
|
||||||
|
internalResource = new InternalConnectResource(herder, restClient);
|
||||||
|
internalResource.uriInfo = uriInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPutConnectorTaskConfigsNoInternalRequestSignature() throws Throwable {
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
||||||
|
expectAndCallbackResult(cb, null).when(herder).putTaskConfigs(
|
||||||
|
eq(CONNECTOR_NAME),
|
||||||
|
eq(TASK_CONFIGS),
|
||||||
|
cb.capture(),
|
||||||
|
any()
|
||||||
|
);
|
||||||
|
expectRequestPath(TASK_CONFIGS_PATH);
|
||||||
|
|
||||||
|
internalResource.putTaskConfigs(CONNECTOR_NAME, NULL_HEADERS, FORWARD, serializeAsBytes(TASK_CONFIGS));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPutConnectorTaskConfigsWithInternalRequestSignature() throws Throwable {
|
||||||
|
final String signatureAlgorithm = "HmacSHA256";
|
||||||
|
final String encodedSignature = "Kv1/OSsxzdVIwvZ4e30avyRIVrngDfhzVUm/kAZEKc4=";
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
||||||
|
final ArgumentCaptor<InternalRequestSignature> signatureCapture = ArgumentCaptor.forClass(InternalRequestSignature.class);
|
||||||
|
expectAndCallbackResult(cb, null).when(herder).putTaskConfigs(
|
||||||
|
eq(CONNECTOR_NAME),
|
||||||
|
eq(TASK_CONFIGS),
|
||||||
|
cb.capture(),
|
||||||
|
signatureCapture.capture()
|
||||||
|
);
|
||||||
|
|
||||||
|
HttpHeaders headers = mock(HttpHeaders.class);
|
||||||
|
when(headers.getHeaderString(InternalRequestSignature.SIGNATURE_ALGORITHM_HEADER))
|
||||||
|
.thenReturn(signatureAlgorithm);
|
||||||
|
when(headers.getHeaderString(InternalRequestSignature.SIGNATURE_HEADER))
|
||||||
|
.thenReturn(encodedSignature);
|
||||||
|
expectRequestPath(TASK_CONFIGS_PATH);
|
||||||
|
|
||||||
|
internalResource.putTaskConfigs(CONNECTOR_NAME, headers, FORWARD, serializeAsBytes(TASK_CONFIGS));
|
||||||
|
|
||||||
|
InternalRequestSignature expectedSignature = new InternalRequestSignature(
|
||||||
|
serializeAsBytes(TASK_CONFIGS),
|
||||||
|
Mac.getInstance(signatureAlgorithm),
|
||||||
|
Base64.getDecoder().decode(encodedSignature)
|
||||||
|
);
|
||||||
|
assertEquals(
|
||||||
|
expectedSignature,
|
||||||
|
signatureCapture.getValue()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testPutConnectorTaskConfigsConnectorNotFound() {
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
||||||
|
expectAndCallbackException(cb, new NotFoundException("not found")).when(herder).putTaskConfigs(
|
||||||
|
eq(CONNECTOR_NAME),
|
||||||
|
eq(TASK_CONFIGS),
|
||||||
|
cb.capture(),
|
||||||
|
any()
|
||||||
|
);
|
||||||
|
expectRequestPath(TASK_CONFIGS_PATH);
|
||||||
|
|
||||||
|
assertThrows(NotFoundException.class, () -> internalResource.putTaskConfigs(CONNECTOR_NAME, NULL_HEADERS,
|
||||||
|
FORWARD, serializeAsBytes(TASK_CONFIGS)));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFenceZombiesNoInternalRequestSignature() throws Throwable {
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
||||||
|
expectAndCallbackResult(cb, null)
|
||||||
|
.when(herder).fenceZombieSourceTasks(eq(CONNECTOR_NAME), cb.capture(), isNull());
|
||||||
|
expectRequestPath(FENCE_PATH);
|
||||||
|
|
||||||
|
internalResource.fenceZombies(CONNECTOR_NAME, NULL_HEADERS, FORWARD, serializeAsBytes(null));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFenceZombiesWithInternalRequestSignature() throws Throwable {
|
||||||
|
final String signatureAlgorithm = "HmacSHA256";
|
||||||
|
final String encodedSignature = "Kv1/OSsxzdVIwvZ4e30avyRIVrngDfhzVUm/kAZEKc4=";
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
||||||
|
final ArgumentCaptor<InternalRequestSignature> signatureCapture = ArgumentCaptor.forClass(InternalRequestSignature.class);
|
||||||
|
expectAndCallbackResult(cb, null)
|
||||||
|
.when(herder).fenceZombieSourceTasks(eq(CONNECTOR_NAME), cb.capture(), signatureCapture.capture());
|
||||||
|
|
||||||
|
HttpHeaders headers = mock(HttpHeaders.class);
|
||||||
|
when(headers.getHeaderString(InternalRequestSignature.SIGNATURE_ALGORITHM_HEADER))
|
||||||
|
.thenReturn(signatureAlgorithm);
|
||||||
|
when(headers.getHeaderString(InternalRequestSignature.SIGNATURE_HEADER))
|
||||||
|
.thenReturn(encodedSignature);
|
||||||
|
expectRequestPath(FENCE_PATH);
|
||||||
|
|
||||||
|
internalResource.fenceZombies(CONNECTOR_NAME, headers, FORWARD, serializeAsBytes(null));
|
||||||
|
|
||||||
|
InternalRequestSignature expectedSignature = new InternalRequestSignature(
|
||||||
|
serializeAsBytes(null),
|
||||||
|
Mac.getInstance(signatureAlgorithm),
|
||||||
|
Base64.getDecoder().decode(encodedSignature)
|
||||||
|
);
|
||||||
|
assertEquals(
|
||||||
|
expectedSignature,
|
||||||
|
signatureCapture.getValue()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFenceZombiesConnectorNotFound() throws Throwable {
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
final ArgumentCaptor<Callback<Void>> cb = ArgumentCaptor.forClass(Callback.class);
|
||||||
|
|
||||||
|
expectAndCallbackException(cb, new NotFoundException("not found"))
|
||||||
|
.when(herder).fenceZombieSourceTasks(eq(CONNECTOR_NAME), cb.capture(), any());
|
||||||
|
expectRequestPath(FENCE_PATH);
|
||||||
|
|
||||||
|
assertThrows(NotFoundException.class,
|
||||||
|
() -> internalResource.fenceZombies(CONNECTOR_NAME, NULL_HEADERS, FORWARD, serializeAsBytes(null)));
|
||||||
|
}
|
||||||
|
|
||||||
|
private <T> byte[] serializeAsBytes(final T value) throws IOException {
|
||||||
|
return new ObjectMapper().writeValueAsBytes(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
private <T> Stubber expectAndCallbackResult(final ArgumentCaptor<Callback<T>> cb, final T value) {
|
||||||
|
return doAnswer(invocation -> {
|
||||||
|
cb.getValue().onCompletion(null, value);
|
||||||
|
return null;
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
private <T> Stubber expectAndCallbackException(final ArgumentCaptor<Callback<T>> cb, final Throwable t) {
|
||||||
|
return doAnswer(invocation -> {
|
||||||
|
cb.getValue().onCompletion(t, null);
|
||||||
|
return null;
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
private void expectRequestPath(String path) {
|
||||||
|
when(uriInfo.getPath()).thenReturn(path);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -16,11 +16,8 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.connect.runtime.rest.util;
|
package org.apache.kafka.connect.runtime.rest.util;
|
||||||
|
|
||||||
import org.apache.kafka.clients.CommonClientConfigs;
|
|
||||||
import org.apache.kafka.common.config.SslConfigs;
|
import org.apache.kafka.common.config.SslConfigs;
|
||||||
import org.apache.kafka.connect.runtime.WorkerConfig;
|
import org.apache.kafka.connect.runtime.rest.RestServerConfig;
|
||||||
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
|
|
||||||
import org.apache.kafka.connect.runtime.standalone.StandaloneConfig;
|
|
||||||
import org.eclipse.jetty.util.ssl.SslContextFactory;
|
import org.eclipse.jetty.util.ssl.SslContextFactory;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -31,17 +28,6 @@ import java.util.Map;
|
||||||
|
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public class SSLUtilsTest {
|
public class SSLUtilsTest {
|
||||||
private static final Map<String, String> DEFAULT_CONFIG = new HashMap<>();
|
|
||||||
static {
|
|
||||||
// The WorkerConfig base class has some required settings without defaults
|
|
||||||
DEFAULT_CONFIG.put(DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "status-topic");
|
|
||||||
DEFAULT_CONFIG.put(DistributedConfig.CONFIG_TOPIC_CONFIG, "config-topic");
|
|
||||||
DEFAULT_CONFIG.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
|
|
||||||
DEFAULT_CONFIG.put(DistributedConfig.GROUP_ID_CONFIG, "connect-test-group");
|
|
||||||
DEFAULT_CONFIG.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
|
|
||||||
DEFAULT_CONFIG.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
|
|
||||||
DEFAULT_CONFIG.put(DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "connect-offsets");
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGetOrDefault() {
|
public void testGetOrDefault() {
|
||||||
|
@ -58,7 +44,7 @@ public class SSLUtilsTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreateServerSideSslContextFactory() {
|
public void testCreateServerSideSslContextFactory() {
|
||||||
Map<String, String> configMap = new HashMap<>(DEFAULT_CONFIG);
|
Map<String, String> configMap = new HashMap<>();
|
||||||
configMap.put("ssl.keystore.location", "/path/to/keystore");
|
configMap.put("ssl.keystore.location", "/path/to/keystore");
|
||||||
configMap.put("ssl.keystore.password", "123456");
|
configMap.put("ssl.keystore.password", "123456");
|
||||||
configMap.put("ssl.key.password", "123456");
|
configMap.put("ssl.key.password", "123456");
|
||||||
|
@ -76,7 +62,7 @@ public class SSLUtilsTest {
|
||||||
configMap.put("ssl.keymanager.algorithm", "SunX509");
|
configMap.put("ssl.keymanager.algorithm", "SunX509");
|
||||||
configMap.put("ssl.trustmanager.algorithm", "PKIX");
|
configMap.put("ssl.trustmanager.algorithm", "PKIX");
|
||||||
|
|
||||||
DistributedConfig config = new DistributedConfig(configMap);
|
RestServerConfig config = RestServerConfig.forPublic(null, configMap);
|
||||||
SslContextFactory ssl = SSLUtils.createServerSideSslContextFactory(config);
|
SslContextFactory ssl = SSLUtils.createServerSideSslContextFactory(config);
|
||||||
|
|
||||||
Assert.assertEquals("file:///path/to/keystore", ssl.getKeyStorePath());
|
Assert.assertEquals("file:///path/to/keystore", ssl.getKeyStorePath());
|
||||||
|
@ -96,7 +82,7 @@ public class SSLUtilsTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreateClientSideSslContextFactory() {
|
public void testCreateClientSideSslContextFactory() {
|
||||||
Map<String, String> configMap = new HashMap<>(DEFAULT_CONFIG);
|
Map<String, String> configMap = new HashMap<>();
|
||||||
configMap.put("ssl.keystore.location", "/path/to/keystore");
|
configMap.put("ssl.keystore.location", "/path/to/keystore");
|
||||||
configMap.put("ssl.keystore.password", "123456");
|
configMap.put("ssl.keystore.password", "123456");
|
||||||
configMap.put("ssl.key.password", "123456");
|
configMap.put("ssl.key.password", "123456");
|
||||||
|
@ -114,7 +100,7 @@ public class SSLUtilsTest {
|
||||||
configMap.put("ssl.keymanager.algorithm", "SunX509");
|
configMap.put("ssl.keymanager.algorithm", "SunX509");
|
||||||
configMap.put("ssl.trustmanager.algorithm", "PKIX");
|
configMap.put("ssl.trustmanager.algorithm", "PKIX");
|
||||||
|
|
||||||
DistributedConfig config = new DistributedConfig(configMap);
|
RestServerConfig config = RestServerConfig.forPublic(null, configMap);
|
||||||
SslContextFactory ssl = SSLUtils.createClientSideSslContextFactory(config);
|
SslContextFactory ssl = SSLUtils.createClientSideSslContextFactory(config);
|
||||||
|
|
||||||
Assert.assertEquals("file:///path/to/keystore", ssl.getKeyStorePath());
|
Assert.assertEquals("file:///path/to/keystore", ssl.getKeyStorePath());
|
||||||
|
@ -134,10 +120,7 @@ public class SSLUtilsTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreateServerSideSslContextFactoryDefaultValues() {
|
public void testCreateServerSideSslContextFactoryDefaultValues() {
|
||||||
Map<String, String> configMap = new HashMap<>(DEFAULT_CONFIG);
|
Map<String, String> configMap = new HashMap<>();
|
||||||
configMap.put(StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG, "/tmp/offset/file");
|
|
||||||
configMap.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
|
|
||||||
configMap.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
|
|
||||||
configMap.put("ssl.keystore.location", "/path/to/keystore");
|
configMap.put("ssl.keystore.location", "/path/to/keystore");
|
||||||
configMap.put("ssl.keystore.password", "123456");
|
configMap.put("ssl.keystore.password", "123456");
|
||||||
configMap.put("ssl.key.password", "123456");
|
configMap.put("ssl.key.password", "123456");
|
||||||
|
@ -147,7 +130,7 @@ public class SSLUtilsTest {
|
||||||
configMap.put("ssl.cipher.suites", "SSL_RSA_WITH_RC4_128_SHA,SSL_RSA_WITH_RC4_128_MD5");
|
configMap.put("ssl.cipher.suites", "SSL_RSA_WITH_RC4_128_SHA,SSL_RSA_WITH_RC4_128_MD5");
|
||||||
configMap.put("ssl.secure.random.implementation", "SHA1PRNG");
|
configMap.put("ssl.secure.random.implementation", "SHA1PRNG");
|
||||||
|
|
||||||
DistributedConfig config = new DistributedConfig(configMap);
|
RestServerConfig config = RestServerConfig.forPublic(null, configMap);
|
||||||
SslContextFactory ssl = SSLUtils.createServerSideSslContextFactory(config);
|
SslContextFactory ssl = SSLUtils.createServerSideSslContextFactory(config);
|
||||||
|
|
||||||
Assert.assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType());
|
Assert.assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType());
|
||||||
|
@ -162,10 +145,7 @@ public class SSLUtilsTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreateClientSideSslContextFactoryDefaultValues() {
|
public void testCreateClientSideSslContextFactoryDefaultValues() {
|
||||||
Map<String, String> configMap = new HashMap<>(DEFAULT_CONFIG);
|
Map<String, String> configMap = new HashMap<>();
|
||||||
configMap.put(StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG, "/tmp/offset/file");
|
|
||||||
configMap.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
|
|
||||||
configMap.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
|
|
||||||
configMap.put("ssl.keystore.location", "/path/to/keystore");
|
configMap.put("ssl.keystore.location", "/path/to/keystore");
|
||||||
configMap.put("ssl.keystore.password", "123456");
|
configMap.put("ssl.keystore.password", "123456");
|
||||||
configMap.put("ssl.key.password", "123456");
|
configMap.put("ssl.key.password", "123456");
|
||||||
|
@ -175,7 +155,7 @@ public class SSLUtilsTest {
|
||||||
configMap.put("ssl.cipher.suites", "SSL_RSA_WITH_RC4_128_SHA,SSL_RSA_WITH_RC4_128_MD5");
|
configMap.put("ssl.cipher.suites", "SSL_RSA_WITH_RC4_128_SHA,SSL_RSA_WITH_RC4_128_MD5");
|
||||||
configMap.put("ssl.secure.random.implementation", "SHA1PRNG");
|
configMap.put("ssl.secure.random.implementation", "SHA1PRNG");
|
||||||
|
|
||||||
DistributedConfig config = new DistributedConfig(configMap);
|
RestServerConfig config = RestServerConfig.forPublic(null, configMap);
|
||||||
SslContextFactory ssl = SSLUtils.createClientSideSslContextFactory(config);
|
SslContextFactory ssl = SSLUtils.createClientSideSslContextFactory(config);
|
||||||
|
|
||||||
Assert.assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType());
|
Assert.assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType());
|
||||||
|
|
|
@ -54,7 +54,7 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
|
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.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.BOOTSTRAP_SERVERS_CONFIG;
|
import static org.apache.kafka.connect.runtime.WorkerConfig.BOOTSTRAP_SERVERS_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.WorkerConfig.LISTENERS_CONFIG;
|
import static org.apache.kafka.connect.runtime.rest.RestServerConfig.LISTENERS_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG;
|
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_TOPIC_CONFIG;
|
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_TOPIC_CONFIG;
|
||||||
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG;
|
import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG;
|
||||||
|
|
Loading…
Reference in New Issue