KAFKA-2366; Initial patch for Copycat

This is an initial patch implementing the basics of Copycat for KIP-26.

The intent here is to start a review of the key pieces of the core API and get a reasonably functional, baseline, non-distributed implementation of Copycat in place to get things rolling. The current patch has a number of known issues that need to be addressed before a final version:

* Some build-related issues. Specifically, requires some locally-installed dependencies (see below), ignores checkstyle for the runtime data library because it's lifted from Avro currently and likely won't last in its current form, and some Gradle task dependencies aren't quite right because I haven't gotten rid of the dependency on `core` (which should now be an easy patch since new consumer groups are in a much better state).
* This patch currently depends on some Confluent trunk code because I prototyped with our Avro serializers w/ schema-registry support. We need to figure out what we want to provide as an example built-in set of serializers. Unlike core Kafka where we could ignore the issue, providing only ByteArray or String serializers, this is pretty central to how Copycat works.
* This patch uses a hacked up version of Avro as its runtime data format. Not sure if we want to go through the entire API discussion just to get some basic code committed, so I filed KAFKA-2367 to handle that separately. The core connector APIs and the runtime data APIs are entirely orthogonal.
* This patch needs some updates to get aligned with recent new consumer changes (specifically, I'm aware of the ConcurrentModificationException issue on exit). More generally, the new consumer is in flux but Copycat depends on it, so there are likely to be some negative interactions.
* The layout feels a bit awkward to me right now because I ported it from a Maven layout. We don't have nearly the same level of granularity in Kafka currently (core and clients, plus the mostly ignored examples, log4j-appender, and a couple of contribs). We might want to reorganize, although keeping data+api separate from runtime and connector plugins is useful for minimizing dependencies.
* There are a variety of other things (e.g., I'm not happy with the exception hierarchy/how they are currently handled, TopicPartition doesn't really need to be duplicated unless we want Copycat entirely isolated from the Kafka APIs, etc), but I expect those we'll cover in the review.

Before commenting on the patch, it's probably worth reviewing https://issues.apache.org/jira/browse/KAFKA-2365 and https://issues.apache.org/jira/browse/KAFKA-2366 to get an idea of what I had in mind for a) what we ultimately want with all the Copycat patches and b) what we aim to cover in this initial patch. My hope is that we can use a WIP patch (after the current obvious deficiencies are addressed) while recognizing that we want to make iterative progress with a bunch of subsequent PRs.

Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Ismael Juma, Gwen Shapira

Closes #99 from ewencp/copycat and squashes the following commits:

a3a47a6 [Ewen Cheslack-Postava] Simplify Copycat exceptions, make them a subclass of KafkaException.
8c108b0 [Ewen Cheslack-Postava] Rename Coordinator to Herder to avoid confusion with the consumer coordinator.
7bf8075 [Ewen Cheslack-Postava] Make Copycat CLI speific to standalone mode, clean up some config and get rid of config storage in standalone mode.
656a003 [Ewen Cheslack-Postava] Clarify and expand the explanation of the Copycat Coordinator interface.
c0e5fdc [Ewen Cheslack-Postava] Merge remote-tracking branch 'origin/trunk' into copycat
0fa7a36 [Ewen Cheslack-Postava] Mark Copycat classes as unstable and reduce visibility of some classes where possible.
d55d31e [Ewen Cheslack-Postava] Reorganize Copycat code to put it all under one top-level directory.
b29cb2c [Ewen Cheslack-Postava] Merge remote-tracking branch 'origin/trunk' into copycat
d713a21 [Ewen Cheslack-Postava] Address Gwen's review comments.
6787a85 [Ewen Cheslack-Postava] Make Converter generic to match serializers since some serialization formats do not require a base class of Object; update many other classes to have generic key and value class type parameters to match this change.
b194c73 [Ewen Cheslack-Postava] Split Copycat converter option into two options for key and value.
0b5a1a0 [Ewen Cheslack-Postava] Normalize naming to use partition for both source and Kafka, adjusting naming in CopycatRecord classes to clearly differentiate.
e345142 [Ewen Cheslack-Postava] Remove Copycat reflection utils, use existing Utils and ConfigDef functionality from clients package.
be5c387 [Ewen Cheslack-Postava] Minor cleanup
122423e [Ewen Cheslack-Postava] Style cleanup
6ba87de [Ewen Cheslack-Postava] Remove most of the Avro-based mock runtime data API, only preserving enough schema functionality to support basic primitive types for an initial patch.
4674d13 [Ewen Cheslack-Postava] Address review comments, clean up some code styling.
25b5739 [Ewen Cheslack-Postava] Fix sink task offset commit concurrency issue by moving it to the worker thread and waking up the consumer to ensure it exits promptly.
0aefe21 [Ewen Cheslack-Postava] Add log4j settings for Copycat.
220e42d [Ewen Cheslack-Postava] Replace Avro serializer with JSON serializer.
1243a7c [Ewen Cheslack-Postava] Merge remote-tracking branch 'origin/trunk' into copycat
5a618c6 [Ewen Cheslack-Postava] Remove offset serializers, instead reusing the existing serializers and removing schema projection support.
e849e10 [Ewen Cheslack-Postava] Remove duplicated TopicPartition implementation.
dec1379 [Ewen Cheslack-Postava] Switch to using new consumer coordinator instead of manually assigning partitions. Remove dependency of copycat-runtime on core.
4a9b4f3 [Ewen Cheslack-Postava] Add some helpful Copycat-specific build and test targets that cover all Copycat packages.
31cd1ca [Ewen Cheslack-Postava] Add CLI tools for Copycat.
e14942c [Ewen Cheslack-Postava] Add Copycat file connector.
0233456 [Ewen Cheslack-Postava] Add copycat-avro and copycat-runtime
11981d2 [Ewen Cheslack-Postava] Add copycat-data and copycat-api
This commit is contained in:
Ewen Cheslack-Postava 2015-08-14 16:00:51 -07:00 committed by Gwen Shapira
parent c8e62c9818
commit f6acfb0891
82 changed files with 11116 additions and 15 deletions

23
bin/copycat-standalone.sh Executable file
View File

@ -0,0 +1,23 @@
#!/bin/sh
# 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.
base_dir=$(dirname $0)
if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then
export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/copycat-log4j.properties"
fi
exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.copycat.cli.CopycatStandalone "$@"

View File

@ -66,6 +66,14 @@ do
CLASSPATH=$CLASSPATH:$file
done
for cc_pkg in "data" "api" "runtime" "file" "json"
do
for file in $base_dir/copycat/${cc_pkg}/build/libs/copycat-${cc_pkg}*.jar $base_dir/copycat/${cc_pkg}/build/dependant-libs/*.jar;
do
CLASSPATH=$CLASSPATH:$file
done
done
# classpath addition for release
for file in $base_dir/libs/*.jar;
do

View File

@ -28,6 +28,11 @@ buildscript {
}
def slf4jlog4j='org.slf4j:slf4j-log4j12:1.7.6'
def slf4japi="org.slf4j:slf4j-api:1.7.6"
def junit='junit:junit:4.6'
def easymock='org.easymock:easymock:3.3.1'
def powermock='org.powermock:powermock-module-junit4:1.6.2'
def powermock_easymock='org.powermock:powermock-api-easymock:1.6.2'
allprojects {
apply plugin: 'idea'
@ -59,7 +64,7 @@ rat {
// And some of the files that we have checked in should also be excluded from this check
excludes.addAll([
'**/.git/**',
'build/**',
'**/build/**',
'CONTRIBUTING.md',
'gradlew',
'gradlew.bat',
@ -204,20 +209,25 @@ for ( sv in ['2_10_5', '2_11_7'] ) {
}
}
tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar', 'log4j-appender:jar', 'tools:jar']) {
def copycatPkgs = ['copycat:data', 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file']
def pkgs = ['clients', 'examples', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'log4j-appender', 'tools'] + copycatPkgs
tasks.create(name: "jarCopycat", dependsOn: copycatPkgs.collect { it + ":jar" }) {}
tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7'] + pkgs.collect { it + ":jar" }) {
}
tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar', 'log4j-appender:srcJar', 'tools:srcJar']) { }
tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7'] + pkgs.collect { it + ":srcJar" }) { }
tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar', 'log4j-appender:docsJar', 'tools:docsJar']) { }
tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7'] + pkgs.collect { it + ":docsJar" }) { }
tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7', 'clients:test', 'log4j-appender:test', 'tools:test']) {
tasks.create(name: "testCopycat", dependsOn: copycatPkgs.collect { it + ":test" }) {}
tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7'] + pkgs.collect { it + ":test" }) {
}
tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) {
}
tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives', 'log4j-appender:uploadArchives', 'tools:uploadArchives']) {
tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7'] + pkgs.collect { it + ":uploadArchives" }) {
}
project(':core') {
@ -239,8 +249,8 @@ project(':core') {
compile 'org.scala-lang.modules:scala-parser-combinators_2.11:1.0.4'
}
testCompile 'junit:junit:4.6'
testCompile 'org.easymock:easymock:3.0'
testCompile "$junit"
testCompile "$easymock"
testCompile 'org.objenesis:objenesis:1.2'
testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5"
@ -371,11 +381,11 @@ project(':clients') {
archivesBaseName = "kafka-clients"
dependencies {
compile "org.slf4j:slf4j-api:1.7.6"
compile "$slf4japi"
compile 'org.xerial.snappy:snappy-java:1.1.1.7'
compile 'net.jpountz.lz4:lz4:1.2.0'
testCompile 'junit:junit:4.6'
testCompile "$junit"
testRuntime "$slf4jlog4j"
}
@ -423,7 +433,7 @@ project(':tools') {
compile 'com.fasterxml.jackson.core:jackson-databind:2.5.4'
compile "$slf4jlog4j"
testCompile 'junit:junit:4.6'
testCompile "$junit"
testCompile project(path: ':clients', configuration: 'archives')
}
@ -471,7 +481,7 @@ project(':log4j-appender') {
compile project(':clients')
compile "$slf4jlog4j"
testCompile 'junit:junit:4.6'
testCompile "$junit"
testCompile project(path: ':clients', configuration: 'archives')
}
@ -496,3 +506,238 @@ project(':log4j-appender') {
}
test.dependsOn('checkstyleMain', 'checkstyleTest')
}
project(':copycat:data') {
apply plugin: 'checkstyle'
archivesBaseName = "copycat-data"
dependencies {
compile project(':clients')
compile "$slf4japi"
testCompile "$junit"
testRuntime "$slf4jlog4j"
}
task testJar(type: Jar) {
classifier = 'test'
from sourceSets.test.output
}
test {
testLogging {
events "passed", "skipped", "failed"
exceptionFormat = 'full'
}
}
javadoc {
include "**/org/apache/kafka/copycat/data/*"
}
artifacts {
archives testJar
}
configurations {
archives.extendsFrom (testCompile)
}
/* FIXME Re-enable this with KAFKA-2367 when the placeholder data API is replaced
checkstyle {
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
}
test.dependsOn('checkstyleMain', 'checkstyleTest') */
}
project(':copycat:api') {
apply plugin: 'checkstyle'
archivesBaseName = "copycat-api"
dependencies {
compile project(':copycat:data')
compile "$slf4japi"
testCompile "$junit"
testRuntime "$slf4jlog4j"
}
task testJar(type: Jar) {
classifier = 'test'
from sourceSets.test.output
}
test {
testLogging {
events "passed", "skipped", "failed"
exceptionFormat = 'full'
}
}
javadoc {
include "**/org/apache/kafka/copycat/*"
}
artifacts {
archives testJar
}
configurations {
archives.extendsFrom (testCompile)
}
checkstyle {
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
}
test.dependsOn('checkstyleMain', 'checkstyleTest')
}
project(':copycat:json') {
apply plugin: 'checkstyle'
archivesBaseName = "copycat-json"
dependencies {
compile project(':copycat:api')
compile "$slf4japi"
compile 'com.fasterxml.jackson.core:jackson-databind:2.5.4'
testCompile "$junit"
testCompile "$easymock"
testCompile "$powermock"
testCompile "$powermock_easymock"
testRuntime "$slf4jlog4j"
}
task testJar(type: Jar) {
classifier = 'test'
from sourceSets.test.output
}
test {
testLogging {
events "passed", "skipped", "failed"
exceptionFormat = 'full'
}
}
javadoc {
include "**/org/apache/kafka/copycat/*"
}
artifacts {
archives testJar
}
configurations {
archives.extendsFrom(testCompile)
}
checkstyle {
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
}
test.dependsOn('checkstyleMain', 'checkstyleTest')
tasks.create(name: "copyDependantLibs", type: Copy) {
from (configurations.runtime) {
exclude('kafka-clients*')
exclude('copycat-*')
}
into "$buildDir/dependant-libs"
}
jar {
dependsOn copyDependantLibs
}
}
project(':copycat:runtime') {
apply plugin: 'checkstyle'
archivesBaseName = "copycat-runtime"
dependencies {
compile project(':copycat:api')
compile project(':clients')
compile "$slf4japi"
testCompile "$junit"
testCompile "$easymock"
testCompile "$powermock"
testCompile "$powermock_easymock"
testRuntime "$slf4jlog4j"
testRuntime project(":copycat:json")
}
task testJar(type: Jar) {
classifier = 'test'
from sourceSets.test.output
}
test {
testLogging {
events "passed", "skipped", "failed"
exceptionFormat = 'full'
}
}
javadoc {
include "**/org/apache/kafka/copycat/*"
}
artifacts {
archives testJar
}
configurations {
archives.extendsFrom(testCompile)
}
checkstyle {
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
}
test.dependsOn('checkstyleMain', 'checkstyleTest')
}
project(':copycat:file') {
apply plugin: 'checkstyle'
archivesBaseName = "copycat-file"
dependencies {
compile project(':copycat:api')
compile "$slf4japi"
testCompile "$junit"
testCompile "$easymock"
testCompile "$powermock"
testCompile "$powermock_easymock"
testRuntime "$slf4jlog4j"
}
task testJar(type: Jar) {
classifier = 'test'
from sourceSets.test.output
}
test {
testLogging {
events "passed", "skipped", "failed"
exceptionFormat = 'full'
}
}
javadoc {
include "**/org/apache/kafka/copycat/*"
}
artifacts {
archives testJar
}
configurations {
archives.extendsFrom(testCompile)
}
checkstyle {
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
}
test.dependsOn('checkstyleMain', 'checkstyleTest')
}

View File

@ -108,4 +108,61 @@
<allow pkg="org.apache.kafka" />
</subpackage>
<subpackage name="copycat">
<allow pkg="org.apache.kafka.common" />
<allow pkg="org.apache.kafka.copycat.data" />
<allow pkg="org.apache.kafka.copycat.errors" />
<subpackage name="source">
<allow pkg="org.apache.kafka.copycat.connector" />
<allow pkg="org.apache.kafka.copycat.storage" />
</subpackage>
<subpackage name="sink">
<allow pkg="org.apache.kafka.copycat.connector" />
<allow pkg="org.apache.kafka.copycat.storage" />
</subpackage>
<subpackage name="runtime">
<allow pkg="org.apache.kafka.copycat" />
<allow pkg="org.apache.kafka.clients" />
<!-- for tests -->
<allow pkg="org.easymock" />
<allow pkg="org.powermock" />
</subpackage>
<subpackage name="cli">
<allow pkg="org.apache.kafka.copycat.runtime" />
<allow pkg="org.apache.kafka.copycat.util" />
<allow pkg="org.apache.kafka.common" />
</subpackage>
<subpackage name="storage">
<allow pkg="org.apache.kafka.copycat" />
<allow pkg="org.apache.kafka.common.serialization" />
<!-- for tests -->
<allow pkg="org.easymock" />
<allow pkg="org.powermock" />
</subpackage>
<subpackage name="util">
<allow pkg="org.apache.kafka.copycat" />
</subpackage>
<subpackage name="json">
<allow pkg="com.fasterxml.jackson" />
<allow pkg="org.apache.kafka.common.serialization" />
<allow pkg="org.apache.kafka.common.errors" />
<allow pkg="org.apache.kafka.copycat.storage" />
</subpackage>
<subpackage name="file">
<allow pkg="org.apache.kafka.copycat" />
<!-- for tests -->
<allow pkg="org.easymock" />
<allow pkg="org.powermock" />
</subpackage>
</subpackage>
</import-control>

View File

