From 60713286c3a1e26c1169cd47a59fe5fac3dbb6d8 Mon Sep 17 00:00:00 2001 From: Suhothayan Date: Sun, 22 Dec 2019 17:36:34 +0530 Subject: [PATCH 1/6] Remove beforeWindow data before sending events to Windows, Add support to extract event by passing event reference, --- .../core/aggregation/AggregationRuntime.java | 2 +- .../core/event/stream/MetaStreamEvent.java | 7 ++ .../core/event/stream/StreamEventCloner.java | 4 +- .../EventVariableFunctionExecutor.java | 73 +++++++++++++++ .../EventTimestampFunctionExecutor.java | 43 ++++++--- .../window/BatchingWindowProcessor.java | 5 ++ .../window/GroupingWindowProcessor.java | 6 ++ .../stream/window/HopingWindowProcessor.java | 6 ++ .../stream/window/SlidingWindowProcessor.java | 5 ++ .../core/util/parser/AggregationParser.java | 2 +- .../core/util/parser/ExpressionParser.java | 89 ++++++++++++++++--- .../core/util/parser/OnDemandQueryParser.java | 2 +- .../util/parser/SingleInputStreamParser.java | 6 +- .../java/io/siddhi/core/window/Window.java | 1 - .../core/query/function/FunctionTestCase.java | 77 ++++++++++++++++ .../store/OnDemandQueryTableTestCase.java | 2 +- .../core/stream/event/EventTestCase.java | 4 +- .../siddhi/query/compiler/SiddhiCompiler.java | 64 +++++++------ 18 files changed, 338 insertions(+), 60 deletions(-) create mode 100644 modules/siddhi-core/src/main/java/io/siddhi/core/executor/EventVariableFunctionExecutor.java diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/AggregationRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/AggregationRuntime.java index 8290acd769..ef86b2379b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/AggregationRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/AggregationRuntime.java @@ -161,7 +161,7 @@ private static void initMetaStreamEvent(MetaStreamEvent metaStreamEvent, Abstrac String inputReferenceId) { metaStreamEvent.addInputDefinition(inputDefinition); metaStreamEvent.setInputReferenceId(inputReferenceId); - metaStreamEvent.initializeAfterWindowData(); + metaStreamEvent.initializeOnAfterWindowData(); inputDefinition.getAttributeList().forEach(metaStreamEvent::addData); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/MetaStreamEvent.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/MetaStreamEvent.java index b942101c65..f8dc0a8f64 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/MetaStreamEvent.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/MetaStreamEvent.java @@ -62,6 +62,13 @@ public List getOutputData() { } } + public void initializeOnAfterWindowData() { + if (onAfterWindowData == null) { + onAfterWindowData = new ArrayList(); + } + } + + @Deprecated public void initializeAfterWindowData() { if (onAfterWindowData == null) { onAfterWindowData = new ArrayList(); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventCloner.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventCloner.java index d986cad2e4..c395c68bc8 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventCloner.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventCloner.java @@ -47,7 +47,9 @@ public StreamEventCloner(MetaStreamEvent metaStreamEvent, StreamEventFactory eve */ public StreamEvent copyStreamEvent(StreamEvent streamEvent) { StreamEvent newEvent = eventFactory.newInstance(); - if (beforeWindowDataSize > 0) { + if (streamEvent.getBeforeWindowData() == null) { + newEvent.setBeforeWindowData(null); + } else if (beforeWindowDataSize > 0) { System.arraycopy(streamEvent.getBeforeWindowData(), 0, newEvent.getBeforeWindowData(), 0, beforeWindowDataSize); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/EventVariableFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/EventVariableFunctionExecutor.java new file mode 100644 index 0000000000..258deaf8c5 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/EventVariableFunctionExecutor.java @@ -0,0 +1,73 @@ +/* + * 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.executor; + +import io.siddhi.core.config.SiddhiQueryContext; +import io.siddhi.core.event.ComplexEvent; +import io.siddhi.core.event.state.StateEvent; +import io.siddhi.core.executor.function.FunctionExecutor; +import io.siddhi.core.util.config.ConfigReader; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.query.api.definition.Attribute; + +import static io.siddhi.core.util.SiddhiConstants.STREAM_EVENT_CHAIN_INDEX; +import static io.siddhi.core.util.SiddhiConstants.STREAM_EVENT_INDEX_IN_CHAIN; +import static io.siddhi.core.util.SiddhiConstants.UNKNOWN_STATE; + +/** + * Executor class for Siddhi that extracts events. + */ +public class EventVariableFunctionExecutor extends FunctionExecutor { + + private int[] position = new int[]{UNKNOWN_STATE, UNKNOWN_STATE}; + + public EventVariableFunctionExecutor(int streamEventChainIndex, int streamEventIndexInChain) { + position[STREAM_EVENT_CHAIN_INDEX] = streamEventChainIndex; + position[STREAM_EVENT_INDEX_IN_CHAIN] = streamEventIndexInChain; + } + + @Override + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { + return null; + } + + @Override + public Attribute.Type getReturnType() { + return Attribute.Type.OBJECT; + } + + @Override + protected Object execute(Object[] data, State state) { + //will not occur + return null; + } + + @Override + protected Object execute(Object data, State state) { + //will not occur + return null; + } + + public Object execute(ComplexEvent event) { + return ((StateEvent) event).getStreamEvent(position); + } + +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/EventTimestampFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/EventTimestampFunctionExecutor.java index faee8af4d5..a07ab0cca6 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/EventTimestampFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/EventTimestampFunctionExecutor.java @@ -20,6 +20,7 @@ import io.siddhi.annotation.Example; import io.siddhi.annotation.Extension; +import io.siddhi.annotation.Parameter; import io.siddhi.annotation.ParameterOverload; import io.siddhi.annotation.ReturnAttribute; import io.siddhi.annotation.util.DataType; @@ -30,7 +31,6 @@ import io.siddhi.core.util.snapshot.state.State; import io.siddhi.core.util.snapshot.state.StateFactory; import io.siddhi.query.api.definition.Attribute; -import io.siddhi.query.api.exception.SiddhiAppValidationException; /** * Executor class for Siddhi cast function. Converts the given parameter according to the castTo parameter. @@ -39,38 +39,55 @@ @Extension( name = "eventTimestamp", namespace = "", - description = "Returns the timestamp of the processed event.", - parameters = {}, + description = "Returns the timestamp of the processed/passed event.", + parameters = { + @Parameter(name = "event", + description = "Event reference.", + type = {DataType.OBJECT}, + dynamic = true, + optional = true, + defaultValue = "Current Event") + }, parameterOverloads = { - @ParameterOverload() + @ParameterOverload(), + @ParameterOverload(parameterNames = {"event"}) }, returnAttributes = @ReturnAttribute( - description = "timestamp of the event.", + description = "Timestamp of the event.", type = DataType.LONG), examples = { @Example( - syntax = "from fooStream\n" + + syntax = "from FooStream\n" + "select symbol as name, eventTimestamp() as eventTimestamp \n" + - "insert into barStream;", - description = "This will extract current events timestamp.") + "insert into BarStream;", + description = "Extracts current event's timestamp."), + @Example( + syntax = "from FooStream as f join FooBarTable as fb\n" + + "select fb.symbol as name, eventTimestamp(f) as eventTimestamp \n" + + "insert into BarStream;", + description = "Extracts FooStream event's timestamp.") } ) public class EventTimestampFunctionExecutor extends FunctionExecutor { + private boolean expectEventObject; + @Override protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { - if (attributeExpressionExecutors.length != 0) { - throw new SiddhiAppValidationException("Invalid no of arguments passed to eventTimestamp() function, " + - "required 0 parameters, but found " + - attributeExpressionExecutors.length); + if (attributeExpressionExecutors.length == 1) { + expectEventObject = true; } return null; } @Override public Object execute(ComplexEvent event) { - return event.getTimestamp(); + if (expectEventObject) { + return ((ComplexEvent) attributeExpressionExecutors[0].execute(event)).getTimestamp(); + } else { + return event.getTimestamp(); + } } @Override diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchingWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchingWindowProcessor.java index 89056159f2..f5bfa9b57f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchingWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchingWindowProcessor.java @@ -71,6 +71,11 @@ protected void processEventChunk(ComplexEventChunk streamEventChunk StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, S state) { streamEventChunk.reset(); + while (streamEventChunk.hasNext()) { + StreamEvent streamEvent = streamEventChunk.next(); + streamEvent.setBeforeWindowData(null); + } + streamEventChunk.reset(); process(streamEventChunk, nextProcessor, streamEventCloner, state); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/GroupingWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/GroupingWindowProcessor.java index 9ce8bde753..dbb96a1181 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/GroupingWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/GroupingWindowProcessor.java @@ -80,6 +80,12 @@ protected abstract StateFactory init(ExpressionExecutor[] attributeExpression protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, S state) { + streamEventChunk.reset(); + while (streamEventChunk.hasNext()) { + StreamEvent streamEvent = streamEventChunk.next(); + streamEvent.setBeforeWindowData(null); + } + streamEventChunk.reset(); processEventChunk(streamEventChunk, nextProcessor, streamEventCloner, groupingKeyPopulator, state); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/HopingWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/HopingWindowProcessor.java index 1491c3a27d..5dbf6aacac 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/HopingWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/HopingWindowProcessor.java @@ -81,6 +81,12 @@ protected abstract StateFactory init(ExpressionExecutor[] attributeExpression protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, S state) { + streamEventChunk.reset(); + while (streamEventChunk.hasNext()) { + StreamEvent streamEvent = streamEventChunk.next(); + streamEvent.setBeforeWindowData(null); + } + streamEventChunk.reset(); processEventChunk(streamEventChunk, nextProcessor, streamEventCloner, hopingTimestampPopulator); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SlidingWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SlidingWindowProcessor.java index 9bf1a24bb8..cbc2cdaf4e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SlidingWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SlidingWindowProcessor.java @@ -66,6 +66,11 @@ protected void processEventChunk(ComplexEventChunk streamEventChunk StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, S state) { streamEventChunk.reset(); + while (streamEventChunk.hasNext()) { + StreamEvent streamEvent = streamEventChunk.next(); + streamEvent.setBeforeWindowData(null); + } + streamEventChunk.reset(); process(streamEventChunk, nextProcessor, streamEventCloner, state); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/AggregationParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/AggregationParser.java index 2d2c48d1b3..8484e38cba 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/AggregationParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/AggregationParser.java @@ -151,7 +151,7 @@ public static AggregationRuntime parse(AggregationDefinition aggregationDefiniti // Example format: AGG_TIMESTAMP, groupByAttribute1, groupByAttribute2, AGG_incAttribute1, AGG_incAttribute2 // AGG_incAttribute1, AGG_incAttribute2 would have the same attribute names as in // finalListOfIncrementalAttributes - incomingMetaStreamEvent.initializeAfterWindowData(); // To enter data as onAfterWindowData + incomingMetaStreamEvent.initializeOnAfterWindowData(); // To enter data as onAfterWindowData List incrementalDurations = getSortedPeriods(aggregationDefinition.getTimePeriod()); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/ExpressionParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/ExpressionParser.java index 30e02f296c..8c7c2a8c39 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/ExpressionParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/ExpressionParser.java @@ -25,6 +25,7 @@ import io.siddhi.core.exception.OperationNotSupportedException; import io.siddhi.core.exception.SiddhiAppCreationException; import io.siddhi.core.executor.ConstantExpressionExecutor; +import io.siddhi.core.executor.EventVariableFunctionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.MultiValueVariableFunctionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -1288,9 +1289,11 @@ private static ExpressionExecutor parseVariable(Variable variable, MetaComplexEv new Attribute(attributeName, type), eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX], eventPosition[SiddhiConstants.STREAM_EVENT_INDEX_IN_CHAIN]); if (((MetaStreamEvent) metaEvent).getEventType() != MetaStreamEvent.EventType.DEFAULT) { - variableExpressionExecutor.getPosition()[SiddhiConstants.STREAM_ATTRIBUTE_TYPE_INDEX] = SiddhiConstants.OUTPUT_DATA_INDEX; - variableExpressionExecutor.getPosition()[SiddhiConstants.STREAM_ATTRIBUTE_INDEX_IN_TYPE] = abstractDefinition - .getAttributePosition(variableExpressionExecutor.getAttribute().getName()); + variableExpressionExecutor.getPosition()[SiddhiConstants.STREAM_ATTRIBUTE_TYPE_INDEX] = + SiddhiConstants.OUTPUT_DATA_INDEX; + variableExpressionExecutor.getPosition()[SiddhiConstants.STREAM_ATTRIBUTE_INDEX_IN_TYPE] = + abstractDefinition + .getAttributePosition(variableExpressionExecutor.getAttribute().getName()); } if (executorList != null) { executorList.add(variableExpressionExecutor); @@ -1345,8 +1348,15 @@ private static ExpressionExecutor parseVariable(Variable variable, MetaComplexEv type = definition.getAttributeType(attributeName); eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX] = currentState; } catch (AttributeNotExistException e) { - throw new SiddhiAppValidationException(e.getMessageWithOutContext() + " Input Stream: " + - definition.getId() + " with reference: " + metaStreamEvent.getInputReferenceId(), e, + ExpressionExecutor functionExecutor = constructEventExpressionExecutor(variable, currentState, + siddhiQueryContext, eventPosition, metaStateEvent); + if (functionExecutor != null) { + return functionExecutor; + } + throw new SiddhiAppValidationException(e.getMessageWithOutContext() + " Input stream '" + + definition.getId() + "' with reference '" + + metaStreamEvent.getInputReferenceId() + "' for attribute '" + + variable.getAttributeName() + "'", e, e.getQueryContextStartIndex(), e.getQueryContextEndIndex()); } } @@ -1366,10 +1376,12 @@ private static ExpressionExecutor parseVariable(Variable variable, MetaComplexEv type = definition.getAttributeType(attributeName); eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX] = i; if (currentState > -1 && metaStreamEvents[currentState].getInputReferenceId() != null - && variable.getStreamIndex() != null && variable.getStreamIndex() <= SiddhiConstants.LAST) { + && variable.getStreamIndex() != null + && variable.getStreamIndex() <= SiddhiConstants.LAST) { if (variable.getStreamId() .equals(metaStreamEvents[currentState].getInputReferenceId())) { - eventPosition[SiddhiConstants.STREAM_EVENT_INDEX_IN_CHAIN] = variable.getStreamIndex(); + eventPosition[SiddhiConstants.STREAM_EVENT_INDEX_IN_CHAIN] = + variable.getStreamIndex(); } } else if (currentState == SiddhiConstants.UNKNOWN_STATE && variable.getStreamIndex() == null) { @@ -1381,8 +1393,18 @@ private static ExpressionExecutor parseVariable(Variable variable, MetaComplexEv } } if (eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX] == SiddhiConstants.UNKNOWN_STATE) { - throw new SiddhiAppValidationException( - "Stream with reference : " + variable.getStreamId() + " not found"); + if (variable.getStreamId() == null) { + ExpressionExecutor functionExecutor = constructEventExpressionExecutor(variable, currentState, siddhiQueryContext, eventPosition, metaStateEvent); + if (functionExecutor != null) { + return functionExecutor; + } + throw new SiddhiAppValidationException( + "No matching stream reference found for attribute '" + variable.getAttributeName() + "'"); + } else { + throw new SiddhiAppValidationException( + "Stream with reference '" + variable.getStreamId() + "' not found for attribute '" + + variable.getAttributeName() + "'"); + } } VariableExpressionExecutor variableExpressionExecutor = new VariableExpressionExecutor( new Attribute(attributeName, type), eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX], @@ -1391,10 +1413,11 @@ private static ExpressionExecutor parseVariable(Variable variable, MetaComplexEv MetaStreamEvent metaStreamEvent = ((MetaStateEvent) metaEvent) .getMetaStreamEvent(eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX]); if (metaStreamEvent.getEventType() != MetaStreamEvent.EventType.DEFAULT) { - variableExpressionExecutor.getPosition()[SiddhiConstants.STREAM_ATTRIBUTE_TYPE_INDEX] = SiddhiConstants.OUTPUT_DATA_INDEX; - variableExpressionExecutor.getPosition()[SiddhiConstants.STREAM_ATTRIBUTE_INDEX_IN_TYPE] = metaStreamEvent - .getLastInputDefinition() - .getAttributePosition(variableExpressionExecutor.getAttribute().getName()); + variableExpressionExecutor.getPosition()[SiddhiConstants.STREAM_ATTRIBUTE_TYPE_INDEX] = + SiddhiConstants.OUTPUT_DATA_INDEX; + variableExpressionExecutor.getPosition()[SiddhiConstants.STREAM_ATTRIBUTE_INDEX_IN_TYPE] = + metaStreamEvent.getLastInputDefinition() + .getAttributePosition(variableExpressionExecutor.getAttribute().getName()); for (Attribute attribute : metaStreamEvent.getLastInputDefinition().getAttributeList()) { metaStreamEvent.addOutputData(new Attribute(attribute.getName(), attribute.getType())); } @@ -1415,6 +1438,46 @@ private static ExpressionExecutor parseVariable(Variable variable, MetaComplexEv } } + private static ExpressionExecutor constructEventExpressionExecutor(Variable variable, int currentState, SiddhiQueryContext siddhiQueryContext, int[] eventPosition, MetaStateEvent metaStateEvent) { + AbstractDefinition definition; + MetaStreamEvent[] metaStreamEvents = metaStateEvent.getMetaStreamEvents(); + for (int i = 0, metaStreamEventsLength = metaStreamEvents.length; i < metaStreamEventsLength; i++) { + MetaStreamEvent metaStreamEvent = metaStreamEvents[i]; + definition = metaStreamEvent.getLastInputDefinition(); + if (metaStreamEvent.getInputReferenceId() == null) { + if (definition.getId().equals(variable.getAttributeName())) { + eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX] = i; + FunctionExecutor functionExecutor = new EventVariableFunctionExecutor( + eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX], + eventPosition[SiddhiConstants.STREAM_EVENT_INDEX_IN_CHAIN]); + functionExecutor.initExecutor(new ExpressionExecutor[0], ProcessingMode.BATCH, + null, false, siddhiQueryContext); + return functionExecutor; + } + } else { + if (metaStreamEvent.getInputReferenceId().equals(variable.getAttributeName())) { + eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX] = i; + if (currentState > -1 && metaStreamEvents[currentState].getInputReferenceId() != null + && variable.getStreamIndex() != null + && variable.getStreamIndex() <= SiddhiConstants.LAST) { + if (variable.getAttributeName() + .equals(metaStreamEvents[currentState].getInputReferenceId())) { + eventPosition[SiddhiConstants.STREAM_EVENT_INDEX_IN_CHAIN] = + variable.getStreamIndex(); + } + } + FunctionExecutor functionExecutor = new EventVariableFunctionExecutor( + eventPosition[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX], + eventPosition[SiddhiConstants.STREAM_EVENT_INDEX_IN_CHAIN]); + functionExecutor.initExecutor(new ExpressionExecutor[0], ProcessingMode.BATCH, + null, false, siddhiQueryContext); + return functionExecutor; + } + } + } + return null; + } + /** * Calculate the return type of arithmetic operation executors.(Ex: add, subtract, etc) * diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/OnDemandQueryParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/OnDemandQueryParser.java index 6339ac96ab..187e3d5044 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/OnDemandQueryParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/OnDemandQueryParser.java @@ -572,7 +572,7 @@ public static MatchingMetaInfoHolder generateMatchingMetaInfoHolderForCacheTable private static void initMetaStreamEvent(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition) { metaStreamEvent.addInputDefinition(inputDefinition); - metaStreamEvent.initializeAfterWindowData(); + metaStreamEvent.initializeOnAfterWindowData(); inputDefinition.getAttributeList().forEach(metaStreamEvent::addData); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SingleInputStreamParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SingleInputStreamParser.java index 4912cafdce..ebcc8ad1cd 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SingleInputStreamParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SingleInputStreamParser.java @@ -152,7 +152,7 @@ public static SingleStreamRuntime parseInputStream(SingleInputStream inputStream } } - metaStreamEvent.initializeAfterWindowData(); + metaStreamEvent.initializeOnAfterWindowData(); return new SingleStreamRuntime(processStreamReceiver, processor, processingMode, metaComplexEvent); } @@ -173,6 +173,10 @@ public static Processor generateProcessor(StreamHandler streamHandler, MetaCompl metaStreamEvent = (MetaStreamEvent) metaEvent; } + if (streamHandler instanceof Window) { + metaStreamEvent.initializeOnAfterWindowData(); + } + ExpressionExecutor[] attributeExpressionExecutors; if (parameters != null) { if (parameters.length > 0) { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/window/Window.java b/modules/siddhi-core/src/main/java/io/siddhi/core/window/Window.java index db5b526aa0..8d1d5908d2 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/window/Window.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/window/Window.java @@ -152,7 +152,6 @@ public void init(Map tableMap, Map eventWindowMap MetaStreamEvent metaStreamEvent = new MetaStreamEvent(); metaStreamEvent.addInputDefinition(windowDefinition); metaStreamEvent.setEventType(MetaStreamEvent.EventType.WINDOW); - metaStreamEvent.initializeAfterWindowData(); for (Attribute attribute : windowDefinition.getAttributeList()) { metaStreamEvent.addOutputData(attribute); } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/function/FunctionTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/function/FunctionTestCase.java index a5702b4cd4..abdfb7b341 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/function/FunctionTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/function/FunctionTestCase.java @@ -300,6 +300,83 @@ public void testFunctionQuery7() throws InterruptedException { siddhiManager.createSiddhiAppRuntime(cseEventStream + query); } + @Test + public void testFunctionQuery7_1() throws InterruptedException { + log.info("eventTimestamp Test1_1"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String appName = "" + + "@app:name('eventTimestamp') " + + "" + + "define stream fooStream (symbol string, time string);" + + "" + + "@info(name = 'query1') " + + "from fooStream " + + "select symbol as name, eventTimestamp() as eventTimestamp " + + "insert into barStream;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(appName); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + + EventPrinter.print(timestamp, inEvents, removeEvents); + for (Event inEvent : inEvents) { + count++; + AssertJUnit.assertEquals(10, inEvent.getTimestamp()); + } + } + + }); + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("fooStream"); + siddhiAppRuntime.start(); + inputHandler.send(10, new Object[]{"WSO2", 50f, 60f, 60L, 6}); + Thread.sleep(100); + AssertJUnit.assertEquals(1, count); + siddhiAppRuntime.shutdown(); + } + + @Test + public void testFunctionQuery7_2() throws InterruptedException { + log.info("eventTimestamp Test1_2"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String app = "" + + "@app:name('eventTimestamp')" + + "" + + "define stream fooStream (symbol string, time string);" + + "define table fooTable (symbol string, time string);" + + "" + + "@info(name = 'query1') " + + "from fooStream as f full outer join fooTable as t " + + "select f.symbol as name, eventTimestamp(f) as eventTimestamp " + + "insert into barStream;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(app); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + + EventPrinter.print(timestamp, inEvents, removeEvents); + for (Event inEvent : inEvents) { + count++; + AssertJUnit.assertEquals(10, inEvent.getTimestamp()); + } + } + + }); + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("fooStream"); + siddhiAppRuntime.start(); + inputHandler.send(10, new Object[]{"WSO2", 50f, 60f, 60L, 6}); + Thread.sleep(100); + AssertJUnit.assertEquals(1, count); + siddhiAppRuntime.shutdown(); + } + @Test(expectedExceptions = SiddhiAppCreationException.class) public void testFunctionQuery8() throws InterruptedException { diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/store/OnDemandQueryTableTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/store/OnDemandQueryTableTestCase.java index 5abc725b71..a2751cedad 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/store/OnDemandQueryTableTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/store/OnDemandQueryTableTestCase.java @@ -880,7 +880,7 @@ public void test21() throws InterruptedException { Assert.fail("Expected OnDemandQueryCreationException exception"); } catch (OnDemandQueryCreationException e) { String expectedCauseBy = "@ Line: 1. Position: 83, near 'AGG_TIMESTAMP2'. " + - "Stream with reference : null not found"; + "No matching stream reference found for attribute 'AGG_TIMESTAMP2'"; Assert.assertTrue(e.getCause().getMessage().endsWith(expectedCauseBy)); } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/stream/event/EventTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/stream/event/EventTestCase.java index 6fbcbb2a91..a489590ad6 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/stream/event/EventTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/stream/event/EventTestCase.java @@ -150,7 +150,7 @@ public void testStreamEventConverter() { MetaStreamEvent metaStreamEvent = new MetaStreamEvent(); metaStreamEvent.addData(volume); - metaStreamEvent.initializeAfterWindowData(); + metaStreamEvent.initializeOnAfterWindowData(); metaStreamEvent.addData(price); metaStreamEvent.addOutputData(symbol); metaStreamEvent.addOutputData(null); //complex attribute @@ -299,7 +299,7 @@ public void testUpdateMetaEvent() { metaStreamEvent.addData(volume); metaStreamEvent.addData(price); metaStreamEvent.addData(symbol); - metaStreamEvent.initializeAfterWindowData(); + metaStreamEvent.initializeOnAfterWindowData(); metaStreamEvent.addData(price); metaStreamEvent.addOutputData(symbol); metaStreamEvent.addOutputData(null); diff --git a/modules/siddhi-query-compiler/src/main/java/io/siddhi/query/compiler/SiddhiCompiler.java b/modules/siddhi-query-compiler/src/main/java/io/siddhi/query/compiler/SiddhiCompiler.java index 3b606348bf..ae8c5c3691 100644 --- a/modules/siddhi-query-compiler/src/main/java/io/siddhi/query/compiler/SiddhiCompiler.java +++ b/modules/siddhi-query-compiler/src/main/java/io/siddhi/query/compiler/SiddhiCompiler.java @@ -42,11 +42,13 @@ import io.siddhi.query.api.execution.query.OnDemandQuery; import io.siddhi.query.api.execution.query.Query; import io.siddhi.query.api.execution.query.StoreQuery; +import io.siddhi.query.api.expression.Expression; import io.siddhi.query.api.expression.constant.TimeConstant; import io.siddhi.query.compiler.exception.SiddhiParserException; import io.siddhi.query.compiler.internal.SiddhiErrorListener; import io.siddhi.query.compiler.internal.SiddhiQLBaseVisitorImpl; -import org.antlr.v4.runtime.ANTLRInputStream; +import org.antlr.v4.runtime.CharStream; +import org.antlr.v4.runtime.CharStreams; import org.antlr.v4.runtime.CommonTokenStream; import org.antlr.v4.runtime.tree.ParseTree; @@ -58,9 +60,8 @@ */ public class SiddhiCompiler { - public static SiddhiApp parse(String source) { - - ANTLRInputStream input = new ANTLRInputStream(source); + public static SiddhiApp parse(String siddhiApp) { + CharStream input = CharStreams.fromString(siddhiApp); SiddhiQLLexer lexer = new SiddhiQLLexer(input); lexer.removeErrorListeners(); lexer.addErrorListener(SiddhiErrorListener.INSTANCE); @@ -76,9 +77,8 @@ public static SiddhiApp parse(String source) { return (SiddhiApp) eval.visit(tree); } - public static StreamDefinition parseStreamDefinition(String source) { - - ANTLRInputStream input = new ANTLRInputStream(source); + public static StreamDefinition parseStreamDefinition(String streamDefinition) { + CharStream input = CharStreams.fromString(streamDefinition); SiddhiQLLexer lexer = new SiddhiQLLexer(input); lexer.removeErrorListeners(); lexer.addErrorListener(SiddhiErrorListener.INSTANCE); @@ -93,9 +93,8 @@ public static StreamDefinition parseStreamDefinition(String source) { return (StreamDefinition) eval.visit(tree); } - public static TableDefinition parseTableDefinition(String source) throws SiddhiParserException { - - ANTLRInputStream input = new ANTLRInputStream(source); + public static TableDefinition parseTableDefinition(String tableDefinition) throws SiddhiParserException { + CharStream input = CharStreams.fromString(tableDefinition); SiddhiQLLexer lexer = new SiddhiQLLexer(input); lexer.removeErrorListeners(); lexer.addErrorListener(SiddhiErrorListener.INSTANCE); @@ -110,9 +109,9 @@ public static TableDefinition parseTableDefinition(String source) throws SiddhiP return (TableDefinition) eval.visit(tree); } - public static AggregationDefinition parseAggregationDefinition(String source) throws SiddhiParserException { - - ANTLRInputStream input = new ANTLRInputStream(source); + public static AggregationDefinition parseAggregationDefinition(String aggregationDefinition) + throws SiddhiParserException { + CharStream input = CharStreams.fromString(aggregationDefinition); SiddhiQLLexer lexer = new SiddhiQLLexer(input); lexer.removeErrorListeners(); lexer.addErrorListener(SiddhiErrorListener.INSTANCE); @@ -127,9 +126,8 @@ public static AggregationDefinition parseAggregationDefinition(String source) th return (AggregationDefinition) eval.visit(tree); } - public static Partition parsePartition(String source) throws SiddhiParserException { - - ANTLRInputStream input = new ANTLRInputStream(source); + public static Partition parsePartition(String partition) throws SiddhiParserException { + CharStream input = CharStreams.fromString(partition); SiddhiQLLexer lexer = new SiddhiQLLexer(input); lexer.removeErrorListeners(); lexer.addErrorListener(SiddhiErrorListener.INSTANCE); @@ -144,9 +142,8 @@ public static Partition parsePartition(String source) throws SiddhiParserExcepti return (Partition) eval.visit(tree); } - public static Query parseQuery(String source) throws SiddhiParserException { - - ANTLRInputStream input = new ANTLRInputStream(source); + public static Query parseQuery(String query) throws SiddhiParserException { + CharStream input = CharStreams.fromString(query); SiddhiQLLexer lexer = new SiddhiQLLexer(input); lexer.removeErrorListeners(); lexer.addErrorListener(SiddhiErrorListener.INSTANCE); @@ -161,8 +158,8 @@ public static Query parseQuery(String source) throws SiddhiParserException { return (Query) eval.visit(tree); } - public static FunctionDefinition parseFunctionDefinition(String source) throws SiddhiParserException { - ANTLRInputStream input = new ANTLRInputStream(source); + public static FunctionDefinition parseFunctionDefinition(String functionDefinition) throws SiddhiParserException { + CharStream input = CharStreams.fromString(functionDefinition); SiddhiQLLexer lexer = new SiddhiQLLexer(input); lexer.removeErrorListeners(); lexer.addErrorListener(SiddhiErrorListener.INSTANCE); @@ -177,8 +174,8 @@ public static FunctionDefinition parseFunctionDefinition(String source) throws S return (FunctionDefinition) eval.visit(tree); } - public static TimeConstant parseTimeConstantDefinition(String source) throws SiddhiParserException { - ANTLRInputStream input = new ANTLRInputStream(source); + public static TimeConstant parseTimeConstantDefinition(String timeConstantDefinition) throws SiddhiParserException { + CharStream input = CharStreams.fromString(timeConstantDefinition); SiddhiQLLexer lexer = new SiddhiQLLexer(input); lexer.removeErrorListeners(); lexer.addErrorListener(SiddhiErrorListener.INSTANCE); @@ -194,8 +191,7 @@ public static TimeConstant parseTimeConstantDefinition(String source) throws Sid } public static OnDemandQuery parseOnDemandQuery(String onDemandQueryString) throws SiddhiParserException { - - ANTLRInputStream input = new ANTLRInputStream(onDemandQueryString); + CharStream input = CharStreams.fromString(onDemandQueryString); SiddhiQLLexer lexer = new SiddhiQLLexer(input); lexer.removeErrorListeners(); lexer.addErrorListener(SiddhiErrorListener.INSTANCE); @@ -216,6 +212,24 @@ public static StoreQuery parseStoreQuery(String storeQuery) throws SiddhiParserE return new StoreQuery(onDemandQuery); } + + public static Expression parseExpression(String expression) { + CharStream input = CharStreams.fromString(expression); + SiddhiQLLexer lexer = new SiddhiQLLexer(input); + lexer.removeErrorListeners(); + lexer.addErrorListener(SiddhiErrorListener.INSTANCE); + + CommonTokenStream tokens = new CommonTokenStream(lexer); + SiddhiQLParser parser = new SiddhiQLParser(tokens); + parser.removeErrorListeners(); + parser.addErrorListener(SiddhiErrorListener.INSTANCE); + ParseTree tree = parser.expression(); + + SiddhiQLVisitor eval = new SiddhiQLBaseVisitorImpl(); + return (Expression) eval.visit(tree); + } + + public static String updateVariables(String siddhiApp) { String updatedSiddhiApp = siddhiApp; if (siddhiApp.contains("$")) { From 8c9f71d017d1f26c1b78515a73fc4efeb5102fd0 Mon Sep 17 00:00:00 2001 From: Suhothayan Date: Mon, 23 Dec 2019 16:30:07 +0530 Subject: [PATCH 2/6] Add sliding expression Window --- .../window/ExpressionWindowProcessor.java | 395 ++++++++++ .../window/ExpressionWindowTestCase.java | 744 ++++++++++++++++++ 2 files changed, 1139 insertions(+) create mode 100644 modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionWindowProcessor.java create mode 100644 modules/siddhi-core/src/test/java/io/siddhi/core/query/window/ExpressionWindowTestCase.java diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionWindowProcessor.java new file mode 100644 index 0000000000..2befdf48a8 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionWindowProcessor.java @@ -0,0 +1,395 @@ +/* + * 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.processor.stream.window; + +import io.siddhi.annotation.Example; +import io.siddhi.annotation.Extension; +import io.siddhi.annotation.Parameter; +import io.siddhi.annotation.ParameterOverload; +import io.siddhi.annotation.util.DataType; +import io.siddhi.core.config.SiddhiOnDemandQueryContext; +import io.siddhi.core.config.SiddhiQueryContext; +import io.siddhi.core.event.ComplexEvent; +import io.siddhi.core.event.ComplexEventChunk; +import io.siddhi.core.event.state.MetaStateEvent; +import io.siddhi.core.event.state.StateEvent; +import io.siddhi.core.event.stream.MetaStreamEvent; +import io.siddhi.core.event.stream.StreamEvent; +import io.siddhi.core.event.stream.StreamEventCloner; +import io.siddhi.core.event.stream.holder.SnapshotableStreamEventQueue; +import io.siddhi.core.event.stream.holder.StreamEventClonerHolder; +import io.siddhi.core.exception.SiddhiAppRuntimeException; +import io.siddhi.core.executor.ConstantExpressionExecutor; +import io.siddhi.core.executor.ExpressionExecutor; +import io.siddhi.core.executor.VariableExpressionExecutor; +import io.siddhi.core.query.processor.ProcessingMode; +import io.siddhi.core.query.processor.Processor; +import io.siddhi.core.table.Table; +import io.siddhi.core.util.collection.operator.CompiledCondition; +import io.siddhi.core.util.collection.operator.MatchingMetaInfoHolder; +import io.siddhi.core.util.collection.operator.Operator; +import io.siddhi.core.util.config.ConfigReader; +import io.siddhi.core.util.parser.ExpressionParser; +import io.siddhi.core.util.parser.OperatorParser; +import io.siddhi.core.util.parser.helper.QueryParserHelper; +import io.siddhi.core.util.snapshot.state.SnapshotStateList; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.query.api.definition.AbstractDefinition; +import io.siddhi.query.api.definition.Attribute; +import io.siddhi.query.api.definition.StreamDefinition; +import io.siddhi.query.api.expression.Expression; +import io.siddhi.query.compiler.SiddhiCompiler; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@link WindowProcessor} which represent a Window operating based on a expression. + */ +@Extension( + name = "expression", + namespace = "", + description = "A sliding window that holds events that satisfy the given `expression`, when they aren't, " + + "they are evaluated from the `first` (oldest) to the `last` (latest/current) and expired " + + "from the oldest until the `expression` is satisfied.\n" + + "**Note**: All the events in window are reevaluated only when the given `expression` is changed.", + parameters = { + @Parameter(name = "expression", + description = "The expression to retain events.", + type = {DataType.STRING}, + dynamic = true + ) + }, + parameterOverloads = { + @ParameterOverload(parameterNames = {"expression"}) + }, + examples = { + @Example( + syntax = "@info(name = 'query1')\n" + + "from StockEventWindow#window.expression('count()>20')\n" + + "select symbol, sum(price) as price\n" + + "insert into OutputStream ;", + description = "This will retain last 20 events in a sliding manner." + ), + @Example( + syntax = "@info(name = 'query1')\n" + + "from StockEventWindow#window.expression(\n" + + " 'sum(price) < 100 and eventTimestamp(last) - eventTimestamp(first) < 3000')\n" + + "select symbol, sum(price) as price\n" + + "insert into OutputStream ;", + description = "This will retain the latest events having their sum(price) < 100, " + + "and the `last` and `first` events are within 3 second difference." + )} +) +public class ExpressionWindowProcessor extends SlidingFindableWindowProcessor { + + private ExpressionExecutor expressionExecutor; + private List variableExpressionExecutors; + private boolean init = false; + private ExpressionExecutor expressionStringExecutor = null; + private String expressionString = null; + + @Override + protected StateFactory init(MetaStreamEvent metaStreamEvent, + AbstractDefinition inputDefinition, + ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { + if (attributeExpressionExecutors[0] instanceof ConstantExpressionExecutor) { + expressionString = (String) ((ConstantExpressionExecutor) + attributeExpressionExecutors[0]).getValue(); + constructExpression(metaStreamEvent, siddhiQueryContext); + } else { + for (Attribute attribute : inputDefinition.getAttributeList()) { + metaStreamEvent.addData(attribute); + } + expressionStringExecutor = attributeExpressionExecutors[0]; + } + return () -> new WindowState(); + } + + @Override + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { + //will not be called. + return null; + } + + @Override + protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, + StreamEventCloner streamEventCloner, WindowState state) { + + if (expressionStringExecutor == null && !init) { + MetaStateEvent metaStateEvent = new MetaStateEvent( + new MetaStreamEvent[]{metaStreamEvent, metaStreamEvent, metaStreamEvent}); + QueryParserHelper.updateVariablePosition(metaStateEvent, variableExpressionExecutors); + init = true; + } + synchronized (state) { + long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); + while (streamEventChunk.hasNext()) { + StreamEvent streamEvent = streamEventChunk.next(); + if (expressionStringExecutor != null) { + String expressionStringNew = (String) expressionStringExecutor.execute(streamEvent); + if (!expressionStringNew.equals(expressionString)) { + expressionString = expressionStringNew; + processAllExpiredEvents(streamEventChunk, streamEventCloner, state, currentTime); + } + } + StreamEvent clonedEvent = streamEventCloner.copyStreamEvent(streamEvent); + clonedEvent.setType(StreamEvent.Type.EXPIRED); + processStreamEvent(streamEventChunk, state, currentTime, clonedEvent); + } + } + nextProcessor.process(streamEventChunk); + } + + private MetaStateEvent constructExpression(MetaStreamEvent metaStreamEvent, + SiddhiQueryContext siddhiQueryContext) { + Expression expression = SiddhiCompiler.parseExpression(expressionString); + MetaStreamEvent metaStreamEventFirst = new MetaStreamEventWrapper(metaStreamEvent); + metaStreamEventFirst.setInputReferenceId("first"); + MetaStreamEvent metaStreamEventLast = new MetaStreamEventWrapper(metaStreamEvent); + metaStreamEventLast.setInputReferenceId("last"); + MetaStateEvent metaStateEvent = new MetaStateEvent( + new MetaStreamEvent[]{metaStreamEvent, metaStreamEventFirst, metaStreamEventLast}); + variableExpressionExecutors = new ArrayList<>(); + SiddhiQueryContext exprQueryContext = new SiddhiOnDemandQueryContext( + siddhiQueryContext.getSiddhiAppContext(), siddhiQueryContext.getName(), + expressionString); + expressionExecutor = ExpressionParser.parseExpression(expression, metaStateEvent, + 0, new HashMap<>(), variableExpressionExecutors, false, + 0, ProcessingMode.SLIDE, true, exprQueryContext); + if (expressionExecutor.getReturnType() != Attribute.Type.BOOL) { + throw new SiddhiAppRuntimeException("Expression (" + expressionString + ") does not return Bool"); + } + return metaStateEvent; + } + + private void processAllExpiredEvents(ComplexEventChunk streamEventChunk, + StreamEventCloner streamEventCloner, WindowState state, + long currentTime) { + MetaStateEvent metaStateEvent = constructExpression(metaStreamEvent, siddhiQueryContext); + QueryParserHelper.updateVariablePosition(metaStateEvent, variableExpressionExecutors); + StreamEvent expiredEvent = state.expiredEventQueue.getFirst(); + state.expiredEventQueue.clear(); + while (expiredEvent != null) { + StreamEvent aExpiredEvent = expiredEvent; + expiredEvent = expiredEvent.getNext(); + aExpiredEvent.setNext(null); + processStreamEvent(streamEventChunk, state, currentTime, aExpiredEvent); + } + state.expiredEventQueue.reset(); + } + + private void processStreamEvent(ComplexEventChunk streamEventChunk, + WindowState state, long currentTime, StreamEvent streamEvent) { + state.expiredEventQueue.add(streamEvent); + StateEvent stateEventCurrent = new StateEvent(3, 0); + stateEventCurrent.setEvent(0, streamEvent); + stateEventCurrent.setEvent(1, state.expiredEventQueue.getFirst()); + stateEventCurrent.setEvent(2, streamEvent); + if (!(Boolean) expressionExecutor.execute(stateEventCurrent)) { + state.expiredEventQueue.reset(); + while (state.expiredEventQueue.hasNext()) { + StreamEvent expiredEvent = state.expiredEventQueue.next(); + state.expiredEventQueue.remove(); + streamEventChunk.insertBeforeCurrent(expiredEvent); + StateEvent stateEvent = new StateEvent(3, 0); + stateEvent.setEvent(0, expiredEvent); + StreamEvent firstEvent = state.expiredEventQueue.getFirst(); + if (firstEvent == null) { + firstEvent = expiredEvent; + } + stateEvent.setEvent(1, firstEvent); + stateEvent.setEvent(2, streamEvent); + stateEvent.setType(ComplexEvent.Type.EXPIRED); + expiredEvent.setTimestamp(currentTime); + if ((Boolean) expressionExecutor.execute(stateEvent)) { + break; + } + + + } + } + } + + @Override + public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, + List variableExpressionExecutors, + Map tableMap, WindowState state, + SiddhiQueryContext siddhiQueryContext) { + return OperatorParser.constructOperator(state.expiredEventQueue, condition, matchingMetaInfoHolder, + variableExpressionExecutors, tableMap, siddhiQueryContext); + } + + @Override + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.expiredEventQueue, streamEventCloner); + + } + + @Override + public void start() { + //Do nothing + } + + @Override + public void stop() { + //Do nothing + } + + + class WindowState extends State { + + private SnapshotableStreamEventQueue expiredEventQueue = + new SnapshotableStreamEventQueue(streamEventClonerHolder); + + @Override + public boolean canDestroy() { + return expiredEventQueue.getFirst() == null; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("ExpiredEventQueue", expiredEventQueue.getSnapshot()); + return state; + } + + @Override + public void restore(Map state) { + expiredEventQueue.clear(); + expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + } + } + + class MetaStreamEventWrapper extends MetaStreamEvent { + private MetaStreamEvent metaStreamEvent; + private String inputReferenceId; + + public MetaStreamEventWrapper(MetaStreamEvent metaStreamEvent) { + this.metaStreamEvent = metaStreamEvent; + } + + @Override + public List getBeforeWindowData() { + return metaStreamEvent.getBeforeWindowData(); + } + + @Override + public List getOnAfterWindowData() { + return metaStreamEvent.getOnAfterWindowData(); + } + + @Override + public List getOutputData() { + return metaStreamEvent.getOutputData(); + } + + @Override + public void initializeOnAfterWindowData() { + metaStreamEvent.initializeOnAfterWindowData(); + } + + @Override + public void initializeAfterWindowData() { + metaStreamEvent.initializeAfterWindowData(); + } + + @Override + public int addData(Attribute attribute) { + return metaStreamEvent.addData(attribute); + } + + @Override + public void addOutputData(Attribute attribute) { + metaStreamEvent.addOutputData(attribute); + } + + @Override + public void addOutputDataAllowingDuplicate(Attribute attribute) { + metaStreamEvent.addOutputDataAllowingDuplicate(attribute); + } + + @Override + public List getInputDefinitions() { + return metaStreamEvent.getInputDefinitions(); + } + + @Override + public void addInputDefinition(AbstractDefinition inputDefinition) { + metaStreamEvent.addInputDefinition(inputDefinition); + } + + @Override + public String getInputReferenceId() { + return inputReferenceId; + } + + @Override + public void setInputReferenceId(String inputReferenceId) { + this.inputReferenceId = inputReferenceId; + } + + @Override + public void setOutputDefinition(StreamDefinition streamDefinition) { + metaStreamEvent.setOutputDefinition(streamDefinition); + } + + @Override + public StreamDefinition getOutputStreamDefinition() { + return metaStreamEvent.getOutputStreamDefinition(); + } + + @Override + public EventType getEventType() { + return metaStreamEvent.getEventType(); + } + + @Override + public void setEventType(EventType eventType) { + metaStreamEvent.setEventType(eventType); + } + + @Override + public AbstractDefinition getLastInputDefinition() { + return metaStreamEvent.getLastInputDefinition(); + } + + @Override + public boolean isMultiValue() { + return metaStreamEvent.isMultiValue(); + } + + @Override + public void setMultiValue(boolean multiValue) { + metaStreamEvent.setMultiValue(multiValue); + } + + @Override + public MetaStreamEvent clone() { + return metaStreamEvent.clone(); + } + } +} diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/ExpressionWindowTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/ExpressionWindowTestCase.java new file mode 100644 index 0000000000..d1544c18db --- /dev/null +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/ExpressionWindowTestCase.java @@ -0,0 +1,744 @@ +/* + * 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.window; + +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 org.apache.log4j.Logger; +import org.testng.AssertJUnit; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class ExpressionWindowTestCase { + private static final Logger log = Logger.getLogger(ExpressionWindowTestCase.class); + private int inEventCount; + private int removeEventCount; + private int count; + private boolean eventArrived; + private Event storedEvent; + + + @BeforeMethod + public void init() { + count = 0; + inEventCount = 0; + removeEventCount = 0; + eventArrived = false; + storedEvent = null; + } + + @Test + public void expressionWindowTest1() throws InterruptedException { + log.info("Testing expression window with no of events"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "" + + "define stream cseEventStream (symbol string, price float, volume int);"; + String query = "" + + "@info(name = 'query1') " + + "from cseEventStream#window.expression('count() <= 2') " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(1, removeEvents.length); + } + eventArrived = true; + } + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + inputHandler.send(new Object[]{"IBM", 700f, 0}); + inputHandler.send(new Object[]{"WSO2", 60.5f, 1}); + inputHandler.send(new Object[]{"WSO2", 61.5f, 2}); + inputHandler.send(new Object[]{"WSO2", 62.5f, 3}); + inputHandler.send(new Object[]{"WSO2", 63.5f, 4}); + AssertJUnit.assertEquals(5, inEventCount); + AssertJUnit.assertEquals(3, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + + @Test + public void expressionWindowTest2() throws InterruptedException { + log.info("Testing expression window 2"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression('last.volume - first.volume <= 2') " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(1, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + inputHandler.send(new Object[]{"WSO2", 60.5f, 0}); + inputHandler.send(new Object[]{"WSO2", 61.5f, 1}); + inputHandler.send(new Object[]{"WSO2", 62.5f, 2}); + inputHandler.send(new Object[]{"WSO2", 63.5f, 3}); + inputHandler.send(new Object[]{"WSO2", 64.5f, 4}); + AssertJUnit.assertEquals(5, inEventCount); + AssertJUnit.assertEquals(2, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + + @Test + public void expressionWindowTest3() throws InterruptedException { + log.info("Testing expression window 3"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression('eventTimestamp(last) - eventTimestamp(first) <= 2') " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(1, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2}); + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4}); + AssertJUnit.assertEquals(5, inEventCount); + AssertJUnit.assertEquals(2, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + + @Test + public void expressionWindowTest4() throws InterruptedException { + log.info("Testing expression window 4"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression('eventTimestamp(last) - eventTimestamp(first) <= 2') " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(7, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(4, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + inputHandler.send(new Event[]{ + new Event(0, new Object[]{"WSO2", 60.5f, 0}), + new Event(1, new Object[]{"WSO2", 61.5f, 1}), + new Event(2, new Object[]{"WSO2", 62.5f, 2}), + new Event(3, new Object[]{"WSO2", 63.5f, 3}), + new Event(4, new Object[]{"WSO2", 64.5f, 4}), + new Event(5, new Object[]{"WSO2", 65.5f, 5}), + new Event(6, new Object[]{"WSO2", 66.5f, 6}), + }); + AssertJUnit.assertEquals(7, inEventCount); + AssertJUnit.assertEquals(4, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + + @Test + public void expressionWindowTest5() throws InterruptedException { + log.info("Testing expression window 5"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(1, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) <= 2"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr}); + AssertJUnit.assertEquals(5, inEventCount); + AssertJUnit.assertEquals(2, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionWindowTest6() throws InterruptedException { + log.info("Testing expression window 6"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(1, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 4"; + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr}); + AssertJUnit.assertEquals(5, inEventCount); + AssertJUnit.assertEquals(1, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionWindowTest7() throws InterruptedException { + log.info("Testing expression window 7"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length || 3 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(7, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionWindowTest8() throws InterruptedException { + log.info("Testing expression window 8"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + AssertJUnit.assertEquals(3, inEventCount); + AssertJUnit.assertEquals(3, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + + @Test + public void expressionWindowTest9() throws InterruptedException { + log.info("Testing expression window 9"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(3, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(3 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(new Event[]{ + new Event(0, new Object[]{"WSO2", 60.5f, 0, expr}), + new Event(1, new Object[]{"WSO2", 61.5f, 1, expr}), + new Event(2, new Object[]{"WSO2", 62.5f, 2, expr}) + }); + AssertJUnit.assertEquals(3, inEventCount); + AssertJUnit.assertEquals(3, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionWindowTest10() throws InterruptedException { + log.info("Testing expression window 10"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(9, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(7 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr1 = "eventTimestamp(last) - eventTimestamp(first) < 2"; + String expr2 = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(new Event[]{ + new Event(0, new Object[]{"WSO2", 60.5f, 0, expr1}), + new Event(1, new Object[]{"WSO2", 61.5f, 1, expr1}), + new Event(2, new Object[]{"WSO2", 62.5f, 2, expr1}), + new Event(3, new Object[]{"WSO2", 63.5f, 3, expr2}), + new Event(4, new Object[]{"WSO2", 64.5f, 4, expr2}), + new Event(5, new Object[]{"WSO2", 65.5f, 5, expr1}), + new Event(6, new Object[]{"WSO2", 66.5f, 6, expr1}), + new Event(7, new Object[]{"WSO2", 67.5f, 7, expr1}), + new Event(8, new Object[]{"WSO2", 68.5f, 8, expr1}) + }); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(7, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionWindowTest11() throws InterruptedException { + log.info("Testing expression window 11"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length || 3 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 4"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(7, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionWindowTest12() throws InterruptedException { + log.info("Testing expression window 12"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price double, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "sum(price) < 50"; + inputHandler.send(0, new Object[]{"WSO2", 10.0, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 10.0, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 10.0, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 10.0, 4, expr}); + inputHandler.send(5, new Object[]{"WSO2", 10.0, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 10.0, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 10.0, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 10.0, 8, expr}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(5, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionWindowTest13() throws InterruptedException { + log.info("Testing expression window 13"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price double, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "sum(price) < 30"; + inputHandler.send(0, new Object[]{"WSO2", 10.0, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 10.0, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 10.0, 3, expr}); + expr = "sum(price) < 40"; + inputHandler.send(4, new Object[]{"WSO2", 10.0, 4, expr}); + inputHandler.send(5, new Object[]{"WSO2", 10.0, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 10.0, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 10.0, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 10.0, 8, expr}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(6, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + + @Test + public void expressionWindowTest14() throws InterruptedException { + log.info("Testing expression window 14"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price double, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length || 3 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "sum(price) < 30"; + inputHandler.send(0, new Object[]{"WSO2", 10.0, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 10.0, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 10.0, 3, expr}); + expr = "sum(price) < 0"; + inputHandler.send(4, new Object[]{"WSO2", 10.0, 4, expr}); + inputHandler.send(5, new Object[]{"WSO2", 10.0, 5, expr}); + expr = "sum(price) < 30"; + inputHandler.send(6, new Object[]{"WSO2", 10.0, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 10.0, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 10.0, 8, expr}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(7, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionWindowTest15() throws InterruptedException { + log.info("Testing expression window 15"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price double, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expression(expr) " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length || 2 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "sum(price) < 30 and eventTimestamp(last) - eventTimestamp(first) < 3"; + inputHandler.send(0, new Object[]{"WSO2", 10.0, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 5.0, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 2, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 3, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 4, expr}); + inputHandler.send(7, new Object[]{"WSO2", 10.0, 5, expr}); + inputHandler.send(8, new Object[]{"WSO2", 15.0, 6, expr}); + inputHandler.send(9, new Object[]{"WSO2", 10.0, 7, expr}); + inputHandler.send(10, new Object[]{"WSO2", 10.0, 8, expr}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(7, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + +} From f553f6e6319386dc64f75c6bad85fc99a781b8ca Mon Sep 17 00:00:00 2001 From: Suhothayan Date: Mon, 23 Dec 2019 16:30:51 +0530 Subject: [PATCH 3/6] Add sliding expression Window --- modules/siddhi-core/src/test/resources/testng.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/modules/siddhi-core/src/test/resources/testng.xml b/modules/siddhi-core/src/test/resources/testng.xml index ea3dcc6b40..091b9db868 100644 --- a/modules/siddhi-core/src/test/resources/testng.xml +++ b/modules/siddhi-core/src/test/resources/testng.xml @@ -110,6 +110,7 @@ + From 2f69ef3607b6bfef693ea546ee8cc534f0e0ff9a Mon Sep 17 00:00:00 2001 From: Suhothayan Date: Tue, 24 Dec 2019 14:12:31 +0530 Subject: [PATCH 4/6] Add Batch expression Window --- .../ExpressionBatchWindowProcessor.java | 589 +++++++++ .../window/ExpressionWindowProcessor.java | 12 +- .../window/ExpressionBatchWindowTestCase.java | 1145 +++++++++++++++++ .../window/LengthBatchWindowTestCase.java | 120 ++ .../siddhi-core/src/test/resources/testng.xml | 1 + 5 files changed, 1861 insertions(+), 6 deletions(-) create mode 100644 modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionBatchWindowProcessor.java create mode 100644 modules/siddhi-core/src/test/java/io/siddhi/core/query/window/ExpressionBatchWindowTestCase.java diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionBatchWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionBatchWindowProcessor.java new file mode 100644 index 0000000000..a9c372d140 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionBatchWindowProcessor.java @@ -0,0 +1,589 @@ +/* + * 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.processor.stream.window; + +import io.siddhi.annotation.Example; +import io.siddhi.annotation.Extension; +import io.siddhi.annotation.Parameter; +import io.siddhi.annotation.ParameterOverload; +import io.siddhi.annotation.util.DataType; +import io.siddhi.core.config.SiddhiOnDemandQueryContext; +import io.siddhi.core.config.SiddhiQueryContext; +import io.siddhi.core.event.ComplexEvent; +import io.siddhi.core.event.ComplexEventChunk; +import io.siddhi.core.event.state.MetaStateEvent; +import io.siddhi.core.event.state.StateEvent; +import io.siddhi.core.event.stream.MetaStreamEvent; +import io.siddhi.core.event.stream.StreamEvent; +import io.siddhi.core.event.stream.StreamEventCloner; +import io.siddhi.core.event.stream.holder.SnapshotableStreamEventQueue; +import io.siddhi.core.event.stream.holder.StreamEventClonerHolder; +import io.siddhi.core.exception.SiddhiAppRuntimeException; +import io.siddhi.core.executor.ConstantExpressionExecutor; +import io.siddhi.core.executor.ExpressionExecutor; +import io.siddhi.core.executor.VariableExpressionExecutor; +import io.siddhi.core.query.processor.ProcessingMode; +import io.siddhi.core.query.processor.Processor; +import io.siddhi.core.table.Table; +import io.siddhi.core.util.collection.operator.CompiledCondition; +import io.siddhi.core.util.collection.operator.MatchingMetaInfoHolder; +import io.siddhi.core.util.collection.operator.Operator; +import io.siddhi.core.util.config.ConfigReader; +import io.siddhi.core.util.parser.ExpressionParser; +import io.siddhi.core.util.parser.OperatorParser; +import io.siddhi.core.util.parser.helper.QueryParserHelper; +import io.siddhi.core.util.snapshot.state.SnapshotStateList; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.query.api.definition.AbstractDefinition; +import io.siddhi.query.api.definition.Attribute; +import io.siddhi.query.api.definition.StreamDefinition; +import io.siddhi.query.api.expression.Expression; +import io.siddhi.query.compiler.SiddhiCompiler; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@link WindowProcessor} which represent a Window operating based on a expression. + */ +@Extension( + name = "expressionBatch", + namespace = "", + description = "A batch window that dynamically shrink and grow based on the `expression`, " + + "it holds events until the `expression` is satisfied, and expires " + + "all when the `expression` is not satisfied." + + "When a string is passed as the `expression` it is evaluated from the `first` (oldest) to the " + + "`last` (latest/current).\n" + + "**Note**: All the events in window are reevaluated only when the given `expression` is changed.", + parameters = { + @Parameter(name = "expression", + description = "The expression to retain events.", + type = {DataType.STRING, DataType.BOOL}, + dynamic = true + ), + @Parameter(name = "include.triggering.event", + description = "Include the event triggered the expiry in to the current event batch.", + type = {DataType.BOOL}, + optional = true, + defaultValue = "false", + dynamic = true + ), + @Parameter(name = "stream.current.event", + description = "Let the window stream the current events out as and when they arrive" + + " to the window while expiring them in batches.", + type = {DataType.BOOL}, + optional = true, + defaultValue = "false" + ) + }, + parameterOverloads = { + @ParameterOverload(parameterNames = {"expression"}), + @ParameterOverload(parameterNames = {"expression", "include.triggering.event"}), + @ParameterOverload(parameterNames = {"expression", "include.triggering.event", "stream.current.event"}) + }, + examples = { + @Example( + syntax = "@info(name = 'query1')\n" + + "from StockEventWindow#window.expressionBatch('count()<=20')\n" + + "select symbol, sum(price) as price\n" + + "insert into OutputStream ;", + description = "Retain and output 20 events at a time as batch." + ), + @Example( + syntax = "@info(name = 'query1')\n" + + "from StockEventWindow#window.expressionBatch(\n" + + " 'sum(price) < 100 and eventTimestamp(last) - eventTimestamp(first) < 3000')\n" + + "select symbol, sum(price) as price\n" + + "insert into OutputStream ;", + description = "Retain and output events having their sum(price) < 100, " + + "and the `last` and `first` events are within 3 second difference as a batch." + ), + @Example( + syntax = "@info(name = 'query1')\n" + + "from StockEventWindow#window.expressionBatch(\n" + + " 'last.symbol==first.symbol')\n" + + "select symbol, sum(price) as price\n" + + "insert into OutputStream ;", + description = "Output events as a batch when a new symbol type arrives." + ), + @Example( + syntax = "@info(name = 'query1')\n" + + "from StockEventWindow#window.expressionBatch(\n" + + " 'flush', true)\n" + + "select symbol, sum(price) as price\n" + + "insert into OutputStream ;", + description = "Output events as a batch when a flush attribute becomes `true`, the output " + + "batch will also contain the triggering event." + ), + @Example( + syntax = "@info(name = 'query1')\n" + + "from StockEventWindow#window.expressionBatch(\n" + + " 'flush', false, true)\n" + + "select symbol, sum(price) as price\n" + + "insert into OutputStream ;", + description = "Arriving events are emitted as soon as they are arrived, and the retained " + + "events are expired when flush attribute becomes `true`, and the output " + + "batch will not contain the triggering event." + )} +) +public class ExpressionBatchWindowProcessor extends + BatchingFindableWindowProcessor { + + private ExpressionExecutor expressionExecutor; + private List variableExpressionExecutors; + private boolean init = false; + private ExpressionExecutor expressionStringExecutor = null; + private String expressionString = null; + private boolean includeTriggeringEvent = false; + private ExpressionExecutor includeTriggeringEventExecutor = null; + private Boolean streamInputEvents = false; + + @Override + protected StateFactory init(MetaStreamEvent metaStreamEvent, + AbstractDefinition inputDefinition, + ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { + if (attributeExpressionExecutors[0] instanceof ConstantExpressionExecutor) { + expressionString = (String) ((ConstantExpressionExecutor) + attributeExpressionExecutors[0]).getValue(); + constructExpression(metaStreamEvent, siddhiQueryContext); + } else { + for (Attribute attribute : inputDefinition.getAttributeList()) { + metaStreamEvent.addData(attribute); + } + expressionStringExecutor = attributeExpressionExecutors[0]; + } + if (attributeExpressionExecutors.length > 1) { + if (attributeExpressionExecutors[1] instanceof ConstantExpressionExecutor) { + includeTriggeringEvent = (Boolean) ((ConstantExpressionExecutor) + attributeExpressionExecutors[1]).getValue(); + } else { + includeTriggeringEventExecutor = attributeExpressionExecutors[1]; + } + if (attributeExpressionExecutors.length > 2 && + attributeExpressionExecutors[2] instanceof ConstantExpressionExecutor) { + streamInputEvents = (Boolean) ((ConstantExpressionExecutor) + attributeExpressionExecutors[2]).getValue(); + } + } + return () -> new WindowState(); + } + + @Override + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { + //will not be called. + return null; + } + + @Override + protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, + StreamEventCloner streamEventCloner, WindowState state) { + + if (expressionStringExecutor == null && !init) { + MetaStateEvent metaStateEvent = new MetaStateEvent( + new MetaStreamEvent[]{metaStreamEvent, metaStreamEvent, metaStreamEvent}); + QueryParserHelper.updateVariablePosition(metaStateEvent, variableExpressionExecutors); + init = true; + } + List> streamEventChunks = new ArrayList<>(); + synchronized (state) { + long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); + while (streamEventChunk.hasNext()) { + StreamEvent streamEvent = streamEventChunk.next(); + streamEventChunk.remove(); + if (expressionStringExecutor != null) { + String expressionStringNew = (String) expressionStringExecutor.execute(streamEvent); + if (!expressionStringNew.equals(expressionString)) { + expressionString = expressionStringNew; + processAllExpiredEvents(streamEventCloner, state, currentTime, streamEventChunks); + } + } + if (!streamInputEvents) { + processStreamEvent(state, currentTime, streamEvent, streamEventCloner, streamEventChunks); + } else { + processStreamEventAsStream(state, currentTime, streamEvent, streamEventCloner, streamEventChunks, + true); + } + } + } + for (ComplexEventChunk outputStreamEventChunk : streamEventChunks) { + nextProcessor.process(outputStreamEventChunk); + } + } + + private MetaStateEvent constructExpression(MetaStreamEvent metaStreamEvent, + SiddhiQueryContext siddhiQueryContext) { + Expression expression = SiddhiCompiler.parseExpression(expressionString); + MetaStreamEvent metaStreamEventFirst = new MetaStreamEventWrapper(metaStreamEvent); + metaStreamEventFirst.setInputReferenceId("first"); + MetaStreamEvent metaStreamEventLast = new MetaStreamEventWrapper(metaStreamEvent); + metaStreamEventLast.setInputReferenceId("last"); + MetaStateEvent metaStateEvent = new MetaStateEvent( + new MetaStreamEvent[]{metaStreamEvent, metaStreamEventFirst, metaStreamEventLast}); + variableExpressionExecutors = new ArrayList<>(); + SiddhiQueryContext exprQueryContext = new SiddhiOnDemandQueryContext( + siddhiQueryContext.getSiddhiAppContext(), siddhiQueryContext.getName(), + expressionString); + expressionExecutor = ExpressionParser.parseExpression(expression, metaStateEvent, + 0, new HashMap<>(), variableExpressionExecutors, false, + 0, ProcessingMode.SLIDE, true, exprQueryContext); + if (expressionExecutor.getReturnType() != Attribute.Type.BOOL) { + throw new SiddhiAppRuntimeException("Expression ('" + expressionString + "') does not return Bool"); + } + return metaStateEvent; + } + + private void processAllExpiredEvents(StreamEventCloner streamEventCloner, WindowState state, + long currentTime, List> streamEventChunks) { + MetaStateEvent metaStateEvent = constructExpression(metaStreamEvent, siddhiQueryContext); + QueryParserHelper.updateVariablePosition(metaStateEvent, variableExpressionExecutors); + if (!streamInputEvents) { + StreamEvent expiredEvent = state.currentEventQueue.getFirst(); + state.currentEventQueue.clear(); + while (expiredEvent != null) { + StreamEvent aExpiredEvent = expiredEvent; + expiredEvent = expiredEvent.getNext(); + aExpiredEvent.setNext(null); + processStreamEvent(state, currentTime, aExpiredEvent, streamEventCloner, streamEventChunks); + } + state.currentEventQueue.reset(); + } else { + StreamEvent expiredEvent = state.expiredEventQueue.getFirst(); + state.expiredEventQueue.clear(); + while (expiredEvent != null) { + StreamEvent aExpiredEvent = expiredEvent; + expiredEvent = expiredEvent.getNext(); + aExpiredEvent.setNext(null); + processStreamEventAsStream(state, currentTime, aExpiredEvent, streamEventCloner, streamEventChunks, + false); + } + state.expiredEventQueue.reset(); + } + } + + private void processStreamEvent(WindowState state, long currentTime, StreamEvent streamEvent, + StreamEventCloner streamEventCloner, + List> streamEventChunks) { + StateEvent stateEventCurrent = new StateEvent(3, 0); + stateEventCurrent.setEvent(0, streamEvent); + if (state.currentEventQueue.getFirst() != null) { + stateEventCurrent.setEvent(1, state.currentEventQueue.getFirst()); + } else { + stateEventCurrent.setEvent(1, streamEvent); + } + stateEventCurrent.setEvent(2, streamEvent); + if (!(Boolean) expressionExecutor.execute(stateEventCurrent)) { + stateEventCurrent.setType(ComplexEvent.Type.RESET); + expressionExecutor.execute(stateEventCurrent); + stateEventCurrent.setType(ComplexEvent.Type.CURRENT); + expressionExecutor.execute(stateEventCurrent); + ComplexEventChunk outputStreamEventChunk = new ComplexEventChunk<>(); + state.expiredEventQueue.reset(); + if (state.expiredEventQueue.getFirst() != null) { + while (state.expiredEventQueue.hasNext()) { + StreamEvent expiredEvent = state.expiredEventQueue.next(); + expiredEvent.setTimestamp(currentTime); + } + outputStreamEventChunk.add(state.expiredEventQueue.getFirst()); + state.expiredEventQueue.clear(); + } + if (state.currentEventQueue.getFirst() != null) { + while (state.currentEventQueue.hasNext()) { + StreamEvent currentEvent = state.currentEventQueue.next(); + currentEvent.setTimestamp(currentTime); + StreamEvent expiredEvent = streamEventCloner.copyStreamEvent(currentEvent); + expiredEvent.setType(ComplexEvent.Type.EXPIRED); + state.expiredEventQueue.add(expiredEvent); + } + outputStreamEventChunk.add(state.currentEventQueue.getFirst()); + state.currentEventQueue.clear(); + if (includeTriggeringEventExecutor != null && + (Boolean) includeTriggeringEventExecutor.execute(streamEvent) || + includeTriggeringEvent) { + outputStreamEventChunk.add(streamEvent); + StreamEvent expiredEvent = streamEventCloner.copyStreamEvent(streamEvent); + expiredEvent.setTimestamp(currentTime); + expiredEvent.setType(ComplexEvent.Type.EXPIRED); + state.expiredEventQueue.add(expiredEvent); + } else { + state.currentEventQueue.add(streamEvent); + } + } else { + StreamEvent expiredEvent = streamEventCloner.copyStreamEvent(streamEvent); + expiredEvent.setType(ComplexEvent.Type.EXPIRED); + expiredEvent.setTimestamp(currentTime); + outputStreamEventChunk.add(expiredEvent); + streamEvent.setTimestamp(currentTime); + outputStreamEventChunk.add(streamEvent); + } + streamEventChunks.add(outputStreamEventChunk); + } else { + state.currentEventQueue.add(streamEvent); + } + } + + private void processStreamEventAsStream(WindowState state, long currentTime, StreamEvent streamEvent, + StreamEventCloner streamEventCloner, + List> streamEventChunks, boolean output) { + StateEvent stateEventCurrent = new StateEvent(3, 0); + stateEventCurrent.setEvent(0, streamEvent); + if (state.expiredEventQueue.getFirst() != null) { + stateEventCurrent.setEvent(1, state.expiredEventQueue.getFirst()); + } else { + stateEventCurrent.setEvent(1, streamEvent); + } + stateEventCurrent.setEvent(2, streamEvent); + ComplexEventChunk outputStreamEventChunk = new ComplexEventChunk<>(); + if (output && state.currentEventQueue.getFirst() != null) { + outputStreamEventChunk.add(state.currentEventQueue.getFirst()); + state.currentEventQueue.clear(); + } + if (!(Boolean) expressionExecutor.execute(stateEventCurrent)) { + stateEventCurrent.setType(ComplexEvent.Type.RESET); + expressionExecutor.execute(stateEventCurrent); + stateEventCurrent.setType(ComplexEvent.Type.CURRENT); + expressionExecutor.execute(stateEventCurrent); + state.expiredEventQueue.reset(); + if (state.expiredEventQueue.getFirst() != null) { + while (state.expiredEventQueue.hasNext()) { + StreamEvent expiredEvent = state.expiredEventQueue.next(); + expiredEvent.setTimestamp(currentTime); + } + StreamEvent expiredEvent = streamEventCloner.copyStreamEvent(streamEvent); + expiredEvent.setType(ComplexEvent.Type.EXPIRED); + if (includeTriggeringEventExecutor != null && + (Boolean) includeTriggeringEventExecutor.execute(streamEvent) || + includeTriggeringEvent) { + outputStreamEventChunk.add(streamEvent); + if (output) { + outputStreamEventChunk.add(state.expiredEventQueue.getFirst()); + expiredEvent.setTimestamp(currentTime); + outputStreamEventChunk.add(expiredEvent); + } else { + state.currentEventQueue.add(state.expiredEventQueue.getFirst()); + state.currentEventQueue.add(expiredEvent); + } + state.expiredEventQueue.clear(); + } else { + if (output) { + outputStreamEventChunk.add(state.expiredEventQueue.getFirst()); + } else { + state.currentEventQueue.add(state.expiredEventQueue.getFirst()); + } + outputStreamEventChunk.add(streamEvent); + state.expiredEventQueue.clear(); + state.expiredEventQueue.add(expiredEvent); + } + + } else if (output) { + StreamEvent expiredEvent = streamEventCloner.copyStreamEvent(streamEvent); + expiredEvent.setType(ComplexEvent.Type.EXPIRED); + expiredEvent.setTimestamp(currentTime); + outputStreamEventChunk.add(expiredEvent); + streamEvent.setTimestamp(currentTime); + outputStreamEventChunk.add(streamEvent); + } else { + state.currentEventQueue.add(streamEvent); + } + if (output) { + streamEventChunks.add(outputStreamEventChunk); + } + } else { + StreamEvent expiredEvent = streamEventCloner.copyStreamEvent(streamEvent); + expiredEvent.setType(ComplexEvent.Type.EXPIRED); + state.expiredEventQueue.add(expiredEvent); + if (output) { + outputStreamEventChunk.add(streamEvent); + streamEventChunks.add(outputStreamEventChunk); + } + } + } + + @Override + public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, + List variableExpressionExecutors, + Map tableMap, WindowState state, + SiddhiQueryContext siddhiQueryContext) { + return OperatorParser.constructOperator(state.currentEventQueue, condition, matchingMetaInfoHolder, + variableExpressionExecutors, tableMap, siddhiQueryContext); + } + + @Override + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.currentEventQueue, streamEventCloner); + + } + + @Override + public void start() { + //Do nothing + } + + @Override + public void stop() { + //Do nothing + } + + + class WindowState extends State { + + private SnapshotableStreamEventQueue currentEventQueue = + new SnapshotableStreamEventQueue(streamEventClonerHolder); + private SnapshotableStreamEventQueue expiredEventQueue = + new SnapshotableStreamEventQueue(streamEventClonerHolder); + + @Override + public boolean canDestroy() { + return currentEventQueue.getFirst() == null && expiredEventQueue.getFirst() == null; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("CurrentEventQueue", currentEventQueue.getSnapshot()); + state.put("ExpiredEventQueue", expiredEventQueue.getSnapshot()); + return state; + } + + @Override + public void restore(Map state) { + currentEventQueue.clear(); + currentEventQueue.restore((SnapshotStateList) state.get("CurrentEventQueue")); + expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + } + } + + class MetaStreamEventWrapper extends MetaStreamEvent { + private MetaStreamEvent metaStreamEvent; + private String inputReferenceId; + + public MetaStreamEventWrapper(MetaStreamEvent metaStreamEvent) { + this.metaStreamEvent = metaStreamEvent; + } + + @Override + public List getBeforeWindowData() { + return metaStreamEvent.getBeforeWindowData(); + } + + @Override + public List getOnAfterWindowData() { + return metaStreamEvent.getOnAfterWindowData(); + } + + @Override + public List getOutputData() { + return metaStreamEvent.getOutputData(); + } + + @Override + public void initializeOnAfterWindowData() { + metaStreamEvent.initializeOnAfterWindowData(); + } + + @Override + public void initializeAfterWindowData() { + metaStreamEvent.initializeAfterWindowData(); + } + + @Override + public int addData(Attribute attribute) { + return metaStreamEvent.addData(attribute); + } + + @Override + public void addOutputData(Attribute attribute) { + metaStreamEvent.addOutputData(attribute); + } + + @Override + public void addOutputDataAllowingDuplicate(Attribute attribute) { + metaStreamEvent.addOutputDataAllowingDuplicate(attribute); + } + + @Override + public List getInputDefinitions() { + return metaStreamEvent.getInputDefinitions(); + } + + @Override + public void addInputDefinition(AbstractDefinition inputDefinition) { + metaStreamEvent.addInputDefinition(inputDefinition); + } + + @Override + public String getInputReferenceId() { + return inputReferenceId; + } + + @Override + public void setInputReferenceId(String inputReferenceId) { + this.inputReferenceId = inputReferenceId; + } + + @Override + public void setOutputDefinition(StreamDefinition streamDefinition) { + metaStreamEvent.setOutputDefinition(streamDefinition); + } + + @Override + public StreamDefinition getOutputStreamDefinition() { + return metaStreamEvent.getOutputStreamDefinition(); + } + + @Override + public EventType getEventType() { + return metaStreamEvent.getEventType(); + } + + @Override + public void setEventType(EventType eventType) { + metaStreamEvent.setEventType(eventType); + } + + @Override + public AbstractDefinition getLastInputDefinition() { + return metaStreamEvent.getLastInputDefinition(); + } + + @Override + public boolean isMultiValue() { + return metaStreamEvent.isMultiValue(); + } + + @Override + public void setMultiValue(boolean multiValue) { + metaStreamEvent.setMultiValue(multiValue); + } + + @Override + public MetaStreamEvent clone() { + return metaStreamEvent.clone(); + } + } +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionWindowProcessor.java index 2befdf48a8..1a0d5ee941 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExpressionWindowProcessor.java @@ -67,7 +67,8 @@ @Extension( name = "expression", namespace = "", - description = "A sliding window that holds events that satisfy the given `expression`, when they aren't, " + + description = "A sliding window that dynamically shrink and grow based on the `expression`, " + + "it holds events that satisfies the given `expression`, when they aren't, " + "they are evaluated from the `first` (oldest) to the `last` (latest/current) and expired " + "from the oldest until the `expression` is satisfied.\n" + "**Note**: All the events in window are reevaluated only when the given `expression` is changed.", @@ -84,7 +85,7 @@ examples = { @Example( syntax = "@info(name = 'query1')\n" + - "from StockEventWindow#window.expression('count()>20')\n" + + "from StockEventWindow#window.expression('count()<=20')\n" + "select symbol, sum(price) as price\n" + "insert into OutputStream ;", description = "This will retain last 20 events in a sliding manner." @@ -152,7 +153,7 @@ protected void process(ComplexEventChunk streamEventChunk, Processo String expressionStringNew = (String) expressionStringExecutor.execute(streamEvent); if (!expressionStringNew.equals(expressionString)) { expressionString = expressionStringNew; - processAllExpiredEvents(streamEventChunk, streamEventCloner, state, currentTime); + processAllExpiredEvents(streamEventChunk, state, currentTime); } } StreamEvent clonedEvent = streamEventCloner.copyStreamEvent(streamEvent); @@ -180,14 +181,13 @@ private MetaStateEvent constructExpression(MetaStreamEvent metaStreamEvent, 0, new HashMap<>(), variableExpressionExecutors, false, 0, ProcessingMode.SLIDE, true, exprQueryContext); if (expressionExecutor.getReturnType() != Attribute.Type.BOOL) { - throw new SiddhiAppRuntimeException("Expression (" + expressionString + ") does not return Bool"); + throw new SiddhiAppRuntimeException("Expression ('" + expressionString + "') does not return Bool"); } return metaStateEvent; } private void processAllExpiredEvents(ComplexEventChunk streamEventChunk, - StreamEventCloner streamEventCloner, WindowState state, - long currentTime) { + WindowState state, long currentTime) { MetaStateEvent metaStateEvent = constructExpression(metaStreamEvent, siddhiQueryContext); QueryParserHelper.updateVariablePosition(metaStateEvent, variableExpressionExecutors); StreamEvent expiredEvent = state.expiredEventQueue.getFirst(); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/ExpressionBatchWindowTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/ExpressionBatchWindowTestCase.java new file mode 100644 index 0000000000..b68ec52d6b --- /dev/null +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/ExpressionBatchWindowTestCase.java @@ -0,0 +1,1145 @@ +/* + * 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.window; + +import io.siddhi.core.SiddhiAppRuntime; +import io.siddhi.core.SiddhiManager; +import io.siddhi.core.event.Event; +import io.siddhi.core.exception.SiddhiAppCreationException; +import io.siddhi.core.query.output.callback.QueryCallback; +import io.siddhi.core.stream.input.InputHandler; +import io.siddhi.core.util.EventPrinter; +import org.apache.log4j.Logger; +import org.testng.AssertJUnit; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class ExpressionBatchWindowTestCase { + private static final Logger log = Logger.getLogger(ExpressionBatchWindowTestCase.class); + private int inEventCount; + private int removeEventCount; + private int count; + private boolean eventArrived; + private Event storedEvent; + + + @BeforeMethod + public void init() { + count = 0; + inEventCount = 0; + removeEventCount = 0; + eventArrived = false; + storedEvent = null; + } + + @Test + public void expressionBatchWindowTest1() throws InterruptedException { + log.info("Testing expression batch window with no of events"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "" + + "define stream cseEventStream (symbol string, price float, volume int);"; + String query = "" + + "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch('count() <= 2') " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(2, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(2, removeEvents.length); + } + eventArrived = true; + } + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + inputHandler.send(new Object[]{"IBM", 700f, 0}); + inputHandler.send(new Object[]{"WSO2", 60.5f, 1}); + inputHandler.send(new Object[]{"WSO2", 61.5f, 2}); + inputHandler.send(new Object[]{"WSO2", 62.5f, 3}); + inputHandler.send(new Object[]{"WSO2", 63.5f, 4}); + AssertJUnit.assertEquals(4, inEventCount); + AssertJUnit.assertEquals(2, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + + @Test + public void expressionBatchWindowTest2() throws InterruptedException { + log.info("Testing expression batch window 2"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch('last.volume - first.volume < 2') " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(2, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(2, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + inputHandler.send(new Object[]{"WSO2", 60.5f, 0}); + inputHandler.send(new Object[]{"WSO2", 61.5f, 1}); + inputHandler.send(new Object[]{"WSO2", 62.5f, 2}); + inputHandler.send(new Object[]{"WSO2", 63.5f, 3}); + inputHandler.send(new Object[]{"WSO2", 64.5f, 4}); + AssertJUnit.assertEquals(4, inEventCount); + AssertJUnit.assertEquals(2, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + + @Test + public void expressionBatchWindowTest3() throws InterruptedException { + log.info("Testing expression batch window 3"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch('eventTimestamp(last) - eventTimestamp(first) < 2') " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(2, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(2, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2}); + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4}); + AssertJUnit.assertEquals(4, inEventCount); + AssertJUnit.assertEquals(2, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + + @Test + public void expressionBatchWindowTest4() throws InterruptedException { + log.info("Testing expression batch window 4"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch('eventTimestamp(last) - eventTimestamp(first) <= 2') " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(3, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(3, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + inputHandler.send(new Event[]{ + new Event(0, new Object[]{"WSO2", 60.5f, 0}), + new Event(1, new Object[]{"WSO2", 61.5f, 1}), + new Event(2, new Object[]{"WSO2", 62.5f, 2}), + new Event(3, new Object[]{"WSO2", 63.5f, 3}), + new Event(4, new Object[]{"WSO2", 64.5f, 4}), + new Event(5, new Object[]{"WSO2", 65.5f, 5}), + new Event(6, new Object[]{"WSO2", 66.5f, 6}), + }); + AssertJUnit.assertEquals(6, inEventCount); + AssertJUnit.assertEquals(3, removeEventCount); + AssertJUnit.assertEquals(2, count); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + + @Test + public void expressionBatchWindowTest5() throws InterruptedException { + log.info("Testing expression batch window 5"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + count++; + AssertJUnit.assertEquals(2, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(2, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr}); + AssertJUnit.assertEquals(4, inEventCount); + AssertJUnit.assertEquals(2, removeEventCount); + AssertJUnit.assertEquals(2, count); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest6() throws InterruptedException { + log.info("Testing expression batch window 6"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(4, inEvents.length); + count++; + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertEquals(4, removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 4"; + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr}); + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr}); + AssertJUnit.assertEquals(8, inEventCount); + AssertJUnit.assertEquals(4, removeEventCount); + AssertJUnit.assertEquals(2, count); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest7() throws InterruptedException { + log.info("Testing expression batch window 7"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + if (inEvents != null) { + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(1 == inEvents.length || 2 == inEvents.length); + } + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(3 == removeEvents.length || 2 == removeEvents.length + || 1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr}); + inputHandler.send(9, new Object[]{"WSO2", 69.5f, 9, expr}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(7, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest8() throws InterruptedException { + log.info("Testing expression batch window 8"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + AssertJUnit.assertEquals(3, inEventCount); + AssertJUnit.assertEquals(3, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + + @Test + public void expressionBatchWindowTest9() throws InterruptedException { + log.info("Testing expression batch window 9"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + count++; + AssertJUnit.assertEquals(1, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(new Event[]{ + new Event(0, new Object[]{"WSO2", 60.5f, 0, expr}), + new Event(1, new Object[]{"WSO2", 61.5f, 1, expr}), + new Event(2, new Object[]{"WSO2", 62.5f, 2, expr}) + }); + AssertJUnit.assertEquals(3, inEventCount); + AssertJUnit.assertEquals(3, removeEventCount); + AssertJUnit.assertEquals(3, count); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest10() throws InterruptedException { + log.info("Testing expression batch window 10"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(1 == inEvents.length || 2 == inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length || 2 == removeEvents.length + || 3 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr1 = "eventTimestamp(last) - eventTimestamp(first) < 2"; + String expr2 = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(new Event[]{ + new Event(0, new Object[]{"WSO2", 60.5f, 0, expr1}), + new Event(1, new Object[]{"WSO2", 61.5f, 1, expr1}), + new Event(2, new Object[]{"WSO2", 62.5f, 2, expr1}), + new Event(3, new Object[]{"WSO2", 63.5f, 3, expr2}), + new Event(4, new Object[]{"WSO2", 64.5f, 4, expr2}), + new Event(5, new Object[]{"WSO2", 65.5f, 5, expr1}), + new Event(6, new Object[]{"WSO2", 66.5f, 6, expr1}), + new Event(7, new Object[]{"WSO2", 67.5f, 7, expr1}), + new Event(8, new Object[]{"WSO2", 68.5f, 8, expr1}), + new Event(9, new Object[]{"WSO2", 69.5f, 9, expr1}) + }); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(7, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest11() throws InterruptedException { + log.info("Testing expression batch window 11"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + count++; + AssertJUnit.assertTrue(2 == inEvents.length || 4 == inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(2 == removeEvents.length || 4 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 4"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr}); + + AssertJUnit.assertEquals(8, inEventCount); + AssertJUnit.assertEquals(6, removeEventCount); + AssertJUnit.assertEquals(3, count); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest12() throws InterruptedException { + log.info("Testing expression batch window 12"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price double, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + count++; + AssertJUnit.assertEquals(4, inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(4 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "sum(price) < 50"; + inputHandler.send(0, new Object[]{"WSO2", 10.0, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 10.0, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 10.0, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 10.0, 4, expr}); + inputHandler.send(5, new Object[]{"WSO2", 10.0, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 10.0, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 10.0, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 10.0, 8, expr}); + + AssertJUnit.assertEquals(8, inEventCount); + AssertJUnit.assertEquals(4, removeEventCount); + AssertJUnit.assertEquals(2, count); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest13() throws InterruptedException { + log.info("Testing expression batch window 13"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price double, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + inEventCount = inEventCount + inEvents.length; + count++; + AssertJUnit.assertTrue(3 == inEvents.length || 4 == inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(3 == removeEvents.length || 4 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "sum(price) < 40"; + inputHandler.send(0, new Object[]{"WSO2", 10.0, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 10.0, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 10.0, 3, expr}); + expr = "sum(price) < 50"; + inputHandler.send(4, new Object[]{"WSO2", 10.0, 4, expr}); + inputHandler.send(5, new Object[]{"WSO2", 10.0, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 10.0, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 10.0, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 10.0, 8, expr}); + + AssertJUnit.assertEquals(7, inEventCount); + AssertJUnit.assertEquals(3, removeEventCount); + AssertJUnit.assertEquals(2, count); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + + @Test + public void expressionBatchWindowTest14() throws InterruptedException { + log.info("Testing expression batch window 14"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price double, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(1 == inEvents.length || 3 == inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length || 4 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "sum(price) < 40"; + inputHandler.send(0, new Object[]{"WSO2", 10.0, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 10.0, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 2, expr}); + inputHandler.send(3, new Object[]{"WSO2", 10.0, 3, expr}); + expr = "sum(price) < 0"; + inputHandler.send(4, new Object[]{"WSO2", 10.0, 4, expr}); + inputHandler.send(5, new Object[]{"WSO2", 10.0, 5, expr}); + expr = "sum(price) < 40"; + inputHandler.send(6, new Object[]{"WSO2", 10.0, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 10.0, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 10.0, 8, expr}); + inputHandler.send(9, new Object[]{"WSO2", 10.0, 9, expr}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(6, removeEventCount); + AssertJUnit.assertEquals(5, count); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest15() throws InterruptedException { + log.info("Testing expression batch window 15"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price double, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr) " + + "select symbol, volume " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(3 == inEvents.length || 2 == inEvents.length); + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(3 == removeEvents.length || 2 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "sum(price) < 30 and eventTimestamp(last) - eventTimestamp(first) < 3"; + inputHandler.send(0, new Object[]{"WSO2", 10.0, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 5.0, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 2, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 3, expr}); + inputHandler.send(2, new Object[]{"WSO2", 10.0, 4, expr}); + inputHandler.send(7, new Object[]{"WSO2", 10.0, 5, expr}); + inputHandler.send(8, new Object[]{"WSO2", 15.0, 6, expr}); + inputHandler.send(9, new Object[]{"WSO2", 10.0, 7, expr}); + inputHandler.send(10, new Object[]{"WSO2", 10.0, 8, expr}); + + AssertJUnit.assertEquals(7, inEventCount); + AssertJUnit.assertEquals(5, removeEventCount); + AssertJUnit.assertEquals(3, count); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest16() throws InterruptedException { + log.info("Testing expression batch window 16"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int, expr string);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr, true) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + if (inEvents != null) { + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(1 == inEvents.length || 2 == inEvents.length); + } + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(3 == removeEvents.length || 2 == removeEvents.length + || 1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + String expr = "eventTimestamp(last) - eventTimestamp(first) < 1"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 1"; + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr}); + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr}); + inputHandler.send(9, new Object[]{"WSO2", 69.5f, 9, expr}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(7, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest17() throws InterruptedException { + log.info("Testing expression batch window 17"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int," + + " expr string, include bool);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr, include) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + if (inEvents != null) { + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(1 == inEvents.length || 2 == inEvents.length); + } + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(3 == removeEvents.length || 2 == removeEvents.length + || 1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + Boolean include = true; + String expr = "eventTimestamp(last) - eventTimestamp(first) < 1"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr, include}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr, include}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr, include}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + include = false; + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr, include}); + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr, include}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr, include}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr, include}); + inputHandler.send(9, new Object[]{"WSO2", 69.5f, 9, expr, include}); + + AssertJUnit.assertEquals(9, inEventCount); + AssertJUnit.assertEquals(7, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest18() throws InterruptedException { + log.info("Testing expression batch window 18"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int," + + " expr string, include bool);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr, include, true) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + if (inEvents != null) { + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(1 == inEvents.length); + } + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(2 == removeEvents.length || 1 == removeEvents.length + || 1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + Boolean include = true; + String expr = "eventTimestamp(last) - eventTimestamp(first) < 1"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr, include}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr, include}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr, include}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + include = false; + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr, include}); + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr, include}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr, include}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr, include}); + inputHandler.send(9, new Object[]{"WSO2", 69.5f, 9, expr, include}); + + AssertJUnit.assertEquals(10, inEventCount); + AssertJUnit.assertEquals(9, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest19() throws InterruptedException { + log.info("Testing expression batch window 19"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int," + + " expr string, include bool);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr, include, true) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + if (inEvents != null) { + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(1 == inEvents.length); + } + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(2 == removeEvents.length || 1 == removeEvents.length + || 1 == removeEvents.length); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + Boolean include = true; + String expr = "eventTimestamp(last)"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr, include}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr, include}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr, include}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + include = false; + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr, include}); + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr, include}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr, include}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr, include}); + inputHandler.send(9, new Object[]{"WSO2", 69.5f, 9, expr, include}); + + AssertJUnit.assertEquals(7, inEventCount); + AssertJUnit.assertEquals(6, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test(expectedExceptions = SiddhiAppCreationException.class) + public void expressionBatchWindowTest20() throws InterruptedException { + log.info("Testing expression batch window 20"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int," + + " include bool);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch('eventTimestamp(last)', include, true) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + siddhiAppRuntime.start(); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest21() throws InterruptedException { + log.info("Testing expression batch window 21"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int," + + " expr string, include bool);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr, include, true) " + + "select symbol, price, count() as totalCount " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + if (inEvents != null) { + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(1 == inEvents.length); + for (Event event : inEvents) { + AssertJUnit.assertTrue(1 == (Long) event.getData(2) + || 2 == (Long) event.getData(2) + || 0 == (Long) event.getData(2)); + } + } + if (removeEvents != null) { + removeEventCount = removeEventCount + removeEvents.length; + AssertJUnit.assertTrue(1 == removeEvents.length); + for (Event event : removeEvents) { + AssertJUnit.assertTrue(0 == (Long) event.getData(2)); + } + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + Boolean include = true; + String expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr, include}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr, include}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr, include}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr, include}); + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr, include}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr, include}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr, include}); + inputHandler.send(9, new Object[]{"WSO2", 69.5f, 9, expr, include}); + + AssertJUnit.assertEquals(8, inEventCount); + AssertJUnit.assertEquals(2, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } + + @Test + public void expressionBatchWindowTest22() throws InterruptedException { + log.info("Testing expression batch window 22"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String cseEventStream = "define stream cseEventStream (symbol string, price float, volume int," + + " expr string, include bool);"; + String query = "@info(name = 'query1') " + + "from cseEventStream#window.expressionBatch(expr, include, true) " + + "select symbol, price, count() as totalCount " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + count++; + if (inEvents != null) { + inEventCount = inEventCount + inEvents.length; + AssertJUnit.assertTrue(1 == inEvents.length); + for (Event event : inEvents) { + AssertJUnit.assertTrue(1 == (Long) event.getData(2) + || 2 == (Long) event.getData(2) + || 0 == (Long) event.getData(2)); + } + } + if (removeEvents != null) { + AssertJUnit.fail("remove events are not expected"); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + Boolean include = false; + String expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(0, new Object[]{"WSO2", 60.5f, 0, expr, include}); + inputHandler.send(1, new Object[]{"WSO2", 61.5f, 1, expr, include}); + inputHandler.send(2, new Object[]{"WSO2", 62.5f, 2, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 0"; + inputHandler.send(3, new Object[]{"WSO2", 63.5f, 3, expr, include}); + inputHandler.send(4, new Object[]{"WSO2", 64.5f, 4, expr, include}); + inputHandler.send(5, new Object[]{"WSO2", 65.5f, 5, expr, include}); + expr = "eventTimestamp(last) - eventTimestamp(first) < 2"; + inputHandler.send(6, new Object[]{"WSO2", 66.5f, 6, expr, include}); + inputHandler.send(7, new Object[]{"WSO2", 67.5f, 7, expr, include}); + inputHandler.send(8, new Object[]{"WSO2", 68.5f, 8, expr, include}); + inputHandler.send(9, new Object[]{"WSO2", 69.5f, 9, expr, include}); + + AssertJUnit.assertEquals(10, inEventCount); + AssertJUnit.assertEquals(0, removeEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + } +} diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/LengthBatchWindowTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/LengthBatchWindowTestCase.java index a342686fe5..70a705abd4 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/LengthBatchWindowTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/LengthBatchWindowTestCase.java @@ -1042,4 +1042,124 @@ public void receive(Event[] events) { siddhiAppRuntime.shutdown(); } + @Test + public void lengthBatchWindowTest21() throws InterruptedException { + log.info("LengthBatchWindow Test21"); + + final int length = 3; + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "define stream cseEventStream (symbol string, price float, volume int);" + + "" + + "@info(name = 'query1') " + + "from cseEventStream#window.lengthBatch(" + length + ", true) " + + "select symbol, price, count() as volumes " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + siddhiAppRuntime.addCallback("outputStream", new StreamCallback() { + + @Override + public void receive(Event[] events) { + + EventPrinter.print(events); + eventArrived = true; + if (events.length == 1) { + inEventCount++; + } else { + AssertJUnit.assertFalse("Event batch with unexpected number of events " + events.length, + false); + } + for (Event event : events) { + AssertJUnit.assertTrue("Count values", ((Long) event.getData(2) == 1 + || (Long) event.getData(2) == 2 || ((Long) event.getData(2)) == 3)); + count++; + } + } + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + inputHandler.send(new Object[]{"IBM", 700f, 1}); + inputHandler.send(new Object[]{"WSO2", 60.5f, 2}); + inputHandler.send(new Object[]{"IBM", 700f, 3}); + inputHandler.send(new Object[]{"WSO2", 60.5f, 4}); + inputHandler.send(new Object[]{"IBM", 700f, 5}); + inputHandler.send(new Object[]{"WSO2", 60.5f, 6}); + inputHandler.send(new Object[]{"WSO2", 60.5f, 4}); + inputHandler.send(new Object[]{"IBM", 700f, 5}); + inputHandler.send(new Object[]{"WSO2", 60.5f, 6}); + AssertJUnit.assertEquals("Total events", 9, count); + AssertJUnit.assertEquals("1 event batch", 9, inEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + + @Test + public void lengthBatchWindowTest22() throws InterruptedException { + log.info("LengthBatchWindow Test22"); + + final int length = 3; + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "define stream cseEventStream (symbol string, price float, volume int);" + + "" + + "@info(name = 'query1') " + + "from cseEventStream#window.lengthBatch(" + length + ", true) " + + "select symbol, price " + + "insert all events into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + } + +// @Override +// public void receive(Event[] events) { +// +// EventPrinter.print(events); +// eventArrived = true; +// if (events.length == 1) { +// inEventCount++; +// } else { +// AssertJUnit.assertFalse("Event batch with unexpected number of events " + events.length, +// false); +// } +// for (Event event : events) { +// AssertJUnit.assertTrue("Count values", ((Long) event.getData(2) == 1 +// || (Long) event.getData(2) == 2 || ((Long) event.getData(2)) == 3)); +// count++; +// } +// } + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + + inputHandler.send(new Event[]{ + new Event(2, new Object[]{"IBM", 700f, 1}), + new Event(2, new Object[]{"WSO2", 60.5f, 2}), + new Event(2, new Object[]{"IBM", 700f, 3}), + new Event(2, new Object[]{"WSO2", 60.5f, 4}), + new Event(2, new Object[]{"IBM", 700f, 5}), + new Event(2, new Object[]{"WSO2", 60.5f, 6}), + new Event(2, new Object[]{"WSO2", 60.5f, 4}), + new Event(2, new Object[]{"IBM", 700f, 5}), + new Event(2, new Object[]{"WSO2", 60.5f, 6}) + }); + AssertJUnit.assertEquals("Total events", 9, count); + AssertJUnit.assertEquals("1 event batch", 9, inEventCount); + AssertJUnit.assertTrue(eventArrived); + siddhiAppRuntime.shutdown(); + + } + } diff --git a/modules/siddhi-core/src/test/resources/testng.xml b/modules/siddhi-core/src/test/resources/testng.xml index 091b9db868..d4dfa7429f 100644 --- a/modules/siddhi-core/src/test/resources/testng.xml +++ b/modules/siddhi-core/src/test/resources/testng.xml @@ -111,6 +111,7 @@ + From 8e029c6fa73d385fd76221031d5d9e73b8e7cdf0 Mon Sep 17 00:00:00 2001 From: Suhothayan Date: Tue, 24 Dec 2019 15:27:51 +0530 Subject: [PATCH 5/6] Add Batch expression Window --- findbugs-exclude.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/findbugs-exclude.xml b/findbugs-exclude.xml index a8db25da03..967ce620d8 100644 --- a/findbugs-exclude.xml +++ b/findbugs-exclude.xml @@ -39,6 +39,14 @@ + + + + + + + + From dd4f1a201e76d4a452f9c6ac199b275cb31a3ff4 Mon Sep 17 00:00:00 2001 From: Suhothayan Date: Tue, 24 Dec 2019 18:13:10 +0530 Subject: [PATCH 6/6] Revert the accidental change of test case --- .../window/LengthBatchWindowTestCase.java | 39 ++++++++----------- 1 file changed, 17 insertions(+), 22 deletions(-) diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/LengthBatchWindowTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/LengthBatchWindowTestCase.java index 70a705abd4..1d9f97fce4 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/LengthBatchWindowTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/LengthBatchWindowTestCase.java @@ -1110,35 +1110,30 @@ public void lengthBatchWindowTest22() throws InterruptedException { "" + "@info(name = 'query1') " + "from cseEventStream#window.lengthBatch(" + length + ", true) " + - "select symbol, price " + + "select symbol, price, count() as total " + "insert all events into outputStream ;"; SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); - siddhiAppRuntime.addCallback("query1", new QueryCallback() { + siddhiAppRuntime.addCallback("outputStream", new StreamCallback() { @Override - public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { - EventPrinter.print(timestamp, inEvents, removeEvents); - } + public void receive(Event[] events) { -// @Override -// public void receive(Event[] events) { -// -// EventPrinter.print(events); -// eventArrived = true; -// if (events.length == 1) { -// inEventCount++; -// } else { -// AssertJUnit.assertFalse("Event batch with unexpected number of events " + events.length, -// false); -// } -// for (Event event : events) { -// AssertJUnit.assertTrue("Count values", ((Long) event.getData(2) == 1 -// || (Long) event.getData(2) == 2 || ((Long) event.getData(2)) == 3)); -// count++; -// } -// } + EventPrinter.print(events); + eventArrived = true; + if (events.length == 1) { + inEventCount++; + } else { + AssertJUnit.assertFalse("Event batch with unexpected number of events " + events.length, + false); + } + for (Event event : events) { + AssertJUnit.assertTrue("Count values", ((Long) event.getData(2) == 1 + || (Long) event.getData(2) == 2 || ((Long) event.getData(2)) == 3)); + count++; + } + } }); InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream");