Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add support to remove stream and query callback dynamically. #1594

Merged
merged 1 commit into from
Dec 20, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,10 @@ public interface SiddhiAppRuntime {

void addCallback(String queryName, QueryCallback callback);

void removeCallback(StreamCallback streamCallback);

void removeCallback(QueryCallback streamCallback);

Event[] query(String onDemandQuery);

@Deprecated
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,7 @@ public void addCallback(String streamId, StreamCallback streamCallback) {
}

public void addCallback(String queryName, QueryCallback callback) {
callback.setQueryName(queryName);
callback.setContext(siddhiAppContext);
QueryRuntime queryRuntime = queryProcessorMap.get(queryName);
if (queryRuntime == null) {
Expand All @@ -278,6 +279,28 @@ public void addCallback(String queryName, QueryCallback callback) {
((QueryRuntimeImpl) queryRuntime).addCallback(callback);
}

public void removeCallback(StreamCallback streamCallback) {
if (streamCallback.getStreamId() == null) {
throw new SiddhiAppRuntimeException("Cannot find streamID in the streamCallback");
}
String streamId = streamCallback.getStreamId();
StreamJunction streamJunction = streamJunctionMap.get(streamId);
if (streamJunction != null) {
streamJunction.unsubscribe(streamCallback);
}
}

public void removeCallback(QueryCallback callback) {
if (callback.getQueryName() == null) {
throw new SiddhiAppRuntimeException("Cannot find QueryName in the queryCallback");
}
String queryName = callback.getQueryName();
QueryRuntime queryRuntime = queryProcessorMap.get(queryName);
if (queryRuntime != null) {
((QueryRuntimeImpl) queryRuntime).removeCallback(callback);
}
}

public Event[] query(String onDemandQuery) {
if (this.running) {
return query(SiddhiCompiler.parseOnDemandQuery(onDemandQuery), onDemandQuery);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,10 @@ public void addCallback(QueryCallback callback) {
outputRateLimiter.addQueryCallback(callback);
}

public void removeCallback(QueryCallback callback) {
outputRateLimiter.removeQueryCallback(callback);
}

public OutputRateLimiter getOutputRateManager() {
return outputRateLimiter;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,11 +40,20 @@ public abstract class QueryCallback {

private SiddhiAppContext siddhiAppContext;
private Query query;
private String queryName;

public void setQuery(Query query) {
this.query = query;
}

public void setQueryName(String queryName) {
this.queryName = queryName;
}

public String getQueryName() {
return queryName;
}

public void setContext(SiddhiAppContext siddhiAppContext) {
this.siddhiAppContext = siddhiAppContext;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,15 @@ public void addQueryCallback(QueryCallback callback) {
hasCallBack = true;
}

public void removeQueryCallback(QueryCallback callback) {
List<QueryCallback> newQueryCallbacks = new ArrayList<>(queryCallbacks);
newQueryCallbacks.remove(callback);
queryCallbacks = newQueryCallbacks;
if (queryCallbacks.size() == 0) {
hasCallBack = false;
}
}

public abstract void process(ComplexEventChunk complexEventChunk);

public void process(List<ComplexEventChunk> complexEventChunks) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -338,6 +338,12 @@ public void subscribe(Receiver receiver) {
}
}

public void unsubscribe(StreamCallback streamCallback) {
List<Receiver> newReceivers = new LinkedList<>(receivers);
newReceivers.remove(streamCallback);
receivers = newReceivers;
}

public String getStreamId() {
return streamDefinition.getId();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* Copyright (c) 2019, 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 io.siddhi.core.query;

import io.siddhi.core.SiddhiAppRuntime;
import io.siddhi.core.SiddhiManager;
import io.siddhi.core.event.Event;
import io.siddhi.core.query.output.callback.QueryCallback;
import io.siddhi.core.stream.input.InputHandler;
import io.siddhi.core.util.EventPrinter;
import io.siddhi.core.util.SiddhiTestHelper;
import org.apache.log4j.Logger;
import org.testng.AssertJUnit;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

/**
* Created on 1/24/15.
*/
public class CallbackTestCase {

private static final Logger log = Logger.getLogger(CallbackTestCase.class);
private volatile int count;
private volatile boolean eventArrived;

@BeforeMethod
public void init() {
count = 0;
eventArrived = false;
}


@Test
public void callbackTest1() throws InterruptedException {
log.info("callback test1");
SiddhiManager siddhiManager = new SiddhiManager();
String siddhiApp = "" +
"define stream cseEventStream (symbol string, price float, volume long);" +
"" +
"@info(name = 'query1') " +
"from cseEventStream " +
"select symbol, price , symbol as sym1 " +
"insert into outputStream ;";
QueryCallback queryCallback = new QueryCallback() {
@Override
public void receive(long timeStamp, Event[] inEvents, Event[] removeEvents) {
EventPrinter.print(timeStamp, inEvents, removeEvents);
eventArrived = true;
if (inEvents != null) {
count += inEvents.length;
}
if (removeEvents != null) {
count += removeEvents.length;
}
}
};
SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp);
siddhiAppRuntime.addCallback("query1", queryCallback);
InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream");
siddhiAppRuntime.start();
inputHandler.send(new Object[]{"IBM", 0f, 100L});
siddhiAppRuntime.removeCallback(queryCallback);
inputHandler.send(new Object[]{"WSO2", 0f, 100L});
SiddhiTestHelper.waitForEvents(10, count, 1, 100);
AssertJUnit.assertEquals(1, count);
AssertJUnit.assertTrue(eventArrived);
siddhiAppRuntime.shutdown();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import io.siddhi.core.stream.input.InputHandler;
import io.siddhi.core.stream.output.StreamCallback;
import io.siddhi.core.util.EventPrinter;
import io.siddhi.core.util.SiddhiTestHelper;
import org.apache.log4j.Logger;
import org.testng.Assert;
import org.testng.AssertJUnit;
Expand Down Expand Up @@ -224,4 +225,39 @@ public void receive(Event[] events) {
siddhiAppRuntime.shutdown();
}

@Test
public void callbackTest5() throws InterruptedException {
log.info("callback test5");
SiddhiManager siddhiManager = new SiddhiManager();
String siddhiApp = "" +
"define stream cseEventStream (symbol string, price float, volume long);" +
"" +
"@info(name = 'query1') " +
"from cseEventStream " +
"select symbol, price , symbol as sym1 " +
"insert into OutputStream ;";

StreamCallback streamCallback = new StreamCallback() {
@Override
public void receive(Event[] events) {
EventPrinter.print(events);
eventArrived = true;
if (events != null) {
count += events.length;
}
}
};
SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp);
siddhiAppRuntime.addCallback("OutputStream", streamCallback);
InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream");
siddhiAppRuntime.start();
inputHandler.send(new Object[]{"IBM", 0f, 100L});
siddhiAppRuntime.removeCallback(streamCallback);
inputHandler.send(new Object[]{"WSO2", 0f, 100L});
SiddhiTestHelper.waitForEvents(10, count, 1, 100);
AssertJUnit.assertEquals(1, count);
AssertJUnit.assertTrue(eventArrived);
siddhiAppRuntime.shutdown();
}

}
1 change: 1 addition & 0 deletions modules/siddhi-core/src/test/resources/testng.xml
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,7 @@
<class name="io.siddhi.core.window.DelayWindowTestCase"/>

<class name="io.siddhi.core.query.BooleanCompareTestCase"/>
<class name="io.siddhi.core.query.CallbackTestCase"/>
<class name="io.siddhi.core.query.FilterTestCase1"/>
<class name="io.siddhi.core.query.FilterTestCase2"/>
<class name="io.siddhi.core.query.IsNullTestCase"/>
Expand Down