mirror of https://github.com/apache/kafka.git
KAFKA-3863: System tests covering connector/task failure and restart
Author: Jason Gustafson <jason@confluent.io> Reviewers: Ewen Cheslack-Postava <ewen@confluent.io> Closes #1519 from hachikuji/KAFKA-3863
This commit is contained in:
parent
8bf18df1b6
commit
36cab7dbdf
|
@ -32,6 +32,9 @@ import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
|
||||||
import org.apache.kafka.connect.source.SourceConnector;
|
import org.apache.kafka.connect.source.SourceConnector;
|
||||||
import org.apache.kafka.connect.storage.ConfigBackingStore;
|
import org.apache.kafka.connect.storage.ConfigBackingStore;
|
||||||
import org.apache.kafka.connect.storage.StatusBackingStore;
|
import org.apache.kafka.connect.storage.StatusBackingStore;
|
||||||
|
import org.apache.kafka.connect.tools.MockConnector;
|
||||||
|
import org.apache.kafka.connect.tools.MockSinkConnector;
|
||||||
|
import org.apache.kafka.connect.tools.MockSourceConnector;
|
||||||
import org.apache.kafka.connect.tools.VerifiableSinkConnector;
|
import org.apache.kafka.connect.tools.VerifiableSinkConnector;
|
||||||
import org.apache.kafka.connect.tools.VerifiableSourceConnector;
|
import org.apache.kafka.connect.tools.VerifiableSourceConnector;
|
||||||
import org.apache.kafka.connect.util.ConnectorTaskId;
|
import org.apache.kafka.connect.util.ConnectorTaskId;
|
||||||
|
@ -87,7 +90,9 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
|
||||||
private static List<ConnectorPluginInfo> validConnectorPlugins;
|
private static List<ConnectorPluginInfo> validConnectorPlugins;
|
||||||
private static final Object LOCK = new Object();
|
private static final Object LOCK = new Object();
|
||||||
private Thread classPathTraverser;
|
private Thread classPathTraverser;
|
||||||
private static final List<Class<? extends Connector>> EXCLUDES = Arrays.<Class<? extends Connector>>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class);
|
private static final List<Class<? extends Connector>> EXCLUDES = Arrays.asList(
|
||||||
|
VerifiableSourceConnector.class, VerifiableSinkConnector.class,
|
||||||
|
MockConnector.class, MockSourceConnector.class, MockSinkConnector.class);
|
||||||
|
|
||||||
public AbstractHerder(Worker worker,
|
public AbstractHerder(Worker worker,
|
||||||
String workerId,
|
String workerId,
|
||||||
|
|
|
@ -401,6 +401,7 @@ public class Worker {
|
||||||
WorkerTask task = getTask(id);
|
WorkerTask task = getTask(id);
|
||||||
stopTask(task);
|
stopTask(task);
|
||||||
awaitStopTask(task, config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG));
|
awaitStopTask(task, config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG));
|
||||||
|
log.info("Task {} completed shutdown.", task.id());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -66,10 +66,9 @@ public class WorkerConnector {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void initialize(ConnectorConfig connectorConfig) {
|
public void initialize(ConnectorConfig connectorConfig) {
|
||||||
log.debug("Initializing connector {} with config {}", connName, config);
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
this.config = connectorConfig.originalsStrings();
|
this.config = connectorConfig.originalsStrings();
|
||||||
|
log.debug("Initializing connector {} with config {}", connName, config);
|
||||||
|
|
||||||
connector.initialize(new ConnectorContext() {
|
connector.initialize(new ConnectorContext() {
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,111 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.tools;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.config.ConfigDef;
|
||||||
|
import org.apache.kafka.common.utils.AppInfoParser;
|
||||||
|
import org.apache.kafka.connect.connector.Connector;
|
||||||
|
import org.apache.kafka.connect.connector.Task;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This connector provides support for mocking certain connector behaviors. For example,
|
||||||
|
* this can be used to simulate connector or task failures. It works by passing a "mock mode"
|
||||||
|
* through configuration from the system test. New mock behavior can be implemented either
|
||||||
|
* in the connector or in the task by providing a new mode implementation.
|
||||||
|
*
|
||||||
|
* At the moment, this connector only supports a single task and shares configuration between
|
||||||
|
* the connector and its tasks.
|
||||||
|
*
|
||||||
|
* @see MockSinkConnector
|
||||||
|
* @see MockSourceConnector
|
||||||
|
*/
|
||||||
|
public class MockConnector extends Connector {
|
||||||
|
public static final String MOCK_MODE_KEY = "mock_mode";
|
||||||
|
public static final String DELAY_MS_KEY = "delay_ms";
|
||||||
|
|
||||||
|
public static final String CONNECTOR_FAILURE = "connector-failure";
|
||||||
|
public static final String TASK_FAILURE = "task-failure";
|
||||||
|
|
||||||
|
public static final long DEFAULT_FAILURE_DELAY_MS = 15000;
|
||||||
|
|
||||||
|
private Map<String, String> config;
|
||||||
|
private ScheduledExecutorService executor;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String version() {
|
||||||
|
return AppInfoParser.getVersion();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void start(Map<String, String> config) {
|
||||||
|
this.config = config;
|
||||||
|
|
||||||
|
if (CONNECTOR_FAILURE.equals(config.get(MOCK_MODE_KEY))) {
|
||||||
|
// Schedule this connector to raise an exception after some delay
|
||||||
|
|
||||||
|
String delayMsString = config.get(DELAY_MS_KEY);
|
||||||
|
long delayMs = DEFAULT_FAILURE_DELAY_MS;
|
||||||
|
if (delayMsString != null)
|
||||||
|
delayMs = Long.parseLong(delayMsString);
|
||||||
|
|
||||||
|
executor = Executors.newSingleThreadScheduledExecutor();
|
||||||
|
executor.schedule(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
context.raiseError(new RuntimeException());
|
||||||
|
}
|
||||||
|
}, delayMs, TimeUnit.MILLISECONDS);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class<? extends Task> taskClass() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Map<String, String>> taskConfigs(int maxTasks) {
|
||||||
|
return Collections.singletonList(config);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stop() {
|
||||||
|
if (executor != null) {
|
||||||
|
executor.shutdownNow();
|
||||||
|
|
||||||
|
try {
|
||||||
|
if (!executor.awaitTermination(20, TimeUnit.SECONDS))
|
||||||
|
throw new RuntimeException("Failed timely termination of scheduler");
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new RuntimeException("Task was interrupted during shutdown");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ConfigDef config() {
|
||||||
|
return new ConfigDef();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,84 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.tools;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.config.Config;
|
||||||
|
import org.apache.kafka.common.config.ConfigDef;
|
||||||
|
import org.apache.kafka.connect.connector.ConnectorContext;
|
||||||
|
import org.apache.kafka.connect.connector.Task;
|
||||||
|
import org.apache.kafka.connect.sink.SinkConnector;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mock sink implementation which delegates to {@link MockConnector}.
|
||||||
|
*/
|
||||||
|
public class MockSinkConnector extends SinkConnector {
|
||||||
|
|
||||||
|
private MockConnector delegate = new MockConnector();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initialize(ConnectorContext ctx) {
|
||||||
|
delegate.initialize(ctx);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initialize(ConnectorContext ctx, List<Map<String, String>> taskConfigs) {
|
||||||
|
delegate.initialize(ctx, taskConfigs);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reconfigure(Map<String, String> props) {
|
||||||
|
delegate.reconfigure(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Config validate(Map<String, String> connectorConfigs) {
|
||||||
|
return delegate.validate(connectorConfigs);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String version() {
|
||||||
|
return delegate.version();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void start(Map<String, String> props) {
|
||||||
|
delegate.start(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class<? extends Task> taskClass() {
|
||||||
|
return MockSinkTask.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Map<String, String>> taskConfigs(int maxTasks) {
|
||||||
|
return delegate.taskConfigs(maxTasks);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stop() {
|
||||||
|
delegate.stop();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ConfigDef config() {
|
||||||
|
return delegate.config();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,71 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.tools;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.utils.AppInfoParser;
|
||||||
|
import org.apache.kafka.connect.sink.SinkRecord;
|
||||||
|
import org.apache.kafka.connect.sink.SinkTask;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class MockSinkTask extends SinkTask {
|
||||||
|
|
||||||
|
private String mockMode;
|
||||||
|
private long startTimeMs;
|
||||||
|
private long failureDelayMs;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String version() {
|
||||||
|
return AppInfoParser.getVersion();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void start(Map<String, String> config) {
|
||||||
|
this.mockMode = config.get(MockConnector.MOCK_MODE_KEY);
|
||||||
|
|
||||||
|
if (MockConnector.TASK_FAILURE.equals(mockMode)) {
|
||||||
|
this.startTimeMs = System.currentTimeMillis();
|
||||||
|
|
||||||
|
String delayMsString = config.get(MockConnector.DELAY_MS_KEY);
|
||||||
|
this.failureDelayMs = MockConnector.DEFAULT_FAILURE_DELAY_MS;
|
||||||
|
if (delayMsString != null)
|
||||||
|
failureDelayMs = Long.parseLong(delayMsString);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void put(Collection<SinkRecord> records) {
|
||||||
|
if (MockConnector.TASK_FAILURE.equals(mockMode)) {
|
||||||
|
long now = System.currentTimeMillis();
|
||||||
|
if (now > startTimeMs + failureDelayMs)
|
||||||
|
throw new RuntimeException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void flush(Map<TopicPartition, OffsetAndMetadata> offsets) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stop() {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,84 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.tools;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.config.Config;
|
||||||
|
import org.apache.kafka.common.config.ConfigDef;
|
||||||
|
import org.apache.kafka.connect.connector.ConnectorContext;
|
||||||
|
import org.apache.kafka.connect.connector.Task;
|
||||||
|
import org.apache.kafka.connect.source.SourceConnector;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mock source implementation which delegates to {@link MockConnector}.
|
||||||
|
*/
|
||||||
|
public class MockSourceConnector extends SourceConnector {
|
||||||
|
|
||||||
|
private MockConnector delegate = new MockConnector();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initialize(ConnectorContext ctx) {
|
||||||
|
delegate.initialize(ctx);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initialize(ConnectorContext ctx, List<Map<String, String>> taskConfigs) {
|
||||||
|
delegate.initialize(ctx, taskConfigs);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void reconfigure(Map<String, String> props) {
|
||||||
|
delegate.reconfigure(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Config validate(Map<String, String> connectorConfigs) {
|
||||||
|
return delegate.validate(connectorConfigs);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String version() {
|
||||||
|
return delegate.version();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void start(Map<String, String> props) {
|
||||||
|
delegate.start(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class<? extends Task> taskClass() {
|
||||||
|
return MockSourceTask.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Map<String, String>> taskConfigs(int maxTasks) {
|
||||||
|
return delegate.taskConfigs(maxTasks);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stop() {
|
||||||
|
delegate.stop();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ConfigDef config() {
|
||||||
|
return delegate.config();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,66 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.tools;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.utils.AppInfoParser;
|
||||||
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
|
import org.apache.kafka.connect.source.SourceTask;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class MockSourceTask extends SourceTask {
|
||||||
|
|
||||||
|
private String mockMode;
|
||||||
|
private long startTimeMs;
|
||||||
|
private long failureDelayMs;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String version() {
|
||||||
|
return AppInfoParser.getVersion();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void start(Map<String, String> config) {
|
||||||
|
this.mockMode = config.get(MockConnector.MOCK_MODE_KEY);
|
||||||
|
|
||||||
|
if (MockConnector.TASK_FAILURE.equals(mockMode)) {
|
||||||
|
this.startTimeMs = System.currentTimeMillis();
|
||||||
|
|
||||||
|
String delayMsString = config.get(MockConnector.DELAY_MS_KEY);
|
||||||
|
this.failureDelayMs = MockConnector.DEFAULT_FAILURE_DELAY_MS;
|
||||||
|
if (delayMsString != null)
|
||||||
|
failureDelayMs = Long.parseLong(delayMsString);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<SourceRecord> poll() throws InterruptedException {
|
||||||
|
if (MockConnector.TASK_FAILURE.equals(mockMode)) {
|
||||||
|
long now = System.currentTimeMillis();
|
||||||
|
if (now > startTimeMs + failureDelayMs)
|
||||||
|
throw new RuntimeException();
|
||||||
|
}
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stop() {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -39,6 +39,9 @@ import org.apache.kafka.connect.runtime.rest.entities.ConfigValueInfo;
|
||||||
import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo;
|
import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo;
|
||||||
import org.apache.kafka.connect.sink.SinkConnector;
|
import org.apache.kafka.connect.sink.SinkConnector;
|
||||||
import org.apache.kafka.connect.source.SourceConnector;
|
import org.apache.kafka.connect.source.SourceConnector;
|
||||||
|
import org.apache.kafka.connect.tools.MockConnector;
|
||||||
|
import org.apache.kafka.connect.tools.MockSinkConnector;
|
||||||
|
import org.apache.kafka.connect.tools.MockSourceConnector;
|
||||||
import org.apache.kafka.connect.tools.VerifiableSinkConnector;
|
import org.apache.kafka.connect.tools.VerifiableSinkConnector;
|
||||||
import org.apache.kafka.connect.tools.VerifiableSourceConnector;
|
import org.apache.kafka.connect.tools.VerifiableSourceConnector;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
|
@ -165,6 +168,9 @@ public class ConnectorPluginsResourceTest {
|
||||||
assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(SinkConnector.class.getCanonicalName())));
|
assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(SinkConnector.class.getCanonicalName())));
|
||||||
assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(VerifiableSourceConnector.class.getCanonicalName())));
|
assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(VerifiableSourceConnector.class.getCanonicalName())));
|
||||||
assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(VerifiableSinkConnector.class.getCanonicalName())));
|
assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(VerifiableSinkConnector.class.getCanonicalName())));
|
||||||
|
assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(MockSourceConnector.class.getCanonicalName())));
|
||||||
|
assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(MockSinkConnector.class.getCanonicalName())));
|
||||||
|
assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(MockConnector.class.getCanonicalName())));
|
||||||
assertTrue(connectorPlugins.contains(new ConnectorPluginInfo(ConnectorPluginsResourceTestConnector.class.getCanonicalName())));
|
assertTrue(connectorPlugins.contains(new ConnectorPluginInfo(ConnectorPluginsResourceTestConnector.class.getCanonicalName())));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -135,6 +135,12 @@ class ConnectServiceBase(KafkaPathResolverMixin, Service):
|
||||||
def get_connector_status(self, name, node=None):
|
def get_connector_status(self, name, node=None):
|
||||||
return self._rest('/connectors/' + name + '/status', node=node)
|
return self._rest('/connectors/' + name + '/status', node=node)
|
||||||
|
|
||||||
|
def restart_connector(self, name, node=None):
|
||||||
|
return self._rest('/connectors/' + name + '/restart', method="POST")
|
||||||
|
|
||||||
|
def restart_task(self, connector_name, task_id, node=None):
|
||||||
|
return self._rest('/connectors/' + connector_name + '/tasks/' + str(task_id) + '/restart', method="POST")
|
||||||
|
|
||||||
def pause_connector(self, name, node=None):
|
def pause_connector(self, name, node=None):
|
||||||
return self._rest('/connectors/' + name + '/pause', method="PUT")
|
return self._rest('/connectors/' + name + '/pause', method="PUT")
|
||||||
|
|
||||||
|
@ -331,3 +337,46 @@ class VerifiableSink(VerifiableConnector):
|
||||||
'tasks.max': self.tasks,
|
'tasks.max': self.tasks,
|
||||||
'topics': ",".join(self.topics)
|
'topics': ",".join(self.topics)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
class MockSink(object):
|
||||||
|
|
||||||
|
def __init__(self, cc, topics, mode=None, delay_sec=10, name="mock-sink"):
|
||||||
|
self.cc = cc
|
||||||
|
self.logger = self.cc.logger
|
||||||
|
self.name = name
|
||||||
|
self.mode = mode
|
||||||
|
self.delay_sec = delay_sec
|
||||||
|
self.topics = topics
|
||||||
|
|
||||||
|
def start(self):
|
||||||
|
self.logger.info("Creating connector MockSinkConnector %s", self.name)
|
||||||
|
self.cc.create_connector({
|
||||||
|
'name': self.name,
|
||||||
|
'connector.class': 'org.apache.kafka.connect.tools.MockSinkConnector',
|
||||||
|
'tasks.max': 1,
|
||||||
|
'topics': ",".join(self.topics),
|
||||||
|
'mock_mode': self.mode,
|
||||||
|
'delay_ms': self.delay_sec * 1000
|
||||||
|
})
|
||||||
|
|
||||||
|
class MockSource(object):
|
||||||
|
|
||||||
|
def __init__(self, cc, topics, mode=None, delay_sec=10, name="mock-source"):
|
||||||
|
self.cc = cc
|
||||||
|
self.logger = self.cc.logger
|
||||||
|
self.name = name
|
||||||
|
self.mode = mode
|
||||||
|
self.delay_sec = delay_sec
|
||||||
|
self.topics = topics
|
||||||
|
|
||||||
|
def start(self):
|
||||||
|
self.logger.info("Creating connector MockSourceConnector %s", self.name)
|
||||||
|
self.cc.create_connector({
|
||||||
|
'name': self.name,
|
||||||
|
'connector.class': 'org.apache.kafka.connect.tools.MockSourceConnector',
|
||||||
|
'tasks.max': 1,
|
||||||
|
'topics': ",".join(self.topics),
|
||||||
|
'mock_mode': self.mode,
|
||||||
|
'delay_ms': self.delay_sec * 1000
|
||||||
|
})
|
||||||
|
|
||||||
|
|
|
@ -17,7 +17,7 @@ from ducktape.tests.test import Test
|
||||||
|
|
||||||
from kafkatest.services.zookeeper import ZookeeperService
|
from kafkatest.services.zookeeper import ZookeeperService
|
||||||
from kafkatest.services.kafka import KafkaService
|
from kafkatest.services.kafka import KafkaService
|
||||||
from kafkatest.services.connect import ConnectDistributedService, VerifiableSource, VerifiableSink, ConnectRestError
|
from kafkatest.services.connect import ConnectDistributedService, VerifiableSource, VerifiableSink, ConnectRestError, MockSink, MockSource
|
||||||
from kafkatest.services.console_consumer import ConsoleConsumer
|
from kafkatest.services.console_consumer import ConsoleConsumer
|
||||||
from kafkatest.services.security.security_config import SecurityConfig
|
from kafkatest.services.security.security_config import SecurityConfig
|
||||||
from ducktape.utils.util import wait_until
|
from ducktape.utils.util import wait_until
|
||||||
|
@ -88,9 +88,23 @@ class ConnectDistributedTest(Test):
|
||||||
except ConnectRestError:
|
except ConnectRestError:
|
||||||
return None
|
return None
|
||||||
|
|
||||||
def _has_state(self, status, state):
|
def _connector_has_state(self, status, state):
|
||||||
return status is not None and status['connector']['state'] == state
|
return status is not None and status['connector']['state'] == state
|
||||||
|
|
||||||
|
def _task_has_state(self, task_id, status, state):
|
||||||
|
if not status:
|
||||||
|
return False
|
||||||
|
|
||||||
|
tasks = status['tasks']
|
||||||
|
if not tasks:
|
||||||
|
return False
|
||||||
|
|
||||||
|
for task in tasks:
|
||||||
|
if task['id'] == task_id:
|
||||||
|
return task['state'] == state
|
||||||
|
|
||||||
|
return False
|
||||||
|
|
||||||
def _all_tasks_have_state(self, status, task_count, state):
|
def _all_tasks_have_state(self, status, task_count, state):
|
||||||
if status is None:
|
if status is None:
|
||||||
return False
|
return False
|
||||||
|
@ -103,11 +117,68 @@ class ConnectDistributedTest(Test):
|
||||||
|
|
||||||
def is_running(self, connector, node=None):
|
def is_running(self, connector, node=None):
|
||||||
status = self._connector_status(connector.name, node)
|
status = self._connector_status(connector.name, node)
|
||||||
return self._has_state(status, 'RUNNING') and self._all_tasks_have_state(status, connector.tasks, 'RUNNING')
|
return self._connector_has_state(status, 'RUNNING') and self._all_tasks_have_state(status, connector.tasks, 'RUNNING')
|
||||||
|
|
||||||
def is_paused(self, connector, node=None):
|
def is_paused(self, connector, node=None):
|
||||||
status = self._connector_status(connector.name, node)
|
status = self._connector_status(connector.name, node)
|
||||||
return self._has_state(status, 'PAUSED') and self._all_tasks_have_state(status, connector.tasks, 'PAUSED')
|
return self._connector_has_state(status, 'PAUSED') and self._all_tasks_have_state(status, connector.tasks, 'PAUSED')
|
||||||
|
|
||||||
|
def connector_is_running(self, connector, node=None):
|
||||||
|
status = self._connector_status(connector.name, node)
|
||||||
|
return self._connector_has_state(status, 'RUNNING')
|
||||||
|
|
||||||
|
def connector_is_failed(self, connector, node=None):
|
||||||
|
status = self._connector_status(connector.name, node)
|
||||||
|
return self._connector_has_state(status, 'FAILED')
|
||||||
|
|
||||||
|
def task_is_failed(self, connector, task_id, node=None):
|
||||||
|
status = self._connector_status(connector.name, node)
|
||||||
|
return self._task_has_state(task_id, status, 'FAILED')
|
||||||
|
|
||||||
|
def task_is_running(self, connector, task_id, node=None):
|
||||||
|
status = self._connector_status(connector.name, node)
|
||||||
|
return self._task_has_state(task_id, status, 'RUNNING')
|
||||||
|
|
||||||
|
def test_restart_failed_connector(self):
|
||||||
|
self.setup_services()
|
||||||
|
self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node))
|
||||||
|
self.cc.start()
|
||||||
|
|
||||||
|
self.sink = MockSink(self.cc, self.topics.keys(), mode='connector-failure', delay_sec=5)
|
||||||
|
self.sink.start()
|
||||||
|
|
||||||
|
wait_until(lambda: self.connector_is_failed(self.sink), timeout_sec=15,
|
||||||
|
err_msg="Failed to see connector transition to the FAILED state")
|
||||||
|
|
||||||
|
self.cc.restart_connector(self.sink.name)
|
||||||
|
|
||||||
|
wait_until(lambda: self.connector_is_running(self.sink), timeout_sec=10,
|
||||||
|
err_msg="Failed to see connector transition to the RUNNING state")
|
||||||
|
|
||||||
|
|
||||||
|
@matrix(connector_type=["source", "sink"])
|
||||||
|
def test_restart_failed_task(self, connector_type):
|
||||||
|
self.setup_services()
|
||||||
|
self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node))
|
||||||
|
self.cc.start()
|
||||||
|
|
||||||
|
connector = None
|
||||||
|
if connector_type == "sink":
|
||||||
|
connector = MockSink(self.cc, self.topics.keys(), mode='task-failure', delay_sec=5)
|
||||||
|
else:
|
||||||
|
connector = MockSource(self.cc, self.topics.keys(), mode='task-failure', delay_sec=5)
|
||||||
|
|
||||||
|
connector.start()
|
||||||
|
|
||||||
|
task_id = 0
|
||||||
|
wait_until(lambda: self.task_is_failed(connector, task_id), timeout_sec=15,
|
||||||
|
err_msg="Failed to see task transition to the FAILED state")
|
||||||
|
|
||||||
|
self.cc.restart_task(connector.name, task_id)
|
||||||
|
|
||||||
|
wait_until(lambda: self.task_is_running(connector, task_id), timeout_sec=10,
|
||||||
|
err_msg="Failed to see task transition to the RUNNING state")
|
||||||
|
|
||||||
|
|
||||||
def test_pause_and_resume_source(self):
|
def test_pause_and_resume_source(self):
|
||||||
"""
|
"""
|
||||||
|
|
Loading…
Reference in New Issue