diff --git a/.gitignore b/.gitignore index f78622ce..3c57bd74 100644 --- a/.gitignore +++ b/.gitignore @@ -18,5 +18,10 @@ *.tar.gz *.rar +tmp_kafka* +.idea +target +*.iml +.DS_Store # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml hs_err_pid* \ No newline at end of file diff --git a/component/input-transport/src/main/java/org/wso2/siddhi/extension/input/transport/kafka/KafkaSource.java b/component/input-transport/src/main/java/org/wso2/siddhi/extension/input/transport/kafka/KafkaSource.java deleted file mode 100644 index ddbe71f5..00000000 --- a/component/input-transport/src/main/java/org/wso2/siddhi/extension/input/transport/kafka/KafkaSource.java +++ /dev/null @@ -1,173 +0,0 @@ -/* - * Copyright (c) 2017, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.wso2.siddhi.extension.input.transport.kafka; - -import org.apache.log4j.Logger; -import org.wso2.siddhi.annotation.Example; -import org.wso2.siddhi.annotation.Extension; -import org.wso2.siddhi.core.config.ExecutionPlanContext; -import org.wso2.siddhi.core.exception.ConnectionUnavailableException; -import org.wso2.siddhi.core.stream.input.source.Source; -import org.wso2.siddhi.core.stream.input.source.SourceEventListener; -import org.wso2.siddhi.core.util.config.ConfigReader; -import org.wso2.siddhi.core.util.transport.OptionHolder; - -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.ScheduledExecutorService; - -/** - * This processes the Kafka messages. - */ -@Extension( - name = "kafka", - namespace = "source", - description = "TBD", - examples = @Example(description = "TBD", syntax = "TBD") -) -public class KafkaSource extends Source { - - protected static final String SINGLE_THREADED = "single.thread"; - protected static final String TOPIC_WISE = "topic.wise"; - protected static final String PARTITION_WISE = "partition.wise"; - private static final Logger log = Logger.getLogger(KafkaSource.class); - private static final String ADAPTOR_SUBSCRIBER_TOPIC = "topic"; - private static final String ADAPTOR_SUBSCRIBER_GROUP_ID = "group.id"; - private static final String ADAPTOR_SUBSCRIBER_ZOOKEEPER_CONNECT_SERVERS = "bootstrap.servers"; - private static final String ADAPTOR_SUBSCRIBER_PARTITION_NO_LIST = "partition.no.list"; - private static final String ADAPTOR_OPTIONAL_CONFIGURATION_PROPERTIES = "optional.configuration"; - private static final String TOPIC_OFFSET_MAP = "topic.offset.map"; - private static final String THREADING_OPTION = "threading.option"; - private static final String HEADER_SEPARATOR = ","; - private static final String ENTRY_SEPARATOR = ":"; - private SourceEventListener sourceEventListener; - private ScheduledExecutorService executorService; - private OptionHolder optionHolder; - private ConsumerKafkaGroup consumerKafkaGroup; - private Map> topicOffsetMap = new HashMap<>(); - - private static Properties createConsumerConfig(String zkServerList, String groupId, String optionalConfigs) { - Properties props = new Properties(); - props.put(ADAPTOR_SUBSCRIBER_ZOOKEEPER_CONNECT_SERVERS, zkServerList); - if (null != groupId) { - props.put(ADAPTOR_SUBSCRIBER_GROUP_ID, groupId); - } - //If it stops heart-beating for a period of time longer than session.timeout.ms then it will be considered dead - // and its partitions will be assigned to another process - props.put("session.timeout.ms", "30000"); - props.put("enable.auto.commit", "false"); - props.put("auto.offset.reset", "earliest"); - props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); - props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); - - if (optionalConfigs != null) { - String[] optionalProperties = optionalConfigs.split(HEADER_SEPARATOR); - if (optionalProperties.length > 0) { - for (String header : optionalProperties) { - try { - String[] configPropertyWithValue = header.split(ENTRY_SEPARATOR, 2); - props.put(configPropertyWithValue[0], configPropertyWithValue[1]); - } catch (Exception e) { - log.warn("Optional property '" + header + "' is not defined in the correct format.", e); - } - } - } - } - return props; - } - - @Override - public void init(SourceEventListener sourceEventListener, OptionHolder optionHolder, - ConfigReader configReader, ExecutionPlanContext executionPlanContext) { - this.sourceEventListener = sourceEventListener; - this.optionHolder = optionHolder; - this.executorService = executionPlanContext.getScheduledExecutorService(); - executionPlanContext.getSnapshotService().addSnapshotable("kafka-sink", this); - } - - @Override - public void connect() throws ConnectionUnavailableException { - String zkServerList = optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_ZOOKEEPER_CONNECT_SERVERS); - String groupID = optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_GROUP_ID, null); - String threadingOption = optionHolder.validateAndGetStaticValue(THREADING_OPTION); - String partitionList; - String partitions[]; - partitionList = optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_PARTITION_NO_LIST, null); - partitions = (partitionList != null) ? partitionList.split(HEADER_SEPARATOR) : null; - String topicList = optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_TOPIC); - String topics[] = topicList.split(HEADER_SEPARATOR); - String optionalConfigs = optionHolder.validateAndGetStaticValue(ADAPTOR_OPTIONAL_CONFIGURATION_PROPERTIES, - null); - consumerKafkaGroup = new ConsumerKafkaGroup(topics, partitions, - KafkaSource.createConsumerConfig(zkServerList, groupID, - optionalConfigs), - topicOffsetMap, threadingOption, this.executorService); - consumerKafkaGroup.run(sourceEventListener); - } - - @Override - public void disconnect() { - if (consumerKafkaGroup != null) { - consumerKafkaGroup.shutdown(); - log.debug("Kafka Adapter disconnected for topic/s" + - optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_TOPIC)); - } - } - - @Override - public void destroy() { - consumerKafkaGroup = null; - } - - @Override - public void pause() { - if (consumerKafkaGroup != null) { - consumerKafkaGroup.pause(); - if (log.isDebugEnabled()) { - log.debug("Kafka Adapter paused for topic/s" + optionHolder.validateAndGetStaticValue - (ADAPTOR_SUBSCRIBER_TOPIC)); - } - } - } - - @Override - public void resume() { - if (consumerKafkaGroup != null) { - consumerKafkaGroup.resume(); - if (log.isDebugEnabled()) { - log.debug("Kafka Adapter resumed for topic/s" + optionHolder.validateAndGetStaticValue - (ADAPTOR_SUBSCRIBER_TOPIC)); - } - } - } - - @Override - public Map currentState() { - Map currentState = new HashMap<>(); - currentState.put(TOPIC_OFFSET_MAP, this.topicOffsetMap); - return currentState; - } - - @Override - public void restoreState(Map state) { - this.topicOffsetMap = (Map>) state.get(TOPIC_OFFSET_MAP); - consumerKafkaGroup.restore(topicOffsetMap); - } -} diff --git a/component/input-transport/src/test/java/org/wso2/siddhi/extension/input/transport/kafka/KafkaSourceTestCase.java b/component/input-transport/src/test/java/org/wso2/siddhi/extension/input/transport/kafka/KafkaSourceTestCase.java deleted file mode 100644 index 2462bff4..00000000 --- a/component/input-transport/src/test/java/org/wso2/siddhi/extension/input/transport/kafka/KafkaSourceTestCase.java +++ /dev/null @@ -1,751 +0,0 @@ -/* - * Copyright (c) 2017, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.wso2.siddhi.extension.input.transport.kafka; - -import kafka.admin.AdminUtils; -import kafka.common.TopicExistsException; -import kafka.server.KafkaConfig; -import kafka.server.KafkaServerStartable; -import kafka.utils.ZKStringSerializer$; -import kafka.utils.ZkUtils; -import org.I0Itec.zkclient.ZkClient; -import org.I0Itec.zkclient.ZkConnection; -import org.I0Itec.zkclient.exception.ZkTimeoutException; -import org.apache.commons.io.FileUtils; -import org.apache.curator.test.TestingServer; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.log4j.Logger; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.wso2.siddhi.core.ExecutionPlanRuntime; -import org.wso2.siddhi.core.SiddhiManager; -import org.wso2.siddhi.core.event.Event; -import org.wso2.siddhi.core.stream.input.source.Source; -import org.wso2.siddhi.core.stream.output.StreamCallback; -import org.wso2.siddhi.core.util.EventPrinter; -import org.wso2.siddhi.core.util.persistence.InMemoryPersistenceStore; -import org.wso2.siddhi.core.util.persistence.PersistenceStore; -import org.wso2.siddhi.extension.input.mapper.text.TextSourceMapper; -import org.wso2.siddhi.extension.output.mapper.text.TextSinkMapper; -import org.wso2.siddhi.query.api.ExecutionPlan; -import org.wso2.siddhi.query.api.annotation.Annotation; -import org.wso2.siddhi.query.api.definition.Attribute; -import org.wso2.siddhi.query.api.definition.StreamDefinition; -import org.wso2.siddhi.query.api.execution.query.Query; -import org.wso2.siddhi.query.api.execution.query.input.stream.InputStream; -import org.wso2.siddhi.query.api.execution.query.selection.Selector; -import org.wso2.siddhi.query.api.expression.Variable; - -import java.io.File; -import java.io.IOException; -import java.rmi.RemoteException; -import java.util.Collection; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class KafkaSourceTestCase { - private static final Logger log = Logger.getLogger(KafkaSourceTestCase.class); - private static TestingServer zkTestServer; - private static KafkaServerStartable kafkaServer; - private static ExecutorService executorService; - private static final String kafkaLogDir = "tmp_kafka_dir"; - private volatile int count; - private volatile boolean eventArrived; - - @BeforeClass - public static void init() throws Exception { - try { - executorService = Executors.newFixedThreadPool(5); - cleanLogDir(); - setupKafkaBroker(); - Thread.sleep(3000); - } catch (Exception e) { - throw new RemoteException("Exception caught when starting server", e); - } - } - - @Before - public void init2() { - count = 0; - eventArrived = false; - } - - @Test - public void testKafkaMultipleTopicPartitionPartitionWiseSubscription() throws InterruptedException { - try { - log.info("Creating test for multiple topics and partitions and thread partition wise"); - String topics[] = new String[]{"kafka_topic", "kafka_topic2"}; - createTopic(topics, 2); - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setExtension("source.mapper:text", TextSourceMapper.class); - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime( - "@Plan:name('TestExecutionPlan') " + - "define stream BarStream (symbol string, price float, volume long); " + - "@info(name = 'query1') " + - "@source(type='kafka', topic='kafka_topic,kafka_topic2', group.id='test', " + - "threading.option='partition.wise', bootstrap.servers='localhost:9092', " + - "partition.no.list='0,1', " + - "@map(type='text'))" + - "Define stream FooStream (symbol string, price float, volume long);" + - "from FooStream select symbol, price, volume insert into BarStream;"); - executionPlanRuntime.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - for (Event event : events) { - log.info(event); - eventArrived = true; - count++; - switch (count) { - case 1: - assertEquals(0, event.getData(2)); - break; - case 2: - assertEquals(0, event.getData(2)); - break; - case 3: - assertEquals(1, event.getData(2)); - break; - case 4: - assertEquals(1, event.getData(2)); - break; - default: - org.junit.Assert.fail(); - } - } - - } - }); - executionPlanRuntime.start(); - Thread.sleep(2000); - kafkaPublisher(topics, 2, 2); - Thread.sleep(5000); - assertEquals(4, count); - assertTrue(eventArrived); - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - log.warn("No zookeeper may not be available.", ex); - } - } - - @Test - public void testAKafkaPauseAndResume() throws InterruptedException { - try { - log.info("Test to verify the pause and resume functionality of Kafka source"); - String topics[] = new String[]{"kafka_topic3"}; - createTopic(topics, 2); - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setExtension("source.mapper:text", TextSourceMapper.class); - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime( - "@Plan:name('TestExecutionPlan') " + - "define stream BarStream (symbol string, price float, volume long); " + - "@info(name = 'query1') " + - "@source(type='kafka', topic='kafka_topic3', group.id='test1', threading" + - ".option='partition.wise', " + - "bootstrap.servers='localhost:9092', partition.no.list='0,1', " + - "@map(type='text'))" + - "Define stream FooStream (symbol string, price float, volume long);" + - "from FooStream select symbol, price, volume insert into BarStream;"); - executionPlanRuntime.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - for (Event event : events) { - log.info(event); - eventArrived = true; - count++; - } - - } - }); - executionPlanRuntime.start(); - Future eventSender = executorService.submit(new Runnable() { - @Override - public void run() { - kafkaPublisher(topics, 2, 4); - } - }); - while (!eventSender.isDone()) { - Thread.sleep(1000); - } - Thread.sleep(2000); - assertEquals(4, count); - assertTrue(eventArrived); - - Collection> sources = executionPlanRuntime.getSources(); - // pause the transports - sources.forEach(e -> e.forEach(Source::pause)); - - init2(); - eventSender = executorService.submit(new Runnable() { - @Override - public void run() { - kafkaPublisher(topics, 2, 4); - } - }); - while (!eventSender.isDone()) { - Thread.sleep(1000); - } - Thread.sleep(5000); - assertFalse(eventArrived); - - // resume the transports - sources.forEach(e -> e.forEach(Source::resume)); - Thread.sleep(2000); - assertEquals(4, count); - assertTrue(eventArrived); - - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - log.warn("No zookeeper may not be available.", ex); - } - } - - @Test - public void testRecoveryOnFailureOfSingleNodeWithKafka() throws InterruptedException { - try { - log.info("Test to verify recovering process of a Siddhi node on a failure when Kafka is the event source"); - String topics[] = new String[]{"kafka_topic4"}; - createTopic(topics, 1); - PersistenceStore persistenceStore = new InMemoryPersistenceStore(); - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setPersistenceStore(persistenceStore); - siddhiManager.setExtension("source.mapper:text", TextSourceMapper.class); - - String query = "@Plan:name('TestExecutionPlan') " + - "define stream BarStream (count long); " + - "@info(name = 'query1') " + - "@source(type='kafka', topic='kafka_topic4', group.id='test', " + - "threading.option='topic.wise', bootstrap.servers='localhost:9092', partition.no.list='0', " + - "@map(type='text'))" + - "Define stream FooStream (symbol string, price float, volume long);" + - "from FooStream select count(symbol) as count insert into BarStream;"; - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime(query); - executionPlanRuntime.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - for (Event event : events) { - eventArrived = true; - log.info(event); - count = Math.toIntExact((long) event.getData(0)); - } - - } - }); - - // start publishing events to Kafka - Future eventSender = executorService.submit(new Runnable() { - @Override - public void run() { - kafkaPublisher(topics, 1, 50, 1000); - } - }); - Thread.sleep(2000); - // start the execution plan - executionPlanRuntime.start(); - - // wait for some time - Thread.sleep(28000); - // initiate a checkpointing task - Future perisistor = executionPlanRuntime.persist(); - // waits till the checkpointing task is done - while (!perisistor.isDone()) { - Thread.sleep(100); - } - // let few more events to be published - Thread.sleep(5000); - // initiate a execution plan shutdown - to demonstrate a node failure - executionPlanRuntime.shutdown(); - // let few events to be published while the execution plan is down - Thread.sleep(5000); - // recreate the execution plan - executionPlanRuntime = siddhiManager.createExecutionPlanRuntime(query); - executionPlanRuntime.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - for (Event event : events) { - eventArrived = true; - log.info(event); - count = Math.toIntExact((long) event.getData(0)); - } - - } - }); - // start the execution plan - executionPlanRuntime.start(); - // immediately trigger a restore from last revision - executionPlanRuntime.restoreLastRevision(); - Thread.sleep(5000); - - // waits till all the events are published - while (!eventSender.isDone()) { - Thread.sleep(2000); - } - - Thread.sleep(20000); - assertTrue(eventArrived); - // assert the count - assertEquals(50, count); - - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - log.warn("No zookeeper may not be available.", ex); - } - } - - @Test - public void testRecoveryOnFailureOfMultipleNodeWithKafka() throws InterruptedException { - try { - log.info("Test to verify recovering process of multiple Siddhi nodes on a failure when Kafka is the event" - + " source"); - String topics[] = new String[]{"kafka_topic5", "kafka_topic6"}; - createTopic(topics, 1); - // 1st node - PersistenceStore persistenceStore = new InMemoryPersistenceStore(); - SiddhiManager siddhiManager1 = new SiddhiManager(); - siddhiManager1.setPersistenceStore(persistenceStore); - siddhiManager1.setExtension("inputmapper:text", TextSourceMapper.class); - - // 2nd node - PersistenceStore persistenceStore1 = new InMemoryPersistenceStore(); - SiddhiManager siddhiManager2 = new SiddhiManager(); - siddhiManager2.setPersistenceStore(persistenceStore1); - siddhiManager2.setExtension("inputmapper:text", TextSourceMapper.class); - - String query1 = "@Plan:name('TestExecutionPlan') " + - "@sink(type='kafka', topic='kafka_topic6', bootstrap.servers='localhost:9092', partition" + - ".no='0', " + - "@map(type='text'))" + - "define stream BarStream (count long); " + - "@source(type='kafka', topic='kafka_topic5', group.id='test', " + - "threading.option='topic.wise', bootstrap.servers='localhost:9092', partition.no.list='0', " + - "@map(type='text'))" + - "Define stream FooStream (symbol string, price float, volume long);" + - "@info(name = 'query1') " + - "from FooStream select count(symbol) as count insert into BarStream;"; - - String query2 = "@Plan:name('TestExecutionPlan') " + - "define stream BarStream (count long); " + - "@source(type='kafka', topic='kafka_topic6', " + - "threading.option='topic.wise', bootstrap.servers='localhost:9092', partition.no.list='0', " + - "@map(type='text'))" + - "Define stream FooStream (number long);" + - "@info(name = 'query1') " + - "from FooStream select count(number) as count insert into BarStream;"; - - ExecutionPlanRuntime executionPlanRuntime1 = siddhiManager1.createExecutionPlanRuntime(query1); - ExecutionPlanRuntime executionPlanRuntime2 = siddhiManager2.createExecutionPlanRuntime(query2); - - executionPlanRuntime2.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - for (Event event : events) { - eventArrived = true; - log.info(event); - count = Math.toIntExact((long) event.getData(0)); - } - - } - }); - - // start the execution plan - executionPlanRuntime1.start(); - executionPlanRuntime2.start(); - // let it initialize - Thread.sleep(2000); - - // start publishing events to Kafka - Future eventSender = executorService.submit(new Runnable() { - @Override - public void run() { - kafkaPublisher(new String[]{"kafka_topic5"}, 1, 50, 1000); - } - }); - - // wait for some time - Thread.sleep(28000); - // initiate a checkpointing task - Future perisistor1 = executionPlanRuntime1.persist(); - Future perisistor2 = executionPlanRuntime2.persist(); - // waits till the checkpointing task is done - while (!perisistor1.isDone() && !perisistor2.isDone()) { - Thread.sleep(100); - } - // let few more events to be published - Thread.sleep(5000); - // initiate a execution plan shutdown - to demonstrate a node failure - executionPlanRuntime1.shutdown(); - executionPlanRuntime2.shutdown(); - // let few events to be published while the execution plan is down - Thread.sleep(5000); - // recreate the execution plan - executionPlanRuntime1 = siddhiManager1.createExecutionPlanRuntime(query1); - executionPlanRuntime2 = siddhiManager2.createExecutionPlanRuntime(query2); - executionPlanRuntime2.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - for (Event event : events) { - eventArrived = true; - log.info(event); - count = Math.toIntExact((long) event.getData(0)); - } - - } - }); - // start the execution plan - executionPlanRuntime1.start(); - executionPlanRuntime2.start(); - // immediately trigger a restore from last revision - executionPlanRuntime1.restoreLastRevision(); - executionPlanRuntime2.restoreLastRevision(); - Thread.sleep(5000); - - // waits till all the events are published - while (!eventSender.isDone()) { - Thread.sleep(2000); - } - - Thread.sleep(20000); - assertTrue(eventArrived); - // assert the count - assertEquals(50, count); - - executionPlanRuntime1.shutdown(); - executionPlanRuntime2.shutdown(); - } catch (ZkTimeoutException ex) { - log.warn("No zookeeper may not be available.", ex); - } - } - - // @Test - public void testKafkaMultipleTopicPartitionTopicWiseSubscription() throws InterruptedException { - try { - log.info("Creating test for multiple topics and partitions and thread topic wise"); - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setExtension("source.mapper:text", TextSourceMapper.class); - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime( - "@Plan:name('TestExecutionPlan') " + - "define stream BarStream (symbol string, price float, volume long); " + - "@info(name = 'query1') " + - "@source(type='kafka', topic='kafka_topic,kafka_topic2', group.id='test', " + - "threading.option='topic.wise', bootstrap.servers='localhost:9092', " + - "partition.no.list='0,1', @map(type='text'))" + - "Define stream FooStream (symbol string, price float, volume long);" + - "from FooStream select symbol, price, volume insert into BarStream;"); - executionPlanRuntime.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - for (Event event : events) { - log.info(event); - } - } - }); - executionPlanRuntime.start(); - Thread.sleep(20000); - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - log.warn("No zookeeper may not be available.", ex); - } - } - - // @Test - public void testKafkaMultipleTopicPartitionSingleThreadSubscription() throws InterruptedException { - try { - log.info("Creating test for multiple topics and partitions on single thread"); - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setExtension("source.mapper:text", TextSourceMapper.class); - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime( - "@Plan:name('TestExecutionPlan') " + - "define stream BarStream (symbol string, price float, volume long); " + - "@info(name = 'query1') " + - "@source(type='kafka', topic='kafka_topic,kafka_topic2', group.id='test', " + - "threading.option='single.thread', bootstrap.servers='localhost:9092', " + - "partition.no.list='0,1', " + - "@map(type='text'))" + - "Define stream FooStream (symbol string, price float, volume long);" + - "from FooStream select symbol, price, volume insert into BarStream;"); - executionPlanRuntime.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - for (Event event : events) { - log.info(event); - } - } - }); - executionPlanRuntime.start(); - Thread.sleep(20000); - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - log.warn("No zookeeper may not be available.", ex); - } - } - - // @Test - public void testKafkaSingleTopicSubscriptionWithPartition() throws InterruptedException { - try { - log.info("Creating test for single topic with multiple partitions on single thread"); - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setExtension("source.mapper:text", TextSourceMapper.class); - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime( - "@Plan:name('TestExecutionPlan') " + - "define stream BarStream (symbol string, price float, volume long); " + - "@info(name = 'query1') " + - "@source(type='kafka', topic='kafka_topic', group.id='test', threading.option='single" + - ".thread', " + - "bootstrap.servers='localhost:9092', partition.no.list='0,1', " + - "@map(type='text'))" + - "Define stream FooStream (symbol string, price float, volume long);" + - "from FooStream select symbol, price, volume insert into BarStream;"); - executionPlanRuntime.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - for (Event event : events) { - log.info(event); - } - } - }); - executionPlanRuntime.start(); - Thread.sleep(20000); - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - log.warn("No zookeeper may not be available.", ex); - } - } - - // @Test - public void testCreatingKafkaSubscriptionWithoutPartition() throws InterruptedException { - try { - log.info("Creating test for multiple topic with no partitions on single thread"); - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setExtension("source.mapper:text", TextSourceMapper.class); - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime( - "@Plan:name('TestExecutionPlan') " + - "define stream BarStream (symbol string, price float, volume long); " + - "@info(name = 'query1') " + - "@source(type='kafka', topic='simple_topic,simple_topic2', group.id='test', " + - "threading.option='single.thread', bootstrap.servers='localhost:9092', " + - "@map(type='text'))" + - "Define stream FooStream (symbol string, price float, volume long);" + - "from FooStream select symbol, price, volume insert into BarStream;"); - executionPlanRuntime.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - EventPrinter.print(events); - } - }); - executionPlanRuntime.start(); - Thread.sleep(20000); - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - log.warn("No zookeeper may not be available.", ex); - } - } - - // @Test - public void testCreatingFullKafkaEventFlow() throws InterruptedException { - Runnable kafkaReceiver = new KafkaFlow(); - Thread t1 = new Thread(kafkaReceiver); - t1.start(); - Thread.sleep(35000); - } - - //---- private methods -------- - private static void setupKafkaBroker() { - try { - // mock zookeeper - zkTestServer = new TestingServer(2181); - // mock kafka - Properties props = new Properties(); - props.put("broker.id", "0"); - props.put("host.name", "localhost"); - props.put("port", "9092"); - props.put("log.dir", kafkaLogDir); - props.put("zookeeper.connect", zkTestServer.getConnectString()); - props.put("replica.socket.timeout.ms", "30000"); - props.put("delete.topic.enable", "true"); - KafkaConfig config = new KafkaConfig(props); - kafkaServer = new KafkaServerStartable(config); - kafkaServer.startup(); - } catch (Exception e) { - log.error("Error running local Kafka broker / Zookeeper", e); - } - } - - private void createTopic(String topics[], int numOfPartitions) { - ZkClient zkClient = new ZkClient(zkTestServer.getConnectString(), 30000, 30000, ZKStringSerializer$.MODULE$); - ZkConnection zkConnection = new ZkConnection(zkTestServer.getConnectString()); - ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false); - for (String topic : topics) { - try { - AdminUtils.createTopic(zkUtils, topic, numOfPartitions, 1, new Properties()); - } catch (TopicExistsException e) { - log.warn("topic exists for: " + topic); - } - } - zkClient.close(); - } - - @AfterClass - public static void stopKafkaBroker() { - try { - if (kafkaServer != null) { - kafkaServer.shutdown(); - } - Thread.sleep(5000); - if (zkTestServer != null) { - zkTestServer.stop(); - } - Thread.sleep(5000); - cleanLogDir(); - } catch (InterruptedException e) { - log.error(e.getMessage(), e); - } catch (IOException e) { - log.error("Error shutting down Kafka broker / Zookeeper", e); - } - } - - private static void cleanLogDir() { - try { - File f = new File(kafkaLogDir); - FileUtils.deleteDirectory(f); - } catch (IOException e) { - log.error("Failed to clean up: " + e); - } - } - - private void kafkaPublisher(String topics[], int numOfPartitions, int numberOfEvents, long sleep) { - - Properties props = new Properties(); - props.put("bootstrap.servers", "localhost:9092"); - props.put("acks", "all"); - props.put("retries", 0); - props.put("batch.size", 16384); - props.put("linger.ms", 1); - props.put("buffer.memory", 33554432); - props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); - props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); - Producer producer = new KafkaProducer(props); - for (int i = 0; i < numberOfEvents; i++) { - String msg = "wso2,12.5," + i; - try { - Thread.sleep(sleep); - } catch (InterruptedException e) { - } - for (String topic : topics) { - if (numOfPartitions > 1) { - log.info("producing: " + msg + " into partition: " + (i % numOfPartitions)); - producer.send(new ProducerRecord<>(topic, String.valueOf(i % numOfPartitions), msg)); - } else { - log.info("Produced " + i); - producer.send(new ProducerRecord<>(topic, msg)); - } - } - } - producer.close(); - } - - private void kafkaPublisher(String topics[], int numOfPartitions, int numberOfEvents) { - kafkaPublisher(topics, numOfPartitions, numberOfEvents, 1000); - } - - private class KafkaFlow implements Runnable { - @Override - public void run() { - try { - StreamDefinition inputDefinition = StreamDefinition.id("FooStream") - .attribute("symbol", Attribute.Type.STRING) - .attribute("price", Attribute.Type.FLOAT) - .attribute("volume", Attribute.Type.INT) - .annotation(Annotation.annotation("source") - .element("type", "kafka") - .element("topic", "receiver_topic") - .element("threads", "1") - .element("partition.no.list", "0,1") - .element("group.id", "group1") - .element("bootstrap.servers", "localhost:9092") - .annotation(Annotation.annotation("map") - .element("type", "text"))); - - StreamDefinition outputDefinition = StreamDefinition.id("BarStream") - .attribute("symbol", Attribute.Type.STRING) - .attribute("price", Attribute.Type.FLOAT) - .attribute("volume", Attribute.Type.INT) - .annotation(Annotation.annotation("sink") - .element("type", "kafka") - .element("topic", "publisher_topic") - .element("partition.no", "0") - .element("bootstrap.servers", "localhost:9092") - .annotation(Annotation.annotation("map") - .element("type", "text"))); - - Query query = Query.query(); - query.from( - InputStream.stream("FooStream") - ); - query.select( - Selector.selector().select(new Variable("symbol")).select(new Variable("price")).select(new - Variable("volume")) - ); - query.insertInto("BarStream"); - - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setExtension("source.mapper:text", TextSourceMapper.class); - siddhiManager.setExtension("sink.mapper:text", TextSinkMapper.class); - - ExecutionPlan executionPlan = new ExecutionPlan("ep1"); - executionPlan.defineStream(inputDefinition); - executionPlan.defineStream(outputDefinition); - executionPlan.addQuery(query); - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime(executionPlan); - - executionPlanRuntime.addCallback("FooStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - log.info("Printing received events !!"); - EventPrinter.print(events); - } - }); - executionPlanRuntime.addCallback("BarStream", new StreamCallback() { - @Override - public void receive(Event[] events) { - log.info("Printing publishing events !!"); - EventPrinter.print(events); - } - }); - executionPlanRuntime.start(); - Thread.sleep(30000); - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - log.warn("No zookeeper may not be available.", ex); - } catch (InterruptedException e) { - log.error(e.getMessage(), e); - } - } - } -} - diff --git a/component/output-transport/pom.xml b/component/output-transport/pom.xml deleted file mode 100644 index 11e84650..00000000 --- a/component/output-transport/pom.xml +++ /dev/null @@ -1,92 +0,0 @@ - - - - - - - org.wso2.extension.siddhi - siddhi-io-kafka - 4.0.0-M5-SNAPSHOT - ../../pom.xml - - 4.0.0 - bundle - - siddhi-io-kafka-output - Siddhi Extension - Kafka Input Transport - - - - - org.wso2.siddhi - siddhi-core - - - org.wso2.siddhi - siddhi-query-api - - - org.wso2.siddhi - siddhi-annotations - - - org.apache.log4j.wso2 - log4j - - - junit - junit - test - - - org.apache.kafka - kafka_2.10 - - - org.wso2.siddhi - siddhi-extension-text-output-mapper - - - - - - - org.apache.felix - maven-bundle-plugin - true - - - ${project.artifactId} - ${project.artifactId} - - org.wso2.siddhi.extension.output.transport.kafka.* - - - *;resolution:=optional - - - META-INF=target/classes/META-INF - - * - - - - - - \ No newline at end of file diff --git a/component/output-transport/src/test/java/org/wso2/siddhi/extension/output/transport/kafka/KafkaSinkTestCase.java b/component/output-transport/src/test/java/org/wso2/siddhi/extension/output/transport/kafka/KafkaSinkTestCase.java deleted file mode 100644 index 9be2f04b..00000000 --- a/component/output-transport/src/test/java/org/wso2/siddhi/extension/output/transport/kafka/KafkaSinkTestCase.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Copyright (c) 2017, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.wso2.siddhi.extension.output.transport.kafka; - -import org.I0Itec.zkclient.exception.ZkTimeoutException; -import org.apache.log4j.Logger; -import org.wso2.siddhi.core.ExecutionPlanRuntime; -import org.wso2.siddhi.core.SiddhiManager; -import org.wso2.siddhi.core.stream.input.InputHandler; -import org.wso2.siddhi.extension.output.mapper.text.TextSinkMapper; - -public class KafkaSinkTestCase { - static final Logger LOG = Logger.getLogger(KafkaSinkTestCase.class); - - // @Test - public void testPublisherWithKafkaTransport() throws InterruptedException { - LOG.info("Creating test for publishing events for static topic with a partition"); - try { - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setExtension("sinkMapper:text", TextSinkMapper.class); - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime( - "@Plan:name('TestExecutionPlan') " + - "define stream FooStream (LOGmbol string, price float, volume long); " + - "@info(name = 'query1') " + - "@sink(type='kafka', topic='kafka_topic', bootstrap.servers='localhost:9092', partition" + - ".no='0', " + - "@map(type='text'))" + - "Define stream BarStream (symbol string, price float, volume long);" + - "from FooStream select symbol, price, volume insert into BarStream;"); - InputHandler fooStream = executionPlanRuntime.getInputHandler("FooStream"); - executionPlanRuntime.start(); - fooStream.send(new Object[]{"WSO2", 55.6f, 100L}); - fooStream.send(new Object[]{"IBM", 75.6f, 100L}); - fooStream.send(new Object[]{"WSO2", 57.6f, 100L}); - Thread.sleep(10000); - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - LOG.warn("No zookeeper may not be available.", ex); - } - } - - // @Test - public void testPublisherWithKafkaTransportWithDynamicTopic() throws InterruptedException { - LOG.info("Creating test for publishing events for dynamic topic without partition"); - try { - SiddhiManager siddhiManager = new SiddhiManager(); - siddhiManager.setExtension("sinkMapper:text", TextSinkMapper.class); - ExecutionPlanRuntime executionPlanRuntime = siddhiManager.createExecutionPlanRuntime( - "@Plan:name('TestExecutionPlan') " + - "define stream FooStream (symbol string, price float, volume long); " + - "@info(name = 'query1') " + - "@sink(type='kafka', topic='{{symbol}}', bootstrap.servers='localhost:9092', " + - "@map(type='text'))" + - "Define stream BarStream (symbol string, price float, volume long);" + - "from FooStream select symbol, price, volume insert into BarStream; " - ); - InputHandler fooStream = executionPlanRuntime.getInputHandler("FooStream"); - executionPlanRuntime.start(); - Thread.sleep(2000); - fooStream.send(new Object[]{"simple_topic", 55.6f, 100L}); - fooStream.send(new Object[]{"simple_topic", 75.6f, 100L}); - fooStream.send(new Object[]{"simple_topic", 57.6f, 100L}); - Thread.sleep(5000); - executionPlanRuntime.shutdown(); - } catch (ZkTimeoutException ex) { - LOG.warn("No zookeeper may not be available.", ex); - } - } -} diff --git a/component/output-transport/src/test/resources/log4j.properties b/component/output-transport/src/test/resources/log4j.properties deleted file mode 100644 index 2c00a08c..00000000 --- a/component/output-transport/src/test/resources/log4j.properties +++ /dev/null @@ -1,31 +0,0 @@ -# -# Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. -# -# WSO2 Inc. licenses this file to you under the Apache License, -# Version 2.0 (the "License"); you may not use this file except -# in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# -# For the general syntax of property based configuration files see the -# documenation of org.apache.log4j.PropertyConfigurator. -# The root category uses the appender called A1. Since no priority is -# specified, the root category assumes the default priority for root -# which is DEBUG in log4j. The root category is the only category that -# has a default priority. All other categories need not be assigned a -# priority in which case they inherit their priority from the -# hierarchy. -#log4j.rootLogger=DEBUG, stdout -log4j.rootLogger=INFO, stdout -log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=%m%n -#log4j.appender.stdout.layout.ConversionPattern=[%t] %-5p %c %x - %m%n diff --git a/component/input-transport/pom.xml b/component/pom.xml similarity index 81% rename from component/input-transport/pom.xml rename to component/pom.xml index 40514e58..37a32d53 100644 --- a/component/input-transport/pom.xml +++ b/component/pom.xml @@ -20,17 +20,16 @@ - org.wso2.extension.siddhi - siddhi-io-kafka + org.wso2.extension.siddhi.io.kafka + siddhi-io-kafka-parent 4.0.0-M5-SNAPSHOT - ../../pom.xml + ../pom.xml 4.0.0 bundle - siddhi-io-kafka-input - Siddhi Extension - Kafka Input Transport - + siddhi-io-kafka + Siddhi Extension - Kafka Transport @@ -50,8 +49,8 @@ log4j - junit - junit + org.testng + testng test @@ -73,19 +72,13 @@ commons-io test - - - - org.wso2.siddhi - siddhi-extension-kafka-output-transport - org.wso2.siddhi - siddhi-extension-text-input-mapper + siddhi-extension-xml-input-mapper org.wso2.siddhi - siddhi-extension-text-output-mapper + siddhi-extension-xml-output-mapper @@ -100,7 +93,7 @@ ${project.artifactId} ${project.artifactId} - org.wso2.siddhi.extension.input.transport.kafka.* + org.wso2.extension.siddhi.io.kafka.* *;resolution:=optional diff --git a/component/output-transport/src/main/java/org/wso2/siddhi/extension/output/transport/kafka/KafkaSink.java b/component/src/main/java/org.wso2.extension.siddhi.io.kafka/sink/KafkaSink.java similarity index 53% rename from component/output-transport/src/main/java/org/wso2/siddhi/extension/output/transport/kafka/KafkaSink.java rename to component/src/main/java/org.wso2.extension.siddhi.io.kafka/sink/KafkaSink.java index 4dc4ccb9..ebf04a28 100644 --- a/component/output-transport/src/main/java/org/wso2/siddhi/extension/output/transport/kafka/KafkaSink.java +++ b/component/src/main/java/org.wso2.extension.siddhi.io.kafka/sink/KafkaSink.java @@ -16,7 +16,7 @@ * under the License. */ -package org.wso2.siddhi.extension.output.transport.kafka; +package org.wso2.extension.siddhi.io.kafka.sink; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; @@ -24,7 +24,9 @@ import org.apache.log4j.Logger; import org.wso2.siddhi.annotation.Example; import org.wso2.siddhi.annotation.Extension; -import org.wso2.siddhi.core.config.ExecutionPlanContext; +import org.wso2.siddhi.annotation.Parameter; +import org.wso2.siddhi.annotation.util.DataType; +import org.wso2.siddhi.core.config.SiddhiAppContext; import org.wso2.siddhi.core.exception.ConnectionUnavailableException; import org.wso2.siddhi.core.stream.output.sink.Sink; import org.wso2.siddhi.core.util.config.ConfigReader; @@ -38,20 +40,71 @@ import java.util.concurrent.ScheduledExecutorService; /** - * Kafka output transport class. + * his class implements a Kafka sink to publish Siddhi events to a kafka cluster. */ @Extension( name = "kafka", namespace = "sink", - description = "TBD", - examples = @Example(description = "TBD", syntax = "TBD") + description = "The Kafka Sink publishes records to a topic with a partition for a Kafka cluster which are in " + + "format such as `text`, `XML` and `JSON`.\n" + + "The Kafka Sink will create the default partition for a given topic, if the topic is not already " + + "been created in the Kafka cluster. The publishing topic and partition can be a dynamic value taken" + + " from the Siddhi event", + parameters = { + @Parameter(name = "topic", + description = "The topic list which the sink should publish to. Only one topic should be " + + "given", + type = {DataType.STRING}), + @Parameter(name = "partition.no", + description = "The partition number for the given topic. Only one partition id can be " + + "defined. If this is not defined, the sink will be publishing to the topic's " + + "default partition.", + type = {DataType.INT}, + optional = true, + defaultValue = "0"), + @Parameter(name = "bootstrap.servers", + description = "This should contain the kafka server list which the kafka sink should be " + + "publishing to. This should be given in comma separated values. " + + "eg: 'localhost:9092,localhost:9093' ", + type = {DataType.STRING}) + }, + examples = { + @Example( + description = "The following query will publish to 'topic_with_partitions' and to its 0th " + + "partition", + syntax = "@App:name('TestExecutionPlan') \n" + + "define stream FooStream (symbol string, price float, volume long); \n" + + "@info(name = 'query1') \n" + + "@sink(" + + "type='kafka', " + + "topic='topic_with_partitions', " + + "partition.no='0', " + + "bootstrap.servers='localhost:9092', " + + "@map(type='xml'))" + + "Define stream BarStream (symbol string, price float, volume long);\n" + + "from FooStream select symbol, price, volume insert into BarStream;\n"), + @Example( + description = "The following query will publish dynamic topic and partitions which will be " + + "taken from the siddhi event. partition number value will be taken from the " + + "'volume' attribute and the topic value will be taken from the 'symbol' attribute.", + syntax = "@App:name('TestExecutionPlan') \n" + + "define stream FooStream (symbol string, price float, volume long); \n" + + "@info(name = 'query1') \n" + + "@sink(" + + "type='kafka', " + + "topic='{{symbol}}', " + + "partition.no='{{volume}}', " + + "bootstrap.servers='localhost:9092', " + + "@map(type='xml'))" + + "Define stream BarStream (symbol string, price float, volume long); \n" + + "from FooStream select symbol, price, volume insert into BarStream; \n")} ) public class KafkaSink extends Sink { private ScheduledExecutorService executorService; private Producer producer; private Option topicOption = null; - private String kafkaConnect; + private String bootstrapServers; private String optionalConfigs; private Option partitionOption; @@ -66,18 +119,18 @@ public class KafkaSink extends Sink { @Override protected void init(StreamDefinition outputStreamDefinition, OptionHolder optionHolder, - ConfigReader sinkConfigReader, ExecutionPlanContext executionPlanContext) { - kafkaConnect = optionHolder.validateAndGetStaticValue(KAFKA_BROKER_LIST); + ConfigReader sinkConfigReader, SiddhiAppContext siddhiAppContext) { + bootstrapServers = optionHolder.validateAndGetStaticValue(KAFKA_BROKER_LIST); optionalConfigs = optionHolder.validateAndGetStaticValue(KAFKA_OPTIONAL_CONFIGURATION_PROPERTIES, null); topicOption = optionHolder.validateAndGetOption(KAFKA_PUBLISH_TOPIC); partitionOption = optionHolder.getOrCreateOption(KAFKA_PARTITION_NO, null); - executorService = executionPlanContext.getScheduledExecutorService(); + executorService = siddhiAppContext.getScheduledExecutorService(); } @Override public void connect() throws ConnectionUnavailableException { Properties props = new Properties(); - props.put("bootstrap.servers", kafkaConnect); + props.put("bootstrap.servers", bootstrapServers); props.put("acks", "all"); props.put("retries", 0); props.put("batch.size", 16384); diff --git a/component/input-transport/src/main/java/org/wso2/siddhi/extension/input/transport/kafka/ConsumerKafkaGroup.java b/component/src/main/java/org.wso2.extension.siddhi.io.kafka/source/ConsumerKafkaGroup.java similarity index 91% rename from component/input-transport/src/main/java/org/wso2/siddhi/extension/input/transport/kafka/ConsumerKafkaGroup.java rename to component/src/main/java/org.wso2.extension.siddhi.io.kafka/source/ConsumerKafkaGroup.java index 67570d93..ab29732d 100644 --- a/component/input-transport/src/main/java/org/wso2/siddhi/extension/input/transport/kafka/ConsumerKafkaGroup.java +++ b/component/src/main/java/org.wso2.extension.siddhi.io.kafka/source/ConsumerKafkaGroup.java @@ -16,7 +16,7 @@ * under the License. */ -package org.wso2.siddhi.extension.input.transport.kafka; +package org.wso2.extension.siddhi.io.kafka.source; import org.apache.log4j.Logger; import org.wso2.siddhi.core.stream.input.source.SourceEventListener; @@ -33,7 +33,7 @@ * This processes the Kafka messages using a thread pool. */ public class ConsumerKafkaGroup { - private static final Logger log = Logger.getLogger(ConsumerKafkaGroup.class); + private static final Logger LOG = Logger.getLogger(ConsumerKafkaGroup.class); private final String topics[]; private final String partitions[]; private final Properties props; @@ -74,9 +74,9 @@ void run(SourceEventListener sourceEventListener) { KafkaConsumerThread kafkaConsumerThread = new KafkaConsumerThread(sourceEventListener, topics, partitions, props, topicOffsetMap); kafkaConsumerThreadList.add(kafkaConsumerThread); - log.info("Kafka Consumer thread starting to listen on topic/s: " + Arrays.toString(topics) + + LOG.info("Kafka Consumer thread starting to listen on topic/s: " + Arrays.toString(topics) + " with partition/s: " + Arrays.toString(partitions)); - kafkaConsumerThread.run(); + executorService.submit(kafkaConsumerThread); } else if (KafkaSource.TOPIC_WISE.equals(threadingOption)) { for (String topic : topics) { KafkaConsumerThread kafkaConsumerThread = @@ -84,7 +84,7 @@ void run(SourceEventListener sourceEventListener) { topicOffsetMap); kafkaConsumerThreadList.add(kafkaConsumerThread); executorService.submit(kafkaConsumerThread); - log.info("Kafka Consumer thread starting to listen on topic: " + topic + + LOG.info("Kafka Consumer thread starting to listen on topic: " + topic + " with partition/s: " + Arrays.toString(partitions)); } } else if (KafkaSource.PARTITION_WISE.equals(threadingOption)) { @@ -95,13 +95,13 @@ void run(SourceEventListener sourceEventListener) { new String[]{partition}, props, topicOffsetMap); kafkaConsumerThreadList.add(kafkaConsumerThread); executorService.submit(kafkaConsumerThread); - log.info("Kafka Consumer thread starting to listen on topic: " + topic + + LOG.info("Kafka Consumer thread starting to listen on topic: " + topic + " with partition: " + partition); } } } } catch (Throwable t) { - log.error("Error while creating KafkaConsumerThread for topic/s: " + Arrays.toString(topics), t); + LOG.error("Error while creating KafkaConsumerThread for topic/s: " + Arrays.toString(topics), t); } } diff --git a/component/input-transport/src/main/java/org/wso2/siddhi/extension/input/transport/kafka/KafkaConsumerThread.java b/component/src/main/java/org.wso2.extension.siddhi.io.kafka/source/KafkaConsumerThread.java similarity index 97% rename from component/input-transport/src/main/java/org/wso2/siddhi/extension/input/transport/kafka/KafkaConsumerThread.java rename to component/src/main/java/org.wso2.extension.siddhi.io.kafka/source/KafkaConsumerThread.java index 29da1faf..121ce3a7 100644 --- a/component/input-transport/src/main/java/org/wso2/siddhi/extension/input/transport/kafka/KafkaConsumerThread.java +++ b/component/src/main/java/org.wso2.extension.siddhi.io.kafka/source/KafkaConsumerThread.java @@ -16,7 +16,7 @@ * under the License. */ -package org.wso2.siddhi.extension.input.transport.kafka; +package org.wso2.extension.siddhi.io.kafka.source; import org.apache.kafka.clients.consumer.CommitFailedException; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -64,6 +64,7 @@ public class KafkaConsumerThread implements Runnable { } for (String partition1 : partitions) { TopicPartition partition = new TopicPartition(topic, Integer.parseInt(partition1)); + LOG.error("Adding partition " + partition1 + " for topic: " + topic); partitionsList.add(partition); } LOG.info("Adding partitions " + Arrays.toString(partitions) + " for topic: " + topic); @@ -124,7 +125,7 @@ public void run() { ConsumerRecords records; try { consumerLock.lock(); - records = consumer.poll(100); + records = consumer.poll(1000); } finally { consumerLock.unlock(); } @@ -150,15 +151,15 @@ public void run() { } } } + } + + void shutdownConsumer() { try { consumerLock.lock(); consumer.close(); } finally { consumerLock.unlock(); } - } - - void shutdownConsumer() { inactive = true; } diff --git a/component/src/main/java/org.wso2.extension.siddhi.io.kafka/source/KafkaSource.java b/component/src/main/java/org.wso2.extension.siddhi.io.kafka/source/KafkaSource.java new file mode 100644 index 00000000..7d2c0f93 --- /dev/null +++ b/component/src/main/java/org.wso2.extension.siddhi.io.kafka/source/KafkaSource.java @@ -0,0 +1,336 @@ +/* + * Copyright (c) 2017, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. + * + * WSO2 Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.wso2.extension.siddhi.io.kafka.source; + +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.PartitionInfo; +import org.apache.log4j.Logger; +import org.wso2.siddhi.annotation.Example; +import org.wso2.siddhi.annotation.Extension; +import org.wso2.siddhi.annotation.Parameter; +import org.wso2.siddhi.annotation.util.DataType; +import org.wso2.siddhi.core.config.SiddhiAppContext; +import org.wso2.siddhi.core.exception.ConnectionUnavailableException; +import org.wso2.siddhi.core.stream.input.source.Source; +import org.wso2.siddhi.core.stream.input.source.SourceEventListener; +import org.wso2.siddhi.core.util.config.ConfigReader; +import org.wso2.siddhi.core.util.transport.OptionHolder; +import org.wso2.siddhi.query.api.exception.SiddhiAppValidationException; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ScheduledExecutorService; + +/** + * This class implements a Kafka source to receives events from a kafka cluster. + */ +@Extension( + name = "kafka", + namespace = "source", + description = "The Kafka Sink publishes records to a topic with a partition for a Kafka cluster which are in " + + "format such as `text`, `XML` and `JSON`.\n" + + "The Kafka Sink will create the default partition for a given topic, if the topic is not already " + + "been created in the Kafka cluster.", + parameters = { + @Parameter(name = "topic.list", + description = "The topic list which the source would be listening to. This should be given " + + "in comma separated values. eg: 'topic_one,topic_two' ", + type = {DataType.STRING}), + @Parameter(name = "partition.no.list", + description = "The partition number list for the given topic. This should be given in " + + "comma separated values. eg: '0,1,1' ", + type = {DataType.STRING}, + optional = true, + defaultValue = "0"), + @Parameter(name = "group.id", + description = "This is used to identify the Kafka source group. And sources with same " + + "topic and partition which are in the same group wont receive the same event", + type = {DataType.STRING}), + @Parameter(name = "threading.option", + description = "Each source can be run in either single thread or in multi threads. The " + + "threading options are `single.thread`, `topic.wise` and `partition.wise` ", + type = {DataType.STRING}), + @Parameter(name = "bootstrap.servers", + description = "This should contain the kafka server list which the kafka source should be " + + "listening to. This should be given in comma separated values. " + + "eg: 'localhost:9092,localhost:9093' ", + type = {DataType.STRING}) + }, + examples = { + @Example( + description = "The following query will listen to 'kafka_topic' and 'kafka_topic2' topics " + + "with 0 and 1 partitions. There will be a thread created for each topic and " + + "partition combination. The receiving xml events will be mapped to a siddhi event " + + "and will be send to the FooStream.", + syntax = "@App:name('TestExecutionPlan') \n" + + "define stream BarStream (symbol string, price float, volume long); \n" + + "@info(name = 'query1') \n" + + "@source(" + + "type='kafka', " + + "topic.list='kafka_topic,kafka_topic2', " + + "group.id='test', " + + "threading.option='partition.wise', " + + "bootstrap.servers='localhost:9092', " + + "partition.no.list='0,1', " + + "@map(type='xml'))\n" + + "Define stream FooStream (symbol string, price float, volume long);\n" + + "from FooStream select symbol, price, volume insert into BarStream;\n"), + @Example( + description = "The following query will listen to 'kafka_topic' topic for the default " + + "partition since there is no 'partition.no.list' is defined. There will be a only " + + "one thread created for the topic. The receiving xml events will be mapped to a " + + "siddhi event and will be send to the FooStream.", + syntax = "@App:name('TestExecutionPlan') \n" + + "define stream BarStream (symbol string, price float, volume long); \n" + + "@info(name = 'query1') \n" + + "@source(" + + "type='kafka', " + + "topic.list='kafka_topic', " + + "group.id='test', " + + "threading.option='single.thread', " + + "bootstrap.servers='localhost:9092', " + + "@map(type='xml'))\n" + + "Define stream FooStream (symbol string, price float, volume long);\n" + + "from FooStream select symbol, price, volume insert into BarStream;\n") + } +) +public class KafkaSource extends Source { + + protected static final String SINGLE_THREADED = "single.thread"; + protected static final String TOPIC_WISE = "topic.wise"; + protected static final String PARTITION_WISE = "partition.wise"; + private static final Logger LOG = Logger.getLogger(KafkaSource.class); + private static final String ADAPTOR_SUBSCRIBER_TOPIC = "topic.list"; + private static final String ADAPTOR_SUBSCRIBER_GROUP_ID = "group.id"; + private static final String ADAPTOR_SUBSCRIBER_ZOOKEEPER_CONNECT_SERVERS = "bootstrap.servers"; + private static final String ADAPTOR_SUBSCRIBER_PARTITION_NO_LIST = "partition.no.list"; + private static final String ADAPTOR_OPTIONAL_CONFIGURATION_PROPERTIES = "optional.configuration"; + private static final String TOPIC_OFFSET_MAP = "topic.offset.map"; + private static final String THREADING_OPTION = "threading.option"; + private static final String HEADER_SEPARATOR = ","; + private static final String ENTRY_SEPARATOR = ":"; + private SourceEventListener sourceEventListener; + private ScheduledExecutorService executorService; + private OptionHolder optionHolder; + private ConsumerKafkaGroup consumerKafkaGroup; + private Map> topicOffsetMap = new HashMap<>(); + private String bootstrapServers; + private String groupID; + private String threadingOption; + private String partitions[]; + private String topics[]; + private String optionalConfigs; + + private static Properties createConsumerConfig(String zkServerList, String groupId, String optionalConfigs) { + Properties props = new Properties(); + props.put(ADAPTOR_SUBSCRIBER_ZOOKEEPER_CONNECT_SERVERS, zkServerList); + props.put(ADAPTOR_SUBSCRIBER_GROUP_ID, groupId); + + //If it stops heart-beating for a period of time longer than session.timeout.ms then it will be considered dead + // and its partitions will be assigned to another process + props.put("session.timeout.ms", "30000"); + props.put("enable.auto.commit", "false"); + props.put("auto.offset.reset", "earliest"); + props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + + if (optionalConfigs != null) { + String[] optionalProperties = optionalConfigs.split(HEADER_SEPARATOR); + if (optionalProperties.length > 0) { + for (String header : optionalProperties) { + try { + String[] configPropertyWithValue = header.split(ENTRY_SEPARATOR, 2); + props.put(configPropertyWithValue[0], configPropertyWithValue[1]); + } catch (Exception e) { + LOG.warn("Optional property '" + header + "' is not defined in the correct format.", e); + } + } + } + } + return props; + } + + @Override + public void init(SourceEventListener sourceEventListener, OptionHolder optionHolder, ConfigReader configReader, + SiddhiAppContext siddhiAppContext) { + this.sourceEventListener = sourceEventListener; + this.optionHolder = optionHolder; + this.executorService = siddhiAppContext.getScheduledExecutorService(); + bootstrapServers = optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_ZOOKEEPER_CONNECT_SERVERS); + groupID = optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_GROUP_ID); + threadingOption = optionHolder.validateAndGetStaticValue(THREADING_OPTION); + String partitionList = optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_PARTITION_NO_LIST, null); + partitions = (partitionList != null) ? partitionList.split(HEADER_SEPARATOR) : null; + String topicList = optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_TOPIC); + topics = topicList.split(HEADER_SEPARATOR); + optionalConfigs = optionHolder.validateAndGetStaticValue(ADAPTOR_OPTIONAL_CONFIGURATION_PROPERTIES, null); + checkTopicsAvailableInCluster(); + checkPartitionsAvailableForTheTopicsInCluster(); + if (KafkaSource.PARTITION_WISE.equals(threadingOption) && null == partitions) { + throw new SiddhiAppValidationException("The threading option is 'partition.wise' but there are no " + + "partition numbers defined."); + } + siddhiAppContext.getSnapshotService().addSnapshotable("kafka-sink", this); + } + + @Override + public void connect() throws ConnectionUnavailableException { + consumerKafkaGroup = new ConsumerKafkaGroup(topics, partitions, + KafkaSource.createConsumerConfig(bootstrapServers, groupID, + optionalConfigs), + topicOffsetMap, threadingOption, this.executorService); + consumerKafkaGroup.run(sourceEventListener); + } + + @Override + public void disconnect() { + if (consumerKafkaGroup != null) { + consumerKafkaGroup.shutdown(); + LOG.debug("Kafka Adapter disconnected for topic/s" + + optionHolder.validateAndGetStaticValue(ADAPTOR_SUBSCRIBER_TOPIC)); + } + } + + @Override + public void destroy() { + consumerKafkaGroup = null; + } + + @Override + public void pause() { + if (consumerKafkaGroup != null) { + consumerKafkaGroup.pause(); + if (LOG.isDebugEnabled()) { + LOG.debug("Kafka Adapter paused for topic/s" + optionHolder.validateAndGetStaticValue + (ADAPTOR_SUBSCRIBER_TOPIC)); + } + } + } + + @Override + public void resume() { + if (consumerKafkaGroup != null) { + consumerKafkaGroup.resume(); + if (LOG.isDebugEnabled()) { + LOG.debug("Kafka Adapter resumed for topic/s" + optionHolder.validateAndGetStaticValue + (ADAPTOR_SUBSCRIBER_TOPIC)); + } + } + } + + @Override + public Map currentState() { + Map currentState = new HashMap<>(); + currentState.put(TOPIC_OFFSET_MAP, this.topicOffsetMap); + return currentState; + } + + @Override + public void restoreState(Map state) { + this.topicOffsetMap = (Map>) state.get(TOPIC_OFFSET_MAP); + consumerKafkaGroup.restore(topicOffsetMap); + } + + private void checkTopicsAvailableInCluster() { + Properties props = new Properties(); + props.put("bootstrap.servers", bootstrapServers); + props.put("group.id", "test-consumer-group"); + props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + KafkaConsumer consumer = new KafkaConsumer(props); + Map> testTopicList = consumer.listTopics(); + boolean topicsAvailable = true; + StringBuilder invalidTopics = new StringBuilder(""); + for (String topic : topics) { + boolean topicAvailable = false; + for (Map.Entry> entry : testTopicList.entrySet()) { + if (entry.getKey().equals(topic)) { + topicAvailable = true; + } + } + if (!topicAvailable) { + topicsAvailable = false; + if ("".equals(invalidTopics.toString())) { + invalidTopics.append(topic); + } else { + invalidTopics.append(',').append(topic); + } + LOG.warn("Topic, " + topic + " is not available."); + } + } + if (null != partitions && !topicsAvailable) { + String errorMessage = "Topic/s " + invalidTopics + " aren't available. Topics wont created since there " + + "are partition numbers defined in the query."; + LOG.error(errorMessage); + throw new SiddhiAppValidationException(errorMessage); + } else if (!topicsAvailable) { + LOG.warn("Topic/s " + invalidTopics + " aren't available. These Topics will be created with the default " + + "partition."); + } + } + + private void checkPartitionsAvailableForTheTopicsInCluster() { + //checking whether the defined partitions are available in the defined topic + Properties configProperties = new Properties(); + configProperties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + configProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArraySerializer"); + configProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + if (PARTITION_WISE.equals(threadingOption) && null == partitions) { + throw new SiddhiAppValidationException("Threading option is selected as 'partition.wise' but there are no" + + " partitions given"); + } + org.apache.kafka.clients.producer.Producer producer = new KafkaProducer(configProperties); + boolean partitionsAvailable = true; + StringBuilder invalidPartitions = new StringBuilder(""); + for (String topic : topics) { + List partitionInfos = producer.partitionsFor(topic); + if (null != partitions) { + for (String partition : partitions) { + boolean partitonAvailable = false; + for (PartitionInfo partitionInfo : partitionInfos) { + if (Integer.parseInt(partition) == partitionInfo.partition()) { + partitonAvailable = true; + } + } + if (!partitonAvailable) { + partitionsAvailable = false; + if ("".equals(invalidPartitions.toString())) { + invalidPartitions.append(partition); + } else { + invalidPartitions.append(',').append(partition); + } + LOG.error("Partition number, " + partition + + " in 'partition.id' is not available in topic partitions"); + } + } + if (!partitionsAvailable) { + throw new SiddhiAppValidationException( + "Partition number/s " + invalidPartitions + " aren't available for " + + "the topic: " + topic); + } + } + } + } +} diff --git a/component/src/test/java/org.wso2.extension.siddhi.io.kafka/KafkaTestUtil.java b/component/src/test/java/org.wso2.extension.siddhi.io.kafka/KafkaTestUtil.java new file mode 100644 index 00000000..38455437 --- /dev/null +++ b/component/src/test/java/org.wso2.extension.siddhi.io.kafka/KafkaTestUtil.java @@ -0,0 +1,215 @@ +/* + * Copyright (c) 2017, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. + * + * WSO2 Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.wso2.extension.siddhi.io.kafka; + +import kafka.admin.AdminUtils; +import kafka.common.TopicExistsException; +import kafka.server.KafkaConfig; +import kafka.server.KafkaServerStartable; +import kafka.utils.ZKStringSerializer$; +import kafka.utils.ZkUtils; +import org.I0Itec.zkclient.ZkClient; +import org.I0Itec.zkclient.ZkConnection; +import org.apache.commons.io.FileUtils; +import org.apache.curator.test.TestingServer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.log4j.Logger; + +import java.io.File; +import java.io.IOException; +import java.util.Properties; + +public class KafkaTestUtil { + private static final Logger log = Logger.getLogger(KafkaTestUtil.class); + private static TestingServer zkTestServer; + private static KafkaServerStartable kafkaServer; + private static KafkaServerStartable kafkaServer2; + private static final String kafkaLogDir = "tmp_kafka_dir"; + private static final String kafkaLogDir2 = "tmp_kafka_dir2"; + + public static void cleanLogDir() { + try { + File f = new File(kafkaLogDir); + FileUtils.deleteDirectory(f); + } catch (IOException e) { + log.error("Failed to clean up: " + e); + } + } + + public static void cleanLogDir2() { + try { + File f = new File(kafkaLogDir2); + FileUtils.deleteDirectory(f); + } catch (IOException e) { + log.error("Failed to clean up: " + e); + } + } + + //---- private methods -------- + public static void setupKafkaBroker() { + try { + log.info("#############################################################################################"); + log.info("################################# ZOOKEEPER STARTED ######################################"); + log.info("#############################################################################################"); + // mock zookeeper + zkTestServer = new TestingServer(2181); + // mock kafka + Properties props = new Properties(); + props.put("broker.id", "0"); + props.put("host.name", "localhost"); + props.put("port", "9092"); + props.put("log.dir", kafkaLogDir); + props.put("zookeeper.connect", zkTestServer.getConnectString()); + props.put("replica.socket.timeout.ms", "30000"); + props.put("delete.topic.enable", "true"); + KafkaConfig config = new KafkaConfig(props); + kafkaServer = new KafkaServerStartable(config); + kafkaServer.startup(); + } catch (Exception e) { + log.error("Error running local Kafka broker / Zookeeper", e); + } + } + + public static void setupKafkaBroker2() { + try { + // mock kafka + Properties props = new Properties(); + props.put("broker.id", "1"); + props.put("host.name", "localhost"); + props.put("port", "9093"); + props.put("log.dir", kafkaLogDir2); + props.put("zookeeper.connect", zkTestServer.getConnectString()); + props.put("replica.socket.timeout.ms", "30000"); + props.put("delete.topic.enable", "true"); + KafkaConfig config = new KafkaConfig(props); + kafkaServer2 = new KafkaServerStartable(config); + kafkaServer2.startup(); + } catch (Exception e) { + log.error("Error running local Kafka broker 2", e); + } + } + + public static void stopKafkaBroker2() { + if (kafkaServer2 != null) { + kafkaServer2.shutdown(); + kafkaServer2.awaitShutdown(); + } + try { + Thread.sleep(5000); + } catch (InterruptedException e) { + log.error(e.getMessage(), e); + } + cleanLogDir2(); + } + + public static void stopKafkaBroker() { + log.info("#############################################################################################"); + log.info("################################# ZOOKEEPER STOPPED ######################################"); + log.info("#############################################################################################"); + try { + if (kafkaServer != null) { + kafkaServer.shutdown(); + kafkaServer.awaitShutdown(); + } + Thread.sleep(5000); + if (zkTestServer != null) { + zkTestServer.stop(); + } + Thread.sleep(5000); + cleanLogDir(); + } catch (InterruptedException e) { + log.error(e.getMessage(), e); + } catch (IOException e) { + log.error("Error shutting down Kafka broker / Zookeeper", e); + } + } + + public static void createTopic(String topics[], int numOfPartitions) { + ZkClient zkClient = new ZkClient("localhost:2181", 30000, 30000, ZKStringSerializer$.MODULE$); + ZkConnection zkConnection = new ZkConnection("localhost:2181"); + ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false); + for (String topic : topics) { + try { + AdminUtils.createTopic(zkUtils, topic, numOfPartitions, 1, new Properties()); + } catch (TopicExistsException e) { + log.warn("topic exists for: " + topic); + } + } + zkClient.close(); + } + + public static void deleteTopic(String topics[]) { + ZkClient zkClient = new ZkClient("localhost:2181", 30000, 30000, ZKStringSerializer$.MODULE$); + ZkConnection zkConnection = new ZkConnection("localhost:2181"); + ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false); + for (String topic : topics) { + AdminUtils.deleteTopic(zkUtils, topic); + } + zkClient.close(); + } + + public static void kafkaPublisher(String topics[], int numOfPartitions, int numberOfEventsPerTopic, boolean + publishWithPartition, String bootstrapServers) { + kafkaPublisher(topics, numOfPartitions, numberOfEventsPerTopic, 1000, publishWithPartition, + bootstrapServers); + } + + public static void kafkaPublisher(String topics[], int numOfPartitions, int numberOfEventsPerTopic, long sleep, + boolean publishWithPartition, String bootstrapServers) { + Properties props = new Properties(); + if (null == bootstrapServers) { + props.put("bootstrap.servers", "localhost:9092"); + } else { + props.put("bootstrap.servers", bootstrapServers); + } + props.put("acks", "all"); + props.put("retries", 0); + props.put("batch.size", 16384); + props.put("linger.ms", 1); + props.put("buffer.memory", 33554432); + props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + Producer producer = new KafkaProducer(props); + for (String topic : topics) { + for (int i = 0; i < numberOfEventsPerTopic; i++) { + String msg = "" + + "" + + "" + topic + "" + + "12.5" + + "" + i + "" + + "" + + ""; + try { + Thread.sleep(sleep); + } catch (InterruptedException e) { + } + if (numOfPartitions > 1 || publishWithPartition) { + log.info("producing: " + msg + " into partition: " + (i % numOfPartitions)); + producer.send(new ProducerRecord<>(topic, String.valueOf(i % numOfPartitions), msg)); + } else { + log.info("producing: " + msg); + producer.send(new ProducerRecord<>(topic, msg)); + } + } + } + producer.close(); + } +} diff --git a/component/src/test/java/org.wso2.extension.siddhi.io.kafka/sink/KafkaSinkTestCase.java b/component/src/test/java/org.wso2.extension.siddhi.io.kafka/sink/KafkaSinkTestCase.java new file mode 100644 index 00000000..b21f98a1 --- /dev/null +++ b/component/src/test/java/org.wso2.extension.siddhi.io.kafka/sink/KafkaSinkTestCase.java @@ -0,0 +1,572 @@ +/* + * Copyright (c) 2017, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. + * + * WSO2 Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.wso2.extension.siddhi.io.kafka.sink; + +import org.I0Itec.zkclient.exception.ZkTimeoutException; +import org.apache.log4j.Logger; +import org.testng.AssertJUnit; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import org.wso2.extension.siddhi.io.kafka.KafkaTestUtil; +import org.wso2.siddhi.core.SiddhiAppRuntime; +import org.wso2.siddhi.core.SiddhiManager; +import org.wso2.siddhi.core.event.Event; +import org.wso2.siddhi.core.stream.input.InputHandler; +import org.wso2.siddhi.core.stream.output.StreamCallback; +import org.wso2.siddhi.extension.input.mapper.xml.XmlSourceMapper; +import org.wso2.siddhi.extension.output.mapper.xml.XMLSinkMapper; + +import java.rmi.RemoteException; +import java.util.ArrayList; +import java.util.List; + +public class KafkaSinkTestCase { + static final Logger LOG = Logger.getLogger(KafkaSinkTestCase.class); + private volatile int count; + private volatile boolean eventArrived; + private volatile List receivedEventNameList; + private volatile List receivedValueList; + + @BeforeClass + public static void init() throws Exception { + try { +// KafkaTestUtil.cleanLogDir(); +// KafkaTestUtil.setupKafkaBroker(); +// Thread.sleep(1000); + } catch (Exception e) { + throw new RemoteException("Exception caught when starting server", e); + } + } + + @AfterClass + public static void stopKafkaBroker() { + KafkaTestUtil.stopKafkaBroker(); + } + + @BeforeMethod + public void init2() { + count = 0; + eventArrived = false; + } + + @Test + public void testPublisherWithTopicWithoutPartitionKafkaTransport_56() throws InterruptedException { + LOG.info("Creating test for publishing events for static topic without a partition"); + String topics[] = new String[]{"single_topic"}; + KafkaTestUtil.createTopic(topics, 1); + receivedEventNameList = new ArrayList<>(3); + receivedValueList = new ArrayList<>(3); + try { + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-output-mapper", XMLSinkMapper.class); + SiddhiAppRuntime siddhiAppRuntimeSource = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan2') " + + "define stream BarStream2 (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='single_topic', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream2 (symbol string, price float, volume long);" + + "from FooStream2 select symbol, price, volume insert into BarStream2;"); + siddhiAppRuntimeSource.addCallback("BarStream2", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + LOG.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntimeSource.start(); + Thread.sleep(4000); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream FooStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@sink(type='kafka', topic='single_topic', bootstrap.servers='localhost:9092', " + + "@map(type='xml'))" + + "Define stream BarStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + InputHandler fooStream = siddhiAppRuntime.getInputHandler("FooStream"); + siddhiAppRuntime.start(); + Thread.sleep(4000); + fooStream.send(new Object[]{"single_topic", 55.6f, 100L}); + fooStream.send(new Object[]{"single_topic2", 75.6f, 102L}); + fooStream.send(new Object[]{"single_topic3", 57.6f, 103L}); + Thread.sleep(4000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("single_topic"); + expectedNames.add("single_topic2"); + expectedNames.add("single_topic3"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(100L); + expectedValues.add(102L); + expectedValues.add(103L); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedValues, receivedValueList); + AssertJUnit.assertEquals(3, count); + KafkaTestUtil.deleteTopic(topics); + siddhiAppRuntime.shutdown(); + siddhiAppRuntimeSource.shutdown(); + } catch (ZkTimeoutException ex) { + LOG.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testPublisherWithTopicWithoutPartitionKafkaTransport_56") + public void testPublisherWithTopicWithPartitionKafkaTransport_56() throws InterruptedException { + LOG.info("Creating test for publishing events for static topic with a partition"); + String topics[] = new String[]{"topic_with_two_partitions_sub0"}; + KafkaTestUtil.createTopic(topics, 2); + receivedEventNameList = new ArrayList<>(3); + receivedValueList = new ArrayList<>(3); + try { + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-output-mapper", XMLSinkMapper.class); + SiddhiAppRuntime siddhiAppRuntimeSource = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan2') " + + "define stream BarStream2 (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='topic_with_two_partitions_sub0', partition.no.list='0'," + + "group.id='test', threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream2 (symbol string, price float, volume long);" + + "from FooStream2 select symbol, price, volume insert into BarStream2;"); + siddhiAppRuntimeSource.addCallback("BarStream2", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + LOG.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntimeSource.start(); + Thread.sleep(4000); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream FooStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@sink(type='kafka', topic='topic_with_two_partitions_sub0', partition.no='0', " + + "bootstrap.servers='localhost:9092', " + + "@map(type='xml'))" + + "Define stream BarStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + InputHandler fooStream = siddhiAppRuntime.getInputHandler("FooStream"); + siddhiAppRuntime.start(); + Thread.sleep(10000); + fooStream.send(new Object[]{"single_topic", 55.6f, 100L}); + fooStream.send(new Object[]{"single_topic2", 75.6f, 102L}); + fooStream.send(new Object[]{"single_topic3", 57.6f, 103L}); + Thread.sleep(4000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("single_topic"); + expectedNames.add("single_topic2"); + expectedNames.add("single_topic3"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(100L); + expectedValues.add(102L); + expectedValues.add(103L); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedValues, receivedValueList); + AssertJUnit.assertEquals(3, count); + KafkaTestUtil.deleteTopic(topics); + siddhiAppRuntime.shutdown(); + siddhiAppRuntimeSource.shutdown(); + } catch (ZkTimeoutException ex) { + LOG.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testPublisherWithTopicWithPartitionKafkaTransport_56") + public void testPublisherWithInvalidTopicKafkaTransport_58() throws InterruptedException { + LOG.info("Creating test for publishing events for invalid topic without a partition"); + String topics[] = new String[]{"invalid_topic_without_partition2"}; + receivedEventNameList = new ArrayList<>(3); + receivedValueList = new ArrayList<>(3); + try { + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-output-mapper", XMLSinkMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream FooStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@sink(type='kafka', topic='invalid_topic_without_partition2', " + + "bootstrap.servers='localhost:9092', " + + "@map(type='xml'))" + + "Define stream BarStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + InputHandler fooStream = siddhiAppRuntime.getInputHandler("FooStream"); + siddhiAppRuntime.start(); + Thread.sleep(2000); + //this event will be published to the cluster and create the topic + fooStream.send(new Object[]{"invalid_topic_without_partition", 55.6f, 100L}); + //this thread sleep is to slowdown the below execution plan because, if there are no topics kafka source + // will create the topic. + Thread.sleep(2000); + SiddhiAppRuntime siddhiAppRuntimeSource = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan2') " + + "define stream BarStream2 (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='invalid_topic_without_partition2', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream2 (symbol string, price float, volume long);" + + "from FooStream2 select symbol, price, volume insert into BarStream2;"); + siddhiAppRuntimeSource.addCallback("BarStream2", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + LOG.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntimeSource.start(); + Thread.sleep(2000); + fooStream.send(new Object[]{"invalid_topic_without_partition2", 75.6f, 102L}); + fooStream.send(new Object[]{"invalid_topic_without_partition3", 57.6f, 103L}); + Thread.sleep(2000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("invalid_topic_without_partition"); + expectedNames.add("invalid_topic_without_partition2"); + expectedNames.add("invalid_topic_without_partition3"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(100L); + expectedValues.add(102L); + expectedValues.add(103L); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedValues, receivedValueList); + AssertJUnit.assertEquals(3, count); + KafkaTestUtil.deleteTopic(topics); + siddhiAppRuntime.shutdown(); + siddhiAppRuntimeSource.shutdown(); + } catch (ZkTimeoutException ex) { + LOG.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testPublisherWithInvalidTopicKafkaTransport_58") + public void testPublisherWithInvalidTopicWithPartitionKafkaTransport_58() throws InterruptedException { + LOG.info("Creating test for publishing events for invalid topic with a partition"); + String topics[] = new String[]{"invalid_topic_with_partition"}; + receivedEventNameList = new ArrayList<>(3); + receivedValueList = new ArrayList<>(3); + try { + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-output-mapper", XMLSinkMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream FooStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@sink(type='kafka', topic='invalid_topic_with_partition', " + + "bootstrap.servers='localhost:9092', partition.no='0', " + + "@map(type='xml'))" + + "Define stream BarStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + InputHandler fooStream = siddhiAppRuntime.getInputHandler("FooStream"); + siddhiAppRuntime.start(); + Thread.sleep(2000); + //this event will be published to the cluster and create the topic + fooStream.send(new Object[]{"invalid_topic_with_partition", 55.6f, 100L}); + //this thread sleep is to slowdown the below execution plan because, if there are no topics kafka source + // will create the topic. + Thread.sleep(2000); + SiddhiAppRuntime siddhiAppRuntimeSource = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan2') " + + "define stream BarStream2 (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='invalid_topic_with_partition', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092', " + + "partition.no.list='0', " + + "@map(type='xml'))" + + "Define stream FooStream2 (symbol string, price float, volume long);" + + "from FooStream2 select symbol, price, volume insert into BarStream2;"); + siddhiAppRuntimeSource.addCallback("BarStream2", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + LOG.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntimeSource.start(); + Thread.sleep(2000); + fooStream.send(new Object[]{"invalid_topic_with_partition2", 75.6f, 102L}); + fooStream.send(new Object[]{"invalid_topic_with_partition3", 57.6f, 103L}); + Thread.sleep(2000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("invalid_topic_with_partition"); + expectedNames.add("invalid_topic_with_partition2"); + expectedNames.add("invalid_topic_with_partition3"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(100L); + expectedValues.add(102L); + expectedValues.add(103L); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedValues, receivedValueList); + AssertJUnit.assertEquals(3, count); + KafkaTestUtil.deleteTopic(topics); + siddhiAppRuntime.shutdown(); + siddhiAppRuntimeSource.shutdown(); + } catch (ZkTimeoutException ex) { + LOG.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testPublisherWithInvalidTopicWithPartitionKafkaTransport_58") + public void testPublisherWithInvalidTopicWithPartitionOtherThan0KafkaTransport_58() throws InterruptedException { + LOG.info("Creating test for publishing events for invalid topic with a partition other than 0 but the source " + + "will be getting events from the default partition"); + String topics[] = new String[]{"invalid_topic_with_partition_2"}; + receivedEventNameList = new ArrayList<>(3); + receivedValueList = new ArrayList<>(3); + try { + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-output-mapper", XMLSinkMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream FooStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@sink(type='kafka', topic='invalid_topic_with_partition_2', " + + "bootstrap.servers='localhost:9092', partition.no='2', " + + "@map(type='xml'))" + + "Define stream BarStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + InputHandler fooStream = siddhiAppRuntime.getInputHandler("FooStream"); + siddhiAppRuntime.start(); + Thread.sleep(2000); + //this event will be published to the cluster and create the topic + fooStream.send(new Object[]{"invalid_topic_with_partition_2", 55.6f, 100L}); + //this thread sleep is to slowdown the below execution plan because, if there are no topics kafka source + // will create the topic. + Thread.sleep(2000); + SiddhiAppRuntime siddhiAppRuntimeSource = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan2') " + + "define stream BarStream2 (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='invalid_topic_with_partition_2', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092', " + + "@map(type='xml'))" + + "Define stream FooStream2 (symbol string, price float, volume long);" + + "from FooStream2 select symbol, price, volume insert into BarStream2;"); + siddhiAppRuntimeSource.addCallback("BarStream2", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + LOG.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntimeSource.start(); + Thread.sleep(2000); + fooStream.send(new Object[]{"invalid_topic_with_partition_22", 75.6f, 102L}); + fooStream.send(new Object[]{"invalid_topic_with_partition_23", 57.6f, 103L}); + Thread.sleep(2000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("invalid_topic_with_partition_2"); + expectedNames.add("invalid_topic_with_partition_22"); + expectedNames.add("invalid_topic_with_partition_23"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(100L); + expectedValues.add(102L); + expectedValues.add(103L); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Sink didnt publish the expected events", expectedValues, receivedValueList); + AssertJUnit.assertEquals(3, count); + KafkaTestUtil.deleteTopic(topics); + siddhiAppRuntime.shutdown(); + siddhiAppRuntimeSource.shutdown(); + } catch (ZkTimeoutException ex) { + LOG.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testPublisherWithInvalidTopicWithPartitionOtherThan0KafkaTransport_58") + public void testPublisherWithKafkaTransportWithDynamicTopic() throws InterruptedException { + LOG.info("Creating test for publishing events for dynamic topic without partition"); + try { + String topics[] = new String[]{"multiple_topic1", "multiple_topic2"}; + receivedEventNameList = new ArrayList<>(4); + receivedValueList = new ArrayList<>(4); + KafkaTestUtil.createTopic(topics, 1); + + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-output-mapper", XMLSinkMapper.class); + SiddhiAppRuntime executionPlanRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream FooStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@sink(type='kafka', topic='{{symbol}}', bootstrap.servers='localhost:9092', " + + "@map(type='xml'))" + + "Define stream BarStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream; "); + InputHandler fooStream = executionPlanRuntime.getInputHandler("FooStream"); + executionPlanRuntime.start(); + + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='multiple_topic1,multiple_topic2', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + LOG.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + + Thread.sleep(2000); + fooStream.send(new Object[]{"multiple_topic1", 55.6f, 100L}); + fooStream.send(new Object[]{"multiple_topic2", 75.6f, 102L}); + Thread.sleep(5000); + + List expectedNames = new ArrayList<>(2); + expectedNames.add("multiple_topic1"); + expectedNames.add("multiple_topic2"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(100L); + expectedValues.add(102L); + AssertJUnit.assertEquals(2, count); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + KafkaTestUtil.deleteTopic(topics); + executionPlanRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + LOG.warn("No zookeeper may not be available.", ex); + } + } + +// @Test + public void testPublisherWithKafkaTransportWithDynamicTopicAndPartition() throws InterruptedException { + LOG.info("Creating test for publishing events for dynamic topic with dynamic partition"); + try { + String topics[] = new String[]{"multiple_topic1_two_par_all_sub1", "multiple_topic2_two_par_all_sub1"}; + receivedEventNameList = new ArrayList<>(4); + receivedValueList = new ArrayList<>(4); + KafkaTestUtil.createTopic(topics, 2); + Thread.sleep(10000); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-output-mapper", XMLSinkMapper.class); + SiddhiAppRuntime executionPlanRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream FooStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@sink(type='kafka', topic='{{symbol}}', partition.no='{{volume}}', " + + "bootstrap.servers='localhost:9092', " + + "@map(type='xml'))" + + "Define stream BarStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream; "); + InputHandler fooStream = executionPlanRuntime.getInputHandler("FooStream"); + executionPlanRuntime.start(); + + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', " + + "topic.list='multiple_topic1_two_par_all_sub1,multiple_topic2_two_par_all_sub1', " + + "group.id='test', threading.option='single.thread', bootstrap.servers='localhost:9092', " + + "partition.no.list='0,1'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + LOG.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + fooStream.send(new Object[]{"multiple_topic1_two_par_all_sub1", 55.6f, 0L}); + Thread.sleep(1000); + fooStream.send(new Object[]{"multiple_topic1_two_par_all_sub1", 55.6f, 1L}); + Thread.sleep(1000); + fooStream.send(new Object[]{"multiple_topic2_two_par_all_sub1", 75.6f, 0L}); + Thread.sleep(1000); + fooStream.send(new Object[]{"multiple_topic2_two_par_all_sub1", 75.6f, 1L}); + Thread.sleep(5000); + + List expectedNames = new ArrayList<>(2); + expectedNames.add("multiple_topic1_two_par_all_sub1"); + expectedNames.add("multiple_topic1_two_par_all_sub1"); + expectedNames.add("multiple_topic2_two_par_all_sub1"); + expectedNames.add("multiple_topic2_two_par_all_sub1"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + expectedValues.add(1L); + expectedValues.add(0L); + expectedValues.add(1L); + AssertJUnit.assertEquals(4, count); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + KafkaTestUtil.deleteTopic(topics); + executionPlanRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + LOG.warn("No zookeeper may not be available.", ex); + } + } +} diff --git a/component/src/test/java/org.wso2.extension.siddhi.io.kafka/source/KafkaSourceHATestCase.java b/component/src/test/java/org.wso2.extension.siddhi.io.kafka/source/KafkaSourceHATestCase.java new file mode 100644 index 00000000..adaf95b0 --- /dev/null +++ b/component/src/test/java/org.wso2.extension.siddhi.io.kafka/source/KafkaSourceHATestCase.java @@ -0,0 +1,459 @@ +/* + * Copyright (c) 2017, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. + * + * WSO2 Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.wso2.extension.siddhi.io.kafka.source; + +import org.I0Itec.zkclient.exception.ZkTimeoutException; +import org.apache.log4j.Logger; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import org.wso2.extension.siddhi.io.kafka.KafkaTestUtil; +import org.wso2.siddhi.core.SiddhiAppRuntime; +import org.wso2.siddhi.core.SiddhiManager; +import org.wso2.siddhi.core.event.Event; +import org.wso2.siddhi.core.stream.input.source.Source; +import org.wso2.siddhi.core.stream.output.StreamCallback; +import org.wso2.siddhi.core.util.EventPrinter; +import org.wso2.siddhi.core.util.persistence.InMemoryPersistenceStore; +import org.wso2.siddhi.core.util.persistence.PersistenceStore; +import org.wso2.siddhi.extension.input.mapper.xml.XmlSourceMapper; +import org.wso2.siddhi.extension.output.mapper.xml.XMLSinkMapper; +import org.wso2.siddhi.query.api.SiddhiApp; +import org.wso2.siddhi.query.api.annotation.Annotation; +import org.wso2.siddhi.query.api.definition.Attribute; +import org.wso2.siddhi.query.api.definition.StreamDefinition; +import org.wso2.siddhi.query.api.execution.query.Query; +import org.wso2.siddhi.query.api.execution.query.input.stream.InputStream; +import org.wso2.siddhi.query.api.execution.query.selection.Selector; +import org.wso2.siddhi.query.api.expression.Variable; + +import java.rmi.RemoteException; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class KafkaSourceHATestCase { + private static final Logger log = Logger.getLogger(KafkaSourceHATestCase.class); + private static ExecutorService executorService; + private volatile int count; + private volatile boolean eventArrived; + + @BeforeClass + public static void init() throws Exception { + try { + executorService = Executors.newFixedThreadPool(5); + KafkaTestUtil.cleanLogDir(); + KafkaTestUtil.setupKafkaBroker(); + Thread.sleep(10000); + } catch (Exception e) { + throw new RemoteException("Exception caught when starting server", e); + } + } + + @BeforeMethod + public void init2() { + count = 0; + eventArrived = false; + } + + @Test + public void testAKafkaPauseAndResume() throws InterruptedException { + try { + log.info("Test to verify the pause and resume functionality of Kafka source"); + String topics[] = new String[]{"kafka_topic3"}; + KafkaTestUtil.createTopic(topics, 2); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='kafka_topic3', group.id='test1', threading" + + ".option='partition.wise', " + + "bootstrap.servers='localhost:9092', partition.no.list='0,1', " + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + } + + } + }); + siddhiAppRuntime.start(); + Future eventSender = executorService.submit(new Runnable() { + @Override + public void run() { + KafkaTestUtil.kafkaPublisher(topics, 2, 4, false, null); + } + }); + while (!eventSender.isDone()) { + Thread.sleep(1000); + } + Thread.sleep(2000); + assertEquals(4, count); + assertTrue(eventArrived); + + Collection> sources = siddhiAppRuntime.getSources(); + // pause the transports + sources.forEach(e -> e.forEach(Source::pause)); + + init2(); + eventSender = executorService.submit(new Runnable() { + @Override + public void run() { + KafkaTestUtil.kafkaPublisher(topics, 2, 4, false, null); + } + }); + while (!eventSender.isDone()) { + Thread.sleep(1000); + } + Thread.sleep(5000); + assertFalse(eventArrived); + + // resume the transports + sources.forEach(e -> e.forEach(Source::resume)); + Thread.sleep(2000); + assertEquals(4, count); + assertTrue(eventArrived); + + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test //(dependsOnMethods = "testAKafkaPauseAndResume") + public void testRecoveryOnFailureOfSingleNodeWithKafka() throws InterruptedException { + try { + log.info("Test to verify recovering process of a Siddhi node on a failure when Kafka is the event source"); + String topics[] = new String[]{"kafka_topic4"}; + KafkaTestUtil.createTopic(topics, 1); + PersistenceStore persistenceStore = new InMemoryPersistenceStore(); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setPersistenceStore(persistenceStore); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + + String query = "@App:name('TestExecutionPlan') " + + "define stream BarStream (count long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='kafka_topic4', group.id='test', " + + "threading.option='topic.wise', bootstrap.servers='localhost:9092', partition.no.list='0', " + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select count(symbol) as count insert into BarStream;"; + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(query); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + eventArrived = true; + log.info(event); + count = Math.toIntExact((long) event.getData(0)); + } + + } + }); + + // start publishing events to Kafka + Future eventSender = executorService.submit(new Runnable() { + @Override + public void run() { + KafkaTestUtil.kafkaPublisher(topics, 1, 5, 1000, false, null); + } + }); + Thread.sleep(2000); + // start the execution app + siddhiAppRuntime.start(); + + // wait for some time + Thread.sleep(28000); + // initiate a checkpointing task + Future perisistor = siddhiAppRuntime.persist().getFuture(); + // waits till the checkpointing task is done + while (!perisistor.isDone()) { + Thread.sleep(100); + } + // let few more events to be published + Thread.sleep(5000); + // initiate a execution app shutdown - to demonstrate a node failure + siddhiAppRuntime.shutdown(); + // let few events to be published while the execution app is down + Thread.sleep(5000); + // recreate the execution app + siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(query); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + eventArrived = true; + log.info(event); + count = Math.toIntExact((long) event.getData(0)); + } + + } + }); + // start the execution app + siddhiAppRuntime.start(); + // immediately trigger a restore from last revision + siddhiAppRuntime.restoreLastRevision(); + Thread.sleep(5000); + + // waits till all the events are published + while (!eventSender.isDone()) { + Thread.sleep(2000); + } + + Thread.sleep(20000); + assertTrue(eventArrived); + // assert the count + assertEquals(5, count); + + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testRecoveryOnFailureOfSingleNodeWithKafka") + public void testRecoveryOnFailureOfMultipleNodeWithKafka() throws InterruptedException { + try { + log.info("Test to verify recovering process of multiple Siddhi nodes on a failure when Kafka is the event" + + " source"); + String topics[] = new String[]{"kafka_topic5", "kafka_topic6"}; + KafkaTestUtil.createTopic(topics, 1); + // 1st node + PersistenceStore persistenceStore = new InMemoryPersistenceStore(); + SiddhiManager siddhiManager1 = new SiddhiManager(); + siddhiManager1.setPersistenceStore(persistenceStore); + siddhiManager1.setExtension("inputmapper:text", XmlSourceMapper.class); + + // 2nd node + PersistenceStore persistenceStore1 = new InMemoryPersistenceStore(); + SiddhiManager siddhiManager2 = new SiddhiManager(); + siddhiManager2.setPersistenceStore(persistenceStore1); + siddhiManager2.setExtension("inputmapper:text", XmlSourceMapper.class); + + String query1 = "@App:name('TestExecutionPlan') " + + "@sink(type='kafka', topic.list='kafka_topic6', bootstrap.servers='localhost:9092', partition" + + ".no='0', " + + "@map(type='xml'))" + + "define stream BarStream (count long); " + + "@source(type='kafka', topic.list='kafka_topic5', group.id='test', " + + "threading.option='topic.wise', bootstrap.servers='localhost:9092', partition.no.list='0', " + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "@info(name = 'query1') " + + "from FooStream select count(symbol) as count insert into BarStream;"; + + String query2 = "@App:name('TestExecutionPlan') " + + "define stream BarStream (count long); " + + "@source(type='kafka', topic.list='kafka_topic6', group.id='test', " + + "threading.option='topic.wise', bootstrap.servers='localhost:9092', partition.no.list='0', " + + "@map(type='xml'))" + + "Define stream FooStream (number long);" + + "@info(name = 'query1') " + + "from FooStream select count(number) as count insert into BarStream;"; + + SiddhiAppRuntime siddhiAppRuntime1 = siddhiManager1.createSiddhiAppRuntime(query1); + SiddhiAppRuntime siddhiAppRuntime2 = siddhiManager2.createSiddhiAppRuntime(query2); + + siddhiAppRuntime2.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + eventArrived = true; + log.info(event); + count = Math.toIntExact((long) event.getData(0)); + } + + } + }); + + // start the execution app + siddhiAppRuntime1.start(); + siddhiAppRuntime2.start(); + // let it initialize + Thread.sleep(2000); + + // start publishing events to Kafka + Future eventSender = executorService.submit(new Runnable() { + @Override + public void run() { + KafkaTestUtil.kafkaPublisher( + new String[]{"kafka_topic5"}, 1, 50, 1000, false, null); + } + }); + + // wait for some time + Thread.sleep(28000); + // initiate a checkpointing task + Future perisistor1 = siddhiAppRuntime1.persist().getFuture(); + Future perisistor2 = siddhiAppRuntime2.persist().getFuture(); + // waits till the checkpointing task is done + while (!perisistor1.isDone() && !perisistor2.isDone()) { + Thread.sleep(100); + } + // let few more events to be published + Thread.sleep(5000); + // initiate a execution app shutdown - to demonstrate a node failure + siddhiAppRuntime1.shutdown(); + siddhiAppRuntime2.shutdown(); + // let few events to be published while the execution app is down + Thread.sleep(5000); + // recreate the execution app + siddhiAppRuntime1 = siddhiManager1.createSiddhiAppRuntime(query1); + siddhiAppRuntime2 = siddhiManager2.createSiddhiAppRuntime(query2); + siddhiAppRuntime2.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + eventArrived = true; + log.info(event); + count = Math.toIntExact((long) event.getData(0)); + } + + } + }); + // start the execution app + siddhiAppRuntime1.start(); + siddhiAppRuntime2.start(); + // immediately trigger a restore from last revision + siddhiAppRuntime1.restoreLastRevision(); + siddhiAppRuntime2.restoreLastRevision(); + Thread.sleep(5000); + + // waits till all the events are published + while (!eventSender.isDone()) { + Thread.sleep(2000); + } + + Thread.sleep(20000); + assertTrue(eventArrived); + // assert the count + assertEquals(50, count); + + siddhiAppRuntime1.shutdown(); + siddhiAppRuntime2.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + // @Test + public void testCreatingFullKafkaEventFlow() throws InterruptedException { + Runnable kafkaReceiver = new KafkaFlow(); + Thread t1 = new Thread(kafkaReceiver); + t1.start(); + Thread.sleep(35000); + } + + @AfterClass + public static void stopKafkaBroker() { + KafkaTestUtil.stopKafkaBroker(); + } + + private class KafkaFlow implements Runnable { + @Override + public void run() { + try { + StreamDefinition inputDefinition = StreamDefinition.id("FooStream") + .attribute("symbol", Attribute.Type.STRING) + .attribute("price", Attribute.Type.FLOAT) + .attribute("volume", Attribute.Type.INT) + .annotation(Annotation.annotation("source") + .element("type", "kafka") + .element("topic", "receiver_topic") + .element("threads", "1") + .element("partition.no.list", "0,1") + .element("group.id", "group1") + .element("bootstrap.servers", "localhost:9092") + .annotation(Annotation.annotation("map") + .element("type", "text"))); + + StreamDefinition outputDefinition = StreamDefinition.id("BarStream") + .attribute("symbol", Attribute.Type.STRING) + .attribute("price", Attribute.Type.FLOAT) + .attribute("volume", Attribute.Type.INT) + .annotation(Annotation.annotation("sink") + .element("type", "kafka") + .element("topic", "publisher_topic") + .element("partition.no", "0") + .element("bootstrap.servers", "localhost:9092") + .annotation(Annotation.annotation("map") + .element("type", "text"))); + + Query query = Query.query(); + query.from( + InputStream.stream("FooStream") + ); + query.select( + Selector.selector().select(new Variable("symbol")).select(new Variable("price")).select(new + Variable("volume")) + ); + query.insertInto("BarStream"); + + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + siddhiManager.setExtension("sink.mapper:text", XMLSinkMapper.class); + + SiddhiApp siddhiApp = new SiddhiApp("ep1"); + siddhiApp.defineStream(inputDefinition); + siddhiApp.defineStream(outputDefinition); + siddhiApp.addQuery(query); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + siddhiAppRuntime.addCallback("FooStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + log.info("Printing received events !!"); + EventPrinter.print(events); + } + }); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + log.info("Printing publishing events !!"); + EventPrinter.print(events); + } + }); + siddhiAppRuntime.start(); + Thread.sleep(30000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } catch (InterruptedException e) { + log.error(e.getMessage(), e); + } + } + } +} + diff --git a/component/src/test/java/org.wso2.extension.siddhi.io.kafka/source/KafkaSourceTestCase.java b/component/src/test/java/org.wso2.extension.siddhi.io.kafka/source/KafkaSourceTestCase.java new file mode 100644 index 00000000..db2b6d55 --- /dev/null +++ b/component/src/test/java/org.wso2.extension.siddhi.io.kafka/source/KafkaSourceTestCase.java @@ -0,0 +1,1024 @@ +/* + * Copyright (c) 2017, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. + * + * WSO2 Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.wso2.extension.siddhi.io.kafka.source; + +import org.I0Itec.zkclient.exception.ZkTimeoutException; +import org.apache.log4j.Logger; +import org.testng.AssertJUnit; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import org.wso2.extension.siddhi.io.kafka.KafkaTestUtil; +import org.wso2.siddhi.core.SiddhiAppRuntime; +import org.wso2.siddhi.core.SiddhiManager; +import org.wso2.siddhi.core.event.Event; +import org.wso2.siddhi.core.stream.output.StreamCallback; +import org.wso2.siddhi.extension.input.mapper.xml.XmlSourceMapper; +import org.wso2.siddhi.query.api.exception.SiddhiAppValidationException; + +import java.rmi.RemoteException; +import java.util.ArrayList; +import java.util.List; + + +public class KafkaSourceTestCase { + private static final Logger log = Logger.getLogger(KafkaSourceTestCase.class); + private volatile int count; + private volatile boolean eventArrived; + private volatile List receivedEventNameList; + private volatile List receivedValueList; + + @BeforeClass + public static void init() throws Exception { + try { + KafkaTestUtil.cleanLogDir(); + KafkaTestUtil.setupKafkaBroker(); + Thread.sleep(1000); + } catch (Exception e) { + throw new RemoteException("Exception caught when starting server", e); + } + } + + @AfterClass + public static void stopKafkaBroker() { +// KafkaTestUtil.stopKafkaBroker(); + } + + @BeforeMethod + public void init2() { + count = 0; + eventArrived = false; + } + + @Test + public void testKafkaSingleTopicSource_53_103() throws InterruptedException { + try { + log.info("Creating test for single topic"); + String topics[] = new String[]{"single_topic"}; + receivedEventNameList = new ArrayList<>(2); + receivedValueList = new ArrayList<>(2); + KafkaTestUtil.createTopic(topics, 1); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='single_topic', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 1, 2, false, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("single_topic"); + expectedNames.add("single_topic"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + expectedValues.add(1L); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + AssertJUnit.assertEquals(2, count); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test(expectedExceptions = SiddhiAppValidationException.class, + dependsOnMethods = "testKafkaSingleTopicSource_53_103") + public void testKafkaWithoutTopicSource_54() { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for without topic"); + log.info("-------------------------------------------------------------------------------------------"); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', group.id='test', threading.option='single.thread', " + + "bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.start(); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaWithoutTopicSource_54") + public void testKafkaMultipleTopicSource_85_98() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for multiple topic"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"multiple_topic1", "multiple_topic2"}; + receivedEventNameList = new ArrayList<>(4); + receivedValueList = new ArrayList<>(4); + KafkaTestUtil.createTopic(topics, 1); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='multiple_topic1,multiple_topic2', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 1, 2, false, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("multiple_topic1"); + expectedNames.add("multiple_topic1"); + expectedNames.add("multiple_topic2"); + expectedNames.add("multiple_topic2"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + expectedValues.add(1L); + expectedValues.add(0L); + expectedValues.add(1L); + AssertJUnit.assertEquals(4, count); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaMultipleTopicSource_85_98") + public void testKafkaSingleTopicWithSpecificSubscribeSource_92() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for single topic with one partition which subscribes for the partition " + + "specifically"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"topic_with_one_partition"}; + receivedEventNameList = new ArrayList<>(4); + receivedValueList = new ArrayList<>(4); + KafkaTestUtil.createTopic(topics, 1); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', " + + "topic.list='topic_with_one_partition', " + + "partition.no.list='0', " + + "group.id='test', " + + "threading.option='single.thread', " + + "bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 1, 1, true, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("topic_with_one_partition"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + AssertJUnit.assertEquals(1, count); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test(expectedExceptions = SiddhiAppValidationException.class, + dependsOnMethods = "testKafkaSingleTopicWithSpecificSubscribeSource_92") + public void testKafkaSpecificSubscribeForUnavailablePartitionSource_105_120() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for single topic with partitions which subscribes for an unavailable partition."); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"topic_without_some_partition"}; + KafkaTestUtil.createTopic(topics, 2); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='topic_without_some_partition', " + + "partition.no.list='0,1,2', group.id='test', threading.option='single.thread', " + + "bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaSpecificSubscribeForUnavailablePartitionSource_105_120") + public void testKafkaMultipleTopic_MultiplePartition_OnePartitionSubscribe_Source_100() throws + InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test to configure Kafka source with multiple topics having multiple partitions " + + "subscribing for single partition id"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"multiple_topic1_two_par_one_sub", "multiple_topic2_two_par_one_sub"}; + receivedEventNameList = new ArrayList<>(4); + receivedValueList = new ArrayList<>(4); + KafkaTestUtil.createTopic(topics, 2); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', " + + "topic.list='multiple_topic1_two_par_one_sub,multiple_topic2_two_par_one_sub', " + + "partition.no.list='1', " + + "group.id='test', " + + "threading.option='single.thread', " + + "bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 2, 2, false, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("multiple_topic1_two_par_one_sub"); + expectedNames.add("multiple_topic2_two_par_one_sub"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(1L); + expectedValues.add(1L); + AssertJUnit.assertEquals(2, count); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaMultipleTopic_MultiplePartition_OnePartitionSubscribe_Source_100") + public void testKafkaMultipleTopic_MultiplePartition_AllPartitionSubscribe_Source_101() throws + InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test to configure Kafka source with multiple topics having multiple partitions " + + "subscribing for all partition ids"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"multiple_topic1_two_par_all_sub", "multiple_topic2_two_par_all_sub"}; + receivedEventNameList = new ArrayList<>(4); + receivedValueList = new ArrayList<>(4); + KafkaTestUtil.createTopic(topics, 2); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', " + + "topic.list='multiple_topic1_two_par_all_sub,multiple_topic2_two_par_all_sub', " + + "partition.no.list='0,1', " + + "group.id='test', " + + "threading.option='single.thread', " + + "bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 2, 2, false, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("multiple_topic1_two_par_all_sub"); + expectedNames.add("multiple_topic1_two_par_all_sub"); + expectedNames.add("multiple_topic2_two_par_all_sub"); + expectedNames.add("multiple_topic2_two_par_all_sub"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + expectedValues.add(1L); + expectedValues.add(0L); + expectedValues.add(1L); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + AssertJUnit.assertEquals(4, count); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test(expectedExceptions = SiddhiAppValidationException.class, + dependsOnMethods = "testKafkaMultipleTopic_MultiplePartition_AllPartitionSubscribe_Source_101") + public void testKafkaWithoutBootstrapServerSource_108() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for without any bootstrap servers defined."); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"no_bootstrap_server_topic"}; + KafkaTestUtil.createTopic(topics, 2); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='no_bootstrap_server_topic', partition.no.list='0,1,2', " + + "group.id='test', threading.option='single.thread'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaWithoutBootstrapServerSource_108") + public void testKafkaMultipleTopicWithThreadingPerTopicSource_111() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for multiple topic with thread per topic"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"multiple_topic1", "multiple_topic2"}; + receivedEventNameList = new ArrayList<>(2); + receivedValueList = new ArrayList<>(2); + KafkaTestUtil.createTopic(topics, 1); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', " + + "topic.list='multiple_topic1,multiple_topic2', " + + "group.id='test', " + + "threading.option='topic.wise', " + + "bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 1, 2, false, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("multiple_topic1"); + expectedNames.add("multiple_topic1"); + expectedNames.add("multiple_topic2"); + expectedNames.add("multiple_topic2"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + expectedValues.add(1L); + expectedValues.add(0L); + expectedValues.add(1L); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + AssertJUnit.assertEquals(4, count); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaMultipleTopicWithThreadingPerTopicSource_111") + public void testKafkaMultipleTopicWithThreadingPerPartitionSource_112() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for multiple topic with thread per partition"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"multiple_topic1_2par", "multiple_topic2_2par"}; + receivedEventNameList = new ArrayList<>(2); + receivedValueList = new ArrayList<>(2); + KafkaTestUtil.createTopic(topics, 2); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', " + + "topic.list='multiple_topic1_2par,multiple_topic2_2par', " + + "group.id='test', " + + "threading.option='partition.wise', " + + "bootstrap.servers='localhost:9092', " + + "partition.no.list='0,1'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 2, 2, false, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("multiple_topic1_2par"); + expectedNames.add("multiple_topic1_2par"); + expectedNames.add("multiple_topic2_2par"); + expectedNames.add("multiple_topic2_2par"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + expectedValues.add(1L); + expectedValues.add(0L); + expectedValues.add(1L); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + AssertJUnit.assertEquals(4, count); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test(expectedExceptions = SiddhiAppValidationException.class, + dependsOnMethods = "testKafkaMultipleTopicWithThreadingPerPartitionSource_112") + public void testKafkaWithoutThreadingOptionSource_113() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for without any threading option defined."); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"no_threading_option_topic"}; + KafkaTestUtil.createTopic(topics, 2); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='no_threading_option_topic', partition.no.list='0,1,2', " + + "group.id='test'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test(expectedExceptions = SiddhiAppValidationException.class, + dependsOnMethods = "testKafkaWithoutThreadingOptionSource_113") + public void testKafkaSingleTopicWithoutGroupIdSource_114_129_143() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for single topic without group ID"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"single_topic_without_groupid"}; + KafkaTestUtil.createTopic(topics, 1); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', " + + "topic.list='single_topic_without_groupid', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaSingleTopicWithoutGroupIdSource_114_129_143") + public void testKafkaSingleTopicDifferentGroupIdsSource_133() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for single topic subscribed by multiple sources with different group ids"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"single_topic_different_group_ids"}; + receivedEventNameList = new ArrayList<>(2); + receivedValueList = new ArrayList<>(2); + KafkaTestUtil.createTopic(topics, 1); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "define stream BarStream2 (symbol string, price float, volume long); " + + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='single_topic_different_group_ids', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long); " + + + "@info(name = 'query2') " + + "@source(type='kafka', topic.list='single_topic_different_group_ids', group.id='test2', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream2 (symbol string, price float, volume long); " + + + "from FooStream select symbol, price, volume insert into BarStream; " + + "from FooStream2 select symbol, price, volume insert into BarStream2; "); + + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.addCallback("BarStream2", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 1, 1, false, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("single_topic_different_group_ids"); + expectedNames.add("single_topic_different_group_ids"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + expectedValues.add(0L); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + AssertJUnit.assertEquals(2, count); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaSingleTopicDifferentGroupIdsSource_133") + public void testKafkaSingleTopicSameGroupIdsSource_140() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for single topic subscribed by multiple sources with same group ids"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"single_topic_same_group_ids"}; + receivedEventNameList = new ArrayList<>(2); + receivedValueList = new ArrayList<>(2); + KafkaTestUtil.createTopic(topics, 1); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "define stream BarStream2 (symbol string, price float, volume long); " + + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='single_topic_same_group_ids', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long); " + + + "@info(name = 'query2') " + + "@source(type='kafka', topic.list='single_topic_same_group_ids', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream2 (symbol string, price float, volume long); " + + + "from FooStream select symbol, price, volume insert into BarStream; " + + "from FooStream2 select symbol, price, volume insert into BarStream2; "); + + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.addCallback("BarStream2", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 1, 1, false, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("single_topic_same_group_ids"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + AssertJUnit.assertEquals(1, count); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaSingleTopicSameGroupIdsSource_140") + public void testKafkaNonExistingTopicSource_121_123() throws InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for non-existing topic. This will create a topic with default partition"); + log.info("-------------------------------------------------------------------------------------------"); + receivedEventNameList = new ArrayList<>(2); + receivedValueList = new ArrayList<>(2); + String topics[] = new String[]{"non_existing_topic1"}; + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='non_existing_topic1', group.id='test', " + + "threading.option='single.thread', bootstrap.servers='localhost:9092'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 1, 1, false, null); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("non_existing_topic1"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + KafkaTestUtil.deleteTopic(topics); + AssertJUnit.assertEquals(1, count); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + + @Test (dependsOnMethods = "testKafkaNonExistingTopicSource_121_123") + public void testKafkaMultipleTopicPartitionPartitionWiseSubscription() throws InterruptedException { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for multiple topics and partitions and thread partition wise"); + log.info("-------------------------------------------------------------------------------------------"); + receivedEventNameList = new ArrayList<>(4); + receivedValueList = new ArrayList<>(4); + String topics[] = new String[]{"kafka_topic", "kafka_topic2"}; + KafkaTestUtil.createTopic(topics, 2); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='kafka_topic,kafka_topic2', group.id='test', " + + "threading.option='partition.wise', bootstrap.servers='localhost:9092', " + + "partition.no.list='0,1', " + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 2, 2, false, null); + Thread.sleep(1000); + AssertJUnit.assertEquals(4, count); + AssertJUnit.assertTrue(eventArrived); + List expectedNames = new ArrayList<>(2); + expectedNames.add("kafka_topic"); + expectedNames.add("kafka_topic"); + expectedNames.add("kafka_topic2"); + expectedNames.add("kafka_topic2"); + List expectedValues = new ArrayList<>(2); + expectedValues.add(0L); + expectedValues.add(1L); + expectedValues.add(0L); + expectedValues.add(1L); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedValues, receivedValueList); + KafkaTestUtil.deleteTopic(topics); + siddhiAppRuntime.shutdown(); + } + + @Test (dependsOnMethods = "testKafkaMultipleTopicPartitionPartitionWiseSubscription") + public void testKafkaMultipleTopicPartitionTopicWiseSubscription() throws InterruptedException { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test for multiple topics and partitions and thread topic wise"); + log.info("-------------------------------------------------------------------------------------------"); + try { + String topics[] = new String[]{"kafka_topic_with_2par", "kafka_topic2_with_2par"}; + receivedEventNameList = new ArrayList<>(4); + receivedValueList = new ArrayList<>(4); + KafkaTestUtil.createTopic(topics, 2); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', topic.list='kafka_topic_with_2par,kafka_topic2_with_2par', " + + "group.id='test', threading.option='topic.wise', bootstrap.servers='localhost:9092', " + + "partition.no.list='0,1', " + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 2, 2, false, null); + Thread.sleep(1000); + AssertJUnit.assertEquals(4, count); + AssertJUnit.assertTrue(eventArrived); + List expectedNames = new ArrayList<>(4); + expectedNames.add("kafka_topic_with_2par"); + expectedNames.add("kafka_topic_with_2par"); + expectedNames.add("kafka_topic2_with_2par"); + expectedNames.add("kafka_topic2_with_2par"); + List expectedValues = new ArrayList<>(4); + expectedValues.add(0L); + expectedValues.add(1L); + expectedValues.add(0L); + expectedValues.add(1L); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(4000); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } + +// @Test + public void testKafkaSingleTopic_MultiplePartition_AllPartitionSubscribe_Source_106() throws + InterruptedException { + try { + log.info("-------------------------------------------------------------------------------------------"); + log.info("Creating test to configure Kafka source with multiple topics having multiple partitions " + + "subscribing for all partition ids"); + log.info("-------------------------------------------------------------------------------------------"); + String topics[] = new String[]{"single_topic1_two_par_two_servers"}; + KafkaTestUtil.setupKafkaBroker2(); + receivedEventNameList = new ArrayList<>(2); + receivedValueList = new ArrayList<>(2); + Thread.sleep(10000); + KafkaTestUtil.createTopic(topics, 2); + Thread.sleep(5000); + SiddhiManager siddhiManager = new SiddhiManager(); + siddhiManager.setExtension("xml-input-mapper", XmlSourceMapper.class); + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime( + "@App:name('TestExecutionPlan') " + + "define stream BarStream (symbol string, price float, volume long); " + + "@info(name = 'query1') " + + "@source(type='kafka', " + + "topic.list='single_topic1_two_par_two_servers', " + + "partition.no.list='0,1', " + + "group.id='test', " + + "threading.option='single.thread', " + + "bootstrap.servers='localhost:9092,localhost:9093'," + + "@map(type='xml'))" + + "Define stream FooStream (symbol string, price float, volume long);" + + "from FooStream select symbol, price, volume insert into BarStream;"); + siddhiAppRuntime.addCallback("BarStream", new StreamCallback() { + @Override + public void receive(Event[] events) { + for (Event event : events) { + log.info(event); + eventArrived = true; + count++; + receivedEventNameList.add(event.getData(0).toString()); + receivedValueList.add((long) event.getData(2)); + } + } + }); + siddhiAppRuntime.start(); + Thread.sleep(2000); + KafkaTestUtil.kafkaPublisher(topics, 2, 2, false, "localhost:9093,localhost:9092"); + Thread.sleep(1000); + List expectedNames = new ArrayList<>(2); + expectedNames.add("single_topic1_two_par_two_servers"); + expectedNames.add("single_topic1_two_par_two_servers"); + expectedNames.add("single_topic1_two_par_two_servers"); + expectedNames.add("single_topic1_two_par_two_servers"); + List expectedValues = new ArrayList<>(2); + AssertJUnit.assertEquals("Kafka Source expected input not received", expectedNames, + receivedEventNameList); + AssertJUnit.assertEquals(2, count); + KafkaTestUtil.deleteTopic(topics); + Thread.sleep(1000); + KafkaTestUtil.stopKafkaBroker2(); + siddhiAppRuntime.shutdown(); + } catch (ZkTimeoutException ex) { + log.warn("No zookeeper may not be available.", ex); + } + } +} + diff --git a/component/input-transport/src/test/resources/log4j.properties b/component/src/test/resources/log4j.properties similarity index 100% rename from component/input-transport/src/test/resources/log4j.properties rename to component/src/test/resources/log4j.properties diff --git a/component/src/test/resources/testing.xml b/component/src/test/resources/testing.xml new file mode 100644 index 00000000..39874747 --- /dev/null +++ b/component/src/test/resources/testing.xml @@ -0,0 +1,30 @@ + + + + + + + + + + + + + + \ No newline at end of file diff --git a/findbugs-exclude.xml b/findbugs-exclude.xml index 94d146e5..bf09b4a7 100644 --- a/findbugs-exclude.xml +++ b/findbugs-exclude.xml @@ -18,24 +18,5 @@ --> - - - - - - - - - - - - - - - - - - - - + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 003062fc..d6ee1e74 100644 --- a/pom.xml +++ b/pom.xml @@ -26,10 +26,10 @@ 4.0.0 pom - org.wso2.extension.siddhi - siddhi-io-kafka - 4.0.0-M5-SNAPSHOT - Siddhi IO Kafka Extensions + org.wso2.extension.siddhi.io.kafka + siddhi-io-kafka-parent + 1.0.0 + Siddhi IO Kafka Extension Parent @@ -38,27 +38,27 @@ true - component/input-transport - component/output-transport + component - 4.0.0-M5-SNAPSHOT + 4.0.0-M9 1.2.17.wso2v1 - 4.12 + 6.8 1.1.1 0.9.0.1 2.7.1 3.4.0 2.5 + 0.9.0.0 - scm:git:https://github.com/wso2-extensions/siddhi-eventtable-rdbms.git - https://github.com/wso2-extensions/siddhi-eventtable-rdbms.git - scm:git:https://github.com/wso2-extensions/siddhi-eventtable-rdbms.git + scm:git:https://github.com/wso2-extensions/siddhi-io-kafka.git + https://github.com/wso2-extensions/siddhi-io-kafka.git + scm:git:https://github.com/wso2-extensions/siddhi-io-kafka.git HEAD @@ -109,9 +109,9 @@ ${commons.logging.version} - junit - junit - ${junit.version} + org.testng + testng + ${testng.version} test @@ -137,21 +137,14 @@ ${commons-io.version} test - - - - org.wso2.siddhi - siddhi-extension-kafka-output-transport - ${siddhi.version} - org.wso2.siddhi - siddhi-extension-text-input-mapper + siddhi-extension-xml-input-mapper ${siddhi.version} org.wso2.siddhi - siddhi-extension-text-output-mapper + siddhi-extension-xml-output-mapper ${siddhi.version} @@ -250,6 +243,16 @@ maven-project-info-reports-plugin 2.4 + + org.apache.maven.plugins + maven-surefire-plugin + 2.16 + + + src/test/resources/testing.xml + + +