@ -18,8 +18,10 @@ package org.apache.kafka.connect.mirror;
@@ -18,8 +18,10 @@ package org.apache.kafka.connect.mirror;
import org.apache.kafka.clients.admin.Admin ;
import org.apache.kafka.clients.consumer.Consumer ;
import org.apache.kafka.clients.consumer.ConsumerRecords ;
import org.apache.kafka.clients.consumer.OffsetAndMetadata ;
import org.apache.kafka.common.TopicPartition ;
import org.apache.kafka.common.utils.Time ;
import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster ;
import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster ;
import org.apache.kafka.test.IntegrationTest ;
@ -61,16 +63,17 @@ public class MirrorConnectorsIntegrationTest {
@@ -61,16 +63,17 @@ public class MirrorConnectorsIntegrationTest {
private static final int RECORD_TRANSFER_DURATION_MS = 10_000 ;
private static final int CHECKPOINT_DURATION_MS = 20_000 ;
private Time time = Time . SYSTEM ;
private Map < String , String > mm2Props ;
private MirrorMakerConfig mm2Config ;
private EmbeddedConnectCluster primary ;
private EmbeddedConnectCluster backup ;
@Before
public void setup ( ) throws InterruptedException {
Properties brokerProps = new Properties ( ) ;
brokerProps . put ( "auto.create.topics.enable" , "false" ) ;
Map < String , String > mm2Props = new HashMap < > ( ) ;
mm2Props = new HashMap < > ( ) ;
mm2Props . put ( "clusters" , "primary, backup" ) ;
mm2Props . put ( "max.tasks" , "10" ) ;
mm2Props . put ( "topics" , "test-topic-.*, primary.test-topic-.*, backup.test-topic-.*" ) ;
@ -130,19 +133,6 @@ public class MirrorConnectorsIntegrationTest {
@@ -130,19 +133,6 @@ public class MirrorConnectorsIntegrationTest {
backup . kafka ( ) . produce ( "test-topic-1" , i % NUM_PARTITIONS , "key" , "message-2-" + i ) ;
}
// create consumers before starting the connectors so we don't need to wait for discovery
Consumer < byte [ ] , byte [ ] > consumer1 = primary . kafka ( ) . createConsumerAndSubscribeTo ( Collections . singletonMap (
"group.id" , "consumer-group-1" ) , "test-topic-1" , "backup.test-topic-1" ) ;
consumer1 . poll ( Duration . ofMillis ( 500 ) ) ;
consumer1 . commitSync ( ) ;
consumer1 . close ( ) ;
Consumer < byte [ ] , byte [ ] > consumer2 = backup . kafka ( ) . createConsumerAndSubscribeTo ( Collections . singletonMap (
"group.id" , "consumer-group-1" ) , "test-topic-1" , "primary.test-topic-1" ) ;
consumer2 . poll ( Duration . ofMillis ( 500 ) ) ;
consumer2 . commitSync ( ) ;
consumer2 . close ( ) ;
log . info ( "primary REST service: {}" , primary . endpointForResource ( "connectors" ) ) ;
log . info ( "backup REST service: {}" , backup . endpointForResource ( "connectors" ) ) ;
@ -152,42 +142,29 @@ public class MirrorConnectorsIntegrationTest {
@@ -152,42 +142,29 @@ public class MirrorConnectorsIntegrationTest {
// now that the brokers are running, we can finish setting up the Connectors
mm2Props . put ( "primary.bootstrap.servers" , primary . kafka ( ) . bootstrapServers ( ) ) ;
mm2Props . put ( "backup.bootstrap.servers" , backup . kafka ( ) . bootstrapServers ( ) ) ;
mm2Config = new MirrorMakerConfig ( mm2Props ) ;
// we wait for the connector and tasks to come up for each connector, so that when we do the
// actual testing, we are certain that the tasks are up and running; this will prevent
// flaky tests where the connector and tasks didn't start up in time for the tests to be
// run
Set < String > connectorNames = new HashSet < > ( Arrays . asList ( "MirrorSourceConnector" ,
"MirrorCheckpointConnector" , "MirrorHeartbeatConnector" ) ) ;
backup . configureConnector ( "MirrorSourceConnector" , mm2Config . connectorBaseConfig ( new SourceAndTarget ( "primary" , "backup" ) ,
MirrorSourceConnector . class ) ) ;
backup . configureConnector ( "MirrorCheckpointConnector" , mm2Config . connectorBaseConfig ( new SourceAndTarget ( "primary" , "backup" ) ,
MirrorCheckpointConnector . class ) ) ;
backup . configureConnector ( "MirrorHeartbeatConnector" , mm2Config . connectorBaseConfig ( new SourceAndTarget ( "primary" , "backup" ) ,
MirrorHeartbeatConnector . class ) ) ;
}
waitUntilMirrorMakerIsRunning ( backup , connectorNames ) ;
primary . configureConnector ( "MirrorSourceConnector" , mm2Config . connectorBaseConfig ( new SourceAndTarget ( "backup" , "primary" ) ,
MirrorSourceConnector . class ) ) ;
private void waitUntilMirrorMakerIsRunning ( EmbeddedConnectCluster connectCluster ,
MirrorMakerConfig mm2Config , String primary , String backup ) throws InterruptedException {
primary . configureConnector ( "MirrorCheckpoint Connector" , mm2Config . connectorBaseConfig ( new SourceAndTarget ( "backup" , "primary" ) ,
MirrorCheckpoint Connector. class ) ) ;
connectCluster . configureConnector ( "MirrorSourceConnector" , mm2Config . connectorBaseConfig ( new SourceAndTarget ( primary , backup ) ,
MirrorSourceConnector . class ) ) ;
primary . configureConnector ( "MirrorHeartbea tConnector" , mm2Config . connectorBaseConfig ( new SourceAndTarget ( "backup" , "primary" ) ,
MirrorHeartbea tConnector. class ) ) ;
connectCluster . configureConnector ( "MirrorCheckpointConnector" , mm2Config . connectorBaseConfig ( new SourceAndTarget ( primary , backup ) ,
MirrorCheckpointConnector . class ) ) ;
waitUntilMirrorMakerIsRunning ( primary , connectorNames ) ;
}
connectCluster . configureConnector ( "MirrorHeartbeatConnector" , mm2Config . connectorBaseConfig ( new SourceAndTarget ( primary , backup ) ,
MirrorHeartbeatConnector . class ) ) ;
// we wait for the connector and tasks to come up for each connector, so that when we do the
// actual testing, we are certain that the tasks are up and running; this will prevent
// flaky tests where the connector and tasks didn't start up in time for the tests to be
// run
Set < String > connectorNames = new HashSet < > ( Arrays . asList ( "MirrorSourceConnector" ,
"MirrorCheckpointConnector" , "MirrorHeartbeatConnector" ) ) ;
private void waitUntilMirrorMakerIsRunning ( EmbeddedConnectCluster connectCluster ,
Set < String > connNames ) throws InterruptedException {
for ( String connector : connNames ) {
for ( String connector : connectorNames ) {
connectCluster . assertions ( ) . assertConnectorAndAtLeastNumTasksAreRunning ( connector , 1 ,
"Connector " + connector + " tasks did not start in time on cluster: " + connectCluster ) ;
}
@ -207,8 +184,30 @@ public class MirrorConnectorsIntegrationTest {
@@ -207,8 +184,30 @@ public class MirrorConnectorsIntegrationTest {
backup . stop ( ) ;
}
@Test
public void testReplication ( ) throws InterruptedException {
// create consumers before starting the connectors so we don't need to wait for discovery
Consumer < byte [ ] , byte [ ] > consumer1 = primary . kafka ( ) . createConsumerAndSubscribeTo ( Collections . singletonMap (
"group.id" , "consumer-group-1" ) , "test-topic-1" , "backup.test-topic-1" ) ;
consumer1 . poll ( Duration . ofMillis ( 500 ) ) ;
consumer1 . commitSync ( ) ;
consumer1 . close ( ) ;
Consumer < byte [ ] , byte [ ] > consumer2 = backup . kafka ( ) . createConsumerAndSubscribeTo ( Collections . singletonMap (
"group.id" , "consumer-group-1" ) , "test-topic-1" , "primary.test-topic-1" ) ;
consumer2 . poll ( Duration . ofMillis ( 500 ) ) ;
consumer2 . commitSync ( ) ;
consumer2 . close ( ) ;
mm2Config = new MirrorMakerConfig ( mm2Props ) ;
waitUntilMirrorMakerIsRunning ( backup , mm2Config , "primary" , "backup" ) ;
waitUntilMirrorMakerIsRunning ( primary , mm2Config , "backup" , "primary" ) ;
MirrorClient primaryClient = new MirrorClient ( mm2Config . clientConfig ( "primary" ) ) ;
MirrorClient backupClient = new MirrorClient ( mm2Config . clientConfig ( "backup" ) ) ;
@ -246,7 +245,7 @@ public class MirrorConnectorsIntegrationTest {
@@ -246,7 +245,7 @@ public class MirrorConnectorsIntegrationTest {
new TopicPartition ( "primary.test-topic-1" , 0 ) ) ) ;
// Failover consumer group to backup cluster.
Consumer < byte [ ] , byte [ ] > consumer1 = backup . kafka ( ) . createConsumer ( Collections . singletonMap ( "group.id" , "consumer-group-1" ) ) ;
consumer1 = backup . kafka ( ) . createConsumer ( Collections . singletonMap ( "group.id" , "consumer-group-1" ) ) ;
consumer1 . assign ( backupOffsets . keySet ( ) ) ;
backupOffsets . forEach ( consumer1 : : seek ) ;
consumer1 . poll ( Duration . ofMillis ( 500 ) ) ;
@ -282,7 +281,7 @@ public class MirrorConnectorsIntegrationTest {
@@ -282,7 +281,7 @@ public class MirrorConnectorsIntegrationTest {
Duration . ofMillis ( CHECKPOINT_DURATION_MS ) ) ;
// Failback consumer group to primary cluster
Consumer < byte [ ] , byte [ ] > consumer2 = primary . kafka ( ) . createConsumer ( Collections . singletonMap ( "group.id" , "consumer-group-1" ) ) ;
consumer2 = primary . kafka ( ) . createConsumer ( Collections . singletonMap ( "group.id" , "consumer-group-1" ) ) ;
consumer2 . assign ( primaryOffsets . keySet ( ) ) ;
primaryOffsets . forEach ( consumer2 : : seek ) ;
consumer2 . poll ( Duration . ofMillis ( 500 ) ) ;
@ -316,6 +315,68 @@ public class MirrorConnectorsIntegrationTest {
@@ -316,6 +315,68 @@ public class MirrorConnectorsIntegrationTest {
backup . kafka ( ) . consume ( NUM_RECORDS_PRODUCED , 2 * RECORD_TRANSFER_DURATION_MS , "primary.test-topic-2" ) . count ( ) ) ;
}
@Test
public void testOneWayReplicationWithAutorOffsetSync1 ( ) throws InterruptedException {
// create consumers before starting the connectors so we don't need to wait for discovery
Consumer < byte [ ] , byte [ ] > consumer1 = primary . kafka ( ) . createConsumerAndSubscribeTo ( Collections . singletonMap (
"group.id" , "consumer-group-1" ) , "test-topic-1" ) ;
consumer1 . poll ( Duration . ofMillis ( 500 ) ) ;
consumer1 . commitSync ( ) ;
consumer1 . close ( ) ;
// enable automated consumer group offset sync
mm2Props . put ( "sync.group.offsets.enabled" , "true" ) ;
mm2Props . put ( "sync.group.offsets.interval.seconds" , "1" ) ;
// one way replication from primary to backup
mm2Props . put ( "backup->primary.enabled" , "false" ) ;
mm2Config = new MirrorMakerConfig ( mm2Props ) ;
waitUntilMirrorMakerIsRunning ( backup , mm2Config , "primary" , "backup" ) ;
// sleep 5 seconds to ensure the automated group offset sync is complete
time . sleep ( 5000 ) ;
// create a consumer at backup cluster with same consumer group Id to consume 1 topic
Consumer < byte [ ] , byte [ ] > consumer = backup . kafka ( ) . createConsumerAndSubscribeTo (
Collections . singletonMap ( "group.id" , "consumer-group-1" ) , "primary.test-topic-1" ) ;
ConsumerRecords records = consumer . poll ( Duration . ofMillis ( 500 ) ) ;
// the size of consumer record should be zero, because the offsets of the same consumer group
// have been automatically synchronized from primary to backup by the background job, so no
// more records to consume from the replicated topic by the same consumer group at backup cluster
assertEquals ( "consumer record size is not zero" , 0 , records . count ( ) ) ;
// now create a new topic in primary cluster
primary . kafka ( ) . createTopic ( "test-topic-2" , NUM_PARTITIONS ) ;
backup . kafka ( ) . createTopic ( "primary.test-topic-2" , 1 ) ;
// produce some records to the new topic in primary cluster
for ( int i = 0 ; i < NUM_RECORDS_PRODUCED ; i + + ) {
primary . kafka ( ) . produce ( "test-topic-2" , i % NUM_PARTITIONS , "key" , "message-1-" + i ) ;
}
// create a consumer at primary cluster to consume the new topic
consumer1 = primary . kafka ( ) . createConsumerAndSubscribeTo ( Collections . singletonMap (
"group.id" , "consumer-group-1" ) , "test-topic-2" ) ;
consumer1 . poll ( Duration . ofMillis ( 500 ) ) ;
consumer1 . commitSync ( ) ;
consumer1 . close ( ) ;
// sleep 5 seconds to ensure the automated group offset sync is complete
time . sleep ( 5000 ) ;
// create a consumer at backup cluster with same consumer group Id to consume old and new topic
consumer = backup . kafka ( ) . createConsumerAndSubscribeTo ( Collections . singletonMap (
"group.id" , "consumer-group-1" ) , "primary.test-topic-1" , "primary.test-topic-2" ) ;
records = consumer . poll ( Duration . ofMillis ( 500 ) ) ;
// similar reasoning as above, no more records to consume by the same consumer group at backup cluster
assertEquals ( "consumer record size is not zero" , 0 , records . count ( ) ) ;
consumer . close ( ) ;
}
private void deleteAllTopics ( EmbeddedKafkaCluster cluster ) {
Admin client = cluster . createAdminClient ( ) ;
try {