@ -271,16 +271,29 @@ public class Utils {
/**
* Instantiate the class
*/
public static Object newInstance(Class<?> c) {
public static <T> T newInstance(Class<T> c) {
try {
return c.newInstance();
} catch (IllegalAccessException e) {
throw new KafkaException("Could not instantiate class " + c.getName(), e);
} catch (InstantiationException e) {
throw new KafkaException("Could not instantiate class " + c.getName() + " Does it have a public no-argument constructor?", e);
} catch (NullPointerException e) {
throw new KafkaException("Requested class was null", e);
}
}
/**
* Look up the class by name and instantiate it.
* @param klass class name
* @param base super class of the class to be instantiated
* @param <T>
* @return the new instance
*/
public static <T> T newInstance(String klass, Class<T> base) throws ClassNotFoundException {
return Utils.newInstance(Class.forName(klass).asSubclass(base));
}
/**
* Generates 32 bit murmur2 hash from byte array
* @param data byte array to hash

View File

@ -0,0 +1,19 @@
# 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.
name=local-console-sink
connector.class=org.apache.kafka.copycat.file.FileStreamSinkConnector
tasks.max=1
topics=test

View File

@ -0,0 +1,19 @@
# 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.
name=local-console-source
connector.class=org.apache.kafka.copycat.file.FileStreamSourceConnector
tasks.max=1
topic=test

View File

@ -0,0 +1,20 @@
# 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.
name=local-file-sink
connector.class=org.apache.kafka.copycat.file.FileStreamSinkConnector
tasks.max=1
file=test.sink.txt
topics=test

View File

@ -0,0 +1,20 @@
# 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.
name=local-file-source
connector.class=org.apache.kafka.copycat.file.FileStreamSourceConnector
tasks.max=1
file=test.txt
topic=test

View File

@ -0,0 +1,23 @@
# 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.
log4j.rootLogger=INFO, stdout
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n
log4j.logger.org.apache.zookeeper=ERROR
log4j.logger.org.I0Itec.zkclient=ERROR

View File

@ -0,0 +1,28 @@
# 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.
# These are defaults. This file just demonstrates how to override some settings.
bootstrap.servers=localhost:9092
key.converter=org.apache.kafka.copycat.json.JsonConverter
value.converter=org.apache.kafka.copycat.json.JsonConverter
key.serializer=org.apache.kafka.copycat.json.JsonSerializer
value.serializer=org.apache.kafka.copycat.json.JsonSerializer
key.deserializer=org.apache.kafka.copycat.json.JsonDeserializer
value.deserializer=org.apache.kafka.copycat.json.JsonDeserializer
offset.storage.file.filename=/tmp/copycat.offsets
# Flush much faster than normal, which is useful for testing/debugging
offset.flush.interval.ms=10000

View File

@ -0,0 +1,117 @@
/**
* 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.copycat.connector;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.List;
import java.util.Properties;
/**
* <p>
* Connectors manage integration of Copycat with another system, either as an input that ingests
* data into Kafka or an output that passes data to an external system. Implementations should
* not use this class directly; they should inherit from SourceConnector or SinkConnector.
* </p>
* <p>
* Connectors have two primary tasks. First, given some configuration, they are responsible for
* creating configurations for a set of {@link Task}s that split up the data processing. For
* example, a database Connector might create Tasks by dividing the set of tables evenly among
* tasks. Second, they are responsible for monitoring inputs for changes that require
* reconfiguration and notifying the Copycat runtime via the ConnectorContext. Continuing the
* previous example, the connector might periodically check for new tables and notify Copycat of
* additions and deletions. Copycat will then request new configurations and update the running
* Tasks.
* </p>
*/
@InterfaceStability.Unstable
public abstract class Connector {
protected ConnectorContext context;
/**
* Initialize this connector, using the provided ConnectorContext to notify the runtime of
* input configuration changes.
* @param ctx context object used to interact with the Copycat runtime
*/
public void initialize(ConnectorContext ctx) {
context = ctx;
}
/**
* <p>
* Initialize this connector, using the provided ConnectorContext to notify the runtime of
* input configuration changes and using the provided set of Task configurations.
* This version is only used to recover from failures.
* </p>
* <p>
* The default implementation ignores the provided Task configurations. During recovery, Copycat will request
* an updated set of configurations and update the running Tasks appropriately. However, Connectors should
* implement special handling of this case if it will avoid unnecessary changes to running Tasks.
* </p>
*
* @param ctx context object used to interact with the Copycat runtime
* @param taskConfigs existing task configurations, which may be used when generating new task configs to avoid
* churn in partition to task assignments
*/
public void initialize(ConnectorContext ctx, List<Properties> taskConfigs) {
context = ctx;
// Ignore taskConfigs. May result in more churn of tasks during recovery if updated configs
// are very different, but reduces the difficulty of implementing a Connector
}
/**
* Start this Connector. This method will only be called on a clean Connector, i.e. it has
* either just been instantiated and initialized or {@link #stop()} has been invoked.
*
* @param props configuration settings
*/
public abstract void start(Properties props);
/**
* Reconfigure this Connector. Most implementations will not override this, using the default
* implementation that calls {@link #stop()} followed by {@link #start(Properties)}.
* Implementations only need to override this if they want to handle this process more
* efficiently, e.g. without shutting down network connections to the external system.
*
* @param props new configuration settings
*/
public void reconfigure(Properties props) {
stop();
start(props);
}
/**
* Returns the Task implementation for this Connector.
*/
public abstract Class<? extends Task> getTaskClass();
/**
* Returns a set of configurations for Tasks based on the current configuration,
* producing at most count configurations.
*
* @param maxTasks maximum number of configurations to generate
* @return configurations for Tasks
*/
public abstract List<Properties> getTaskConfigs(int maxTasks);
/**
* Stop this connector.
*/
public abstract void stop();
}

View File

@ -0,0 +1,33 @@
/**
* 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.copycat.connector;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* ConnectorContext allows Connectors to proactively interact with the Copycat runtime.
*/
@InterfaceStability.Unstable
public interface ConnectorContext {
/**
* Requests that the runtime reconfigure the Tasks for this source. This should be used to
* indicate to the runtime that something about the input/output has changed (e.g. partitions
* added/removed) and the running Tasks will need to be modified.
*/
void requestTaskReconfiguration();
}

View File

@ -0,0 +1,103 @@
/**
* 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.copycat.connector;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* <p>
* Base class for records containing data to be copied to/from Kafka. This corresponds closely to
* Kafka's ProducerRecord and ConsumerRecord classes, and holds the data that may be used by both
* sources and sinks (topic, kafkaPartition, key, value). Although both implementations include a
* notion of offset, it is not included here because they differ in type.
* </p>
*/
@InterfaceStability.Unstable
public abstract class CopycatRecord {
private final String topic;
private final Integer kafkaPartition;
private final Object key;
private final Object value;
public CopycatRecord(String topic, Integer kafkaPartition, Object value) {
this(topic, kafkaPartition, null, value);
}
public CopycatRecord(String topic, Integer kafkaPartition, Object key, Object value) {
this.topic = topic;
this.kafkaPartition = kafkaPartition;
this.key = key;
this.value = value;
}
public String getTopic() {
return topic;
}
public Integer getKafkaPartition() {
return kafkaPartition;
}
public Object getKey() {
return key;
}
public Object getValue() {
return value;
}
@Override
public String toString() {
return "CopycatRecord{" +
"topic='" + topic + '\'' +
", kafkaPartition=" + kafkaPartition +
", key=" + key +
", value=" + value +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
CopycatRecord that = (CopycatRecord) o;
if (key != null ? !key.equals(that.key) : that.key != null)
return false;
if (kafkaPartition != null ? !kafkaPartition.equals(that.kafkaPartition) : that.kafkaPartition != null)
return false;
if (topic != null ? !topic.equals(that.topic) : that.topic != null)
return false;
if (value != null ? !value.equals(that.value) : that.value != null)
return false;
return true;
}
@Override
public int hashCode() {
int result = topic != null ? topic.hashCode() : 0;
result = 31 * result + (kafkaPartition != null ? kafkaPartition.hashCode() : 0);
result = 31 * result + (key != null ? key.hashCode() : 0);
result = 31 * result + (value != null ? value.hashCode() : 0);
return result;
}
}

View File

@ -0,0 +1,49 @@
/**
* 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.copycat.connector;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Properties;
/**
* <p>
* Tasks contain the code that actually copies data to/from another system. They receive
* a configuration from their parent Connector, assigning them a fraction of a Copycat job's work.
* The Copycat framework then pushes/pulls data from the Task. The Task must also be able to
* respond to reconfiguration requests.
* </p>
* <p>
* Task only contains the minimal shared functionality between
* {@link org.apache.kafka.copycat.source.SourceTask} and
* {@link org.apache.kafka.copycat.sink.SinkTask}.
* </p>
*/
@InterfaceStability.Unstable
public interface Task {
/**
* Start the Task
* @param props initial configuration
*/
void start(Properties props);
/**
* Stop this task.
*/
void stop();
}

View File

@ -0,0 +1,40 @@
/**
* 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.copycat.errors;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* CopycatException is the top-level exception type generated by Copycat and connectors.
*/
@InterfaceStability.Unstable
public class CopycatException extends KafkaException {
public CopycatException(String s) {
super(s);
}
public CopycatException(String s, Throwable throwable) {
super(s, throwable);
}
public CopycatException(Throwable throwable) {
super(throwable);
}
}

View File

@ -0,0 +1,40 @@
/**
* 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.copycat.sink;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.copycat.connector.Connector;
/**
* SinkConnectors implement the Connector interface to send Kafka data to another system.
*/
@InterfaceStability.Unstable
public abstract class SinkConnector extends Connector {
/**
* <p>
* Configuration key for the list of input topics for this connector.
* </p>
* <p>
* Usually this setting is only relevant to the Copycat framework, but is provided here for
* the convenience of Connector developers if they also need to know the set of topics.
* </p>
*/
public static final String TOPICS_CONFIG = "topics";
}

View File

@ -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.copycat.sink;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.copycat.connector.CopycatRecord;
/**
* SinkRecord is a CopycatRecord that has been read from Kafka and includes the kafkaOffset of
* the record in the Kafka topic-partition in addition to the standard fields. This information
* should be used by the SinkTask to coordinate kafkaOffset commits.
*/
@InterfaceStability.Unstable
public class SinkRecord extends CopycatRecord {
private final long kafkaOffset;
public SinkRecord(String topic, int partition, Object key, Object value, long kafkaOffset) {
super(topic, partition, key, value);
this.kafkaOffset = kafkaOffset;
}
public long getKafkaOffset() {
return kafkaOffset;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
if (!super.equals(o))
return false;
SinkRecord that = (SinkRecord) o;
if (kafkaOffset != that.kafkaOffset)
return false;
return true;
}
@Override
public int hashCode() {
int result = super.hashCode();
result = 31 * result + (int) (kafkaOffset ^ (kafkaOffset >>> 32));
return result;
}
@Override
public String toString() {
return "SinkRecord{" +
"kafkaOffset=" + kafkaOffset +
"} " + super.toString();
}
}

View File

@ -0,0 +1,64 @@
/**
* 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.copycat.sink;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.copycat.connector.Task;
import java.util.Collection;
import java.util.Map;
/**
* SinkTask is a Task takes records loaded from Kafka and sends them to another system. In
* addition to the basic {@link #put} interface, SinkTasks must also implement {@link #flush}
* to support offset commits.
*/
@InterfaceStability.Unstable
public abstract class SinkTask implements Task {
/**
* <p>
* The configuration key that provides the list of topics that are inputs for this
* SinkTask.
* </p>
*/
public static final String TOPICS_CONFIG = "topics";
protected SinkTaskContext context;
public void initialize(SinkTaskContext context) {
this.context = context;
}
/**
* Put the records in the sink. Usually this should send the records to the sink asynchronously
* and immediately return.
*
* @param records the set of records to send
*/
public abstract void put(Collection<SinkRecord> records);
/**
* Flush all records that have been {@link #put} for the specified topic-partitions. The
* offsets are provided for convenience, but could also be determined by tracking all offsets
* included in the SinkRecords passed to {@link #put}.
*
* @param offsets mapping of TopicPartition to committed offset
*/
public abstract void flush(Map<TopicPartition, Long> offsets);
}

View File

@ -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.copycat.sink;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.HashMap;
import java.util.Map;
/**
* Context passed to SinkTasks, allowing them to access utilities in the copycat runtime.
*/
@InterfaceStability.Unstable
public abstract class SinkTaskContext {
private Map<TopicPartition, Long> offsets;
public SinkTaskContext() {
offsets = new HashMap<>();
}
/**
* Reset the consumer offsets for the given topic partitions. SinkTasks should use this when they are started
* if they manage offsets in the sink data store rather than using Kafka consumer offsets. For example, an HDFS
* connector might record offsets in HDFS to provide exactly once delivery. When the SinkTask is started or
* a rebalance occurs, the task would reload offsets from HDFS and use this method to reset the consumer to those
* offsets.
*
* SinkTasks that do not manage their own offsets do not need to use this method.
*
* @param offsets map of offsets for topic partitions
*/
public void resetOffset(Map<TopicPartition, Long> offsets) {
this.offsets = offsets;
}
/**
* Get offsets that the SinkTask has submitted to be reset. Used by the Copycat framework.
* @return the map of offsets
*/
public Map<TopicPartition, Long> getOffsets() {
return offsets;
}
}

View File

@ -0,0 +1,29 @@
/**
* 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.copycat.source;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.copycat.connector.Connector;
/**
* SourceConnectors implement the connector interface to pull data from another system and send
* it to Kafka.
*/
@InterfaceStability.Unstable
public abstract class SourceConnector extends Connector {
}

View File

@ -0,0 +1,103 @@
/**
* 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.copycat.source;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.copycat.connector.CopycatRecord;
/**
* <p>
* SourceRecords are generated by SourceTasks and passed to Copycat for storage in
* Kafka. In addition to the standard fields in CopycatRecord which specify where data is stored
* in Kafka, they also include a sourcePartition and sourceOffset.
* </p>
* <p>
* The sourcePartition represents a single input sourcePartition that the record came from (e.g. a filename, table
* name, or topic-partition). The sourceOffset represents a position in that sourcePartition which can be used
* to resume consumption of data.
* </p>
* <p>
* These values can have arbitrary structure and should be represented using
* org.apache.kafka.copycat.data objects (or primitive values). For example, a database connector
* might specify the sourcePartition as a record containing { "db": "database_name", "table":
* "table_name"} and the sourceOffset as a Long containing the timestamp of the row.
* </p>
*/
@InterfaceStability.Unstable
public class SourceRecord extends CopycatRecord {
private final Object sourcePartition;
private final Object sourceOffset;
public SourceRecord(Object sourcePartition, Object sourceOffset, String topic, Integer partition, Object value) {
this(sourcePartition, sourceOffset, topic, partition, null, value);
}
public SourceRecord(Object sourcePartition, Object sourceOffset, String topic, Object value) {
this(sourcePartition, sourceOffset, topic, null, null, value);
}
public SourceRecord(Object sourcePartition, Object sourceOffset, String topic, Integer partition,
Object key, Object value) {
super(topic, partition, key, value);
this.sourcePartition = sourcePartition;
this.sourceOffset = sourceOffset;
}
public Object getSourcePartition() {
return sourcePartition;
}
public Object getSourceOffset() {
return sourceOffset;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
if (!super.equals(o))
return false;
SourceRecord that = (SourceRecord) o;
if (sourceOffset != null ? !sourceOffset.equals(that.sourceOffset) : that.sourceOffset != null)
return false;
if (sourcePartition != null ? !sourcePartition.equals(that.sourcePartition) : that.sourcePartition != null)
return false;
return true;
}
@Override
public int hashCode() {
int result = super.hashCode();
result = 31 * result + (sourcePartition != null ? sourcePartition.hashCode() : 0);
result = 31 * result + (sourceOffset != null ? sourceOffset.hashCode() : 0);
return result;
}
@Override
public String toString() {
return "SourceRecord{" +
"sourcePartition=" + sourcePartition +
", sourceOffset=" + sourceOffset +
"} " + super.toString();
}
}

View File

@ -0,0 +1,62 @@
/**
* 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.copycat.source;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.copycat.connector.Task;
import java.util.List;
/**
* SourceTask is a Task that pulls records from another system for storage in Kafka.
*/
@InterfaceStability.Unstable
public abstract class SourceTask implements Task {
protected SourceTaskContext context;
/**
* Initialize this SourceTask with the specified context object.
*/
public void initialize(SourceTaskContext context) {
this.context = context;
}
/**
* Poll this SourceTask for new records. This method should block if no data is currently
* available.
*
* @return a list of source records
*/
public abstract List<SourceRecord> poll() throws InterruptedException;
/**
* <p>
* Commit the offsets, up to the offsets that have been returned by {@link #poll()}. This
* method should block until the commit is complete.
* </p>
* <p>
* SourceTasks are not required to implement this functionality; Copycat will record offsets
* automatically. This hook is provided for systems that also need to store offsets internally
* in their own system.
* </p>
*/
public void commit() throws InterruptedException {
// This space intentionally left blank.
}
}

View File

@ -0,0 +1,40 @@
/**
* 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.copycat.source;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.copycat.storage.OffsetStorageReader;
/**
* SourceTaskContext is provided to SourceTasks to allow them to interact with the underlying
* runtime.
*/
@InterfaceStability.Unstable
public class SourceTaskContext {
private final OffsetStorageReader reader;
public SourceTaskContext(OffsetStorageReader reader) {
this.reader = reader;
}
/**
* Get the OffsetStorageReader for this SourceTask.
*/
public OffsetStorageReader getOffsetStorageReader() {
return reader;
}
}

View File

@ -0,0 +1,45 @@
/**
* 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.copycat.storage;
import org.apache.kafka.common.annotation.InterfaceStability;
/**
* The Converter interface provides support for translating between Copycat's runtime data format
* and the "native" runtime format used by the serialization layer. This is used to translate
* two types of data: records and offsets. The (de)serialization is performed by a separate
* component -- the producer or consumer serializer or deserializer for records or a Copycat
* serializer or deserializer for offsets.
*/
@InterfaceStability.Unstable
public interface Converter<T> {
/**
* Convert a Copycat data object to a native object for serialization.
* @param value
* @return
*/
T fromCopycatData(Object value);
/**
* Convert a native object to a Copycat data object.
* @param value
* @return
*/
Object toCopycatData(T value);
}

View File

@ -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.copycat.storage;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.Map;
/**
* OffsetStorageReader provides access to the offset storage used by sources. This can be used by
* connectors to determine offsets to start consuming data from. This is most commonly used during
* initialization of a task, but can also be used during runtime, e.g. when reconfiguring a task.
*/
@InterfaceStability.Unstable
public interface OffsetStorageReader {
/**
* Get the offset for the specified partition. If the data isn't already available locally, this
* gets it from the backing store, which may require some network round trips.
*
* @param partition object uniquely identifying the partition of data
* @return object uniquely identifying the offset in the partition of data
*/
Object getOffset(Object partition);
/**
* <p>
* Get a set of offsets for the specified partition identifiers. This may be more efficient
* than calling {@link #getOffset(Object)} repeatedly.
* </p>
* <p>
* Note that when errors occur, this method omits the associated data and tries to return as
* many of the requested values as possible. This allows a task that's managing many partitions to
* still proceed with any available data. Therefore, implementations should take care to check
* that the data is actually available in the returned response. The only case when an
* exception will be thrown is if the entire request failed, e.g. because the underlying
* storage was unavailable.
* </p>
*
* @param partitions set of identifiers for partitions of data
* @return a map of partition identifiers to decoded offsets
*/
Map<Object, Object> getOffsets(Collection<Object> partitions);
}

View File

@ -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
*
* 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.copycat.util;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.ArrayList;
import java.util.List;
/**
* Utilities that connector implementations might find useful. Contains common building blocks
* for writing connectors.
*/
@InterfaceStability.Unstable
public class ConnectorUtils {
/**
* Given a list of elements and a target number of groups, generates list of groups of
* elements to match the target number of groups, spreading them evenly among the groups.
* This generates groups with contiguous elements, which results in intuitive ordering if
* your elements are also ordered (e.g. alphabetical lists of table names if you sort
* table names alphabetically to generate the raw partitions) or can result in efficient
* partitioning if elements are sorted according to some criteria that affects performance
* (e.g. topic partitions with the same leader).
*
* @param elements list of elements to partition
* @param numGroups the number of output groups to generate.
*/
public static <T> List<List<T>> groupPartitions(List<T> elements, int numGroups) {
if (numGroups <= 0)
throw new IllegalArgumentException("Number of groups must be positive.");
List<List<T>> result = new ArrayList<>(numGroups);
// Each group has either n+1 or n raw partitions
int perGroup = elements.size() / numGroups;
int leftover = elements.size() - (numGroups * perGroup);
int assigned = 0;
for (int group = 0; group < numGroups; group++) {
int numThisGroup = group < leftover ? perGroup + 1 : perGroup;
List<T> groupList = new ArrayList<>(numThisGroup);
for (int i = 0; i < numThisGroup; i++) {
groupList.add(elements.get(assigned));
assigned++;
}
result.add(groupList);
}
return result;
}
}

View File

@ -0,0 +1,76 @@
/**
* 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.copycat.connector;
import org.apache.kafka.copycat.errors.CopycatException;
import org.junit.Test;
import java.util.List;
import java.util.Properties;
import static org.junit.Assert.assertEquals;
public class ConnectorReconfigurationTest {
@Test
public void testDefaultReconfigure() throws Exception {
TestConnector conn = new TestConnector(false);
conn.reconfigure(new Properties());
assertEquals(conn.stopOrder, 0);
assertEquals(conn.configureOrder, 1);
}
@Test(expected = CopycatException.class)
public void testReconfigureStopException() throws Exception {
TestConnector conn = new TestConnector(true);
conn.reconfigure(new Properties());
}
private static class TestConnector extends Connector {
private boolean stopException;
private int order = 0;
public int stopOrder = -1;
public int configureOrder = -1;
public TestConnector(boolean stopException) {
this.stopException = stopException;
}
@Override
public void start(Properties props) {
configureOrder = order++;
}
@Override
public Class<? extends Task> getTaskClass() {
return null;
}
@Override
public List<Properties> getTaskConfigs(int count) {
return null;
}
@Override
public void stop() {
stopOrder = order++;
if (stopException)
throw new CopycatException("error");
}
}
}

View File

@ -0,0 +1,67 @@
/**
* 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.copycat.util;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import static org.junit.Assert.assertEquals;
public class ConnectorUtilsTest {
private static final List<Integer> FIVE_ELEMENTS = Arrays.asList(1, 2, 3, 4, 5);
@Test
public void testGroupPartitions() {
List<List<Integer>> grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 1);
assertEquals(Arrays.asList(FIVE_ELEMENTS), grouped);
grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 2);
assertEquals(Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5)), grouped);
grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 3);
assertEquals(Arrays.asList(Arrays.asList(1, 2),
Arrays.asList(3, 4),
Arrays.asList(5)), grouped);
grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 5);
assertEquals(Arrays.asList(Arrays.asList(1),
Arrays.asList(2),
Arrays.asList(3),
Arrays.asList(4),
Arrays.asList(5)), grouped);
grouped = ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 7);
assertEquals(Arrays.asList(Arrays.asList(1),
Arrays.asList(2),
Arrays.asList(3),
Arrays.asList(4),
Arrays.asList(5),
Collections.EMPTY_LIST,
Collections.EMPTY_LIST), grouped);
}
@Test(expected = IllegalArgumentException.class)
public void testGroupPartitionsInvalidCount() {
ConnectorUtils.groupPartitions(FIVE_ELEMENTS, 0);
}
}

View File

@ -0,0 +1,36 @@
/**
* 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.copycat.data;
/** Base Avro exception. */
public class DataRuntimeException extends RuntimeException {
public DataRuntimeException(Throwable cause) {
super(cause);
}
public DataRuntimeException(String message) {
super(message);
}
public DataRuntimeException(String message, Throwable cause) {
super(message, cause);
}
}

View File

@ -0,0 +1,33 @@
/**
* 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.copycat.data;
/** Thrown when an illegal type is used. */
public class DataTypeException extends DataRuntimeException {
public DataTypeException(String message) {
super(message);
}
public DataTypeException(String message, Throwable cause) {
super(message, cause);
}
}

View File

@ -0,0 +1,85 @@
/**
* 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.copycat.data;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
/**
* Base class for objects that have Object-valued properties.
*/
public abstract class ObjectProperties {
public static class Null {
private Null() {
}
}
/** A value representing a JSON <code>null</code>. */
public static final Null NULL_VALUE = new Null();
Map<String, Object> props = new LinkedHashMap<String, Object>(1);
private Set<String> reserved;
ObjectProperties(Set<String> reserved) {
this.reserved = reserved;
}
/**
* Returns the value of the named, string-valued property in this schema.
* Returns <tt>null</tt> if there is no string-valued property with that name.
*/
public String getProp(String name) {
Object value = getObjectProp(name);
return (value instanceof String) ? (String) value : null;
}
/**
* Returns the value of the named property in this schema.
* Returns <tt>null</tt> if there is no property with that name.
*/
public synchronized Object getObjectProp(String name) {
return props.get(name);
}
/**
* Adds a property with the given name <tt>name</tt> and
* value <tt>value</tt>. Neither <tt>name</tt> nor <tt>value</tt> can be
* <tt>null</tt>. It is illegal to add a property if another with
* the same name but different value already exists in this schema.
*
* @param name The name of the property to add
* @param value The value for the property to add
*/
public synchronized void addProp(String name, Object value) {
if (reserved.contains(name))
throw new DataRuntimeException("Can't set reserved property: " + name);
if (value == null)
throw new DataRuntimeException("Can't set a property to null: " + name);
Object old = props.get(name);
if (old == null)
props.put(name, value);
else if (!old.equals(value))
throw new DataRuntimeException("Can't overwrite property: " + name);
}
}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,32 @@
/**
* 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.copycat.data;
/** Thrown for errors parsing schemas and protocols. */
public class SchemaParseException extends DataRuntimeException {
public SchemaParseException(Throwable cause) {
super(cause);
}
public SchemaParseException(String message) {
super(message);
}
}

View File

@ -0,0 +1,62 @@
/**
* 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.copycat.file;
import org.apache.kafka.copycat.connector.Task;
import org.apache.kafka.copycat.sink.SinkConnector;
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
/**
* Very simple connector that works with the console. This connector supports both source and
* sink modes via its 'mode' setting.
*/
public class FileStreamSinkConnector extends SinkConnector {
public static final String FILE_CONFIG = "file";
private String filename;
@Override
public void start(Properties props) {
filename = props.getProperty(FILE_CONFIG);
}
@Override
public Class<? extends Task> getTaskClass() {
return FileStreamSinkTask.class;
}
@Override
public List<Properties> getTaskConfigs(int maxTasks) {
ArrayList<Properties> configs = new ArrayList<>();
for (int i = 0; i < maxTasks; i++) {
Properties config = new Properties();
if (filename != null)
config.setProperty(FILE_CONFIG, filename);
configs.add(config);
}
return configs;
}
@Override
public void stop() {
// Nothing to do since FileStreamSinkConnector has no background monitoring.
}
}

View File

@ -0,0 +1,79 @@
/**
* 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.copycat.file;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.sink.SinkRecord;
import org.apache.kafka.copycat.sink.SinkTask;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.PrintStream;
import java.util.Collection;
import java.util.Map;
import java.util.Properties;
/**
* FileStreamSinkTask writes records to stdout or a file.
*/
public class FileStreamSinkTask extends SinkTask {
private static final Logger log = LoggerFactory.getLogger(FileStreamSinkTask.class);
private PrintStream outputStream;
public FileStreamSinkTask() {
}
// for testing
public FileStreamSinkTask(PrintStream outputStream) {
this.outputStream = outputStream;
}
@Override
public void start(Properties props) {
String filename = props.getProperty(FileStreamSinkConnector.FILE_CONFIG);
if (filename == null) {
outputStream = System.out;
} else {
try {
outputStream = new PrintStream(new File(filename));
} catch (FileNotFoundException e) {
throw new CopycatException("Couldn't find or create file for FileStreamSinkTask: {}", e);
}
}
}
@Override
public void put(Collection<SinkRecord> sinkRecords) {
for (SinkRecord record : sinkRecords) {
outputStream.println(record.getValue());
}
}
@Override
public void flush(Map<TopicPartition, Long> offsets) {
outputStream.flush();
}
@Override
public void stop() {
}
}

View File

@ -0,0 +1,70 @@
/**
* 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.copycat.file;
import org.apache.kafka.copycat.connector.Task;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.source.SourceConnector;
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
/**
* Very simple connector that works with the console. This connector supports both source and
* sink modes via its 'mode' setting.
*/
public class FileStreamSourceConnector extends SourceConnector {
public static final String TOPIC_CONFIG = "topic";
public static final String FILE_CONFIG = "file";
private String filename;
private String topic;
@Override
public void start(Properties props) {
filename = props.getProperty(FILE_CONFIG);
topic = props.getProperty(TOPIC_CONFIG);
if (topic == null || topic.isEmpty())
throw new CopycatException("FileStreamSourceConnector configuration must include 'topic' setting");
if (topic.contains(","))
throw new CopycatException("FileStreamSourceConnector should only have a single topic when used as a source.");
}
@Override
public Class<? extends Task> getTaskClass() {
return FileStreamSourceTask.class;
}
@Override
public List<Properties> getTaskConfigs(int maxTasks) {
ArrayList<Properties> configs = new ArrayList<>();
// Only one input stream makes sense.
Properties config = new Properties();
if (filename != null)
config.setProperty(FILE_CONFIG, filename);
config.setProperty(TOPIC_CONFIG, topic);
configs.add(config);
return configs;
}
@Override
public void stop() {
// Nothing to do since FileStreamSourceConnector has no background monitoring.
}
}

View File

@ -0,0 +1,176 @@
/**
* 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.copycat.file;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.source.SourceRecord;
import org.apache.kafka.copycat.source.SourceTask;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.*;
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
/**
* FileStreamSourceTask reads from stdin or a file.
*/
public class FileStreamSourceTask extends SourceTask {
private static final Logger log = LoggerFactory.getLogger(FileStreamSourceTask.class);
private InputStream stream;
private BufferedReader reader = null;
private char[] buffer = new char[1024];
private int offset = 0;
private String topic = null;
private Long streamOffset;
@Override
public void start(Properties props) {
String filename = props.getProperty(FileStreamSourceConnector.FILE_CONFIG);
if (filename == null) {
stream = System.in;
// Tracking offset for stdin doesn't make sense
streamOffset = null;
} else {
try {
stream = new FileInputStream(filename);
Long lastRecordedOffset = (Long) context.getOffsetStorageReader().getOffset(null);
if (lastRecordedOffset != null) {
log.debug("Found previous offset, trying to skip to file offset {}", lastRecordedOffset);
long skipLeft = lastRecordedOffset;
while (skipLeft > 0) {
try {
long skipped = stream.skip(skipLeft);
skipLeft -= skipped;
} catch (IOException e) {
log.error("Error while trying to seek to previous offset in file: ", e);
throw new CopycatException(e);
}
}
log.debug("Skipped to offset {}", lastRecordedOffset);
}
streamOffset = (lastRecordedOffset != null) ? lastRecordedOffset : 0L;
} catch (FileNotFoundException e) {
throw new CopycatException("Couldn't find file for FileStreamSourceTask: {}", e);
}
}
topic = props.getProperty(FileStreamSourceConnector.TOPIC_CONFIG);
if (topic == null)
throw new CopycatException("ConsoleSourceTask config missing topic setting");
reader = new BufferedReader(new InputStreamReader(stream));
}
@Override
public List<SourceRecord> poll() throws InterruptedException {
// Unfortunately we can't just use readLine() because it blocks in an uninterruptible way.
// Instead we have to manage splitting lines ourselves, using simple backoff when no new data
// is available.
try {
final BufferedReader readerCopy;
synchronized (this) {
readerCopy = reader;
}
if (readerCopy == null)
return null;
ArrayList<SourceRecord> records = null;
int nread = 0;
while (readerCopy.ready()) {
nread = readerCopy.read(buffer, offset, buffer.length - offset);
if (nread > 0) {
offset += nread;
if (offset == buffer.length) {
char[] newbuf = new char[buffer.length * 2];
System.arraycopy(buffer, 0, newbuf, 0, buffer.length);
buffer = newbuf;
}
String line;
do {
line = extractLine();
if (line != null) {
if (records == null)
records = new ArrayList<>();
records.add(new SourceRecord(null, streamOffset, topic, line));
}
new ArrayList<SourceRecord>();
} while (line != null);
}
}
if (nread <= 0)
Thread.sleep(1);
return records;
} catch (IOException e) {
// Underlying stream was killed, probably as a result of calling stop. Allow to return
// null, and driving thread will handle any shutdown if necessary.
}
return null;
}
private String extractLine() {
int until = -1, newStart = -1;
for (int i = 0; i < offset; i++) {
if (buffer[i] == '\n') {
until = i;
newStart = i + 1;
break;
} else if (buffer[i] == '\r') {
// We need to check for \r\n, so we must skip this if we can't check the next char
if (i + 1 >= offset)
return null;
until = i;
newStart = (buffer[i + 1] == '\n') ? i + 2 : i + 1;
break;
}
}
if (until != -1) {
String result = new String(buffer, 0, until);
System.arraycopy(buffer, newStart, buffer, 0, buffer.length - newStart);
offset = offset - newStart;
if (streamOffset != null)
streamOffset += newStart;
return result;
} else {
return null;
}
}
@Override
public void stop() {
log.trace("Stopping");
synchronized (this) {
try {
stream.close();
log.trace("Closed input stream");
} catch (IOException e) {
log.error("Failed to close ConsoleSourceTask stream: ", e);
}
reader = null;
stream = null;
}
}
}

View File

@ -0,0 +1,85 @@
/**
* 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.copycat.file;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.copycat.connector.ConnectorContext;
import org.apache.kafka.copycat.sink.SinkConnector;
import org.junit.Before;
import org.junit.Test;
import org.powermock.api.easymock.PowerMock;
import java.util.Arrays;
import java.util.List;
import java.util.Properties;
import static org.junit.Assert.assertEquals;
public class FileStreamSinkConnectorTest {
private static final String MULTIPLE_TOPICS = "test1,test2";
private static final String[] MULTIPLE_TOPICS_LIST
= MULTIPLE_TOPICS.split(",");
private static final List<TopicPartition> MULTIPLE_TOPICS_PARTITIONS = Arrays.asList(
new TopicPartition("test1", 1), new TopicPartition("test2", 2)
);
private static final String FILENAME = "/afilename";
private FileStreamSinkConnector connector;
private ConnectorContext ctx;
private Properties sinkProperties;
@Before
public void setup() {
connector = new FileStreamSinkConnector();
ctx = PowerMock.createMock(ConnectorContext.class);
connector.initialize(ctx);
sinkProperties = new Properties();
sinkProperties.setProperty(SinkConnector.TOPICS_CONFIG, MULTIPLE_TOPICS);
sinkProperties.setProperty(FileStreamSinkConnector.FILE_CONFIG, FILENAME);
}
@Test
public void testSinkTasks() {
PowerMock.replayAll();
connector.start(sinkProperties);
List<Properties> taskConfigs = connector.getTaskConfigs(1);
assertEquals(1, taskConfigs.size());
assertEquals(FILENAME, taskConfigs.get(0).getProperty(FileStreamSinkConnector.FILE_CONFIG));
taskConfigs = connector.getTaskConfigs(2);
assertEquals(2, taskConfigs.size());
for (int i = 0; i < 2; i++) {
assertEquals(FILENAME, taskConfigs.get(0).getProperty(FileStreamSinkConnector.FILE_CONFIG));
}
PowerMock.verifyAll();
}
@Test
public void testTaskClass() {
PowerMock.replayAll();
connector.start(sinkProperties);
assertEquals(FileStreamSinkTask.class, connector.getTaskClass());
PowerMock.verifyAll();
}
}

View File

@ -0,0 +1,67 @@
/**
* 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.copycat.file;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.copycat.sink.SinkRecord;
import org.junit.Before;
import org.junit.Test;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.util.Arrays;
import java.util.HashMap;
import static org.junit.Assert.assertEquals;
public class FileStreamSinkTaskTest {
private FileStreamSinkTask task;
private ByteArrayOutputStream os;
private PrintStream printStream;
@Before
public void setup() {
os = new ByteArrayOutputStream();
printStream = new PrintStream(os);
task = new FileStreamSinkTask(printStream);
}
@Test
public void testPutFlush() {
HashMap<TopicPartition, Long> offsets = new HashMap<>();
// We do not call task.start() since it would override the output stream
task.put(Arrays.asList(
new SinkRecord("topic1", 0, null, "line1", 1)
));
offsets.put(new TopicPartition("topic1", 0), 1L);
task.flush(offsets);
assertEquals("line1\n", os.toString());
task.put(Arrays.asList(
new SinkRecord("topic1", 0, null, "line2", 2),
new SinkRecord("topic2", 0, null, "line3", 1)
));
offsets.put(new TopicPartition("topic1", 0), 2L);
offsets.put(new TopicPartition("topic2", 0), 1L);
task.flush(offsets);
assertEquals("line1\nline2\nline3\n", os.toString());
}
}

View File

@ -0,0 +1,104 @@
/**
* 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.copycat.file;
import org.apache.kafka.copycat.connector.ConnectorContext;
import org.apache.kafka.copycat.errors.CopycatException;
import org.junit.Before;
import org.junit.Test;
import org.powermock.api.easymock.PowerMock;
import java.util.List;
import java.util.Properties;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
public class FileStreamSourceConnectorTest {
private static final String SINGLE_TOPIC = "test";
private static final String MULTIPLE_TOPICS = "test1,test2";
private static final String FILENAME = "/somefilename";
private FileStreamSourceConnector connector;
private ConnectorContext ctx;
private Properties sourceProperties;
@Before
public void setup() {
connector = new FileStreamSourceConnector();
ctx = PowerMock.createMock(ConnectorContext.class);
connector.initialize(ctx);
sourceProperties = new Properties();
sourceProperties.setProperty(FileStreamSourceConnector.TOPIC_CONFIG, SINGLE_TOPIC);
sourceProperties.setProperty(FileStreamSourceConnector.FILE_CONFIG, FILENAME);
}
@Test
public void testSourceTasks() {
PowerMock.replayAll();
connector.start(sourceProperties);
List<Properties> taskConfigs = connector.getTaskConfigs(1);
assertEquals(1, taskConfigs.size());
assertEquals(FILENAME,
taskConfigs.get(0).getProperty(FileStreamSourceConnector.FILE_CONFIG));
assertEquals(SINGLE_TOPIC,
taskConfigs.get(0).getProperty(FileStreamSourceConnector.TOPIC_CONFIG));
// Should be able to return fewer than requested #
taskConfigs = connector.getTaskConfigs(2);
assertEquals(1, taskConfigs.size());
assertEquals(FILENAME,
taskConfigs.get(0).getProperty(FileStreamSourceConnector.FILE_CONFIG));
assertEquals(SINGLE_TOPIC,
taskConfigs.get(0).getProperty(FileStreamSourceConnector.TOPIC_CONFIG));
PowerMock.verifyAll();
}
@Test
public void testSourceTasksStdin() {
PowerMock.replayAll();
sourceProperties.remove(FileStreamSourceConnector.FILE_CONFIG);
connector.start(sourceProperties);
List<Properties> taskConfigs = connector.getTaskConfigs(1);
assertEquals(1, taskConfigs.size());
assertNull(taskConfigs.get(0).getProperty(FileStreamSourceConnector.FILE_CONFIG));
PowerMock.verifyAll();
}
@Test(expected = CopycatException.class)
public void testMultipleSourcesInvalid() {
sourceProperties.setProperty(FileStreamSourceConnector.TOPIC_CONFIG, MULTIPLE_TOPICS);
connector.start(sourceProperties);
}
@Test
public void testTaskClass() {
PowerMock.replayAll();
connector.start(sourceProperties);
assertEquals(FileStreamSourceTask.class, connector.getTaskClass());
PowerMock.verifyAll();
}
}

View File

@ -0,0 +1,140 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
**/
package org.apache.kafka.copycat.file;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.source.SourceRecord;
import org.apache.kafka.copycat.source.SourceTaskContext;
import org.apache.kafka.copycat.storage.OffsetStorageReader;
import org.easymock.EasyMock;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.powermock.api.easymock.PowerMock;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.List;
import java.util.Properties;
import static org.junit.Assert.assertEquals;
public class FileStreamSourceTaskTest {
private static final String TOPIC = "test";
private File tempFile;
private Properties config;
private OffsetStorageReader offsetStorageReader;
private FileStreamSourceTask task;
private boolean verifyMocks = false;
@Before
public void setup() throws IOException {
tempFile = File.createTempFile("file-stream-source-task-test", null);
config = new Properties();
config.setProperty(FileStreamSourceConnector.FILE_CONFIG, tempFile.getAbsolutePath());
config.setProperty(FileStreamSourceConnector.TOPIC_CONFIG, TOPIC);
task = new FileStreamSourceTask();
offsetStorageReader = PowerMock.createMock(OffsetStorageReader.class);
task.initialize(new SourceTaskContext(offsetStorageReader));
}
@After
public void teardown() {
tempFile.delete();
if (verifyMocks)
PowerMock.verifyAll();
}
private void replay() {
PowerMock.replayAll();
verifyMocks = true;
}
@Test
public void testNormalLifecycle() throws InterruptedException, IOException {
expectOffsetLookupReturnNone();
replay();
task.start(config);
FileOutputStream os = new FileOutputStream(tempFile);
assertEquals(null, task.poll());
os.write("partial line".getBytes());
os.flush();
assertEquals(null, task.poll());
os.write(" finished\n".getBytes());
os.flush();
List<SourceRecord> records = task.poll();
assertEquals(1, records.size());
assertEquals(TOPIC, records.get(0).getTopic());
assertEquals("partial line finished", records.get(0).getValue());
assertEquals(22L, records.get(0).getSourceOffset());
assertEquals(null, task.poll());
// Different line endings, and make sure the final \r doesn't result in a line until we can
// read the subsequent byte.
os.write("line1\rline2\r\nline3\nline4\n\r".getBytes());
os.flush();
records = task.poll();
assertEquals(4, records.size());
assertEquals("line1", records.get(0).getValue());
assertEquals(28L, records.get(0).getSourceOffset());
assertEquals("line2", records.get(1).getValue());
assertEquals(35L, records.get(1).getSourceOffset());
assertEquals("line3", records.get(2).getValue());
assertEquals(41L, records.get(2).getSourceOffset());
assertEquals("line4", records.get(3).getValue());
assertEquals(47L, records.get(3).getSourceOffset());
os.write("subsequent text".getBytes());
os.flush();
records = task.poll();
assertEquals(1, records.size());
assertEquals("", records.get(0).getValue());
assertEquals(48L, records.get(0).getSourceOffset());
task.stop();
}
@Test(expected = CopycatException.class)
public void testMissingTopic() {
expectOffsetLookupReturnNone();
replay();
config.remove(FileStreamSourceConnector.TOPIC_CONFIG);
task.start(config);
}
@Test(expected = CopycatException.class)
public void testInvalidFile() {
config.setProperty(FileStreamSourceConnector.FILE_CONFIG, "bogusfilename");
task.start(config);
}
private void expectOffsetLookupReturnNone() {
EasyMock.expect(
offsetStorageReader.getOffset(EasyMock.anyObject(Object.class)))
.andReturn(null);
}
}

View File

@ -0,0 +1,265 @@
/**
* 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.copycat.json;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import com.fasterxml.jackson.databind.node.ObjectNode;
import org.apache.kafka.copycat.data.*;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.storage.Converter;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.*;
/**
* Implementation of Converter that uses JSON to store schemas and objects.
*/
public class JsonConverter implements Converter<JsonNode> {
private static final HashMap<String, JsonToCopycatTypeConverter> TO_COPYCAT_CONVERTERS
= new HashMap<>();
static {
TO_COPYCAT_CONVERTERS.put(JsonSchema.BOOLEAN_TYPE_NAME, new JsonToCopycatTypeConverter() {
@Override
public Object convert(JsonNode jsonSchema, JsonNode value) {
return value.booleanValue();
}
});
TO_COPYCAT_CONVERTERS.put(JsonSchema.INT_TYPE_NAME, new JsonToCopycatTypeConverter() {
@Override
public Object convert(JsonNode jsonSchema, JsonNode value) {
return value.intValue();
}
});
TO_COPYCAT_CONVERTERS.put(JsonSchema.LONG_TYPE_NAME, new JsonToCopycatTypeConverter() {
@Override
public Object convert(JsonNode jsonSchema, JsonNode value) {
return value.longValue();
}
});
TO_COPYCAT_CONVERTERS.put(JsonSchema.FLOAT_TYPE_NAME, new JsonToCopycatTypeConverter() {
@Override
public Object convert(JsonNode jsonSchema, JsonNode value) {
return value.floatValue();
}
});
TO_COPYCAT_CONVERTERS.put(JsonSchema.DOUBLE_TYPE_NAME, new JsonToCopycatTypeConverter() {
@Override
public Object convert(JsonNode jsonSchema, JsonNode value) {
return value.doubleValue();
}
});
TO_COPYCAT_CONVERTERS.put(JsonSchema.BYTES_TYPE_NAME, new JsonToCopycatTypeConverter() {
@Override
public Object convert(JsonNode jsonSchema, JsonNode value) {
try {
return value.binaryValue();
} catch (IOException e) {
throw new CopycatException("Invalid bytes field", e);
}
}
});
TO_COPYCAT_CONVERTERS.put(JsonSchema.STRING_TYPE_NAME, new JsonToCopycatTypeConverter() {
@Override
public Object convert(JsonNode jsonSchema, JsonNode value) {
return value.textValue();
}
});
TO_COPYCAT_CONVERTERS.put(JsonSchema.ARRAY_TYPE_NAME, new JsonToCopycatTypeConverter() {
@Override
public Object convert(JsonNode jsonSchema, JsonNode value) {
JsonNode elemSchema = jsonSchema.get(JsonSchema.ARRAY_ITEMS_FIELD_NAME);
if (elemSchema == null)
throw new CopycatException("Array schema did not specify the element type");
ArrayList<Object> result = new ArrayList<>();
for (JsonNode elem : value) {
result.add(convertToCopycat(elemSchema, elem));
}
return result;
}
});
}
@Override
public JsonNode fromCopycatData(Object value) {
return convertToJsonWithSchemaEnvelope(value);
}
@Override
public Object toCopycatData(JsonNode value) {
if (!value.isObject() || value.size() != 2 || !value.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME) || !value.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME))
throw new CopycatException("JSON value converted to Copycat must be in envelope containing schema");
return convertToCopycat(value.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME), value.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
private static JsonNode asJsonSchema(Schema schema) {
switch (schema.getType()) {
case BOOLEAN:
return JsonSchema.BOOLEAN_SCHEMA;
case BYTES:
return JsonSchema.BYTES_SCHEMA;
case DOUBLE:
return JsonSchema.DOUBLE_SCHEMA;
case FLOAT:
return JsonSchema.FLOAT_SCHEMA;
case INT:
return JsonSchema.INT_SCHEMA;
case LONG:
return JsonSchema.LONG_SCHEMA;
case NULL:
throw new UnsupportedOperationException("null schema not supported");
case STRING:
return JsonSchema.STRING_SCHEMA;
case UNION: {
throw new UnsupportedOperationException("union schema not supported");
}
case ARRAY:
return JsonNodeFactory.instance.objectNode().put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.ARRAY_TYPE_NAME)
.set(JsonSchema.ARRAY_ITEMS_FIELD_NAME, asJsonSchema(schema.getElementType()));
case ENUM:
throw new UnsupportedOperationException("enum schema not supported");
case MAP:
throw new UnsupportedOperationException("map schema not supported");
default:
throw new CopycatException("Couldn't translate unsupported schema type " + schema.getType().getName() + ".");
}
}
private static Schema asCopycatSchema(JsonNode jsonSchema) {
if (jsonSchema.isNull())
return null;
JsonNode schemaTypeNode = jsonSchema.get(JsonSchema.SCHEMA_TYPE_FIELD_NAME);
if (schemaTypeNode == null || !schemaTypeNode.isTextual())
throw new CopycatException("Schema must contain 'type' field");
switch (schemaTypeNode.textValue()) {
case JsonSchema.BOOLEAN_TYPE_NAME:
return SchemaBuilder.builder().booleanType();
case JsonSchema.INT_TYPE_NAME:
return SchemaBuilder.builder().intType();
case JsonSchema.LONG_TYPE_NAME:
return SchemaBuilder.builder().longType();
case JsonSchema.FLOAT_TYPE_NAME:
return SchemaBuilder.builder().floatType();
case JsonSchema.DOUBLE_TYPE_NAME:
return SchemaBuilder.builder().doubleType();
case JsonSchema.BYTES_TYPE_NAME:
return SchemaBuilder.builder().bytesType();
case JsonSchema.STRING_TYPE_NAME:
return SchemaBuilder.builder().stringType();
case JsonSchema.ARRAY_TYPE_NAME:
JsonNode elemSchema = jsonSchema.get(JsonSchema.ARRAY_ITEMS_FIELD_NAME);
if (elemSchema == null)
throw new CopycatException("Array schema did not specify the element type");
return Schema.createArray(asCopycatSchema(elemSchema));
default:
throw new CopycatException("Unknown schema type: " + schemaTypeNode.textValue());
}
}
/**
* Convert this object, in org.apache.kafka.copycat.data format, into a JSON object with an envelope object
* containing schema and payload fields.
* @param value
* @return
*/
private static JsonNode convertToJsonWithSchemaEnvelope(Object value) {
return convertToJson(value).toJsonNode();
}
/**
* Convert this object, in the org.apache.kafka.copycat.data format, into a JSON object, returning both the schema
* and the converted object.
*/
private static JsonSchema.Envelope convertToJson(Object value) {
if (value == null) {
return JsonSchema.nullEnvelope();
} else if (value instanceof Boolean) {
return JsonSchema.booleanEnvelope((Boolean) value);
} else if (value instanceof Byte) {
return JsonSchema.intEnvelope((Byte) value);
} else if (value instanceof Short) {
return JsonSchema.intEnvelope((Short) value);
} else if (value instanceof Integer) {
return JsonSchema.intEnvelope((Integer) value);
} else if (value instanceof Long) {
return JsonSchema.longEnvelope((Long) value);
} else if (value instanceof Float) {
return JsonSchema.floatEnvelope((Float) value);
} else if (value instanceof Double) {
return JsonSchema.doubleEnvelope((Double) value);
} else if (value instanceof byte[]) {
return JsonSchema.bytesEnvelope((byte[]) value);
} else if (value instanceof ByteBuffer) {
return JsonSchema.bytesEnvelope(((ByteBuffer) value).array());
} else if (value instanceof CharSequence) {
return JsonSchema.stringEnvelope(value.toString());
} else if (value instanceof Collection) {
Collection collection = (Collection) value;
ObjectNode schema = JsonNodeFactory.instance.objectNode().put(JsonSchema.SCHEMA_TYPE_FIELD_NAME, JsonSchema.ARRAY_TYPE_NAME);
ArrayNode list = JsonNodeFactory.instance.arrayNode();
JsonNode itemSchema = null;
for (Object elem : collection) {
JsonSchema.Envelope fieldSchemaAndValue = convertToJson(elem);
if (itemSchema == null) {
itemSchema = fieldSchemaAndValue.schema;
schema.set(JsonSchema.ARRAY_ITEMS_FIELD_NAME, itemSchema);
} else {
if (!itemSchema.equals(fieldSchemaAndValue.schema))
throw new CopycatException("Mismatching schemas found in a list.");
}
list.add(fieldSchemaAndValue.payload);
}
return new JsonSchema.Envelope(schema, list);
}
throw new CopycatException("Couldn't convert " + value + " to Avro.");
}
private static Object convertToCopycat(JsonNode jsonSchema, JsonNode jsonValue) {
if (jsonSchema.isNull())
return null;
JsonNode schemaTypeNode = jsonSchema.get(JsonSchema.SCHEMA_TYPE_FIELD_NAME);
if (schemaTypeNode == null || !schemaTypeNode.isTextual())
throw new CopycatException("Schema must contain 'type' field. Schema: " + jsonSchema.toString());
JsonToCopycatTypeConverter typeConverter = TO_COPYCAT_CONVERTERS.get(schemaTypeNode.textValue());
if (typeConverter != null)
return typeConverter.convert(jsonSchema, jsonValue);
throw new CopycatException("Unknown schema type: " + schemaTypeNode);
}
private interface JsonToCopycatTypeConverter {
Object convert(JsonNode schema, JsonNode value);
}
}

