Add copycat-avro and copycat-runtime

This commit is contained in:
Ewen Cheslack-Postava 2015-07-24 14:59:54 -07:00
parent 11981d2eaa
commit 0233456c29
51 changed files with 6327 additions and 49 deletions

View File

@ -33,6 +33,7 @@ allprojects {
apply plugin: 'idea'
repositories {
mavenCentral()
mavenLocal() // FIXME Currently required to get io.confluent:kafka-avro-serializer:2.0-SNAPSHOT
}
}
@ -239,7 +240,7 @@ project(':core') {
}
testCompile 'junit:junit:4.6'
testCompile 'org.easymock:easymock:3.0'
testCompile 'org.easymock:easymock:3.3.1'
testCompile 'org.objenesis:objenesis:1.2'
testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5"
@ -514,7 +515,7 @@ project(':copycat-api') {
}
javadoc {
include "**/org/apache/kafka/copycat/api/*"
include "**/org/apache/kafka/copycat/*"
}
artifacts {
@ -525,9 +526,103 @@ project(':copycat-api') {
archives.extendsFrom (testCompile)
}
/* FIXME
checkstyle {
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
}
test.dependsOn('checkstyleMain', 'checkstyleTest') */
test.dependsOn('checkstyleMain', 'checkstyleTest')
}
project(':copycat-avro') {
apply plugin: 'checkstyle'
archivesBaseName = "copycat-avro"
dependencies {
compile project(':copycat-api')
compile "org.slf4j:slf4j-api:1.7.6"
compile "io.confluent:kafka-avro-serializer:2.0-SNAPSHOT"
testCompile 'junit:junit:4.6'
testCompile 'org.easymock:easymock:3.3.1'
testCompile 'org.powermock:powermock-module-junit4:1.6.2'
testCompile 'org.powermock:powermock-api-easymock:1.6.2'
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-runtime') {
apply plugin: 'checkstyle'
archivesBaseName = "copycat-runtime"
dependencies {
compile project(':copycat-api')
compile project(':copycat-avro')
compile project(':clients')
compile "org.slf4j:slf4j-api:1.7.6"
// FIXME we shouldn't depend on core since we only need clients, but currently we're doing sink task topic-partition assignment manually until we switch over to new consumer group functionality
compile project(':core')
testCompile 'junit:junit:4.6'
testCompile 'org.easymock:easymock:3.3.1'
testCompile 'org.powermock:powermock-module-junit4:1.6.2'
testCompile 'org.powermock:powermock-api-easymock:1.6.2'
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

@ -119,6 +119,49 @@
<allow pkg="org.apache.kafka.copycat.connector" />
<allow pkg="org.apache.kafka.copycat.storage" />
</subpackage>
<subpackage name="avro">
<allow pkg="org.apache.avro" />
<allow pkg="org.apache.kafka.copycat" />
<!-- FIXME/move Avro plugin out of Kafka -->
<allow pkg="io.confluent.kafka.serializers" />
<allow pkg="io.confluent.kafka.schemaregistry" />
</subpackage>
<subpackage name="runtime">
<allow pkg="org.apache.kafka.copycat" />
<allow pkg="org.apache.kafka.common" />
<allow pkg="org.apache.kafka.clients" />
<!-- for tests -->
<allow pkg="org.easymock" />
<allow pkg="org.powermock" />
<!-- FIXME Remove when dependency on core for sink partition assignment is removed -->
<allow pkg="kafka.utils" />
<allow pkg="org.I0Itec.zkclient" />
</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" />
<!-- for tests -->
<allow pkg="org.easymock" />
<allow pkg="org.powermock" />
</subpackage>
<subpackage name="util">
<allow pkg="org.apache.kafka.copycat" />
<!-- FIXME Remove when dependency on core for sink partition assignment is removed -->
<allow pkg="kafka.utils" />
<allow pkg="org.I0Itec.zkclient" />
<allow pkg="scala" />
</subpackage>
</subpackage>
</import-control>

View File

@ -0,0 +1,35 @@
/**
* 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.avro;
import org.apache.kafka.copycat.storage.Converter;
/**
* Implementation of Converter that uses Avro schemas and objects.
*/
public class AvroConverter implements Converter {
@Override
public Object fromCopycatData(Object value) {
return AvroData.convertToAvro(value);
}
@Override
public Object toCopycatData(Object value) {
return AvroData.convertFromAvro(value);
}
}

View File

@ -0,0 +1,350 @@
/**
* 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.avro;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.io.*;
import org.apache.kafka.copycat.data.GenericRecord;
import org.apache.kafka.copycat.data.GenericRecordBuilder;
import org.apache.kafka.copycat.data.Schema;
import org.apache.kafka.copycat.data.SchemaBuilder;
import org.apache.kafka.copycat.errors.CopycatRuntimeException;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.*;
/**
* Utilities for converting between our runtime data format and Avro, and (de)serializing that data.
*/
public class AvroData {
private static final Map<String, org.apache.avro.Schema> PRIMITIVE_SCHEMAS;
static {
org.apache.avro.Schema.Parser parser = new org.apache.avro.Schema.Parser();
PRIMITIVE_SCHEMAS = new HashMap<String, org.apache.avro.Schema>();
PRIMITIVE_SCHEMAS.put("Null", createPrimitiveSchema(parser, "null"));
PRIMITIVE_SCHEMAS.put("Boolean", createPrimitiveSchema(parser, "boolean"));
PRIMITIVE_SCHEMAS.put("Integer", createPrimitiveSchema(parser, "int"));
PRIMITIVE_SCHEMAS.put("Long", createPrimitiveSchema(parser, "long"));
PRIMITIVE_SCHEMAS.put("Float", createPrimitiveSchema(parser, "float"));
PRIMITIVE_SCHEMAS.put("Double", createPrimitiveSchema(parser, "double"));
PRIMITIVE_SCHEMAS.put("String", createPrimitiveSchema(parser, "string"));
PRIMITIVE_SCHEMAS.put("Bytes", createPrimitiveSchema(parser, "bytes"));
}
private static final EncoderFactory ENCODER_FACTORY = EncoderFactory.get();
private static final DecoderFactory DECODER_FACTORY = DecoderFactory.get();
/**
* Convert this object, in the org.apache.kafka.copycat.data format, into an Avro object.
*/
public static Object convertToAvro(Object value) {
if (value == null || value instanceof Byte || value instanceof Short ||
value instanceof Integer || value instanceof Long ||
value instanceof Float || value instanceof Double ||
value instanceof Boolean ||
value instanceof byte[] || value instanceof ByteBuffer ||
value instanceof CharSequence) {
// Note that using CharSequence allows Utf8 -- either copycat or Avro variants -- to pass
// through, but this should be fine as long as they are always handled as CharSequences
return value;
} else if (value instanceof GenericRecord) {
GenericRecord recordValue = (GenericRecord) value;
org.apache.avro.Schema avroSchema = asAvroSchema(recordValue.getSchema());
org.apache.avro.generic.GenericRecordBuilder builder
= new org.apache.avro.generic.GenericRecordBuilder(avroSchema);
for (Schema.Field field : recordValue.getSchema().getFields()) {
builder.set(field.name(), convertToAvro(recordValue.get(field.name())));
}
return builder.build();
} else if (value instanceof Collection) {
Collection collection = (Collection) value;
List<Object> converted = new ArrayList<Object>(collection.size());
for (Object elem : collection) {
converted.add(convertToAvro(elem));
}
return converted;
} else if (value instanceof Map) {
Map<String, Object> map = (Map<String, Object>) value;
Map<String, Object> converted = new TreeMap<String, Object>();
for (Map.Entry<String, Object> entry : map.entrySet()) {
converted.put(entry.getKey(), convertToAvro(entry.getValue()));
}
return converted;
}
// Fixed and enum are handled by byte[] and String
throw new RuntimeException("Couldn't convert " + value + " to Avro.");
}
/**
* Serialize this object, in org.apache.kafka.copycat.data format, to Avro's binary encoding
*/
public static ByteBuffer serializeToAvro(Object value) {
Object asAvro = convertToAvro(value);
if (value == null) {
return null;
}
try {
org.apache.avro.Schema schema = getSchema(asAvro);
ByteArrayOutputStream out = new ByteArrayOutputStream();
BinaryEncoder encoder = ENCODER_FACTORY.directBinaryEncoder(out, null);
DatumWriter<Object> writer;
writer = new GenericDatumWriter<Object>(schema);
writer.write(asAvro, encoder);
encoder.flush();
return ByteBuffer.wrap(out.toByteArray());
} catch (IOException e) {
throw new CopycatRuntimeException("Couldn't serialize object to Avro", e);
}
}
/**
* Convert the given object, in Avro format, into an org.apache.kafka.copycat.data object.
*/
public static Object convertFromAvro(Object value) {
if (value == null || value instanceof Byte || value instanceof Short ||
value instanceof Integer || value instanceof Long ||
value instanceof Float || value instanceof Double ||
value instanceof Boolean ||
value instanceof byte[] || value instanceof ByteBuffer) {
return value;
} else if (value instanceof CharSequence) {
// We need to be careful about CharSequences. This could be a String or a Utf8. If we passed
// Utf8 values directly through using Avro's implementation, equality wouldn't work.
if (value instanceof org.apache.avro.util.Utf8) {
return value.toString();
} else {
return value;
}
} else if (value instanceof org.apache.avro.generic.GenericRecord) {
org.apache.avro.generic.GenericRecord recordValue
= (org.apache.avro.generic.GenericRecord) value;
Schema copycatSchema = asCopycatSchema(recordValue.getSchema());
GenericRecordBuilder builder = new GenericRecordBuilder(copycatSchema);
for (org.apache.avro.Schema.Field field : recordValue.getSchema().getFields()) {
builder.set(field.name(), convertFromAvro(recordValue.get(field.name())));
}
return builder.build();
} else if (value instanceof Collection) {
Collection collection = (Collection) value;
List<Object> converted = new ArrayList<Object>(collection.size());
for (Object elem : collection) {
converted.add(convertFromAvro(elem));
}
return converted;
} else if (value instanceof Map) {
Map<String, Object> map = (Map<String, Object>) value;
Map<String, Object> converted = new TreeMap<String, Object>();
for (Map.Entry<String, Object> entry : map.entrySet()) {
converted.put(entry.getKey(), convertFromAvro(entry.getValue()));
}
return converted;
}
// Fixed and enum are handled by byte[] and String
throw new RuntimeException("Couldn't convert " + value + " from Avro.");
}
/**
* Deserialize and convert the provided binary Avro-serialized bytes into
* org.apache.kafka.copycat.data format.
*/
public static Object deserializeFromAvro(ByteBuffer serialized, Schema schema) {
org.apache.avro.Schema avroSchema = asAvroSchema(schema);
DatumReader reader = new GenericDatumReader(avroSchema);
try {
Object deserialized =
reader.read(null, DECODER_FACTORY.binaryDecoder(serialized.array(), null));
return convertFromAvro(deserialized);
} catch (IOException e) {
throw new CopycatRuntimeException("Couldn't deserialize object from Avro", e);
}
}
private static org.apache.avro.Schema createPrimitiveSchema(
org.apache.avro.Schema.Parser parser, String type) {
String schemaString = String.format("{\"type\" : \"%s\"}", type);
return parser.parse(schemaString);
}
private static org.apache.avro.Schema getSchema(Object object) {
if (object == null) {
return PRIMITIVE_SCHEMAS.get("Null");
} else if (object instanceof Boolean) {
return PRIMITIVE_SCHEMAS.get("Boolean");
} else if (object instanceof Integer) {
return PRIMITIVE_SCHEMAS.get("Integer");
} else if (object instanceof Long) {
return PRIMITIVE_SCHEMAS.get("Long");
} else if (object instanceof Float) {
return PRIMITIVE_SCHEMAS.get("Float");
} else if (object instanceof Double) {
return PRIMITIVE_SCHEMAS.get("Double");
} else if (object instanceof CharSequence) {
return PRIMITIVE_SCHEMAS.get("String");
} else if (object instanceof byte[] || object instanceof ByteBuffer) {
return PRIMITIVE_SCHEMAS.get("Bytes");
} else if (object instanceof org.apache.avro.generic.GenericContainer) {
return ((org.apache.avro.generic.GenericContainer) object).getSchema();
} else {
throw new IllegalArgumentException(
"Unsupported Avro type: " + object.getClass());
}
}
// Implementation note -- I considered trying to unify asAvroSchema and asCopycatSchema through
// a generic implementation. In practice this probably won't be worth the relatively minor
// amount of code duplication avoided since a) the generic version is messy and b) the Copycat
// type system and Avro type system are likely to diverge anyway, which would eventually
// require splitting them up.
public static org.apache.avro.Schema asAvroSchema(Schema schema) {
switch (schema.getType()) {
case BOOLEAN:
return org.apache.avro.SchemaBuilder.builder().booleanType();
case BYTES:
return org.apache.avro.SchemaBuilder.builder().bytesType();
case DOUBLE:
return org.apache.avro.SchemaBuilder.builder().doubleType();
case FLOAT:
return org.apache.avro.SchemaBuilder.builder().floatType();
case INT:
return org.apache.avro.SchemaBuilder.builder().intType();
case LONG:
return org.apache.avro.SchemaBuilder.builder().longType();
case NULL:
return org.apache.avro.SchemaBuilder.builder().nullType();
case STRING:
return org.apache.avro.SchemaBuilder.builder().stringType();
case RECORD: {
List<org.apache.avro.Schema.Field> fields = new ArrayList<org.apache.avro.Schema.Field>();
for (Schema.Field field : schema.getFields()) {
// NOTE: Order ignored
// TODO: Providing a default value would require translating since Avro's is a JsonNode
fields.add(new org.apache.avro.Schema.Field(
field.name(), asAvroSchema(field.schema()), field.doc(), null));
}
org.apache.avro.Schema result = org.apache.avro.Schema.createRecord(
schema.getName(), schema.getDoc(), schema.getNamespace(), schema.isError());
result.setFields(fields);
return result;
}
case UNION: {
List<org.apache.avro.Schema> unionTypes = new ArrayList<org.apache.avro.Schema>();
for (Schema origType : schema.getTypes()) {
unionTypes.add(asAvroSchema(origType));
}
return org.apache.avro.Schema.createUnion(unionTypes);
}
case ARRAY:
return org.apache.avro.Schema.createArray(asAvroSchema(schema.getElementType()));
case ENUM:
return org.apache.avro.Schema.createEnum(schema.getName(), schema.getDoc(), schema
.getNamespace(), schema.getEnumSymbols());
case FIXED:
return org.apache.avro.Schema.createFixed(schema.getName(), schema.getDoc(), schema
.getNamespace(), schema.getFixedSize());
case MAP:
return org.apache.avro.Schema.createMap(asAvroSchema(schema.getValueType()));
default:
throw new CopycatRuntimeException("Couldn't translate unsupported schema type "
+ schema.getType().getName() + ".");
}
}
public static Schema asCopycatSchema(org.apache.avro.Schema schema) {
switch (schema.getType()) {
case BOOLEAN:
return SchemaBuilder.builder().booleanType();
case BYTES:
return SchemaBuilder.builder().bytesType();
case DOUBLE:
return SchemaBuilder.builder().doubleType();
case FLOAT:
return SchemaBuilder.builder().floatType();
case INT:
return SchemaBuilder.builder().intType();
case LONG:
return SchemaBuilder.builder().longType();
case NULL:
return SchemaBuilder.builder().nullType();
case STRING:
return SchemaBuilder.builder().stringType();
case RECORD: {
List<Schema.Field> fields = new ArrayList<Schema.Field>();
for (org.apache.avro.Schema.Field field : schema.getFields()) {
// NOTE: Order ignored
// TODO: Providing a default value would require translating since Avro's is a JsonNode
fields.add(new Schema.Field(
field.name(), asCopycatSchema(field.schema()), field.doc(), null));
}
Schema result = Schema.createRecord(
schema.getName(), schema.getDoc(), schema.getNamespace(), schema.isError());
result.setFields(fields);
return result;
}
case UNION: {
List<Schema> unionTypes = new ArrayList<Schema>();
for (org.apache.avro.Schema origType : schema.getTypes()) {
unionTypes.add(asCopycatSchema(origType));
}
return Schema.createUnion(unionTypes);
}
case ARRAY:
return Schema.createArray(asCopycatSchema(schema.getElementType()));
case ENUM:
return Schema.createEnum(schema.getName(), schema.getDoc(), schema
.getNamespace(), schema.getEnumSymbols());
case FIXED:
return Schema.createFixed(schema.getName(), schema.getDoc(), schema
.getNamespace(), schema.getFixedSize());
case MAP:
return Schema.createMap(asCopycatSchema(schema.getValueType()));
default:
throw new CopycatRuntimeException("Couldn't translate unsupported schema type "
+ schema.getType().getName() + ".");
}
}
}

View File

@ -0,0 +1,50 @@
/**
* 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.avro;
import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer;
import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig;
import org.apache.kafka.copycat.data.Schema;
import org.apache.kafka.copycat.storage.OffsetDeserializer;
import java.util.Map;
public class AvroDeserializer extends AbstractKafkaAvroDeserializer implements OffsetDeserializer<Object> {
@Override
public void configure(Map<String, ?> configs, boolean isKey) {
configure(new KafkaAvroDeserializerConfig(configs));
}
@Override
public Object deserializeOffset(String connector, byte[] data) {
// TODO: Support schema projection
return deserialize(data);
}
@Override
public Object deserializeOffset(String connector, byte[] data, Schema schema) {
// TODO: Support schema projection
return deserialize(data);
}
@Override
public void close() {
}
}

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
* <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.avro;
import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig;
import io.confluent.kafka.serializers.AbstractKafkaAvroSerializer;
import org.apache.kafka.copycat.errors.CopycatRuntimeException;
import org.apache.kafka.copycat.storage.OffsetSerializer;
import java.util.Map;
public class AvroSerializer extends AbstractKafkaAvroSerializer implements OffsetSerializer<Object> {
private boolean isKey;
@Override
public void configure(Map<String, ?> configs, boolean isKey) {
this.isKey = isKey;
Object url = configs.get(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG);
if (url == null) {
throw new CopycatRuntimeException("Missing Schema registry url!");
}
Object maxSchemaObject = configs.get(
AbstractKafkaAvroSerDeConfig.MAX_SCHEMAS_PER_SUBJECT_CONFIG);
if (maxSchemaObject == null) {
schemaRegistry = new CachedSchemaRegistryClient(
(String) url, AbstractKafkaAvroSerDeConfig.MAX_SCHEMAS_PER_SUBJECT_DEFAULT);
} else {
schemaRegistry = new CachedSchemaRegistryClient(
(String) url, (Integer) maxSchemaObject);
}
}
@Override
public byte[] serializeOffset(String connector, Object data) {
String subject;
if (isKey) {
subject = connector + "-key";
} else {
subject = connector + "-value";
}
return serializeImpl(subject, data);
}
@Override
public void close() {
}
}

View File

@ -0,0 +1,154 @@
/**
* 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.avro;
import org.apache.kafka.copycat.data.GenericRecord;
import org.apache.kafka.copycat.data.GenericRecordBuilder;
import org.apache.kafka.copycat.data.Schema;
import org.apache.kafka.copycat.data.SchemaBuilder;
import org.junit.Assert;
import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import static org.junit.Assert.*;
// Tests AvroData's conversion of Copycat -> Avro
public class AvroDataToAvroTest {
// All the primitive types are pass-through
@Test
public void testNull() {
assertNull(AvroData.convertToAvro(null));
}
@Test
public void testBoolean() {
Assert.assertEquals(true, AvroData.convertToAvro(true));
}
@Test
public void testInteger() {
Assert.assertEquals(12, AvroData.convertToAvro(12));
}
@Test
public void testLong() {
Assert.assertEquals(12L, AvroData.convertToAvro(12L));
}
@Test
public void testFloat() {
Assert.assertEquals(12.2f, AvroData.convertToAvro(12.2f));
}
@Test
public void testDouble() {
Assert.assertEquals(12.2, AvroData.convertToAvro(12.2));
}
@Test
public void testBytes() {
Object converted = AvroData.convertToAvro("foo".getBytes());
assertTrue(converted instanceof byte[]);
assertEquals(ByteBuffer.wrap("foo".getBytes()), ByteBuffer.wrap((byte[]) converted));
Assert.assertEquals(ByteBuffer.wrap("foo".getBytes()),
AvroData.convertToAvro(ByteBuffer.wrap("foo".getBytes())));
}
@Test
public void testString() {
Assert.assertEquals("string", AvroData.convertToAvro("string"));
}
@Test
public void testComplex() {
Schema schema = SchemaBuilder.record("record").fields()
.name("null").type().nullType().noDefault()
.requiredBoolean("boolean")
.requiredInt("int")
.requiredLong("long")
.requiredFloat("float")
.requiredDouble("double")
.requiredBytes("bytes")
.requiredString("string")
.name("union").type().unionOf().nullType().and().intType().endUnion().noDefault()
.name("array").type().array().items().intType().noDefault()
.name("map").type().map().values().intType().noDefault()
.name("fixed").type().fixed("fixed").size(3).noDefault()
.name("enum").type().enumeration("enum").symbols("one", "two").noDefault()
.endRecord();
GenericRecord record = new GenericRecordBuilder(schema)
.set("null", null)
.set("boolean", true)
.set("int", 12)
.set("long", 12L)
.set("float", 12.2f)
.set("double", 12.2)
.set("bytes", ByteBuffer.wrap("foo".getBytes()))
.set("string", "string-value")
.set("union", 12)
.set("array", Arrays.asList(1, 2, 3))
.set("map", Collections.singletonMap("field", 1))
.set("fixed", ByteBuffer.wrap("foo".getBytes()))
.set("enum", "one")
.build();
Object convertedRecord = AvroData.convertToAvro(record);
org.apache.avro.Schema avroSchema = org.apache.avro.SchemaBuilder.record("record").fields()
.name("null").type().nullType().noDefault()
.requiredBoolean("boolean")
.requiredInt("int")
.requiredLong("long")
.requiredFloat("float")
.requiredDouble("double")
.requiredBytes("bytes")
.requiredString("string")
.name("union").type().unionOf().nullType().and().intType().endUnion().noDefault()
.name("array").type().array().items().intType().noDefault()
.name("map").type().map().values().intType().noDefault()
.name("fixed").type().fixed("fixed").size(3).noDefault()
.name("enum").type().enumeration("enum").symbols("one", "two").noDefault()
.endRecord();
org.apache.avro.generic.GenericRecord avroRecord
= new org.apache.avro.generic.GenericRecordBuilder(avroSchema)
.set("null", null)
.set("boolean", true)
.set("int", 12)
.set("long", 12L)
.set("float", 12.2f)
.set("double", 12.2)
.set("bytes", ByteBuffer.wrap("foo".getBytes()))
.set("string", "string-value")
.set("union", 12)
.set("array", Arrays.asList(1, 2, 3))
.set("map", Collections.singletonMap("field", 1))
.set("fixed", ByteBuffer.wrap("foo".getBytes()))
.set("enum", "one")
.build();
assertEquals(avroSchema, ((org.apache.avro.generic.GenericRecord) convertedRecord).getSchema());
assertEquals(avroRecord, convertedRecord);
}
}

View File

@ -0,0 +1,154 @@
/**
* 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.avro;
import org.apache.kafka.copycat.data.GenericRecord;
import org.apache.kafka.copycat.data.GenericRecordBuilder;
import org.apache.kafka.copycat.data.Schema;
import org.apache.kafka.copycat.data.SchemaBuilder;
import org.junit.Assert;
import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import static org.junit.Assert.*;
// Tests AvroData's conversion of Avro -> Copycat
public class AvroDataToCopycatTest {
// All the primitive types are pass-through
@Test
public void testNull() {
assertNull(AvroData.convertFromAvro(null));
}
@Test
public void testBoolean() {
Assert.assertEquals(true, AvroData.convertFromAvro(true));
}
@Test
public void testInteger() {
Assert.assertEquals(12, AvroData.convertFromAvro(12));
}
@Test
public void testLong() {
Assert.assertEquals(12L, AvroData.convertFromAvro(12L));
}
@Test
public void testFloat() {
Assert.assertEquals(12.2f, AvroData.convertFromAvro(12.2f));
}
@Test
public void testDouble() {
Assert.assertEquals(12.2, AvroData.convertFromAvro(12.2));
}
@Test
public void testBytes() {
Object converted = AvroData.convertFromAvro("foo".getBytes());
assertTrue(converted instanceof byte[]);
assertEquals(ByteBuffer.wrap("foo".getBytes()), ByteBuffer.wrap((byte[]) converted));
Assert.assertEquals(ByteBuffer.wrap("foo".getBytes()),
AvroData.convertFromAvro(ByteBuffer.wrap("foo".getBytes())));
}
@Test
public void testString() {
Assert.assertEquals("string", AvroData.convertFromAvro("string"));
}
@Test
public void testComplex() {
org.apache.avro.Schema avroSchema = org.apache.avro.SchemaBuilder.record("record").fields()
.name("null").type().nullType().noDefault()
.requiredBoolean("boolean")
.requiredInt("int")
.requiredLong("long")
.requiredFloat("float")
.requiredDouble("double")
.requiredBytes("bytes")
.requiredString("string")
.name("union").type().unionOf().nullType().and().intType().endUnion().noDefault()
.name("array").type().array().items().intType().noDefault()
.name("map").type().map().values().intType().noDefault()
.name("fixed").type().fixed("fixed").size(3).noDefault()
.name("enum").type().enumeration("enum").symbols("one", "two").noDefault()
.endRecord();
org.apache.avro.generic.GenericRecord avroRecord
= new org.apache.avro.generic.GenericRecordBuilder(avroSchema)
.set("null", null)
.set("boolean", true)
.set("int", 12)
.set("long", 12L)
.set("float", 12.2f)
.set("double", 12.2)
.set("bytes", ByteBuffer.wrap("foo".getBytes()))
.set("string", "string-value")
.set("union", 12)
.set("array", Arrays.asList(1, 2, 3))
.set("map", Collections.singletonMap("field", 1))
.set("fixed", ByteBuffer.wrap("foo".getBytes()))
.set("enum", "one")
.build();
Object convertedRecord = AvroData.convertFromAvro(avroRecord);
Schema schema = SchemaBuilder.record("record").fields()
.name("null").type().nullType().noDefault()
.requiredBoolean("boolean")
.requiredInt("int")
.requiredLong("long")
.requiredFloat("float")
.requiredDouble("double")
.requiredBytes("bytes")
.requiredString("string")
.name("union").type().unionOf().nullType().and().intType().endUnion().noDefault()
.name("array").type().array().items().intType().noDefault()
.name("map").type().map().values().intType().noDefault()
.name("fixed").type().fixed("fixed").size(3).noDefault()
.name("enum").type().enumeration("enum").symbols("one", "two").noDefault()
.endRecord();
GenericRecord record = new GenericRecordBuilder(schema)
.set("null", null)
.set("boolean", true)
.set("int", 12)
.set("long", 12L)
.set("float", 12.2f)
.set("double", 12.2)
.set("bytes", ByteBuffer.wrap("foo".getBytes()))
.set("string", "string-value")
.set("union", 12)
.set("array", Arrays.asList(1, 2, 3))
.set("map", Collections.singletonMap("field", 1))
.set("fixed", ByteBuffer.wrap("foo".getBytes()))
.set("enum", "one")
.build();
assertEquals(schema, ((GenericRecord) convertedRecord).getSchema());
assertEquals(record, convertedRecord);
}
}

View File

@ -0,0 +1,229 @@
/**
* 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.avro;
/*
import org.apache.kafka.copycat.data.Schema;
import org.apache.kafka.copycat.data.SchemaBuilder;
import org.apache.kafka.copycat.errors.CopycatRuntimeException;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.junit.Before;
import org.junit.Test;
import org.powermock.api.easymock.PowerMock;
import java.nio.ByteBuffer;
import java.util.*;
import java.util.concurrent.Future;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
*/
public class AvroOffsetStorageReaderTest {
/* FIXME
private static final String namespace = "namespace";
private OffsetBackingStore offsetBackingStore;
private OffsetStorageReaderImpl offsetReader;
// Most requests will make a request to the underlying storage and want to verify some info
// about that request
private Capture<Collection<ByteBuffer>> requestKeys;
private Capture<Callback<Map<ByteBuffer, ByteBuffer>>> callback;
private Future<Map<ByteBuffer, ByteBuffer>> storeFuture;
// Cover a few different types to verify serialization works and that for keys we're allowed to
// mix-and-match within a single request
// TODO Add testing of complex types for stream IDs & offsets to these primitive types
private static final List<Object> streamIds = Arrays.asList((Object) "stream1", (Object) 52);
private static final Object nullStreamId = null;
// Same as streamIds but with extra, unserializable entry
private static final List<Object> streamIdsWithUnserializable
= Arrays.asList((Object) "stream1", (Object) 52, new Date());
private static final List<Object> longOffsets = Arrays.asList((Object) 12L, (Object) 24L);
private static final List<Object> stringOffsets = Arrays.asList(
(Object) "offset1", (Object) "offset2");
private static final Schema longSchema = SchemaBuilder.builder().longType();
private static final Schema stringSchema = SchemaBuilder.builder().stringType();
// Serialized form of data to be returned by the storage layer
private static final Map<ByteBuffer, ByteBuffer> longsSerialized
= new HashMap<ByteBuffer, ByteBuffer>();
private static final Map<ByteBuffer, ByteBuffer> stringsSerialized
= new HashMap<ByteBuffer, ByteBuffer>();
private static final Map<ByteBuffer, ByteBuffer> singleLongSerialized
= new HashMap<ByteBuffer, ByteBuffer>();
private static final Map<ByteBuffer, ByteBuffer> nullsSerialized
= new HashMap<ByteBuffer, ByteBuffer>();
private static final Map<ByteBuffer, ByteBuffer> longsSerializedWithInvalid
= new HashMap<ByteBuffer, ByteBuffer>();
static {
for (int i = 0; i < longOffsets.size(); i++) {
longsSerialized.put(
AvroData.serializeToAvro(streamIds.get(i)),
AvroData.serializeToAvro(longOffsets.get(i)));
}
for (int i = 0; i < stringOffsets.size(); i++) {
stringsSerialized.put(
AvroData.serializeToAvro(streamIds.get(i)),
AvroData.serializeToAvro(stringOffsets.get(i)));
}
singleLongSerialized.put(
AvroData.serializeToAvro(streamIds.get(0)),
AvroData.serializeToAvro(longOffsets.get(0)));
nullsSerialized.put(null, null);
longsSerializedWithInvalid.put(
AvroData.serializeToAvro(streamIds.get(0)),
AvroData.serializeToAvro(longOffsets.get(0)));
// You need to be careful about the bytes specified here because Avro's variable length
// encoding allows some short byte sequences to be valid values for a long
longsSerializedWithInvalid.put(
AvroData.serializeToAvro(streamIds.get(1)),
ByteBuffer.wrap(new byte[0]));
}
@Before
public void setup() {
offsetBackingStore = PowerMock.createMock(OffsetBackingStore.class);
offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, namespace);
requestKeys = EasyMock.newCapture();
callback = EasyMock.newCapture();
storeFuture = PowerMock.createMock(Future.class);
}
@Test
public void testGetOffsetsStringValues() throws Exception {
expectStoreRequest(stringsSerialized);
PowerMock.replayAll();
Map<Object, Object> result = offsetReader.getOffsets(streamIds, stringSchema);
assertEquals(2, requestKeys.getValue().size());
assertEquals(mapFromLists(streamIds, stringOffsets), result);
PowerMock.verifyAll();
}
@Test
public void testGetOffsetsLongValues() throws Exception {
expectStoreRequest(longsSerialized);
PowerMock.replayAll();
Map<Object, Object> result = offsetReader.getOffsets(streamIds, longSchema);
assertEquals(2, requestKeys.getValue().size());
assertEquals(mapFromLists(streamIds, longOffsets), result);
PowerMock.verifyAll();
}
// getOffset() isn't too interesting since it's just a degenerate form of getOffsets(), so we
// just do one simple validation
@Test
public void testGetOffset() throws Exception {
expectStoreRequest(singleLongSerialized);
PowerMock.replayAll();
Object result = offsetReader.getOffset(streamIds.get(0), longSchema);
assertEquals(1, requestKeys.getValue().size());
assertEquals(longOffsets.get(0), result);
PowerMock.verifyAll();
}
@Test
public void testGetOffsetsNulls() throws Exception {
// Should be able to store/load null values
expectStoreRequest(nullsSerialized);
PowerMock.replayAll();
Object result = offsetReader.getOffset(null, longSchema);
assertEquals(1, requestKeys.getValue().size());
assertNull(result);
PowerMock.verifyAll();
}
@Test
public void testSerializationErrorReturnsOtherResults() throws Exception {
expectStoreRequest(longsSerialized);
PowerMock.replayAll();
Map<Object, Object> result = offsetReader.getOffsets(streamIdsWithUnserializable, longSchema);
assertEquals(2, requestKeys.getValue().size());
assertEquals(mapFromLists(streamIds, longOffsets), result);
PowerMock.verifyAll();
}
@Test(expected = CopycatRuntimeException.class)
public void testStorageGetFailed() throws Exception {
// backing store failed -> CopycatRuntimeException
EasyMock.expect(offsetBackingStore.get(EasyMock.eq(namespace), EasyMock.capture(requestKeys),
EasyMock.capture(callback)))
.andReturn(storeFuture);
EasyMock.expect(storeFuture.get())
.andThrow(new RuntimeException("failed to get data from backing store"));
PowerMock.replayAll();
Map<Object, Object> result = offsetReader.getOffsets(streamIds, longSchema);
assertEquals(2, requestKeys.getValue().size()); // throws
}
@Test
public void testUndeserializeableData() throws Exception {
// Should return whatever data it can, ignoring the unserializeable data
expectStoreRequest(longsSerializedWithInvalid);
PowerMock.replayAll();
Map<Object, Object> result = offsetReader.getOffsets(streamIds, longSchema);
assertEquals(2, requestKeys.getValue().size());
assertEquals(mapFromLists(streamIds.subList(0, 1), longOffsets.subList(0, 1)), result);
PowerMock.verifyAll();
}
private void expectStoreRequest(Map<ByteBuffer, ByteBuffer> result) throws Exception {
EasyMock.expect(offsetBackingStore.get(EasyMock.eq(namespace), EasyMock.capture(requestKeys),
EasyMock.capture(callback)))
.andReturn(storeFuture);
EasyMock.expect(storeFuture.get()).andReturn(result);
}
private Map<Object, Object> mapFromLists(List<Object> keys, List<Object> values) {
Map<Object, Object> result = new HashMap<Object, Object>();
for (int i = 0; i < keys.size(); i++) {
result.put(keys.get(i), values.get(i));
}
return result;
}
*/
}

View File

@ -0,0 +1,110 @@
/**
* 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.config.ConfigException;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.copycat.runtime.Coordinator;
import org.apache.kafka.copycat.runtime.Copycat;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.runtime.standalone.StandaloneCoordinator;
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.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>
*/
public class CopycatCommand {
private static final Logger log = LoggerFactory.getLogger(CopycatCommand.class);
public static void main(String[] args) throws Exception {
CopycatCommandConfig config;
Properties workerProps;
Properties connectorProps;
try {
config = CopycatCommandConfig.parseCommandLineArgs(args);
} catch (ConfigException e) {
log.error(e.getMessage());
log.info("Usage: copycat [--worker-config worker.properties]"
+ " [--create-connectors connector1.properties,connector2.properties,...]"
+ " [--delete-connectors connector1-name,connector2-name,...]");
System.exit(1);
return;
}
String workerPropsFile = config.getString(CopycatCommandConfig.WORKER_PROPERTIES_CONFIG);
workerProps = !workerPropsFile.isEmpty() ? Utils.loadProps(workerPropsFile) : new Properties();
WorkerConfig workerConfig = new WorkerConfig(workerProps);
Worker worker = new Worker(workerConfig);
Coordinator coordinator = new StandaloneCoordinator(worker, workerConfig.getUnusedProperties());
final Copycat copycat = new Copycat(worker, coordinator);
copycat.start();
try {
// Destroy any requested connectors
for (final String connName : config.getList(CopycatCommandConfig.DELETE_CONNECTORS_CONFIG)) {
FutureCallback cb = new FutureCallback(new Callback<Void>() {
@Override
public void onCompletion(Throwable error, Void result) {
if (error != null) {
log.error("Failed to stop job {}", connName);
}
}
});
coordinator.deleteConnector(connName, cb);
cb.get();
}
// Create any new connectors
for (final String connectorPropsFile : config
.getList(CopycatCommandConfig.CREATE_CONNECTORS_CONFIG)) {
connectorProps = Utils.loadProps(connectorPropsFile);
FutureCallback 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);
}
}
});
coordinator.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,96 @@
/**
* 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.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;
public class CopycatCommandConfig extends AbstractConfig {
public static final String WORKER_PROPERTIES_CONFIG = "worker-config";
public static final String WORKER_PROPERTIES_CONFIG_DEFAULT = "";
private static final String WORKER_PROPERTIES_CONFIG_DOC =
"Path to a properties file with worker configuration.";
public static final String CREATE_CONNECTORS_CONFIG = "create-connectors";
public static final String CREATE_CONNECTORS_CONFIG_DEFAULT = "";
private static final String CREATE_CONNECTORS_CONFIG_DOC =
"List of paths to properties files with connector properties to use to create new connectors";
public static final String DELETE_CONNECTORS_CONFIG = "delete-connectors";
public static final String DELETE_CONNECTORS_CONFIG_DEFAULT = "";
private static final String DELETE_CONNECTORS_CONFIG_DOC = "List of names of a connectors to "
+ "stop and delete.";
private static ConfigDef config;
static {
config = new ConfigDef()
.define(WORKER_PROPERTIES_CONFIG, Type.STRING, WORKER_PROPERTIES_CONFIG_DEFAULT,
Importance.HIGH, WORKER_PROPERTIES_CONFIG_DOC)
.define(CREATE_CONNECTORS_CONFIG, Type.LIST, CREATE_CONNECTORS_CONFIG_DEFAULT,
Importance.HIGH, CREATE_CONNECTORS_CONFIG_DOC)
.define(DELETE_CONNECTORS_CONFIG, Type.LIST, DELETE_CONNECTORS_CONFIG_DEFAULT,
Importance.HIGH, DELETE_CONNECTORS_CONFIG_DOC);
}
private Properties originalProperties;
public CopycatCommandConfig(Properties props) {
super(config, props);
this.originalProperties = props;
}
/**
* Parses command line arguments into a Properties object and instantiate a
* CopycatCommandConfig with it.
* @param args
* @return
*/
public static CopycatCommandConfig parseCommandLineArgs(String[] args) {
Properties props = new Properties();
for (int i = 0; i < args.length; i++) {
String arg = args[i];
String key, value;
// Check for foo=bar or --foo=bar syntax
if (arg.contains("=")) {
String[] parts = arg.split("=", 1);
key = parts[0];
value = parts[1];
} else {
key = args[i];
i += 1;
value = args[i];
}
// Check for -- prefix on key
if (key.startsWith("--")) {
key = key.substring(2);
}
props.setProperty(key, value);
}
return new CopycatCommandConfig(props);
}
}

View File

@ -0,0 +1,227 @@
/**
* 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.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.
*/
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 ZOOKEEPER_CONNECT_CONFIG = "zookeeper.connect";
private static final String ZOOKEEPER_CONNECT_DOC =
"Specifies the ZooKeeper connection string in the form "
+ "hostname:port where host and port are the host and port of a ZooKeeper server. To allow connecting "
+ "through other ZooKeeper nodes when that ZooKeeper machine is down you can also specify multiple hosts "
+ "in the form hostname1:port1,hostname2:port2,hostname3:port3.\n"
+ "\n"
+ "The server may also have a ZooKeeper chroot path as part of it's ZooKeeper connection string which puts "
+ "its data under some path in the global ZooKeeper namespace. If so the consumer should use the same "
+ "chroot path in its connection string. For example to give a chroot path of /chroot/path you would give "
+ "the connection string as hostname1:port1,hostname2:port2,hostname3:port3/chroot/path.";
public static final String ZOOKEEPER_CONNECT_DEFAULT = "localhost:2181";
public static final String ZOOKEEPER_SESSION_TIMEOUT_MS_CONFIG = "zookeeper.session.timeout.ms";
private static final String ZOOKEEPER_SESSION_TIMEOUT_MS_DOC
= "Session timeout for ZooKeeper connections.";
public static final String ZOOKEEPER_SESSION_TIMEOUT_MS_DEFAULT = "30000";
public static final String ZOOKEEPER_CONNECTION_TIMEOUT_MS_CONFIG
= "zookeeper.session.connection.ms";
private static final String ZOOKEEPER_CONNECTION_TIMEOUT_MS_DOC
= "Connection timeout for ZooKeeper.";
public static final String ZOOKEEPER_CONNECTION_TIMEOUT_MS_DEFAULT = "30000";
public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
public static final String BOOSTRAP_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 CONVERTER_CLASS_CONFIG = "converter";
public static final String CONVERTER_CLASS_DOC =
"Converter class for Copycat data that implements the <code>Converter</code> interface.";
public static final String CONVERTER_CLASS_DEFAULT
= "org.apache.kafka.copycat.avro.AvroConverter"; // TODO: Non-avro built-in?
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 KEY_SERIALIZER_CLASS_DEFAULT
= "io.confluent.kafka.serializers.KafkaAvroSerializer"; // TODO: Non-avro built-in?
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 VALUE_SERIALIZER_CLASS_DEFAULT
= "io.confluent.kafka.serializers.KafkaAvroSerializer"; // TODO: Non-avro built-in?
public static final String OFFSET_KEY_SERIALIZER_CLASS_CONFIG = "offset.key.serializer";
public static final String OFFSET_KEY_SERIALIZER_CLASS_DOC =
"Serializer class for key that implements the <code>OffsetSerializer</code> interface.";
public static final String OFFSET_KEY_SERIALIZER_CLASS_DEFAULT
= "org.apache.kafka.copycat.avro.AvroSerializer"; // TODO: Non-avro built-in?
public static final String OFFSET_VALUE_SERIALIZER_CLASS_CONFIG = "offset.value.serializer";
public static final String OFFSET_VALUE_SERIALIZER_CLASS_DOC =
"Serializer class for value that implements the <code>OffsetSerializer</code> interface.";
public static final String OFFSET_VALUE_SERIALIZER_CLASS_DEFAULT
= "org.apache.kafka.copycat.avro.AvroSerializer"; // TODO: Non-avro built-in?
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 KEY_DESERIALIZER_CLASS_DEFAULT
= "io.confluent.kafka.serializers.KafkaAvroDeserializer"; // TODO: Non-avro built-in?
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 VALUE_DESERIALIZER_CLASS_DEFAULT
= "io.confluent.kafka.serializers.KafkaAvroDeserializer"; // TODO: Non-avro built-in?
public static final String OFFSET_KEY_DESERIALIZER_CLASS_CONFIG = "offset.key.deserializer";
public static final String OFFSET_KEY_DESERIALIZER_CLASS_DOC =
"Deserializer class for key that implements the <code>OffsetDeserializer</code> interface.";
public static final String OFFSET_KEY_DESERIALIZER_CLASS_DEFAULT
= "org.apache.kafka.copycat.avro.AvroDeserializer"; // TODO: Non-avro built-in?
public static final String OFFSET_VALUE_DESERIALIZER_CLASS_CONFIG = "offset.value.deserializer";
public static final String OFFSET_VALUE_DESERIALIZER_CLASS_DOC =
"Deserializer class for value that implements the <code>OffsetDeserializer</code> interface.";
public static final String OFFSET_VALUE_DESERIALIZER_CLASS_DEFAULT
= "org.apache.kafka.copycat.avro.AvroDeserializer"; // TODO: Non-avro built-in?
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_STORAGE_CLASS_CONFIG = "offset.storage.class";
private static final String OFFSET_STORAGE_CLASS_DOC =
"OffsetBackingStore implementation to use for storing stream offset data";
public static final String OFFSET_STORAGE_CLASS_DEFAULT
= "org.apache.kafka.copycat.storage.MemoryOffsetBackingStore";
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 stream 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(ZOOKEEPER_CONNECT_CONFIG, Type.STRING, ZOOKEEPER_CONNECT_DEFAULT,
Importance.HIGH, ZOOKEEPER_CONNECT_DOC)
.define(ZOOKEEPER_SESSION_TIMEOUT_MS_CONFIG, Type.INT,
ZOOKEEPER_SESSION_TIMEOUT_MS_DEFAULT,
Importance.LOW, ZOOKEEPER_SESSION_TIMEOUT_MS_DOC)
.define(ZOOKEEPER_CONNECTION_TIMEOUT_MS_CONFIG, Type.INT,
ZOOKEEPER_CONNECTION_TIMEOUT_MS_DEFAULT,
Importance.LOW, ZOOKEEPER_CONNECTION_TIMEOUT_MS_DOC)
.define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, BOOTSTRAP_SERVERS_DEFAULT,
Importance.HIGH, BOOSTRAP_SERVERS_DOC)
.define(CONVERTER_CLASS_CONFIG, Type.CLASS, CONVERTER_CLASS_DEFAULT,
Importance.HIGH, CONVERTER_CLASS_DOC)
.define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, KEY_SERIALIZER_CLASS_DEFAULT,
Importance.HIGH, KEY_SERIALIZER_CLASS_DOC)
.define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, VALUE_SERIALIZER_CLASS_DEFAULT,
Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC)
.define(OFFSET_KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, OFFSET_KEY_SERIALIZER_CLASS_DEFAULT,
Importance.HIGH, OFFSET_KEY_SERIALIZER_CLASS_DOC)
.define(OFFSET_VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS,
OFFSET_VALUE_SERIALIZER_CLASS_DEFAULT,
Importance.HIGH, OFFSET_VALUE_SERIALIZER_CLASS_DOC)
.define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS, KEY_DESERIALIZER_CLASS_DEFAULT,
Importance.HIGH, KEY_DESERIALIZER_CLASS_DOC)
.define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, VALUE_DESERIALIZER_CLASS_DEFAULT,
Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC)
.define(OFFSET_KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS,
OFFSET_KEY_DESERIALIZER_CLASS_DEFAULT,
Importance.HIGH, OFFSET_KEY_DESERIALIZER_CLASS_DOC)
.define(OFFSET_VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS,
OFFSET_VALUE_DESERIALIZER_CLASS_DEFAULT,
Importance.HIGH, OFFSET_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_STORAGE_CLASS_CONFIG, Type.CLASS, OFFSET_STORAGE_CLASS_DEFAULT,
Importance.LOW, OFFSET_STORAGE_CLASS_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);
}
private Properties originalProperties;
public WorkerConfig() {
this(new Properties());
}
public WorkerConfig(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;
}
public Properties getOriginalProperties() {
return originalProperties;
}
}

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,52 @@
/**
* 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;
/**
* The Coordinator interface works with coordinators on other workers to manage a set of Jobs.
* Each job is a Connector instance with associated config and triggers tasks that are then run
* in workers. The assignment and tracking of those tasks in workers is also managed by the
* coordinator.
*/
public interface Coordinator {
void start();
void stop();
/**
* Submit a connector job to the cluster. This works from any node by forwarding the request to
* the leader coordinator 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,92 @@
/**
* 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.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 (coordinator, worker,
* storage, command interface), managing their lifecycle.
*/
public class Copycat {
private static final Logger log = LoggerFactory.getLogger(Copycat.class);
private final Worker worker;
private final Coordinator coordinator;
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, Coordinator coordinator) {
log.debug("Copycat created");
this.worker = worker;
this.coordinator = coordinator;
shutdownHook = new ShutdownHook();
}
public void start() {
log.info("Copycat starting");
Runtime.getRuntime().addShutdownHook(shutdownHook);
worker.start();
coordinator.start();
log.info("Copycat started");
startLatch.countDown();
}
public void stop() {
boolean wasShuttingDown = shutdown.getAndSet(true);
if (!wasShuttingDown) {
log.info("Copycat stopping");
coordinator.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,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;
public class SinkTaskContextImpl extends SinkTaskContext {
}

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
* <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>
*/
public 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<?>> commiters
= new HashMap<ConnectorTaskId, ScheduledFuture<?>>();
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);
commiters.put(id, commitFuture);
}
public void remove(ConnectorTaskId id) {
ScheduledFuture<?> commitFuture = commiters.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,261 @@
/**
* 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.common.utils.Time;
import kafka.utils.ZKStringSerializer$;
import org.I0Itec.zkclient.ZkClient;
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.errors.CopycatRuntimeException;
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.apache.kafka.copycat.util.Reflection;
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 {
private static final Logger log = LoggerFactory.getLogger(Worker.class);
private Time time;
private WorkerConfig config;
private Converter converter;
private OffsetBackingStore offsetBackingStore;
private OffsetSerializer offsetKeySerializer;
private OffsetSerializer offsetValueSerializer;
private OffsetDeserializer offsetKeyDeserializer;
private OffsetDeserializer offsetValueDeserializer;
private HashMap<ConnectorTaskId, WorkerTask> tasks = new HashMap<ConnectorTaskId, WorkerTask>();
private KafkaProducer producer;
private ZkClient zkClient;
private SourceTaskOffsetCommitter sourceTaskOffsetCommitter;
public Worker(WorkerConfig config) {
this(new SystemTime(), config,
Reflection.instantiateConfigurable(
config.getClass(WorkerConfig.OFFSET_STORAGE_CLASS_CONFIG).getName(),
OffsetBackingStore.class, config.getUnusedProperties()),
null, null, null, null,
createZkClient(config));
}
public Worker(Time time, WorkerConfig config, OffsetBackingStore offsetBackingStore,
OffsetSerializer offsetKeySerializer, OffsetSerializer offsetValueSerializer,
OffsetDeserializer offsetKeyDeserializer, OffsetDeserializer offsetValueDeserializer,
ZkClient zkClient) {
this.time = time;
this.config = config;
this.converter = Reflection.instantiate(config.getClass(WorkerConfig.CONVERTER_CLASS_CONFIG).getName(),
Converter.class);
this.offsetBackingStore = offsetBackingStore;
if (offsetKeySerializer != null) {
this.offsetKeySerializer = offsetKeySerializer;
} else {
this.offsetKeySerializer = Reflection.instantiate(
config.getClass(WorkerConfig.OFFSET_KEY_SERIALIZER_CLASS_CONFIG).getName(),
OffsetSerializer.class);
this.offsetKeySerializer.configure(config.getOriginalProperties(), true);
}
if (offsetValueSerializer != null) {
this.offsetValueSerializer = offsetValueSerializer;
} else {
this.offsetValueSerializer = Reflection.instantiate(
config.getClass(WorkerConfig.OFFSET_VALUE_SERIALIZER_CLASS_CONFIG).getName(),
OffsetSerializer.class);
this.offsetValueSerializer.configure(config.getOriginalProperties(), false);
}
if (offsetKeyDeserializer != null) {
this.offsetKeyDeserializer = offsetKeyDeserializer;
} else {
this.offsetKeyDeserializer = Reflection.instantiate(
config.getClass(WorkerConfig.OFFSET_KEY_DESERIALIZER_CLASS_CONFIG).getName(),
OffsetDeserializer.class);
this.offsetKeyDeserializer.configure(config.getOriginalProperties(), true);
}
if (offsetValueDeserializer != null) {
this.offsetValueDeserializer = offsetValueDeserializer;
} else {
this.offsetValueDeserializer = Reflection.instantiate(
config.getClass(WorkerConfig.OFFSET_VALUE_DESERIALIZER_CLASS_CONFIG).getName(),
OffsetDeserializer.class);
this.offsetValueDeserializer.configure(config.getOriginalProperties(), false);
}
this.zkClient = zkClient;
}
private static ZkClient createZkClient(WorkerConfig config) {
return new ZkClient(config.getString(WorkerConfig.ZOOKEEPER_CONNECT_CONFIG),
config.getInt(WorkerConfig.ZOOKEEPER_SESSION_TIMEOUT_MS_CONFIG),
config.getInt(WorkerConfig.ZOOKEEPER_CONNECTION_TIMEOUT_MS_CONFIG),
ZKStringSerializer$.MODULE$);
}
public void start() {
log.info("Worker starting");
Properties unusedConfigs = config.getUnusedProperties();
Map<String, Object> avroProps = new HashMap<String, Object>();
avroProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, Utils.join(config.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ","));
avroProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, config.getClass(WorkerConfig.KEY_SERIALIZER_CLASS_CONFIG).getName());
avroProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, config.getClass(WorkerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName());
for (String propName : unusedConfigs.stringPropertyNames()) {
avroProps.put(propName, unusedConfigs.getProperty(propName));
}
producer = new KafkaProducer<Object, Object>(avroProps);
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; coordinator 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)
throws CopycatException {
if (tasks.containsKey(id)) {
String msg = "Task already exists in this worker; the coordinator should not have requested "
+ "that this : " + id;
log.error(msg);
throw new CopycatRuntimeException(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(), converter,
offsetKeySerializer, offsetValueDeserializer);
OffsetStorageWriter offsetWriter
= new OffsetStorageWriter(offsetBackingStore, id.getConnector(), converter,
offsetKeySerializer, offsetValueSerializer);
workerTask = new WorkerSourceTask(id, sourceTask, converter, producer,
offsetReader, offsetWriter,
config, time);
} else if (task instanceof SinkTask) {
workerTask = new WorkerSinkTask(id, (SinkTask) task, config, converter, 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) {
return Reflection.instantiate(taskClassName, Task.class);
}
public void stopTask(ConnectorTaskId id) throws CopycatException {
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);
}
public ZkClient getZkClient() {
return zkClient;
}
private WorkerTask getTask(ConnectorTaskId id) {
WorkerTask task = tasks.get(id);
if (task == null) {
log.error("Task not found: " + id);
throw new CopycatRuntimeException();
}
return task;
}
}

View File

@ -0,0 +1,240 @@
/**
* 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.utils.Utils;
import org.apache.kafka.copycat.cli.WorkerConfig;
import org.apache.kafka.copycat.connector.TopicPartition;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.errors.CopycatRuntimeException;
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.
*/
public class WorkerSinkTask 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 converter;
private WorkerSinkTaskThread workThread;
private KafkaConsumer<Object, Object> consumer;
private final SinkTaskContext context;
public WorkerSinkTask(ConnectorTaskId id, SinkTask task, WorkerConfig workerConfig,
Converter converter, Time time) {
this.id = id;
this.task = task;
this.workerConfig = workerConfig;
this.converter = converter;
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() throws CopycatException {
task.stop();
commitOffsets(time.milliseconds(), true, -1, false);
if (workThread != null) {
workThread.startGracefulShutdown();
}
// Closing the consumer has to wait until we're sure the work thread has exited so it won't
// call poll() anymore.
}
@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) {
log.trace("{} polling consumer with timeout {} ms", id, timeoutMs);
ConsumerRecords<Object, Object> msgs = consumer.poll(timeoutMs);
log.trace("{} polling returned {} messages", id, msgs.count());
deliverMessages(msgs);
}
/**
* 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) {
// Because of the different representations, we need to build two copies of the same map
HashMap<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>();
HashMap<org.apache.kafka.common.TopicPartition, Long> offsetsKafka
= new HashMap<org.apache.kafka.common.TopicPartition, Long>();
for (org.apache.kafka.common.TopicPartition tp : consumer.subscriptions()) {
long pos = consumer.position(tp);
offsets.put(new TopicPartition(tp.topic(), tp.partition()), pos);
offsetsKafka.put(tp, pos);
}
// 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<org.apache.kafka.common.TopicPartition, Long> offsets, Exception error) {
workThread.onCommitCompleted(error, seqno);
}
};
consumer.commit(offsetsKafka, sync ? CommitType.SYNC : CommitType.ASYNC, cb);
}
public Time getTime() {
return time;
}
public WorkerConfig getWorkerConfig() {
return workerConfig;
}
private KafkaConsumer<Object, Object> createConsumer(Properties taskProps) {
// 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<Object, Object> newConsumer;
try {
newConsumer = new KafkaConsumer<Object, Object>(props);
} catch (Throwable t) {
throw new CopycatRuntimeException("Failed to create consumer", t);
}
List<TopicPartition> topicPartitions = getInputTopicPartitions(taskProps);
log.debug("Task {} subscribing to topic-partitions {}", id, topicPartitions);
Map<TopicPartition, Long> offsets = context.getOffsets();
for (TopicPartition tp : topicPartitions) {
org.apache.kafka.common.TopicPartition kafkatp = new
org.apache.kafka.common.TopicPartition(tp.topic(), tp.partition());
newConsumer.subscribe(kafkatp);
if (offsets.containsKey(tp)) {
long offset = offsets.get(tp);
newConsumer.seek(kafkatp, offset);
}
}
return newConsumer;
}
private WorkerSinkTaskThread createWorkerThread() {
return new WorkerSinkTaskThread(this, "WorkerSinkTask-" + id, time, workerConfig);
}
private List<TopicPartition> getInputTopicPartitions(Properties taskProps) {
String topicPartitionsStr = taskProps.getProperty(SinkTask.TOPICPARTITIONS_CONFIG);
if (topicPartitionsStr == null || topicPartitionsStr.isEmpty()) {
throw new CopycatRuntimeException("Sink tasks require a list of topic partitions, which "
+ "copycat should generate automatically. This might "
+ "indicate your Task class inherits from SinkTask, but "
+ "your Connector class does not inherit from "
+ "SinkConnector.");
}
List<TopicPartition> topicPartitions = new ArrayList<TopicPartition>();
for (String topicPartitionStr : Arrays.asList(topicPartitionsStr.split(","))) {
topicPartitions.add(new TopicPartition(topicPartitionStr));
}
return topicPartitions;
}
private void deliverMessages(ConsumerRecords<Object, Object> msgs) {
// Finally, deliver this batch to the sink
if (msgs.count() > 0) {
List<SinkRecord> records = new ArrayList<SinkRecord>();
for (ConsumerRecord<Object, Object> msg : msgs) {
log.trace("Consuming message with key {}, value {}", msg.key(), msg.value());
records.add(
new SinkRecord(msg.topic(), msg.partition(),
converter.toCopycatData(msg.key()),
converter.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,110 @@
/**
* 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();
}
}
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,316 @@
/**
* 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.errors.CopycatException;
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.
*/
public class WorkerSourceTask implements WorkerTask {
private static final Logger log = LoggerFactory.getLogger(WorkerSourceTask.class);
private ConnectorTaskId id;
private SourceTask task;
private final Converter converter;
private KafkaProducer<Object, Object> producer;
private WorkerSourceTaskThread workThread;
private OffsetStorageReader offsetReader;
private OffsetStorageWriter 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<Object, Object>, ProducerRecord<Object, Object>>
outstandingMessages;
// A second buffer is used while an offset flush is running
private IdentityHashMap<ProducerRecord<Object, Object>, ProducerRecord<Object, Object>>
outstandingMessagesBacklog;
private boolean flushing;
public WorkerSourceTask(ConnectorTaskId id, SourceTask task,
Converter converter,
KafkaProducer<Object, Object> producer,
OffsetStorageReader offsetReader, OffsetStorageWriter offsetWriter,
WorkerConfig workerConfig, Time time) {
this.id = id;
this.task = task;
this.converter = converter;
this.producer = producer;
this.offsetReader = offsetReader;
this.offsetWriter = offsetWriter;
this.workerConfig = workerConfig;
this.time = time;
this.outstandingMessages
= new IdentityHashMap<ProducerRecord<Object, Object>, ProducerRecord<Object, Object>>();
this.outstandingMessagesBacklog
= new IdentityHashMap<ProducerRecord<Object, Object>, ProducerRecord<Object, Object>>();
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() throws CopycatException {
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<Object, Object> producerRecord
= new ProducerRecord<Object, Object>(record.getTopic(), record.getPartition(),
converter.fromCopycatData(record.getKey()),
converter.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 to Avro & serialized in the OffsetWriter
offsetWriter.setOffset(record.getStream(), record.getOffset());
}
}
private synchronized void recordSent(final ProducerRecord<Object, Object> record) {
ProducerRecord<Object, Object> 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<Object, Object>, ProducerRecord<Object, Object>> 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,58 @@
/**
* 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.errors.CopycatException;
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.
*/
public 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.
*
* @throws CopycatException
*/
void stop() throws CopycatException;
/**
* 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,60 @@
/**
* 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 java.util.Collection;
import java.util.Properties;
/**
* Interface used by StandaloneController to store configuration data for jobs. To be fault
* tolerant, all data required to resume jobs is stored here.
*/
public interface ConfigStorage {
/**
* Configure this storage engine.
* @param props configuration properties
*/
void configure(Properties props);
/**
* Close this storage engine.
*/
void close();
/**
* Commit the new connector config.
* @param connector
* @param properties
*/
void putConnectorConfig(String connector, Properties properties);
/**
* Read back the config for the given connector.
* @param connector
* @return
*/
Properties getConnectorConfig(String connector);
/**
* Get a list of connector names that have associated state in the store.
* @return
*/
Collection<String> getConnectors();
}

View File

@ -0,0 +1,110 @@
/**
* 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.errors.CopycatRuntimeException;
import java.io.*;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
/**
* <p>
* Implementation of ConfigStorage that saves state to a local file. This allows a standalone
* node to tolerate faults/restarts.
* </p>
* <p>
* Currently the implementation is naive, inefficient, and only meant for testing or a small
* number of jobs.
* </p>
*/
public class FileConfigStorage implements ConfigStorage {
public static final String FILE_CONFIG = "config.storage.file";
public static final String FILE_DEFAULT = "configs.db";
private String filename;
private Map<String, Properties> connectorConfig = new HashMap<String, Properties>();
@Override
public void configure(Properties props) {
filename = props.getProperty(FILE_CONFIG);
if (filename == null)
filename = FILE_DEFAULT;
load();
}
@Override
public void close() {
}
@Override
public void putConnectorConfig(String connector, Properties properties) {
if (properties == null) {
connectorConfig.remove(connector);
} else {
connectorConfig.put(connector, properties);
}
save();
}
@Override
public Properties getConnectorConfig(String connector) {
return connectorConfig.get(connector);
}
@Override
public Collection<String> getConnectors() {
return connectorConfig.keySet();
}
/**
* Saves the current state to disk, overwriting previous data. This action is performed
* atomically.
*/
private void save() {
try {
String tempFilename = filename + ".temp";
ObjectOutputStream os = new ObjectOutputStream(new FileOutputStream(tempFilename));
os.writeObject(connectorConfig);
os.close();
// Overwrite the original. Since the nio file package is JDK7+ only, this is the best we
// can do.
File tempFile = new File(tempFilename);
tempFile.renameTo(new File(filename));
} catch (IOException e) {
throw new CopycatRuntimeException("Failed to save config data to file", e);
}
}
private void load() {
try {
ObjectInputStream is = new ObjectInputStream(new FileInputStream(filename));
connectorConfig = (Map<String, Properties>) is.readObject();
} catch (FileNotFoundException e) {
// Expected on first run
} catch (IOException e) {
throw new CopycatRuntimeException("Failed to load config data", e);
} catch (ClassNotFoundException e) {
throw new CopycatRuntimeException("Failed to load config data", e);
}
}
}

View File

@ -0,0 +1,60 @@
/**
* 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.I0Itec.zkclient.ZkClient;
import org.apache.kafka.copycat.connector.ConnectorContext;
import org.apache.kafka.copycat.connector.TopicPartition;
import org.apache.kafka.copycat.util.KafkaUtils;
import java.util.List;
/**
* ConnectorContext for use with the StandaloneCoordinator, which maintains all connectors and tasks
* in a single process.
*/
public class StandaloneConnectorContext implements ConnectorContext {
private StandaloneCoordinator coordinator;
private String connectorName;
private ZkClient zkClient;
public StandaloneConnectorContext(StandaloneCoordinator coordinator, String connectorName,
ZkClient zkClient) {
this.coordinator = coordinator;
this.connectorName = connectorName;
this.zkClient = zkClient;
}
@Override
public void requestTaskReconfiguration() {
// This is trivial to forward since there is only one coordinator and it's in memory in this
// process
coordinator.requestTaskReconfiguration(connectorName);
}
@Override
public List<TopicPartition> getTopicPartitions(String... topics) {
return KafkaUtils.getTopicPartitions(zkClient, topics);
}
@Override
public List<TopicPartition> getTopicPartitions(List<String> topics) {
return KafkaUtils.getTopicPartitions(zkClient, topics);
}
}

View File

@ -0,0 +1,315 @@
/**
* 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.connector.TopicPartition;
import org.apache.kafka.copycat.errors.CopycatException;
import org.apache.kafka.copycat.errors.CopycatRuntimeException;
import org.apache.kafka.copycat.runtime.ConnectorConfig;
import org.apache.kafka.copycat.runtime.Coordinator;
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 "coordinator". Useful for a standalone copycat process.
*/
public class StandaloneCoordinator implements Coordinator {
private static final Logger log = LoggerFactory.getLogger(StandaloneCoordinator.class);
public static final String STORAGE_CONFIG = "coordinator.standalone.storage";
private Worker worker;
private Properties configs;
private ConfigStorage configStorage;
private HashMap<String, ConnectorState> connectors = new HashMap<String, ConnectorState>();
public StandaloneCoordinator(Worker worker, Properties props) {
this.worker = worker;
this.configs = props;
}
public synchronized void start() {
log.info("Coordinator starting");
String storage = configs.getProperty(STORAGE_CONFIG);
if (storage != null && !storage.isEmpty()) {
configStorage = Reflection.instantiate(storage, ConfigStorage.class);
configStorage.configure(configs);
} else {
configStorage = null;
}
restoreConnectors();
log.info("Coordinator started");
}
public synchronized void stop() {
log.info("Coordinator 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();
if (configStorage != null) {
configStorage.close();
configStorage = null;
}
log.info("Coordinator 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 (CopycatRuntimeException 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 (CopycatRuntimeException 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 CopycatRuntimeException("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 CopycatRuntimeException("Failed to create connector instance", t);
}
connector.initialize(new StandaloneConnectorContext(this, connName, worker.getZkClient()));
try {
connector.start(configs);
} catch (CopycatException e) {
throw new CopycatRuntimeException("Connector threw an exception while starting", e);
}
ConnectorState state = new ConnectorState(connName, connector, maxTasks, topics);
connectors.put(connName, state);
if (configStorage != null) {
configStorage.putConnectorConfig(connName, connectorProps);
}
log.info("Finished creating connector {}", connName);
return state;
}
private static Connector instantiateConnector(String className) {
return Reflection.instantiate(className, Connector.class);
}
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 CopycatRuntimeException("Connector does not exist");
}
stopConnector(state);
connectors.remove(state.name);
if (configStorage != null) {
configStorage.putConnectorConfig(state.name, null);
}
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);
int maxTasks = state.maxTasks;
// For sink tasks, we may also be limited by the number of input topic partitions
List<TopicPartition> topicPartitions = null;
if (state.connector instanceof SinkConnector) {
topicPartitions = KafkaUtils.getTopicPartitions(worker.getZkClient(), state.inputTopics);
maxTasks = Math.min(maxTasks, topicPartitions.size());
}
List<Properties> taskConfigs = state.connector.getTaskConfigs(maxTasks);
// If necessary, figure out how to distribute input topic partitions
// TODO: This needs to run periodically so we detect new partitions
List<List<TopicPartition>> taskAssignments = null;
if (state.connector instanceof SinkConnector) {
taskAssignments = ConnectorUtils.groupPartitions(topicPartitions, taskConfigs.size());
}
// Generate the final configs, including framework provided settings
Map<ConnectorTaskId, Properties> taskProps = new HashMap<ConnectorTaskId, Properties>();
for (int i = 0; i < taskConfigs.size(); i++) {
ConnectorTaskId taskId = new ConnectorTaskId(state.name, i);
Properties config = taskConfigs.get(i);
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.TOPICPARTITIONS_CONFIG, Utils.join(taskAssignments.get(i), ","));
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);
}
private void restoreConnectors() {
if (configStorage == null) {
return;
}
Collection<String> connNames = configStorage.getConnectors();
for (String connName : connNames) {
log.info("Restoring connector {}", connName);
Properties connProps = configStorage.getConnectorConfig(connName);
ConnectorState connState = createConnector(connProps);
// Because this coordinator is standalone, connectors are only restored when this process
// starts and we know there can't be any existing tasks. So in this special case we're able
// to just create the tasks rather than having to check for existing tasks and sort out
// whether they need to be reconfigured.
createConnectorTasks(connState);
}
}
/**
* 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<ConnectorTaskId>();
}
}
}

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.storage;
import org.apache.kafka.copycat.errors.CopycatRuntimeException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.*;
import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
/**
* 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(Properties props) {
super.configure(props);
String filename = props.getProperty(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");
}
private void load() {
try {
ObjectInputStream is = new ObjectInputStream(new FileInputStream(file));
HashMap<String, Map<byte[], byte[]>> raw
= (HashMap<String, Map<byte[], byte[]>>) is.readObject();
data = new HashMap<String, Map<ByteBuffer, ByteBuffer>>();
for (Map.Entry<String, Map<byte[], byte[]>> entry : raw.entrySet()) {
HashMap<ByteBuffer, ByteBuffer> converted = new HashMap<ByteBuffer, ByteBuffer>();
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 e) {
// Ignore, may be new
} catch (EOFException e) {
// Ignore, this means the file was missing or corrupt
} catch (IOException e) {
throw new CopycatRuntimeException(e);
} catch (ClassNotFoundException e) {
throw new CopycatRuntimeException(e);
}
}
protected void save() {
try {
ObjectOutputStream os = new ObjectOutputStream(new FileOutputStream(file));
HashMap<String, Map<byte[], byte[]>> raw = new HashMap<String, Map<byte[], byte[]>>();
for (Map.Entry<String, Map<ByteBuffer, ByteBuffer>> entry : data.entrySet()) {
HashMap<byte[], byte[]> converted = new HashMap<byte[], byte[]>();
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 CopycatRuntimeException(e);
}
}
}

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.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.Properties;
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<String, Map<ByteBuffer, ByteBuffer>>();
protected ExecutorService executor = Executors.newSingleThreadExecutor();
public MemoryOffsetBackingStore() {
}
@Override
public void configure(Properties 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<ByteBuffer, ByteBuffer>();
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<ByteBuffer, ByteBuffer>();
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.copycat.util.Callback;
import org.apache.kafka.copycat.util.Configurable;
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,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.data.Schema;
import org.apache.kafka.copycat.errors.CopycatRuntimeException;
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 implements OffsetStorageReader {
private static final Logger log = LoggerFactory.getLogger(OffsetStorageReaderImpl.class);
private final OffsetBackingStore backingStore;
private final String namespace;
private final Converter converter;
private final OffsetSerializer keySerializer;
private final OffsetDeserializer valueDeserializer;
public OffsetStorageReaderImpl(OffsetBackingStore backingStore, String namespace,
Converter converter,
OffsetSerializer keySerializer,
OffsetDeserializer valueDeserializer) {
this.backingStore = backingStore;
this.namespace = namespace;
this.converter = converter;
this.keySerializer = keySerializer;
this.valueDeserializer = valueDeserializer;
}
@Override
public Object getOffset(Object stream, Schema schema) {
return getOffsets(Arrays.asList(stream), schema).get(stream);
}
@Override
public Map<Object, Object> getOffsets(Collection<Object> streams, Schema schema) {
// Serialize keys so backing store can work with them
Map<ByteBuffer, Object> serializedToOriginal = new HashMap<ByteBuffer, Object>(streams.size());
for (Object key : streams) {
try {
byte[] keySerialized = keySerializer.serializeOffset(namespace,
converter.fromCopycatData(key));
ByteBuffer keyBuffer = (keySerialized != null) ? ByteBuffer.wrap(keySerialized) : null;
serializedToOriginal.put(keyBuffer, key);
} catch (Throwable t) {
log.error("CRITICAL: Failed to serialize stream 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 CopycatRuntimeException("Failed to fetch offsets.", e);
}
// Deserialize all the values and map back to the original keys
Map<Object, Object> result = new HashMap<Object, Object>(streams.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 stream-offset key, backing "
+ "store may have returned invalid data", rawEntry.getKey());
continue;
}
Object origKey = serializedToOriginal.get(rawEntry.getKey());
Object deserializedValue = converter.toCopycatData(
valueDeserializer.deserializeOffset(namespace, rawEntry.getValue().array(), schema)
);
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,204 @@
/**
* 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.CopycatRuntimeException;
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 streams 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 streams 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
* stream A, writing offsets; reconfiguration causes stream A to be reassigned elsewhere;
* reconfiguration causes stream 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 {
private static final Logger log = LoggerFactory.getLogger(OffsetStorageWriter.class);
private final OffsetBackingStore backingStore;
private final Converter converter;
private final OffsetSerializer keySerializer;
private final OffsetSerializer valueSerializer;
private final String namespace;
private Map<Object, Object> data = new HashMap<Object, Object>();
// 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 converter,
OffsetSerializer keySerializer, OffsetSerializer valueSerializer) {
this.backingStore = backingStore;
this.namespace = namespace;
this.converter = converter;
this.keySerializer = keySerializer;
this.valueSerializer = valueSerializer;
}
public synchronized void setOffset(Object stream, Object offset) {
data.put(stream, 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 CopycatRuntimeException("OffsetStorageWriter is already flushing");
}
if (data.isEmpty()) {
return false;
}
assert !flushing();
toFlush = data;
data = new HashMap<Object, Object>();
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<ByteBuffer, ByteBuffer>();
for (Map.Entry<Object, Object> entry : toFlush.entrySet()) {
byte[] key = keySerializer.serializeOffset(namespace,
converter.fromCopycatData(entry.getKey()));
ByteBuffer keyBuffer = (key != null) ? ByteBuffer.wrap(key) : null;
byte[] value = valueSerializer.serializeOffset(namespace,
converter.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 stream 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

@ -5,9 +5,9 @@
* 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
*
* <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.
@ -17,22 +17,11 @@
package org.apache.kafka.copycat.util;
import org.junit.Test;
import java.util.Properties;
import java.util.Arrays;
import java.util.Collections;
import static org.junit.Assert.assertEquals;
public class StringUtilsTest {
@Test
public void testJoin() {
assertEquals("", StringUtils.join(Collections.EMPTY_LIST, ""));
assertEquals("", StringUtils.join(Collections.EMPTY_LIST, ","));
assertEquals("ab", StringUtils.join(Arrays.asList("a", "b"), ""));
assertEquals("a,b,c", StringUtils.join(Arrays.asList("a", "b", "c"), ","));
}
/**
* Class that can be configured immediately after instantiation with a set of properties.
*/
public interface Configurable {
public void configure(Properties props);
}

View File

@ -0,0 +1,75 @@
/**
* 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 underlying;
private CountDownLatch finishedLatch;
private T result = null;
private Throwable exception = null;
public FutureCallback(Callback 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,53 @@
/**
* 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 kafka.utils.ZkUtils;
import org.I0Itec.zkclient.ZkClient;
import org.apache.kafka.copycat.connector.TopicPartition;
import scala.collection.Seq;
import scala.collection.mutable.Map;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import static scala.collection.JavaConversions.*;
/**
* Some utilities for working with Kafka that aren't included with Kafka itself.
*/
public class KafkaUtils {
public static List<TopicPartition> getTopicPartitions(ZkClient zkClient, String... topics) {
return getTopicPartitions(zkClient, Arrays.asList(topics));
}
public static List<TopicPartition> getTopicPartitions(ZkClient zkClient, List<String> topics) {
Seq<String> scalaTopics = asScalaIterable(topics).toSeq();
List<TopicPartition> result = new ArrayList<TopicPartition>();
Map<String, Seq<Object>> partitionsByTopic
= ZkUtils.getPartitionsForTopics(zkClient, scalaTopics);
for (java.util.Map.Entry<String, Seq<Object>> entry : asJavaMap(partitionsByTopic).entrySet()) {
for (Object partition : asJavaIterable(entry.getValue())) {
result.add(new TopicPartition(entry.getKey(), (Integer) partition));
}
}
return result;
}
}

View File

@ -0,0 +1,75 @@
/**
* 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.apache.kafka.copycat.errors.CopycatRuntimeException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.lang.reflect.InvocationTargetException;
import java.util.Properties;
public class Reflection {
private static final Logger log = LoggerFactory.getLogger(Reflection.class);
/**
* Instantiate the class, which is a subclass of the specified type.
* @param className fully qualified name of the class to instantiate
* @param superType required supertype of the class
* @param <T>
* @return a new instance of the class instantiated with the default constructor
*/
public static <T> T instantiate(String className, Class<T> superType) {
try {
Class<? extends T> taskClass = Class.forName(className).asSubclass(superType);
return taskClass.getConstructor().newInstance();
} catch (ClassNotFoundException e) {
log.error("Class not found: " + className, e);
throw new CopycatRuntimeException(e);
} catch (ClassCastException e) {
log.error("Specified class " + className + " is not a subclass of " + superType + ":", e);
throw new CopycatRuntimeException(e);
} catch (NoSuchMethodException e) {
log.error("Class does not have a default constructor: " + className, e);
throw new CopycatRuntimeException(e);
} catch (InstantiationException e) {
log.error("Class couldn't be instantiated: " + className, e);
throw new CopycatRuntimeException(e);
} catch (InvocationTargetException e) {
log.error("Class constructor threw an exception: " + className, e);
throw new CopycatRuntimeException(e);
} catch (IllegalAccessException e) {
log.error("Class couldn't be instantiated due to IllegalAccessException: " + className, e);
throw new CopycatRuntimeException(e);
}
}
/**
* Instantiate the class, which is a subclass of the specified type, and start it.
* @param className fully qualified name of the class to instantiate
* @param superType required supertype of the class
* @param <T>
* @return a new instance of the class instantiated with the default constructor
*/
public static <T extends Configurable> T instantiateConfigurable(
String className, Class<T> superType, Properties props) {
T result = instantiate(className, superType);
result.configure(props);
return result;
}
}

View File

@ -0,0 +1,150 @@
/**
* 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 e) {
log.error("Thread {} exiting with uncaught exception: ", getName(), e);
throw e;
} catch (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,421 @@
/**
* 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.data.GenericRecord;
import org.apache.kafka.copycat.data.GenericRecordBuilder;
import org.apache.kafka.copycat.data.Schema;
import org.apache.kafka.copycat.data.SchemaBuilder;
import org.apache.kafka.copycat.errors.CopycatRuntimeException;
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.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
private static final String TOPIC = "test";
private static final int PARTITION = 12;
private static final long FIRST_OFFSET = 45;
private static final String KEY = "KEY";
private static final String VALUE = "VALUE";
private static final String TOPIC_PARTITION_STR = "test-12";
private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, PARTITION);
private static final org.apache.kafka.copycat.connector.TopicPartition TOPIC_PARTITION_COPYCAT =
new org.apache.kafka.copycat.connector.TopicPartition(TOPIC, PARTITION);
private ConnectorTaskId taskId = new ConnectorTaskId("job", 0);
private Time time;
private SinkTask sinkTask;
private WorkerConfig workerConfig;
private Converter converter;
private WorkerSinkTask workerTask;
private KafkaConsumer<Object, Object> consumer;
private WorkerSinkTaskThread workerThread;
private long recordsReturned;
@Override
public void setup() {
super.setup();
time = new MockTime();
sinkTask = PowerMock.createMock(SinkTask.class);
workerConfig = new WorkerConfig();
converter = PowerMock.createMock(Converter.class);
workerTask = PowerMock.createPartialMock(
WorkerSinkTask.class, new String[]{"createConsumer", "createWorkerThread"},
taskId, sinkTask, workerConfig, converter, time);
recordsReturned = 0;
}
@Test
public void testGetInputTopicPartitions() throws Exception {
Properties props = new Properties();
props.setProperty(SinkTask.TOPICPARTITIONS_CONFIG, "topic-1,foo-2");
assertEquals(
Arrays.asList(new org.apache.kafka.copycat.connector.TopicPartition("topic", 1),
new org.apache.kafka.copycat.connector.TopicPartition("foo", 2)),
Whitebox.invokeMethod(workerTask, "getInputTopicPartitions", props)
);
}
@Test
public void testPollsInBackground() throws Exception {
Properties taskProps = new Properties();
taskProps.setProperty(SinkTask.TOPICPARTITIONS_CONFIG, TOPIC_PARTITION_STR);
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
Schema schema = SchemaBuilder.record("sample").fields().endRecord();
GenericRecord record = new GenericRecordBuilder(schema).build();
byte[] rawKey = "key".getBytes(), rawValue = "value".getBytes();
ConsumerRecords<Object, Object> records = new ConsumerRecords<Object, Object>(
Collections.singletonMap(
new TopicPartition("topic", 0),
Collections.singletonList(
new ConsumerRecord<Object, Object>("topic", 0, 0, rawKey, rawValue))));
// Exact data doesn't matter, but should be passed directly to sink task
EasyMock.expect(converter.toCopycatData(rawKey))
.andReturn(record);
EasyMock.expect(converter.toCopycatData(rawValue))
.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();
taskProps.setProperty(SinkTask.TOPICPARTITIONS_CONFIG, TOPIC_PARTITION_STR);
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();
taskProps.setProperty(SinkTask.TOPICPARTITIONS_CONFIG, TOPIC_PARTITION_STR);
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();
taskProps.setProperty(SinkTask.TOPICPARTITIONS_CONFIG, TOPIC_PARTITION_STR);
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();
taskProps.setProperty(SinkTask.TOPICPARTITIONS_CONFIG, TOPIC_PARTITION_STR);
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();
}
@Test
public void testGetInputPartitionsSingle() throws Exception {
Properties taskProps = new Properties();
taskProps.setProperty(SinkTask.TOPICPARTITIONS_CONFIG, "test-1");
assertEquals(Arrays.asList(new org.apache.kafka.copycat.connector.TopicPartition("test", 1)),
Whitebox.invokeMethod(workerTask, "getInputTopicPartitions", taskProps));
}
@Test
public void testGetInputPartitionsList() throws Exception {
Properties taskProps = new Properties();
taskProps.setProperty(SinkTask.TOPICPARTITIONS_CONFIG, "test-1,foo-2,bar-3");
assertEquals(Arrays.asList(
new org.apache.kafka.copycat.connector.TopicPartition("test", 1),
new org.apache.kafka.copycat.connector.TopicPartition("foo", 2),
new org.apache.kafka.copycat.connector.TopicPartition("bar", 3)),
Whitebox.invokeMethod(workerTask, "getInputTopicPartitions", taskProps));
}
@Test(expected = CopycatRuntimeException.class)
public void testGetInputPartitionsMissing() throws Exception {
// Missing setting
Whitebox.invokeMethod(workerTask, "getInputTopicPartitions", new Properties());
}
private KafkaConsumer<Object, Object> expectInitializeTask(Properties taskProps)
throws Exception {
sinkTask.initialize(EasyMock.anyObject(SinkTaskContext.class));
PowerMock.expectLastCall();
sinkTask.start(taskProps);
PowerMock.expectLastCall();
consumer = PowerMock.createMock(KafkaConsumer.class);
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();
// Triggers final offset commit
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;
}
});
final Capture<ConsumerCommitCallback> capturedCallback = EasyMock.newCapture();
consumer.commit(EasyMock.eq(Collections.singletonMap(TOPIC_PARTITION, finalOffset)),
EasyMock.eq(CommitType.SYNC),
EasyMock.capture(capturedCallback));
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<Object, Object>>() {
@Override
public ConsumerRecords<Object, Object> answer() throws Throwable {
// "Sleep" so time will progress
time.sleep(pollDelayMs);
ConsumerRecords<Object, Object> records = new ConsumerRecords<Object, Object>(
Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), Arrays.asList(
new ConsumerRecord<Object, Object>(TOPIC, PARTITION,
FIRST_OFFSET + recordsReturned, KEY,
VALUE))));
recordsReturned++;
return records;
}
});
EasyMock.expect(converter.toCopycatData(KEY)).andReturn(KEY).anyTimes();
EasyMock.expect(converter.toCopycatData(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_COPYCAT, 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,275 @@
/**
* 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.data.GenericRecord;
import org.apache.kafka.copycat.data.GenericRecordBuilder;
import org.apache.kafka.copycat.data.Schema;
import org.apache.kafka.copycat.data.SchemaBuilder;
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.powermock.api.easymock.PowerMock;
import org.powermock.reflect.Whitebox;
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.*;
public class WorkerSourceTaskTest extends ThreadedTest {
private static final byte[] STREAM_BYTES = "stream".getBytes();
private static final byte[] OFFSET_BYTES = "offset-1".getBytes();
private static final Schema RECORD_SCHEMA = SchemaBuilder.record("sample").fields().endRecord();
private static final GenericRecord RECORD = new GenericRecordBuilder(RECORD_SCHEMA).build();
// 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 String CONVERTED_RECORD = "converted-record";
private ConnectorTaskId taskId = new ConnectorTaskId("job", 0);
private WorkerConfig config = new WorkerConfig();
private SourceTask sourceTask;
private Converter converter;
private KafkaProducer<Object, Object> producer;
private OffsetStorageReader offsetReader;
private OffsetStorageWriter offsetWriter;
private WorkerSourceTask workerTask;
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(STREAM_BYTES, OFFSET_BYTES,
"topic", RECORD)
);
@Override
public void setup() {
super.setup();
sourceTask = PowerMock.createMock(SourceTask.class);
converter = PowerMock.createMock(Converter.class);
producer = PowerMock.createMock(KafkaProducer.class);
offsetReader = PowerMock.createMock(OffsetStorageReader.class);
offsetWriter = PowerMock.createMock(OffsetStorageWriter.class);
sendFuture = PowerMock.createMock(Future.class);
producerCallbacks = EasyMock.newCapture();
}
private void createWorkerTask() {
workerTask = new WorkerSourceTask(taskId, sourceTask, converter, 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<SourceRecord>();
// Can just use the same record for key and value
records.add(new SourceRecord(STREAM_BYTES, OFFSET_BYTES, "topic", null, RECORD, RECORD));
Capture<ProducerRecord> sent = expectSendRecord();
PowerMock.replayAll();
Whitebox.invokeMethod(workerTask, "sendRecords", records);
assertEquals(CONVERTED_RECORD, 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> expectSendRecord() throws InterruptedException {
EasyMock.expect(converter.fromCopycatData(null)).andStubReturn(null);
EasyMock.expect(converter.fromCopycatData(RECORD)).andStubReturn(CONVERTED_RECORD);
Capture<ProducerRecord> 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(STREAM_BYTES, OFFSET_BYTES);
PowerMock.expectLastCall().anyTimes();
return sent;
}
private void awaitPolls(CountDownLatch latch) throws InterruptedException {
latch.await(1000, TimeUnit.MILLISECONDS);
}
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,177 @@
/**
* 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.I0Itec.zkclient.ZkClient;
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.errors.CopycatRuntimeException;
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 OffsetSerializer offsetKeySerializer = PowerMock.createMock(OffsetSerializer.class);
private OffsetSerializer offsetValueSerializer = PowerMock.createMock(OffsetSerializer.class);
private OffsetDeserializer offsetKeyDeserializer = PowerMock.createMock(OffsetDeserializer.class);
private OffsetDeserializer offsetValueDeserializer = PowerMock.createMock(OffsetDeserializer.class);
@Before
public void setup() {
super.setup();
// TODO: Remove schema registry URL
Properties workerProps = new Properties();
workerProps.setProperty("schema.registry.url", "http://localhost:8081");
WorkerConfig config = new WorkerConfig(workerProps);
ZkClient zkClient = PowerMock.createMock(ZkClient.class);
worker = new Worker(new MockTime(), config, offsetBackingStore,
offsetKeySerializer, offsetValueSerializer,
offsetKeyDeserializer, offsetValueDeserializer,
zkClient);
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(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 = CopycatRuntimeException.class)
public void testStopInvalidTask() throws CopycatException {
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(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<Object, Object> {
public TestSourceTask() {
}
@Override
public void start(Properties props) {
}
@Override
public List<SourceRecord> poll() throws InterruptedException {
return null;
}
@Override
public void stop() throws CopycatException {
}
}
}

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.standalone;
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.util.Callback;
import org.apache.kafka.copycat.util.KafkaUtils;
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.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
import java.io.File;
import java.util.Properties;
@RunWith(PowerMockRunner.class)
@PrepareForTest({StandaloneCoordinator.class, KafkaUtils.class})
@PowerMockIgnore("javax.management.*")
public class StandaloneCoordinatorRestoreTest extends StandaloneCoordinatorTestBase {
private File coordinatorConfigFile;
@Before
public void setup() throws Exception {
worker = PowerMock.createMock(Worker.class);
Properties coordinatorProps = new Properties();
coordinatorProps.setProperty(StandaloneCoordinator.STORAGE_CONFIG,
FileConfigStorage.class.getName());
coordinatorConfigFile = File.createTempFile("test-coordinator-config", null);
coordinatorConfigFile.delete(); // Delete since we just needed a random file path
coordinatorProps.setProperty(FileConfigStorage.FILE_CONFIG,
coordinatorConfigFile.getAbsolutePath());
coordinator = new StandaloneCoordinator(worker, coordinatorProps);
createCallback = PowerMock.createMock(Callback.class);
connectorProps = new Properties();
connectorProps.setProperty(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME);
connectorProps.setProperty(SinkConnector.TOPICS_CONFIG, TOPICS_LIST_STR);
PowerMock.mockStatic(StandaloneCoordinator.class);
PowerMock.mockStatic(KafkaUtils.class);
// These can be anything since connectors can pass along whatever they want.
taskProps = new Properties();
taskProps.setProperty("foo", "bar");
}
@After
public void tearDown() {
coordinatorConfigFile.delete();
}
@Test
public void testRestoreConnectors() throws Exception {
connector = PowerMock.createMock(BogusSourceClass.class);
expectAdd(BogusSourceClass.class, BogusSourceTask.class, false);
expectStop();
// Restarting should recreate the same connector
expectRestore(BogusSourceClass.class, BogusSourceTask.class);
expectStop();
PowerMock.replayAll();
// One run to seed the config storage with the job
coordinator.start();
coordinator.addConnector(connectorProps, createCallback);
coordinator.stop();
// Second run should restore the connector
coordinator.start();
coordinator.stop();
PowerMock.verifyAll();
}
}

View File

@ -0,0 +1,100 @@
/**
* 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.runtime.ConnectorConfig;
import org.apache.kafka.copycat.runtime.Worker;
import org.apache.kafka.copycat.sink.SinkConnector;
import org.apache.kafka.copycat.util.Callback;
import org.apache.kafka.copycat.util.FutureCallback;
import org.apache.kafka.copycat.util.KafkaUtils;
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.Properties;
import java.util.concurrent.TimeUnit;
@RunWith(PowerMockRunner.class)
@PrepareForTest({StandaloneCoordinator.class, KafkaUtils.class})
@PowerMockIgnore("javax.management.*")
public class StandaloneCoordinatorTest extends StandaloneCoordinatorTestBase {
@Before
public void setup() {
worker = PowerMock.createMock(Worker.class);
coordinator = new StandaloneCoordinator(worker, new Properties());
createCallback = PowerMock.createMock(Callback.class);
connectorProps = new Properties();
connectorProps.setProperty(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME);
connectorProps.setProperty(SinkConnector.TOPICS_CONFIG, TOPICS_LIST_STR);
PowerMock.mockStatic(StandaloneCoordinator.class);
PowerMock.mockStatic(KafkaUtils.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();
coordinator.addConnector(connectorProps, createCallback);
PowerMock.verifyAll();
}
@Test
public void testCreateSinkConnector() throws Exception {
connector = PowerMock.createMock(BogusSinkClass.class);
expectAdd(BogusSinkClass.class, BogusSinkTask.class, true);
PowerMock.replayAll();
coordinator.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();
coordinator.addConnector(connectorProps, createCallback);
FutureCallback<Void> futureCb = new FutureCallback<Void>(new Callback() {
@Override
public void onCompletion(Throwable error, Object result) {
}
});
coordinator.deleteConnector(CONNECTOR_NAME, futureCb);
futureCb.get(1000L, TimeUnit.MILLISECONDS);
PowerMock.verifyAll();
}
}

View File

@ -0,0 +1,133 @@
/**
* 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.I0Itec.zkclient.ZkClient;
import org.apache.kafka.copycat.connector.Connector;
import org.apache.kafka.copycat.connector.Task;
import org.apache.kafka.copycat.connector.TopicPartition;
import org.apache.kafka.copycat.errors.CopycatException;
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.KafkaUtils;
import org.easymock.EasyMock;
import org.powermock.api.easymock.PowerMock;
import java.util.Arrays;
import java.util.List;
import java.util.Properties;
public class StandaloneCoordinatorTestBase {
protected static final String CONNECTOR_NAME = "test";
protected static final String TOPICS_LIST_STR = "topic1,topic2";
protected static final List<String> TOPICS_LIST = Arrays.asList("topic1", "topic2");
protected static final List<TopicPartition> TOPIC_PARTITIONS = Arrays.asList(
new TopicPartition("topic1", 1), new TopicPartition("topic2", 1));
protected static final String TOPIC_PARTITIONS_STR = "topic1-1,topic2-1";
protected StandaloneCoordinator coordinator;
protected Worker worker;
protected Connector connector;
protected Callback<String> createCallback;
protected Properties connectorProps;
protected Properties taskProps;
protected void expectAdd(Class<? extends Connector> connClass,
Class<? extends Task> taskClass,
boolean sink) throws Exception {
expectCreate(connClass, taskClass, sink, true);
}
protected 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);
}
protected 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(StandaloneCoordinator.class, "instantiateConnector", connClass.getName())
.andReturn(connector);
if (expectCallback) {
createCallback.onCompletion(null, CONNECTOR_NAME);
PowerMock.expectLastCall();
}
ZkClient zkClient = PowerMock.createMock(ZkClient.class);
EasyMock.expect(worker.getZkClient()).andStubReturn(zkClient);
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);
if (sink) {
EasyMock.expect(KafkaUtils.getTopicPartitions(zkClient, TOPICS_LIST))
.andReturn(TOPIC_PARTITIONS);
}
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.TOPICPARTITIONS_CONFIG, TOPIC_PARTITIONS_STR);
}
worker.addTask(new ConnectorTaskId(CONNECTOR_NAME, 0), taskClass.getName(), generatedTaskProps);
PowerMock.expectLastCall();
}
protected void expectStop() throws CopycatException {
worker.stopTask(new ConnectorTaskId(CONNECTOR_NAME, 0));
EasyMock.expectLastCall();
connector.stop();
EasyMock.expectLastCall();
}
protected void expectDestroy() throws CopycatException {
expectStop();
}
// We need to use a real class here due to some issue with mocking java.lang.Class
protected abstract class BogusSourceClass extends SourceConnector {
}
protected abstract class BogusSourceTask extends SourceTask {
}
protected abstract class BogusSinkClass extends SinkConnector {
}
protected abstract class BogusSinkTask extends SourceTask {
}
}

View File

@ -0,0 +1,120 @@
/**
* 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 java.util.Properties;
import static org.junit.Assert.assertEquals;
public class FileOffsetBackingStoreTest {
FileOffsetBackingStore store;
Properties props;
File tempFile;
private static Map<ByteBuffer, ByteBuffer> firstSet = new HashMap<ByteBuffer, ByteBuffer>();
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 Properties();
props.setProperty(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() {
Callback<Void> setCallback = PowerMock.createMock(Callback.class);
setCallback.onCompletion(null, null);
PowerMock.expectLastCall();
return setCallback;
}
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,233 @@
/**
* 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.CopycatRuntimeException;
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.powermock.api.easymock.PowerMock;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.*;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class OffsetStorageWriterTest {
private static final String NAMESPACE = "namespace";
private static final String OFFSET_KEY = "key";
private static final String OFFSET_VALUE = "value";
private static final String OFFSET_KEY_CONVERTED = "key-converted";
private static final String OFFSET_VALUE_CONVERTED = "value-converted";
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));
private OffsetBackingStore store;
private Converter converter;
private OffsetSerializer keySerializer;
private OffsetSerializer valueSerializer;
private OffsetStorageWriter writer;
private static Exception exception = new RuntimeException("error");
private ExecutorService service;
@Before
public void setup() {
store = PowerMock.createMock(OffsetBackingStore.class);
converter = PowerMock.createMock(Converter.class);
keySerializer = PowerMock.createMock(OffsetSerializer.class);
valueSerializer = PowerMock.createMock(OffsetSerializer.class);
writer = new OffsetStorageWriter(store, NAMESPACE, converter, keySerializer, valueSerializer);
service = Executors.newFixedThreadPool(1);
}
@After
public void teardown() {
service.shutdownNow();
}
@Test
public void testWriteFlush() throws Exception {
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.
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 = CopycatRuntimeException.class)
public void testAlreadyFlushing() throws Exception {
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 {
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(converter.fromCopycatData(OFFSET_KEY)).andReturn(OFFSET_KEY_CONVERTED);
EasyMock.expect(keySerializer.serializeOffset(NAMESPACE, OFFSET_KEY_CONVERTED)).andReturn(OFFSET_KEY_SERIALIZED);
EasyMock.expect(converter.fromCopycatData(OFFSET_VALUE)).andReturn(OFFSET_VALUE_CONVERTED);
EasyMock.expect(valueSerializer.serializeOffset(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

@ -16,30 +16,34 @@
**/
package org.apache.kafka.copycat.util;
/**
* General string utilities that are missing from the standard library and may commonly be
* required by Connector or Task implementations.
*/
public class StringUtils {
import org.apache.kafka.common.utils.Time;
/**
* Generate a String by appending all the @{elements}, converted to Strings, delimited by
* @{delim}.
* @param elements list of elements to concatenate
* @param delim delimiter to place between each element
* @return the concatenated string with delimiters
*/
public static <T> String join(Iterable<T> elements, String delim) {
StringBuilder result = new StringBuilder();
boolean first = true;
for (T elem : elements) {
if (first) {
first = false;
} else {
result.append(delim);
}
result.append(elem);
}
return result.toString();
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,39 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
* <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

@ -15,4 +15,4 @@
apply from: file('scala.gradle')
include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients',
'log4j-appender', 'copycat-data', 'copycat-api'
'log4j-appender', 'copycat-data', 'copycat-api', 'copycat-runtime', 'copycat-avro'