View File

@ -0,0 +1,87 @@
/**
* 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.copycat.json;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import com.fasterxml.jackson.databind.node.ObjectNode;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.serialization.Deserializer;
import java.util.Map;
/**
* JSON deserializer for Jackson's JsonNode tree model. Using the tree model allows it to work with arbitrarily
* structured data without having associated Java classes. This deserializer also supports Copycat schemas.
*/
public class JsonDeserializer implements Deserializer<JsonNode> {
private static final ObjectNode CATCH_ALL_OBJECT_SCHEMA = JsonNodeFactory.instance.objectNode();
private static final ObjectNode CATCH_ALL_ARRAY_SCHEMA = JsonNodeFactory.instance.objectNode();
private static final ArrayNode ALL_SCHEMAS_LIST = JsonNodeFactory.instance.arrayNode();
private static final ObjectNode CATCH_ALL_SCHEMA = JsonNodeFactory.instance.objectNode();
static {
CATCH_ALL_OBJECT_SCHEMA.put("type", "object")
.putArray("field").add(JsonNodeFactory.instance.objectNode().put("*", "all"));
CATCH_ALL_ARRAY_SCHEMA.put("type", "array").put("items", "all");
ALL_SCHEMAS_LIST.add("boolean").add("int").add("long").add("float").add("double").add("bytes").add("string")
.add(CATCH_ALL_ARRAY_SCHEMA).add(CATCH_ALL_OBJECT_SCHEMA);
CATCH_ALL_SCHEMA.put("name", "all").set("type", ALL_SCHEMAS_LIST);
}
private ObjectMapper objectMapper = new ObjectMapper();
/**
* Default constructor needed by Kafka
*/
public JsonDeserializer() {
}
@Override
public void configure(Map<String, ?> props, boolean isKey) {
}
@Override
public JsonNode deserialize(String topic, byte[] bytes) {
JsonNode data;
try {
data = objectMapper.readTree(bytes);
} catch (Exception e) {
throw new SerializationException(e);
}
// The deserialized data should either be an envelope object containing the schema and the payload or the schema
// was stripped during serialization and we need to fill in an all-encompassing schema.
if (!data.isObject() || data.size() != 2 || !data.has("schema") || !data.has("payload")) {
ObjectNode envelope = JsonNodeFactory.instance.objectNode();
envelope.set("schema", CATCH_ALL_SCHEMA);
envelope.set("payload", data);
data = envelope;
}
return data;
}
@Override
public void close() {
}
}

View File

@ -0,0 +1,114 @@
/**
* 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.copycat.json;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import com.fasterxml.jackson.databind.node.ObjectNode;
import java.nio.ByteBuffer;
public class JsonSchema {
static final String ENVELOPE_SCHEMA_FIELD_NAME = "schema";
static final String ENVELOPE_PAYLOAD_FIELD_NAME = "payload";
static final String SCHEMA_TYPE_FIELD_NAME = "type";
static final String SCHEMA_NAME_FIELD_NAME = "name";
static final String ARRAY_ITEMS_FIELD_NAME = "items";
static final String BOOLEAN_TYPE_NAME = "boolean";
static final JsonNode BOOLEAN_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, BOOLEAN_TYPE_NAME);
static final String INT_TYPE_NAME = "int";
static final JsonNode INT_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, INT_TYPE_NAME);
static final String LONG_TYPE_NAME = "long";
static final JsonNode LONG_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, LONG_TYPE_NAME);
static final String FLOAT_TYPE_NAME = "float";
static final JsonNode FLOAT_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, FLOAT_TYPE_NAME);
static final String DOUBLE_TYPE_NAME = "double";
static final JsonNode DOUBLE_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, DOUBLE_TYPE_NAME);
static final String BYTES_TYPE_NAME = "bytes";
static final JsonNode BYTES_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, BYTES_TYPE_NAME);
static final String STRING_TYPE_NAME = "string";
static final JsonNode STRING_SCHEMA = JsonNodeFactory.instance.objectNode().put(SCHEMA_TYPE_FIELD_NAME, STRING_TYPE_NAME);
static final String ARRAY_TYPE_NAME = "array";
public static ObjectNode envelope(JsonNode schema, JsonNode payload) {
ObjectNode result = JsonNodeFactory.instance.objectNode();
result.set(ENVELOPE_SCHEMA_FIELD_NAME, schema);
result.set(ENVELOPE_PAYLOAD_FIELD_NAME, payload);
return result;
}
static class Envelope {
public JsonNode schema;
public JsonNode payload;
public Envelope(JsonNode schema, JsonNode payload) {
this.schema = schema;
this.payload = payload;
}
public ObjectNode toJsonNode() {
return envelope(schema, payload);
}
}
public static Envelope nullEnvelope() {
return new Envelope(null, null);
}
public static Envelope booleanEnvelope(boolean value) {
return new Envelope(JsonSchema.BOOLEAN_SCHEMA, JsonNodeFactory.instance.booleanNode(value));
}
public static Envelope intEnvelope(byte value) {
return new Envelope(JsonSchema.INT_SCHEMA, JsonNodeFactory.instance.numberNode(value));
}
public static Envelope intEnvelope(short value) {
return new Envelope(JsonSchema.INT_SCHEMA, JsonNodeFactory.instance.numberNode(value));
}
public static Envelope intEnvelope(int value) {
return new Envelope(JsonSchema.INT_SCHEMA, JsonNodeFactory.instance.numberNode(value));
}
public static Envelope longEnvelope(long value) {
return new Envelope(JsonSchema.LONG_SCHEMA, JsonNodeFactory.instance.numberNode(value));
}
public static Envelope floatEnvelope(float value) {
return new Envelope(JsonSchema.FLOAT_SCHEMA, JsonNodeFactory.instance.numberNode(value));
}
public static Envelope doubleEnvelope(double value) {
return new Envelope(JsonSchema.DOUBLE_SCHEMA, JsonNodeFactory.instance.numberNode(value));
}
public static Envelope bytesEnvelope(byte[] value) {
return new Envelope(JsonSchema.BYTES_SCHEMA, JsonNodeFactory.instance.binaryNode(value));
}
public static Envelope bytesEnvelope(ByteBuffer value) {
return new Envelope(JsonSchema.BYTES_SCHEMA, JsonNodeFactory.instance.binaryNode(value.array()));
}
public static Envelope stringEnvelope(CharSequence value) {
return new Envelope(JsonSchema.STRING_SCHEMA, JsonNodeFactory.instance.textNode(value.toString()));
}
}

View File

@ -0,0 +1,72 @@
/**
* 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.copycat.json;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.serialization.Serializer;
import java.util.Map;
/**
* Serialize Jackson JsonNode tree model objects to UTF-8 JSON. Using the tree model allows handling arbitrarily
* structured data without corresponding Java classes. This serializer also supports Copycat schemas.
*/
public class JsonSerializer implements Serializer<JsonNode> {
private static final String SCHEMAS_ENABLE_CONFIG = "schemas.enable";
private static final boolean SCHEMAS_ENABLE_DEFAULT = true;
private final ObjectMapper objectMapper = new ObjectMapper();
private boolean enableSchemas = SCHEMAS_ENABLE_DEFAULT;
/**
* Default constructor needed by Kafka
*/
public JsonSerializer() {
}
@Override
public void configure(Map<String, ?> config, boolean isKey) {
Object enableConfigsVal = config.get(SCHEMAS_ENABLE_CONFIG);
if (enableConfigsVal != null)
enableSchemas = enableConfigsVal.toString().equals("true");
}
@Override
public byte[] serialize(String topic, JsonNode data) {
// This serializer works for Copycat data that requires a schema to be included, so we expect it to have a
// specific format: { "schema": {...}, "payload": ... }.
if (!data.isObject() || data.size() != 2 || !data.has("schema") || !data.has("payload"))
throw new SerializationException("JsonSerializer requires \"schema\" and \"payload\" fields and may not contain additional fields");
try {
if (!enableSchemas)
data = data.get("payload");
return objectMapper.writeValueAsBytes(data);
} catch (Exception e) {
throw new SerializationException("Error serializing JSON message", e);
}
}
@Override
public void close() {
}
}

View File

@ -0,0 +1,173 @@
/**
* 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.copycat.json;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import org.junit.Test;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class JsonConverterTest {
ObjectMapper objectMapper = new ObjectMapper();
JsonConverter converter = new JsonConverter();
@Test
public void booleanToCopycat() {
assertEquals(true, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"boolean\" }, \"payload\": true }")));
assertEquals(false, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"boolean\" }, \"payload\": false }")));
}
@Test
public void intToCopycat() {
assertEquals(12, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"int\" }, \"payload\": 12 }")));
}
@Test
public void longToCopycat() {
assertEquals(12L, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"long\" }, \"payload\": 12 }")));
assertEquals(4398046511104L, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"long\" }, \"payload\": 4398046511104 }")));
}
@Test
public void floatToCopycat() {
assertEquals(12.34f, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"float\" }, \"payload\": 12.34 }")));
}
@Test
public void doubleToCopycat() {
assertEquals(12.34, converter.toCopycatData(parse("{ \"schema\": { \"type\": \"double\" }, \"payload\": 12.34 }")));
}
@Test
public void bytesToCopycat() throws UnsupportedEncodingException {
ByteBuffer reference = ByteBuffer.wrap("test-string".getBytes("UTF-8"));
String msg = "{ \"schema\": { \"type\": \"bytes\" }, \"payload\": \"dGVzdC1zdHJpbmc=\" }";
ByteBuffer converted = ByteBuffer.wrap((byte[]) converter.toCopycatData(parse(msg)));
assertEquals(reference, converted);
}
@Test
public void stringToCopycat() {
assertEquals("foo-bar-baz", converter.toCopycatData(parse("{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }")));
}
@Test
public void arrayToCopycat() {
JsonNode arrayJson = parse("{ \"schema\": { \"type\": \"array\", \"items\": { \"type\" : \"int\" } }, \"payload\": [1, 2, 3] }");
assertEquals(Arrays.asList(1, 2, 3), converter.toCopycatData(arrayJson));
}
@Test
public void booleanToJson() {
JsonNode converted = converter.fromCopycatData(true);
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"boolean\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(true, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).booleanValue());
}
@Test
public void intToJson() {
JsonNode converted = converter.fromCopycatData(12);
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"int\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(12, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).intValue());
}
@Test
public void longToJson() {
JsonNode converted = converter.fromCopycatData(4398046511104L);
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"long\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(4398046511104L, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).longValue());
}
@Test
public void floatToJson() {
JsonNode converted = converter.fromCopycatData(12.34f);
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"float\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(12.34f, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).floatValue(), 0.001);
}
@Test
public void doubleToJson() {
JsonNode converted = converter.fromCopycatData(12.34);
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"double\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(12.34, converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).doubleValue(), 0.001);
}
@Test
public void bytesToJson() throws IOException {
JsonNode converted = converter.fromCopycatData("test-string".getBytes());
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"bytes\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(ByteBuffer.wrap("test-string".getBytes()),
ByteBuffer.wrap(converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).binaryValue()));
}
@Test
public void stringToJson() {
JsonNode converted = converter.fromCopycatData("test-string");
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"string\" }"), converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals("test-string", converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME).textValue());
}
@Test
public void arrayToJson() {
JsonNode converted = converter.fromCopycatData(Arrays.asList(1, 2, 3));
validateEnvelope(converted);
assertEquals(parse("{ \"type\": \"array\", \"items\": { \"type\": \"int\" } }"),
converted.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertEquals(JsonNodeFactory.instance.arrayNode().add(1).add(2).add(3),
converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
private JsonNode parse(String json) {
try {
return objectMapper.readTree(json);
} catch (IOException e) {
fail("IOException during JSON parse: " + e.getMessage());
throw new RuntimeException("failed");
}
}
private void validateEnvelope(JsonNode env) {
assertNotNull(env);
assertTrue(env.isObject());
assertEquals(2, env.size());
assertTrue(env.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME));
assertTrue(env.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isObject());
assertTrue(env.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
}
}

View File

@ -0,0 +1,87 @@
/**
* 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.copycat.cli;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.runtime.Herder;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.runtime.standalone.StandaloneHerder;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.FutureCallback;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Arrays;
import java.util.Properties;
/**
* <p>
* Command line utility that runs Copycat as a standalone process. In this mode, work is not
* distributed. Instead, all the normal Copycat machinery works within a single process. This is
* useful for ad hoc, small, or experimental jobs.
* </p>
* <p>
* By default, no job configs or offset data is persistent. You can make jobs persistent and
* fault tolerant by overriding the settings to use file storage for both.
* </p>
*/
@InterfaceStability.Unstable
public class CopycatStandalone {
private static final Logger log = LoggerFactory.getLogger(CopycatStandalone.class);
public static void main(String[] args) throws Exception {
Properties workerProps;
Properties connectorProps;
if (args.length < 2) {
log.info("Usage: CopycatStandalone worker.properties connector1.properties [connector2.properties ...]");
System.exit(1);
}
String workerPropsFile = args[0];
workerProps = !workerPropsFile.isEmpty() ? Utils.loadProps(workerPropsFile) : new Properties();
WorkerConfig workerConfig = new WorkerConfig(workerProps);
Worker worker = new Worker(workerConfig);
Herder herder = new StandaloneHerder(worker);
final org.apache.kafka.copycat.runtime.Copycat copycat = new org.apache.kafka.copycat.runtime.Copycat(worker, herder);
copycat.start();
try {
for (final String connectorPropsFile : Arrays.copyOfRange(args, 1, args.length)) {
connectorProps = Utils.loadProps(connectorPropsFile);
FutureCallback<String> cb = new FutureCallback<>(new Callback<String>() {
@Override
public void onCompletion(Throwable error, String id) {
if (error != null)
log.error("Failed to create job for {}", connectorPropsFile);
}
});
herder.addConnector(connectorProps, cb);
cb.get();
}
} catch (Throwable t) {
log.error("Stopping after connector error", t);
copycat.stop();
}
// Shutdown will be triggered by Ctrl-C or via HTTP shutdown request
copycat.awaitStop();
}
}

View File

@ -0,0 +1,141 @@
/**
* 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.copycat.cli;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import java.util.Properties;
import java.util.Set;
/**
* Configuration for standalone workers.
*/
@InterfaceStability.Unstable
public class WorkerConfig extends AbstractConfig {
public static final String CLUSTER_CONFIG = "cluster";
private static final String
CLUSTER_CONFIG_DOC =
"ID for this cluster, which is used to provide a namespace so multiple Copycat clusters "
+ "or instances may co-exist while sharing a single Kafka cluster.";
public static final String CLUSTER_DEFAULT = "copycat";
public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
public static final String BOOTSTRAP_SERVERS_DOC
= "A list of host/port pairs to use for establishing the initial connection to the Kafka "
+ "cluster. The client will make use of all servers irrespective of which servers are "
+ "specified here for bootstrapping&mdash;this list only impacts the initial hosts used "
+ "to discover the full set of servers. This list should be in the form "
+ "<code>host1:port1,host2:port2,...</code>. Since these servers are just used for the "
+ "initial connection to discover the full cluster membership (which may change "
+ "dynamically), this list need not contain the full set of servers (you may want more "
+ "than one, though, in case a server is down).";
public static final String BOOTSTRAP_SERVERS_DEFAULT = "localhost:9092";
public static final String KEY_CONVERTER_CLASS_CONFIG = "key.converter";
public static final String KEY_CONVERTER_CLASS_DOC =
"Converter class for key Copycat data that implements the <code>Converter</code> interface.";
public static final String VALUE_CONVERTER_CLASS_CONFIG = "value.converter";
public static final String VALUE_CONVERTER_CLASS_DOC =
"Converter class for value Copycat data that implements the <code>Converter</code> interface.";
public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer";
public static final String KEY_SERIALIZER_CLASS_DOC =
"Serializer class for key that implements the <code>Serializer</code> interface.";
public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer";
public static final String VALUE_SERIALIZER_CLASS_DOC =
"Serializer class for value that implements the <code>Serializer</code> interface.";
public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer";
public static final String KEY_DESERIALIZER_CLASS_DOC =
"Serializer class for key that implements the <code>Deserializer</code> interface.";
public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer";
public static final String VALUE_DESERIALIZER_CLASS_DOC =
"Deserializer class for value that implements the <code>Deserializer</code> interface.";
public static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG
= "task.shutdown.graceful.timeout.ms";
private static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DOC =
"Amount of time to wait for tasks to shutdown gracefully. This is the total amount of time,"
+ " not per task. All task have shutdown triggered, then they are waited on sequentially.";
private static final String TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DEFAULT = "5000";
public static final String OFFSET_COMMIT_INTERVAL_MS_CONFIG = "offset.flush.interval.ms";
private static final String OFFSET_COMMIT_INTERVAL_MS_DOC
= "Interval at which to try committing offsets for tasks.";
public static final long OFFSET_COMMIT_INTERVAL_MS_DEFAULT = 60000L;
public static final String OFFSET_COMMIT_TIMEOUT_MS_CONFIG = "offset.flush.timeout.ms";
private static final String OFFSET_COMMIT_TIMEOUT_MS_DOC
= "Maximum number of milliseconds to wait for records to flush and partition offset data to be"
+ " committed to offset storage before cancelling the process and restoring the offset "
+ "data to be committed in a future attempt.";
public static final long OFFSET_COMMIT_TIMEOUT_MS_DEFAULT = 5000L;
private static ConfigDef config;
static {
config = new ConfigDef()
.define(CLUSTER_CONFIG, Type.STRING, CLUSTER_DEFAULT, Importance.HIGH, CLUSTER_CONFIG_DOC)
.define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, BOOTSTRAP_SERVERS_DEFAULT,
Importance.HIGH, BOOTSTRAP_SERVERS_DOC)
.define(KEY_CONVERTER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, KEY_CONVERTER_CLASS_DOC)
.define(VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, VALUE_CONVERTER_CLASS_DOC)
.define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, KEY_SERIALIZER_CLASS_DOC)
.define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC)
.define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, KEY_DESERIALIZER_CLASS_DOC)
.define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS,
Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC)
.define(TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG, Type.LONG,
TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DEFAULT, Importance.LOW,
TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_DOC)
.define(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Type.LONG, OFFSET_COMMIT_INTERVAL_MS_DEFAULT,
Importance.LOW, OFFSET_COMMIT_INTERVAL_MS_DOC)
.define(OFFSET_COMMIT_TIMEOUT_MS_CONFIG, Type.LONG, OFFSET_COMMIT_TIMEOUT_MS_DEFAULT,
Importance.LOW, OFFSET_COMMIT_TIMEOUT_MS_DOC);
}
public WorkerConfig() {
this(new Properties());
}
public WorkerConfig(Properties props) {
super(config, props);
}
public Properties getUnusedProperties() {
Set<String> unusedKeys = this.unused();
Properties unusedProps = new Properties();
for (String key : unusedKeys) {
unusedProps.put(key, this.originals().get(key));
}
return unusedProps;
}
}

View File

@ -0,0 +1,87 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import java.util.Properties;
import java.util.Set;
/**
* <p>
* Configuration options for Connectors. These only include Copycat system-level configuration
* options (e.g. Connector class name, timeouts used by Copycat to control the connector) but does
* not include Connector-specific options (e.g. database connection settings).
* </p>
* <p>
* Note that some of these options are not required for all connectors. For example TOPICS_CONFIG
* is sink-specific.
* </p>
*/
public class ConnectorConfig extends AbstractConfig {
public static final String NAME_CONFIG = "name";
private static final String NAME_DOC = "Globally unique name to use for this connector.";
public static final String CONNECTOR_CLASS_CONFIG = "connector.class";
private static final String CONNECTOR_CLASS_DOC =
"Name of the class for this connector. Must be a subclass of org.apache.kafka.copycat.connector"
+ ".Connector";
public static final String TASKS_MAX_CONFIG = "tasks.max";
private static final String TASKS_MAX_DOC = "Maximum number of tasks to use for this connector.";
public static final int TASKS_MAX_DEFAULT = 1;
public static final String TOPICS_CONFIG = "topics";
private static final String TOPICS_DOC = "";
public static final String TOPICS_DEFAULT = "";
private static ConfigDef config;
static {
config = new ConfigDef()
.define(NAME_CONFIG, Type.STRING, Importance.HIGH, NAME_DOC)
.define(CONNECTOR_CLASS_CONFIG, Type.STRING, Importance.HIGH, CONNECTOR_CLASS_DOC)
.define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, Importance.HIGH, TASKS_MAX_DOC)
.define(TOPICS_CONFIG, Type.LIST, TOPICS_DEFAULT, Importance.HIGH, TOPICS_DOC);
}
private Properties originalProperties;
public ConnectorConfig() {
this(new Properties());
}
public ConnectorConfig(Properties props) {
super(config, props);
this.originalProperties = props;
}
public Properties getUnusedProperties() {
Set<String> unusedKeys = this.unused();
Properties unusedProps = new Properties();
for (String key : unusedKeys) {
unusedProps.setProperty(key, originalProperties.getProperty(key));
}
return unusedProps;
}
}

View File

@ -0,0 +1,94 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* This class ties together all the components of a Copycat process (herder, worker,
* storage, command interface), managing their lifecycle.
*/
@InterfaceStability.Unstable
public class Copycat {
private static final Logger log = LoggerFactory.getLogger(Copycat.class);
private final Worker worker;
private final Herder herder;
private final CountDownLatch startLatch = new CountDownLatch(1);
private final CountDownLatch stopLatch = new CountDownLatch(1);
private final AtomicBoolean shutdown = new AtomicBoolean(false);
private final ShutdownHook shutdownHook;
public Copycat(Worker worker, Herder herder) {
log.debug("Copycat created");
this.worker = worker;
this.herder = herder;
shutdownHook = new ShutdownHook();
}
public void start() {
log.info("Copycat starting");
Runtime.getRuntime().addShutdownHook(shutdownHook);
worker.start();
herder.start();
log.info("Copycat started");
startLatch.countDown();
}
public void stop() {
boolean wasShuttingDown = shutdown.getAndSet(true);
if (!wasShuttingDown) {
log.info("Copycat stopping");
herder.stop();
worker.stop();
log.info("Copycat stopped");
}
stopLatch.countDown();
}
public void awaitStop() {
try {
stopLatch.await();
} catch (InterruptedException e) {
log.error("Interrupted waiting for Copycat to shutdown");
}
}
private class ShutdownHook extends Thread {
@Override
public void run() {
try {
startLatch.await();
Copycat.this.stop();
} catch (InterruptedException e) {
log.error("Interrupted in shutdown hook while waiting for copycat startup to finish");
}
}
}
}

View File

@ -0,0 +1,67 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.copycat.util.Callback;
import java.util.Properties;
/**
* <p>
* The herder interface tracks and manages workers and connectors. It is the main interface for external components
* to make changes to the state of the cluster. For example, in distributed mode, an implementation of this class
* knows how to accept a connector configuration, may need to route it to the current leader worker for the cluster so
* the config can be written to persistent storage, and then ensures the new connector is correctly instantiated on one
* of the workers.
* </p>
* <p>
* This class must implement all the actions that can be taken on the cluster (add/remove connectors, pause/resume tasks,
* get state of connectors and tasks, etc). The non-Java interfaces to the cluster (REST API and CLI) are very simple
* wrappers of the functionality provided by this interface.
* </p>
* <p>
* In standalone mode, this implementation of this class will be trivial because no coordination is needed. In that case,
* the implementation will mainly be delegating tasks directly to other components. For example, when creating a new
* connector in standalone mode, there is no need to persist the config and the connector and its tasks must run in the
* same process, so the standalone herder implementation can immediately instantiate and start the connector and its
* tasks.
* </p>
*/
public interface Herder {
void start();
void stop();
/**
* Submit a connector job to the cluster. This works from any node by forwarding the request to
* the leader herder if necessary.
*
* @param connectorProps user-specified properties for this job
* @param callback callback to invoke when the request completes
*/
void addConnector(Properties connectorProps, Callback<String> callback);
/**
* Delete a connector job by name.
*
* @param name name of the connector job to shutdown and delete
* @param callback callback to invoke when the request completes
*/
void deleteConnector(String name, Callback<Void> callback);
}

View File

@ -0,0 +1,24 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.copycat.sink.SinkTaskContext;
class SinkTaskContextImpl extends SinkTaskContext {
}

View File

@ -0,0 +1,103 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.copycat.cli.WorkerConfig;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashMap;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
/**
* <p>
* Manages offset commit scheduling and execution for SourceTasks.
* </p>
* <p>
* Unlike sink tasks which directly manage their offset commits in the main poll() thread since
* they drive the event loop and control (for all intents and purposes) the timeouts, source
* tasks are at the whim of the connector and cannot be guaranteed to wake up on the necessary
* schedule. Instead, this class tracks all the active tasks, their schedule for commits, and
* ensures they are invoked in a timely fashion.
* </p>
* <p>
* The current implementation uses a single thread to process commits and
* </p>
*/
class SourceTaskOffsetCommitter {
private static final Logger log = LoggerFactory.getLogger(SourceTaskOffsetCommitter.class);
private Time time;
private WorkerConfig config;
private ScheduledExecutorService commitExecutorService = null;
private HashMap<ConnectorTaskId, ScheduledFuture<?>> committers = new HashMap<>();
SourceTaskOffsetCommitter(Time time, WorkerConfig config) {
this.time = time;
this.config = config;
commitExecutorService = Executors.newSingleThreadScheduledExecutor();
}
public void close(long timeoutMs) {
commitExecutorService.shutdown();
try {
if (!commitExecutorService.awaitTermination(timeoutMs, TimeUnit.MILLISECONDS)) {
log.error("Graceful shutdown of offset commitOffsets thread timed out.");
}
} catch (InterruptedException e) {
// ignore and allow to exit immediately
}
}
public void schedule(final ConnectorTaskId id, final WorkerSourceTask workerTask) {
long commitIntervalMs = config.getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG);
ScheduledFuture<?> commitFuture = commitExecutorService.scheduleWithFixedDelay(new Runnable() {
@Override
public void run() {
commit(workerTask);
}
}, commitIntervalMs, commitIntervalMs, TimeUnit.MILLISECONDS);
committers.put(id, commitFuture);
}
public void remove(ConnectorTaskId id) {
ScheduledFuture<?> commitFuture = committers.remove(id);
commitFuture.cancel(false);
}
public void commit(WorkerSourceTask workerTask) {
try {
log.debug("Committing offsets for {}", workerTask);
boolean success = workerTask.commitOffsets();
if (!success) {
log.error("Failed to commit offsets for {}", workerTask);
}
} catch (Throwable t) {
// We're very careful about exceptions here since any uncaught exceptions in the commit
// thread would cause the fixed interval schedule on the ExecutorService to stop running
// for that task
log.error("Unhandled exception when committing {}: ", workerTask, t);
}
}
}

View File

@ -0,0 +1,236 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.SystemTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.cli.WorkerConfig;
import org.apache.kafka.copycat.connector.Task;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.sink.SinkTask;
import org.apache.kafka.copycat.source.SourceTask;
import org.apache.kafka.copycat.storage.*;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
/**
* <p>
* Worker runs a (dynamic) set of tasks in a set of threads, doing the work of actually moving
* data to/from Kafka.
* </p>
* <p>
* Since each task has a dedicated thread, this is mainly just a container for them.
* </p>
*/
public class Worker<K, V> {
private static final Logger log = LoggerFactory.getLogger(Worker.class);
private Time time;
private WorkerConfig config;
private Converter<K> keyConverter;
private Converter<V> valueConverter;
private OffsetBackingStore offsetBackingStore;
private Serializer<K> offsetKeySerializer;
private Serializer<V> offsetValueSerializer;
private Deserializer<K> offsetKeyDeserializer;
private Deserializer<V> offsetValueDeserializer;
private HashMap<ConnectorTaskId, WorkerTask> tasks = new HashMap<>();
private KafkaProducer<K, V> producer;
private SourceTaskOffsetCommitter sourceTaskOffsetCommitter;
public Worker(WorkerConfig config) {
this(new SystemTime(), config, null, null, null, null, null);
}
@SuppressWarnings("unchecked")
public Worker(Time time, WorkerConfig config, OffsetBackingStore offsetBackingStore,
Serializer offsetKeySerializer, Serializer offsetValueSerializer,
Deserializer offsetKeyDeserializer, Deserializer offsetValueDeserializer) {
this.time = time;
this.config = config;
this.keyConverter = config.getConfiguredInstance(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, Converter.class);
this.valueConverter = config.getConfiguredInstance(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, Converter.class);
if (offsetBackingStore != null) {
this.offsetBackingStore = offsetBackingStore;
} else {
this.offsetBackingStore = new FileOffsetBackingStore();
this.offsetBackingStore.configure(config.originals());
}
if (offsetKeySerializer != null) {
this.offsetKeySerializer = offsetKeySerializer;
} else {
this.offsetKeySerializer = config.getConfiguredInstance(WorkerConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class);
this.offsetKeySerializer.configure(config.originals(), true);
}
if (offsetValueSerializer != null) {
this.offsetValueSerializer = offsetValueSerializer;
} else {
this.offsetValueSerializer = config.getConfiguredInstance(WorkerConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class);
this.offsetValueSerializer.configure(config.originals(), false);
}
if (offsetKeyDeserializer != null) {
this.offsetKeyDeserializer = offsetKeyDeserializer;
} else {
this.offsetKeyDeserializer = config.getConfiguredInstance(WorkerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class);
this.offsetKeyDeserializer.configure(config.originals(), true);
}
if (offsetValueDeserializer != null) {
this.offsetValueDeserializer = offsetValueDeserializer;
} else {
this.offsetValueDeserializer = config.getConfiguredInstance(WorkerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class);
this.offsetValueDeserializer.configure(config.originals(), false);
}
}
public void start() {
log.info("Worker starting");
Properties unusedConfigs = config.getUnusedProperties();
Map<String, Object> producerProps = new HashMap<>();
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, Utils.join(config.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ","));
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, config.getClass(WorkerConfig.KEY_SERIALIZER_CLASS_CONFIG).getName());
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, config.getClass(WorkerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName());
for (String propName : unusedConfigs.stringPropertyNames()) {
producerProps.put(propName, unusedConfigs.getProperty(propName));
}
producer = new KafkaProducer<>(producerProps);
offsetBackingStore.start();
sourceTaskOffsetCommitter = new SourceTaskOffsetCommitter(time, config);
log.info("Worker started");
}
public void stop() {
log.info("Worker stopping");
long started = time.milliseconds();
long limit = started + config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG);
for (Map.Entry<ConnectorTaskId, WorkerTask> entry : tasks.entrySet()) {
WorkerTask task = entry.getValue();
log.warn("Shutting down task {} uncleanly; herder should have shut down "
+ "tasks before the Worker is stopped.", task);
try {
task.stop();
} catch (CopycatException e) {
log.error("Error while shutting down task " + task, e);
}
}
for (Map.Entry<ConnectorTaskId, WorkerTask> entry : tasks.entrySet()) {
WorkerTask task = entry.getValue();
log.debug("Waiting for task {} to finish shutting down", task);
if (!task.awaitStop(Math.max(limit - time.milliseconds(), 0)))
log.error("Graceful shutdown of task {} failed.", task);
task.close();
}
long timeoutMs = limit - time.milliseconds();
sourceTaskOffsetCommitter.close(timeoutMs);
offsetBackingStore.start();
log.info("Worker stopped");
}
/**
* Add a new task.
* @param id Globally unique ID for this task.
* @param taskClassName name of the {@link org.apache.kafka.copycat.connector.Task}
* class to instantiate. Must be a subclass of either
* {@link org.apache.kafka.copycat.source.SourceTask} or
* {@link org.apache.kafka.copycat.sink.SinkTask}.
* @param props configuration options for the task
*/
public void addTask(ConnectorTaskId id, String taskClassName, Properties props) {
if (tasks.containsKey(id)) {
String msg = "Task already exists in this worker; the herder should not have requested "
+ "that this : " + id;
log.error(msg);
throw new CopycatException(msg);
}
final Task task = instantiateTask(taskClassName);
// Decide which type of worker task we need based on the type of task.
final WorkerTask workerTask;
if (task instanceof SourceTask) {
SourceTask sourceTask = (SourceTask) task;
OffsetStorageReader offsetReader = new OffsetStorageReaderImpl<>(offsetBackingStore, id.getConnector(),
keyConverter, valueConverter, offsetKeySerializer, offsetValueDeserializer);
OffsetStorageWriter<K, V> offsetWriter = new OffsetStorageWriter<>(offsetBackingStore, id.getConnector(),
keyConverter, valueConverter, offsetKeySerializer, offsetValueSerializer);
workerTask = new WorkerSourceTask<>(id, sourceTask, keyConverter, valueConverter, producer,
offsetReader, offsetWriter, config, time);
} else if (task instanceof SinkTask) {
workerTask = new WorkerSinkTask<>(id, (SinkTask) task, config, keyConverter, valueConverter, time);
} else {
log.error("Tasks must be a subclass of either SourceTask or SinkTask", task);
throw new CopycatException("Tasks must be a subclass of either SourceTask or SinkTask");
}
// Start the task before adding modifying any state, any exceptions are caught higher up the
// call chain and there's no cleanup to do here
workerTask.start(props);
tasks.put(id, workerTask);
}
private static Task instantiateTask(String taskClassName) {
try {
return Utils.newInstance(Class.forName(taskClassName).asSubclass(Task.class));
} catch (ClassNotFoundException e) {
throw new CopycatException("Task class not found", e);
}
}
public void stopTask(ConnectorTaskId id) {
WorkerTask task = getTask(id);
task.stop();
if (!task.awaitStop(config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG)))
log.error("Graceful stop of task {} failed.", task);
task.close();
tasks.remove(id);
}
private WorkerTask getTask(ConnectorTaskId id) {
WorkerTask task = tasks.get(id);
if (task == null) {
log.error("Task not found: " + id);
throw new CopycatException("Task not found: " + id);
}
return task;
}
}

View File

@ -0,0 +1,226 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.clients.consumer.*;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.cli.WorkerConfig;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.sink.SinkRecord;
import org.apache.kafka.copycat.sink.SinkTask;
import org.apache.kafka.copycat.sink.SinkTaskContext;
import org.apache.kafka.copycat.storage.Converter;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.*;
import java.util.concurrent.TimeUnit;
/**
* WorkerTask that uses a SinkTask to export data from Kafka.
*/
class WorkerSinkTask<K, V> implements WorkerTask {
private static final Logger log = LoggerFactory.getLogger(WorkerSinkTask.class);
private final ConnectorTaskId id;
private final SinkTask task;
private final WorkerConfig workerConfig;
private final Time time;
private final Converter<K> keyConverter;
private final Converter<V> valueConverter;
private WorkerSinkTaskThread workThread;
private KafkaConsumer<K, V> consumer;
private final SinkTaskContext context;
public WorkerSinkTask(ConnectorTaskId id, SinkTask task, WorkerConfig workerConfig,
Converter<K> keyConverter, Converter<V> valueConverter, Time time) {
this.id = id;
this.task = task;
this.workerConfig = workerConfig;
this.keyConverter = keyConverter;
this.valueConverter = valueConverter;
context = new SinkTaskContextImpl();
this.time = time;
}
@Override
public void start(Properties props) {
task.initialize(context);
task.start(props);
consumer = createConsumer(props);
workThread = createWorkerThread();
workThread.start();
}
@Override
public void stop() {
// Offset commit is handled upon exit in work thread
task.stop();
if (workThread != null)
workThread.startGracefulShutdown();
consumer.wakeup();
}
@Override
public boolean awaitStop(long timeoutMs) {
if (workThread != null) {
try {
boolean success = workThread.awaitShutdown(timeoutMs, TimeUnit.MILLISECONDS);
if (!success)
workThread.forceShutdown();
return success;
} catch (InterruptedException e) {
return false;
}
}
return true;
}
@Override
public void close() {
// FIXME Kafka needs to add a timeout parameter here for us to properly obey the timeout
// passed in
if (consumer != null)
consumer.close();
}
/** Poll for new messages with the given timeout. Should only be invoked by the worker thread. */
public void poll(long timeoutMs) {
try {
log.trace("{} polling consumer with timeout {} ms", id, timeoutMs);
ConsumerRecords<K, V> msgs = consumer.poll(timeoutMs);
log.trace("{} polling returned {} messages", id, msgs.count());
deliverMessages(msgs);
} catch (ConsumerWakeupException we) {
log.trace("{} consumer woken up", id);
}
}
/**
* Starts an offset commit by flushing outstanding messages from the task and then starting
* the write commit. This should only be invoked by the WorkerSinkTaskThread.
**/
public void commitOffsets(long now, boolean sync, final int seqno, boolean flush) {
HashMap<TopicPartition, Long> offsets = new HashMap<>();
for (TopicPartition tp : consumer.subscriptions()) {
offsets.put(tp, consumer.position(tp));
}
// We only don't flush the task in one case: when shutting down, the task has already been
// stopped and all data should have already been flushed
if (flush) {
try {
task.flush(offsets);
} catch (Throwable t) {
log.error("Commit of {} offsets failed due to exception while flushing: {}", this, t);
workThread.onCommitCompleted(t, seqno);
return;
}
}
ConsumerCommitCallback cb = new ConsumerCommitCallback() {
@Override
public void onComplete(Map<TopicPartition, Long> offsets, Exception error) {
workThread.onCommitCompleted(error, seqno);
}
};
consumer.commit(offsets, sync ? CommitType.SYNC : CommitType.ASYNC, cb);
}
public Time getTime() {
return time;
}
public WorkerConfig getWorkerConfig() {
return workerConfig;
}
private KafkaConsumer<K, V> createConsumer(Properties taskProps) {
String topicsStr = taskProps.getProperty(SinkTask.TOPICS_CONFIG);
if (topicsStr == null || topicsStr.isEmpty())
throw new CopycatException("Sink tasks require a list of topics.");
String[] topics = topicsStr.split(",");
// Include any unknown worker configs so consumer configs can be set globally on the worker
// and through to the task
Properties props = workerConfig.getUnusedProperties();
props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "copycat-" + id.toString());
props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG,
Utils.join(workerConfig.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ","));
props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
workerConfig.getClass(WorkerConfig.KEY_DESERIALIZER_CLASS_CONFIG).getName());
props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
workerConfig.getClass(WorkerConfig.VALUE_DESERIALIZER_CLASS_CONFIG).getName());
KafkaConsumer<K, V> newConsumer;
try {
newConsumer = new KafkaConsumer<>(props);
} catch (Throwable t) {
throw new CopycatException("Failed to create consumer", t);
}
log.debug("Task {} subscribing to topics {}", id, topics);
newConsumer.subscribe(topics);
// Seek to any user-provided offsets. This is useful if offsets are tracked in the downstream system (e.g., to
// enable exactly once delivery to that system).
//
// To do this correctly, we need to first make sure we have been assigned partitions, which poll() will guarantee.
// We ask for offsets after this poll to make sure any offsets committed before the rebalance are picked up correctly.
newConsumer.poll(0);
Map<TopicPartition, Long> offsets = context.getOffsets();
for (TopicPartition tp : newConsumer.subscriptions()) {
Long offset = offsets.get(tp);
if (offset != null)
newConsumer.seek(tp, offset);
}
return newConsumer;
}
private WorkerSinkTaskThread createWorkerThread() {
return new WorkerSinkTaskThread(this, "WorkerSinkTask-" + id, time, workerConfig);
}
private void deliverMessages(ConsumerRecords<K, V> msgs) {
// Finally, deliver this batch to the sink
if (msgs.count() > 0) {
List<SinkRecord> records = new ArrayList<>();
for (ConsumerRecord<K, V> msg : msgs) {
log.trace("Consuming message with key {}, value {}", msg.key(), msg.value());
records.add(
new SinkRecord(msg.topic(), msg.partition(),
keyConverter.toCopycatData(msg.key()),
valueConverter.toCopycatData(msg.value()),
msg.offset())
);
}
try {
task.put(records);
} catch (CopycatException e) {
log.error("Exception from SinkTask {}: ", id, e);
} catch (Throwable t) {
log.error("Unexpected exception from SinkTask {}: ", id, t);
}
}
}
}

View File

@ -0,0 +1,112 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.copycat.cli.WorkerConfig;
import org.apache.kafka.copycat.util.ShutdownableThread;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Worker thread for a WorkerSinkTask. These classes are very tightly coupled, but separated to
* simplify testing.
*/
class WorkerSinkTaskThread extends ShutdownableThread {
private static final Logger log = LoggerFactory.getLogger(WorkerSinkTask.class);
private final WorkerSinkTask task;
private long nextCommit;
private boolean committing;
private int commitSeqno;
private long commitStarted;
private int commitFailures;
public WorkerSinkTaskThread(WorkerSinkTask task, String name, Time time,
WorkerConfig workerConfig) {
super(name);
this.task = task;
this.nextCommit = time.milliseconds() +
workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG);
this.committing = false;
this.commitSeqno = 0;
this.commitStarted = -1;
this.commitFailures = 0;
}
@Override
public void execute() {
while (getRunning()) {
iteration();
}
// Make sure any uncommitted data has committed
task.commitOffsets(task.getTime().milliseconds(), true, -1, false);
}
public void iteration() {
long now = task.getTime().milliseconds();
// Maybe commit
if (!committing && now >= nextCommit) {
synchronized (this) {
committing = true;
commitSeqno += 1;
commitStarted = now;
}
task.commitOffsets(now, false, commitSeqno, true);
nextCommit += task.getWorkerConfig().getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG);
}
// Check for timed out commits
long commitTimeout = commitStarted + task.getWorkerConfig().getLong(
WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG);
if (committing && now >= commitTimeout) {
log.warn("Commit of {} offsets timed out", this);
commitFailures++;
committing = false;
}
// And process messages
long timeoutMs = Math.max(nextCommit - now, 0);
task.poll(timeoutMs);
}
public void onCommitCompleted(Throwable error, long seqno) {
synchronized (this) {
if (commitSeqno != seqno) {
log.debug("Got callback for timed out commit {}: {}, but most recent commit is {}",
this,
seqno, commitSeqno);
} else {
if (error != null) {
log.error("Commit of {} offsets threw an unexpected exception: ", this, error);
commitFailures++;
} else {
log.debug("Finished {} offset commit successfully in {} ms",
this, task.getTime().milliseconds() - commitStarted);
commitFailures = 0;
}
committing = false;
}
}
}
public int getCommitFailures() {
return commitFailures;
}
}

View File

@ -0,0 +1,310 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.copycat.cli.WorkerConfig;
import org.apache.kafka.copycat.source.SourceRecord;
import org.apache.kafka.copycat.source.SourceTask;
import org.apache.kafka.copycat.source.SourceTaskContext;
import org.apache.kafka.copycat.storage.Converter;
import org.apache.kafka.copycat.storage.OffsetStorageReader;
import org.apache.kafka.copycat.storage.OffsetStorageWriter;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.apache.kafka.copycat.util.ShutdownableThread;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
/**
* WorkerTask that uses a SourceTask to ingest data into Kafka.
*/
class WorkerSourceTask<K, V> implements WorkerTask {
private static final Logger log = LoggerFactory.getLogger(WorkerSourceTask.class);
private ConnectorTaskId id;
private SourceTask task;
private final Converter<K> keyConverter;
private final Converter<V> valueConverter;
private KafkaProducer<K, V> producer;
private WorkerSourceTaskThread workThread;
private OffsetStorageReader offsetReader;
private OffsetStorageWriter<K, V> offsetWriter;
private final WorkerConfig workerConfig;
private final Time time;
// Use IdentityHashMap to ensure correctness with duplicate records. This is a HashMap because
// there is no IdentityHashSet.
private IdentityHashMap<ProducerRecord<K, V>, ProducerRecord<K, V>>
outstandingMessages;
// A second buffer is used while an offset flush is running
private IdentityHashMap<ProducerRecord<K, V>, ProducerRecord<K, V>>
outstandingMessagesBacklog;
private boolean flushing;
public WorkerSourceTask(ConnectorTaskId id, SourceTask task,
Converter<K> keyConverter, Converter<V> valueConverter,
KafkaProducer<K, V> producer,
OffsetStorageReader offsetReader, OffsetStorageWriter<K, V> offsetWriter,
WorkerConfig workerConfig, Time time) {
this.id = id;
this.task = task;
this.keyConverter = keyConverter;
this.valueConverter = valueConverter;
this.producer = producer;
this.offsetReader = offsetReader;
this.offsetWriter = offsetWriter;
this.workerConfig = workerConfig;
this.time = time;
this.outstandingMessages = new IdentityHashMap<>();
this.outstandingMessagesBacklog = new IdentityHashMap<>();
this.flushing = false;
}
@Override
public void start(Properties props) {
task.initialize(new SourceTaskContext(offsetReader));
task.start(props);
workThread = new WorkerSourceTaskThread("WorkerSourceTask-" + id);
workThread.start();
}
@Override
public void stop() {
task.stop();
commitOffsets();
if (workThread != null)
workThread.startGracefulShutdown();
}
@Override
public boolean awaitStop(long timeoutMs) {
if (workThread != null) {
try {
boolean success = workThread.awaitShutdown(timeoutMs, TimeUnit.MILLISECONDS);
if (!success)
workThread.forceShutdown();
return success;
} catch (InterruptedException e) {
return false;
}
}
return true;
}
@Override
public void close() {
// Nothing to do
}
/**
* Send a batch of records. This is atomic up to the point of getting the messages into the
* Producer and recorded in our set of outstanding messages, so either all or none will be sent
* @param records
*/
private synchronized void sendRecords(List<SourceRecord> records) {
for (SourceRecord record : records) {
final ProducerRecord<K, V> producerRecord
= new ProducerRecord<>(record.getTopic(), record.getKafkaPartition(),
keyConverter.fromCopycatData(record.getKey()),
valueConverter.fromCopycatData(record.getValue()));
log.trace("Appending record with key {}, value {}", record.getKey(), record.getValue());
if (!flushing) {
outstandingMessages.put(producerRecord, producerRecord);
} else {
outstandingMessagesBacklog.put(producerRecord, producerRecord);
}
producer.send(
producerRecord,
new Callback() {
@Override
public void onCompletion(RecordMetadata recordMetadata, Exception e) {
if (e != null) {
log.error("Failed to send record: ", e);
} else {
log.trace("Wrote record successfully: topic {} partition {} offset {}",
recordMetadata.topic(), recordMetadata.partition(),
recordMetadata.offset());
}
recordSent(producerRecord);
}
});
// Offsets are converted & serialized in the OffsetWriter
offsetWriter.setOffset(record.getSourcePartition(), record.getSourceOffset());
}
}
private synchronized void recordSent(final ProducerRecord<K, V> record) {
ProducerRecord<K, V> removed = outstandingMessages.remove(record);
// While flushing, we may also see callbacks for items in the backlog
if (removed == null && flushing)
removed = outstandingMessagesBacklog.remove(record);
// But if neither one had it, something is very wrong
if (removed == null) {
log.error("Saw callback for record that was not present in the outstanding message set: "
+ "{}", record);
} else if (flushing && outstandingMessages.isEmpty()) {
// flush thread may be waiting on the outstanding messages to clear
this.notifyAll();
}
}
public boolean commitOffsets() {
long commitTimeoutMs = workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG);
long started = time.milliseconds();
long timeout = started + commitTimeoutMs;
synchronized (this) {
// First we need to make sure we snapshot everything in exactly the current state. This
// means both the current set of messages we're still waiting to finish, stored in this
// class, which setting flushing = true will handle by storing any new values into a new
// buffer; and the current set of user-specified offsets, stored in the
// OffsetStorageWriter, for which we can use beginFlush() to initiate the snapshot.
flushing = true;
boolean flushStarted = offsetWriter.beginFlush();
// Still wait for any producer records to flush, even if there aren't any offsets to write
// to persistent storage
// Next we need to wait for all outstanding messages to finish sending
while (!outstandingMessages.isEmpty()) {
try {
long timeoutMs = timeout - time.milliseconds();
if (timeoutMs <= 0) {
log.error(
"Failed to flush {}, timed out while waiting for producer to flush outstanding "
+ "messages", this.toString());
finishFailedFlush();
return false;
}
this.wait(timeoutMs);
} catch (InterruptedException e) {
// ignore
}
}
if (!flushStarted) {
// There was nothing in the offsets to process, but we still waited for the data in the
// buffer to flush. This is useful since this can feed into metrics to monitor, e.g.
// flush time, which can be used for monitoring even if the connector doesn't record any
// offsets.
finishSuccessfulFlush();
log.debug("Finished {} offset commitOffsets successfully in {} ms",
this, time.milliseconds() - started);
return true;
}
}
// Now we can actually flush the offsets to user storage.
Future<Void> flushFuture = offsetWriter.doFlush(new org.apache.kafka.copycat.util.Callback<Void>() {
@Override
public void onCompletion(Throwable error, Void result) {
if (error != null) {
log.error("Failed to flush {} offsets to storage: ", this, error);
} else {
log.trace("Finished flushing {} offsets to storage", this);
}
}
});
// Very rare case: offsets were unserializable and we finished immediately, unable to store
// any data
if (flushFuture == null) {
finishFailedFlush();
return false;
}
try {
flushFuture.get(Math.max(timeout - time.milliseconds(), 0), TimeUnit.MILLISECONDS);
// There's a small race here where we can get the callback just as this times out (and log
// success), but then catch the exception below and cancel everything. This won't cause any
// errors, is only wasteful in this minor edge case, and the worst result is that the log
// could look a little confusing.
} catch (InterruptedException e) {
log.warn("Flush of {} offsets interrupted, cancelling", this);
finishFailedFlush();
return false;
} catch (ExecutionException e) {
log.error("Flush of {} offsets threw an unexpected exception: ", this, e);
finishFailedFlush();
return false;
} catch (TimeoutException e) {
log.error("Timed out waiting to flush {} offsets to storage", this);
finishFailedFlush();
return false;
}
finishSuccessfulFlush();
log.debug("Finished {} commitOffsets successfully in {} ms",
this, time.milliseconds() - started);
return true;
}
private synchronized void finishFailedFlush() {
offsetWriter.cancelFlush();
outstandingMessages.putAll(outstandingMessagesBacklog);
outstandingMessagesBacklog.clear();
flushing = false;
}
private void finishSuccessfulFlush() {
// If we were successful, we can just swap instead of replacing items back into the original map
IdentityHashMap<ProducerRecord<K, V>, ProducerRecord<K, V>> temp = outstandingMessages;
outstandingMessages = outstandingMessagesBacklog;
outstandingMessagesBacklog = temp;
flushing = false;
}
private class WorkerSourceTaskThread extends ShutdownableThread {
public WorkerSourceTaskThread(String name) {
super(name);
}
@Override
public void execute() {
try {
while (getRunning()) {
List<SourceRecord> records = task.poll();
if (records == null)
continue;
sendRecords(records);
}
} catch (InterruptedException e) {
// Ignore and allow to exit.
}
}
}
@Override
public String toString() {
return "WorkerSourceTask{" +
"id=" + id +
'}';
}
}

View File

@ -0,0 +1,54 @@
/**
* 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.copycat.runtime;
import java.util.Properties;
/**
* Handles processing for an individual task. This interface only provides the basic methods
* used by {@link Worker} to manage the tasks. Implementations combine a user-specified Task with
* Kafka to create a data flow.
*/
interface WorkerTask {
/**
* Start the Task
* @param props initial configuration
*/
void start(Properties props);
/**
* Stop this task from processing messages. This method does not block, it only triggers
* shutdown. Use #{@link #awaitStop} to block until completion.
*/
void stop();
/**
* Wait for this task to finish stopping.
*
* @param timeoutMs
* @return true if successful, false if the timeout was reached
*/
boolean awaitStop(long timeoutMs);
/**
* Close this task. This is different from #{@link #stop} and #{@link #awaitStop} in that the
* stop methods ensure processing has stopped but may leave resources allocated. This method
* should clean up all resources.
*/
void close();
}

View File

@ -0,0 +1,42 @@
/**
* 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.copycat.runtime.standalone;
import org.apache.kafka.copycat.connector.ConnectorContext;
/**
* ConnectorContext for use with the StandaloneHerder, which maintains all connectors and tasks
* in a single process.
*/
class StandaloneConnectorContext implements ConnectorContext {
private StandaloneHerder herder;
private String connectorName;
public StandaloneConnectorContext(StandaloneHerder herder, String connectorName) {
this.herder = herder;
this.connectorName = connectorName;
}
@Override
public void requestTaskReconfiguration() {
// This is trivial to forward since there is only one herder and it's in memory in this
// process
herder.requestTaskReconfiguration(connectorName);
}
}

View File

@ -0,0 +1,257 @@
/**
* 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.copycat.runtime.standalone;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.connector.Connector;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.runtime.ConnectorConfig;
import org.apache.kafka.copycat.runtime.Herder;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.sink.SinkConnector;
import org.apache.kafka.copycat.sink.SinkTask;
import org.apache.kafka.copycat.util.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.*;
/**
* Single process, in-memory "herder". Useful for a standalone copycat process.
*/
public class StandaloneHerder implements Herder {
private static final Logger log = LoggerFactory.getLogger(StandaloneHerder.class);
private Worker worker;
private HashMap<String, ConnectorState> connectors = new HashMap<>();
public StandaloneHerder(Worker worker) {
this.worker = worker;
}
public synchronized void start() {
log.info("Herder starting");
log.info("Herder started");
}
public synchronized void stop() {
log.info("Herder stopping");
// There's no coordination/hand-off to do here since this is all standalone. Instead, we
// should just clean up the stuff we normally would, i.e. cleanly checkpoint and shutdown all
// the tasks.
for (Map.Entry<String, ConnectorState> entry : connectors.entrySet()) {
ConnectorState state = entry.getValue();
stopConnector(state);
}
connectors.clear();
log.info("Herder stopped");
}
@Override
public synchronized void addConnector(Properties connectorProps,
Callback<String> callback) {
try {
ConnectorState connState = createConnector(connectorProps);
if (callback != null)
callback.onCompletion(null, connState.name);
// This should always be a new job, create jobs from scratch
createConnectorTasks(connState);
} catch (CopycatException e) {
if (callback != null)
callback.onCompletion(e, null);
}
}
@Override
public synchronized void deleteConnector(String name, Callback<Void> callback) {
try {
destroyConnector(name);
if (callback != null)
callback.onCompletion(null, null);
} catch (CopycatException e) {
if (callback != null)
callback.onCompletion(e, null);
}
}
// Creates the and configures the connector. Does not setup any tasks
private ConnectorState createConnector(Properties connectorProps) {
ConnectorConfig connConfig = new ConnectorConfig(connectorProps);
String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG);
String className = connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG);
log.info("Creating connector {} of type {}", connName, className);
int maxTasks = connConfig.getInt(ConnectorConfig.TASKS_MAX_CONFIG);
List<String> topics = connConfig.getList(ConnectorConfig.TOPICS_CONFIG); // Sinks only
Properties configs = connConfig.getUnusedProperties();
if (connectors.containsKey(connName)) {
log.error("Ignoring request to create connector due to conflicting connector name");
throw new CopycatException("Connector with name " + connName + " already exists");
}
final Connector connector;
try {
connector = instantiateConnector(className);
} catch (Throwable t) {
// Catches normal exceptions due to instantiation errors as well as any runtime errors that
// may be caused by user code
throw new CopycatException("Failed to create connector instance", t);
}
connector.initialize(new StandaloneConnectorContext(this, connName));
try {
connector.start(configs);
} catch (CopycatException e) {
throw new CopycatException("Connector threw an exception while starting", e);
}
ConnectorState state = new ConnectorState(connName, connector, maxTasks, topics);
connectors.put(connName, state);
log.info("Finished creating connector {}", connName);
return state;
}
private static Connector instantiateConnector(String className) {
try {
return Utils.newInstance(className, Connector.class);
} catch (ClassNotFoundException e) {
throw new CopycatException("Couldn't instantiate connector class", e);
}
}
private void destroyConnector(String connName) {
log.info("Destroying connector {}", connName);
ConnectorState state = connectors.get(connName);
if (state == null) {
log.error("Failed to destroy connector {} because it does not exist", connName);
throw new CopycatException("Connector does not exist");
}
stopConnector(state);
connectors.remove(state.name);
log.info("Finished destroying connector {}", connName);
}
// Stops a connectors tasks, then the connector
private void stopConnector(ConnectorState state) {
removeConnectorTasks(state);
try {
state.connector.stop();
} catch (CopycatException e) {
log.error("Error shutting down connector {}: ", state.connector, e);
}
}
private void createConnectorTasks(ConnectorState state) {
String taskClassName = state.connector.getTaskClass().getName();
log.info("Creating tasks for connector {} of type {}", state.name, taskClassName);
List<Properties> taskConfigs = state.connector.getTaskConfigs(state.maxTasks);
// Generate the final configs, including framework provided settings
Map<ConnectorTaskId, Properties> taskProps = new HashMap<>();
for (int i = 0; i < taskConfigs.size(); i++) {
ConnectorTaskId taskId = new ConnectorTaskId(state.name, i);
Properties config = taskConfigs.get(i);
// TODO: This probably shouldn't be in the Herder. It's nice to have Copycat ensure the list of topics
// is automatically provided to tasks since it is required by the framework, but this
String subscriptionTopics = Utils.join(state.inputTopics, ",");
if (state.connector instanceof SinkConnector) {
// Make sure we don't modify the original since the connector may reuse it internally
Properties configForSink = new Properties();
configForSink.putAll(config);
configForSink.setProperty(SinkTask.TOPICS_CONFIG, subscriptionTopics);
config = configForSink;
}
taskProps.put(taskId, config);
}
// And initiate the tasks
for (int i = 0; i < taskConfigs.size(); i++) {
ConnectorTaskId taskId = new ConnectorTaskId(state.name, i);
Properties config = taskProps.get(taskId);
try {
worker.addTask(taskId, taskClassName, config);
// We only need to store the task IDs so we can clean up.
state.tasks.add(taskId);
} catch (Throwable e) {
log.error("Failed to add task {}: ", taskId, e);
// Swallow this so we can continue updating the rest of the tasks
// FIXME what's the proper response? Kill all the tasks? Consider this the same as a task
// that died after starting successfully.
}
}
}
private void removeConnectorTasks(ConnectorState state) {
Iterator<ConnectorTaskId> taskIter = state.tasks.iterator();
while (taskIter.hasNext()) {
ConnectorTaskId taskId = taskIter.next();
try {
worker.stopTask(taskId);
taskIter.remove();
} catch (CopycatException e) {
log.error("Failed to stop task {}: ", taskId, e);
// Swallow this so we can continue stopping the rest of the tasks
// FIXME: Forcibly kill the task?
}
}
}
private void updateConnectorTasks(ConnectorState state) {
removeConnectorTasks(state);
createConnectorTasks(state);
}
/**
* Requests reconfiguration of the task. This should only be triggered by
* {@link StandaloneConnectorContext}.
*
* @param connName name of the connector that should be reconfigured
*/
public synchronized void requestTaskReconfiguration(String connName) {
ConnectorState state = connectors.get(connName);
if (state == null) {
log.error("Task that requested reconfiguration does not exist: {}", connName);
return;
}
updateConnectorTasks(state);
}
private static class ConnectorState {
public String name;
public Connector connector;
public int maxTasks;
public List<String> inputTopics;
Set<ConnectorTaskId> tasks;
public ConnectorState(String name, Connector connector, int maxTasks,
List<String> inputTopics) {
this.name = name;
this.connector = connector;
this.maxTasks = maxTasks;
this.inputTopics = inputTopics;
this.tasks = new HashSet<>();
}
}
}

View File

@ -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.copycat.storage;
import org.apache.kafka.copycat.errors.CopycatException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.*;
import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.Map;
/**
* Implementation of OffsetBackingStore that saves data locally to a file. To ensure this behaves
* similarly to a real backing store, operations are executed asynchronously on a background thread.
*/
public class FileOffsetBackingStore extends MemoryOffsetBackingStore {
private static final Logger log = LoggerFactory.getLogger(FileOffsetBackingStore.class);
public final static String OFFSET_STORAGE_FILE_FILENAME_CONFIG = "offset.storage.file.filename";
private File file;
public FileOffsetBackingStore() {
}
@Override
public void configure(Map<String, ?> props) {
super.configure(props);
String filename = (String) props.get(OFFSET_STORAGE_FILE_FILENAME_CONFIG);
file = new File(filename);
}
@Override
public synchronized void start() {
super.start();
log.info("Starting FileOffsetBackingStore with file {}", file);
load();
}
@Override
public synchronized void stop() {
super.stop();
// Nothing to do since this doesn't maintain any outstanding connections/data
log.info("Stopped FileOffsetBackingStore");
}
@SuppressWarnings("unchecked")
private void load() {
try {
ObjectInputStream is = new ObjectInputStream(new FileInputStream(file));
Object obj = is.readObject();
if (!(obj instanceof HashMap))
throw new CopycatException("Expected HashMap but found " + obj.getClass());
HashMap<String, Map<byte[], byte[]>> raw = (HashMap<String, Map<byte[], byte[]>>) obj;
data = new HashMap<>();
for (Map.Entry<String, Map<byte[], byte[]>> entry : raw.entrySet()) {
HashMap<ByteBuffer, ByteBuffer> converted = new HashMap<>();
for (Map.Entry<byte[], byte[]> mapEntry : entry.getValue().entrySet()) {
ByteBuffer key = (mapEntry.getKey() != null) ? ByteBuffer.wrap(mapEntry.getKey()) : null;
ByteBuffer value = (mapEntry.getValue() != null) ? ByteBuffer.wrap(mapEntry.getValue()) :
null;
converted.put(key, value);
}
data.put(entry.getKey(), converted);
}
is.close();
} catch (FileNotFoundException | EOFException e) {
// FileNotFoundException: Ignore, may be new.
// EOFException: Ignore, this means the file was missing or corrupt
} catch (IOException | ClassNotFoundException e) {
throw new CopycatException(e);
}
}
protected void save() {
try {
ObjectOutputStream os = new ObjectOutputStream(new FileOutputStream(file));
HashMap<String, Map<byte[], byte[]>> raw = new HashMap<>();
for (Map.Entry<String, Map<ByteBuffer, ByteBuffer>> entry : data.entrySet()) {
HashMap<byte[], byte[]> converted = new HashMap<>();
for (Map.Entry<ByteBuffer, ByteBuffer> mapEntry : entry.getValue().entrySet()) {
byte[] key = (mapEntry.getKey() != null) ? mapEntry.getKey().array() : null;
byte[] value = (mapEntry.getValue() != null) ? mapEntry.getValue().array() : null;
converted.put(key, value);
}
raw.put(entry.getKey(), converted);
}
os.writeObject(raw);
os.close();
} catch (IOException e) {
throw new CopycatException(e);
}
}
}

View File

@ -0,0 +1,113 @@
/**
* 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.copycat.storage;
import org.apache.kafka.copycat.util.Callback;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
/**
* Implementation of OffsetBackingStore that doesn't actually persist any data. To ensure this
* behaves similarly to a real backing store, operations are executed asynchronously on a
* background thread.
*/
public class MemoryOffsetBackingStore implements OffsetBackingStore {
private static final Logger log = LoggerFactory.getLogger(MemoryOffsetBackingStore.class);
protected HashMap<String, Map<ByteBuffer, ByteBuffer>> data = new HashMap<>();
protected ExecutorService executor = Executors.newSingleThreadExecutor();
public MemoryOffsetBackingStore() {
}
@Override
public void configure(Map<String, ?> props) {
}
@Override
public synchronized void start() {
}
@Override
public synchronized void stop() {
// Nothing to do since this doesn't maintain any outstanding connections/data
}
@Override
public Future<Map<ByteBuffer, ByteBuffer>> get(
final String namespace, final Collection<ByteBuffer> keys,
final Callback<Map<ByteBuffer, ByteBuffer>> callback) {
return executor.submit(new Callable<Map<ByteBuffer, ByteBuffer>>() {
@Override
public Map<ByteBuffer, ByteBuffer> call() throws Exception {
Map<ByteBuffer, ByteBuffer> result = new HashMap<>();
synchronized (MemoryOffsetBackingStore.this) {
Map<ByteBuffer, ByteBuffer> namespaceData = data.get(namespace);
if (namespaceData == null)
return result;
for (ByteBuffer key : keys) {
result.put(key, namespaceData.get(key));
}
}
if (callback != null)
callback.onCompletion(null, result);
return result;
}
});
}
@Override
public Future<Void> set(final String namespace, final Map<ByteBuffer, ByteBuffer> values,
final Callback<Void> callback) {
return executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
synchronized (MemoryOffsetBackingStore.this) {
Map<ByteBuffer, ByteBuffer> namespaceData = data.get(namespace);
if (namespaceData == null) {
namespaceData = new HashMap<>();
data.put(namespace, namespaceData);
}
for (Map.Entry<ByteBuffer, ByteBuffer> entry : values.entrySet()) {
namespaceData.put(entry.getKey(), entry.getValue());
}
save();
}
if (callback != null)
callback.onCompletion(null, null);
return null;
}
});
}
// Hook to allow subclasses to persist data
protected void save() {
}
}

View File

@ -0,0 +1,74 @@
/**
* 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.copycat.storage;
import org.apache.kafka.common.Configurable;
import org.apache.kafka.copycat.util.Callback;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.Map;
import java.util.concurrent.Future;
/**
* <p>
* OffsetBackingStore is an interface for storage backends that store key-value data. The backing
* store doesn't need to handle serialization or deserialization. It only needs to support
* reading/writing bytes. Since it is expected these operations will require network
* operations, only bulk operations are supported.
* </p>
* <p>
* Since OffsetBackingStore is a shared resource that may be used by many OffsetStorage instances
* that are associated with individual tasks, all operations include a namespace which should be
* used to isolate different key spaces.
* </p>
*/
public interface OffsetBackingStore extends Configurable {
/**
* Start this offset store.
*/
public void start();
/**
* Stop the backing store. Implementations should attempt to shutdown gracefully, but not block
* indefinitely.
*/
public void stop();
/**
* Get the values for the specified keys
* @param namespace prefix for the keys in this request
* @param keys list of keys to look up
* @param callback callback to invoke on completion
* @return future for the resulting map from key to value
*/
public Future<Map<ByteBuffer, ByteBuffer>> get(
String namespace, Collection<ByteBuffer> keys,
Callback<Map<ByteBuffer, ByteBuffer>> callback);
/**
* Set the specified keys and values.
* @param namespace prefix for the keys in this request
* @param values map from key to value
* @param callback callback to invoke on completion
* @return void future for the operation
*/
public Future<Void> set(String namespace, Map<ByteBuffer, ByteBuffer> values,
Callback<Void> callback);
}

View File

@ -0,0 +1,114 @@
/**
* 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.copycat.storage;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.copycat.errors.CopycatException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
/**
* Implementation of OffsetStorageReader. Unlike OffsetStorageWriter which is implemented
* directly, the interface is only separate from this implementation because it needs to be
* included in the public API package.
*/
public class OffsetStorageReaderImpl<K, V> implements OffsetStorageReader {
private static final Logger log = LoggerFactory.getLogger(OffsetStorageReaderImpl.class);
private final OffsetBackingStore backingStore;
private final String namespace;
private final Converter<K> keyConverter;
private final Converter<V> valueConverter;
private final Serializer<K> keySerializer;
private final Deserializer<V> valueDeserializer;
public OffsetStorageReaderImpl(OffsetBackingStore backingStore, String namespace,
Converter<K> keyConverter, Converter<V> valueConverter,
Serializer<K> keySerializer, Deserializer<V> valueDeserializer) {
this.backingStore = backingStore;
this.namespace = namespace;
this.keyConverter = keyConverter;
this.valueConverter = valueConverter;
this.keySerializer = keySerializer;
this.valueDeserializer = valueDeserializer;
}
@Override
public Object getOffset(Object partition) {
return getOffsets(Arrays.asList(partition)).get(partition);
}
@Override
public Map<Object, Object> getOffsets(Collection<Object> partitions) {
// Serialize keys so backing store can work with them
Map<ByteBuffer, Object> serializedToOriginal = new HashMap<>(partitions.size());
for (Object key : partitions) {
try {
byte[] keySerialized = keySerializer.serialize(namespace, keyConverter.fromCopycatData(key));
ByteBuffer keyBuffer = (keySerialized != null) ? ByteBuffer.wrap(keySerialized) : null;
serializedToOriginal.put(keyBuffer, key);
} catch (Throwable t) {
log.error("CRITICAL: Failed to serialize partition key when getting offsets for task with "
+ "namespace {}. No value for this data will be returned, which may break the "
+ "task or cause it to skip some data.", namespace, t);
}
}
// Get serialized key -> serialized value from backing store
Map<ByteBuffer, ByteBuffer> raw;
try {
raw = backingStore.get(namespace, serializedToOriginal.keySet(), null).get();
} catch (Exception e) {
log.error("Failed to fetch offsets from namespace {}: ", namespace, e);
throw new CopycatException("Failed to fetch offsets.", e);
}
// Deserialize all the values and map back to the original keys
Map<Object, Object> result = new HashMap<>(partitions.size());
for (Map.Entry<ByteBuffer, ByteBuffer> rawEntry : raw.entrySet()) {
try {
// Since null could be a valid key, explicitly check whether map contains the key
if (!serializedToOriginal.containsKey(rawEntry.getKey())) {
log.error("Should be able to map {} back to a requested partition-offset key, backing "
+ "store may have returned invalid data", rawEntry.getKey());
continue;
}
Object origKey = serializedToOriginal.get(rawEntry.getKey());
Object deserializedValue = valueConverter.toCopycatData(
valueDeserializer.deserialize(namespace, rawEntry.getValue().array())
);
result.put(origKey, deserializedValue);
} catch (Throwable t) {
log.error("CRITICAL: Failed to deserialize offset data when getting offsets for task with"
+ " namespace {}. No value for this data will be returned, which may break the "
+ "task or cause it to skip some data. This could either be due to an error in "
+ "the connector implementation or incompatible schema.", namespace, t);
}
}
return result;
}
}

View File

@ -0,0 +1,208 @@
/**
* 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.copycat.storage;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.util.Callback;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.Future;
/**
* <p>
* OffsetStorageWriter is a buffered writer that wraps the simple OffsetBackingStore interface.
* It maintains a copy of the key-value data in memory and buffers writes. It allows you to take
* a snapshot, which can then be asynchronously flushed to the backing store while new writes
* continue to be processed. This allows Copycat to process offset commits in the background
* while continuing to process messages.
* </p>
* <p>
* Copycat uses an OffsetStorage implementation to save state about the current progress of
* source (import to Kafka) jobs, which may have many input partitions and "offsets" may not be as
* simple as they are for Kafka partitions or files. Offset storage is not required for sink jobs
* because they can use Kafka's native offset storage (or the sink data store can handle offset
* storage to achieve exactly once semantics).
* </p>
* <p>
* Both partitions and offsets are generic data objects. This allows different connectors to use
* whatever representation they need, even arbitrarily complex records. These are translated
* internally into the serialized form the OffsetBackingStore uses.
* </p>
* <p>
* Note that this only provides write functionality. This is intentional to ensure stale data is
* never read. Offset data should only be read during startup or reconfiguration of a task. By
* always serving those requests by reading the values from the backing store, we ensure we never
* accidentally use stale data. (One example of how this can occur: a task is processing input
* partition A, writing offsets; reconfiguration causes partition A to be reassigned elsewhere;
* reconfiguration causes partition A to be reassigned to this node, but now the offset data is out
* of date). Since these offsets are created and managed by the connector itself, there's no way
* for the offset management layer to know which keys are "owned" by which tasks at any given
* time.
* </p>
* <p>
* This class is not thread-safe. It should only be accessed from a Task's processing thread.
* </p>
*/
public class OffsetStorageWriter<K, V> {
private static final Logger log = LoggerFactory.getLogger(OffsetStorageWriter.class);
private final OffsetBackingStore backingStore;
private final Converter<K> keyConverter;
private final Converter<V> valueConverter;
private final Serializer<K> keySerializer;
private final Serializer<V> valueSerializer;
private final String namespace;
// Offset data in Copycat format
private Map<Object, Object> data = new HashMap<>();
// Not synchronized, should only be accessed by flush thread
private Map<Object, Object> toFlush = null;
// Unique ID for each flush request to handle callbacks after timeouts
private long currentFlushId = 0;
public OffsetStorageWriter(OffsetBackingStore backingStore,
String namespace, Converter<K> keyConverter, Converter<V> valueConverter,
Serializer<K> keySerializer, Serializer<V> valueSerializer) {
this.backingStore = backingStore;
this.namespace = namespace;
this.keyConverter = keyConverter;
this.valueConverter = valueConverter;
this.keySerializer = keySerializer;
this.valueSerializer = valueSerializer;
}
/**
* Set an offset for a partition using Copycat data values
* @param partition the partition to store an offset for
* @param offset the offset
*/
public synchronized void setOffset(Object partition, Object offset) {
data.put(partition, offset);
}
private boolean flushing() {
return toFlush != null;
}
/**
* Performs the first step of a flush operation, snapshotting the current state. This does not
* actually initiate the flush with the underlying storage.
*
* @return true if a flush was initiated, false if no data was available
*/
public synchronized boolean beginFlush() {
if (flushing()) {
log.error("Invalid call to OffsetStorageWriter flush() while already flushing, the "
+ "framework should not allow this");
throw new CopycatException("OffsetStorageWriter is already flushing");
}
if (data.isEmpty())
return false;
assert !flushing();
toFlush = data;
data = new HashMap<>();
return true;
}
/**
* Flush the current offsets and clear them from this writer. This is non-blocking: it
* moves the current set of offsets out of the way, serializes the data, and asynchronously
* writes the data to the backing store. If no offsets need to be written, the callback is
* still invoked, but no Future is returned.
*
* @return a Future, or null if there are no offsets to commitOffsets
*/
public Future<Void> doFlush(final Callback<Void> callback) {
final long flushId = currentFlushId;
// Serialize
Map<ByteBuffer, ByteBuffer> offsetsSerialized;
try {
offsetsSerialized = new HashMap<>();
for (Map.Entry<Object, Object> entry : toFlush.entrySet()) {
byte[] key = keySerializer.serialize(namespace, keyConverter.fromCopycatData(entry.getKey()));
ByteBuffer keyBuffer = (key != null) ? ByteBuffer.wrap(key) : null;
byte[] value = valueSerializer.serialize(namespace, valueConverter.fromCopycatData(entry.getValue()));
ByteBuffer valueBuffer = (value != null) ? ByteBuffer.wrap(value) : null;
offsetsSerialized.put(keyBuffer, valueBuffer);
}
} catch (Throwable t) {
// Must handle errors properly here or the writer will be left mid-flush forever and be
// unable to make progress.
log.error("CRITICAL: Failed to serialize offset data, making it impossible to commit "
+ "offsets under namespace {}. This likely won't recover unless the "
+ "unserializable partition or offset information is overwritten.", namespace);
callback.onCompletion(t, null);
return null;
}
// And submit the data
log.debug("Submitting {} entries to backing store", offsetsSerialized.size());
return backingStore.set(namespace, offsetsSerialized, new Callback<Void>() {
@Override
public void onCompletion(Throwable error, Void result) {
boolean isCurrent = handleFinishWrite(flushId, error, result);
if (isCurrent && callback != null)
callback.onCompletion(error, result);
}
});
}
/**
* Cancel a flush that has been initiated by {@link #beginFlush}. This should not be called if
* {@link #doFlush} has already been invoked. It should be used if an operation performed
* between beginFlush and doFlush failed.
*/
public synchronized void cancelFlush() {
// Verify we're still flushing data to handle a race between cancelFlush() calls from up the
// call stack and callbacks from the write request to underlying storage
if (flushing()) {
// Just recombine the data and place it back in the primary storage
toFlush.putAll(data);
data = toFlush;
currentFlushId++;
toFlush = null;
}
}
/**
* Handle completion of a write. Returns true if this callback is for the current flush
* operation, false if it's for an old operation that should now be ignored.
*/
private synchronized boolean handleFinishWrite(long flushId, Throwable error, Void result) {
// Callbacks need to be handled carefully since the flush operation may have already timed
// out and been cancelled.
if (flushId != currentFlushId)
return false;
if (error != null) {
cancelFlush();
} else {
currentFlushId++;
toFlush = null;
}
return true;
}
}

View File

@ -0,0 +1,31 @@
/**
* 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.copycat.util;
/**
* Generic interface for callbacks
*/
public interface Callback<V> {
/**
* Invoked upon completion of the operation.
*
* @param error the error that caused the operation to fail, or null if no error occurred
* @param result the return value, or null if the operation failed
*/
void onCompletion(Throwable error, V result);
}

View File

@ -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.copycat.util;
import java.io.Serializable;
/**
* Unique ID for a single task. It includes a unique connector ID and a task ID that is unique within
* the connector.
*/
public class ConnectorTaskId implements Serializable {
private final String connector;
private final int task;
public ConnectorTaskId(String job, int task) {
this.connector = job;
this.task = task;
}
public String getConnector() {
return connector;
}
public int getTask() {
return task;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
ConnectorTaskId that = (ConnectorTaskId) o;
if (task != that.task)
return false;
if (connector != null ? !connector.equals(that.connector) : that.connector != null)
return false;
return true;
}
@Override
public int hashCode() {
int result = connector != null ? connector.hashCode() : 0;
result = 31 * result + task;
return result;
}
@Override
public String toString() {
return connector + '-' + task;
}
}

View File

@ -0,0 +1,76 @@
/**
* 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.copycat.util;
import java.util.concurrent.*;
public class FutureCallback<T> implements Callback<T>, Future<T> {
private Callback<T> underlying;
private CountDownLatch finishedLatch;
private T result = null;
private Throwable exception = null;
public FutureCallback(Callback<T> underlying) {
this.underlying = underlying;
this.finishedLatch = new CountDownLatch(1);
}
@Override
public void onCompletion(Throwable error, T result) {
underlying.onCompletion(error, result);
this.exception = error;
this.result = result;
finishedLatch.countDown();
}
@Override
public boolean cancel(boolean b) {
return false;
}
@Override
public boolean isCancelled() {
return false;
}
@Override
public boolean isDone() {
return finishedLatch.getCount() == 0;
}
@Override
public T get() throws InterruptedException, ExecutionException {
finishedLatch.await();
return getResult();
}
@Override
public T get(long l, TimeUnit timeUnit)
throws InterruptedException, ExecutionException, TimeoutException {
finishedLatch.await(l, timeUnit);
return getResult();
}
private T getResult() throws ExecutionException {
if (exception != null) {
throw new ExecutionException(exception);
}
return result;
}
}

View File

@ -0,0 +1,145 @@
/**
* 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.copycat.util;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* <p>
* Thread class with support for triggering graceful and forcible shutdown. In graceful shutdown,
* a flag is set, which the thread should detect and try to exit gracefully from. In forcible
* shutdown, the thread is interrupted. These can be combined to give a thread a chance to exit
* gracefully, but then force it to exit if it takes too long.
* </p>
* <p>
* Implementations should override the {@link #execute} method and check {@link #getRunning} to
* determine whether they should try to gracefully exit.
* </p>
*/
public abstract class ShutdownableThread extends Thread {
private static final Logger log = LoggerFactory.getLogger(ShutdownableThread.class);
private AtomicBoolean isRunning = new AtomicBoolean(true);
private CountDownLatch shutdownLatch = new CountDownLatch(1);
/**
* An UncaughtExceptionHandler to register on every instance of this class. This is useful for
* testing, where AssertionExceptions in the thread may not cause the test to fail. Since one
* instance is used for all threads, it must be thread-safe.
*/
volatile public static UncaughtExceptionHandler funcaughtExceptionHandler = null;
public ShutdownableThread(String name) {
// The default is daemon=true so that these threads will not prevent shutdown. We use this
// default because threads that are running user code that may not clean up properly, even
// when we attempt to forcibly shut them down.
this(name, true);
}
public ShutdownableThread(String name, boolean daemon) {
super(name);
this.setDaemon(daemon);
if (funcaughtExceptionHandler != null)
this.setUncaughtExceptionHandler(funcaughtExceptionHandler);
}
/**
* Implementations should override this method with the main body for the thread.
*/
public abstract void execute();
/**
* Returns true if the thread hasn't exited yet and none of the shutdown methods have been
* invoked
*/
public boolean getRunning() {
return isRunning.get();
}
@Override
public void run() {
try {
execute();
} catch (Error | RuntimeException e) {
log.error("Thread {} exiting with uncaught exception: ", getName(), e);
throw e;
} finally {
shutdownLatch.countDown();
}
}
/**
* Shutdown the thread, first trying to shut down gracefully using the specified timeout, then
* forcibly interrupting the thread.
* @param gracefulTimeout the maximum time to wait for a graceful exit
* @param unit the time unit of the timeout argument
*/
public void shutdown(long gracefulTimeout, TimeUnit unit)
throws InterruptedException {
boolean success = gracefulShutdown(gracefulTimeout, unit);
if (!success)
forceShutdown();
}
/**
* Attempt graceful shutdown
* @param timeout the maximum time to wait
* @param unit the time unit of the timeout argument
* @return true if successful, false if the timeout elapsed
*/
public boolean gracefulShutdown(long timeout, TimeUnit unit) throws InterruptedException {
startGracefulShutdown();
return awaitShutdown(timeout, unit);
}
/**
* Start shutting down this thread gracefully, but do not block waiting for it to exit.
*/
public void startGracefulShutdown() {
log.info("Starting graceful shutdown of thread {}", getName());
isRunning.set(false);
}
/**
* Awaits shutdown of this thread, waiting up to the timeout.
* @param timeout the maximum time to wait
* @param unit the time unit of the timeout argument
* @return true if successful, false if the timeout elapsed
* @throws InterruptedException
*/
public boolean awaitShutdown(long timeout, TimeUnit unit) throws InterruptedException {
return shutdownLatch.await(timeout, unit);
}
/**
* Immediately tries to force the thread to shut down by interrupting it. This does not try to
* wait for the thread to truly exit because forcible shutdown is not always possible. By
* default, threads are marked as daemon threads so they will not prevent the process from
* exiting.
*/
public void forceShutdown() throws InterruptedException {
log.info("Forcing shutdown of thread {}", getName());
isRunning.set(false);
interrupt();
}
}

View File

@ -0,0 +1,367 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.clients.consumer.*;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.copycat.cli.WorkerConfig;
import org.apache.kafka.copycat.sink.SinkRecord;
import org.apache.kafka.copycat.sink.SinkTask;
import org.apache.kafka.copycat.sink.SinkTaskContext;
import org.apache.kafka.copycat.storage.Converter;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.apache.kafka.copycat.util.MockTime;
import org.apache.kafka.copycat.util.ThreadedTest;
import org.easymock.*;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock;
import org.powermock.api.easymock.annotation.Mock;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import org.powermock.reflect.Whitebox;
import java.util.*;
import static org.junit.Assert.assertEquals;
@RunWith(PowerMockRunner.class)
@PrepareForTest(WorkerSinkTask.class)
@PowerMockIgnore("javax.management.*")
public class WorkerSinkTaskTest extends ThreadedTest {
// These are fixed to keep this code simpler. In this example we assume byte[] raw values
// with mix of integer/string in Copycat
private static final String TOPIC = "test";
private static final int PARTITION = 12;
private static final long FIRST_OFFSET = 45;
private static final int KEY = 12;
private static final String VALUE = "VALUE";
private static final byte[] RAW_KEY = "key".getBytes();
private static final byte[] RAW_VALUE = "value".getBytes();
private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, PARTITION);
private ConnectorTaskId taskId = new ConnectorTaskId("job", 0);
private Time time;
@Mock private SinkTask sinkTask;
private WorkerConfig workerConfig;
@Mock private Converter<byte[]> keyConverter;
@Mock
private Converter<byte[]> valueConverter;
private WorkerSinkTask<Integer, String> workerTask;
@Mock private KafkaConsumer<byte[], byte[]> consumer;
private WorkerSinkTaskThread workerThread;
private long recordsReturned;
@SuppressWarnings("unchecked")
@Override
public void setup() {
super.setup();
time = new MockTime();
Properties workerProps = new Properties();
workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("key.serializer", "org.apache.kafka.copycat.json.JsonSerializer");
workerProps.setProperty("value.serializer", "org.apache.kafka.copycat.json.JsonSerializer");
workerProps.setProperty("key.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer");
workerProps.setProperty("value.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer");
workerConfig = new WorkerConfig(workerProps);
workerTask = PowerMock.createPartialMock(
WorkerSinkTask.class, new String[]{"createConsumer", "createWorkerThread"},
taskId, sinkTask, workerConfig, keyConverter, valueConverter, time);
recordsReturned = 0;
}
@Test
public void testPollsInBackground() throws Exception {
Properties taskProps = new Properties();
expectInitializeTask(taskProps);
Capture<Collection<SinkRecord>> capturedRecords = expectPolls(1L);
expectStopTask(10L);
PowerMock.replayAll();
workerTask.start(taskProps);
for (int i = 0; i < 10; i++) {
workerThread.iteration();
}
workerTask.stop();
// No need for awaitStop since the thread is mocked
workerTask.close();
// Verify contents match expected values, i.e. that they were translated properly. With max
// batch size 1 and poll returns 1 message at a time, we should have a matching # of batches
assertEquals(10, capturedRecords.getValues().size());
int offset = 0;
for (Collection<SinkRecord> recs : capturedRecords.getValues()) {
assertEquals(1, recs.size());
for (SinkRecord rec : recs) {
SinkRecord referenceSinkRecord
= new SinkRecord(TOPIC, PARTITION, KEY, VALUE, FIRST_OFFSET + offset);
assertEquals(referenceSinkRecord, rec);
offset++;
}
}
PowerMock.verifyAll();
}
@Test
public void testDeliverConvertsData() throws Exception {
// Validate conversion is performed when data is delivered
Integer record = 12;
ConsumerRecords<byte[], byte[]> records = new ConsumerRecords<>(
Collections.singletonMap(
new TopicPartition("topic", 0),
Collections.singletonList(new ConsumerRecord<>("topic", 0, 0, RAW_KEY, RAW_VALUE))));
// Exact data doesn't matter, but should be passed directly to sink task
EasyMock.expect(keyConverter.toCopycatData(RAW_KEY)).andReturn(record);
EasyMock.expect(valueConverter.toCopycatData(RAW_VALUE)).andReturn(record);
Capture<Collection<SinkRecord>> capturedRecords
= EasyMock.newCapture(CaptureType.ALL);
sinkTask.put(EasyMock.capture(capturedRecords));
EasyMock.expectLastCall();
PowerMock.replayAll();
Whitebox.invokeMethod(workerTask, "deliverMessages", records);
assertEquals(record, capturedRecords.getValue().iterator().next().getKey());
assertEquals(record, capturedRecords.getValue().iterator().next().getValue());
PowerMock.verifyAll();
}
@Test
public void testCommit() throws Exception {
Properties taskProps = new Properties();
expectInitializeTask(taskProps);
// Make each poll() take the offset commit interval
Capture<Collection<SinkRecord>> capturedRecords
= expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT);
expectOffsetFlush(1L, null, null, 0, true);
expectStopTask(2);
PowerMock.replayAll();
workerTask.start(taskProps);
// First iteration gets one record
workerThread.iteration();
// Second triggers commit, gets a second offset
workerThread.iteration();
// Commit finishes synchronously for testing so we can check this immediately
assertEquals(0, workerThread.getCommitFailures());
workerTask.stop();
workerTask.close();
assertEquals(2, capturedRecords.getValues().size());
PowerMock.verifyAll();
}
@Test
public void testCommitTaskFlushFailure() throws Exception {
Properties taskProps = new Properties();
expectInitializeTask(taskProps);
Capture<Collection<SinkRecord>> capturedRecords
= expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT);
expectOffsetFlush(1L, new RuntimeException(), null, 0, true);
expectStopTask(2);
PowerMock.replayAll();
workerTask.start(taskProps);
// Second iteration triggers commit
workerThread.iteration();
workerThread.iteration();
assertEquals(1, workerThread.getCommitFailures());
assertEquals(false, Whitebox.getInternalState(workerThread, "committing"));
workerTask.stop();
workerTask.close();
PowerMock.verifyAll();
}
@Test
public void testCommitConsumerFailure() throws Exception {
Properties taskProps = new Properties();
expectInitializeTask(taskProps);
Capture<Collection<SinkRecord>> capturedRecords
= expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT);
expectOffsetFlush(1L, null, new Exception(), 0, true);
expectStopTask(2);
PowerMock.replayAll();
workerTask.start(taskProps);
// Second iteration triggers commit
workerThread.iteration();
workerThread.iteration();
// TODO Response to consistent failures?
assertEquals(1, workerThread.getCommitFailures());
assertEquals(false, Whitebox.getInternalState(workerThread, "committing"));
workerTask.stop();
workerTask.close();
PowerMock.verifyAll();
}
@Test
public void testCommitTimeout() throws Exception {
Properties taskProps = new Properties();
expectInitializeTask(taskProps);
// Cut down amount of time to pass in each poll so we trigger exactly 1 offset commit
Capture<Collection<SinkRecord>> capturedRecords
= expectPolls(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_DEFAULT / 2);
expectOffsetFlush(2L, null, null, WorkerConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, false);
expectStopTask(4);
PowerMock.replayAll();
workerTask.start(taskProps);
// Third iteration triggers commit, fourth gives a chance to trigger the timeout but doesn't
// trigger another commit
workerThread.iteration();
workerThread.iteration();
workerThread.iteration();
workerThread.iteration();
// TODO Response to consistent failures?
assertEquals(1, workerThread.getCommitFailures());
assertEquals(false, Whitebox.getInternalState(workerThread, "committing"));
workerTask.stop();
workerTask.close();
PowerMock.verifyAll();
}
private KafkaConsumer<byte[], byte[]> expectInitializeTask(Properties taskProps)
throws Exception {
sinkTask.initialize(EasyMock.anyObject(SinkTaskContext.class));
PowerMock.expectLastCall();
sinkTask.start(taskProps);
PowerMock.expectLastCall();
PowerMock.expectPrivate(workerTask, "createConsumer", taskProps)
.andReturn(consumer);
workerThread = PowerMock.createPartialMock(WorkerSinkTaskThread.class, new String[]{"start"},
workerTask, "mock-worker-thread", time,
workerConfig);
PowerMock.expectPrivate(workerTask, "createWorkerThread")
.andReturn(workerThread);
workerThread.start();
PowerMock.expectLastCall();
return consumer;
}
private void expectStopTask(final long expectedMessages) throws Exception {
final long finalOffset = FIRST_OFFSET + expectedMessages - 1;
sinkTask.stop();
PowerMock.expectLastCall();
// No offset commit since it happens in the mocked worker thread, but the main thread does need to wake up the
// consumer so it exits quickly
consumer.wakeup();
PowerMock.expectLastCall();
consumer.close();
PowerMock.expectLastCall();
}
// Note that this can only be called once per test currently
private Capture<Collection<SinkRecord>> expectPolls(final long pollDelayMs) throws Exception {
// Stub out all the consumer stream/iterator responses, which we just want to verify occur,
// but don't care about the exact details here.
EasyMock.expect(consumer.poll(EasyMock.anyLong())).andStubAnswer(
new IAnswer<ConsumerRecords<byte[], byte[]>>() {
@Override
public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
// "Sleep" so time will progress
time.sleep(pollDelayMs);
ConsumerRecords<byte[], byte[]> records = new ConsumerRecords<>(
Collections.singletonMap(
new TopicPartition(TOPIC, PARTITION),
Arrays.asList(
new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, RAW_KEY, RAW_VALUE)
)));
recordsReturned++;
return records;
}
});
EasyMock.expect(keyConverter.toCopycatData(RAW_KEY)).andReturn(KEY).anyTimes();
EasyMock.expect(valueConverter.toCopycatData(RAW_VALUE)).andReturn(VALUE).anyTimes();
Capture<Collection<SinkRecord>> capturedRecords = EasyMock.newCapture(CaptureType.ALL);
sinkTask.put(EasyMock.capture(capturedRecords));
EasyMock.expectLastCall().anyTimes();
return capturedRecords;
}
private Capture<ConsumerCommitCallback> expectOffsetFlush(final long expectedMessages,
final RuntimeException flushError,
final Exception consumerCommitError,
final long consumerCommitDelayMs,
final boolean invokeCallback)
throws Exception {
final long finalOffset = FIRST_OFFSET + expectedMessages - 1;
EasyMock.expect(consumer.subscriptions()).andReturn(Collections.singleton(TOPIC_PARTITION));
EasyMock.expect(consumer.position(TOPIC_PARTITION)).andAnswer(
new IAnswer<Long>() {
@Override
public Long answer() throws Throwable {
return FIRST_OFFSET + recordsReturned - 1;
}
}
);
sinkTask.flush(Collections.singletonMap(TOPIC_PARTITION, finalOffset));
IExpectationSetters<Object> flushExpectation = PowerMock.expectLastCall();
if (flushError != null) {
flushExpectation.andThrow(flushError).once();
return null;
}
final Capture<ConsumerCommitCallback> capturedCallback = EasyMock.newCapture();
final Map<TopicPartition, Long> offsets = Collections.singletonMap(TOPIC_PARTITION, finalOffset);
consumer.commit(EasyMock.eq(offsets),
EasyMock.eq(CommitType.ASYNC),
EasyMock.capture(capturedCallback));
PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
@Override
public Object answer() throws Throwable {
time.sleep(consumerCommitDelayMs);
if (invokeCallback)
capturedCallback.getValue().onComplete(offsets, consumerCommitError);
return null;
}
});
return capturedCallback;
}
}

View File

@ -0,0 +1,279 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.utils.SystemTime;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.copycat.cli.WorkerConfig;
import org.apache.kafka.copycat.source.SourceRecord;
import org.apache.kafka.copycat.source.SourceTask;
import org.apache.kafka.copycat.source.SourceTaskContext;
import org.apache.kafka.copycat.storage.Converter;
import org.apache.kafka.copycat.storage.OffsetStorageReader;
import org.apache.kafka.copycat.storage.OffsetStorageWriter;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.apache.kafka.copycat.util.ThreadedTest;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.easymock.IAnswer;
import org.easymock.IExpectationSetters;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock;
import org.powermock.api.easymock.annotation.Mock;
import org.powermock.modules.junit4.PowerMockRunner;
import org.powermock.reflect.Whitebox;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.junit.Assert.*;
@RunWith(PowerMockRunner.class)
public class WorkerSourceTaskTest extends ThreadedTest {
private static final byte[] PARTITION_BYTES = "partition".getBytes();
private static final byte[] OFFSET_BYTES = "offset-1".getBytes();
// Copycat-format data
private static final Integer KEY = -1;
private static final Long RECORD = 12L;
// Native-formatted data. The actual format of this data doesn't matter -- we just want to see that the right version
// is used in the right place.
private static final ByteBuffer CONVERTED_KEY = ByteBuffer.wrap("converted-key".getBytes());
private static final String CONVERTED_RECORD = "converted-record";
private ConnectorTaskId taskId = new ConnectorTaskId("job", 0);
private WorkerConfig config;
@Mock private SourceTask sourceTask;
@Mock private Converter<ByteBuffer> keyConverter;
@Mock private Converter<String> valueConverter;
@Mock private KafkaProducer<ByteBuffer, String> producer;
@Mock private OffsetStorageReader offsetReader;
@Mock private OffsetStorageWriter<ByteBuffer, String> offsetWriter;
private WorkerSourceTask<ByteBuffer, String> workerTask;
@Mock private Future<RecordMetadata> sendFuture;
private Capture<org.apache.kafka.clients.producer.Callback> producerCallbacks;
private static final Properties EMPTY_TASK_PROPS = new Properties();
private static final List<SourceRecord> RECORDS = Arrays.asList(
new SourceRecord(PARTITION_BYTES, OFFSET_BYTES, "topic", null, KEY, RECORD)
);
@Override
public void setup() {
super.setup();
Properties workerProps = new Properties();
workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("key.serializer", "org.apache.kafka.copycat.json.JsonSerializer");
workerProps.setProperty("value.serializer", "org.apache.kafka.copycat.json.JsonSerializer");
workerProps.setProperty("key.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer");
workerProps.setProperty("value.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer");
config = new WorkerConfig(workerProps);
producerCallbacks = EasyMock.newCapture();
}
private void createWorkerTask() {
workerTask = new WorkerSourceTask<>(taskId, sourceTask, keyConverter, valueConverter, producer,
offsetReader, offsetWriter, config, new SystemTime());
}
@Test
public void testPollsInBackground() throws Exception {
createWorkerTask();
sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
EasyMock.expectLastCall();
sourceTask.start(EMPTY_TASK_PROPS);
EasyMock.expectLastCall();
final CountDownLatch pollLatch = expectPolls(10);
// In this test, we don't flush, so nothing goes any further than the offset writer
sourceTask.stop();
EasyMock.expectLastCall();
expectOffsetFlush(true);
PowerMock.replayAll();
workerTask.start(EMPTY_TASK_PROPS);
awaitPolls(pollLatch);
workerTask.stop();
assertEquals(true, workerTask.awaitStop(1000));
PowerMock.verifyAll();
}
@Test
public void testCommit() throws Exception {
// Test that the task commits properly when prompted
createWorkerTask();
sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
EasyMock.expectLastCall();
sourceTask.start(EMPTY_TASK_PROPS);
EasyMock.expectLastCall();
// We'll wait for some data, then trigger a flush
final CountDownLatch pollLatch = expectPolls(1);
expectOffsetFlush(true);
sourceTask.stop();
EasyMock.expectLastCall();
expectOffsetFlush(true);
PowerMock.replayAll();
workerTask.start(EMPTY_TASK_PROPS);
awaitPolls(pollLatch);
assertTrue(workerTask.commitOffsets());
workerTask.stop();
assertEquals(true, workerTask.awaitStop(1000));
PowerMock.verifyAll();
}
@Test
public void testCommitFailure() throws Exception {
// Test that the task commits properly when prompted
createWorkerTask();
sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class));
EasyMock.expectLastCall();
sourceTask.start(EMPTY_TASK_PROPS);
EasyMock.expectLastCall();
// We'll wait for some data, then trigger a flush
final CountDownLatch pollLatch = expectPolls(1);
expectOffsetFlush(false);
sourceTask.stop();
EasyMock.expectLastCall();
expectOffsetFlush(true);
PowerMock.replayAll();
workerTask.start(EMPTY_TASK_PROPS);
awaitPolls(pollLatch);
assertFalse(workerTask.commitOffsets());
workerTask.stop();
assertEquals(true, workerTask.awaitStop(1000));
PowerMock.verifyAll();
}
@Test
public void testSendRecordsConvertsData() throws Exception {
createWorkerTask();
List<SourceRecord> records = new ArrayList<>();
// Can just use the same record for key and value
records.add(new SourceRecord(PARTITION_BYTES, OFFSET_BYTES, "topic", null, KEY, RECORD));
Capture<ProducerRecord<ByteBuffer, String>> sent = expectSendRecord();
PowerMock.replayAll();
Whitebox.invokeMethod(workerTask, "sendRecords", records);
assertEquals(CONVERTED_KEY, sent.getValue().key());
assertEquals(CONVERTED_RECORD, sent.getValue().value());
PowerMock.verifyAll();
}
private CountDownLatch expectPolls(int count) throws InterruptedException {
final CountDownLatch latch = new CountDownLatch(count);
// Note that we stub these to allow any number of calls because the thread will continue to
// run. The count passed in + latch returned just makes sure we get *at least* that number of
// calls
EasyMock.expect(sourceTask.poll())
.andStubAnswer(new IAnswer<List<SourceRecord>>() {
@Override
public List<SourceRecord> answer() throws Throwable {
latch.countDown();
return RECORDS;
}
});
// Fallout of the poll() call
expectSendRecord();
return latch;
}
private Capture<ProducerRecord<ByteBuffer, String>> expectSendRecord() throws InterruptedException {
EasyMock.expect(keyConverter.fromCopycatData(KEY)).andStubReturn(CONVERTED_KEY);
EasyMock.expect(valueConverter.fromCopycatData(RECORD)).andStubReturn(CONVERTED_RECORD);
Capture<ProducerRecord<ByteBuffer, String>> sent = EasyMock.newCapture();
// 1. Converted data passed to the producer, which will need callbacks invoked for flush to work
EasyMock.expect(
producer.send(EasyMock.capture(sent),
EasyMock.capture(producerCallbacks)))
.andStubAnswer(new IAnswer<Future<RecordMetadata>>() {
@Override
public Future<RecordMetadata> answer() throws Throwable {
synchronized (producerCallbacks) {
for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) {
cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0), null);
}
producerCallbacks.reset();
}
return sendFuture;
}
});
// 2. Offset data is passed to the offset storage.
offsetWriter.setOffset(PARTITION_BYTES, OFFSET_BYTES);
PowerMock.expectLastCall().anyTimes();
return sent;
}
private void awaitPolls(CountDownLatch latch) throws InterruptedException {
latch.await(1000, TimeUnit.MILLISECONDS);
}
@SuppressWarnings("unchecked")
private void expectOffsetFlush(boolean succeed) throws Exception {
EasyMock.expect(offsetWriter.beginFlush()).andReturn(true);
Future<Void> flushFuture = PowerMock.createMock(Future.class);
EasyMock.expect(offsetWriter.doFlush(EasyMock.anyObject(Callback.class))).andReturn(flushFuture);
// Should throw for failure
IExpectationSetters<Void> futureGetExpect = EasyMock.expect(
flushFuture.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class)));
if (succeed) {
futureGetExpect.andReturn(null);
} else {
futureGetExpect.andThrow(new TimeoutException());
offsetWriter.cancelFlush();
PowerMock.expectLastCall();
}
}
}

View File

@ -0,0 +1,179 @@
/**
* 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.copycat.runtime;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.copycat.cli.WorkerConfig;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.source.SourceRecord;
import org.apache.kafka.copycat.source.SourceTask;
import org.apache.kafka.copycat.storage.*;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.apache.kafka.copycat.util.MockTime;
import org.apache.kafka.copycat.util.ThreadedTest;
import org.easymock.EasyMock;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import java.util.List;
import java.util.Properties;
@RunWith(PowerMockRunner.class)
@PrepareForTest(Worker.class)
@PowerMockIgnore("javax.management.*")
public class WorkerTest extends ThreadedTest {
private ConnectorTaskId taskId = new ConnectorTaskId("job", 0);
private Worker worker;
private OffsetBackingStore offsetBackingStore = PowerMock.createMock(OffsetBackingStore.class);
private Serializer offsetKeySerializer = PowerMock.createMock(Serializer.class);
private Serializer offsetValueSerializer = PowerMock.createMock(Serializer.class);
private Deserializer offsetKeyDeserializer = PowerMock.createMock(Deserializer.class);
private Deserializer offsetValueDeserializer = PowerMock.createMock(Deserializer.class);
@Before
public void setup() {
super.setup();
Properties workerProps = new Properties();
workerProps.setProperty("key.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("value.converter", "org.apache.kafka.copycat.json.JsonConverter");
workerProps.setProperty("key.serializer", "org.apache.kafka.copycat.json.JsonSerializer");
workerProps.setProperty("value.serializer", "org.apache.kafka.copycat.json.JsonSerializer");
workerProps.setProperty("key.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer");
workerProps.setProperty("value.deserializer", "org.apache.kafka.copycat.json.JsonDeserializer");
WorkerConfig config = new WorkerConfig(workerProps);
worker = new Worker(new MockTime(), config, offsetBackingStore,
offsetKeySerializer, offsetValueSerializer,
offsetKeyDeserializer, offsetValueDeserializer);
worker.start();
}
@Test
public void testAddRemoveTask() throws Exception {
ConnectorTaskId taskId = new ConnectorTaskId("job", 0);
// Create
TestSourceTask task = PowerMock.createMock(TestSourceTask.class);
WorkerSourceTask workerTask = PowerMock.createMock(WorkerSourceTask.class);
PowerMock.mockStatic(Worker.class);
PowerMock.expectPrivate(Worker.class, "instantiateTask", TestSourceTask.class.getName()).andReturn(task);
PowerMock.expectNew(
WorkerSourceTask.class, EasyMock.eq(taskId), EasyMock.eq(task),
EasyMock.anyObject(Converter.class),
EasyMock.anyObject(Converter.class),
EasyMock.anyObject(KafkaProducer.class),
EasyMock.anyObject(OffsetStorageReader.class),
EasyMock.anyObject(OffsetStorageWriter.class),
EasyMock.anyObject(WorkerConfig.class),
EasyMock.anyObject(Time.class))
.andReturn(workerTask);
Properties origProps = new Properties();
workerTask.start(origProps);
EasyMock.expectLastCall();
// Remove
workerTask.stop();
EasyMock.expectLastCall();
EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true);
workerTask.close();
EasyMock.expectLastCall();
PowerMock.replayAll();
worker.addTask(taskId, TestSourceTask.class.getName(), origProps);
worker.stopTask(taskId);
// Nothing should be left, so this should effectively be a nop
worker.stop();
PowerMock.verifyAll();
}
@Test(expected = CopycatException.class)
public void testStopInvalidTask() {
worker.stopTask(taskId);
}
@Test
public void testCleanupTasksOnStop() throws Exception {
// Create
TestSourceTask task = PowerMock.createMock(TestSourceTask.class);
WorkerSourceTask workerTask = PowerMock.createMock(WorkerSourceTask.class);
PowerMock.mockStatic(Worker.class);
PowerMock.expectPrivate(Worker.class, "instantiateTask", TestSourceTask.class.getName()).andReturn(task);
PowerMock.expectNew(
WorkerSourceTask.class, EasyMock.eq(taskId), EasyMock.eq(task),
EasyMock.anyObject(Converter.class),
EasyMock.anyObject(Converter.class),
EasyMock.anyObject(KafkaProducer.class),
EasyMock.anyObject(OffsetStorageReader.class),
EasyMock.anyObject(OffsetStorageWriter.class),
EasyMock.anyObject(WorkerConfig.class),
EasyMock.anyObject(Time.class))
.andReturn(workerTask);
Properties origProps = new Properties();
workerTask.start(origProps);
EasyMock.expectLastCall();
// Remove on Worker.stop()
workerTask.stop();
EasyMock.expectLastCall();
EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andReturn(true);
// Note that in this case we *do not* commit offsets since it's an unclean shutdown
workerTask.close();
EasyMock.expectLastCall();
PowerMock.replayAll();
worker.addTask(taskId, TestSourceTask.class.getName(), origProps);
worker.stop();
PowerMock.verifyAll();
}
private static class TestSourceTask extends SourceTask {
public TestSourceTask() {
}
@Override
public void start(Properties props) {
}
@Override
public List<SourceRecord> poll() throws InterruptedException {
return null;
}
@Override
public void stop() {
}
}
}

View File

@ -0,0 +1,186 @@
/**
* 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.copycat.runtime.standalone;
import org.apache.kafka.copycat.connector.Connector;
import org.apache.kafka.copycat.connector.Task;
import org.apache.kafka.copycat.runtime.ConnectorConfig;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.sink.SinkConnector;
import org.apache.kafka.copycat.sink.SinkTask;
import org.apache.kafka.copycat.source.SourceConnector;
import org.apache.kafka.copycat.source.SourceTask;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.ConnectorTaskId;
import org.apache.kafka.copycat.util.FutureCallback;
import org.easymock.EasyMock;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock;
import org.powermock.api.easymock.annotation.Mock;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import java.util.Arrays;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
@RunWith(PowerMockRunner.class)
@PrepareForTest({StandaloneHerder.class})
@PowerMockIgnore("javax.management.*")
public class StandaloneHerderTest {
private static final String CONNECTOR_NAME = "test";
private static final String TOPICS_LIST_STR = "topic1,topic2";
private StandaloneHerder herder;
@Mock protected Worker worker;
private Connector connector;
@Mock protected Callback<String> createCallback;
private Properties connectorProps;
private Properties taskProps;
@Before
public void setup() {
worker = PowerMock.createMock(Worker.class);
herder = new StandaloneHerder(worker);
connectorProps = new Properties();
connectorProps.setProperty(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME);
connectorProps.setProperty(SinkConnector.TOPICS_CONFIG, TOPICS_LIST_STR);
PowerMock.mockStatic(StandaloneHerder.class);
// These can be anything since connectors can pass along whatever they want.
taskProps = new Properties();
taskProps.setProperty("foo", "bar");
}
@Test
public void testCreateSourceConnector() throws Exception {
connector = PowerMock.createMock(BogusSourceClass.class);
expectAdd(BogusSourceClass.class, BogusSourceTask.class, false);
PowerMock.replayAll();
herder.addConnector(connectorProps, createCallback);
PowerMock.verifyAll();
}
@Test
public void testCreateSinkConnector() throws Exception {
connector = PowerMock.createMock(BogusSinkClass.class);
expectAdd(BogusSinkClass.class, BogusSinkTask.class, true);
PowerMock.replayAll();
herder.addConnector(connectorProps, createCallback);
PowerMock.verifyAll();
}
@Test
public void testDestroyConnector() throws Exception {
connector = PowerMock.createMock(BogusSourceClass.class);
expectAdd(BogusSourceClass.class, BogusSourceTask.class, false);
expectDestroy();
PowerMock.replayAll();
herder.addConnector(connectorProps, createCallback);
FutureCallback<Void> futureCb = new FutureCallback<>(new Callback<Void>() {
@Override
public void onCompletion(Throwable error, Void result) {
}
});
herder.deleteConnector(CONNECTOR_NAME, futureCb);
futureCb.get(1000L, TimeUnit.MILLISECONDS);
PowerMock.verifyAll();
}
private void expectAdd(Class<? extends Connector> connClass,
Class<? extends Task> taskClass,
boolean sink) throws Exception {
expectCreate(connClass, taskClass, sink, true);
}
private void expectRestore(Class<? extends Connector> connClass,
Class<? extends Task> taskClass) throws Exception {
// Restore never uses a callback. These tests always use sources
expectCreate(connClass, taskClass, false, false);
}
private void expectCreate(Class<? extends Connector> connClass,
Class<? extends Task> taskClass,
boolean sink, boolean expectCallback) throws Exception {
connectorProps.setProperty(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connClass.getName());
PowerMock.expectPrivate(StandaloneHerder.class, "instantiateConnector", connClass.getName())
.andReturn(connector);
if (expectCallback) {
createCallback.onCompletion(null, CONNECTOR_NAME);
PowerMock.expectLastCall();
}
connector.initialize(EasyMock.anyObject(StandaloneConnectorContext.class));
PowerMock.expectLastCall();
connector.start(new Properties());
PowerMock.expectLastCall();
// Just return the connector properties for the individual task we generate by default
EasyMock.<Class<? extends Task>>expect(connector.getTaskClass()).andReturn(taskClass);
EasyMock.expect(connector.getTaskConfigs(ConnectorConfig.TASKS_MAX_DEFAULT))
.andReturn(Arrays.asList(taskProps));
// And we should instantiate the tasks. For a sink task, we should see added properties for
// the input topic partitions
Properties generatedTaskProps = new Properties();
generatedTaskProps.putAll(taskProps);
if (sink)
generatedTaskProps.setProperty(SinkTask.TOPICS_CONFIG, TOPICS_LIST_STR);
worker.addTask(new ConnectorTaskId(CONNECTOR_NAME, 0), taskClass.getName(), generatedTaskProps);
PowerMock.expectLastCall();
}
private void expectStop() {
worker.stopTask(new ConnectorTaskId(CONNECTOR_NAME, 0));
EasyMock.expectLastCall();
connector.stop();
EasyMock.expectLastCall();
}
private void expectDestroy() {
expectStop();
}
// We need to use a real class here due to some issue with mocking java.lang.Class
private abstract class BogusSourceClass extends SourceConnector {
}
private abstract class BogusSourceTask extends SourceTask {
}
private abstract class BogusSinkClass extends SinkConnector {
}
private abstract class BogusSinkTask extends SourceTask {
}
}

View File

@ -0,0 +1,117 @@
/**
* 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.copycat.storage;
import org.apache.kafka.copycat.util.Callback;
import org.easymock.EasyMock;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.powermock.api.easymock.PowerMock;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import static org.junit.Assert.assertEquals;
public class FileOffsetBackingStoreTest {
FileOffsetBackingStore store;
Map<String, Object> props;
File tempFile;
private static Map<ByteBuffer, ByteBuffer> firstSet = new HashMap<>();
static {
firstSet.put(buffer("key"), buffer("value"));
firstSet.put(null, null);
}
@Before
public void setup() throws IOException {
store = new FileOffsetBackingStore();
tempFile = File.createTempFile("fileoffsetbackingstore", null);
props = new HashMap<>();
props.put(FileOffsetBackingStore.OFFSET_STORAGE_FILE_FILENAME_CONFIG, tempFile.getAbsolutePath());
store.configure(props);
store.start();
}
@After
public void teardown() {
tempFile.delete();
}
@Test
public void testGetSet() throws Exception {
Callback<Void> setCallback = expectSuccessfulSetCallback();
Callback<Map<ByteBuffer, ByteBuffer>> getCallback = expectSuccessfulGetCallback();
PowerMock.replayAll();
store.set("namespace", firstSet, setCallback).get();
Map<ByteBuffer, ByteBuffer> values = store.get("namespace", Arrays.asList(buffer("key"), buffer("bad")), getCallback).get();
assertEquals(buffer("value"), values.get(buffer("key")));
assertEquals(null, values.get(buffer("bad")));
PowerMock.verifyAll();
}
@Test
public void testSaveRestore() throws Exception {
Callback<Void> setCallback = expectSuccessfulSetCallback();
Callback<Map<ByteBuffer, ByteBuffer>> getCallback = expectSuccessfulGetCallback();
PowerMock.replayAll();
store.set("namespace", firstSet, setCallback).get();
store.stop();
// Restore into a new store to ensure correct reload from scratch
FileOffsetBackingStore restore = new FileOffsetBackingStore();
restore.configure(props);
restore.start();
Map<ByteBuffer, ByteBuffer> values = restore.get("namespace", Arrays.asList(buffer("key")), getCallback).get();
assertEquals(buffer("value"), values.get(buffer("key")));
PowerMock.verifyAll();
}
private static ByteBuffer buffer(String v) {
return ByteBuffer.wrap(v.getBytes());
}
private Callback<Void> expectSuccessfulSetCallback() {
@SuppressWarnings("unchecked")
Callback<Void> setCallback = PowerMock.createMock(Callback.class);
setCallback.onCompletion(EasyMock.isNull(Throwable.class), EasyMock.isNull(Void.class));
PowerMock.expectLastCall();
return setCallback;
}
@SuppressWarnings("unchecked")
private Callback<Map<ByteBuffer, ByteBuffer>> expectSuccessfulGetCallback() {
Callback<Map<ByteBuffer, ByteBuffer>> getCallback = PowerMock.createMock(Callback.class);
getCallback.onCompletion(EasyMock.isNull(Throwable.class), EasyMock.anyObject(Map.class));
PowerMock.expectLastCall();
return getCallback;
}
}

View File

@ -0,0 +1,242 @@
/**
* 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.copycat.storage;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.util.Callback;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.easymock.IAnswer;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock;
import org.powermock.api.easymock.annotation.Mock;
import org.powermock.modules.junit4.PowerMockRunner;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.*;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@RunWith(PowerMockRunner.class)
public class OffsetStorageWriterTest {
private static final String NAMESPACE = "namespace";
// Copycat format - any types should be accepted here
private static final List<String> OFFSET_KEY = Arrays.asList("key", "key");
private static final String OFFSET_VALUE = "value";
// Native objects - must match serializer types
private static final int OFFSET_KEY_CONVERTED = 12;
private static final String OFFSET_VALUE_CONVERTED = "value-converted";
// Serialized
private static final byte[] OFFSET_KEY_SERIALIZED = "key-serialized".getBytes();
private static final byte[] OFFSET_VALUE_SERIALIZED = "value-serialized".getBytes();
private static final Map<ByteBuffer, ByteBuffer> OFFSETS_SERIALIZED
= Collections.singletonMap(ByteBuffer.wrap(OFFSET_KEY_SERIALIZED),
ByteBuffer.wrap(OFFSET_VALUE_SERIALIZED));
@Mock private OffsetBackingStore store;
@Mock private Converter<Integer> keyConverter;
@Mock private Converter<String> valueConverter;
@Mock private Serializer<Integer> keySerializer;
@Mock private Serializer<String> valueSerializer;
private OffsetStorageWriter<Integer, String> writer;
private static Exception exception = new RuntimeException("error");
private ExecutorService service;
@Before
public void setup() {
writer = new OffsetStorageWriter<>(store, NAMESPACE, keyConverter, valueConverter, keySerializer, valueSerializer);
service = Executors.newFixedThreadPool(1);
}
@After
public void teardown() {
service.shutdownNow();
}
@Test
public void testWriteFlush() throws Exception {
@SuppressWarnings("unchecked")
Callback<Void> callback = PowerMock.createMock(Callback.class);
expectStore(callback, false);
PowerMock.replayAll();
writer.setOffset(OFFSET_KEY, OFFSET_VALUE);
assertTrue(writer.beginFlush());
writer.doFlush(callback).get(1000, TimeUnit.MILLISECONDS);
PowerMock.verifyAll();
}
@Test
public void testNoOffsetsToFlush() {
// If no offsets are flushed, we should finish immediately and not have made any calls to the
// underlying storage layer
PowerMock.replayAll();
// Should not return a future
assertFalse(writer.beginFlush());
PowerMock.verifyAll();
}
@Test
public void testFlushFailureReplacesOffsets() throws Exception {
// When a flush fails, we shouldn't just lose the offsets. Instead, they should be restored
// such that a subsequent flush will write them.
@SuppressWarnings("unchecked")
final Callback<Void> callback = PowerMock.createMock(Callback.class);
// First time the write fails
expectStore(callback, true);
// Second time it succeeds
expectStore(callback, false);
// Third time it has no data to flush so we won't get past beginFlush()
PowerMock.replayAll();
writer.setOffset(OFFSET_KEY, OFFSET_VALUE);
assertTrue(writer.beginFlush());
writer.doFlush(callback).get(1000, TimeUnit.MILLISECONDS);
assertTrue(writer.beginFlush());
writer.doFlush(callback).get(1000, TimeUnit.MILLISECONDS);
assertFalse(writer.beginFlush());
PowerMock.verifyAll();
}
@Test(expected = CopycatException.class)
public void testAlreadyFlushing() throws Exception {
@SuppressWarnings("unchecked")
final Callback<Void> callback = PowerMock.createMock(Callback.class);
// Trigger the send, but don't invoke the callback so we'll still be mid-flush
CountDownLatch allowStoreCompleteCountdown = new CountDownLatch(1);
expectStore(null, false, allowStoreCompleteCountdown);
PowerMock.replayAll();
writer.setOffset(OFFSET_KEY, OFFSET_VALUE);
assertTrue(writer.beginFlush());
writer.doFlush(callback);
assertTrue(writer.beginFlush()); // should throw
PowerMock.verifyAll();
}
@Test
public void testCancelBeforeAwaitFlush() {
PowerMock.replayAll();
writer.setOffset(OFFSET_KEY, OFFSET_VALUE);
assertTrue(writer.beginFlush());
writer.cancelFlush();
PowerMock.verifyAll();
}
@Test
public void testCancelAfterAwaitFlush() throws Exception {
@SuppressWarnings("unchecked")
Callback<Void> callback = PowerMock.createMock(Callback.class);
CountDownLatch allowStoreCompleteCountdown = new CountDownLatch(1);
// In this test, the write should be cancelled so the callback will not be invoked and is not
// passed to the expectStore call
expectStore(null, false, allowStoreCompleteCountdown);
PowerMock.replayAll();
writer.setOffset(OFFSET_KEY, OFFSET_VALUE);
assertTrue(writer.beginFlush());
// Start the flush, then immediately cancel before allowing the mocked store request to finish
Future<Void> flushFuture = writer.doFlush(callback);
writer.cancelFlush();
allowStoreCompleteCountdown.countDown();
flushFuture.get(1000, TimeUnit.MILLISECONDS);
PowerMock.verifyAll();
}
private void expectStore(final Callback<Void> callback, final boolean fail) {
expectStore(callback, fail, null);
}
/**
* Expect a request to store data to the underlying OffsetBackingStore.
*
* @param callback the callback to invoke when completed, or null if the callback isn't
* expected to be invoked
* @param fail if true, treat
* @param waitForCompletion if non-null, a CountDownLatch that should be awaited on before
* invoking the callback. A (generous) timeout is still imposed to
* ensure tests complete.
* @return the captured set of ByteBuffer key-value pairs passed to the storage layer
*/
private void expectStore(final Callback<Void> callback,
final boolean fail,
final CountDownLatch waitForCompletion) {
EasyMock.expect(keyConverter.fromCopycatData(OFFSET_KEY)).andReturn(OFFSET_KEY_CONVERTED);
EasyMock.expect(keySerializer.serialize(NAMESPACE, OFFSET_KEY_CONVERTED)).andReturn(OFFSET_KEY_SERIALIZED);
EasyMock.expect(valueConverter.fromCopycatData(OFFSET_VALUE)).andReturn(OFFSET_VALUE_CONVERTED);
EasyMock.expect(valueSerializer.serialize(NAMESPACE, OFFSET_VALUE_CONVERTED)).andReturn(OFFSET_VALUE_SERIALIZED);
final Capture<Callback<Void>> storeCallback = Capture.newInstance();
EasyMock.expect(store.set(EasyMock.eq(NAMESPACE), EasyMock.eq(OFFSETS_SERIALIZED),
EasyMock.capture(storeCallback)))
.andAnswer(new IAnswer<Future<Void>>() {
@Override
public Future<Void> answer() throws Throwable {
return service.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
if (waitForCompletion != null)
assertTrue(waitForCompletion.await(10000, TimeUnit.MILLISECONDS));
if (fail) {
storeCallback.getValue().onCompletion(exception, null);
} else {
storeCallback.getValue().onCompletion(null, null);
}
return null;
}
});
}
});
if (callback != null) {
if (fail) {
callback.onCompletion(EasyMock.eq(exception), EasyMock.eq((Void) null));
} else {
callback.onCompletion(null, null);
}
}
PowerMock.expectLastCall();
}
}

View File

@ -0,0 +1,49 @@
/**
* 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.copycat.util;
import org.apache.kafka.common.utils.Time;
import java.util.concurrent.TimeUnit;
/**
* A clock that you can manually advance by calling sleep
*/
public class MockTime implements Time {
private long nanos = 0;
public MockTime() {
this.nanos = System.nanoTime();
}
@Override
public long milliseconds() {
return TimeUnit.MILLISECONDS.convert(this.nanos, TimeUnit.NANOSECONDS);
}
@Override
public long nanoseconds() {
return nanos;
}
@Override
public void sleep(long ms) {
this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS);
}
}

View File

@ -0,0 +1,72 @@
/**
* 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.copycat.util;
import org.junit.Test;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class ShutdownableThreadTest {
@Test
public void testGracefulShutdown() throws InterruptedException {
ShutdownableThread thread = new ShutdownableThread("graceful") {
@Override
public void execute() {
while (getRunning()) {
try {
Thread.sleep(1);
} catch (InterruptedException e) {
// Ignore
}
}
}
};
thread.start();
Thread.sleep(10);
assertTrue(thread.gracefulShutdown(1000, TimeUnit.MILLISECONDS));
}
@Test
public void testForcibleShutdown() throws InterruptedException {
final CountDownLatch startedLatch = new CountDownLatch(1);
ShutdownableThread thread = new ShutdownableThread("forcible") {
@Override
public void execute() {
try {
startedLatch.countDown();
Thread.sleep(100000);
} catch (InterruptedException e) {
// Ignore
}
}
};
thread.start();
startedLatch.await();
thread.forceShutdown();
// Not all threads can be forcibly stopped since interrupt() doesn't work on threads in
// certain conditions, but in this case we know the thread is interruptible so we should be
// able join() it
thread.join(1000);
assertFalse(thread.isAlive());
}
}

View File

@ -0,0 +1,37 @@
/**
* 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.copycat.util;
/**
* An UncaughtExceptionHandler that can be registered with one or more threads which tracks the
* first exception so the main thread can check for uncaught exceptions.
*/
public class TestBackgroundThreadExceptionHandler implements Thread.UncaughtExceptionHandler {
private Throwable firstException = null;
@Override
public void uncaughtException(Thread t, Throwable e) {
if (this.firstException == null)
this.firstException = e;
}
public void verifyNoExceptions() {
if (this.firstException != null)
throw new AssertionError(this.firstException);
}
}

View File

@ -0,0 +1,43 @@
/**
* 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.copycat.util;
import org.junit.After;
import org.junit.Before;
/**
* Base class for tests that use threads. It sets up uncaught exception handlers for all known
* thread classes and checks for errors at the end of the test so that failures in background
* threads will cause the test to fail.
*/
public class ThreadedTest {
protected TestBackgroundThreadExceptionHandler backgroundThreadExceptionHandler;
@Before
public void setup() {
backgroundThreadExceptionHandler = new TestBackgroundThreadExceptionHandler();
ShutdownableThread.funcaughtExceptionHandler = backgroundThreadExceptionHandler;
}
@After
public void teardown() {
backgroundThreadExceptionHandler.verifyNoExceptions();
ShutdownableThread.funcaughtExceptionHandler = null;
}
}

View File

@ -14,5 +14,5 @@
// limitations under the License.
apply from: file('scala.gradle')
include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'log4j-appender'
include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'log4j-appender',
'copycat:data', 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file'