diff --git a/checkstyle-suppressions.xml b/checkstyle-suppressions.xml index f76bb72a1e..d8f2437f1c 100644 --- a/checkstyle-suppressions.xml +++ b/checkstyle-suppressions.xml @@ -24,8 +24,8 @@ - - + + diff --git a/docs/api/latest.md b/docs/api/latest.md index f10a5db218..65edb147dc 100644 --- a/docs/api/latest.md +++ b/docs/api/latest.md @@ -1,4 +1,4 @@ -# API Docs - v5.0.0-m4 +# API Docs - v5.0.0-SNAPSHOT ## Core diff --git a/docs/documentation/siddhi-4.0.md b/docs/documentation/siddhi-4.0.md index 364dbfa78d..386dc84c5a 100644 --- a/docs/documentation/siddhi-4.0.md +++ b/docs/documentation/siddhi-4.0.md @@ -2706,7 +2706,7 @@ Siddhi supports following extension types: For each event, it consumes zero or more parameters as input parameters and returns a single attribute with aggregated results. This can be used in conjunction with a window in order to find the aggregated results based on the given window like any Aggregate Function operation. - This is implemented by extending `io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator`. + This is implemented by extending `io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregatorExecutor`. Example : diff --git a/findbugs-exclude.xml b/findbugs-exclude.xml index 9f1618f21a..b6a2a448e8 100644 --- a/findbugs-exclude.xml +++ b/findbugs-exclude.xml @@ -172,6 +172,26 @@ + + + + + + + + + + + + + + + + + + + + diff --git a/modules/siddhi-annotations/src/main/java/io/siddhi/annotation/processor/SiddhiAnnotationProcessor.java b/modules/siddhi-annotations/src/main/java/io/siddhi/annotation/processor/SiddhiAnnotationProcessor.java index dbfde98a50..f40a15432b 100644 --- a/modules/siddhi-annotations/src/main/java/io/siddhi/annotation/processor/SiddhiAnnotationProcessor.java +++ b/modules/siddhi-annotations/src/main/java/io/siddhi/annotation/processor/SiddhiAnnotationProcessor.java @@ -79,7 +79,7 @@ public boolean process(Set annotations, RoundEnvironment AnnotationConstants.SINK_MAPPER_SUPER_CLASS, AnnotationConstants.SINK_SUPER_CLASS, AnnotationConstants.FUNCTION_EXECUTOR_SUPER_CLASS, - AnnotationConstants.AGGREGATION_ATTRIBUTE_SUPER_CLASS, + AnnotationConstants.AGGREGATION_ATTRIBUTE_EXECUTOR_SUPER_CLASS, AnnotationConstants.DISTRIBUTION_STRATEGY_SUPER_CLASS, AnnotationConstants.STREAM_PROCESSOR_SUPER_CLASS, AnnotationConstants.STREAM_FUNCTION_PROCESSOR_SUPER_CLASS, @@ -117,7 +117,7 @@ public boolean process(Set annotations, RoundEnvironment abstractAnnotationProcessor = new FunctionExecutorValidationAnnotationProcessor(extensionClassFullName); break; - case AnnotationConstants.AGGREGATION_ATTRIBUTE_SUPER_CLASS: + case AnnotationConstants.AGGREGATION_ATTRIBUTE_EXECUTOR_SUPER_CLASS: abstractAnnotationProcessor = new AggregationAttributeValidationAnnotationProcessor(extensionClassFullName); break; diff --git a/modules/siddhi-annotations/src/main/java/io/siddhi/annotation/util/AnnotationConstants.java b/modules/siddhi-annotations/src/main/java/io/siddhi/annotation/util/AnnotationConstants.java index 6b3e486bb0..cfb6a72604 100644 --- a/modules/siddhi-annotations/src/main/java/io/siddhi/annotation/util/AnnotationConstants.java +++ b/modules/siddhi-annotations/src/main/java/io/siddhi/annotation/util/AnnotationConstants.java @@ -28,8 +28,8 @@ public class AnnotationConstants { "io.siddhi.core.function.Script"; public static final String FUNCTION_EXECUTOR_SUPER_CLASS = "io.siddhi.core.executor.function.FunctionExecutor"; - public static final String AGGREGATION_ATTRIBUTE_SUPER_CLASS = - "io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator"; + public static final String AGGREGATION_ATTRIBUTE_EXECUTOR_SUPER_CLASS = + "io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregatorExecutor"; public static final String DISTRIBUTION_STRATEGY_SUPER_CLASS = "io.siddhi.core.stream.output.sink.distributed.DistributionStrategy"; public static final String STREAM_PROCESSOR_SUPER_CLASS = diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/SiddhiAppRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/SiddhiAppRuntime.java index 87ead5f81b..776f7bf9c7 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/SiddhiAppRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/SiddhiAppRuntime.java @@ -52,7 +52,7 @@ import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.SiddhiConstants; import io.siddhi.core.util.StringUtil; -import io.siddhi.core.util.extension.holder.EternalReferencedHolder; +import io.siddhi.core.util.extension.holder.ExternalReferencedHolder; import io.siddhi.core.util.parser.StoreQueryParser; import io.siddhi.core.util.parser.helper.QueryParserHelper; import io.siddhi.core.util.persistence.util.PersistenceHelper; @@ -243,7 +243,7 @@ public Set getQueryNames() { public Map> getPartitionedInnerStreamDefinitionMap() { Map> innerStreams = new HashMap<>(); for (PartitionRuntime partition : partitionMap.values()) { - innerStreams.put(partition.getElementId(), partition.getLocalStreamDefinitionMap()); + innerStreams.put(partition.getPartitionName(), partition.getLocalStreamDefinitionMap()); } return innerStreams; } @@ -390,8 +390,9 @@ public synchronized void startWithoutSources() { if (siddhiAppContext.isStatsEnabled() && siddhiAppContext.getStatisticsManager() != null) { siddhiAppContext.getStatisticsManager().startReporting(); } - for (EternalReferencedHolder eternalReferencedHolder : siddhiAppContext.getEternalReferencedHolders()) { - eternalReferencedHolder.start(); + for (ExternalReferencedHolder externalReferencedHolder : + siddhiAppContext.getExternalReferencedHolders()) { + externalReferencedHolder.start(); } for (List sinks : sinkMap.values()) { for (Sink sink : sinks) { @@ -427,7 +428,7 @@ public void setPurgingEnabled(boolean purgingEnabled) { this.incrementalDataPurging = purgingEnabled; } - public synchronized void startSources() { + public void startSources() { if (running) { log.warn("Error calling startSources() for Siddhi App '" + siddhiAppContext.getName() + "', " + "SiddhiApp already started with the sources."); @@ -465,7 +466,7 @@ public synchronized void shutdown() { try { if (sourceHandlerManager != null) { sourceHandlerManager.unregisterSourceHandler(source.getMapper().getHandler(). - getElementId()); + getId()); } source.shutdown(); } catch (Throwable t) { @@ -494,7 +495,7 @@ public synchronized void shutdown() { for (Sink sink : sinks) { try { if (sinkHandlerManager != null) { - sinkHandlerManager.unregisterSinkHandler(sink.getHandler().getElementId()); + sinkHandlerManager.unregisterSinkHandler(sink.getHandler().getId()); } sink.shutdown(); } catch (Throwable t) { @@ -514,7 +515,7 @@ public synchronized void shutdown() { String elementId = null; RecordTableHandler recordTableHandler = table.getHandler(); if (recordTableHandler != null) { - elementId = recordTableHandler.getElementId(); + elementId = recordTableHandler.getId(); } if (elementId != null) { recordTableHandlerManager.unregisterRecordTableHandler(elementId); @@ -523,13 +524,13 @@ public synchronized void shutdown() { table.shutdown(); } - for (EternalReferencedHolder eternalReferencedHolder : siddhiAppContext.getEternalReferencedHolders()) { + for (ExternalReferencedHolder externalReferencedHolder : siddhiAppContext.getExternalReferencedHolders()) { try { - eternalReferencedHolder.stop(); + externalReferencedHolder.stop(); } catch (Throwable t) { log.error(StringUtil.removeCRLFCharacters(ExceptionUtil.getMessageWithContext(t, siddhiAppContext)) + - " Error while stopping EternalReferencedHolder '" + - StringUtil.removeCRLFCharacters(eternalReferencedHolder.toString()) + "' down Siddhi app '" + + " Error while stopping ExternalReferencedHolder '" + + StringUtil.removeCRLFCharacters(externalReferencedHolder.toString()) + "' down Siddhi app '" + StringUtil.removeCRLFCharacters(siddhiAppContext.getName()) + "'.", t); } } 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 d1fff03c33..717e385029 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 @@ -18,7 +18,6 @@ package io.siddhi.core.aggregation; -import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.MetaStateEvent; @@ -68,61 +67,61 @@ public class AggregationRuntime implements MemoryCalculable { private final AggregationDefinition aggregationDefinition; private final Map incrementalExecutorMap; private final Map incrementalExecutorMapForPartitions; + private final List baseExecutorsForFind; + private ExpressionExecutor shouldUpdateTimestamp; private final Map aggregationTables; - private final SiddhiAppContext siddhiAppContext; private final MetaStreamEvent tableMetaStreamEvent; private final MetaStreamEvent aggregateMetaSteamEvent; private final LatencyTracker latencyTrackerFind; private final ThroughputTracker throughputTrackerFind; - private final List> aggregateProcessingExecutorsList; private final List groupByKeyGeneratorList; private List incrementalDurations; private SingleStreamRuntime singleStreamRuntime; - private List baseExecutors; private List outputExpressionExecutors; private RecreateInMemoryData recreateInMemoryData; private boolean processingOnExternalTime; private boolean isFirstEventArrived; private long lastExecutorsRefreshedTime = -1; private IncrementalDataPurging incrementalDataPurging; - private ExpressionExecutor shouldUpdateExpressionExecutor; private String shardId; + private List> aggregateProcessExpressionExecutorsListForFind; public AggregationRuntime(AggregationDefinition aggregationDefinition, Map incrementalExecutorMap, Map aggregationTables, SingleStreamRuntime singleStreamRuntime, List incrementalDurations, - SiddhiAppContext siddhiAppContext, List baseExecutors, MetaStreamEvent tableMetaStreamEvent, List outputExpressionExecutors, LatencyTracker latencyTrackerFind, ThroughputTracker throughputTrackerFind, RecreateInMemoryData recreateInMemoryData, boolean processingOnExternalTime, - List> aggregateProcessingExecutorsList, List groupByKeyGeneratorList, IncrementalDataPurging incrementalDataPurging, - ExpressionExecutor shouldUpdateExpressionExecutor, String shardId, - Map incrementalExecutorMapForPartitions) { + String shardId, + Map incrementalExecutorMapForPartitions, + ExpressionExecutor shouldUpdateTimestamp, + List> aggregateProcessExpressionExecutorsListForFind) { this.aggregationDefinition = aggregationDefinition; this.incrementalExecutorMap = incrementalExecutorMap; this.aggregationTables = aggregationTables; this.incrementalDurations = incrementalDurations; - this.siddhiAppContext = siddhiAppContext; this.singleStreamRuntime = singleStreamRuntime; - this.baseExecutors = baseExecutors; this.tableMetaStreamEvent = tableMetaStreamEvent; this.outputExpressionExecutors = outputExpressionExecutors; this.latencyTrackerFind = latencyTrackerFind; this.throughputTrackerFind = throughputTrackerFind; this.recreateInMemoryData = recreateInMemoryData; this.processingOnExternalTime = processingOnExternalTime; - this.aggregateProcessingExecutorsList = aggregateProcessingExecutorsList; this.groupByKeyGeneratorList = groupByKeyGeneratorList; this.incrementalDataPurging = incrementalDataPurging; - this.shouldUpdateExpressionExecutor = shouldUpdateExpressionExecutor; this.shardId = shardId; this.incrementalExecutorMapForPartitions = incrementalExecutorMapForPartitions; - aggregateMetaSteamEvent = new MetaStreamEvent(); + this.shouldUpdateTimestamp = shouldUpdateTimestamp; + this.aggregateProcessExpressionExecutorsListForFind = aggregateProcessExpressionExecutorsListForFind; + this.aggregateMetaSteamEvent = new MetaStreamEvent(); + //Without timestamp executor + this.baseExecutorsForFind = aggregateProcessExpressionExecutorsListForFind.get(0).subList(1, + aggregateProcessExpressionExecutorsListForFind.get(0).size()); aggregationDefinition.getAttributeList().forEach(aggregateMetaSteamEvent::addOutputData); } @@ -196,11 +195,11 @@ public SingleStreamRuntime getSingleStreamRuntime() { return singleStreamRuntime; } - public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + SiddhiQueryContext siddhiQueryContext) { try { - SnapshotService.getSkipSnapshotableThreadLocal().set(true); - if (latencyTrackerFind != null && siddhiAppContext.isStatsEnabled()) { + SnapshotService.getSkipStateStorageThreadLocal().set(true); + if (latencyTrackerFind != null && siddhiQueryContext.getSiddhiAppContext().isStatsEnabled()) { latencyTrackerFind.markIn(); throughputTrackerFind.eventIn(); } @@ -215,12 +214,13 @@ public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCond } return ((IncrementalAggregateCompileCondition) compiledCondition).find(matchingEvent, aggregationDefinition, incrementalExecutorMap, aggregationTables, incrementalDurations, - baseExecutors, outputExpressionExecutors, siddhiAppContext, - aggregateProcessingExecutorsList, groupByKeyGeneratorList, shouldUpdateExpressionExecutor, + baseExecutorsForFind, outputExpressionExecutors, siddhiQueryContext, + aggregateProcessExpressionExecutorsListForFind, + groupByKeyGeneratorList, shouldUpdateTimestamp, incrementalExecutorMapForPartitions); } finally { - SnapshotService.getSkipSnapshotableThreadLocal().set(null); - if (latencyTrackerFind != null && siddhiAppContext.isStatsEnabled()) { + SnapshotService.getSkipStateStorageThreadLocal().set(null); + if (latencyTrackerFind != null && siddhiQueryContext.getSiddhiAppContext().isStatsEnabled()) { latencyTrackerFind.markOut(); } } @@ -348,8 +348,7 @@ public CompiledCondition compileExpression(Expression expression, Within within, // "on stream1.name == aggregator.nickName ..." in the join query) must be executed on that data. // This condition is used for that purpose. onCompiledCondition = OperatorParser.constructOperator(new ComplexEventChunk<>(true), expression, - matchingMetaInfoHolder, variableExpressionExecutors, tableMap, - siddhiQueryContext); + matchingMetaInfoHolder, variableExpressionExecutors, tableMap, siddhiQueryContext); return new IncrementalAggregateCompileCondition(withinTableCompiledConditions, withinInMemoryCompileCondition, onCompiledCondition, tableMetaStreamEvent, aggregateMetaSteamEvent, additionalAttributes, diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/BaseIncrementalValueStore.java b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/BaseIncrementalValueStore.java index 832888e35b..5d8a960ca1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/BaseIncrementalValueStore.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/BaseIncrementalValueStore.java @@ -19,12 +19,16 @@ package io.siddhi.core.aggregation; import io.siddhi.core.config.SiddhiAppContext; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.executor.ExpressionExecutor; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.executor.VariableExpressionExecutor; +import io.siddhi.core.util.snapshot.state.PartitionSyncStateHolder; +import io.siddhi.core.util.snapshot.state.SingleSyncStateHolder; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateHolder; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -33,50 +37,48 @@ * Store for maintaining the base values related to incremental aggregation. (e.g. for average, * the base incremental values would be sum and count. The timestamp too is stored here. */ -public class BaseIncrementalValueStore implements Snapshotable { - private long timestamp; // This is the starting timeStamp of aggregates - private Object[] values; +public class BaseIncrementalValueStore { + private StateHolder valueStateHolder; + private StateHolder storeStateHolder; private List expressionExecutors; - private boolean isProcessed = false; - private StreamEventPool streamEventPool; - private String elementId; - private SiddhiAppContext siddhiAppContext; - private String aggregatorName; - private ExpressionExecutor shouldUpdateExpressionExecutor; - - public BaseIncrementalValueStore(long timeStamp, List expressionExecutors, - StreamEventPool streamEventPool, - SiddhiAppContext siddhiAppContext, String aggregatorName, - ExpressionExecutor shouldUpdateExpressionExecutor) { - this.timestamp = timeStamp; - this.values = new Object[expressionExecutors.size() + 1]; + private StreamEventFactory streamEventFactory; + private ExpressionExecutor shouldUpdateTimestamp; + private long initialTimestamp; + + public BaseIncrementalValueStore(List expressionExecutors, + StreamEventFactory streamEventFactory, + SiddhiQueryContext siddhiQueryContext, String aggregatorName, + ExpressionExecutor shouldUpdateTimestamp, long initialTimestamp, + boolean groupBy, boolean local) { this.expressionExecutors = expressionExecutors; - this.streamEventPool = streamEventPool; - this.siddhiAppContext = siddhiAppContext; - this.aggregatorName = aggregatorName; - this.shouldUpdateExpressionExecutor = shouldUpdateExpressionExecutor; - if (elementId == null) { - elementId = "IncrementalBaseStore-" + siddhiAppContext.getElementIdGenerator().createNewId(); - } - if (aggregatorName != null) { - siddhiAppContext.getSnapshotService().addSnapshotable(aggregatorName, this); + this.streamEventFactory = streamEventFactory; + this.shouldUpdateTimestamp = shouldUpdateTimestamp; + this.initialTimestamp = initialTimestamp; + if (!local) { + this.valueStateHolder = siddhiQueryContext.generateStateHolder(aggregatorName + "-" + + this.getClass().getName() + "-value", groupBy, () -> new ValueState()); + this.storeStateHolder = siddhiQueryContext.generateStateHolder(aggregatorName + "-" + + this.getClass().getName(), false, () -> new StoreState()); + } else { + this.valueStateHolder = new PartitionSyncStateHolder(() -> new ValueState()); + this.storeStateHolder = new SingleSyncStateHolder(() -> new StoreState()); } } - public void clearValues() { - this.values = new Object[expressionExecutors.size() + 1]; + public void clearValues(long startTimeOfNewAggregates, StreamEvent resetEvent) { + this.initialTimestamp = startTimeOfNewAggregates; + setTimestamp(startTimeOfNewAggregates); + setProcessed(false); + this.valueStateHolder.cleanGroupByStates(); } public void setValue(Object value, int position) { - values[position] = value; - } - - public long getTimestamp() { - return timestamp; - } - - public void setTimestamp(long timestamp) { - this.timestamp = timestamp; + ValueState state = this.valueStateHolder.getState(); + try { + state.values[position] = value; + } finally { + this.valueStateHolder.returnState(state); + } } public List getExpressionExecutors() { @@ -84,66 +86,208 @@ public List getExpressionExecutors() { } public boolean isProcessed() { - return isProcessed; + StoreState state = this.storeStateHolder.getState(); + try { + return state.isProcessed; + } finally { + this.storeStateHolder.returnState(state); + } } public void setProcessed(boolean isProcessed) { - this.isProcessed = isProcessed; + StoreState state = this.storeStateHolder.getState(); + try { + state.isProcessed = isProcessed; + } finally { + this.storeStateHolder.returnState(state); + } } - public StreamEvent createStreamEvent() { - StreamEvent streamEvent = streamEventPool.borrowEvent(); - streamEvent.setTimestamp(timestamp); - setValue(timestamp, 0); - streamEvent.setOutputData(values); - return streamEvent; + private long getTimestamp() { + StoreState state = this.storeStateHolder.getState(); + try { + return state.timestamp; + } finally { + this.storeStateHolder.returnState(state); + } } - public BaseIncrementalValueStore cloneStore(String key, long timestamp) { - List newExpressionExecutors = new ArrayList<>(expressionExecutors.size()); - expressionExecutors - .forEach(expressionExecutor -> newExpressionExecutors.add(expressionExecutor.cloneExecutor(key))); - ExpressionExecutor newShouldUpdateExpressionExecutor = - (shouldUpdateExpressionExecutor == null) ? null : shouldUpdateExpressionExecutor.cloneExecutor(key); - return new BaseIncrementalValueStore(timestamp, newExpressionExecutors, streamEventPool, siddhiAppContext, - aggregatorName, newShouldUpdateExpressionExecutor); + private void setTimestamp(long timestamp) { + StoreState state = this.storeStateHolder.getState(); + try { + state.timestamp = timestamp; + } finally { + this.storeStateHolder.returnState(state); + } } - public ExpressionExecutor getShouldUpdateExpressionExecutor() { - return shouldUpdateExpressionExecutor; + public Map getGroupedByEvents() { + Map groupedByEvents = new HashMap<>(); + + if (isProcessed()) { + Map baseIncrementalValueStoreMap = this.valueStateHolder.getAllGroupByStates(); + try { + for (Map.Entry state : baseIncrementalValueStoreMap.entrySet()) { + StreamEvent streamEvent = streamEventFactory.newInstance(); + long timestamp = getTimestamp(); + streamEvent.setTimestamp(timestamp); + state.getValue().setValue(timestamp, 0); + streamEvent.setOutputData(state.getValue().values); + groupedByEvents.put(state.getKey(), streamEvent); + } + } finally { + this.valueStateHolder.returnGroupByStates(baseIncrementalValueStoreMap); + } + } + return groupedByEvents; } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("Timestamp", timestamp); - state.put("Values", values); - state.put("IsProcessed", isProcessed); - return state; + public void process(StreamEvent streamEvent) { + ValueState state = valueStateHolder.getState(); + try { + boolean shouldUpdate = true; + if (shouldUpdateTimestamp != null) { + shouldUpdate = (boolean) shouldUpdate( + shouldUpdateTimestamp.execute(streamEvent), state); + } + for (int i = 0; i < expressionExecutors.size(); i++) { // keeping timestamp value location as null + if (shouldUpdate) { + ExpressionExecutor expressionExecutor = expressionExecutors.get(i); + state.setValue(expressionExecutor.execute(streamEvent), i + 1); + } else { + ExpressionExecutor expressionExecutor = expressionExecutors.get(i); + if (!(expressionExecutor instanceof VariableExpressionExecutor)) { + state.setValue(expressionExecutor.execute(streamEvent), i + 1); + } + } + } + setProcessed(true); + } finally { + valueStateHolder.returnState(state); + } } - @Override - public void restoreState(Map state) { - timestamp = (long) state.get("Timestamp"); - values = (Object[]) state.get("Values"); - isProcessed = (boolean) state.get("IsProcessed"); + public void process(Map groupedByEvents) { + for (Map.Entry eventEntry : groupedByEvents.entrySet()) { + synchronized (this) { + SiddhiAppContext.startGroupByFlow(eventEntry.getKey() + "-" + + eventEntry.getValue().getTimestamp()); + ValueState state = valueStateHolder.getState(); + try { + boolean shouldUpdate = true; + if (shouldUpdateTimestamp != null) { + shouldUpdate = (boolean) shouldUpdate( + shouldUpdateTimestamp.execute(eventEntry.getValue()), state); + } + for (int i = 0; i < expressionExecutors.size(); i++) { // keeping timestamp value location as null + if (shouldUpdate) { + ExpressionExecutor expressionExecutor = expressionExecutors.get(i); + state.setValue(expressionExecutor.execute(eventEntry.getValue()), i + 1); + } else { + ExpressionExecutor expressionExecutor = expressionExecutors.get(i); + if (!(expressionExecutor instanceof VariableExpressionExecutor)) { + state.setValue(expressionExecutor.execute(eventEntry.getValue()), i + 1); + } + } + } + setProcessed(true); + } finally { + valueStateHolder.returnState(state); + SiddhiAppContext.stopGroupByFlow(); + } + } + } } - @Override - public String getElementId() { - return elementId; + private Object shouldUpdate(Object data, ValueState state) { + long timestamp = (long) data; + if (timestamp >= state.lastTimestamp) { + state.lastTimestamp = timestamp; + return true; + } + return false; } - @Override - public void clean() { - for (ExpressionExecutor expressionExecutor : expressionExecutors) { - expressionExecutor.clean(); + public Object shouldUpdate(Object data) { + ValueState state = this.valueStateHolder.getState(); + try { + long timestamp = (long) data; + if (timestamp >= state.lastTimestamp) { + state.lastTimestamp = timestamp; + return true; + } + return false; + } finally { + this.valueStateHolder.returnState(state); } - if (shouldUpdateExpressionExecutor != null) { - shouldUpdateExpressionExecutor.clean(); + + } + + class StoreState extends State { + private long timestamp; + private boolean isProcessed = false; + + public StoreState() { + this.timestamp = initialTimestamp; } - if (aggregatorName != null) { - siddhiAppContext.getSnapshotService().removeSnapshotable(aggregatorName, this); + + @Override + public boolean canDestroy() { + return !isProcessed; } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Timestamp", timestamp); + state.put("IsProcessed", isProcessed); + return state; + } + + @Override + public void restore(Map state) { + timestamp = (long) state.get("Timestamp"); + isProcessed = (boolean) state.get("IsProcessed"); + } + + public void setIfAbsentTimestamp(long timestamp) { + if (this.timestamp == -1) { + this.timestamp = timestamp; + } + } + } + + class ValueState extends State { + private Object[] values; + public long lastTimestamp = 0; + + public ValueState() { + this.values = new Object[expressionExecutors.size() + 1]; + } + + @Override + public boolean canDestroy() { + return values == null && lastTimestamp == 0; + } + + public void setValue(Object value, int position) { + values[position] = value; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Values", values); + state.put("LastTimestamp", lastTimestamp); + return state; + } + + @Override + public void restore(Map state) { + values = (Object[]) state.get("Values"); + lastTimestamp = (Long) state.get("LastTimestamp"); + } + } + } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalAggregationProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalAggregationProcessor.java index ec165fecc1..6d4eb6ae69 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalAggregationProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalAggregationProcessor.java @@ -23,7 +23,7 @@ import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.exception.SiddhiAppCreationException; import io.siddhi.core.exception.SiddhiAppRuntimeException; import io.siddhi.core.executor.ExpressionExecutor; @@ -31,6 +31,7 @@ import io.siddhi.core.query.processor.Processor; import io.siddhi.core.util.statistics.LatencyTracker; import io.siddhi.core.util.statistics.ThroughputTracker; +import org.apache.log4j.Logger; import java.util.List; @@ -39,12 +40,13 @@ */ public class IncrementalAggregationProcessor implements Processor { private final List incomingExpressionExecutors; - private final StreamEventPool streamEventPool; + private final StreamEventFactory streamEventFactory; private final LatencyTracker latencyTrackerInsert; private final ThroughputTracker throughputTrackerInsert; private SiddhiAppContext siddhiAppContext; private AggregationRuntime aggregationRuntime; private boolean isFirstEventArrived; + private static final Logger log = Logger.getLogger(IncrementalAggregationProcessor.class); public IncrementalAggregationProcessor(AggregationRuntime aggregationRuntime, List incomingExpressionExecutors, @@ -55,7 +57,7 @@ public IncrementalAggregationProcessor(AggregationRuntime aggregationRuntime, this.isFirstEventArrived = false; this.aggregationRuntime = aggregationRuntime; this.incomingExpressionExecutors = incomingExpressionExecutors; - this.streamEventPool = new StreamEventPool(processedMetaStreamEvent, 5); + this.streamEventFactory = new StreamEventFactory(processedMetaStreamEvent); this.latencyTrackerInsert = latencyTrackerInsert; this.throughputTrackerInsert = throughputTrackerInsert; this.siddhiAppContext = siddhiAppContext; @@ -76,16 +78,16 @@ public void process(ComplexEventChunk complexEventChunk) { aggregationRuntime.recreateInMemoryData(true, false); isFirstEventArrived = true; } - StreamEvent borrowedEvent = streamEventPool.borrowEvent(); + StreamEvent newEvent = streamEventFactory.newInstance(); for (int i = 0; i < incomingExpressionExecutors.size(); i++) { ExpressionExecutor expressionExecutor = incomingExpressionExecutors.get(i); Object outputData = expressionExecutor.execute(complexEvent); if (expressionExecutor instanceof IncrementalUnixTimeFunctionExecutor && outputData == null) { throw new SiddhiAppRuntimeException("Cannot retrieve the timestamp of event"); } - borrowedEvent.setOutputData(outputData, i); + newEvent.setOutputData(outputData, i); } - streamEventChunk.add(borrowedEvent); + streamEventChunk.add(newEvent); noOfEvents++; } aggregationRuntime.processEvents(streamEventChunk); @@ -115,16 +117,4 @@ public void setToLast(Processor processor) { throw new SiddhiAppCreationException("IncrementalAggregationProcessor does not support any " + "next/last processor"); } - - @Override - public Processor cloneProcessor(String key) { - throw new SiddhiAppCreationException("IncrementalAggregationProcessor cannot be cloned"); - } - - @Override - public void clean() { - for (ExpressionExecutor expressionExecutor : incomingExpressionExecutors) { - expressionExecutor.clean(); - } - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalDataAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalDataAggregator.java index b9b75dd972..e00b6beaf3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalDataAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalDataAggregator.java @@ -22,15 +22,22 @@ import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; import io.siddhi.core.util.IncrementalTimeConverterUtil; +import io.siddhi.core.util.parser.AggregationParser; +import io.siddhi.core.util.snapshot.state.PartitionSyncStateHolder; +import io.siddhi.core.util.snapshot.state.SingleSyncStateHolder; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.query.api.aggregation.TimePeriod; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; /** * This class implements the logic to aggregate data that is in-memory or in tables, in incremental data processing. @@ -41,103 +48,154 @@ */ public class IncrementalDataAggregator { private final List incrementalDurations; - private final TimePeriod.Duration aggregateForDuration; - private final BaseIncrementalValueStore baseIncrementalValueStore; - private final Map baseIncrementalValueStoreGroupByMap; + private final TimePeriod.Duration durationToAggregate; + private final List baseExecutorsForFind; + private final StateHolder valueStateHolder; + private final StreamEvent resetEvent; + private final long oldestEventTimestamp; + private ExpressionExecutor shouldUpdateTimestamp; + private final StreamEventFactory streamEventFactory; public IncrementalDataAggregator(List incrementalDurations, - TimePeriod.Duration aggregateForDuration, long oldestEventTimeStamp, - List baseExecutors, - MetaStreamEvent metaStreamEvent, SiddhiAppContext siddhiAppContext, - ExpressionExecutor shouldUpdateExpressionExecutor) { + TimePeriod.Duration durationToAggregate, long oldestEventTimestamp, + List baseExecutorsForFind, + MetaStreamEvent metaStreamEvent, + ExpressionExecutor shouldUpdateTimestamp, + boolean groupBy) { this.incrementalDurations = incrementalDurations; - this.aggregateForDuration = aggregateForDuration; - StreamEventPool streamEventPool = new StreamEventPool(metaStreamEvent, 10); - this.baseIncrementalValueStore = new BaseIncrementalValueStore(oldestEventTimeStamp, baseExecutors, - streamEventPool, siddhiAppContext, null, shouldUpdateExpressionExecutor); - this.baseIncrementalValueStoreGroupByMap = new HashMap<>(); + this.durationToAggregate = durationToAggregate; + this.oldestEventTimestamp = oldestEventTimestamp; + this.baseExecutorsForFind = baseExecutorsForFind; + this.streamEventFactory = new StreamEventFactory(metaStreamEvent); + if (groupBy) { + this.valueStateHolder = new PartitionSyncStateHolder(() -> new ValueState()); + } else { + this.valueStateHolder = new SingleSyncStateHolder(() -> new ValueState()); + } + this.resetEvent = AggregationParser.createRestEvent(metaStreamEvent, streamEventFactory.newInstance()); + this.shouldUpdateTimestamp = shouldUpdateTimestamp; } public ComplexEventChunk aggregateInMemoryData( Map incrementalExecutorMap) { - int startIndex = incrementalDurations.indexOf(aggregateForDuration); - for (int i = startIndex; i >= 0; i--) { - TimePeriod.Duration duration = incrementalDurations.get(i); + int startIndex = incrementalDurations.indexOf(durationToAggregate); + Set groupByKeys = new HashSet<>(); + for (int k = startIndex; k >= 0; k--) { + TimePeriod.Duration duration = incrementalDurations.get(k); IncrementalExecutor incrementalExecutor = incrementalExecutorMap.get(duration); - BaseIncrementalValueStore baseIncrementalValueStore = incrementalExecutor.getBaseIncrementalValueStore(); - Map baseIncrementalValueStoreGroupByMap = null; - if (incrementalExecutor.getBaseIncrementalValueStoreGroupByMap() != null) { - baseIncrementalValueStoreGroupByMap - = new HashMap<>(incrementalExecutor.getBaseIncrementalValueStoreGroupByMap()); - } - if (baseIncrementalValueStoreGroupByMap != null) { - for (Map.Entry entry : - baseIncrementalValueStoreGroupByMap.entrySet()) { - BaseIncrementalValueStore aBaseIncrementalValueStore = entry.getValue(); - if (aBaseIncrementalValueStore.isProcessed()) { - processInMemoryAggregates(aBaseIncrementalValueStore.createStreamEvent(), - aBaseIncrementalValueStore.getTimestamp(), entry.getKey()); + BaseIncrementalValueStore aBaseIncrementalValueStore = incrementalExecutor.getBaseIncrementalValueStore(); + Map groupedByEvents = aBaseIncrementalValueStore.getGroupedByEvents(); + for (Map.Entry eventEntry : groupedByEvents.entrySet()) { + long startTimeOfAggregates = IncrementalTimeConverterUtil.getStartTimeOfAggregates( + eventEntry.getValue().getTimestamp(), durationToAggregate); + String groupByKey = eventEntry.getKey() + "-" + startTimeOfAggregates; + synchronized (this) { + groupByKeys.add(groupByKey); + SiddhiAppContext.startGroupByFlow(groupByKey); + ValueState state = (ValueState) valueStateHolder.getState(); + try { + boolean shouldUpdate = true; + if (shouldUpdateTimestamp != null) { + shouldUpdate = (boolean) shouldUpdate( + shouldUpdateTimestamp.execute(eventEntry.getValue()), state); + } else { + state.lastTimestamp = oldestEventTimestamp; + } + // keeping timestamp value location as null + for (int i = 0; i < baseExecutorsForFind.size(); i++) { + if (shouldUpdate) { + ExpressionExecutor expressionExecutor = baseExecutorsForFind.get(i); + state.setValue(expressionExecutor.execute(eventEntry.getValue()), i + 1); + } else { + ExpressionExecutor expressionExecutor = baseExecutorsForFind.get(i); + if (!(expressionExecutor instanceof VariableExpressionExecutor)) { + state.setValue(expressionExecutor.execute(eventEntry.getValue()), i + 1); + } + } + } + } finally { + valueStateHolder.returnState(state); + SiddhiAppContext.stopGroupByFlow(); } } - } else if (baseIncrementalValueStore.isProcessed()) { - processInMemoryAggregates(baseIncrementalValueStore.createStreamEvent(), - baseIncrementalValueStore.getTimestamp(), null); } } - return createEventChunkFromAggregatedData(); + //clean all executors + for (String groupByKey : groupByKeys) { + SiddhiAppContext.startGroupByFlow(groupByKey); + try { + for (ExpressionExecutor expressionExecutor : baseExecutorsForFind) { + expressionExecutor.execute(resetEvent); + } + } finally { + SiddhiAppContext.stopGroupByFlow(); + } + } + return getProcessedEventChunk(); } - private void processInMemoryAggregates(StreamEvent streamEvent, long timestamp, String groupByKey) { - long startTimeOfAggregates = IncrementalTimeConverterUtil.getStartTimeOfAggregates(timestamp, - aggregateForDuration); - synchronized (this) { - if (groupByKey != null) { - BaseIncrementalValueStore aBaseIncrementalValueStore = - this.baseIncrementalValueStoreGroupByMap.computeIfAbsent( - groupByKey, k -> baseIncrementalValueStore.cloneStore(k, startTimeOfAggregates)); - process(streamEvent, aBaseIncrementalValueStore); - } else { - process(streamEvent, this.baseIncrementalValueStore); + + private ComplexEventChunk getProcessedEventChunk() { + ComplexEventChunk streamEventChunk = new ComplexEventChunk<>(true); + Map valueStoreMap = this.valueStateHolder.getAllGroupByStates(); + try { + for (State aState : valueStoreMap.values()) { + ValueState state = (ValueState) aState; + StreamEvent streamEvent = streamEventFactory.newInstance(); + long timestamp = state.lastTimestamp; + streamEvent.setTimestamp(timestamp); + state.setValue(timestamp, 0); + streamEvent.setOutputData(state.values); + streamEventChunk.add(streamEvent); } + } finally { + this.valueStateHolder.returnGroupByStates(valueStoreMap); } + return streamEventChunk; } - private void process(StreamEvent streamEvent, BaseIncrementalValueStore baseIncrementalValueStore) { - List expressionExecutors = baseIncrementalValueStore.getExpressionExecutors(); - boolean shouldUpdate = true; - ExpressionExecutor shouldUpdateExpressionExecutor = - baseIncrementalValueStore.getShouldUpdateExpressionExecutor(); - if (shouldUpdateExpressionExecutor != null) { - shouldUpdate = ((boolean) shouldUpdateExpressionExecutor.execute(streamEvent)); - } - for (int i = 0; i < expressionExecutors.size(); i++) { // keeping timestamp value location as null - if (shouldUpdate) { - ExpressionExecutor expressionExecutor = expressionExecutors.get(i); - baseIncrementalValueStore.setValue(expressionExecutor.execute(streamEvent), i + 1); - } else { - ExpressionExecutor expressionExecutor = expressionExecutors.get(i); - if (!(expressionExecutor instanceof VariableExpressionExecutor)) { - baseIncrementalValueStore.setValue(expressionExecutor.execute(streamEvent), i + 1); - } - } + private Object shouldUpdate(Object data, ValueState state) { + long timestamp = (long) data; + if (timestamp >= state.lastTimestamp) { + state.lastTimestamp = timestamp; + return true; } - baseIncrementalValueStore.setProcessed(true); + return false; } - private ComplexEventChunk createEventChunkFromAggregatedData() { - ComplexEventChunk processedInMemoryEventChunk = new ComplexEventChunk<>(true); - if (this.baseIncrementalValueStoreGroupByMap.size() == 0) { - if (this.baseIncrementalValueStore.isProcessed()) { - processedInMemoryEventChunk.add(this.baseIncrementalValueStore.createStreamEvent()); - } - } else { - for (Map.Entry entryAgainstGroupBy : - baseIncrementalValueStoreGroupByMap.entrySet()) { - processedInMemoryEventChunk.add(entryAgainstGroupBy.getValue().createStreamEvent()); - } + class ValueState extends State { + private Object[] values; + public long lastTimestamp; + + public ValueState() { + this.lastTimestamp = 0; + this.values = new Object[baseExecutorsForFind.size() + 1]; + } + + @Override + public boolean canDestroy() { + return values == null && lastTimestamp == 0; + } + + public void setValue(Object value, int position) { + values[position] = value; } - return processedInMemoryEventChunk; + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Values", values); + state.put("LastTimestamp", lastTimestamp); + return state; + } + + @Override + public void restore(Map state) { + values = (Object[]) state.get("Values"); + lastTimestamp = (Long) state.get("LastTimestamp"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalDataPurging.java b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalDataPurging.java index c12dad630d..2bf893dfa1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalDataPurging.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalDataPurging.java @@ -24,7 +24,7 @@ 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.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.exception.DataPurgingException; import io.siddhi.core.exception.SiddhiAppCreationException; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -66,7 +66,7 @@ public class IncrementalDataPurging implements Runnable { private long purgeExecutionInterval = Expression.Time.minute(15).value(); private boolean purgingEnabled = true; private Map retentionPeriods = new EnumMap<>(TimePeriod.Duration.class); - private StreamEventPool streamEventPool; + private StreamEventFactory streamEventFactory; private Map aggregationTables; private SiddhiQueryContext siddhiQueryContext; private ScheduledFuture scheduledPurgingTaskStatus; @@ -80,13 +80,13 @@ public class IncrementalDataPurging implements Runnable { private Map tableMap = new HashMap<>(); private AggregationDefinition aggregationDefinition; - public void init(AggregationDefinition aggregationDefinition, StreamEventPool streamEventPool, + public void init(AggregationDefinition aggregationDefinition, StreamEventFactory streamEventFactory, Map aggregationTables, Boolean isProcessingOnExternalTime, SiddhiQueryContext siddhiQueryContext) { this.siddhiQueryContext = siddhiQueryContext; this.aggregationDefinition = aggregationDefinition; List annotations = aggregationDefinition.getAnnotations(); - this.streamEventPool = streamEventPool; + this.streamEventFactory = streamEventFactory; this.aggregationTables = aggregationTables; if (isProcessingOnExternalTime) { purgingTimestampField = EXTERNAL_AGG_TIMESTAMP_FIELD; @@ -144,11 +144,11 @@ public void init(AggregationDefinition aggregationDefinition, StreamEventPool st } if (purgingEnabled) { // If interval is defined, default value of 15 min will be replaced by user input value - if (purge.getElement(SiddhiConstants.NAMESPACE_INTERVAL) != null) { - String interval = purge.getElement(SiddhiConstants.NAMESPACE_INTERVAL); + if (purge.getElement(SiddhiConstants.ANNOTATION_ELEMENT_INTERVAL) != null) { + String interval = purge.getElement(SiddhiConstants.ANNOTATION_ELEMENT_INTERVAL); purgeExecutionInterval = timeToLong(interval); } - List retentions = purge.getAnnotations(SiddhiConstants.NAMESPACE_RETENTION); + List retentions = purge.getAnnotations(SiddhiConstants.NAMESPACE_RETENTION_PERIOD); if (retentions != null && !retentions.isEmpty()) { Annotation retention = retentions.get(0); List elements = retention.getElements(); @@ -296,7 +296,7 @@ public void executeIncrementalDataPurging() { * creating stream event method **/ private StateEvent createStreamEvent(Object[] values, Long timestamp) { - StreamEvent streamEvent = streamEventPool.borrowEvent(); + StreamEvent streamEvent = streamEventFactory.newInstance(); streamEvent.setTimestamp(timestamp); streamEvent.setOutputData(values); StateEvent stateEvent = new StateEvent(2, 1); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalExecutor.java index 0f59048429..2fb45f8acc 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalExecutor.java @@ -19,19 +19,20 @@ package io.siddhi.core.aggregation; import io.siddhi.core.config.SiddhiAppContext; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.executor.ExpressionExecutor; -import io.siddhi.core.executor.VariableExpressionExecutor; import io.siddhi.core.query.selector.GroupByKeyGenerator; -import io.siddhi.core.query.selector.attribute.processor.executor.GroupByAggregationAttributeExecutor; import io.siddhi.core.table.Table; import io.siddhi.core.util.IncrementalTimeConverterUtil; import io.siddhi.core.util.Scheduler; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.util.parser.AggregationParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.query.api.aggregation.TimePeriod; import org.apache.log4j.Logger; @@ -42,64 +43,43 @@ /** * Incremental executor class which is responsible for performing incremental aggregation. */ -public class IncrementalExecutor implements Executor, Snapshotable { +public class IncrementalExecutor implements Executor { private static final Logger LOG = Logger.getLogger(IncrementalExecutor.class); private final StreamEvent resetEvent; private final ExpressionExecutor timestampExpressionExecutor; - private final String aggregatorName; + private final StateHolder stateHolder; private TimePeriod.Duration duration; private Table table; private GroupByKeyGenerator groupByKeyGenerator; - private StreamEventPool streamEventPool; - private long nextEmitTime = -1; - private long startTimeOfAggregates = -1; - private boolean timerStarted = false; - private boolean isGroupBy; + private StreamEventFactory streamEventFactory; private Executor next; private Scheduler scheduler; private boolean isRoot; - private String elementId; private boolean isProcessingExecutor; - private SiddhiAppContext siddhiAppContext; private BaseIncrementalValueStore baseIncrementalValueStore = null; - private Map baseIncrementalValueStoreGroupByMap = null; public IncrementalExecutor(TimePeriod.Duration duration, List processExpressionExecutors, GroupByKeyGenerator groupByKeyGenerator, MetaStreamEvent metaStreamEvent, IncrementalExecutor child, boolean isRoot, Table table, - SiddhiAppContext siddhiAppContext, String aggregatorName, - ExpressionExecutor shouldUpdateExpressionExecutor) { + SiddhiQueryContext siddhiQueryContext, String aggregatorName, + ExpressionExecutor shouldUpdateTimestamp) { this.duration = duration; this.next = child; this.isRoot = isRoot; this.table = table; - this.siddhiAppContext = siddhiAppContext; - this.aggregatorName = aggregatorName; - this.streamEventPool = new StreamEventPool(metaStreamEvent, 10); + this.streamEventFactory = new StreamEventFactory(metaStreamEvent); this.timestampExpressionExecutor = processExpressionExecutors.remove(0); - this.baseIncrementalValueStore = new BaseIncrementalValueStore(-1, processExpressionExecutors, - streamEventPool, siddhiAppContext, aggregatorName, shouldUpdateExpressionExecutor); this.isProcessingExecutor = false; - - if (groupByKeyGenerator != null) { - this.isGroupBy = true; - this.groupByKeyGenerator = groupByKeyGenerator; - this.baseIncrementalValueStoreGroupByMap = new HashMap<>(); - } else { - this.isGroupBy = false; - } - - this.resetEvent = streamEventPool.borrowEvent(); - this.resetEvent.setType(ComplexEvent.Type.RESET); + this.groupByKeyGenerator = groupByKeyGenerator; + this.baseIncrementalValueStore = new BaseIncrementalValueStore(processExpressionExecutors, streamEventFactory, + siddhiQueryContext, aggregatorName, shouldUpdateTimestamp, -1, true, false); + this.resetEvent = AggregationParser.createRestEvent(metaStreamEvent, streamEventFactory.newInstance()); setNextExecutor(child); - if (elementId == null) { - elementId = "IncrementalExecutor-" + siddhiAppContext.getElementIdGenerator().createNewId(); - } - siddhiAppContext.getSnapshotService().addSnapshotable(aggregatorName, this); - + this.stateHolder = siddhiQueryContext.generateStateHolder( + aggregatorName + "-" + this.getClass().getName(), false, () -> new ExecutorState()); } public void setScheduler(Scheduler scheduler) { @@ -116,40 +96,44 @@ public void execute(ComplexEventChunk streamEventChunk) { while (streamEventChunk.hasNext()) { StreamEvent streamEvent = (StreamEvent) streamEventChunk.next(); streamEventChunk.remove(); - - long timestamp = getTimestamp(streamEvent); - - startTimeOfAggregates = IncrementalTimeConverterUtil.getStartTimeOfAggregates(timestamp, duration); - - if (timestamp >= nextEmitTime) { - nextEmitTime = IncrementalTimeConverterUtil.getNextEmitTime(timestamp, duration, null); - dispatchAggregateEvents(startTimeOfAggregates); - sendTimerEvent(); - } - if (streamEvent.getType() == ComplexEvent.Type.CURRENT) { - processAggregates(streamEvent); + ExecutorState executorState = stateHolder.getState(); + try { + long timestamp = getTimestamp(streamEvent, executorState); + executorState.startTimeOfAggregates = IncrementalTimeConverterUtil.getStartTimeOfAggregates( + timestamp, duration); + if (timestamp >= executorState.nextEmitTime) { + executorState.nextEmitTime = IncrementalTimeConverterUtil.getNextEmitTime( + timestamp, duration, null); + dispatchAggregateEvents(executorState.startTimeOfAggregates); + sendTimerEvent(executorState); + } + if (streamEvent.getType() == ComplexEvent.Type.CURRENT) { + processAggregates(streamEvent, executorState); + } + } finally { + stateHolder.returnState(executorState); } } } - private void sendTimerEvent() { + private void sendTimerEvent(ExecutorState executorState) { if (getNextExecutor() != null) { - StreamEvent timerEvent = streamEventPool.borrowEvent(); + StreamEvent timerEvent = streamEventFactory.newInstance(); timerEvent.setType(ComplexEvent.Type.TIMER); - timerEvent.setTimestamp(startTimeOfAggregates); + timerEvent.setTimestamp(executorState.startTimeOfAggregates); ComplexEventChunk timerStreamEventChunk = new ComplexEventChunk<>(true); timerStreamEventChunk.add(timerEvent); next.execute(timerStreamEventChunk); } } - private long getTimestamp(StreamEvent streamEvent) { + private long getTimestamp(StreamEvent streamEvent, ExecutorState executorState) { long timestamp; if (streamEvent.getType() == ComplexEvent.Type.CURRENT) { timestamp = (long) timestampExpressionExecutor.execute(streamEvent); - if (isRoot && !timerStarted) { + if (isRoot && !executorState.timerStarted) { scheduler.notifyAt(IncrementalTimeConverterUtil.getNextEmitTime(timestamp, duration, null)); - timerStarted = true; + executorState.timerStarted = true; } } else { timestamp = streamEvent.getTimestamp(); @@ -171,127 +155,84 @@ public void setNextExecutor(Executor nextExecutor) { next = nextExecutor; } - private void processAggregates(StreamEvent streamEvent) { + private void processAggregates(StreamEvent streamEvent, ExecutorState executorState) { synchronized (this) { - if (isGroupBy) { + if (groupByKeyGenerator != null) { try { String groupedByKey = groupByKeyGenerator.constructEventKey(streamEvent); - GroupByAggregationAttributeExecutor.getKeyThreadLocal().set(groupedByKey); - BaseIncrementalValueStore aBaseIncrementalValueStore = baseIncrementalValueStoreGroupByMap - .computeIfAbsent(groupedByKey, - k -> baseIncrementalValueStore.cloneStore(k, startTimeOfAggregates)); - process(streamEvent, aBaseIncrementalValueStore); + SiddhiAppContext.startGroupByFlow(groupedByKey); + baseIncrementalValueStore.process(streamEvent); } finally { - GroupByAggregationAttributeExecutor.getKeyThreadLocal().remove(); + SiddhiAppContext.stopGroupByFlow(); } } else { - process(streamEvent, baseIncrementalValueStore); + baseIncrementalValueStore.process(streamEvent); } } } - private void process(StreamEvent streamEvent, BaseIncrementalValueStore baseIncrementalValueStore) { - List expressionExecutors = baseIncrementalValueStore.getExpressionExecutors(); - boolean shouldUpdate = true; - ExpressionExecutor shouldUpdateExpressionExecutor = - baseIncrementalValueStore.getShouldUpdateExpressionExecutor(); - if (shouldUpdateExpressionExecutor != null) { - shouldUpdate = ((boolean) shouldUpdateExpressionExecutor.execute(streamEvent)); - } - - for (int i = 0; i < expressionExecutors.size(); i++) { // keeping timestamp value location as null - if (shouldUpdate) { - ExpressionExecutor expressionExecutor = expressionExecutors.get(i); - baseIncrementalValueStore.setValue(expressionExecutor.execute(streamEvent), i + 1); - } else { - ExpressionExecutor expressionExecutor = expressionExecutors.get(i); - if (!(expressionExecutor instanceof VariableExpressionExecutor)) { - baseIncrementalValueStore.setValue(expressionExecutor.execute(streamEvent), i + 1); - } - } - } - baseIncrementalValueStore.setProcessed(true); - } private void dispatchAggregateEvents(long startTimeOfNewAggregates) { - if (isGroupBy) { - dispatchEvents(baseIncrementalValueStoreGroupByMap); - } else { - dispatchEvent(startTimeOfNewAggregates, baseIncrementalValueStore); - } + dispatchEvent(startTimeOfNewAggregates, baseIncrementalValueStore); } private void dispatchEvent(long startTimeOfNewAggregates, BaseIncrementalValueStore aBaseIncrementalValueStore) { if (aBaseIncrementalValueStore.isProcessed()) { - StreamEvent streamEvent = aBaseIncrementalValueStore.createStreamEvent(); + Map streamEventMap = aBaseIncrementalValueStore.getGroupedByEvents(); ComplexEventChunk eventChunk = new ComplexEventChunk<>(true); - eventChunk.add(streamEvent); - if (LOG.isDebugEnabled()) { - LOG.debug("Event dispatched by " + this.duration + " incremental executor: " + eventChunk.toString()); - } - if (isProcessingExecutor) { - table.addEvents(eventChunk, 1); - } - if (getNextExecutor() != null) { - next.execute(eventChunk); - } - } - cleanBaseIncrementalValueStore(startTimeOfNewAggregates, aBaseIncrementalValueStore); - } - - private void dispatchEvents(Map baseIncrementalValueGroupByStore) { - int noOfEvents = baseIncrementalValueGroupByStore.size(); - if (noOfEvents > 0) { - ComplexEventChunk eventChunk = new ComplexEventChunk<>(true); - for (BaseIncrementalValueStore aBaseIncrementalValueStore : baseIncrementalValueGroupByStore.values()) { - StreamEvent streamEvent = aBaseIncrementalValueStore.createStreamEvent(); - eventChunk.add(streamEvent); + for (StreamEvent event : streamEventMap.values()) { + eventChunk.add(event); } if (LOG.isDebugEnabled()) { LOG.debug("Event dispatched by " + this.duration + " incremental executor: " + eventChunk.toString()); } if (isProcessingExecutor) { - table.addEvents(eventChunk, noOfEvents); + table.addEvents(eventChunk, streamEventMap.size()); } if (getNextExecutor() != null) { next.execute(eventChunk); } } - for (BaseIncrementalValueStore baseIncrementalValueStore : baseIncrementalValueGroupByStore.values()) { - baseIncrementalValueStore.clean(); - } - baseIncrementalValueGroupByStore.clear(); + cleanBaseIncrementalValueStore(startTimeOfNewAggregates, aBaseIncrementalValueStore); } private void cleanBaseIncrementalValueStore(long startTimeOfNewAggregates, BaseIncrementalValueStore baseIncrementalValueStore) { - baseIncrementalValueStore.clearValues(); - baseIncrementalValueStore.setTimestamp(startTimeOfNewAggregates); - baseIncrementalValueStore.setProcessed(false); + baseIncrementalValueStore.clearValues(startTimeOfNewAggregates, resetEvent); for (ExpressionExecutor expressionExecutor : baseIncrementalValueStore.getExpressionExecutors()) { expressionExecutor.execute(resetEvent); } } - - Map getBaseIncrementalValueStoreGroupByMap() { - return baseIncrementalValueStoreGroupByMap; - } - BaseIncrementalValueStore getBaseIncrementalValueStore() { return baseIncrementalValueStore; } public long getAggregationStartTimestamp() { - return this.startTimeOfAggregates; + ExecutorState state = stateHolder.getState(); + try { + return state.startTimeOfAggregates; + } finally { + stateHolder.returnState(state); + } } public long getNextEmitTime() { - return nextEmitTime; + ExecutorState state = stateHolder.getState(); + try { + return state.nextEmitTime; + } finally { + stateHolder.returnState(state); + } } public void setValuesForInMemoryRecreateFromTable(long emitTimeOfLatestEventInTable) { - this.nextEmitTime = emitTimeOfLatestEventInTable; + ExecutorState state = stateHolder.getState(); + try { + state.nextEmitTime = emitTimeOfLatestEventInTable; + } finally { + stateHolder.returnState(state); + } } public boolean isProcessingExecutor() { @@ -303,45 +244,38 @@ public void setProcessingExecutor(boolean processingExecutor) { } public void clearExecutor() { - if (isGroupBy) { - this.baseIncrementalValueStoreGroupByMap.clear(); - } else { - cleanBaseIncrementalValueStore(-1, this.baseIncrementalValueStore); - } + cleanBaseIncrementalValueStore(-1, this.baseIncrementalValueStore); } - @Override - public Map currentState() { - Map state = new HashMap<>(); + class ExecutorState extends State { + private long nextEmitTime = -1; + private long startTimeOfAggregates = -1; + private boolean timerStarted = false; + private boolean canDestroy = false; - state.put("NextEmitTime", nextEmitTime); - state.put("StartTimeOfAggregates", startTimeOfAggregates); - state.put("TimerStarted", timerStarted); - return state; - } + @Override + public boolean canDestroy() { + return canDestroy; + } - @Override - public void restoreState(Map state) { - nextEmitTime = (long) state.get("NextEmitTime"); - startTimeOfAggregates = (long) state.get("StartTimeOfAggregates"); - timerStarted = (boolean) state.get("TimerStarted"); - } + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("NextEmitTime", nextEmitTime); + state.put("StartTimeOfAggregates", startTimeOfAggregates); + state.put("TimerStarted", timerStarted); + return state; + } - @Override - public String getElementId() { - return elementId; - } + @Override + public void restore(Map state) { + nextEmitTime = (long) state.get("NextEmitTime"); + startTimeOfAggregates = (long) state.get("StartTimeOfAggregates"); + timerStarted = (boolean) state.get("TimerStarted"); + } - @Override - public void clean() { - timestampExpressionExecutor.clean(); - baseIncrementalValueStore.clean(); - scheduler.clean(); - if (baseIncrementalValueStoreGroupByMap != null) { - for (BaseIncrementalValueStore aBaseIncrementalValueStore : baseIncrementalValueStoreGroupByMap.values()) { - aBaseIncrementalValueStore.clean(); - } + public void setCanDestroy(boolean canDestroy) { + this.canDestroy = canDestroy; } - siddhiAppContext.getSnapshotService().removeSnapshotable(aggregatorName, this); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalExternalTimestampDataAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalExternalTimestampDataAggregator.java index 92b88ee3fe..74e0c7f58b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalExternalTimestampDataAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/IncrementalExternalTimestampDataAggregator.java @@ -18,17 +18,25 @@ package io.siddhi.core.aggregation; import io.siddhi.core.config.SiddhiAppContext; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; import io.siddhi.core.query.selector.GroupByKeyGenerator; +import io.siddhi.core.util.parser.AggregationParser; +import io.siddhi.core.util.snapshot.state.PartitionSyncStateHolder; +import io.siddhi.core.util.snapshot.state.SingleSyncStateHolder; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateHolder; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; /** * This class implements logic to process aggregates(after retrieval from tables) that were aggregated @@ -37,67 +45,133 @@ public class IncrementalExternalTimestampDataAggregator { private final GroupByKeyGenerator groupByKeyGenerator; - private final BaseIncrementalValueStore baseIncrementalValueStore; - private final Map baseIncrementalValueGroupByStore; + private final List baseExecutors; + private final StateHolder valueStateHolder; + private final StreamEvent resetEvent; + private StreamEventFactory streamEventFactory; + private final ExpressionExecutor shouldUpdateTimestamp; public IncrementalExternalTimestampDataAggregator(List baseExecutors, GroupByKeyGenerator groupByKeyGenerator, MetaStreamEvent metaStreamEvent, - SiddhiAppContext siddhiAppContext, - ExpressionExecutor shouldUpdateExpressionExecutor) { - StreamEventPool streamEventPool = new StreamEventPool(metaStreamEvent, 10); - - List expressionExecutorsWithoutTime = baseExecutors.subList(1, baseExecutors.size()); - this.baseIncrementalValueStore = new BaseIncrementalValueStore(-1, - expressionExecutorsWithoutTime, streamEventPool, siddhiAppContext, null, - shouldUpdateExpressionExecutor); - this.baseIncrementalValueGroupByStore = new HashMap<>(); + SiddhiQueryContext siddhiQueryContext, + ExpressionExecutor shouldUpdateTimestamp) { + this.baseExecutors = baseExecutors.subList(1, baseExecutors.size()); + this.streamEventFactory = new StreamEventFactory(metaStreamEvent); + this.shouldUpdateTimestamp = shouldUpdateTimestamp; this.groupByKeyGenerator = groupByKeyGenerator; + if (groupByKeyGenerator != null) { + this.valueStateHolder = new PartitionSyncStateHolder(() -> new ValueState()); + } else { + this.valueStateHolder = new SingleSyncStateHolder(() -> new ValueState()); + } + this.resetEvent = AggregationParser.createRestEvent(metaStreamEvent, streamEventFactory.newInstance()); } public ComplexEventChunk aggregateData(ComplexEventChunk retrievedData) { + Set groupByKeys = new HashSet<>(); while (retrievedData.hasNext()) { StreamEvent streamEvent = retrievedData.next(); String groupByKey = groupByKeyGenerator.constructEventKey(streamEvent); - BaseIncrementalValueStore baseIncrementalValueStore = baseIncrementalValueGroupByStore - .computeIfAbsent( - groupByKey, k -> this.baseIncrementalValueStore.cloneStore(k, -1) - ); - process(streamEvent, baseIncrementalValueStore); + groupByKeys.add(groupByKey); + SiddhiAppContext.startGroupByFlow(groupByKey); + ValueState state = (ValueState) valueStateHolder.getState(); + try { + boolean shouldUpdate = true; + if (shouldUpdateTimestamp != null) { + shouldUpdate = (boolean) shouldUpdate(shouldUpdateTimestamp.execute(streamEvent), state); + } + for (int i = 0; i < baseExecutors.size(); i++) { // keeping timestamp value location as null + if (shouldUpdate) { + ExpressionExecutor expressionExecutor = baseExecutors.get(i); + state.setValue(expressionExecutor.execute(streamEvent), i + 1); + } else { + ExpressionExecutor expressionExecutor = baseExecutors.get(i); + if (!(expressionExecutor instanceof VariableExpressionExecutor)) { + state.setValue(expressionExecutor.execute(streamEvent), i + 1); + } + } + } + } finally { + valueStateHolder.returnState(state); + SiddhiAppContext.stopGroupByFlow(); + } + } + + //clean all executors + for (String groupByKey : groupByKeys) { + SiddhiAppContext.startGroupByFlow(groupByKey); + try { + for (ExpressionExecutor expressionExecutor : baseExecutors) { + expressionExecutor.execute(resetEvent); + } + } finally { + SiddhiAppContext.stopGroupByFlow(); + } } return createEventChunkFromAggregatedData(); } - private void process(StreamEvent streamEvent, BaseIncrementalValueStore baseIncrementalValueStore) { - List expressionExecutors = baseIncrementalValueStore.getExpressionExecutors(); - boolean shouldUpdate = true; - ExpressionExecutor shouldUpdateExpressionExecutor = - baseIncrementalValueStore.getShouldUpdateExpressionExecutor(); - if (shouldUpdateExpressionExecutor != null) { - shouldUpdate = ((boolean) shouldUpdateExpressionExecutor.execute(streamEvent)); + private Object shouldUpdate(Object data, ValueState state) { + long timestamp = (long) data; + if (timestamp >= state.lastTimestamp) { + state.lastTimestamp = timestamp; + return true; } + return false; + } - for (int i = 0; i < expressionExecutors.size(); i++) { // keeping timestamp value location as null - if (shouldUpdate) { - ExpressionExecutor expressionExecutor = expressionExecutors.get(i); - baseIncrementalValueStore.setValue(expressionExecutor.execute(streamEvent), i + 1); - } else { - ExpressionExecutor expressionExecutor = expressionExecutors.get(i); - if (!(expressionExecutor instanceof VariableExpressionExecutor)) { - baseIncrementalValueStore.setValue(expressionExecutor.execute(streamEvent), i + 1); - } + + private ComplexEventChunk createEventChunkFromAggregatedData() { + ComplexEventChunk streamEventChunk = new ComplexEventChunk<>(true); + Map valueStoreMap = this.valueStateHolder.getAllGroupByStates(); + try { + for (State aState : valueStoreMap.values()) { + ValueState state = (ValueState) aState; + StreamEvent streamEvent = streamEventFactory.newInstance(); + long timestamp = state.lastTimestamp; + streamEvent.setTimestamp(timestamp); + state.setValue(timestamp, 0); + streamEvent.setOutputData(state.values); + streamEventChunk.add(streamEvent); } + } finally { + this.valueStateHolder.returnGroupByStates(valueStoreMap); } - baseIncrementalValueStore.setProcessed(true); + return streamEventChunk; } - private ComplexEventChunk createEventChunkFromAggregatedData() { - ComplexEventChunk processedInMemoryEventChunk = new ComplexEventChunk<>(true); - for (Map.Entry entryAgainstTime : - baseIncrementalValueGroupByStore.entrySet()) { - processedInMemoryEventChunk.add(entryAgainstTime.getValue().createStreamEvent()); + class ValueState extends State { + private Object[] values; + private long lastTimestamp = 0; + + public ValueState() { + this.values = new Object[baseExecutors.size() + 1]; + } + + @Override + public boolean canDestroy() { + return values == null && lastTimestamp == 0; + } + + public void setValue(Object value, int position) { + values[position] = value; } - return processedInMemoryEventChunk; + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Values", values); + state.put("LastTimestamp", lastTimestamp); + return state; + } + + @Override + public void restore(Map state) { + values = (Object[]) state.get("Values"); + lastTimestamp = (Long) state.get("LastTimestamp"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/RecreateInMemoryData.java b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/RecreateInMemoryData.java index 1259e821ff..3a70245fc7 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/RecreateInMemoryData.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/aggregation/RecreateInMemoryData.java @@ -23,7 +23,7 @@ import io.siddhi.core.event.Event; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.query.StoreQueryRuntime; import io.siddhi.core.table.Table; import io.siddhi.core.util.IncrementalTimeConverterUtil; @@ -50,7 +50,7 @@ public class RecreateInMemoryData { private final Map incrementalExecutorMap; private final Map incrementalExecutorMapForPartitions; private final SiddhiAppContext siddhiAppContext; - private final StreamEventPool streamEventPool; + private final StreamEventFactory streamEventFactory; private final Map tableMap; private final Map windowMap; private final Map aggregationMap; @@ -67,7 +67,7 @@ public RecreateInMemoryData(List incrementalDurations, this.aggregationTables = aggregationTables; this.incrementalExecutorMap = incrementalExecutorMap; this.siddhiAppContext = siddhiAppContext; - this.streamEventPool = new StreamEventPool(metaStreamEvent, 10); + this.streamEventFactory = new StreamEventFactory(metaStreamEvent); this.tableMap = tableMap; this.windowMap = windowMap; this.aggregationMap = aggregationMap; @@ -163,7 +163,7 @@ public void recreateInMemoryData(boolean refreshReadingExecutors) { ComplexEventChunk complexEventChunk = new ComplexEventChunk<>(false); for (Event event : events) { - StreamEvent streamEvent = streamEventPool.borrowEvent(); + StreamEvent streamEvent = streamEventFactory.newInstance(); streamEvent.setOutputData(event.getData()); complexEventChunk.add(streamEvent); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/config/SiddhiAppContext.java b/modules/siddhi-core/src/main/java/io/siddhi/core/config/SiddhiAppContext.java index 6fa92c2b6d..1cd2966595 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/config/SiddhiAppContext.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/config/SiddhiAppContext.java @@ -20,11 +20,17 @@ import com.lmax.disruptor.ExceptionHandler; import io.siddhi.core.function.Script; -import io.siddhi.core.util.ElementIdGenerator; +import io.siddhi.core.util.IdGenerator; import io.siddhi.core.util.Scheduler; +import io.siddhi.core.util.SiddhiConstants; import io.siddhi.core.util.ThreadBarrier; -import io.siddhi.core.util.extension.holder.EternalReferencedHolder; +import io.siddhi.core.util.extension.holder.ExternalReferencedHolder; import io.siddhi.core.util.snapshot.SnapshotService; +import io.siddhi.core.util.snapshot.state.EmptyStateHolder; +import io.siddhi.core.util.snapshot.state.SingleStateHolder; +import io.siddhi.core.util.snapshot.state.SingleSyncStateHolder; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.core.util.statistics.StatisticsManager; import io.siddhi.core.util.timestamp.TimestampGenerator; import io.siddhi.query.api.SiddhiApp; @@ -53,12 +59,12 @@ public class SiddhiAppContext { private ExecutorService executorService; private ScheduledExecutorService scheduledExecutorService; - private List eternalReferencedHolders; + private List externalReferencedHolders; private SnapshotService snapshotService; private ThreadBarrier threadBarrier = null; private TimestampGenerator timestampGenerator = null; - private ElementIdGenerator elementIdGenerator; + private IdGenerator idGenerator; private Map scriptFunctionMap; private ExceptionHandler disruptorExceptionHandler; private ExceptionListener runtimeExceptionListener; @@ -67,13 +73,43 @@ public class SiddhiAppContext { private List includedMetrics; private boolean transportChannelCreationEnabled; private List schedulerList; + private static final ThreadLocal GROUP_BY_KEY = new ThreadLocal<>(); + private static final ThreadLocal PARTITION_KEY = new ThreadLocal<>(); public SiddhiAppContext() { - this.eternalReferencedHolders = Collections.synchronizedList(new LinkedList<>()); + this.externalReferencedHolders = Collections.synchronizedList(new LinkedList<>()); this.scriptFunctionMap = new HashMap(); this.schedulerList = new ArrayList(); } + public static void startGroupByFlow(String key) { + GROUP_BY_KEY.set(key); + } + + public static void stopGroupByFlow() { + GROUP_BY_KEY.set(null); + } + + public static void startPartitionFlow(String key) { + PARTITION_KEY.set(key); + } + + public static void stopPartitionFlow() { + PARTITION_KEY.set(null); + } + + public static String getCurrentFlowId() { + return PARTITION_KEY.get() + "--" + GROUP_BY_KEY.get(); + } + + public static String getPartitionFlowId() { + return PARTITION_KEY.get(); + } + + public static String getGroupByFlowId() { + return GROUP_BY_KEY.get(); + } + public SiddhiContext getSiddhiContext() { return siddhiContext; } @@ -130,12 +166,12 @@ public void setScheduledExecutorService(ScheduledExecutorService scheduledExecut this.scheduledExecutorService = scheduledExecutorService; } - public synchronized void addEternalReferencedHolder(EternalReferencedHolder eternalReferencedHolder) { - eternalReferencedHolders.add(eternalReferencedHolder); + public void addEternalReferencedHolder(ExternalReferencedHolder externalReferencedHolder) { + externalReferencedHolders.add(externalReferencedHolder); } - public List getEternalReferencedHolders() { - return Collections.unmodifiableList(new ArrayList<>(eternalReferencedHolders)); + public List getExternalReferencedHolders() { + return Collections.unmodifiableList(new ArrayList<>(externalReferencedHolders)); } public ThreadBarrier getThreadBarrier() { @@ -170,12 +206,12 @@ public void setSnapshotService(SnapshotService snapshotService) { this.snapshotService = snapshotService; } - public ElementIdGenerator getElementIdGenerator() { - return elementIdGenerator; - } +// public IdGenerator getElementIdGenerator() { +// return idGenerator; +// } - public void setElementIdGenerator(ElementIdGenerator elementIdGenerator) { - this.elementIdGenerator = elementIdGenerator; + public void setIdGenerator(IdGenerator idGenerator) { + this.idGenerator = idGenerator; } public Script getScript(String name) { @@ -249,4 +285,28 @@ public void addScheduler(Scheduler scheduler) { public List getSchedulerList() { return schedulerList; } + + public StateHolder generateStateHolder(String name, StateFactory stateFactory) { + return generateStateHolder(name, stateFactory, false); + } + + public StateHolder generateStateHolder(String name, StateFactory stateFactory, boolean unSafe) { + if (stateFactory != null) { + StateHolder stateHolder; + if (unSafe) { + stateHolder = new SingleStateHolder(stateFactory); + } else { + stateHolder = new SingleSyncStateHolder(stateFactory); + } + if (SnapshotService.getSkipStateStorageThreadLocal().get() == null || + !SnapshotService.getSkipStateStorageThreadLocal().get()) { + Map stateHolderMap = getSnapshotService().getStateHolderMap( + SiddhiConstants.PARTITION_ID_DEFAULT, SiddhiConstants.PARTITION_ID_DEFAULT); + stateHolderMap.put(name + "-" + idGenerator.createNewId(), stateHolder); + } + return stateHolder; + } else { + return new EmptyStateHolder(); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/config/SiddhiQueryContext.java b/modules/siddhi-core/src/main/java/io/siddhi/core/config/SiddhiQueryContext.java index 82f3b4db82..636a278b62 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/config/SiddhiQueryContext.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/config/SiddhiQueryContext.java @@ -18,10 +18,22 @@ package io.siddhi.core.config; +import io.siddhi.core.util.IdGenerator; +import io.siddhi.core.util.SiddhiConstants; +import io.siddhi.core.util.snapshot.SnapshotService; +import io.siddhi.core.util.snapshot.state.EmptyStateHolder; +import io.siddhi.core.util.snapshot.state.PartitionStateHolder; +import io.siddhi.core.util.snapshot.state.PartitionSyncStateHolder; +import io.siddhi.core.util.snapshot.state.SingleStateHolder; +import io.siddhi.core.util.snapshot.state.SingleSyncStateHolder; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.core.util.statistics.LatencyTracker; import io.siddhi.query.api.SiddhiApp; import io.siddhi.query.api.execution.query.output.stream.OutputStream; +import java.util.Map; + /** * Holder object for context information of {@link SiddhiApp}. */ @@ -29,13 +41,22 @@ public class SiddhiQueryContext { private SiddhiAppContext siddhiAppContext = null; private String name; + private String partitionId; + private boolean partitioned; private OutputStream.OutputEventType outputEventType; private LatencyTracker latencyTracker; + private Map stateHolderMap; + private IdGenerator idGenerator; public SiddhiQueryContext(SiddhiAppContext siddhiAppContext, String queryName) { + this(siddhiAppContext, queryName, SiddhiConstants.PARTITION_ID_DEFAULT); + } + public SiddhiQueryContext(SiddhiAppContext siddhiAppContext, String queryName, String partitionId) { this.siddhiAppContext = siddhiAppContext; this.name = queryName; + this.partitionId = partitionId; + this.idGenerator = new IdGenerator(); } public String getName() { @@ -73,4 +94,48 @@ public void setLatencyTracker(LatencyTracker latencyTracker) { public LatencyTracker getLatencyTracker() { return latencyTracker; } + + public boolean isPartitioned() { + return partitioned; + } + + public void setPartitioned(boolean partitionable) { + partitioned = partitionable; + } + + public String generateNewId() { + return idGenerator.createNewId(); + } + + public StateHolder generateStateHolder(String name, boolean groupBy, StateFactory stateFactory) { + return generateStateHolder(name, groupBy, stateFactory, false); + } + + public StateHolder generateStateHolder(String name, boolean groupBy, StateFactory stateFactory, boolean unSafe) { + if (stateFactory != null) { + StateHolder stateHolder; + if (unSafe) { + if (partitioned || groupBy) { + stateHolder = new PartitionStateHolder(stateFactory); + } else { + stateHolder = new SingleStateHolder(stateFactory); + } + } else { + if (partitioned || groupBy) { + stateHolder = new PartitionSyncStateHolder(stateFactory); + } else { + stateHolder = new SingleSyncStateHolder(stateFactory); + } + } + + if (SnapshotService.getSkipStateStorageThreadLocal().get() == null || + !SnapshotService.getSkipStateStorageThreadLocal().get()) { + stateHolderMap = siddhiAppContext.getSnapshotService().getStateHolderMap(partitionId, this.getName()); + stateHolderMap.put(idGenerator.createNewId() + "-" + name, stateHolder); + } + return stateHolder; + } else { + return new EmptyStateHolder(); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/debugger/SiddhiDebugger.java b/modules/siddhi-core/src/main/java/io/siddhi/core/debugger/SiddhiDebugger.java index fd6f7aa234..8d7b8ab5cb 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/debugger/SiddhiDebugger.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/debugger/SiddhiDebugger.java @@ -20,7 +20,6 @@ import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.util.snapshot.SnapshotService; -import io.siddhi.core.util.snapshot.Snapshotable; import org.apache.log4j.Logger; import java.util.HashMap; @@ -204,8 +203,7 @@ public void setDebuggerCallback(SiddhiDebuggerCallback siddhiDebuggerCallback) { } /** - * Get all the events stored in the {@link Snapshotable} entities of the given - * query. + * Get all the event states stored in the given query. * * @param queryName name of the Siddhi query * @return QueryState internal state of the query diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventCloner.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventCloner.java index 33ea14f022..bb87f32598 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventCloner.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventCloner.java @@ -28,11 +28,11 @@ public class StateEventCloner { private final int streamEventSize; // private final int preOutputDataSize; private final int outputDataSize; - private final StateEventPool stateEventPool; + private final StateEventFactory stateEventFactory; - public StateEventCloner(MetaStateEvent metaStateEvent, StateEventPool stateEventPool) { + public StateEventCloner(MetaStateEvent metaStateEvent, StateEventFactory stateEventFactory) { - this.stateEventPool = stateEventPool; + this.stateEventFactory = stateEventFactory; this.streamEventSize = metaStateEvent.getStreamEventCount(); // this.preOutputDataSize = metaStateEvent.getPreOutputDataAttributes().size(); this.outputDataSize = metaStateEvent.getOutputDataAttributes().size(); @@ -46,16 +46,16 @@ public StateEventCloner(MetaStateEvent metaStateEvent, StateEventPool stateEvent * @return StateEvent */ public StateEvent copyStateEvent(StateEvent stateEvent) { - StateEvent borrowedEvent = stateEventPool.borrowEvent(); + StateEvent newEvent = stateEventFactory.newInstance(); if (outputDataSize > 0) { - System.arraycopy(stateEvent.getOutputData(), 0, borrowedEvent.getOutputData(), 0, outputDataSize); + System.arraycopy(stateEvent.getOutputData(), 0, newEvent.getOutputData(), 0, outputDataSize); } if (streamEventSize > 0) { - System.arraycopy(stateEvent.getStreamEvents(), 0, borrowedEvent.getStreamEvents(), 0, streamEventSize); + System.arraycopy(stateEvent.getStreamEvents(), 0, newEvent.getStreamEvents(), 0, streamEventSize); } - borrowedEvent.setType(stateEvent.getType()); - borrowedEvent.setTimestamp(stateEvent.getTimestamp()); - borrowedEvent.setId(stateEvent.getId()); - return borrowedEvent; + newEvent.setType(stateEvent.getType()); + newEvent.setTimestamp(stateEvent.getTimestamp()); + newEvent.setId(stateEvent.getId()); + return newEvent; } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventFactory.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventFactory.java index 2b9e78cf78..6c465efa8c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventFactory.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventFactory.java @@ -32,6 +32,11 @@ public StateEventFactory(int eventSize, int outputDataSize) { this.outputDataSize = outputDataSize; } + public StateEventFactory(MetaStateEvent metaStateEvent) { + this.eventSize = metaStateEvent.getStreamEventCount(); + this.outputDataSize = metaStateEvent.getOutputDataAttributes().size(); + } + public StateEvent newInstance() { return new StateEvent(eventSize, outputDataSize); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventPool.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventPool.java deleted file mode 100644 index e431315af4..0000000000 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/state/StateEventPool.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package io.siddhi.core.event.state; - -/** - * Event pool containing StreamEvent for reuse - * This is not a thread safe implementation - */ -public class StateEventPool { - - private StateEventFactory eventFactory; - private int size; - private int index = 0; - private StateEvent stateEventList; - private long id = 0; - - public StateEventPool(MetaStateEvent metaStateEvent, int size) { - eventFactory = new StateEventFactory(metaStateEvent.getStreamEventCount(), - metaStateEvent.getOutputDataAttributes().size()); - this.size = size; - } - -// public StateEventPool(int beforeWindowDataSize, int onAfterWindowDataSize, int outputDataSize, int poolSize) { -// eventFactory = new StreamEventFactory(beforeWindowDataSize, onAfterWindowDataSize, outputDataSize); -// this.size = poolSize; -// } - - /** - * Borrowing an StateEvent - * - * @return if StateEvent exist in the pool an existing event if not a new StateEvent will be returned - */ - public StateEvent borrowEvent() { - if (index > 0) { - StateEvent event = stateEventList; - stateEventList = stateEventList.getNext(); - event.setNext(null); - index--; - event.setId(++id); - return event; - } else { - StateEvent event = eventFactory.newInstance(); - event.setId(++id); - return event; - } - } - - /** - * Collects the used InnerStreamEvents - * If the pool has space the returned event will be added to the pool else it will be dropped - * - * @param stateEvent used event - */ - public void returnEvents(StateEvent stateEvent) { - if (stateEvent != null) { - if (index < size) { - StateEvent first = stateEvent; - StateEvent last = stateEvent; - while (stateEvent != null) { - last = stateEvent; - index++; - stateEvent = stateEvent.getNext(); - } - last.setNext(stateEventList); - stateEventList = first; - } - } - - } - - /** - * @return Occupied buffer size - */ - public int getBufferedEventsSize() { - return index; - } - - public int getSize() { - return size; - } -} 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 dbae1a194b..d986cad2e4 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 @@ -28,11 +28,11 @@ public class StreamEventCloner { private final int beforeWindowDataSize; private final int onAfterWindowDataSize; private final int outputDataSize; - private final StreamEventPool streamEventPool; + private final StreamEventFactory eventFactory; - public StreamEventCloner(MetaStreamEvent metaStreamEvent, StreamEventPool streamEventPool) { + public StreamEventCloner(MetaStreamEvent metaStreamEvent, StreamEventFactory eventFactory) { - this.streamEventPool = streamEventPool; + this.eventFactory = eventFactory; this.beforeWindowDataSize = metaStreamEvent.getBeforeWindowData().size(); this.onAfterWindowDataSize = metaStreamEvent.getOnAfterWindowData().size(); this.outputDataSize = metaStreamEvent.getOutputData().size(); @@ -46,20 +46,20 @@ public StreamEventCloner(MetaStreamEvent metaStreamEvent, StreamEventPool stream * @return StreamEvent */ public StreamEvent copyStreamEvent(StreamEvent streamEvent) { - StreamEvent borrowedEvent = streamEventPool.borrowEvent(); + StreamEvent newEvent = eventFactory.newInstance(); if (beforeWindowDataSize > 0) { - System.arraycopy(streamEvent.getBeforeWindowData(), 0, borrowedEvent.getBeforeWindowData(), 0, + System.arraycopy(streamEvent.getBeforeWindowData(), 0, newEvent.getBeforeWindowData(), 0, beforeWindowDataSize); } if (onAfterWindowDataSize > 0) { - System.arraycopy(streamEvent.getOnAfterWindowData(), 0, borrowedEvent.getOnAfterWindowData(), 0, + System.arraycopy(streamEvent.getOnAfterWindowData(), 0, newEvent.getOnAfterWindowData(), 0, onAfterWindowDataSize); } if (outputDataSize > 0) { - System.arraycopy(streamEvent.getOutputData(), 0, borrowedEvent.getOutputData(), 0, outputDataSize); + System.arraycopy(streamEvent.getOutputData(), 0, newEvent.getOutputData(), 0, outputDataSize); } - borrowedEvent.setType(streamEvent.getType()); - borrowedEvent.setTimestamp(streamEvent.getTimestamp()); - return borrowedEvent; + newEvent.setType(streamEvent.getType()); + newEvent.setTimestamp(streamEvent.getTimestamp()); + return newEvent; } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventFactory.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventFactory.java index 12ceff5bd1..61350da1be 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventFactory.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventFactory.java @@ -31,6 +31,17 @@ public class StreamEventFactory implements EventFactory, Serializab private int onAfterWindowDataSize; private int outputDataSize; + /** + * Initialization of the factory using MetaStreamEvent + * + * @param metaStreamEvent meta stream event + */ + public StreamEventFactory(MetaStreamEvent metaStreamEvent) { + this.beforeWindowDataSize = metaStreamEvent.getBeforeWindowData().size(); + this.onAfterWindowDataSize = metaStreamEvent.getOnAfterWindowData().size(); + this.outputDataSize = metaStreamEvent.getOutputData().size(); + } + /** * Initialization of the factory with event data sizes * diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventPool.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventPool.java deleted file mode 100644 index 877b0e2f57..0000000000 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/StreamEventPool.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package io.siddhi.core.event.stream; - -import java.io.Serializable; - -/** - * Event pool containing StreamEvent for reuse - * This is not a thread safe implementation - */ -public class StreamEventPool implements Serializable { - - private static final long serialVersionUID = -1743558131917334571L; - private StreamEventFactory eventFactory; - private int size; - private int index = 0; - private StreamEvent streamEventList; - - public StreamEventPool(MetaStreamEvent metaStreamEvent, int size) { - eventFactory = new StreamEventFactory(metaStreamEvent.getBeforeWindowData().size(), - metaStreamEvent.getOnAfterWindowData().size(), - metaStreamEvent.getOutputData().size()); - this.size = size; - } - - public StreamEventPool(int beforeWindowDataSize, int onAfterWindowDataSize, int outputDataSize, int poolSize) { - eventFactory = new StreamEventFactory(beforeWindowDataSize, onAfterWindowDataSize, outputDataSize); - this.size = poolSize; - } - - /** - * Borrowing an StreamEvent - * - * @return if StreamEvent exist in the pool an existing event if not a new StreamEvent will be returned - */ - public StreamEvent borrowEvent() { - if (index > 0) { - StreamEvent event = streamEventList; - streamEventList = streamEventList.getNext(); - event.setNext(null); - index--; - return event; - } else { - return eventFactory.newInstance(); - } - } - - /** - * Collects the used InnerStreamEvents - * If the pool has space the returned event will be added to the pool else it will be dropped - * - * @param streamEvent used event - */ - public void returnEvents(StreamEvent streamEvent) { - if (streamEvent != null) { - if (index < size) { - StreamEvent first = streamEvent; - StreamEvent last = streamEvent; - while (streamEvent != null) { - last = streamEvent; - index++; - streamEvent = streamEvent.getNext(); - } - last.setNext(streamEventList); - streamEventList = first; - } - } - - } - - /** - * @return Occupied buffer size - */ - public int getBufferedEventsSize() { - return index; - } - - public int getSize() { - return size; - } -} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/ConversionStreamEventChunk.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/ConversionStreamEventChunk.java index 22cacf9d18..30c988ed79 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/ConversionStreamEventChunk.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/ConversionStreamEventChunk.java @@ -23,7 +23,7 @@ import io.siddhi.core.event.Event; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; /** * A StreamEvent holder that can also convert other events into StreamEvents @@ -32,53 +32,54 @@ public class ConversionStreamEventChunk extends ComplexEventChunk { private static final long serialVersionUID = 2754352338846132676L; private StreamEventConverter streamEventConverter; - private StreamEventPool streamEventPool; + private StreamEventFactory streamEventFactory; - public ConversionStreamEventChunk(MetaStreamEvent metaStreamEvent, StreamEventPool streamEventPool) { + public ConversionStreamEventChunk(MetaStreamEvent metaStreamEvent, StreamEventFactory streamEventFactory) { super(false); - this.streamEventPool = streamEventPool; + this.streamEventFactory = streamEventFactory; streamEventConverter = StreamEventConverterFactory.constructEventConverter(metaStreamEvent); } - public ConversionStreamEventChunk(StreamEventConverter streamEventConverter, StreamEventPool streamEventPool) { + public ConversionStreamEventChunk(StreamEventConverter streamEventConverter, + StreamEventFactory streamEventFactory) { super(false); this.streamEventConverter = streamEventConverter; - this.streamEventPool = streamEventPool; + this.streamEventFactory = streamEventFactory; } public void convertAndAssign(Event event) { - StreamEvent borrowedEvent = streamEventPool.borrowEvent(); - streamEventConverter.convertEvent(event, borrowedEvent); - first = borrowedEvent; + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertEvent(event, newEvent); + first = newEvent; last = first; } public void convertAndAssign(long timestamp, Object[] data) { - StreamEvent borrowedEvent = streamEventPool.borrowEvent(); - streamEventConverter.convertData(timestamp, data, borrowedEvent); - first = borrowedEvent; + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertData(timestamp, data, newEvent); + first = newEvent; last = first; } public void convertAndAssign(ComplexEvent complexEvent) { - first = streamEventPool.borrowEvent(); + first = streamEventFactory.newInstance(); last = convertAllStreamEvents(complexEvent, first); } // @Override // public void convertAndAssignFirst(StreamEvent streamEvent) { -// StreamEvent borrowedEvent = streamEventPool.borrowEvent(); -// eventConverter.convertComplexEvent(streamEvent, borrowedEvent); -// first = borrowedEvent; +// StreamEvent newEvent = streamEventFactory.borrowEvent(); +// eventConverter.convertComplexEvent(streamEvent, newEvent); +// first = newEvent; // last = first; // } public void convertAndAssign(Event[] events) { - StreamEvent firstEvent = streamEventPool.borrowEvent(); + StreamEvent firstEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(events[0], firstEvent); StreamEvent currentEvent = firstEvent; for (int i = 1, eventsLength = events.length; i < eventsLength; i++) { - StreamEvent nextEvent = streamEventPool.borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(events[i], nextEvent); currentEvent.setNext(nextEvent); currentEvent = nextEvent; @@ -88,15 +89,15 @@ public void convertAndAssign(Event[] events) { } public void convertAndAdd(Event event) { - StreamEvent borrowedEvent = streamEventPool.borrowEvent(); - streamEventConverter.convertEvent(event, borrowedEvent); + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertEvent(event, newEvent); if (first == null) { - first = borrowedEvent; + first = newEvent; last = first; } else { - last.setNext(borrowedEvent); - last = borrowedEvent; + last.setNext(newEvent); + last = newEvent; } } @@ -106,7 +107,7 @@ private StreamEvent convertAllStreamEvents(ComplexEvent complexEvents, StreamEve StreamEvent currentEvent = firstEvent; complexEvents = complexEvents.getNext(); while (complexEvents != null) { - StreamEvent nextEvent = streamEventPool.borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); streamEventConverter.convertComplexEvent(complexEvents, nextEvent); currentEvent.setNext(nextEvent); currentEvent = nextEvent; diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/FaultStreamEventConverter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/FaultStreamEventConverter.java index a566d4285b..efc0361d84 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/FaultStreamEventConverter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/FaultStreamEventConverter.java @@ -21,7 +21,7 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.Event; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import java.util.List; @@ -30,32 +30,32 @@ */ public class FaultStreamEventConverter { - private StreamEventPool streamEventPool; + private StreamEventFactory streamEventFactory; - public FaultStreamEventConverter(StreamEventPool streamEventPool) { - this.streamEventPool = streamEventPool; + public FaultStreamEventConverter(StreamEventFactory streamEventFactory) { + this.streamEventFactory = streamEventFactory; } public StreamEvent convert(Event event, Exception e) { - StreamEvent borrowedEvent = streamEventPool.borrowEvent(); - convertEvent(event, borrowedEvent, e); - return borrowedEvent; + StreamEvent newEvent = streamEventFactory.newInstance(); + convertEvent(event, newEvent, e); + return newEvent; } public StreamEvent convert(long timestamp, Object[] data, Exception e) { - StreamEvent borrowedEvent = streamEventPool.borrowEvent(); - convertData(timestamp, data, borrowedEvent, e); - return borrowedEvent; + StreamEvent newEvent = streamEventFactory.newInstance(); + convertData(timestamp, data, newEvent, e); + return newEvent; } public StreamEvent convert(ComplexEvent complexEvents, Exception e) { - StreamEvent firstEvent = streamEventPool.borrowEvent(); + StreamEvent firstEvent = streamEventFactory.newInstance(); convertComplexEvent(complexEvents, firstEvent, e); StreamEvent currentEvent = firstEvent; complexEvents = complexEvents.getNext(); while (complexEvents != null) { - StreamEvent nextEvent = streamEventPool.borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); convertComplexEvent(complexEvents, nextEvent, e); currentEvent.setNext(nextEvent); currentEvent = nextEvent; @@ -65,11 +65,11 @@ public StreamEvent convert(ComplexEvent complexEvents, Exception e) { } public StreamEvent convert(Event[] events, Exception e) { - StreamEvent firstEvent = streamEventPool.borrowEvent(); + StreamEvent firstEvent = streamEventFactory.newInstance(); convertEvent(events[0], firstEvent, e); StreamEvent currentEvent = firstEvent; for (int i = 1, eventsLength = events.length; i < eventsLength; i++) { - StreamEvent nextEvent = streamEventPool.borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); convertEvent(events[i], nextEvent, e); currentEvent.setNext(nextEvent); currentEvent = nextEvent; @@ -78,11 +78,11 @@ public StreamEvent convert(Event[] events, Exception e) { } public StreamEvent convert(List events, Exception e) { - StreamEvent firstEvent = streamEventPool.borrowEvent(); + StreamEvent firstEvent = streamEventFactory.newInstance(); convertEvent(events.get(0), firstEvent, e); StreamEvent currentEvent = firstEvent; for (int i = 1, eventsLength = events.size(); i < eventsLength; i++) { - StreamEvent nextEvent = streamEventPool.borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); convertEvent(events.get(i), nextEvent, e); currentEvent.setNext(nextEvent); currentEvent = nextEvent; @@ -90,28 +90,28 @@ public StreamEvent convert(List events, Exception e) { return firstEvent; } - private void convertData(long timestamp, Object[] data, StreamEvent.Type type, StreamEvent borrowedEvent, + private void convertData(long timestamp, Object[] data, StreamEvent.Type type, StreamEvent newEvent, Exception e) { - System.arraycopy(data, 0, borrowedEvent.getOutputData(), 0, data.length); - borrowedEvent.setOutputData(e, data.length); - borrowedEvent.setType(type); - borrowedEvent.setTimestamp(timestamp); + System.arraycopy(data, 0, newEvent.getOutputData(), 0, data.length); + newEvent.setOutputData(e, data.length); + newEvent.setType(type); + newEvent.setTimestamp(timestamp); } - private void convertEvent(Event event, StreamEvent borrowedEvent, Exception e) { + private void convertEvent(Event event, StreamEvent newEvent, Exception e) { convertData(event.getTimestamp(), event.getData(), event.isExpired() ? StreamEvent.Type.EXPIRED : StreamEvent.Type.CURRENT, - borrowedEvent, e); + newEvent, e); } - private void convertComplexEvent(ComplexEvent complexEvent, StreamEvent borrowedEvent, Exception e) { + private void convertComplexEvent(ComplexEvent complexEvent, StreamEvent newEvent, Exception e) { convertData(complexEvent.getTimestamp(), complexEvent.getOutputData(), complexEvent.getType(), - borrowedEvent, e); + newEvent, e); } - private void convertData(long timeStamp, Object[] data, StreamEvent borrowedEvent, Exception e) { - convertData(timeStamp, data, StreamEvent.Type.CURRENT, borrowedEvent, e); + private void convertData(long timeStamp, Object[] data, StreamEvent newEvent, Exception e) { + convertData(timeStamp, data, StreamEvent.Type.CURRENT, newEvent, e); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/SelectiveStreamEventConverter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/SelectiveStreamEventConverter.java index d9ef080390..15b5c91807 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/SelectiveStreamEventConverter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/SelectiveStreamEventConverter.java @@ -36,44 +36,44 @@ public SelectiveStreamEventConverter(List conversionMappings) this.conversionMappings = conversionMappings; } - public void convertData(long timestamp, Object[] data, StreamEvent.Type type, StreamEvent borrowedEvent) { + public void convertData(long timestamp, Object[] data, StreamEvent.Type type, StreamEvent newEvent) { for (ConversionMapping conversionMapping : conversionMappings) { int[] position = conversionMapping.getToPosition(); int fromPosition = conversionMapping.getFromPosition(); switch (position[0]) { case 0: - borrowedEvent.setBeforeWindowData(data[fromPosition], position[1]); + newEvent.setBeforeWindowData(data[fromPosition], position[1]); break; case 1: - borrowedEvent.setOnAfterWindowData(data[fromPosition], position[1]); + newEvent.setOnAfterWindowData(data[fromPosition], position[1]); break; case 2: - borrowedEvent.setOutputData(data[fromPosition], position[1]); + newEvent.setOutputData(data[fromPosition], position[1]); break; default: //can not happen } } - borrowedEvent.setType(type); - borrowedEvent.setTimestamp(timestamp); + newEvent.setType(type); + newEvent.setTimestamp(timestamp); } - public void convertEvent(Event event, StreamEvent borrowedEvent) { + public void convertEvent(Event event, StreamEvent newEvent) { convertData(event.getTimestamp(), event.getData(), event.isExpired() ? StreamEvent.Type.EXPIRED : StreamEvent .Type.CURRENT, - borrowedEvent); + newEvent); } - public void convertComplexEvent(ComplexEvent complexEvent, StreamEvent borrowedEvent) { + public void convertComplexEvent(ComplexEvent complexEvent, StreamEvent newEvent) { convertData(complexEvent.getTimestamp(), complexEvent.getOutputData(), complexEvent.getType(), - borrowedEvent); + newEvent); } @Override - public void convertData(long timeStamp, Object[] data, StreamEvent borrowedEvent) { - convertData(timeStamp, data, StreamEvent.Type.CURRENT, borrowedEvent); + public void convertData(long timeStamp, Object[] data, StreamEvent newEvent) { + convertData(timeStamp, data, StreamEvent.Type.CURRENT, newEvent); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/SimpleStreamEventConverter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/SimpleStreamEventConverter.java index 53fb782676..b3e46439d5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/SimpleStreamEventConverter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/SimpleStreamEventConverter.java @@ -35,28 +35,28 @@ public SimpleStreamEventConverter(List conversionMappings) { this.conversionMappings = conversionMappings; } - public void convertData(long timestamp, Object[] data, StreamEvent.Type type, StreamEvent borrowedEvent) { + public void convertData(long timestamp, Object[] data, StreamEvent.Type type, StreamEvent newEvent) { for (ConversionMapping element : conversionMappings) { - borrowedEvent.setOutputData(data[element.getFromPosition()], element.getToPosition()[1]); + newEvent.setOutputData(data[element.getFromPosition()], element.getToPosition()[1]); } - borrowedEvent.setType(type); - borrowedEvent.setTimestamp(timestamp); + newEvent.setType(type); + newEvent.setTimestamp(timestamp); } - public void convertEvent(Event event, StreamEvent borrowedEvent) { + public void convertEvent(Event event, StreamEvent newEvent) { convertData(event.getTimestamp(), event.getData(), event.isExpired() ? StreamEvent.Type.EXPIRED : StreamEvent .Type.CURRENT, - borrowedEvent); + newEvent); } - public void convertComplexEvent(ComplexEvent complexEvent, StreamEvent borrowedEvent) { + public void convertComplexEvent(ComplexEvent complexEvent, StreamEvent newEvent) { convertData(complexEvent.getTimestamp(), complexEvent.getOutputData(), complexEvent.getType(), - borrowedEvent); + newEvent); } @Override - public void convertData(long timestamp, Object[] data, StreamEvent borrowedEvent) { - convertData(timestamp, data, StreamEvent.Type.CURRENT, borrowedEvent); + public void convertData(long timestamp, Object[] data, StreamEvent newEvent) { + convertData(timestamp, data, StreamEvent.Type.CURRENT, newEvent); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/StreamEventConverter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/StreamEventConverter.java index 3089c4f5a5..4f600d5c99 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/StreamEventConverter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/StreamEventConverter.java @@ -31,37 +31,37 @@ public interface StreamEventConverter { /** * Method to construct StreamEvent form Event * - * @param event Event to be converted - * @param borrowedEvent Event that will be populated + * @param event Event to be converted + * @param newEvent Event that will be populated */ - void convertEvent(Event event, StreamEvent borrowedEvent); + void convertEvent(Event event, StreamEvent newEvent); /** * Method to construct(change format) new StreamEvent from StreamEvent * - * @param complexEvent StreamEvent to be Converted - * @param borrowedEvent Event that will be populated + * @param complexEvent StreamEvent to be Converted + * @param newEvent Event that will be populated */ - void convertComplexEvent(ComplexEvent complexEvent, StreamEvent borrowedEvent); + void convertComplexEvent(ComplexEvent complexEvent, StreamEvent newEvent); /** * Method to construct(change format) timeStamp and data from StreamEvent * - * @param timeStamp timeStamp of the event - * @param data output data of the event - * @param borrowedEvent Event that will be populated + * @param timeStamp timeStamp of the event + * @param data output data of the event + * @param newEvent Event that will be populated */ - void convertData(long timeStamp, Object[] data, StreamEvent borrowedEvent); + void convertData(long timeStamp, Object[] data, StreamEvent newEvent); /** * Method to construct(change format) timeStamp and data from StreamEvent * - * @param timeStamp timeStamp of the event - * @param data output data of the event - * @param type output type of the event - * @param borrowedEvent Event that will be populated + * @param timeStamp timeStamp of the event + * @param data output data of the event + * @param type output type of the event + * @param newEvent Event that will be populated */ - void convertData(long timeStamp, Object[] data, StreamEvent.Type type, StreamEvent borrowedEvent); + void convertData(long timeStamp, Object[] data, StreamEvent.Type type, StreamEvent newEvent); /** * Element to hold information about event conversion diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/ZeroStreamEventConverter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/ZeroStreamEventConverter.java index f9cc78eab9..9ff9281dd4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/ZeroStreamEventConverter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/converter/ZeroStreamEventConverter.java @@ -30,26 +30,25 @@ public class ZeroStreamEventConverter implements StreamEventConverter, Serializa private static final long serialVersionUID = 2223375413250580450L; - public void convertData(long timestamp, Object[] data, StreamEvent.Type type, StreamEvent borrowedEvent) { - System.arraycopy(data, 0, borrowedEvent.getOutputData(), 0, data.length); - borrowedEvent.setType(type); - borrowedEvent.setTimestamp(timestamp); + public void convertData(long timestamp, Object[] data, StreamEvent.Type type, StreamEvent newEvent) { + System.arraycopy(data, 0, newEvent.getOutputData(), 0, data.length); + newEvent.setType(type); + newEvent.setTimestamp(timestamp); } - public void convertEvent(Event event, StreamEvent borrowedEvent) { - convertData(event.getTimestamp(), event.getData(), event.isExpired() ? StreamEvent.Type.EXPIRED : StreamEvent - .Type.CURRENT, - borrowedEvent); + public void convertEvent(Event event, StreamEvent newEvent) { + convertData(event.getTimestamp(), event.getData(), event.isExpired() ? StreamEvent.Type.EXPIRED : + StreamEvent.Type.CURRENT, newEvent); } - public void convertComplexEvent(ComplexEvent complexEvent, StreamEvent borrowedEvent) { + public void convertComplexEvent(ComplexEvent complexEvent, StreamEvent newEvent) { convertData(complexEvent.getTimestamp(), complexEvent.getOutputData(), complexEvent.getType(), - borrowedEvent); + newEvent); } @Override - public void convertData(long timestamp, Object[] data, StreamEvent borrowedEvent) { - convertData(timestamp, data, StreamEvent.Type.CURRENT, borrowedEvent); + public void convertData(long timestamp, Object[] data, StreamEvent newEvent) { + convertData(timestamp, data, StreamEvent.Type.CURRENT, newEvent); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/holder/SnapshotableStreamEventQueue.java b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/holder/SnapshotableStreamEventQueue.java index 7da77adb9e..2c1e3a2fd5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/holder/SnapshotableStreamEventQueue.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/event/stream/holder/SnapshotableStreamEventQueue.java @@ -21,7 +21,7 @@ import io.siddhi.core.event.stream.Operation.Operator; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.util.snapshot.SnapshotRequest; -import io.siddhi.core.util.snapshot.state.SnapshotState; +import io.siddhi.core.util.snapshot.state.Snapshot; import io.siddhi.core.util.snapshot.state.SnapshotStateList; import java.io.Serializable; @@ -269,12 +269,12 @@ public int hashCode() { return result; } - public SnapshotState getSnapshot() { + public Snapshot getSnapshot() { if (isFullSnapshot()) { forceFullSnapshot = false; - return new SnapshotState(this.getFirst(), false); + return new Snapshot(this.getFirst(), false); } else { - SnapshotState snapshot = new SnapshotState(operationChangeLog, true); + Snapshot snapshot = new Snapshot(operationChangeLog, true); operationChangeLog = new ArrayList<>(); return snapshot; } @@ -287,11 +287,11 @@ private boolean isFullSnapshot() { } public void restore(SnapshotStateList snapshotStatelist) { - TreeMap revisions = snapshotStatelist.getSnapshotStates(); - Iterator> itr = revisions.entrySet().iterator(); + TreeMap revisions = snapshotStatelist.getSnapshotStates(); + Iterator> itr = revisions.entrySet().iterator(); this.isOperationLogEnabled = false; while (itr.hasNext()) { - Map.Entry snapshotEntry = itr.next(); + Map.Entry snapshotEntry = itr.next(); if (!snapshotEntry.getValue().isIncrementalSnapshot()) { this.clear(); this.add((StreamEvent) snapshotEntry.getValue().getState()); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/ConstantExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/ConstantExpressionExecutor.java index 71a5ec0d70..f9fa8ec278 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/ConstantExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/ConstantExpressionExecutor.java @@ -42,18 +42,8 @@ public Attribute.Type getReturnType() { return type; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return this; - } - public Object getValue() { return value; } - @Override - public void clean() { - //ignore - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/ExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/ExpressionExecutor.java index 8296288350..36b6cb9e62 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/ExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/ExpressionExecutor.java @@ -30,8 +30,4 @@ public interface ExpressionExecutor { Attribute.Type getReturnType(); - ExpressionExecutor cloneExecutor(String key); - - void clean(); - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/VariableExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/VariableExpressionExecutor.java index 629435c1a1..edb79a2322 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/VariableExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/VariableExpressionExecutor.java @@ -51,11 +51,6 @@ public Attribute.Type getReturnType() { return attribute.getType(); } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return this; - } - public Attribute getAttribute() { return attribute; } @@ -83,9 +78,4 @@ public void setPosition(int[] position) { } } - @Override - public void clean() { - //ignore - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/AndConditionExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/AndConditionExpressionExecutor.java index 51fb1b23da..9afcacc4af 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/AndConditionExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/AndConditionExpressionExecutor.java @@ -74,16 +74,4 @@ public Boolean execute(ComplexEvent event) { return Boolean.FALSE; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new AndConditionExpressionExecutor(leftConditionExecutor.cloneExecutor(key), rightConditionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftConditionExecutor.clean(); - rightConditionExecutor.clean(); - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/BoolConditionExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/BoolConditionExpressionExecutor.java index 512ad7ec0d..437e2a0869 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/BoolConditionExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/BoolConditionExpressionExecutor.java @@ -49,14 +49,4 @@ public Boolean execute(ComplexEvent event) { } } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new BoolConditionExpressionExecutor(conditionExecutor.cloneExecutor(key)); - } - - @Override - public void clean() { - conditionExecutor.clean(); - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/InConditionExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/InConditionExpressionExecutor.java index e26106f9d3..fddfa3dff4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/InConditionExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/InConditionExpressionExecutor.java @@ -21,7 +21,6 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.table.Table; import io.siddhi.core.util.collection.FinderStateEvent; import io.siddhi.core.util.collection.operator.CompiledCondition; @@ -36,19 +35,18 @@ public class InConditionExpressionExecutor extends ConditionExpressionExecutor { private final int matchingStreamIndex; private final CompiledCondition compiledCondition; private Table table; - private FinderStateEvent finderStateEvent; public InConditionExpressionExecutor(Table table, CompiledCondition compiledCondition, int streamEventSize, boolean isMatchingEventAStateEvent, int matchingStreamIndex) { this.streamEventSize = streamEventSize; this.isMatchingEventAStateEvent = isMatchingEventAStateEvent; this.matchingStreamIndex = matchingStreamIndex; - this.finderStateEvent = new FinderStateEvent(streamEventSize, 0); this.table = table; this.compiledCondition = compiledCondition; } - public synchronized Boolean execute(ComplexEvent event) { + public Boolean execute(ComplexEvent event) { + FinderStateEvent finderStateEvent = new FinderStateEvent(streamEventSize, 0); try { if (isMatchingEventAStateEvent) { finderStateEvent.setEvent((StateEvent) event); @@ -65,16 +63,4 @@ public synchronized Boolean execute(ComplexEvent event) { } } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new InConditionExpressionExecutor(table, compiledCondition.cloneCompilation(key), - streamEventSize, isMatchingEventAStateEvent, matchingStreamIndex); - } - - @Override - public void clean() { - //ignore - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/IsNullConditionExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/IsNullConditionExpressionExecutor.java index b81356f068..edbf2dcb76 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/IsNullConditionExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/IsNullConditionExpressionExecutor.java @@ -41,14 +41,4 @@ public Boolean execute(ComplexEvent event) { } } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new IsNullConditionExpressionExecutor(expressionExecutor.cloneExecutor(key)); - } - - @Override - public void clean() { - expressionExecutor.clean(); - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/IsNullStreamConditionExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/IsNullStreamConditionExpressionExecutor.java index 80b6374d8e..b3a2506982 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/IsNullStreamConditionExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/IsNullStreamConditionExpressionExecutor.java @@ -21,7 +21,6 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.executor.ExpressionExecutor; /** * Executor class for IsNullStream condition. Condition evaluation logic is implemented within executor. @@ -51,14 +50,4 @@ public Boolean execute(ComplexEvent event) { } } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new IsNullStreamConditionExpressionExecutor(eventPosition); - } - - @Override - public void clean() { - //ignore - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/NotConditionExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/NotConditionExpressionExecutor.java index 4057e30f8c..e0e36637b9 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/NotConditionExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/NotConditionExpressionExecutor.java @@ -49,14 +49,4 @@ public Boolean execute(ComplexEvent event) { } } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotConditionExpressionExecutor(conditionExecutor.cloneExecutor(key)); - } - - @Override - public void clean() { - conditionExecutor.clean(); - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/OrConditionExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/OrConditionExpressionExecutor.java index 828ddd2a17..705be9cd04 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/OrConditionExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/OrConditionExpressionExecutor.java @@ -75,16 +75,4 @@ public Boolean execute(ComplexEvent event) { return Boolean.FALSE; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new OrConditionExpressionExecutor(leftConditionExecutor.cloneExecutor(key), rightConditionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftConditionExecutor.clean(); - rightConditionExecutor.clean(); - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/CompareConditionExpressionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/CompareConditionExpressionExecutor.java index 76fe863607..198784fad1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/CompareConditionExpressionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/CompareConditionExpressionExecutor.java @@ -44,9 +44,4 @@ public Boolean execute(ComplexEvent event) { protected abstract Boolean execute(Object left, Object right); - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorBoolBool.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorBoolBool.java index 5a4aafafc4..9d606d410a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorBoolBool.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorBoolBool.java @@ -35,9 +35,4 @@ protected Boolean execute(Object left, Object right) { return ((Boolean) left).booleanValue() == ((Boolean) right).booleanValue(); } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorBoolBool(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleDouble.java index ce891e84e0..3953ae9299 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorDoubleDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleFloat.java index 1748e5281f..0856c00479 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleFloat.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorDoubleFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleInt.java index 0d4deb7d26..cfc227dd69 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleInt.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Double) left == ((Integer) right).doubleValue(); } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorDoubleInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleLong.java index caf5bd0514..2d83db15e8 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorDoubleLong.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Double) left == ((Long) right).doubleValue(); } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorDoubleLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatDouble.java index 1ed7191c2e..b6fb36b292 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatDouble.java @@ -37,10 +37,4 @@ protected Boolean execute(Object left, Object right) { return ((Float) left).doubleValue() == (Double) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorFloatDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatFloat.java index f71f33cb44..7f4331d5b3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatFloat.java @@ -36,9 +36,4 @@ protected Boolean execute(Object left, Object right) { return ((Float) left).floatValue() == ((Float) right).floatValue(); } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorFloatFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatInt.java index 288f833bc5..3c94a85d36 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatInt.java @@ -37,10 +37,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left == ((Integer) right).floatValue(); } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorFloatInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatLong.java index 053f4cfe7d..21b98d6c63 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorFloatLong.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return ((Float) left).doubleValue() == ((Long) right).doubleValue(); } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorFloatLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntDouble.java index b887f6d159..92248a5f0e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntDouble.java @@ -37,10 +37,4 @@ protected Boolean execute(Object left, Object right) { return ((Integer) left).doubleValue() == (Double) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorIntDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntFloat.java index b0739e5d7e..e59b26162b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntFloat.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return ((Integer) left).floatValue() == (Float) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorIntFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntInt.java index fea843feaa..1db67071b2 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntInt.java @@ -36,10 +36,4 @@ protected Boolean execute(Object left, Object right) { return ((Integer) left).intValue() == (Integer) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorIntInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntLong.java index b15a388a7c..f28eb4cfac 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorIntLong.java @@ -36,12 +36,5 @@ public EqualCompareConditionExpressionExecutorIntLong( @Override protected Boolean execute(Object left, Object right) { return ((Integer) left).longValue() == (Long) right; - - } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorIntLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongDouble.java index de96eb8f82..c346777b51 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongDouble.java @@ -37,10 +37,4 @@ protected Boolean execute(Object left, Object right) { return ((Long) left).doubleValue() == (Double) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorLongDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongFloat.java index 37ba30aa3c..facae9c84b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongFloat.java @@ -35,12 +35,5 @@ public EqualCompareConditionExpressionExecutorLongFloat( @Override protected Boolean execute(Object left, Object right) { return ((Long) left).doubleValue() == ((Float) right).doubleValue(); - - } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorLongFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongInt.java index c9a182668b..bc06b16fcb 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongInt.java @@ -35,12 +35,5 @@ public EqualCompareConditionExpressionExecutorLongInt( @Override protected Boolean execute(Object left, Object right) { return (Long) left == ((Integer) right).longValue(); - - } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorLongInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongLong.java index 95678a8d4e..b5f9d82bbc 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorLongLong.java @@ -35,12 +35,5 @@ public EqualCompareConditionExpressionExecutorLongLong( @Override protected Boolean execute(Object left, Object right) { return ((Long) left).longValue() == (Long) right; - - } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorLongLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorStringString.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorStringString.java index 1a9df95c35..e62eb8694e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorStringString.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/equal/EqualCompareConditionExpressionExecutorStringString.java @@ -33,10 +33,4 @@ public EqualCompareConditionExpressionExecutorStringString(ExpressionExecutor le protected Boolean execute(Object left, Object right) { return left.equals(right); } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new EqualCompareConditionExpressionExecutorStringString(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleDouble.java index f882adb887..aa6c5c994a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleDouble.java @@ -35,12 +35,5 @@ public GreaterThanCompareConditionExpressionExecutorDoubleDouble( @Override protected Boolean execute(Object left, Object right) { return (Double) left > (Double) right; - - } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorDoubleDouble(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleFloat.java index 5b2d2a511e..6812e07d22 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleFloat.java @@ -34,12 +34,5 @@ public GreaterThanCompareConditionExpressionExecutorDoubleFloat(ExpressionExecut @Override protected Boolean execute(Object left, Object right) { return (Double) left > (Float) right; - - } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorDoubleFloat(leftExpressionExecutor.cloneExecutor(key) - , rightExpressionExecutor.cloneExecutor(key)); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleInt.java index a89c7161f8..161d73386a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleInt.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorDoubleInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleLong.java index 8833349a1e..3e40bd8f75 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorDoubleLong.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorDoubleLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatDouble.java index cf3dc5acc7..44555d2bc3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatDouble.java @@ -37,10 +37,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left > (Double) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorFloatDouble(leftExpressionExecutor.cloneExecutor(key) - , rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatFloat.java index 1626d6ecf5..bdef8190e9 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatFloat.java @@ -36,9 +36,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left > (Float) right; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorFloatFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatInt.java index 9b9ecfad3c..3a18f9cd84 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatInt.java @@ -35,12 +35,6 @@ public GreaterThanCompareConditionExpressionExecutorFloatInt( @Override protected Boolean execute(Object left, Object right) { return (Float) left > (Integer) right; - } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorFloatInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatLong.java index c45e5715d6..f68220a530 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorFloatLong.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left > (Long) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorFloatLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntDouble.java index 3b3b2f25d7..829cc575c6 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntDouble.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorIntDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntFloat.java index b63addf0e7..78c8f23e2c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntFloat.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorIntFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntInt.java index a1fbf7b752..6ed334021d 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntInt.java @@ -36,9 +36,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorIntInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntLong.java index f7e762cf79..73995c1bdb 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorIntLong.java @@ -37,10 +37,4 @@ protected Boolean execute(Object left, Object right) { return (Integer) left > (Long) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorIntLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongDouble.java index af0c05308f..ae9c68c9a5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongDouble.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorLongDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongFloat.java index 53680f7f58..fde875b74e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongFloat.java @@ -36,10 +36,4 @@ protected Boolean execute(Object left, Object right) { return (Long) left > (Float) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorLongFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongInt.java index 9ff28f680f..fd6f2a617f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongInt.java @@ -36,10 +36,4 @@ protected Boolean execute(Object left, Object right) { return (Long) left > (Integer) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorLongInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongLong.java index 253133762b..0bb71c53cf 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthan/GreaterThanCompareConditionExpressionExecutorLongLong.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanCompareConditionExpressionExecutorLongLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleDouble.java index 5ad830f685..4f65f11bdb 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleDouble.java @@ -39,10 +39,4 @@ protected Boolean execute(Object left, Object right) { } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorDoubleDouble(leftExpressionExecutor - .cloneExecutor(key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleFloat.java index 8c0f5d3b29..eb2cf39da2 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleFloat.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorDoubleFloat(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleInt.java index 1e9fd82b0b..47e32360ba 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleInt.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorDoubleInt(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleLong.java index 32084bf72b..02eb30283b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorDoubleLong.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorDoubleLong(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatDouble.java index d2294288bd..c771cd657e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatDouble.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left >= (Double) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorFloatDouble(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatFloat.java index da0c82f022..a19fe71f97 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatFloat.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left >= (Float) right; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorFloatFloat(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatInt.java index bbb6ea9871..26fd7d4d10 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatInt.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorFloatInt(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatLong.java index e6021e4f0a..4efbdd21b1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorFloatLong.java @@ -39,10 +39,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left >= (Long) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorFloatLong(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntDouble.java index a70f7255ae..9dc9961c05 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorIntDouble(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntFloat.java index ce7dc3f07d..476d04918c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntFloat.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Integer) left >= (Float) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorIntFloat(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntInt.java index c55043f2ed..3aaa659814 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntInt.java @@ -37,10 +37,4 @@ protected Boolean execute(Object left, Object right) { return (Integer) left >= (Integer) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorIntInt(leftExpressionExecutor.cloneExecutor(key) - , rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntLong.java index 90bfed8f15..baa603dbc4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorIntLong.java @@ -40,9 +40,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorIntLong(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongDouble.java index 1159c5fee7..24c1d182d0 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorLongDouble(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongFloat.java index 8325183483..f452f12a1a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongFloat.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorLongFloat(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongInt.java index 76836a7167..5c26b5cdd8 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongInt.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorLongInt(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongLong.java index d468a388f8..21c42bc672 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/greaterthanequal/GreaterThanEqualCompareConditionExpressionExecutorLongLong.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new GreaterThanEqualCompareConditionExpressionExecutorLongLong(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleDouble.java index ad0623142f..66899ad40f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorDoubleDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleFloat.java index e96cd3587d..76e7c0deaa 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleFloat.java @@ -38,9 +38,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorDoubleFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleInt.java index 1ab1777e78..a38eb58770 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleInt.java @@ -38,9 +38,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorDoubleInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleLong.java index 3a5fa4d974..d32e21cc6f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorDoubleLong.java @@ -38,9 +38,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorDoubleLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatDouble.java index 6be4fb224e..907867f445 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorFloatDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatFloat.java index 11b5567592..6da22ffd20 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatFloat.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left < (Float) right; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorFloatFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatInt.java index 3d49eb0d13..7e7bfe1319 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatInt.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left < (Integer) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorFloatInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatLong.java index 53dbbbd489..e687bd2183 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorFloatLong.java @@ -40,9 +40,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorFloatLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntDouble.java index dff0b10e23..0bc30b5abb 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntDouble.java @@ -37,10 +37,4 @@ protected Boolean execute(Object left, Object right) { return (Integer) left < (Double) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorIntDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntFloat.java index aecb5e63ca..d539e65db5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntFloat.java @@ -38,9 +38,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorIntFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntInt.java index 647db45296..76b3df407a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntInt.java @@ -36,10 +36,4 @@ protected Boolean execute(Object left, Object right) { return (Integer) left < (Integer) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorIntInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntLong.java index 8561606c5f..9613ea9d66 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorIntLong.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorIntLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongDouble.java index e72e341098..3a26483c3d 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongDouble.java @@ -38,9 +38,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorLongDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongFloat.java index cbab2302bf..9eefcaedb1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongFloat.java @@ -38,9 +38,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorLongFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongInt.java index d3246f88a1..0744433393 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongInt.java @@ -38,9 +38,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorLongInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongLong.java index ec9b546152..f190720cd7 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthan/LessThanCompareConditionExpressionExecutorLongLong.java @@ -38,9 +38,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanCompareConditionExpressionExecutorLongLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleDouble.java index 2f2f44decf..d0a217b2f5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorDoubleDouble(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleFloat.java index 512594e3ef..4a5d9b8911 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleFloat.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorDoubleFloat(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleInt.java index 21e51e01d7..ca36eb635a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleInt.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Double) left <= (Integer) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorDoubleInt(leftExpressionExecutor.cloneExecutor(key) - , rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleLong.java index 55e5ddc598..02de1c77e3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorDoubleLong.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorDoubleLong(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatDouble.java index 0eeab083e4..945208a06c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatDouble.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left <= (Double) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorFloatDouble(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatFloat.java index 784dbc4a70..daa7132422 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatFloat.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left <= (Float) right; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorFloatFloat(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatInt.java index 683b58ab8d..a780eaa5fa 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatInt.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left <= (Integer) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorFloatInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatLong.java index 507ae84353..9bec1286d5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorFloatLong.java @@ -39,10 +39,4 @@ protected Boolean execute(Object left, Object right) { return (Float) left <= (Long) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorFloatLong(leftExpressionExecutor.cloneExecutor(key) - , rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntDouble.java index 458f36d644..9c3674e027 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorIntDouble(leftExpressionExecutor.cloneExecutor(key) - , rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntFloat.java index 1b8696b96e..ba67945b22 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntFloat.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorIntFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntInt.java index a18b136633..52e0ac38fe 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntInt.java @@ -37,10 +37,4 @@ protected Boolean execute(Object left, Object right) { return (Integer) left <= (Integer) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorIntInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntLong.java index b2b47f4885..bcf279c829 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorIntLong.java @@ -40,9 +40,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorIntLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongDouble.java index 80c2a221e3..ce41db7db7 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorLongDouble(leftExpressionExecutor.cloneExecutor - (key), rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongFloat.java index c970ac2518..8a7ce4b9b5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongFloat.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorLongFloat(leftExpressionExecutor.cloneExecutor(key) - , rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongInt.java index ba7e0a48f9..90f02e6f2f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongInt.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Long) left <= (Integer) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorLongInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongLong.java index 45f94a0a21..8f429972d6 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/lessthanequal/LessThanEqualCompareConditionExpressionExecutorLongLong.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return (Long) left <= (Long) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new LessThanEqualCompareConditionExpressionExecutorLongLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorBoolBool.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorBoolBool.java index 4c4b58bf41..9570e875a3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorBoolBool.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorBoolBool.java @@ -35,9 +35,4 @@ protected Boolean execute(Object left, Object right) { return ((Boolean) left).booleanValue() != ((Boolean) right).booleanValue(); } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorBoolBool(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleDouble.java index 0cbef5b160..c769cf26d4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleDouble.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return ((Double) left).doubleValue() != ((Double) right).doubleValue(); } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorDoubleDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleFloat.java index 4df7dbc531..ceea82cf4a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleFloat.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorDoubleFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleInt.java index a75a254f4c..1f978ecb8c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleInt.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorDoubleInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleLong.java index 4a9910cbe9..d4119b269e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorDoubleLong.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorDoubleLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatDouble.java index b5b8ff6970..6a0f370b90 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorFloatDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatFloat.java index a8b912a61a..6d23e6ea19 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatFloat.java @@ -37,9 +37,4 @@ protected Boolean execute(Object left, Object right) { return ((Float) left).floatValue() != ((Float) right).floatValue(); } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorFloatFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatInt.java index 2ec08f0853..36fba05204 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatInt.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorFloatInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatLong.java index 14c727ca4c..707f3ac08b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorFloatLong.java @@ -40,9 +40,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorFloatLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntDouble.java index 168a0d703f..1c2f71aee9 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorIntDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntFloat.java index ba33093399..8ec94498e9 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntFloat.java @@ -38,10 +38,4 @@ protected Boolean execute(Object left, Object right) { return ((Integer) left).floatValue() != (Float) right; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorIntFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntInt.java index b691819642..6a5b179bc3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntInt.java @@ -38,9 +38,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorIntInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntLong.java index c2040eb207..8dfd2751c2 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorIntLong.java @@ -40,9 +40,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorIntLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongDouble.java index f3355a7129..d936fa75ab 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongDouble.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorLongDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongFloat.java index aca53ab49e..87be1eac9e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongFloat.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorLongFloat(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongInt.java index 45cd7ae86b..a9f9c0c4ad 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongInt.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorLongInt(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongLong.java index a630fb51da..efb5c6317b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorLongLong.java @@ -39,9 +39,4 @@ protected Boolean execute(Object left, Object right) { } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorLongLong(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorStringString.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorStringString.java index bf7438d569..e15f95896a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorStringString.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/condition/compare/notequal/NotEqualCompareConditionExpressionExecutorStringString.java @@ -35,9 +35,4 @@ protected Boolean execute(Object left, Object right) { return !(left.equals(right)); } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new NotEqualCompareConditionExpressionExecutorStringString(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CastFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CastFunctionExecutor.java index 2a72c78522..aa0ed69ff1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CastFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CastFunctionExecutor.java @@ -27,11 +27,11 @@ import io.siddhi.core.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for Siddhi cast function. Converts the given parameter according to the castTo parameter. * Incompatible arguments cause {@link ClassCastException} if further processed. @@ -69,8 +69,8 @@ public class CastFunctionExecutor extends FunctionExecutor { private Attribute.Type returnType = Attribute.Type.OBJECT; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 2) { throw new SiddhiAppValidationException("Invalid no of arguments passed to common:cast() function, " + "required 2 parameters, but found " + @@ -99,10 +99,11 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea "string"); } } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { if (returnType == Attribute.Type.LONG && data[0] instanceof Integer) { return ((Integer) data[0]).longValue(); } @@ -110,7 +111,7 @@ protected Object execute(Object[] data) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return null; } @@ -119,15 +120,6 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; //No need to maintain a state. - } - - @Override - public void restoreState(Map state) { - //Since there's no need to maintain a state, nothing needs to be done here. - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CoalesceFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CoalesceFunctionExecutor.java index 06720c5bda..c709da0dc2 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CoalesceFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CoalesceFunctionExecutor.java @@ -25,11 +25,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for coalesce function. Returns the value of the first input parameter that is not null. */ @@ -73,8 +73,8 @@ public class CoalesceFunctionExecutor extends FunctionExecutor { private Attribute.Type returnType; @Override - public void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + public StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length == 0) { throw new SiddhiAppValidationException("Coalesce must have at least one parameter"); } @@ -85,6 +85,7 @@ public void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader } } returnType = type; + return null; } @Override @@ -93,7 +94,7 @@ public Attribute.Type getReturnType() { } - protected Object execute(Object[] obj) { + protected Object execute(Object[] obj, State state) { for (Object aObj : obj) { if (aObj != null) { return aObj; @@ -103,18 +104,7 @@ protected Object execute(Object[] obj) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data; } - - @Override - public Map currentState() { - //No states - return null; - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/ConvertFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/ConvertFunctionExecutor.java index c556bb2bf8..a5198d7c13 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/ConvertFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/ConvertFunctionExecutor.java @@ -26,11 +26,11 @@ import io.siddhi.core.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for convert function. Function execution logic is implemented in execute here. */ @@ -73,8 +73,8 @@ public class ConvertFunctionExecutor extends FunctionExecutor { private Attribute.Type inputType; @Override - public void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + public StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 2) { throw new SiddhiAppValidationException("convert() must have at 2 parameters, attribute and to be " + "converted type"); @@ -115,6 +115,7 @@ public void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader "(STRING, INT, LONG, FLOAT, DOUBLE, BOOL), but found '" + type + "'"); } + return null; } @Override @@ -123,7 +124,7 @@ public Attribute.Type getReturnType() { } - protected Object execute(Object[] obj) { + protected Object execute(Object[] obj, State state) { Object data = obj[0]; if (data != null) { try { @@ -231,19 +232,8 @@ protected Object execute(Object[] obj) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { //will not occur return null; } - - @Override - public Map currentState() { - //No states - return null; - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CreateSetFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CreateSetFunctionExecutor.java index b67deffe98..2d5d7ad3fe 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CreateSetFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CreateSetFunctionExecutor.java @@ -27,11 +27,12 @@ import io.siddhi.core.exception.OperationNotSupportedException; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; import java.util.HashSet; -import java.util.Map; import java.util.Set; /** @@ -61,8 +62,8 @@ public class CreateSetFunctionExecutor extends FunctionExecutor { @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("createSet() function has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length + " parameters provided"); @@ -71,10 +72,11 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea throw new OperationNotSupportedException("createSet() function not supported for type: " + attributeExpressionExecutors[0].getReturnType()); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the createSet function takes in only 1 parameter, this method does not get called. // Hence, not implemented. } @@ -83,10 +85,11 @@ protected Object execute(Object[] data) { * return set object, containing only one element: data. * * @param data array of Double values + * @param state * @return the set object */ @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { Set set = new HashSet(); set.add(data); return set; @@ -97,16 +100,6 @@ public Attribute.Type getReturnType() { return Attribute.Type.OBJECT; } - @Override - public Map currentState() { - return null; //no state is maintained. - } - - @Override - public void restoreState(Map state) { - //Does nothing as no state is maintained. - } - private boolean isAttributeTypeSupported(Attribute.Type type) { switch (type) { case FLOAT: diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CurrentTimeMillisFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CurrentTimeMillisFunctionExecutor.java index cf7bbd9e26..f08476181b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CurrentTimeMillisFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/CurrentTimeMillisFunctionExecutor.java @@ -26,12 +26,12 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.executor.ExpressionExecutor; 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.core.util.timestamp.TimestampGenerator; import io.siddhi.query.api.definition.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for getting Siddhi application timestamp. */ @@ -56,14 +56,15 @@ public class CurrentTimeMillisFunctionExecutor extends FunctionExecutor { private TimestampGenerator timestampGenerator; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + 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); } timestampGenerator = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator(); + return null; } @Override @@ -72,13 +73,13 @@ public Object execute(ComplexEvent event) { } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { //will not occur return null; } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { //will not occur return null; } @@ -87,16 +88,6 @@ protected Object execute(Object data) { public Attribute.Type getReturnType() { return Attribute.Type.LONG; } - - @Override - public Map currentState() { - return null; //No need to maintain a state. - } - - @Override - public void restoreState(Map state) { - //Since there's no need to maintain a state, nothing needs to be done here. - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/DefaultFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/DefaultFunctionExecutor.java index 1bca93f953..baf6b21b90 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/DefaultFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/DefaultFunctionExecutor.java @@ -26,12 +26,12 @@ import io.siddhi.core.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; import org.apache.log4j.Logger; -import java.util.Map; - /** * Executor class for default function. Function execution sets the default value assigned by the user when attribute * value is null. @@ -66,8 +66,8 @@ public class DefaultFunctionExecutor extends FunctionExecutor { Attribute.Type returnType; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 2) { // check whether all the arguments passed throw new SiddhiAppValidationException("Invalid no of parameters passed to default() function, " + @@ -87,10 +87,11 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, attributeExpressionExecutors[1].getReturnType()); } returnType = attributeExpressionExecutors[0].getReturnType(); + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { if (data[0] == null) { return data[1]; } else { @@ -99,7 +100,7 @@ protected Object execute(Object[] data) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { //this will not occur return null; } @@ -109,14 +110,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; // No need to maintain a state. - } - - @Override - public void restoreState(Map state) { - // Since there's no need to maintain a state, nothing needs to be done here. - } - } 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 80239c47ae..95c8cd8cf4 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 @@ -26,11 +26,11 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for Siddhi cast function. Converts the given parameter according to the castTo parameter. * Incompatible arguments cause {@link ClassCastException} if further processed. @@ -54,13 +54,14 @@ public class EventTimestampFunctionExecutor extends FunctionExecutor { @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + 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); } + return null; } @Override @@ -69,13 +70,13 @@ public Object execute(ComplexEvent event) { } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { //will not occur return null; } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { //will not occur return null; } @@ -85,15 +86,6 @@ public Attribute.Type getReturnType() { return Attribute.Type.LONG; } - @Override - public Map currentState() { - return null; //No need to maintain a state. - } - - @Override - public void restoreState(Map state) { - //Since there's no need to maintain a state, nothing needs to be done here. - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/FunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/FunctionExecutor.java index 086a43e703..1d8a8a9d46 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/FunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/FunctionExecutor.java @@ -24,63 +24,43 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.util.config.ConfigReader; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.core.util.snapshot.state.StateHolder; import org.apache.log4j.Logger; /** * Parent abstract class for Function Executors. Function executor will have one or more input parameters and single * return value. + * @param current state for the Function Executor */ -public abstract class FunctionExecutor implements ExpressionExecutor, Snapshotable { +public abstract class FunctionExecutor implements ExpressionExecutor { private static final Logger log = Logger.getLogger(FunctionExecutor.class); protected ExpressionExecutor[] attributeExpressionExecutors; protected SiddhiQueryContext siddhiQueryContext; - protected String elementId; protected String functionId; protected ProcessingMode processingMode; private ConfigReader configReader; private int attributeSize; + private StateHolder stateHolder; public void initExecutor(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + boolean groupBy, SiddhiQueryContext siddhiQueryContext) { this.processingMode = processingMode; this.configReader = configReader; this.siddhiQueryContext = siddhiQueryContext; try { this.attributeExpressionExecutors = attributeExpressionExecutors; attributeSize = attributeExpressionExecutors.length; - if (elementId == null) { - elementId = "FunctionExecutor-" + this.siddhiQueryContext.getSiddhiAppContext(). - getElementIdGenerator().createNewId(); - } - this.siddhiQueryContext.getSiddhiAppContext().getSnapshotService().addSnapshotable( - siddhiQueryContext.getName(), this); - init(attributeExpressionExecutors, configReader, this.siddhiQueryContext); + StateFactory stateFactory = init(attributeExpressionExecutors, configReader, this.siddhiQueryContext); + stateHolder = this.siddhiQueryContext.generateStateHolder(this.getClass().getName(), groupBy, stateFactory); } catch (Throwable t) { throw new SiddhiAppCreationException(t); } } - @Override - public ExpressionExecutor cloneExecutor(String key) { - try { - FunctionExecutor functionExecutor = this.getClass().newInstance(); - ExpressionExecutor[] innerExpressionExecutors = new ExpressionExecutor[attributeSize]; - for (int i = 0; i < attributeSize; i++) { - innerExpressionExecutors[i] = attributeExpressionExecutors[i].cloneExecutor(key); - } - functionExecutor.elementId = elementId + "-" + key; - functionExecutor.functionId = functionId; - functionExecutor.initExecutor(innerExpressionExecutors, processingMode, - configReader, siddhiQueryContext); - return functionExecutor; - } catch (Exception e) { - throw new SiddhiAppRuntimeException("Exception in cloning " + this.getClass().getCanonicalName(), e); - } - } - /** * The initialization method for FunctionExecutor, this method will be called before the other methods * @@ -88,8 +68,9 @@ public ExpressionExecutor cloneExecutor(String key) { * @param configReader This hold the {@link FunctionExecutor} extensions configuration reader. * @param siddhiQueryContext the context of the siddhi query */ - protected abstract void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext); + protected abstract StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext); /** * The main execution method which will be called upon event arrival @@ -100,17 +81,22 @@ protected abstract void init(ExpressionExecutor[] attributeExpressionExecutors, @Override public Object execute(ComplexEvent event) { try { - switch (attributeSize) { - case 0: - return execute((Object) null); - case 1: - return execute(attributeExpressionExecutors[0].execute(event)); - default: - Object[] data = new Object[attributeSize]; - for (int i = 0; i < attributeSize; i++) { - data[i] = attributeExpressionExecutors[i].execute(event); - } - return execute(data); + S state = stateHolder.getState(); + try { + switch (attributeSize) { + case 0: + return execute((Object) null, state); + case 1: + return execute(attributeExpressionExecutors[0].execute(event), state); + default: + Object[] data = new Object[attributeSize]; + for (int i = 0; i < attributeSize; i++) { + data[i] = attributeExpressionExecutors[i].execute(event); + } + return execute(data, state); + } + } finally { + stateHolder.returnState(state); } } catch (Exception e) { throw new SiddhiAppRuntimeException(e.getMessage() + ". Exception on class '" + this.getClass().getName() @@ -125,7 +111,14 @@ public Object execute(ComplexEvent event) { * @param data the runtime values of function parameters * @return the function result */ - protected abstract Object execute(Object[] data); + protected Object execute(Object[] data) { + S state = stateHolder.getState(); + try { + return execute(data, state); + } finally { + stateHolder.returnState(state); + } + } /** * The main execution method which will be called upon event arrival @@ -135,19 +128,33 @@ public Object execute(ComplexEvent event) { * runtime data value of the function parameter * @return the function result */ - protected abstract Object execute(Object data); - - @Override - public String getElementId() { - return elementId; - } - - @Override - public void clean() { - for (ExpressionExecutor expressionExecutor : attributeExpressionExecutors) { - expressionExecutor.clean(); + protected Object execute(Object data) { + S state = stateHolder.getState(); + try { + return execute(data, state); + } finally { + stateHolder.returnState(state); } - siddhiQueryContext.getSiddhiAppContext().getSnapshotService().removeSnapshotable( - siddhiQueryContext.getName(), this); } + + /** + * The main execution method which will be called upon event arrival + * when there are more then one function parameter + * + * @param data the runtime values of function parameters + * @param state current query state + * @return the function result + */ + protected abstract Object execute(Object[] data, S state); + + /** + * The main execution method which will be called upon event arrival + * when there are zero or one function parameter + * + * @param data null if the function parameter count is zero or + * runtime data value of the function parameter + * @param state current query state + * @return the function result + */ + protected abstract Object execute(Object data, S state); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/IfThenElseFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/IfThenElseFunctionExecutor.java index f03fb9410a..306b72ca9a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/IfThenElseFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/IfThenElseFunctionExecutor.java @@ -27,12 +27,12 @@ import io.siddhi.core.exception.SiddhiAppRuntimeException; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; import org.apache.log4j.Logger; -import java.util.Map; - /** * Executor class for ifThenElse function. Function execution logic is implemented in execute here. */ @@ -90,8 +90,8 @@ public class IfThenElseFunctionExecutor extends FunctionExecutor { Attribute.Type returnType; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 3) { // check whether all the arguments passed throw new SiddhiAppValidationException("Invalid no of arguments passed to ifThenElse() function, " + @@ -110,10 +110,11 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, } else { returnType = attributeExpressionExecutors[1].getReturnType(); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { // check whether first argument true or null if (Boolean.TRUE.equals(data[0])) { return data[1]; @@ -123,7 +124,7 @@ protected Object execute(Object[] data) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { // Since the e function takes in multiple parameters, this method does not get called. Hence, not implemented. return null; } @@ -133,16 +134,6 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; // No need to maintain a state. - } - - @Override - public void restoreState(Map state) { - // Since there's no need to maintain a state, nothing needs to be done here. - } - @Override public Object execute(ComplexEvent event) { try { @@ -152,8 +143,7 @@ public Object execute(ComplexEvent event) { condition, (condition) ? attributeExpressionExecutors[1].execute(event) : null, (!condition) ? attributeExpressionExecutors[2].execute(event) : null - } - ); + }); } catch (Exception e) { throw new SiddhiAppRuntimeException(e.getMessage() + ". Exception on class '" + this.getClass().getName() + "'", e); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfBooleanFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfBooleanFunctionExecutor.java index 5537f1d613..0ff4f40ae7 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfBooleanFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfBooleanFunctionExecutor.java @@ -26,11 +26,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for instanceOf Boolean function. Function execution logic is implemented in execute here. */ @@ -69,23 +69,24 @@ public class InstanceOfBooleanFunctionExecutor extends FunctionExecutor { Attribute.Type returnType = Attribute.Type.BOOL; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("Invalid no of arguments passed to instanceOfBoolean() " + "function, " + "required only 1, but found " + attributeExpressionExecutors.length); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the instanceOfBoolean function takes in 1 parameter, this method does not get called. // Hence, not implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data instanceof Boolean; } @@ -94,13 +95,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfDoubleFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfDoubleFunctionExecutor.java index 617f683efe..17ecc06650 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfDoubleFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfDoubleFunctionExecutor.java @@ -26,11 +26,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for instanceOf Double function. Function execution logic is implemented in execute here. */ @@ -69,22 +69,23 @@ public class InstanceOfDoubleFunctionExecutor extends FunctionExecutor { Attribute.Type returnType = Attribute.Type.BOOL; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("Invalid no of arguments passed to instanceOfDouble() " + "function, required only 1, but found " + attributeExpressionExecutors.length); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the instanceOfDouble function takes in 1 parameter, this method does not get called. // Hence, not implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data instanceof Double; } @@ -93,13 +94,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfFloatFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfFloatFunctionExecutor.java index d7c4dfdf60..5bca3a1552 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfFloatFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfFloatFunctionExecutor.java @@ -26,11 +26,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for instanceOf Double function. Function execution logic is implemented in execute here. */ @@ -68,22 +68,23 @@ public class InstanceOfFloatFunctionExecutor extends FunctionExecutor { Attribute.Type returnType = Attribute.Type.BOOL; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("Invalid no of arguments passed to instanceOfFloat() " + "function, required only 1, but found " + attributeExpressionExecutors.length); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the instanceOfFloat function takes in 1 parameter, this method does not get called. Hence, // not implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data instanceof Float; } @@ -92,13 +93,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfIntegerFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfIntegerFunctionExecutor.java index d99998b4e5..b1a48553e4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfIntegerFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfIntegerFunctionExecutor.java @@ -26,11 +26,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for instanceOf Integer function. Function execution logic is implemented in execute here. */ @@ -69,22 +69,23 @@ public class InstanceOfIntegerFunctionExecutor extends FunctionExecutor { Attribute.Type returnType = Attribute.Type.BOOL; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("Invalid no of arguments passed to instanceOfInteger() " + "function, required only 1, but found " + attributeExpressionExecutors.length); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the instanceOfInteger function takes in 1 parameter, this method does not get called. // Hence, not implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data instanceof Integer; } @@ -93,13 +94,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfLongFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfLongFunctionExecutor.java index 8b6d64a065..fb8ffc1559 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfLongFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfLongFunctionExecutor.java @@ -26,11 +26,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for instanceOf Long function. Function execution logic is implemented in execute here. */ @@ -68,22 +68,23 @@ public class InstanceOfLongFunctionExecutor extends FunctionExecutor { Attribute.Type returnType = Attribute.Type.BOOL; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("Invalid no of arguments passed to instanceOfLong() function, " + "required only 1, but found " + attributeExpressionExecutors.length); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the instanceOfLong function takes in 1 parameter, this method does not get called. Hence, // not implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data instanceof Long; } @@ -92,13 +93,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfStringFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfStringFunctionExecutor.java index bfda93bf15..78e352b235 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfStringFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/InstanceOfStringFunctionExecutor.java @@ -26,11 +26,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for instanceOf String function. Function execution logic is implemented in execute here. */ @@ -69,22 +69,23 @@ public class InstanceOfStringFunctionExecutor extends FunctionExecutor { Attribute.Type returnType = Attribute.Type.BOOL; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("Invalid no of arguments passed to instanceOfString() " + "function, required only 1, but found " + attributeExpressionExecutors.length); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the instanceOfString function takes in 1 parameter, this method does not get called. // Hence, not implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data instanceof java.lang.String; } @@ -93,13 +94,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/MaximumFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/MaximumFunctionExecutor.java index b123b85545..4577bbd1b1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/MaximumFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/MaximumFunctionExecutor.java @@ -26,11 +26,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for Maximum function. Function execution logic is implemented in execute here. */ @@ -60,8 +60,8 @@ public class MaximumFunctionExecutor extends FunctionExecutor { private Attribute.Type returnType; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { Attribute.Type attributeTypeOne = attributeExpressionExecutors[0].getReturnType(); if (!((attributeTypeOne == Attribute.Type.DOUBLE) || (attributeTypeOne == Attribute.Type.INT) || (attributeTypeOne == Attribute.Type.FLOAT) || (attributeTypeOne == Attribute.Type.LONG))) { @@ -89,16 +89,18 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea } returnType = attributeTypeOne; + return null; } /** * return maximum of arbitrary long set of Double values * * @param data array of Double values + * @param state * @return max */ @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { double max = Double.MIN_VALUE; for (Object aObj : data) { @@ -133,7 +135,7 @@ protected Object execute(Object[] data) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data; } @@ -142,13 +144,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/MinimumFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/MinimumFunctionExecutor.java index 7b5c3f66ec..996c87e106 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/MinimumFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/MinimumFunctionExecutor.java @@ -27,11 +27,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for Minimum function. Function execution logic is implemented in execute here. */ @@ -63,8 +63,8 @@ public class MinimumFunctionExecutor extends FunctionExecutor { private Attribute.Type returnType; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { Attribute.Type attributeTypeOne = attributeExpressionExecutors[0].getReturnType(); if (!((attributeTypeOne == Attribute.Type.DOUBLE) || (attributeTypeOne == Attribute.Type.INT) || (attributeTypeOne == Attribute.Type.FLOAT) || (attributeTypeOne == Attribute.Type.LONG))) { @@ -90,16 +90,18 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea } returnType = attributeTypeOne; + return null; } /** * return maximum of arbitrary long set of Double values * * @param data array of Double values + * @param state * @return max */ @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { double min = Double.MAX_VALUE; for (Object aObj : data) { Double value = Double.MAX_VALUE; @@ -133,7 +135,7 @@ protected Object execute(Object[] data) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data; } @@ -142,13 +144,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/ScriptFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/ScriptFunctionExecutor.java index dece3d493c..7e89173186 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/ScriptFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/ScriptFunctionExecutor.java @@ -22,11 +22,11 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.function.Script; 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 org.apache.log4j.Logger; -import java.util.Map; - /** * Executor class for Script function. Function execution logic is implemented in execute here. */ @@ -49,29 +49,21 @@ public Attribute.Type getReturnType() { } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { returnType = siddhiQueryContext.getSiddhiAppContext().getScript(functionId).getReturnType(); script = siddhiQueryContext.getSiddhiAppContext().getScript(functionId); + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return script.eval(functionId, data); } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return script.eval(functionId, new Object[]{data}); } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/SizeOfSetFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/SizeOfSetFunctionExecutor.java index fdc8088eff..c2669f49e5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/SizeOfSetFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/SizeOfSetFunctionExecutor.java @@ -28,10 +28,11 @@ import io.siddhi.core.exception.SiddhiAppRuntimeException; import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; import java.util.Set; /** @@ -70,8 +71,8 @@ public class SizeOfSetFunctionExecutor extends FunctionExecutor { @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("sizeOfSet() function has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length + " parameters provided"); @@ -80,22 +81,24 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea throw new OperationNotSupportedException("Parameter given for sizeOfSet() function has to be of type " + "object, but found: " + attributeExpressionExecutors[0].getReturnType()); } + return null; } /** * return maximum of arbitrary long set of Double values * * @param data array of Double values + * @param state * @return max */ @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the sizeOfSet function takes in only 1 parameter, this method does not get called. // Hence, not implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { if (data == null) { return 0; } @@ -112,13 +115,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.INT; } - @Override - public Map currentState() { - return null; //no state is maintained. - } - - @Override - public void restoreState(Map state) { - //Does nothing as no state is maintained. - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/UUIDFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/UUIDFunctionExecutor.java index 90eccdcdec..a363de7356 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/UUIDFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/function/UUIDFunctionExecutor.java @@ -25,9 +25,10 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; 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 java.util.Map; import java.util.UUID; /** @@ -58,19 +59,20 @@ public class UUIDFunctionExecutor extends FunctionExecutor { Attribute.Type returnType = Attribute.Type.STRING; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { //Nothing to be done. + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the e function takes in no parameters, this method does not get called. Hence, not // implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return UUID.randomUUID().toString(); } @@ -79,13 +81,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; //No need to maintain a state. - } - - @Override - public void restoreState(Map state) { - //Since there's no need to maintain a state, nothing needs to be done here. - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalAggregateBaseTimeFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalAggregateBaseTimeFunctionExecutor.java index f4f9a568e3..2b796f5029 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalAggregateBaseTimeFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalAggregateBaseTimeFunctionExecutor.java @@ -24,12 +24,12 @@ import io.siddhi.core.executor.function.FunctionExecutor; import io.siddhi.core.util.IncrementalTimeConverterUtil; 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.aggregation.TimePeriod; import io.siddhi.query.api.definition.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /** * Executor class for finding the start time and end time of the within clause in incremental processing. * This is important when retrieving incremental aggregate values by specifying a time range with 'within' clause. @@ -37,8 +37,8 @@ public class IncrementalAggregateBaseTimeFunctionExecutor extends FunctionExecutor { @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 2) { throw new SiddhiAppValidationException("incrementalAggregator:getAggregationStartTime() function accepts " + "two arguments, but found " + attributeExpressionExecutors.length); @@ -48,10 +48,11 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea "incrementalAggregator:getAggregationStartTime() function accepts should be of type 'STRING', " + "but found '" + attributeExpressionExecutors[1].getReturnType() + "'."); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { long time = (long) data[0]; String durationName = (String) data[1]; TimePeriod.Duration duration; @@ -83,7 +84,7 @@ protected Object execute(Object[] data) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { //More than one attribute executors return null; } @@ -92,14 +93,4 @@ protected Object execute(Object data) { public Attribute.Type getReturnType() { return Attribute.Type.OBJECT; } - - @Override - public Map currentState() { - return null; // No states - } - - @Override - public void restoreState(Map state) { - // Nothing to be done - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalShouldUpdateFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalShouldUpdateFunctionExecutor.java index e6e0c933c7..29ddc949b3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalShouldUpdateFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalShouldUpdateFunctionExecutor.java @@ -22,6 +22,8 @@ import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; @@ -31,12 +33,12 @@ /** * Execute class for shouldUpdate() function. */ -public class IncrementalShouldUpdateFunctionExecutor extends FunctionExecutor { - private long lastTimestamp = 0; +public class IncrementalShouldUpdateFunctionExecutor + extends FunctionExecutor { @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("shouldUpdate() function has to have exactly 1 parameter, " + "currently " + attributeExpressionExecutors.length + " parameters provided"); @@ -45,10 +47,11 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea throw new OperationNotSupportedException("Parameter given for shouldUpdate() function has to be of type " + "long, but found: " + attributeExpressionExecutors[0].getReturnType()); } + return () -> new FunctionState(); } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, FunctionState state) { //will not occur return null; } @@ -56,14 +59,15 @@ protected Object execute(Object[] data) { /** * return true/false based on timestamp values * - * @param data of Long type + * @param data of Long type + * @param state function state * @return true/false */ @Override - protected Object execute(Object data) { + protected Object execute(Object data, FunctionState state) { long timestamp = (long) data; - if (timestamp >= this.lastTimestamp) { - this.lastTimestamp = timestamp; + if (timestamp >= state.lastTimestamp) { + state.lastTimestamp = timestamp; return true; } return false; @@ -74,15 +78,24 @@ public Attribute.Type getReturnType() { return Attribute.Type.BOOL; } - @Override - public Map currentState() { - HashMap state = new HashMap<>(); - state.put("lastTimestamp", this.lastTimestamp); - return state; - } + class FunctionState extends State { + private long lastTimestamp = 0; - @Override - public void restoreState(Map state) { - this.lastTimestamp = ((long) state.get("lastTimestamp")); + @Override + public boolean canDestroy() { + return lastTimestamp == 0; + } + + @Override + public Map snapshot() { + HashMap state = new HashMap<>(); + state.put("lastTimestamp", this.lastTimestamp); + return state; + } + + @Override + public void restore(Map state) { + this.lastTimestamp = ((long) state.get("lastTimestamp")); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalStartTimeEndTimeFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalStartTimeEndTimeFunctionExecutor.java index c402e86268..1e2f044e7a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalStartTimeEndTimeFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalStartTimeEndTimeFunctionExecutor.java @@ -24,13 +24,14 @@ import io.siddhi.core.executor.function.FunctionExecutor; import io.siddhi.core.util.IncrementalTimeConverterUtil; 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.aggregation.TimePeriod; import io.siddhi.query.api.definition.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.regex.Pattern; /** @@ -73,8 +74,8 @@ private static List getSupportedRegexPatterns(int withinStringLength) { } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length == 1) { if (attributeExpressionExecutors[0].getReturnType() != Attribute.Type.STRING) { throw new SiddhiAppValidationException("Only string values are supported for single within clause " @@ -95,10 +96,11 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea throw new SiddhiAppValidationException("incrementalAggregator:startTimeEndTime() function accepts " + "only one or two arguments, but found " + attributeExpressionExecutors.length); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { long startTime; long endTime; @@ -122,7 +124,7 @@ protected Object execute(Object[] data) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return getStartTimeEndTime(data.toString().trim()); } @@ -131,16 +133,6 @@ public Attribute.Type getReturnType() { return Attribute.Type.OBJECT; } - @Override - public Map currentState() { - return null; // No states - } - - @Override - public void restoreState(Map state) { - // Nothing to be done - } - private Long[] getStartTimeEndTime(String singleWithinTimeAsString) { long startTime; long endTime; diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalTimeGetTimeZone.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalTimeGetTimeZone.java index a986215479..a984c64300 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalTimeGetTimeZone.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalTimeGetTimeZone.java @@ -23,12 +23,13 @@ import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; import java.time.Instant; import java.time.ZoneOffset; -import java.util.Map; import java.util.regex.Pattern; /** @@ -56,8 +57,8 @@ public static String getTimeZone(String stringTimeStamp) { } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (!(attributeExpressionExecutors.length == 0 || attributeExpressionExecutors.length == 1)) { throw new SiddhiAppValidationException("incrementalAggregator:getTimeZone() function " + "accepts zero or one argument, but found " + attributeExpressionExecutors.length); @@ -68,16 +69,17 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea "string values, but found " + attributeExpressionExecutors[0].getReturnType()); } } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the getTimeZone function takes in 1 parameter, this method does not get called. // Hence, not implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { if (data == null) { return ZoneOffset.systemDefault().getRules().getOffset(Instant.now()).getId(); } @@ -89,13 +91,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.STRING; } - @Override - public Map currentState() { - return null; //No states - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalUnixTimeFunctionExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalUnixTimeFunctionExecutor.java index 7064e90b04..6e72f2f2d9 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalUnixTimeFunctionExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/incremental/IncrementalUnixTimeFunctionExecutor.java @@ -23,6 +23,8 @@ import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; @@ -30,7 +32,6 @@ import java.time.ZoneId; import java.time.ZoneOffset; import java.time.ZonedDateTime; -import java.util.Map; import java.util.regex.Pattern; /** @@ -66,8 +67,8 @@ public static long getUnixTimeStamp(String stringTimeStamp) { } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppValidationException("incrementalAggregator:timestampInMilliseconds() function " + "accepts only one argument, but found " + attributeExpressionExecutors.length); @@ -76,16 +77,17 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea throw new SiddhiAppValidationException("Only string values can be converted to unix time, but found " + attributeExpressionExecutors[0].getReturnType()); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { return null; //Since the timestampInMilliseconds function takes in 1 parameter, this method does not get called. // Hence, not implemented. } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return getUnixTimeStamp(data.toString()); } @@ -94,13 +96,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.LONG; } - @Override - public Map currentState() { - return null; //No states - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorDouble.java index 2299c5e83f..9d97d1579f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorDouble.java @@ -49,16 +49,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.DOUBLE; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new AddExpressionExecutorDouble(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorFloat.java index c4885e21bf..c91716680e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorFloat.java @@ -48,17 +48,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.FLOAT; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new AddExpressionExecutorFloat(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorInt.java index b021456ada..902171ce57 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorInt.java @@ -49,17 +49,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.INT; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new AddExpressionExecutorInt(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorLong.java index fdd02b2e79..f1fd687cae 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/add/AddExpressionExecutorLong.java @@ -50,16 +50,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.LONG; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new AddExpressionExecutorLong(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorDouble.java index 7d0de39d07..0d10cf898b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorDouble.java @@ -54,17 +54,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.DOUBLE; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new DivideExpressionExecutorDouble(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorFloat.java index 6e15f4ecc1..4092bfadf1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorFloat.java @@ -53,17 +53,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.FLOAT; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new DivideExpressionExecutorFloat(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorInt.java index 5e508d0f9d..a6df3fd97c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorInt.java @@ -53,17 +53,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.INT; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new DivideExpressionExecutorInt(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorLong.java index b339ace70b..0538f4984f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/divide/DivideExpressionExecutorLong.java @@ -53,17 +53,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.LONG; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new DivideExpressionExecutorLong(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorDouble.java index 16146c471d..e84e5703a6 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorDouble.java @@ -52,18 +52,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.DOUBLE; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new ModExpressionExecutorDouble(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorFloat.java index e09a070aa0..95e2894ca3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorFloat.java @@ -52,18 +52,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.FLOAT; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new ModExpressionExecutorFloat(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorInt.java index 5d60df1f49..10a4140da0 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorInt.java @@ -52,18 +52,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.INT; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new ModExpressionExecutorInt(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorLong.java index f10d1b5325..cf47a9af45 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/mod/ModExpressionExecutorLong.java @@ -52,18 +52,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.LONG; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new ModExpressionExecutorLong(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorDouble.java index 0757e83746..8ae4bfa805 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorDouble.java @@ -49,18 +49,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.DOUBLE; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new MultiplyExpressionExecutorDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorFloat.java index 170e17dead..ed02f0bff2 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorFloat.java @@ -48,18 +48,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.FLOAT; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new MultiplyExpressionExecutorFloat(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorInt.java index 73b4d86d4e..46e5679412 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorInt.java @@ -50,17 +50,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.INT; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new MultiplyExpressionExecutorInt(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorLong.java index 9283d906bf..6d5595bab4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/multiply/MultiplyExpressionExecutorLong.java @@ -49,18 +49,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.LONG; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new MultiplyExpressionExecutorLong(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorDouble.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorDouble.java index 47c1ec753d..18710494cb 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorDouble.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorDouble.java @@ -49,18 +49,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.DOUBLE; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new SubtractExpressionExecutorDouble(leftExpressionExecutor.cloneExecutor(key), - rightExpressionExecutor.cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorFloat.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorFloat.java index 4998bccb73..fb9ff841fc 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorFloat.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorFloat.java @@ -48,18 +48,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.FLOAT; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new SubtractExpressionExecutorFloat(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorInt.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorInt.java index e2999e1781..03c1f3d67d 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorInt.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorInt.java @@ -48,18 +48,4 @@ public Object execute(ComplexEvent event) { public Attribute.Type getReturnType() { return Attribute.Type.INT; } - - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new SubtractExpressionExecutorInt(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorLong.java b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorLong.java index ddd4c75ec6..7e85ca7fbf 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorLong.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/executor/math/subtract/SubtractExpressionExecutorLong.java @@ -49,17 +49,4 @@ public Attribute.Type getReturnType() { return Attribute.Type.LONG; } - @Override - public ExpressionExecutor cloneExecutor(String key) { - return new SubtractExpressionExecutorLong(leftExpressionExecutor.cloneExecutor(key), rightExpressionExecutor - .cloneExecutor(key)); - } - - @Override - public void clean() { - leftExpressionExecutor.clean(); - rightExpressionExecutor.clean(); - } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionCreationListener.java b/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionCreationListener.java new file mode 100644 index 0000000000..14e8151910 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionCreationListener.java @@ -0,0 +1,30 @@ +/* + * 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.partition; + +/** + * To notify partition creations + */ +public interface PartitionCreationListener { + + /** + * Get called when partition is created + */ + void partitionCreated(); +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionInstanceRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionInstanceRuntime.java deleted file mode 100644 index 3d29d2770b..0000000000 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionInstanceRuntime.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package io.siddhi.core.partition; - -import io.siddhi.core.query.QueryRuntime; - -import java.util.List; - -/** - * Class to represent each partition key instance. These will be created dynamically and will contain - * {@link QueryRuntime}s belonging to that partition key. - */ -public class PartitionInstanceRuntime { - private String key; - private List queryRuntimeList; - - public PartitionInstanceRuntime(String key, List queryRuntimeList) { - this.key = key; - this.queryRuntimeList = queryRuntimeList; - } - - public List getQueryRuntimeList() { - return queryRuntimeList; - } - - public String getKey() { - return key; - } - -} - - diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionRuntime.java index 013c0e6403..46a3d636e6 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionRuntime.java @@ -33,8 +33,10 @@ import io.siddhi.core.util.SiddhiConstants; import io.siddhi.core.util.parser.helper.DefinitionParserHelper; import io.siddhi.core.util.parser.helper.QueryParserHelper; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.core.util.statistics.MemoryUsageTracker; +import io.siddhi.query.api.annotation.Annotation; import io.siddhi.query.api.annotation.Element; import io.siddhi.query.api.definition.AbstractDefinition; import io.siddhi.query.api.definition.StreamDefinition; @@ -51,45 +53,49 @@ import io.siddhi.query.api.execution.query.input.stream.SingleInputStream; import io.siddhi.query.api.execution.query.input.stream.StateInputStream; import io.siddhi.query.api.execution.query.output.stream.InsertIntoStream; +import io.siddhi.query.api.expression.Expression; import io.siddhi.query.api.util.AnnotationHelper; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.UUID; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; /** * Runtime class to handle partitioning. It will hold all information regarding current partitions and wil create * partition dynamically during runtime. */ -public class PartitionRuntime implements Snapshotable { +public class PartitionRuntime { - private String partitionId; - private String elementId; + private final StateHolder stateHolder; + //default every 5 min + private long purgeExecutionInterval = 300000; + private boolean purgingEnabled = false; + private long purgeIdlePeriod = 0; + private String partitionName; private Partition partition; - private ConcurrentMap localStreamJunctionMap = new ConcurrentHashMap(); //contains definition - private ConcurrentMap localStreamDefinitionMap = new ConcurrentHashMap(); //contains stream definition + private ConcurrentMap localStreamJunctionMap = new ConcurrentHashMap<>(); + private ConcurrentMap innerPartitionStreamReceiverStreamJunctionMap = + new ConcurrentHashMap<>(); //contains definition + private ConcurrentMap localStreamDefinitionMap = + new ConcurrentHashMap<>(); //contains stream definition private ConcurrentMap streamDefinitionMap; private ConcurrentMap windowDefinitionMap; private ConcurrentMap streamJunctionMap; - private ConcurrentMap metaQueryRuntimeMap = new ConcurrentHashMap(); - private ConcurrentMap partitionInstanceRuntimeMap = new - ConcurrentHashMap(); - private ConcurrentMap partitionStreamReceivers = new ConcurrentHashMap(); + private List queryRuntimeList = new ArrayList(); + private ConcurrentMap partitionStreamReceivers = new ConcurrentHashMap<>(); private SiddhiAppContext siddhiAppContext; - private MemoryUsageTracker memoryUsageTracker; public PartitionRuntime(ConcurrentMap streamDefinitionMap, ConcurrentMap windowDefinitionMap, ConcurrentMap streamJunctionMap, - Partition partition, SiddhiAppContext siddhiAppContext) { + Partition partition, int partitionIndex, SiddhiAppContext siddhiAppContext) { this.siddhiAppContext = siddhiAppContext; if (partition.getPartitionTypeMap().isEmpty()) { throw new SiddhiAppCreationException("Partition must have at least one executor. But found none."); @@ -98,25 +104,54 @@ public PartitionRuntime(ConcurrentMap streamDefiniti Element element = AnnotationHelper.getAnnotationElement("info", "name", partition.getAnnotations()); if (element != null) { - this.partitionId = element.getValue(); + this.partitionName = element.getValue(); } } catch (DuplicateAnnotationException e) { throw new DuplicateAnnotationException(e.getMessageWithOutContext() + " for the same Query " + partition.toString(), e, e.getQueryContextStartIndex(), e.getQueryContextEndIndex(), siddhiAppContext.getName(), siddhiAppContext.getSiddhiAppString()); } - if (partitionId == null) { - this.partitionId = UUID.randomUUID().toString(); + if (partitionName == null) { + this.partitionName = "partition_" + partitionIndex; + } + + Annotation purge = AnnotationHelper.getAnnotation(SiddhiConstants.NAMESPACE_PURGE, partition.getAnnotations()); + if (purge != null) { + if (purge.getElement(SiddhiConstants.ANNOTATION_ELEMENT_ENABLE) != null) { + String purgeEnable = purge.getElement(SiddhiConstants.ANNOTATION_ELEMENT_ENABLE); + if (!("true".equalsIgnoreCase(purgeEnable) || "false".equalsIgnoreCase(purgeEnable))) { + throw new SiddhiAppCreationException("Invalid value for enable: " + purgeEnable + "." + + " Please use 'true' or 'false'"); + } else { + purgingEnabled = Boolean.parseBoolean(purgeEnable); + } + } else { + throw new SiddhiAppCreationException("Annotation @" + SiddhiConstants.NAMESPACE_PURGE + + " is missing element '" + SiddhiConstants.ANNOTATION_ELEMENT_ENABLE + "'"); + } + if (purge.getElement(SiddhiConstants.ANNOTATION_ELEMENT_IDLE_PERIOD) != null) { + String purgeIdle = purge.getElement(SiddhiConstants.ANNOTATION_ELEMENT_IDLE_PERIOD); + purgeIdlePeriod = Expression.Time.timeToLong(purgeIdle); + + } else { + throw new SiddhiAppCreationException("Annotation @" + SiddhiConstants.NAMESPACE_PURGE + + " is missing element '" + SiddhiConstants.ANNOTATION_ELEMENT_IDLE_PERIOD + "'"); + } + + if (purge.getElement(SiddhiConstants.ANNOTATION_ELEMENT_INTERVAL) != null) { + String interval = purge.getElement(SiddhiConstants.ANNOTATION_ELEMENT_INTERVAL); + purgeExecutionInterval = Expression.Time.timeToLong(interval); + } } - elementId = "PartitionRuntime-" + siddhiAppContext.getElementIdGenerator().createNewId(); - siddhiAppContext.getSnapshotService().addSnapshotable("partition", this); this.partition = partition; this.streamDefinitionMap = streamDefinitionMap; this.windowDefinitionMap = windowDefinitionMap; this.streamJunctionMap = streamJunctionMap; + + this.stateHolder = siddhiAppContext.generateStateHolder(partitionName, () -> new PartitionState()); } - public QueryRuntime addQuery(QueryRuntime metaQueryRuntime) { + public void addQuery(QueryRuntime metaQueryRuntime) { Query query = metaQueryRuntime.getQuery(); if (query.getOutputStream() instanceof InsertIntoStream && @@ -174,9 +209,28 @@ public QueryRuntime addQuery(QueryRuntime metaQueryRuntime) { insertIntoWindowCallback.getWindow().setPublisher(streamJunctionMap.get(insertIntoWindowCallback .getOutputStreamDefinition().getId()).constructPublisher()); } - metaQueryRuntimeMap.put(metaQueryRuntime.getQueryId(), metaQueryRuntime); - return metaQueryRuntime; + if (metaQueryRuntime.isFromLocalStream()) { + for (int i = 0; i < metaQueryRuntime.getStreamRuntime().getSingleStreamRuntimes().size(); i++) { + String streamId = metaQueryRuntime.getStreamRuntime().getSingleStreamRuntimes().get(i) + .getProcessStreamReceiver().getStreamId(); + if (streamId.startsWith("#")) { + StreamDefinition streamDefinition = (StreamDefinition) localStreamDefinitionMap.get(streamId); + StreamJunction streamJunction = localStreamJunctionMap.get(streamId); + if (streamJunction == null) { + streamJunction = new StreamJunction(streamDefinition, siddhiAppContext + .getExecutorService(), + siddhiAppContext.getBufferSize(), + null, siddhiAppContext); + localStreamJunctionMap.put(streamId, streamJunction); + } + streamJunction.subscribe(metaQueryRuntime.getStreamRuntime().getSingleStreamRuntimes().get + (i).getProcessStreamReceiver()); + } + } + } + + queryRuntimeList.add(metaQueryRuntime); } public void addPartitionReceiver(QueryRuntime queryRuntime, List executors, @@ -204,8 +258,8 @@ public void addPartitionReceiver(QueryRuntime queryRuntime, List> partitionExecutors, int - executorIndex) { + List> partitionExecutors, + int executorIndex) { if (stateElement instanceof EveryStateElement) { return addPartitionReceiverForStateElement(((EveryStateElement) stateElement).getStateElement(), metaEvent, partitionExecutors, executorIndex); @@ -244,91 +298,31 @@ private void addPartitionReceiver(String streamId, boolean isInnerStream, MetaSt partitionStreamReceivers.put(partitionStreamReceiver.getStreamId(), partitionStreamReceiver); streamJunctionMap.get(partitionStreamReceiver.getStreamId()).subscribe(partitionStreamReceiver); } - } - - /** - * clone all the queries of the partition for a given partition key if they are not available - * - * @param key partition key - */ - public void cloneIfNotExist(String key) { - if (!partitionInstanceRuntimeMap.containsKey(key)) { - clonePartition(key); - } } - private synchronized void clonePartition(String key) { - PartitionInstanceRuntime partitionInstance = this.partitionInstanceRuntimeMap.get(key); - - if (partitionInstance == null) { - List queryRuntimeList = new ArrayList(); - List partitionedQueryRuntimeList = new ArrayList(); - - for (QueryRuntime queryRuntime : metaQueryRuntimeMap.values()) { - - QueryRuntime clonedQueryRuntime = queryRuntime.clone(key, localStreamJunctionMap); - queryRuntimeList.add(clonedQueryRuntime); - - QueryParserHelper.registerMemoryUsageTracking(clonedQueryRuntime.getQueryId(), queryRuntime, - SiddhiConstants.METRIC_INFIX_QUERIES, siddhiAppContext, memoryUsageTracker); - - if (queryRuntime.isFromLocalStream()) { - for (int i = 0; i < clonedQueryRuntime.getStreamRuntime().getSingleStreamRuntimes().size(); i++) { - String streamId = queryRuntime.getStreamRuntime().getSingleStreamRuntimes().get(i) - .getProcessStreamReceiver().getStreamId(); - StreamDefinition streamDefinition = null; - if (streamId.startsWith("#")) { - streamDefinition = (StreamDefinition) localStreamDefinitionMap.get(streamId); - } else { - streamDefinition = (StreamDefinition) streamDefinitionMap.get(streamId); - if (streamDefinition == null) { - streamDefinition = (StreamDefinition) windowDefinitionMap.get(streamId); - } - } - StreamJunction streamJunction = localStreamJunctionMap.get(streamId + key); - if (streamJunction == null) { - streamJunction = new StreamJunction(streamDefinition, siddhiAppContext - .getExecutorService(), - siddhiAppContext.getBufferSize(), - null, siddhiAppContext); - localStreamJunctionMap.put(streamId + key, streamJunction); - } - streamJunction.subscribe(clonedQueryRuntime.getStreamRuntime().getSingleStreamRuntimes().get - (i).getProcessStreamReceiver()); - } - } else { - partitionedQueryRuntimeList.add(clonedQueryRuntime); - } - } - partitionInstanceRuntimeMap.putIfAbsent(key, new PartitionInstanceRuntime(key, queryRuntimeList)); - updatePartitionStreamReceivers(key, partitionedQueryRuntimeList); - } - + private StreamJunction createStreamJunction(StreamDefinition streamDefinition) { + return new StreamJunction(streamDefinition, siddhiAppContext.getExecutorService(), + siddhiAppContext.getBufferSize(), null, siddhiAppContext); } - private void updatePartitionStreamReceivers(String key, List partitionedQueryRuntimeList) { - for (PartitionStreamReceiver partitionStreamReceiver : partitionStreamReceivers.values()) { - partitionStreamReceiver.addStreamJunction(key, partitionedQueryRuntimeList); - } + public void addInnerpartitionStreamReceiverStreamJunction(String key, StreamJunction streamJunction) { + innerPartitionStreamReceiverStreamJunctionMap.put(key, streamJunction); } - public void addStreamJunction(String key, StreamJunction streamJunction) { - localStreamJunctionMap.put(key, streamJunction); + public ConcurrentMap getInnerPartitionStreamReceiverStreamJunctionMap() { + return innerPartitionStreamReceiverStreamJunctionMap; } public void init() { for (PartitionStreamReceiver partitionStreamReceiver : partitionStreamReceivers.values()) { + partitionStreamReceiver.addStreamJunction(queryRuntimeList); partitionStreamReceiver.init(); } } - public String getPartitionId() { - return partitionId; - } - - public ConcurrentMap getMetaQueryRuntimeMap() { - return metaQueryRuntimeMap; + public String getPartitionName() { + return partitionName; } public ConcurrentMap getLocalStreamDefinitionMap() { @@ -339,33 +333,104 @@ public ConcurrentMap getLocalStreamJunctionMap() { return localStreamJunctionMap; } - @Override - public Map currentState() { - Map state = new HashMap<>(); - List partitionKeys = new ArrayList<>(partitionInstanceRuntimeMap.keySet()); - state.put("PartitionKeys", partitionKeys); - return state; - } - @Override - public void restoreState(Map state) { - List partitionKeys = (List) state.get("PartitionKeys"); - for (String key : partitionKeys) { - clonePartition(key); + public void setMemoryUsageTracker(MemoryUsageTracker memoryUsageTracker) { + for (QueryRuntime queryRuntime : queryRuntimeList) { + QueryParserHelper.registerMemoryUsageTracking(queryRuntime.getQueryId(), queryRuntime, + SiddhiConstants.METRIC_INFIX_QUERIES, siddhiAppContext, memoryUsageTracker); } } - @Override - public String getElementId() { - return elementId; + public void initPartition() { + PartitionState state = stateHolder.getState(); + try { + Long time = state.partitionKeys.get(SiddhiAppContext.getPartitionFlowId()); + if (time == null) { + synchronized (state) { + time = state.partitionKeys.get(SiddhiAppContext.getPartitionFlowId()); + if (time == null) { + for (QueryRuntime queryRuntime : queryRuntimeList) { + queryRuntime.initPartition(); + } + } + state.partitionKeys.put(SiddhiAppContext.getPartitionFlowId(), + siddhiAppContext.getTimestampGenerator().currentTime()); + } + } else { + state.partitionKeys.put(SiddhiAppContext.getPartitionFlowId(), + siddhiAppContext.getTimestampGenerator().currentTime()); + } + } finally { + stateHolder.returnState(state); + } + if (purgingEnabled) { + siddhiAppContext.getScheduledExecutorService().scheduleWithFixedDelay(new Runnable() { + @Override + public void run() { + long currentTime = siddhiAppContext.getTimestampGenerator().currentTime(); + PartitionState state = stateHolder.getState(); + try { + synchronized (state) { + HashMap partitions = new HashMap<>(state.partitionKeys); + for (Map.Entry partition : partitions.entrySet()) { + if (partition.getValue() + purgeIdlePeriod < currentTime) { + state.partitionKeys.remove(partition.getKey()); + SiddhiAppContext.startPartitionFlow(partition.getKey()); + try { + for (QueryRuntime queryRuntime : queryRuntimeList) { + Map elementHolderMap = + siddhiAppContext.getSnapshotService().getStateHolderMap( + partitionName, queryRuntime.getQueryId()); + for (StateHolder stateHolder : elementHolderMap.values()) { + stateHolder.cleanGroupByStates(); + } + } + } finally { + SiddhiAppContext.stopPartitionFlow(); + } + } + } + } + } finally { + stateHolder.returnState(state); + } + } + }, purgeExecutionInterval, purgeExecutionInterval, TimeUnit.MILLISECONDS); + } } - public void setMemoryUsageTracker(MemoryUsageTracker memoryUsageTracker) { - this.memoryUsageTracker = memoryUsageTracker; + public Set getPartitionKeys() { + PartitionState state = stateHolder.getState(); + try { + return new HashSet<>(state.partitionKeys.keySet()); + } finally { + stateHolder.returnState(state); + } } - @Override - public void clean() { - siddhiAppContext.getSnapshotService().removeSnapshotable("partition", this); + /** + * State of partition + */ + public class PartitionState extends State { + + private Map partitionKeys = new ConcurrentHashMap<>(); + + @Override + public boolean canDestroy() { + return partitionKeys.isEmpty(); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("PartitionKeys", partitionKeys); + return state; + } + + @Override + public void restore(Map state) { + partitionKeys = (Map) state.get("PartitionKeys"); + } } + } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionStreamReceiver.java b/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionStreamReceiver.java index 838baae3ff..425f707277 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionStreamReceiver.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/partition/PartitionStreamReceiver.java @@ -23,7 +23,7 @@ import io.siddhi.core.event.Event; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.event.stream.converter.StreamEventConverterFactory; import io.siddhi.core.partition.executor.PartitionExecutor; @@ -32,9 +32,9 @@ import io.siddhi.core.stream.StreamJunction; import io.siddhi.query.api.definition.StreamDefinition; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; /** * Specific {@link StreamJunction.Receiver} implementation to pump events into partitions. This will send the event @@ -42,7 +42,7 @@ */ public class PartitionStreamReceiver implements StreamJunction.Receiver { - private final StreamEventPool eventPool; + private final StreamEventFactory streamEventFactory; private StreamEventConverter streamEventConverter; private String streamId; private MetaStreamEvent metaStreamEvent; @@ -50,7 +50,7 @@ public class PartitionStreamReceiver implements StreamJunction.Receiver { private SiddhiAppContext siddhiAppContext; private PartitionRuntime partitionRuntime; private List partitionExecutors; - private Map cachedStreamJunctionMap = new ConcurrentHashMap(); + private Map streamJunctionMap = new HashMap<>(); public PartitionStreamReceiver(SiddhiAppContext siddhiAppContext, MetaStreamEvent metaStreamEvent, @@ -63,7 +63,7 @@ public PartitionStreamReceiver(SiddhiAppContext siddhiAppContext, MetaStreamEven this.partitionExecutors = partitionExecutors; this.siddhiAppContext = siddhiAppContext; this.streamId = streamDefinition.getId(); - this.eventPool = new StreamEventPool(metaStreamEvent, 5); + this.streamEventFactory = new StreamEventFactory(metaStreamEvent); } @@ -84,19 +84,19 @@ public void receive(ComplexEvent complexEvent) { ComplexEventChunk outputEventChunk = new ComplexEventChunk(false); ComplexEvent aComplexEvent = complexEvent; while (aComplexEvent != null) { - StreamEvent borrowedEvent = borrowEvent(); - streamEventConverter.convertComplexEvent(aComplexEvent, borrowedEvent); - outputEventChunk.add(borrowedEvent); + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertComplexEvent(aComplexEvent, newEvent); + outputEventChunk.add(newEvent); aComplexEvent = aComplexEvent.getNext(); } send(outputEventChunk.getFirst()); } else { if (complexEvent.getNext() == null) { for (PartitionExecutor partitionExecutor : partitionExecutors) { - StreamEvent borrowedEvent = borrowEvent(); - streamEventConverter.convertComplexEvent(complexEvent, borrowedEvent); - String key = partitionExecutor.execute(borrowedEvent); - send(key, borrowedEvent); + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertComplexEvent(complexEvent, newEvent); + String key = partitionExecutor.execute(newEvent); + send(key, newEvent); } } else { ComplexEventChunk complexEventChunk = new ComplexEventChunk(false); @@ -106,11 +106,11 @@ public void receive(ComplexEvent complexEvent) { while (complexEventChunk.hasNext()) { ComplexEvent aEvent = complexEventChunk.next(); complexEventChunk.remove(); - StreamEvent borrowedEvent = borrowEvent(); - streamEventConverter.convertComplexEvent(aEvent, borrowedEvent); + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertComplexEvent(aEvent, newEvent); boolean currentEventMatchedPrevPartitionExecutor = false; for (PartitionExecutor partitionExecutor : partitionExecutors) { - String key = partitionExecutor.execute(borrowedEvent); + String key = partitionExecutor.execute(newEvent); if (key != null) { if (currentKey == null) { currentKey = key; @@ -125,13 +125,13 @@ public void receive(ComplexEvent complexEvent) { send(currentKey, firstEvent); currentKey = key; outputEventChunk.clear(); - StreamEvent cloneEvent = borrowEvent(); + StreamEvent cloneEvent = streamEventFactory.newInstance(); streamEventConverter.convertComplexEvent(aEvent, cloneEvent); outputEventChunk.add(cloneEvent); } } if (!currentEventMatchedPrevPartitionExecutor) { - outputEventChunk.add(borrowedEvent); + outputEventChunk.add(newEvent); } currentEventMatchedPrevPartitionExecutor = true; } @@ -146,55 +146,51 @@ public void receive(ComplexEvent complexEvent) { @Override public void receive(Event event) { - StreamEvent borrowedEvent = borrowEvent(); - streamEventConverter.convertEvent(event, borrowedEvent); + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertEvent(event, newEvent); for (PartitionExecutor partitionExecutor : partitionExecutors) { - String key = partitionExecutor.execute(borrowedEvent); - send(key, borrowedEvent); + String key = partitionExecutor.execute(newEvent); + send(key, newEvent); } if (partitionExecutors.size() == 0) { - send(borrowedEvent); + send(newEvent); } - returnEvents(borrowedEvent); } @Override public void receive(long timestamp, Object[] data) { - StreamEvent borrowedEvent = borrowEvent(); - streamEventConverter.convertData(timestamp, data, borrowedEvent); + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertData(timestamp, data, newEvent); if (partitionExecutors.size() == 0) { - send(borrowedEvent); + send(newEvent); } else { for (PartitionExecutor partitionExecutor : partitionExecutors) { - String key = partitionExecutor.execute(borrowedEvent); - send(key, borrowedEvent); + String key = partitionExecutor.execute(newEvent); + send(key, newEvent); } } - returnEvents(borrowedEvent); } @Override public void receive(Event[] events) { if (partitionExecutors.size() == 0) { StreamEvent currentEvent; - StreamEvent firstEvent = borrowEvent(); + StreamEvent firstEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(events[0], firstEvent); currentEvent = firstEvent; for (int i = 1; i < events.length; i++) { - StreamEvent nextEvent = borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(events[i], nextEvent); currentEvent.setNext(nextEvent); currentEvent = nextEvent; } send(firstEvent); - returnEvents(firstEvent); - } else { String key = null; StreamEvent firstEvent = null; StreamEvent currentEvent = null; for (Event event : events) { - StreamEvent nextEvent = borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(event, nextEvent); for (PartitionExecutor partitionExecutor : partitionExecutors) { String currentKey = partitionExecutor.execute(nextEvent); @@ -204,7 +200,6 @@ public void receive(Event[] events) { firstEvent = nextEvent; } else if (!currentKey.equals(key)) { send(key, firstEvent); - returnEvents(firstEvent); key = currentKey; firstEvent = nextEvent; } else { @@ -215,7 +210,6 @@ public void receive(Event[] events) { } } send(key, firstEvent); - returnEvents(firstEvent); } } @@ -226,7 +220,7 @@ public void receive(List events) { StreamEvent firstEvent = null; StreamEvent currentEvent = null; for (Event event : events) { - StreamEvent nextEvent = borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(event, nextEvent); if (firstEvent == null) { firstEvent = nextEvent; @@ -236,13 +230,12 @@ public void receive(List events) { currentEvent = nextEvent; } send(firstEvent); - returnEvents(firstEvent); } else { String key = null; StreamEvent firstEvent = null; StreamEvent currentEvent = null; for (Event event : events) { - StreamEvent nextEvent = borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(event, nextEvent); for (PartitionExecutor partitionExecutor : partitionExecutors) { String currentKey = partitionExecutor.execute(nextEvent); @@ -252,7 +245,6 @@ public void receive(List events) { firstEvent = nextEvent; } else if (!currentKey.equals(key)) { send(key, firstEvent); - returnEvents(firstEvent); key = currentKey; firstEvent = nextEvent; } else { @@ -263,20 +255,29 @@ public void receive(List events) { } } send(key, firstEvent); - returnEvents(firstEvent); } } private void send(String key, ComplexEvent event) { if (key != null) { - partitionRuntime.cloneIfNotExist(key); - cachedStreamJunctionMap.get(streamId + key).sendEvent(event); + SiddhiAppContext.startPartitionFlow(key); + try { + partitionRuntime.initPartition(); + streamJunctionMap.get(streamId).sendEvent(event); + } finally { + SiddhiAppContext.stopPartitionFlow(); + } } } private void send(ComplexEvent event) { - for (StreamJunction streamJunction : cachedStreamJunctionMap.values()) { - streamJunction.sendEvent(event); + for (String key : partitionRuntime.getPartitionKeys()) { + SiddhiAppContext.startPartitionFlow(key); + try { + streamJunctionMap.get(streamId).sendEvent(event); + } finally { + SiddhiAppContext.stopPartitionFlow(); + } } } @@ -284,24 +285,23 @@ private void send(ComplexEvent event) { * create local streamJunctions through which events received by partitionStreamReceiver, are sent to * queryStreamReceivers * - * @param key partitioning key * @param queryRuntimeList queryRuntime list of the partition */ - public void addStreamJunction(String key, List queryRuntimeList) { - StreamJunction streamJunction = cachedStreamJunctionMap.get(streamId + key); + public void addStreamJunction(List queryRuntimeList) { + StreamJunction streamJunction = streamJunctionMap.get(streamId); if (streamJunction == null) { - streamJunction = partitionRuntime.getLocalStreamJunctionMap().get(streamId + key); + streamJunction = partitionRuntime.getInnerPartitionStreamReceiverStreamJunctionMap().get(streamId); if (streamJunction == null) { streamJunction = createStreamJunction(); - partitionRuntime.addStreamJunction(streamId + key, streamJunction); + partitionRuntime.addInnerpartitionStreamReceiverStreamJunction(streamId, streamJunction); } - cachedStreamJunctionMap.put(streamId + key, streamJunction); + streamJunctionMap.put(streamId, streamJunction); } for (QueryRuntime queryRuntime : queryRuntimeList) { StreamRuntime streamRuntime = queryRuntime.getStreamRuntime(); for (int i = 0; i < queryRuntime.getInputStreamId().size(); i++) { if ((streamRuntime.getSingleStreamRuntimes().get(i)). - getProcessStreamReceiver().getStreamId().equals(streamId + key)) { + getProcessStreamReceiver().getStreamId().equals(streamId)) { streamJunction.subscribe((streamRuntime.getSingleStreamRuntimes().get(i)) .getProcessStreamReceiver()); } @@ -314,11 +314,4 @@ private StreamJunction createStreamJunction() { siddhiAppContext.getBufferSize(), null, siddhiAppContext); } - private synchronized StreamEvent borrowEvent() { - return eventPool.borrowEvent(); - } - - private synchronized void returnEvents(StreamEvent events) { - eventPool.returnEvents(events); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/DeleteStoreQueryRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/DeleteStoreQueryRuntime.java index 93ce48157a..430c446fb9 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/DeleteStoreQueryRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/DeleteStoreQueryRuntime.java @@ -52,7 +52,7 @@ private ComplexEventChunk generateResetComplexEventChunk(MetaStrea metaStreamEvent.getOnAfterWindowData().size(), metaStreamEvent.getOutputData().size()); streamEvent.setType(ComplexEvent.Type.RESET); - StateEvent stateEvent = stateEventPool.borrowEvent(); + StateEvent stateEvent = stateEventFactory.newInstance(); stateEvent.addEvent(0, streamEvent); stateEvent.setType(ComplexEvent.Type.RESET); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/FindStoreQueryRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/FindStoreQueryRuntime.java index 3985557b65..6ce681e054 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/FindStoreQueryRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/FindStoreQueryRuntime.java @@ -18,6 +18,7 @@ package io.siddhi.core.query; import io.siddhi.core.aggregation.AggregationRuntime; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.Event; @@ -38,6 +39,7 @@ public class FindStoreQueryRuntime extends StoreQueryRuntime { private CompiledCondition compiledCondition; + private SiddhiQueryContext siddhiQueryContext; private Table table; private Window window; private MetaStreamEvent.EventType eventType; @@ -64,9 +66,10 @@ public FindStoreQueryRuntime(Window window, CompiledCondition compiledCondition, } public FindStoreQueryRuntime(AggregationRuntime aggregation, CompiledCondition compiledCondition, String queryName, - MetaStreamEvent metaStreamEvent) { + MetaStreamEvent metaStreamEvent, SiddhiQueryContext siddhiQueryContext) { this.aggregation = aggregation; this.compiledCondition = compiledCondition; + this.siddhiQueryContext = siddhiQueryContext; this.queryName = queryName; this.eventType = metaStreamEvent.getEventType(); this.metaStreamEvent = metaStreamEvent; @@ -89,7 +92,7 @@ public Event[] execute() { stateEvent = new StateEvent(2, 0); StreamEvent streamEvent = new StreamEvent(0, 2, 0); stateEvent.addEvent(0, streamEvent); - streamEvents = aggregation.find(stateEvent, compiledCondition); + streamEvents = aggregation.find(stateEvent, compiledCondition, siddhiQueryContext); break; case DEFAULT: break; @@ -130,7 +133,7 @@ private ComplexEventChunk generateResetComplexEventChunk(MetaStrea metaStreamEvent.getOnAfterWindowData().size(), metaStreamEvent.getOutputData().size()); streamEvent.setType(ComplexEvent.Type.RESET); - StateEvent stateEvent = stateEventPool.borrowEvent(); + StateEvent stateEvent = stateEventFactory.newInstance(); if (eventType == MetaStreamEvent.EventType.AGGREGATE) { stateEvent.addEvent(1, streamEvent); } else { @@ -151,7 +154,7 @@ private Event[] executeSelector(StreamEvent streamEvents, MetaStreamEvent.EventT streamEvents = streamEvents.getNext(); streamEvent.setNext(null); - StateEvent stateEvent = stateEventPool.borrowEvent(); + StateEvent stateEvent = stateEventFactory.newInstance(); if (eventType == MetaStreamEvent.EventType.AGGREGATE) { stateEvent.addEvent(1, streamEvent); } else { @@ -171,6 +174,5 @@ private Event[] executeSelector(StreamEvent streamEvents, MetaStreamEvent.EventT } else { return null; } - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/InsertStoreQueryRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/InsertStoreQueryRuntime.java index cb517a7711..4791b36fb9 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/InsertStoreQueryRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/InsertStoreQueryRuntime.java @@ -52,7 +52,7 @@ private ComplexEventChunk generateResetComplexEventChunk(MetaStrea metaStreamEvent.getOnAfterWindowData().size(), metaStreamEvent.getOutputData().size()); streamEvent.setType(ComplexEvent.Type.RESET); - StateEvent stateEvent = stateEventPool.borrowEvent(); + StateEvent stateEvent = stateEventFactory.newInstance(); stateEvent.addEvent(0, streamEvent); stateEvent.setType(ComplexEvent.Type.RESET); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/QueryRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/QueryRuntime.java index 7b366d6a17..366f329d20 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/QueryRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/QueryRuntime.java @@ -22,14 +22,12 @@ import io.siddhi.core.query.input.MultiProcessStreamReceiver; import io.siddhi.core.query.input.stream.StreamRuntime; import io.siddhi.core.query.input.stream.single.SingleStreamRuntime; +import io.siddhi.core.query.input.stream.state.StateStreamRuntime; import io.siddhi.core.query.output.callback.OutputCallback; import io.siddhi.core.query.output.callback.QueryCallback; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; import io.siddhi.core.query.selector.QuerySelector; -import io.siddhi.core.stream.StreamJunction; -import io.siddhi.core.util.lock.LockWrapper; -import io.siddhi.core.util.parser.OutputParser; -import io.siddhi.core.util.parser.helper.QueryParserHelper; +import io.siddhi.core.util.extension.holder.ExternalReferencedHolder; import io.siddhi.core.util.statistics.MemoryCalculable; import io.siddhi.query.api.definition.StreamDefinition; import io.siddhi.query.api.execution.query.Query; @@ -38,19 +36,16 @@ import io.siddhi.query.api.execution.query.input.stream.StateInputStream; import java.util.List; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.locks.ReentrantLock; /** * Query Runtime represent holder object for a single Siddhi query and holds all runtime objects related to that query. */ -public class QueryRuntime implements MemoryCalculable { +public class QueryRuntime implements MemoryCalculable, ExternalReferencedHolder { private StreamRuntime streamRuntime; private OutputRateLimiter outputRateLimiter; private Query query; private OutputCallback outputCallback; - private boolean synchronised; private SiddhiQueryContext siddhiQueryContext; private StreamDefinition outputStreamDefinition; private boolean toLocalStream; @@ -59,13 +54,12 @@ public class QueryRuntime implements MemoryCalculable { public QueryRuntime(Query query, StreamRuntime streamRuntime, QuerySelector selector, OutputRateLimiter outputRateLimiter, OutputCallback outputCallback, - MetaComplexEvent metaComplexEvent, boolean synchronised, + MetaComplexEvent metaComplexEvent, SiddhiQueryContext siddhiQueryContext) { this.query = query; this.streamRuntime = streamRuntime; this.selector = selector; this.outputCallback = outputCallback; - this.synchronised = synchronised; this.siddhiQueryContext = siddhiQueryContext; outputRateLimiter.setOutputCallback(outputCallback); setOutputRateLimiter(outputRateLimiter); @@ -118,40 +112,6 @@ public boolean isFromLocalStream() { return false; } - public QueryRuntime clone(String key, ConcurrentMap localStreamJunctionMap) { - - LockWrapper lockWrapper = null; - if (synchronised) { - lockWrapper = new LockWrapper(""); - lockWrapper.setLock(new ReentrantLock()); - } - StreamRuntime clonedStreamRuntime = this.streamRuntime.clone(key); - QuerySelector clonedSelector = this.selector.clone(key); - OutputRateLimiter clonedOutputRateLimiter = outputRateLimiter.clone(key); - clonedOutputRateLimiter.init(lockWrapper, siddhiQueryContext); - - QueryRuntime queryRuntime = new QueryRuntime(query, clonedStreamRuntime, clonedSelector, - clonedOutputRateLimiter, outputCallback, this.metaComplexEvent, - synchronised, siddhiQueryContext); - QueryParserHelper.initStreamRuntime(clonedStreamRuntime, metaComplexEvent, lockWrapper, - siddhiQueryContext.getName()); - - queryRuntime.setToLocalStream(toLocalStream); - - if (!toLocalStream) { - queryRuntime.outputRateLimiter.setOutputCallback(outputCallback); - queryRuntime.outputCallback = this.outputCallback; - } else { - OutputCallback clonedQueryOutputCallback = OutputParser.constructOutputCallback(query.getOutputStream(), - key, localStreamJunctionMap, outputStreamDefinition, siddhiQueryContext); - queryRuntime.outputRateLimiter.setOutputCallback(clonedQueryOutputCallback); - queryRuntime.outputCallback = clonedQueryOutputCallback; - } - queryRuntime.outputRateLimiter.start(); - return queryRuntime; - - } - private void setOutputRateLimiter(OutputRateLimiter outputRateLimiter) { this.outputRateLimiter = outputRateLimiter; selector.setNextProcessor(outputRateLimiter); @@ -196,4 +156,18 @@ public QuerySelector getSelector() { return selector; } + public void initPartition() { + if (streamRuntime instanceof StateStreamRuntime) { + ((StateStreamRuntime) streamRuntime).initPartition(); + } + outputRateLimiter.partitionCreated(); + } + + public void start() { + initPartition(); + } + + @Override + public void stop() { + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/StoreQueryRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/StoreQueryRuntime.java index 8c361386ee..da9b90c2fa 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/StoreQueryRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/StoreQueryRuntime.java @@ -20,7 +20,7 @@ import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.Event; import io.siddhi.core.event.state.StateEvent; -import io.siddhi.core.event.state.StateEventPool; +import io.siddhi.core.event.state.StateEventFactory; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.exception.StoreQueryRuntimeException; @@ -37,7 +37,7 @@ public abstract class StoreQueryRuntime { String queryName; MetaStreamEvent.EventType eventType; QuerySelector selector; - StateEventPool stateEventPool; + StateEventFactory stateEventFactory; MetaStreamEvent metaStreamEvent; Attribute[] outputAttributes; @@ -71,10 +71,10 @@ public Event[] execute() { /** * This method sets a state event pool for store query runtime. * - * @param stateEventPool stateEventPool for the store query runtime + * @param stateEventFactory stateEventFactory for the store query runtime */ - public void setStateEventPool(StateEventPool stateEventPool) { - this.stateEventPool = stateEventPool; + public void setStateEventFactory(StateEventFactory stateEventFactory) { + this.stateEventFactory = stateEventFactory; } /** diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/UpdateOrInsertStoreQueryRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/UpdateOrInsertStoreQueryRuntime.java index cd2a627005..57b68f2c32 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/UpdateOrInsertStoreQueryRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/UpdateOrInsertStoreQueryRuntime.java @@ -52,7 +52,7 @@ private ComplexEventChunk generateResetComplexEventChunk(MetaStrea metaStreamEvent.getOnAfterWindowData().size(), metaStreamEvent.getOutputData().size()); streamEvent.setType(ComplexEvent.Type.RESET); - StateEvent stateEvent = stateEventPool.borrowEvent(); + StateEvent stateEvent = stateEventFactory.newInstance(); stateEvent.addEvent(0, streamEvent); stateEvent.setType(ComplexEvent.Type.RESET); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/UpdateStoreQueryRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/UpdateStoreQueryRuntime.java index 62c0ec918d..249406ec57 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/UpdateStoreQueryRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/UpdateStoreQueryRuntime.java @@ -52,7 +52,7 @@ private ComplexEventChunk generateResetComplexEventChunk(MetaStrea metaStreamEvent.getOnAfterWindowData().size(), metaStreamEvent.getOutputData().size()); streamEvent.setType(ComplexEvent.Type.RESET); - StateEvent stateEvent = stateEventPool.borrowEvent(); + StateEvent stateEvent = stateEventFactory.newInstance(); stateEvent.addEvent(0, streamEvent); stateEvent.setType(ComplexEvent.Type.RESET); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/MultiProcessStreamReceiver.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/MultiProcessStreamReceiver.java index 6cfd14b819..4c69c7c434 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/MultiProcessStreamReceiver.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/MultiProcessStreamReceiver.java @@ -23,7 +23,7 @@ import io.siddhi.core.event.Event; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.event.stream.converter.StreamEventConverterFactory; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; @@ -41,23 +41,23 @@ public class MultiProcessStreamReceiver extends ProcessStreamReceiver { private static ThreadLocal multiProcessReturn = new ThreadLocal<>(); protected Processor[] nextProcessors; - protected int processCount; protected int[] eventSequence; + private final Object patternSyncObject; protected OutputRateLimiter outputRateLimiter; private MetaStreamEvent[] metaStreamEvents; - private StreamEventPool[] streamEventPools; + private StreamEventFactory[] streamEventFactorys; private StreamEventConverter[] streamEventConverters; public MultiProcessStreamReceiver(String streamId, int processCount, - SiddhiQueryContext siddhiQueryContext) { + Object patternSyncObject, SiddhiQueryContext siddhiQueryContext) { super(streamId, siddhiQueryContext); - this.processCount = processCount; nextProcessors = new Processor[processCount]; metaStreamEvents = new MetaStreamEvent[processCount]; - streamEventPools = new StreamEventPool[processCount]; + streamEventFactorys = new StreamEventFactory[processCount]; streamEventConverters = new StreamEventConverter[processCount]; eventSequence = new int[processCount]; + this.patternSyncObject = patternSyncObject; for (int i = 0; i < eventSequence.length; i++) { eventSequence[i] = i; } @@ -67,12 +67,7 @@ public static ThreadLocal getMultiProcessReturn() { return multiProcessReturn; } - public MultiProcessStreamReceiver clone(String key) { - return new MultiProcessStreamReceiver(streamId + key, processCount, - siddhiQueryContext); - } - - private void process(int eventSequence, StreamEvent borrowedEvent) { + private void process(int eventSequence, StreamEvent newEvent) { if (lockWrapper != null) { lockWrapper.lock(); } @@ -81,12 +76,12 @@ private void process(int eventSequence, StreamEvent borrowedEvent) { if (latencyTracker != null) { try { latencyTracker.markIn(); - processAndClear(eventSequence, borrowedEvent); + processAndClear(eventSequence, newEvent); } finally { latencyTracker.markOut(); } } else { - processAndClear(eventSequence, borrowedEvent); + processAndClear(eventSequence, newEvent); } } finally { if (lockWrapper != null) { @@ -98,213 +93,150 @@ private void process(int eventSequence, StreamEvent borrowedEvent) { @Override public void receive(ComplexEvent complexEvent) { ComplexEvent aComplexEvent = complexEvent; - while (aComplexEvent != null) { - if (outputRateLimiter == null) { - synchronized (this) { + List returnEventHolderList = new ArrayList<>(eventSequence.length); + synchronized (patternSyncObject) { + while (aComplexEvent != null) { + try { + multiProcessReturn.set(new ReturnEventHolder()); stabilizeStates(); for (int anEventSequence : eventSequence) { StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertComplexEvent(aComplexEvent, borrowedEvent); - process(anEventSequence, borrowedEvent); - } - } - } else { - List returnEventHolderList = new ArrayList<>(eventSequence.length); - try { - multiProcessReturn.set(new ReturnEventHolder()); - synchronized (this) { - stabilizeStates(); - for (int anEventSequence : eventSequence) { - StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertComplexEvent(aComplexEvent, borrowedEvent); - process(anEventSequence, borrowedEvent); - if (multiProcessReturn.get() != null && - multiProcessReturn.get().complexEventChunk != null) { - returnEventHolderList.add(multiProcessReturn.get()); - multiProcessReturn.set(new ReturnEventHolder()); - } + StreamEventFactory aStreamEventFactory = streamEventFactorys[anEventSequence]; + StreamEvent newEvent = aStreamEventFactory.newInstance(); + aStreamEventConverter.convertComplexEvent(aComplexEvent, newEvent); + process(anEventSequence, newEvent); + if (multiProcessReturn.get() != null && + multiProcessReturn.get().complexEventChunk != null) { + returnEventHolderList.add(multiProcessReturn.get()); + multiProcessReturn.set(new ReturnEventHolder()); } } } finally { multiProcessReturn.set(null); } - for (ReturnEventHolder returnEventHolder : returnEventHolderList) { - outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); - } + aComplexEvent = aComplexEvent.getNext(); + } + for (ReturnEventHolder returnEventHolder : returnEventHolderList) { + outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); } - aComplexEvent = aComplexEvent.getNext(); } + } @Override public void receive(Event event) { - if (outputRateLimiter == null) { - synchronized (this) { + List returnEventHolderList = new ArrayList<>(eventSequence.length); + synchronized (patternSyncObject) { + try { + multiProcessReturn.set(new ReturnEventHolder()); + stabilizeStates(); for (int anEventSequence : eventSequence) { StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertEvent(event, borrowedEvent); - process(anEventSequence, borrowedEvent); - } - } - } else { - List returnEventHolderList = new ArrayList<>(eventSequence.length); - try { - multiProcessReturn.set(new ReturnEventHolder()); - synchronized (this) { - stabilizeStates(); - for (int anEventSequence : eventSequence) { - StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertEvent(event, borrowedEvent); - process(anEventSequence, borrowedEvent); - if (multiProcessReturn.get() != null && - multiProcessReturn.get().complexEventChunk != null) { - returnEventHolderList.add(multiProcessReturn.get()); - multiProcessReturn.set(new ReturnEventHolder()); - } + StreamEventFactory aStreamEventFactory = streamEventFactorys[anEventSequence]; + StreamEvent newEvent = aStreamEventFactory.newInstance(); + aStreamEventConverter.convertEvent(event, newEvent); + process(anEventSequence, newEvent); + if (multiProcessReturn.get() != null && + multiProcessReturn.get().complexEventChunk != null) { + returnEventHolderList.add(multiProcessReturn.get()); + multiProcessReturn.set(new ReturnEventHolder()); } } } finally { multiProcessReturn.set(null); } - for (ReturnEventHolder returnEventHolder : returnEventHolderList) { - outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); - } + } + for (ReturnEventHolder returnEventHolder : returnEventHolderList) { + outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); } } @Override public void receive(Event[] events) { - for (Event event : events) { - if (outputRateLimiter == null) { - synchronized (this) { + List returnEventHolderList = new ArrayList<>(eventSequence.length); + synchronized (patternSyncObject) { + for (Event event : events) { + try { + multiProcessReturn.set(new ReturnEventHolder()); stabilizeStates(); for (int anEventSequence : eventSequence) { StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertEvent(event, borrowedEvent); - process(anEventSequence, borrowedEvent); - } - } - } else { - List returnEventHolderList = new ArrayList<>(eventSequence.length); - try { - multiProcessReturn.set(new ReturnEventHolder()); - synchronized (this) { - stabilizeStates(); - for (int anEventSequence : eventSequence) { - StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertEvent(event, borrowedEvent); - process(anEventSequence, borrowedEvent); - if (multiProcessReturn.get() != null && - multiProcessReturn.get().complexEventChunk != null) { - returnEventHolderList.add(multiProcessReturn.get()); - multiProcessReturn.set(new ReturnEventHolder()); - } + StreamEventFactory aStreamEventFactory = streamEventFactorys[anEventSequence]; + StreamEvent newEvent = aStreamEventFactory.newInstance(); + aStreamEventConverter.convertEvent(event, newEvent); + process(anEventSequence, newEvent); + if (multiProcessReturn.get() != null && + multiProcessReturn.get().complexEventChunk != null) { + returnEventHolderList.add(multiProcessReturn.get()); + multiProcessReturn.set(new ReturnEventHolder()); } } } finally { multiProcessReturn.set(null); } - for (ReturnEventHolder returnEventHolder : returnEventHolderList) { - outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); - } } } + for (ReturnEventHolder returnEventHolder : returnEventHolderList) { + outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); + } } @Override public void receive(List events) { - for (Event event : events) { - if (outputRateLimiter == null) { - synchronized (this) { + List returnEventHolderList = new ArrayList<>(eventSequence.length); + synchronized (patternSyncObject) { + for (Event event : events) { + try { + multiProcessReturn.set(new ReturnEventHolder()); stabilizeStates(); for (int anEventSequence : eventSequence) { StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertEvent(event, borrowedEvent); - process(anEventSequence, borrowedEvent); - } - } - } else { - List returnEventHolderList = new ArrayList<>(eventSequence.length); - try { - multiProcessReturn.set(new ReturnEventHolder()); - synchronized (this) { - stabilizeStates(); - for (int anEventSequence : eventSequence) { - StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertEvent(event, borrowedEvent); - process(anEventSequence, borrowedEvent); - if (multiProcessReturn.get() != null && - multiProcessReturn.get().complexEventChunk != null) { - returnEventHolderList.add(multiProcessReturn.get()); - multiProcessReturn.set(new ReturnEventHolder()); - } + StreamEventFactory aStreamEventFactory = streamEventFactorys[anEventSequence]; + StreamEvent newEvent = aStreamEventFactory.newInstance(); + aStreamEventConverter.convertEvent(event, newEvent); + process(anEventSequence, newEvent); + if (multiProcessReturn.get() != null && + multiProcessReturn.get().complexEventChunk != null) { + returnEventHolderList.add(multiProcessReturn.get()); + multiProcessReturn.set(new ReturnEventHolder()); } } } finally { multiProcessReturn.set(null); } - for (ReturnEventHolder returnEventHolder : returnEventHolderList) { - outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); - } } } + for (ReturnEventHolder returnEventHolder : returnEventHolderList) { + outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); + } } @Override public void receive(long timestamp, Object[] data) { - if (outputRateLimiter == null) { - synchronized (this) { + List returnEventHolderList = new ArrayList<>(eventSequence.length); + synchronized (patternSyncObject) { + try { + multiProcessReturn.set(new ReturnEventHolder()); stabilizeStates(); for (int anEventSequence : eventSequence) { StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertData(timestamp, data, borrowedEvent); - process(anEventSequence, borrowedEvent); - } - } - } else { - List returnEventHolderList = new ArrayList<>(eventSequence.length); - try { - multiProcessReturn.set(new ReturnEventHolder()); - synchronized (this) { - stabilizeStates(); - for (int anEventSequence : eventSequence) { - StreamEventConverter aStreamEventConverter = streamEventConverters[anEventSequence]; - StreamEventPool aStreamEventPool = streamEventPools[anEventSequence]; - StreamEvent borrowedEvent = aStreamEventPool.borrowEvent(); - aStreamEventConverter.convertData(timestamp, data, borrowedEvent); - process(anEventSequence, borrowedEvent); - if (multiProcessReturn.get() != null && - multiProcessReturn.get().complexEventChunk != null) { - returnEventHolderList.add(multiProcessReturn.get()); - multiProcessReturn.set(new ReturnEventHolder()); - } + StreamEventFactory aStreamEventFactory = streamEventFactorys[anEventSequence]; + StreamEvent newEvent = aStreamEventFactory.newInstance(); + aStreamEventConverter.convertData(timestamp, data, newEvent); + process(anEventSequence, newEvent); + if (multiProcessReturn.get() != null && + multiProcessReturn.get().complexEventChunk != null) { + returnEventHolderList.add(multiProcessReturn.get()); + multiProcessReturn.set(new ReturnEventHolder()); } } } finally { multiProcessReturn.set(null); } - for (ReturnEventHolder returnEventHolder : returnEventHolderList) { - outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); - } + } + for (ReturnEventHolder returnEventHolder : returnEventHolderList) { + outputRateLimiter.sendToCallBacks(returnEventHolder.complexEventChunk); } } @@ -344,18 +276,17 @@ public boolean toStream() { metaStreamEvents[0].getEventType() == MetaStreamEvent.EventType.WINDOW; } - public void setStreamEventPool(StreamEventPool streamEventPool) { - for (int i = 0, nextLength = streamEventPools.length; i < nextLength; i++) { - StreamEventPool eventPool = streamEventPools[i]; + public void setStreamEventFactory(StreamEventFactory streamEventFactory) { + for (int i = 0, nextLength = streamEventFactorys.length; i < nextLength; i++) { + StreamEventFactory eventPool = streamEventFactorys[i]; if (eventPool == null) { - streamEventPools[i] = streamEventPool; + streamEventFactorys[i] = streamEventFactory; break; } } } public void init() { - for (int i = 0, nextLength = streamEventConverters.length; i < nextLength; i++) { StreamEventConverter streamEventConverter = streamEventConverters[i]; if (streamEventConverter == null) { @@ -374,7 +305,6 @@ public void setOutputRateLimiter(OutputRateLimiter outputRateLimiter) { */ public class ReturnEventHolder { ComplexEventChunk complexEventChunk; - public void setReturnEvents(ComplexEventChunk complexEventChunk) { if (this.complexEventChunk == null) { this.complexEventChunk = new ComplexEventChunk(complexEventChunk.isBatch()); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/ProcessStreamReceiver.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/ProcessStreamReceiver.java index 8e6aa472ae..161aedf937 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/ProcessStreamReceiver.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/ProcessStreamReceiver.java @@ -24,7 +24,7 @@ import io.siddhi.core.event.Event; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.event.stream.converter.StreamEventConverterFactory; import io.siddhi.core.query.input.stream.state.PreStateProcessor; @@ -51,7 +51,7 @@ public class ProcessStreamReceiver implements StreamJunction.Receiver { protected boolean batchProcessingAllowed; private StreamEventConverter streamEventConverter; private MetaStreamEvent metaStreamEvent; - private StreamEventPool streamEventPool; + private StreamEventFactory streamEventFactory; private SiddhiDebugger siddhiDebugger; public ProcessStreamReceiver(String streamId, @@ -65,13 +65,6 @@ public String getStreamId() { return streamId; } - public ProcessStreamReceiver clone(String key) { - ProcessStreamReceiver processStreamReceiver = new ProcessStreamReceiver( - streamId + key, siddhiQueryContext); - processStreamReceiver.batchProcessingAllowed = this.batchProcessingAllowed; - return processStreamReceiver; - } - public void setSiddhiDebugger(SiddhiDebugger siddhiDebugger) { this.siddhiDebugger = siddhiDebugger; } @@ -105,12 +98,12 @@ public void receive(ComplexEvent complexEvents) { siddhiDebugger.checkBreakPoint(siddhiQueryContext.getName(), SiddhiDebugger.QueryTerminal.IN, complexEvents); } - StreamEvent firstEvent = streamEventPool.borrowEvent(); + StreamEvent firstEvent = streamEventFactory.newInstance(); streamEventConverter.convertComplexEvent(complexEvents, firstEvent); StreamEvent currentEvent = firstEvent; complexEvents = complexEvents.getNext(); while (complexEvents != null) { - StreamEvent nextEvent = streamEventPool.borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); streamEventConverter.convertComplexEvent(complexEvents, nextEvent); currentEvent.setNext(nextEvent); currentEvent = nextEvent; @@ -122,23 +115,23 @@ public void receive(ComplexEvent complexEvents) { @Override public void receive(Event event) { if (event != null) { - StreamEvent borrowedEvent = streamEventPool.borrowEvent(); - streamEventConverter.convertEvent(event, borrowedEvent); + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertEvent(event, newEvent); if (siddhiDebugger != null) { siddhiDebugger.checkBreakPoint(siddhiQueryContext.getName(), - SiddhiDebugger.QueryTerminal.IN, borrowedEvent); + SiddhiDebugger.QueryTerminal.IN, newEvent); } - process(new ComplexEventChunk(borrowedEvent, borrowedEvent, this.batchProcessingAllowed)); + process(new ComplexEventChunk(newEvent, newEvent, this.batchProcessingAllowed)); } } @Override public void receive(Event[] events) { - StreamEvent firstEvent = streamEventPool.borrowEvent(); + StreamEvent firstEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(events[0], firstEvent); StreamEvent currentEvent = firstEvent; for (int i = 1, eventsLength = events.length; i < eventsLength; i++) { - StreamEvent nextEvent = streamEventPool.borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(events[i], nextEvent); currentEvent.setNext(nextEvent); currentEvent = nextEvent; @@ -154,7 +147,7 @@ public void receive(List events) { StreamEvent firstEvent = null; StreamEvent currentEvent = null; for (Event event : events) { - StreamEvent nextEvent = streamEventPool.borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); streamEventConverter.convertEvent(event, nextEvent); if (firstEvent == null) { firstEvent = nextEvent; @@ -172,14 +165,14 @@ public void receive(List events) { @Override public void receive(long timestamp, Object[] data) { - StreamEvent borrowedEvent = streamEventPool.borrowEvent(); - streamEventConverter.convertData(timestamp, data, borrowedEvent); + StreamEvent newEvent = streamEventFactory.newInstance(); + streamEventConverter.convertData(timestamp, data, newEvent); // Send to debugger if (siddhiDebugger != null) { siddhiDebugger.checkBreakPoint(siddhiQueryContext.getName(), - SiddhiDebugger.QueryTerminal.IN, borrowedEvent); + SiddhiDebugger.QueryTerminal.IN, newEvent); } - process(new ComplexEventChunk(borrowedEvent, borrowedEvent, this.batchProcessingAllowed)); + process(new ComplexEventChunk(newEvent, newEvent, this.batchProcessingAllowed)); } protected void processAndClear(ComplexEventChunk streamEventChunk) { @@ -204,8 +197,8 @@ public void setNext(Processor next) { this.next = next; } - public void setStreamEventPool(StreamEventPool streamEventPool) { - this.streamEventPool = streamEventPool; + public void setStreamEventFactory(StreamEventFactory streamEventFactory) { + this.streamEventFactory = streamEventFactory; } public void setLockWrapper(LockWrapper lockWrapper) { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/SingleProcessStreamReceiver.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/SingleProcessStreamReceiver.java index 66bf0d3f24..2dcc0f88b4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/SingleProcessStreamReceiver.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/SingleProcessStreamReceiver.java @@ -31,14 +31,13 @@ */ public class SingleProcessStreamReceiver extends ProcessStreamReceiver { - protected final String lockKey; - protected ComplexEventChunk currentStreamEventChunk = new ComplexEventChunk - (batchProcessingAllowed); + private final Object patternSyncObject; private QuerySelector querySelector; - public SingleProcessStreamReceiver(String streamId, String lockKey, SiddhiQueryContext siddhiQueryContext) { + public SingleProcessStreamReceiver(String streamId, Object patternSyncObject, + SiddhiQueryContext siddhiQueryContext) { super(streamId, siddhiQueryContext); - this.lockKey = lockKey; + this.patternSyncObject = patternSyncObject; } public void setNext(Processor next) { @@ -46,21 +45,18 @@ public void setNext(Processor next) { this.querySelector = (QuerySelector) ((StreamPreStateProcessor) next).getThisLastProcessor().getNextProcessor(); } - public SingleProcessStreamReceiver clone(String key) { - return new SingleProcessStreamReceiver(streamId + key, key, - siddhiQueryContext); - } - protected void processAndClear(ComplexEventChunk streamEventChunk) { - ComplexEventChunk retEventChunk = new ComplexEventChunk(false); - synchronized (lockKey) { + ComplexEventChunk retEventChunk = new ComplexEventChunk<>(false); + ComplexEventChunk currentStreamEventChunk = new ComplexEventChunk<>( + batchProcessingAllowed); + synchronized (patternSyncObject) { while (streamEventChunk.hasNext()) { StreamEvent streamEvent = streamEventChunk.next(); streamEventChunk.remove(); stabilizeStates(); currentStreamEventChunk.add(streamEvent); - ComplexEventChunk eventChunk = ((StreamPreStateProcessor) next).processAndReturn - (currentStreamEventChunk); + ComplexEventChunk eventChunk = ((StreamPreStateProcessor) next). + processAndReturn(currentStreamEventChunk); if (eventChunk.getFirst() != null) { retEventChunk.add(eventChunk.getFirst()); } @@ -72,7 +68,7 @@ protected void processAndClear(ComplexEventChunk streamEventChunk) while (retEventChunk.hasNext()) { StateEvent stateEvent = retEventChunk.next(); retEventChunk.remove(); - querySelector.process(new ComplexEventChunk(stateEvent, stateEvent, false)); + querySelector.process(new ComplexEventChunk<>(stateEvent, stateEvent, false)); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/StateMultiProcessStreamReceiver.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/StateMultiProcessStreamReceiver.java index 472f7ae7ac..e607d5987d 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/StateMultiProcessStreamReceiver.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/StateMultiProcessStreamReceiver.java @@ -33,8 +33,9 @@ public class StateMultiProcessStreamReceiver extends MultiProcessStreamReceiver private QuerySelector querySelector; - public StateMultiProcessStreamReceiver(String streamId, int processCount, SiddhiQueryContext siddhiQueryContext) { - super(streamId, processCount, siddhiQueryContext); + public StateMultiProcessStreamReceiver(String streamId, int processCount, Object patternSyncObject, + SiddhiQueryContext siddhiQueryContext) { + super(streamId, processCount, patternSyncObject, siddhiQueryContext); } public void setNext(Processor next) { @@ -43,10 +44,6 @@ public void setNext(Processor next) { .getNextProcessor(); } - public StateMultiProcessStreamReceiver clone(String key) { - return new StateMultiProcessStreamReceiver(streamId + key, processCount, siddhiQueryContext); - } - protected void processAndClear(int processIndex, StreamEvent streamEvent) { ComplexEventChunk retEventChunk = new ComplexEventChunk(batchProcessingAllowed); ComplexEventChunk currentStreamEventChunk = new ComplexEventChunk(streamEvent, diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/StreamRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/StreamRuntime.java index 7438c3595c..aa29177636 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/StreamRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/StreamRuntime.java @@ -31,8 +31,6 @@ public interface StreamRuntime { List getSingleStreamRuntimes(); - StreamRuntime clone(String key); - void setCommonProcessor(Processor commonProcessor); MetaComplexEvent getMetaComplexEvent(); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/join/JoinProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/join/JoinProcessor.java index c32666c7bd..f1ce8555bf 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/join/JoinProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/join/JoinProcessor.java @@ -20,7 +20,7 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.StateEvent; -import io.siddhi.core.event.state.StateEventPool; +import io.siddhi.core.event.state.StateEventFactory; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.query.processor.Processor; import io.siddhi.core.query.processor.stream.window.FindableProcessor; @@ -39,7 +39,7 @@ public class JoinProcessor implements Processor { private boolean outerJoinProcessor = false; private int matchingStreamIndex; private boolean preJoinProcessor; - private StateEventPool stateEventPool; + private StateEventFactory stateEventFactory; private CompiledCondition compiledCondition; private FindableProcessor findableProcessor; private Processor nextProcessor; @@ -162,28 +162,6 @@ public void setToLast(Processor processor) { } } - /** - * Clone a copy of processor. - * - * @param key partition key - * @return Cloned Processor - */ - @Override - public Processor cloneProcessor(String key) { - JoinProcessor joinProcessor = new JoinProcessor(leftJoinProcessor, preJoinProcessor, outerJoinProcessor, - matchingStreamIndex); - joinProcessor.setTrigger(trigger); - if (trigger) { - joinProcessor.setCompiledCondition(compiledCondition.cloneCompilation(key)); - } - return joinProcessor; - } - - @Override - public void clean() { - //ignore - } - public void setFindableProcessor(FindableProcessor findableProcessor) { this.findableProcessor = findableProcessor; } @@ -200,8 +178,8 @@ public void setTrigger(boolean trigger) { this.trigger = trigger; } - public void setStateEventPool(StateEventPool stateEventPool) { - this.stateEventPool = stateEventPool; + public void setStateEventFactory(StateEventFactory stateEventFactory) { + this.stateEventFactory = stateEventFactory; } /** @@ -213,7 +191,7 @@ public void setStateEventPool(StateEventPool stateEventPool) { * @return StateEvent state event */ public StateEvent joinEventBuilder(StreamEvent leftStream, StreamEvent rightStream, ComplexEvent.Type type) { - StateEvent returnEvent = stateEventPool.borrowEvent(); + StateEvent returnEvent = stateEventFactory.newInstance(); returnEvent.setEvent(0, leftStream); returnEvent.setEvent(1, rightStream); returnEvent.setType(type); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/join/JoinStreamRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/join/JoinStreamRuntime.java index 92009eb7f7..3c78e9701b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/join/JoinStreamRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/join/JoinStreamRuntime.java @@ -24,8 +24,6 @@ import io.siddhi.core.query.input.stream.single.SingleStreamRuntime; import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; -import io.siddhi.core.query.processor.stream.window.FindableProcessor; -import io.siddhi.core.query.processor.stream.window.WindowProcessor; import java.util.ArrayList; import java.util.List; @@ -36,13 +34,11 @@ public class JoinStreamRuntime implements StreamRuntime { private List singleStreamRuntimeList = new ArrayList(); - private SiddhiQueryContext siddhiQueryContext; private MetaStateEvent metaStateEvent; private ProcessingMode overallProcessingMode = ProcessingMode.BATCH; public JoinStreamRuntime(SiddhiQueryContext siddhiQueryContext, MetaStateEvent metaStateEvent) { - this.siddhiQueryContext = siddhiQueryContext; this.metaStateEvent = metaStateEvent; } @@ -57,45 +53,6 @@ public List getSingleStreamRuntimes() { return singleStreamRuntimeList; } - @Override - public StreamRuntime clone(String key) { - - JoinStreamRuntime joinStreamRuntime = new JoinStreamRuntime(siddhiQueryContext, metaStateEvent); - for (SingleStreamRuntime singleStreamRuntime : singleStreamRuntimeList) { - joinStreamRuntime.addRuntime((SingleStreamRuntime) singleStreamRuntime.clone(key)); - } - SingleStreamRuntime leftSingleStreamRuntime = joinStreamRuntime.getSingleStreamRuntimes().get(0); - SingleStreamRuntime rightSingleStreamRuntime = joinStreamRuntime.getSingleStreamRuntimes().get(1); - - Processor lastLeftProcessor = leftSingleStreamRuntime.getProcessorChain(); - - while (!(lastLeftProcessor instanceof JoinProcessor)) { - lastLeftProcessor = lastLeftProcessor.getNextProcessor(); - } - - JoinProcessor leftPreJoinProcessor = (JoinProcessor) lastLeftProcessor; - WindowProcessor leftWindowProcessor = (WindowProcessor) leftPreJoinProcessor.getNextProcessor(); - JoinProcessor leftPostJoinProcessor = (JoinProcessor) leftWindowProcessor.getNextProcessor(); - - Processor lastRightProcessor = rightSingleStreamRuntime.getProcessorChain(); - - while (!(lastRightProcessor instanceof JoinProcessor)) { - lastRightProcessor = lastRightProcessor.getNextProcessor(); - } - - JoinProcessor rightPreJoinProcessor = (JoinProcessor) lastRightProcessor; - WindowProcessor rightWindowProcessor = (WindowProcessor) rightPreJoinProcessor.getNextProcessor(); - JoinProcessor rightPostJoinProcessor = (JoinProcessor) rightWindowProcessor.getNextProcessor(); - - rightPostJoinProcessor.setFindableProcessor((FindableProcessor) leftWindowProcessor); - rightPreJoinProcessor.setFindableProcessor((FindableProcessor) leftWindowProcessor); - - leftPreJoinProcessor.setFindableProcessor((FindableProcessor) rightWindowProcessor); - leftPostJoinProcessor.setFindableProcessor((FindableProcessor) rightWindowProcessor); - - return joinStreamRuntime; - } - @Override public void setCommonProcessor(Processor commonProcessor) { for (SingleStreamRuntime singleStreamRuntime : singleStreamRuntimeList) { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/EntryValveExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/EntryValveExecutor.java index a3c2eae2e8..3bddf69311 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/EntryValveExecutor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/EntryValveExecutor.java @@ -42,8 +42,12 @@ public EntryValveExecutor(SiddhiAppContext siddhiAppContext) { */ @Override public void execute(ComplexEventChunk complexEventChunk) { - threadBarrier.pass(); - next.execute(complexEventChunk); + threadBarrier.enter(); + try { + next.execute(complexEventChunk); + } finally { + threadBarrier.exit(); + } } /** diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/EntryValveProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/EntryValveProcessor.java index bb48d16b65..a84ba6deeb 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/EntryValveProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/EntryValveProcessor.java @@ -31,10 +31,8 @@ public class EntryValveProcessor implements Processor, Schedulable { private Processor next; private ThreadBarrier threadBarrier; - private SiddhiAppContext siddhiAppContext; public EntryValveProcessor(SiddhiAppContext siddhiAppContext) { - this.siddhiAppContext = siddhiAppContext; threadBarrier = siddhiAppContext.getThreadBarrier(); } @@ -46,8 +44,12 @@ public EntryValveProcessor(SiddhiAppContext siddhiAppContext) { */ @Override public void process(ComplexEventChunk complexEventChunk) { - threadBarrier.pass(); - next.process(complexEventChunk); + threadBarrier.enter(); + try { + next.process(complexEventChunk); + } finally { + threadBarrier.exit(); + } } /** @@ -85,20 +87,4 @@ public void setToLast(Processor processor) { } } - /** - * Clone a copy of processor - * - * @param key partition key - * @return cloned processor - */ - @Override - public Processor cloneProcessor(String key) { - return new EntryValveProcessor(siddhiAppContext); - } - - @Override - public void clean() { - //ignore - } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/SingleStreamRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/SingleStreamRuntime.java index e6456f6b79..c3e221f649 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/SingleStreamRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/single/SingleStreamRuntime.java @@ -20,11 +20,8 @@ import io.siddhi.core.event.MetaComplexEvent; import io.siddhi.core.query.input.ProcessStreamReceiver; import io.siddhi.core.query.input.stream.StreamRuntime; -import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; -import io.siddhi.core.query.processor.SchedulingProcessor; -import io.siddhi.core.query.selector.QuerySelector; import java.util.ArrayList; import java.util.List; @@ -60,10 +57,6 @@ public ProcessStreamReceiver getProcessStreamReceiver() { return processStreamReceiver; } - public void setProcessStreamReceiver(ProcessStreamReceiver processStreamReceiver) { - this.processStreamReceiver = processStreamReceiver; - } - @Override public List getSingleStreamRuntimes() { List list = new ArrayList(1); @@ -72,39 +65,6 @@ public List getSingleStreamRuntimes() { } - @Override - public StreamRuntime clone(String key) { - ProcessStreamReceiver clonedProcessStreamReceiver = this.processStreamReceiver.clone(key); - EntryValveProcessor entryValveProcessor = null; - SchedulingProcessor schedulingProcessor; - Processor clonedProcessorChain = null; - if (processorChain != null) { - if (!(processorChain instanceof QuerySelector || processorChain instanceof OutputRateLimiter)) { - clonedProcessorChain = processorChain.cloneProcessor(key); - if (clonedProcessorChain instanceof EntryValveProcessor) { - entryValveProcessor = (EntryValveProcessor) clonedProcessorChain; - } - } - Processor processor = processorChain.getNextProcessor(); - while (processor != null) { - if (!(processor instanceof QuerySelector || processor instanceof OutputRateLimiter)) { - Processor clonedProcessor = processor.cloneProcessor(key); - clonedProcessorChain.setToLast(clonedProcessor); - if (clonedProcessor instanceof EntryValveProcessor) { - entryValveProcessor = (EntryValveProcessor) clonedProcessor; - } else if (clonedProcessor instanceof SchedulingProcessor) { - schedulingProcessor = (SchedulingProcessor) clonedProcessor; - schedulingProcessor.setScheduler(((SchedulingProcessor) processor).getScheduler().clone( - key, entryValveProcessor)); - } - } - processor = processor.getNextProcessor(); - } - } - return new SingleStreamRuntime(clonedProcessStreamReceiver, clonedProcessorChain, overallProcessingMode, - metaComplexEvent); - } - @Override public void setCommonProcessor(Processor commonProcessor) { if (processorChain == null) { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentLogicalPostStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentLogicalPostStateProcessor.java index 1676df7482..b88159ba39 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentLogicalPostStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentLogicalPostStateProcessor.java @@ -48,10 +48,4 @@ protected void process(StateEvent stateEvent, ComplexEventChunk complexEventChun ((AbsentPreStateProcessor) thisStatePreProcessor).updateLastArrivalTime(streamEvent.getTimestamp()); } - @Override - public PostStateProcessor cloneProcessor(String key) { - AbsentLogicalPostStateProcessor logicalPostStateProcessor = new AbsentLogicalPostStateProcessor(type); - cloneProperties(logicalPostStateProcessor); - return logicalPostStateProcessor; - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentLogicalPreStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentLogicalPreStateProcessor.java index 2d3c80e15a..369fb01148 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentLogicalPreStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentLogicalPreStateProcessor.java @@ -18,17 +18,17 @@ package io.siddhi.core.query.input.stream.state; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.query.input.stream.single.EntryValveProcessor; import io.siddhi.core.util.Scheduler; -import io.siddhi.core.util.parser.SchedulerParser; import io.siddhi.query.api.execution.query.input.state.LogicalStateElement; import io.siddhi.query.api.execution.query.input.stream.StateInputStream; import io.siddhi.query.api.expression.constant.TimeConstant; import java.util.Iterator; +import java.util.Map; /** * Logical not processor. @@ -45,62 +45,52 @@ public class AbsentLogicalPreStateProcessor extends LogicalPreStateProcessor imp */ private long waitingTime = -1; - /** - * The timestamp of the last event received by this processor. - */ - private volatile long lastArrivalTime; - - /** - * This flag turns to false after processing the first event if 'every' is not used. - */ - private boolean active = true; - - /** - * TimeConstant to be used by cloneProcessor method. - */ - private TimeConstant waitingTimeConstant; - public AbsentLogicalPreStateProcessor(LogicalStateElement.Type type, StateInputStream.Type stateType, TimeConstant waitingTime) { super(type, stateType); if (waitingTime != null) { this.waitingTime = waitingTime.value(); - this.waitingTimeConstant = waitingTime; } } + public void init(SiddhiQueryContext siddhiQueryContext) { + this.siddhiQueryContext = siddhiQueryContext; + this.stateHolder = siddhiQueryContext.generateStateHolder( + this.getClass().getName(), + false, () -> new LogicalStreamPreState()); + } + @Override public void updateLastArrivalTime(long timestamp) { - + StreamPreState state = stateHolder.getState(); this.lock.lock(); try { - this.lastArrivalTime = timestamp; + ((LogicalStreamPreState) state).lastArrivalTime = timestamp; } finally { this.lock.unlock(); + stateHolder.returnState(state); } } - @Override - public void addState(StateEvent stateEvent) { - - if (!this.active) { + protected void addState(StateEvent stateEvent, StreamPreState state) { + if (!((LogicalStreamPreState) state).active) { return; } - this.lock.lock(); + lock.lock(); try { - super.addState(stateEvent); + super.addState(stateEvent, state); if (!isStartState) { if (waitingTime != -1) { scheduler.notifyAt(stateEvent.getTimestamp() + waitingTime); if (partnerStatePreProcessor instanceof AbsentLogicalPreStateProcessor) { - ((AbsentLogicalPreStateProcessor) partnerStatePreProcessor).scheduler.notifyAt(stateEvent - .getTimestamp() + ((AbsentLogicalPreStateProcessor) partnerStatePreProcessor) - .waitingTime); + ((AbsentLogicalPreStateProcessor) partnerStatePreProcessor).scheduler.notifyAt( + stateEvent.getTimestamp() + + ((AbsentLogicalPreStateProcessor) partnerStatePreProcessor).waitingTime); } } } } finally { - this.lock.unlock(); + lock.unlock(); } } @@ -114,98 +104,100 @@ public void addEveryState(StateEvent stateEvent) { } clonedEvent.setEvent(stateId, null); clonedEvent.setEvent(partnerStatePreProcessor.stateId, null); - // Start state takes events from newAndEveryStateEventList - newAndEveryStateEventList.add(clonedEvent); - partnerStatePreProcessor.newAndEveryStateEventList.add(clonedEvent); + StreamPreState state = stateHolder.getState(); + try { + // Start state takes events from newAndEveryStateEventList + state.getNewAndEveryStateEventList().add(clonedEvent); + partnerStatePreProcessor.addEventToNewAndEveryStateEventList(clonedEvent); + } finally { + stateHolder.returnState(state); + } } @Override public void process(ComplexEventChunk complexEventChunk) { - if (!this.active) { - return; - } - boolean notProcessed = true; - ComplexEventChunk retEventChunk = new ComplexEventChunk<>(false); - this.lock.lock(); + StreamPreState state = stateHolder.getState(); try { - long currentTime = complexEventChunk.getFirst().getTimestamp(); - if (currentTime >= this.lastArrivalTime + waitingTime) { - - - Iterator iterator; - if (isStartState && stateType == StateInputStream.Type.SEQUENCE && newAndEveryStateEventList.isEmpty() - && pendingStateEventList.isEmpty()) { - - StateEvent stateEvent = stateEventPool.borrowEvent(); - addState(stateEvent); - } else if (stateType == StateInputStream.Type.SEQUENCE && !newAndEveryStateEventList.isEmpty()) { - this.resetState(); - } - - this.updateState(); - iterator = pendingStateEventList.iterator(); - - while (iterator.hasNext()) { - StateEvent stateEvent = iterator.next(); - - // Remove expired events based on within - if (isExpired(stateEvent, currentTime)) { - iterator.remove(); - if (withinEveryPreStateProcessor != null) { - withinEveryPreStateProcessor.addEveryState(stateEvent); - withinEveryPreStateProcessor.updateState(); - } - continue; + if (!((LogicalStreamPreState) state).active) { + return; + } + boolean notProcessed = true; + ComplexEventChunk retEventChunk = new ComplexEventChunk<>(false); + this.lock.lock(); + try { + long currentTime = complexEventChunk.getFirst().getTimestamp(); + if (currentTime >= ((LogicalStreamPreState) state).lastArrivalTime + waitingTime) { + Iterator iterator; + if (isStartState && stateType == StateInputStream.Type.SEQUENCE && + state.getNewAndEveryStateEventList().isEmpty() && + state.getPendingStateEventList().isEmpty()) { + + StateEvent stateEvent = stateEventFactory.newInstance(); + addState(stateEvent); + } else if (stateType == StateInputStream.Type.SEQUENCE && + !state.getNewAndEveryStateEventList().isEmpty()) { + this.resetState(); } - - // Collect the events that came before the waiting time - if (waitingTimePassed(currentTime, stateEvent)) { - - iterator.remove(); - - if (logicalType == LogicalStateElement.Type.OR && stateEvent.getStreamEvent - (partnerStatePreProcessor.getStateId()) == null) { - // OR Partner not received - stateEvent.addEvent(stateId, streamEventPool.borrowEvent()); - retEventChunk.add(stateEvent); - } else if (logicalType == LogicalStateElement.Type.AND && stateEvent.getStreamEvent - (partnerStatePreProcessor.getStateId()) != null) { - // AND partner received but didn't send out - retEventChunk.add(stateEvent); - } else if (logicalType == LogicalStateElement.Type.AND && stateEvent.getStreamEvent - (partnerStatePreProcessor.getStateId()) == null) { - // AND partner didn't receive - // Let the partner to process or not - stateEvent.addEvent(stateId, streamEventPool.borrowEvent()); + this.updateState(); + iterator = state.getPendingStateEventList().iterator(); + while (iterator.hasNext()) { + StateEvent stateEvent = iterator.next(); + // Remove expired events based on within + if (isExpired(stateEvent, currentTime)) { + iterator.remove(); + if (withinEveryPreStateProcessor != null) { + withinEveryPreStateProcessor.addEveryState(stateEvent); + withinEveryPreStateProcessor.updateState(); + } + continue; + } + // Collect the events that came before the waiting time + if (waitingTimePassed(currentTime, stateEvent)) { + iterator.remove(); + if (logicalType == LogicalStateElement.Type.OR && stateEvent.getStreamEvent + (partnerStatePreProcessor.getStateId()) == null) { + // OR Partner not received + stateEvent.addEvent(stateId, streamEventFactory.newInstance()); + retEventChunk.add(stateEvent); + } else if (logicalType == LogicalStateElement.Type.AND && stateEvent.getStreamEvent + (partnerStatePreProcessor.getStateId()) != null) { + // AND partner received but didn't send out + retEventChunk.add(stateEvent); + } else if (logicalType == LogicalStateElement.Type.AND && stateEvent.getStreamEvent + (partnerStatePreProcessor.getStateId()) == null) { + // AND partner didn't receive + // Let the partner to process or not + stateEvent.addEvent(stateId, streamEventFactory.newInstance()); + } } } + retEventChunk.reset(); + notProcessed = retEventChunk.getFirst() == null; + while (retEventChunk.hasNext()) { + StateEvent stateEvent = retEventChunk.next(); + retEventChunk.remove(); + sendEvent(stateEvent, state); + } + ((LogicalStreamPreState) state).lastArrivalTime = 0; } + } finally { + this.lock.unlock(); + } - retEventChunk.reset(); - notProcessed = retEventChunk.getFirst() == null; - while (retEventChunk.hasNext()) { - StateEvent stateEvent = retEventChunk.next(); - retEventChunk.remove(); - sendEvent(stateEvent); + if (thisStatePostProcessor.nextEveryStatePreProcessor != null || (notProcessed && isStartState)) { + // If every or (notProcessed and startState), schedule again + long nextBreak; + if (((LogicalStreamPreState) state).lastArrivalTime == 0) { + nextBreak = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime() + + waitingTime; + } else { + nextBreak = ((LogicalStreamPreState) state).lastArrivalTime + waitingTime; } - - this.lastArrivalTime = 0; + this.scheduler.notifyAt(nextBreak); } } finally { - this.lock.unlock(); - } - - if (thisStatePostProcessor.nextEveryStatePreProcessor != null || (notProcessed && isStartState)) { - // If every or (notProcessed and startState), schedule again - long nextBreak; - if (lastArrivalTime == 0) { - nextBreak = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime() + - waitingTime; - } else { - nextBreak = lastArrivalTime + waitingTime; - } - this.scheduler.notifyAt(nextBreak); + stateHolder.returnState(state); } } @@ -219,7 +211,7 @@ private boolean waitingTimePassed(long currentTime, StateEvent event) { } } - private void sendEvent(StateEvent stateEvent) { + private void sendEvent(StateEvent stateEvent, StreamPreState state) { if (thisStatePostProcessor.nextProcessor != null) { thisStatePostProcessor.nextProcessor.process(new ComplexEventChunk<>(stateEvent, stateEvent, false)); @@ -230,10 +222,10 @@ private void sendEvent(StateEvent stateEvent) { if (thisStatePostProcessor.nextEveryStatePreProcessor != null) { thisStatePostProcessor.nextEveryStatePreProcessor.addEveryState(stateEvent); } else if (isStartState) { - this.active = false; + ((LogicalStreamPreState) state).active = false; if (logicalType == LogicalStateElement.Type.OR && partnerStatePreProcessor instanceof AbsentLogicalPreStateProcessor) { - ((AbsentLogicalPreStateProcessor) partnerStatePreProcessor).active = false; + ((AbsentLogicalPreStateProcessor) partnerStatePreProcessor).setActive(false); } } if (thisStatePostProcessor.callbackPreStateProcessor != null) { @@ -241,67 +233,81 @@ private void sendEvent(StateEvent stateEvent) { } } + private void setActive(boolean active) { + StreamPreState state = stateHolder.getState(); + try { + ((LogicalStreamPreState) state).active = active; + } finally { + stateHolder.returnState(state); + } + } + @Override public ComplexEventChunk processAndReturn(ComplexEventChunk complexEventChunk) { ComplexEventChunk returnEventChunk = new ComplexEventChunk<>(false); - - if (!this.active) { - return returnEventChunk; - } - complexEventChunk.reset(); - StreamEvent streamEvent = (StreamEvent) complexEventChunk.next(); //Sure only one will be sent - - this.lock.lock(); + StreamPreState state = stateHolder.getState(); try { - for (Iterator iterator = pendingStateEventList.iterator(); iterator.hasNext(); ) { - StateEvent stateEvent = iterator.next(); - if (isExpired(stateEvent, streamEvent.getTimestamp())) { - if (withinEveryPreStateProcessor != null) { - withinEveryPreStateProcessor.addEveryState(stateEvent); - withinEveryPreStateProcessor.updateState(); + + if (!((LogicalStreamPreState) state).active) { + return returnEventChunk; + } + complexEventChunk.reset(); + StreamEvent streamEvent = (StreamEvent) complexEventChunk.next(); //Sure only one will be sent + this.lock.lock(); + try { + for (Iterator iterator = state.getPendingStateEventList().iterator(); + iterator.hasNext(); ) { + StateEvent stateEvent = iterator.next(); + if (isExpired(stateEvent, streamEvent.getTimestamp())) { + if (withinEveryPreStateProcessor != null) { + withinEveryPreStateProcessor.addEveryState(stateEvent); + withinEveryPreStateProcessor.updateState(); + } + iterator.remove(); + continue; } - iterator.remove(); - continue; - } - if (logicalType == LogicalStateElement.Type.OR && - stateEvent.getStreamEvent(partnerStatePreProcessor.getStateId()) != null) { - iterator.remove(); - continue; - } - StreamEvent currentStreamEvent = stateEvent.getStreamEvent(stateId); - stateEvent.setEvent(stateId, streamEventCloner.copyStreamEvent(streamEvent)); - process(stateEvent); - if (waitingTime != -1 || (stateType == StateInputStream.Type.SEQUENCE && - logicalType == LogicalStateElement.Type.AND && thisStatePostProcessor - .nextEveryStatePreProcessor != null)) { - // Reset to the original state after processing - stateEvent.setEvent(stateId, currentStreamEvent); - } - if (this.thisLastProcessor.isEventReturned()) { - this.thisLastProcessor.clearProcessedEvent(); - // The event has passed the filter condition. So remove from being an absent candidate. - iterator.remove(); - if (stateType == StateInputStream.Type.SEQUENCE) { - partnerStatePreProcessor.pendingStateEventList.remove(stateEvent); + if (logicalType == LogicalStateElement.Type.OR && + stateEvent.getStreamEvent(partnerStatePreProcessor.getStateId()) != null) { + iterator.remove(); + continue; } - } - if (!stateChanged) { - switch (stateType) { - case PATTERN: - stateEvent.setEvent(stateId, currentStreamEvent); - break; - case SEQUENCE: - stateEvent.setEvent(stateId, currentStreamEvent); - iterator.remove(); - break; + StreamEvent currentStreamEvent = stateEvent.getStreamEvent(stateId); + stateEvent.setEvent(stateId, streamEventCloner.copyStreamEvent(streamEvent)); + process(stateEvent); + if (waitingTime != -1 || (stateType == StateInputStream.Type.SEQUENCE && + logicalType == LogicalStateElement.Type.AND && thisStatePostProcessor + .nextEveryStatePreProcessor != null)) { + // Reset to the original state after processing + stateEvent.setEvent(stateId, currentStreamEvent); + } + if (this.thisLastProcessor.isEventReturned()) { + this.thisLastProcessor.clearProcessedEvent(); + // The event has passed the filter condition. So remove from being an absent candidate. + iterator.remove(); + if (stateType == StateInputStream.Type.SEQUENCE) { + partnerStatePreProcessor.getPendingStateEventList().remove(stateEvent); + } + } + if (!state.isStateChanged()) { + switch (stateType) { + case PATTERN: + stateEvent.setEvent(stateId, currentStreamEvent); + break; + case SEQUENCE: + stateEvent.setEvent(stateId, currentStreamEvent); + iterator.remove(); + break; + } } } + } finally { + this.lock.unlock(); } + return returnEventChunk; } finally { - this.lock.unlock(); + stateHolder.returnState(state); } - return returnEventChunk; } @Override @@ -315,70 +321,94 @@ public void setScheduler(Scheduler scheduler) { } @Override - public PreStateProcessor cloneProcessor(String key) { - AbsentLogicalPreStateProcessor logicalPreStateProcessor = new AbsentLogicalPreStateProcessor(logicalType, - stateType, waitingTimeConstant); - cloneProperties(logicalPreStateProcessor, key); - logicalPreStateProcessor.init(siddhiQueryContext); - - // Set the scheduler - siddhiQueryContext.getSiddhiAppContext().addEternalReferencedHolder(logicalPreStateProcessor); - EntryValveProcessor entryValveProcessor = new EntryValveProcessor(siddhiQueryContext.getSiddhiAppContext()); - entryValveProcessor.setToLast(logicalPreStateProcessor); - Scheduler scheduler = SchedulerParser.parse(entryValveProcessor, siddhiQueryContext.getSiddhiAppContext()); - logicalPreStateProcessor.setScheduler(scheduler); - - return logicalPreStateProcessor; - } - - @Override - public void start() { - if (isStartState && waitingTime != -1 && active) { - this.lock.lock(); - try { - this.scheduler.notifyAt(this.siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator() - .currentTime() + waitingTime); - } finally { - this.lock.unlock(); + public void partitionCreated() { + StreamPreState state = stateHolder.getState(); + try { + if (!state.isStarted()) { + state.started(); + if (isStartState && waitingTime != -1 && ((LogicalStreamPreState) state).active) { + this.lock.lock(); + try { + this.scheduler.notifyAt( + this.siddhiQueryContext.getSiddhiAppContext(). + getTimestampGenerator().currentTime() + waitingTime); + } finally { + this.lock.unlock(); + } + } } + } finally { + stateHolder.returnState(state); } } - @Override - public void stop() { - // Scheduler will be stopped automatically - // Nothing to stop here - } - public boolean partnerCanProceed(StateEvent stateEvent) { - - boolean process; - if (stateType == StateInputStream.Type.SEQUENCE && thisStatePostProcessor.nextEveryStatePreProcessor == null - && this.lastArrivalTime > 0) { - process = false; - } else { - if (this.waitingTime == -1) { - // for time is not defined and event is not received by absent processor - if (thisStatePostProcessor.nextEveryStatePreProcessor == null) { - process = stateEvent.getStreamEvent(this.stateId) == null; - } else { - // Every - if (this.lastArrivalTime > 0) { - process = false; - this.lastArrivalTime = 0; - this.init(); + StreamPreState state = stateHolder.getState(); + try { + boolean process; + if (stateType == StateInputStream.Type.SEQUENCE && + thisStatePostProcessor.nextEveryStatePreProcessor == null && + ((LogicalStreamPreState) state).lastArrivalTime > 0) { + process = false; + } else { + if (this.waitingTime == -1) { + // for time is not defined and event is not received by absent processor + if (thisStatePostProcessor.nextEveryStatePreProcessor == null) { + process = stateEvent.getStreamEvent(this.stateId) == null; } else { - process = true; + // Every + if (((LogicalStreamPreState) state).lastArrivalTime > 0) { + process = false; + ((LogicalStreamPreState) state).lastArrivalTime = 0; + init(); + } else { + process = true; + } } + } else if (stateEvent.getStreamEvent(this.stateId) != null) { + // for time is defined + process = true; + } else { + process = false; } - } else if (stateEvent.getStreamEvent(this.stateId) != null) { - // for time is defined - process = true; - } else { - process = false; } + + return process; + } finally { + stateHolder.returnState(state); + } + } + + class LogicalStreamPreState extends StreamPreState { + + /** + * The timestamp of the last event received by this processor. + */ + private volatile long lastArrivalTime; + + /** + * This flag turns to false after processing the first event if 'every' is not used. + */ + private boolean active = true; + + @Override + public boolean canDestroy() { + return super.canDestroy() && lastArrivalTime == 0; + } + + @Override + public Map snapshot() { + Map snapshot = super.snapshot(); + snapshot.put("IsActive", active); + snapshot.put("LastArrivalTime", lastArrivalTime); + return snapshot; } - return process; + @Override + public void restore(Map state) { + super.restore(state); + active = (Boolean) state.get("IsActive"); + lastArrivalTime = (Long) state.get("LastArrivalTime"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentPreStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentPreStateProcessor.java index c8a71134f8..1aff0cb3c1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentPreStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentPreStateProcessor.java @@ -18,13 +18,13 @@ package io.siddhi.core.query.input.stream.state; +import io.siddhi.core.partition.PartitionCreationListener; import io.siddhi.core.query.processor.SchedulingProcessor; -import io.siddhi.core.util.extension.holder.EternalReferencedHolder; /** * PreStateProcessor of events not received by Siddhi. */ -public interface AbsentPreStateProcessor extends SchedulingProcessor, EternalReferencedHolder { +public interface AbsentPreStateProcessor extends SchedulingProcessor, PartitionCreationListener { /** * Update the timestamp of the event arrived to this processor and met the filter conditions. diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentStreamPostStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentStreamPostStateProcessor.java index 0ba4c2880b..43e0ec4c93 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentStreamPostStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentStreamPostStateProcessor.java @@ -55,10 +55,4 @@ protected void process(StateEvent stateEvent, ComplexEventChunk complexEventChun ((AbsentPreStateProcessor) thisStatePreProcessor).updateLastArrivalTime(streamEvent.getTimestamp()); } - @Override - public PostStateProcessor cloneProcessor(String key) { - AbsentStreamPostStateProcessor streamPostStateProcessor = new AbsentStreamPostStateProcessor(); - cloneProperties(streamPostStateProcessor); - return streamPostStateProcessor; - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentStreamPreStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentStreamPreStateProcessor.java index fdae11708a..cc53cae38e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentStreamPreStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/AbsentStreamPreStateProcessor.java @@ -18,14 +18,14 @@ package io.siddhi.core.query.input.stream.state; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.StateEvent; -import io.siddhi.core.query.input.stream.single.EntryValveProcessor; import io.siddhi.core.util.Scheduler; -import io.siddhi.core.util.parser.SchedulerParser; import io.siddhi.query.api.execution.query.input.stream.StateInputStream; import java.util.Iterator; +import java.util.Map; /** * Pre processor of not operator. @@ -42,17 +42,6 @@ public class AbsentStreamPreStateProcessor extends StreamPreStateProcessor imple */ private long waitingTime = -1; - /** - * The timestamp of the last event scheduled by this processor. - */ - private long lastScheduledTime; - - /** - * This flag turns to false after processing the first event if 'every' is not used. - * This is used to process only one pattern if 'every' is not used. - */ - private boolean active = true; - /** * Construct an AbsentStreamPreStateProcessor object. * @@ -66,34 +55,45 @@ public AbsentStreamPreStateProcessor(StateInputStream.Type stateType, this.waitingTime = waitingTime; } + public void init(SiddhiQueryContext siddhiQueryContext) { + this.siddhiQueryContext = siddhiQueryContext; + this.stateHolder = siddhiQueryContext.generateStateHolder( + this.getClass().getName(), + false, () -> new LogicalStreamPreState()); + } + @Override public void updateLastArrivalTime(long timestamp) { - synchronized (this) { - this.lastScheduledTime = timestamp + waitingTime; - scheduler.notifyAt(lastScheduledTime); + LogicalStreamPreState state = (LogicalStreamPreState) stateHolder.getState(); + this.lock.lock(); + try { + state.lastScheduledTime = timestamp + waitingTime; + scheduler.notifyAt(state.lastScheduledTime); + } finally { + this.lock.unlock(); + stateHolder.returnState(state); } } - @Override - public void addState(StateEvent stateEvent) { - - if (!this.active) { + protected void addState(StateEvent stateEvent, StreamPreState preState) { + LogicalStreamPreState state = (LogicalStreamPreState) preState; + if (!state.active) { // 'every' keyword is not used and already a pattern is processed return; } lock.lock(); try { if (stateType == StateInputStream.Type.SEQUENCE) { - newAndEveryStateEventList.clear(); - newAndEveryStateEventList.add(stateEvent); + state.getNewAndEveryStateEventList().clear(); + state.getNewAndEveryStateEventList().add(stateEvent); } else { - newAndEveryStateEventList.add(stateEvent); + state.getNewAndEveryStateEventList().add(stateEvent); } // If this is the first processor, nothing to receive from previous patterns if (!isStartState) { // Start the scheduler - lastScheduledTime = stateEvent.getTimestamp() + waitingTime; - scheduler.notifyAt(lastScheduledTime); + state.lastScheduledTime = stateEvent.getTimestamp() + waitingTime; + scheduler.notifyAt(state.lastScheduledTime); } } finally { lock.unlock(); @@ -102,30 +102,33 @@ public void addState(StateEvent stateEvent) { @Override public void addEveryState(StateEvent stateEvent) { + LogicalStreamPreState state = (LogicalStreamPreState) stateHolder.getState(); lock.lock(); try { StateEvent clonedEvent = stateEventCloner.copyStateEvent(stateEvent); - newAndEveryStateEventList.add(clonedEvent); + state.getNewAndEveryStateEventList().add(clonedEvent); // Start the scheduler - lastScheduledTime = stateEvent.getTimestamp() + waitingTime; - scheduler.notifyAt(lastScheduledTime); + state.lastScheduledTime = stateEvent.getTimestamp() + waitingTime; + scheduler.notifyAt(state.lastScheduledTime); } finally { lock.unlock(); + stateHolder.returnState(state); } } @Override public void resetState() { + LogicalStreamPreState state = (LogicalStreamPreState) stateHolder.getState(); lock.lock(); try { // Clear the events added by the previous processor - pendingStateEventList.clear(); + state.getPendingStateEventList().clear(); if (isStartState) { if (stateType == StateInputStream.Type.SEQUENCE && thisStatePostProcessor.nextEveryStatePreProcessor == null && !((StreamPreStateProcessor) thisStatePostProcessor.nextStatePreProcessor) - .pendingStateEventList.isEmpty()) { + .getPendingStateEventList().isEmpty()) { // Sequence without 'every' keyword and the next processor has pending events to be processed return; } @@ -134,82 +137,90 @@ public void resetState() { } } finally { lock.unlock(); + stateHolder.returnState(state); } } @Override public void process(ComplexEventChunk complexEventChunk) { - if (!this.active) { - // Every keyword is not used and already a pattern is processed - return; - } - boolean notProcessed = true; - long currentTime = complexEventChunk.getFirst().getTimestamp(); - ComplexEventChunk retEventChunk = new ComplexEventChunk<>(false); - lock.lock(); + LogicalStreamPreState state = (LogicalStreamPreState) stateHolder.getState(); try { - // If the process method is called, it is guaranteed that the waitingTime is passed - boolean initialize = isStartState && newAndEveryStateEventList.isEmpty() - && pendingStateEventList.isEmpty(); - if (initialize && stateType == StateInputStream.Type.SEQUENCE && - thisStatePostProcessor.nextEveryStatePreProcessor == null && this.lastScheduledTime > 0) { - // Sequence with no every but an event arrived - initialize = false; + if (!state.active) { + // Every keyword is not used and already a pattern is processed + return; } + boolean notProcessed = true; + long currentTime = complexEventChunk.getFirst().getTimestamp(); + ComplexEventChunk retEventChunk = new ComplexEventChunk<>(false); + lock.lock(); + try { + // If the process method is called, it is guaranteed that the waitingTime is passed + boolean initialize = isStartState && state.getNewAndEveryStateEventList().isEmpty() + && state.getPendingStateEventList().isEmpty(); + if (initialize && stateType == StateInputStream.Type.SEQUENCE && + thisStatePostProcessor.nextEveryStatePreProcessor == null && state.lastScheduledTime > 0) { + // Sequence with no every but an event arrived + initialize = false; + } - if (initialize) { - // This is the first processor and no events received so far - StateEvent stateEvent = stateEventPool.borrowEvent(); - addState(stateEvent); - } else if (stateType == StateInputStream.Type.SEQUENCE && !newAndEveryStateEventList.isEmpty()) { - this.resetState(); - } - this.updateState(); - - Iterator iterator = pendingStateEventList.iterator(); - while (iterator.hasNext()) { - StateEvent event = iterator.next(); - // Remove expired events based on within - if (isExpired(event, currentTime)) { - iterator.remove(); - if (withinEveryPreStateProcessor != null - && thisStatePostProcessor.nextEveryStatePreProcessor != this) { - thisStatePostProcessor.nextEveryStatePreProcessor.addEveryState(event); + if (initialize) { + // This is the first processor and no events received so far + StateEvent stateEvent = stateEventFactory.newInstance(); + addState(stateEvent); + } else if (stateType == StateInputStream.Type.SEQUENCE && + !state.getNewAndEveryStateEventList().isEmpty()) { + this.resetState(); + } + this.updateState(); + + Iterator iterator = state.getPendingStateEventList().iterator(); + while (iterator.hasNext()) { + StateEvent event = iterator.next(); + // Remove expired events based on within + if (isExpired(event, currentTime)) { + iterator.remove(); + if (withinEveryPreStateProcessor != null + && thisStatePostProcessor.nextEveryStatePreProcessor != this) { + thisStatePostProcessor.nextEveryStatePreProcessor.addEveryState(event); + } + continue; + } + // Collect the events that came before the waiting time + if (event.getTimestamp() == -1 && currentTime >= state.lastScheduledTime || + event.getTimestamp() != -1 && currentTime >= event.getTimestamp() + waitingTime) { + iterator.remove(); + event.setTimestamp(currentTime); + retEventChunk.add(event); } - continue; } - // Collect the events that came before the waiting time - if (event.getTimestamp() == -1 && currentTime >= lastScheduledTime || - event.getTimestamp() != -1 && currentTime >= event.getTimestamp() + waitingTime) { - iterator.remove(); - event.setTimestamp(currentTime); - retEventChunk.add(event); + if (withinEveryPreStateProcessor != null) { + withinEveryPreStateProcessor.updateState(); } + } finally { + lock.unlock(); } - if (withinEveryPreStateProcessor != null) { - withinEveryPreStateProcessor.updateState(); + + notProcessed = retEventChunk.getFirst() == null; + while (retEventChunk.hasNext()) { + StateEvent stateEvent = retEventChunk.next(); + retEventChunk.remove(); + sendEvent(stateEvent, state); } - } finally { - lock.unlock(); - } - notProcessed = retEventChunk.getFirst() == null; - while (retEventChunk.hasNext()) { - StateEvent stateEvent = retEventChunk.next(); - retEventChunk.remove(); - sendEvent(stateEvent); - } - long actualCurrentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); - if (actualCurrentTime > waitingTime + currentTime) { - lastScheduledTime = actualCurrentTime + waitingTime; - } - if (notProcessed && lastScheduledTime < currentTime) { - lastScheduledTime = currentTime + waitingTime; - this.scheduler.notifyAt(lastScheduledTime); + long actualCurrentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); + if (actualCurrentTime > waitingTime + currentTime) { + state.lastScheduledTime = actualCurrentTime + waitingTime; + } + if (notProcessed && state.lastScheduledTime < currentTime) { + state.lastScheduledTime = currentTime + waitingTime; + this.scheduler.notifyAt(state.lastScheduledTime); + } + } finally { + stateHolder.returnState(state); } } - private void sendEvent(StateEvent stateEvent) { + private void sendEvent(StateEvent stateEvent, LogicalStreamPreState state) { if (thisStatePostProcessor.nextProcessor != null) { thisStatePostProcessor.nextProcessor.process(new ComplexEventChunk<>(stateEvent, stateEvent, false)); } @@ -219,7 +230,7 @@ private void sendEvent(StateEvent stateEvent) { if (thisStatePostProcessor.nextEveryStatePreProcessor != null) { thisStatePostProcessor.nextEveryStatePreProcessor.addEveryState(stateEvent); } else if (isStartState) { - this.active = false; + state.active = false; } if (thisStatePostProcessor.callbackPreStateProcessor != null) { @@ -229,18 +240,22 @@ private void sendEvent(StateEvent stateEvent) { @Override public ComplexEventChunk processAndReturn(ComplexEventChunk complexEventChunk) { + LogicalStreamPreState state = (LogicalStreamPreState) stateHolder.getState(); + try { + if (!state.active) { + return new ComplexEventChunk<>(false); + } + ComplexEventChunk event = super.processAndReturn(complexEventChunk); - if (!this.active) { - return new ComplexEventChunk<>(false); - } - ComplexEventChunk event = super.processAndReturn(complexEventChunk); - - StateEvent firstEvent = event.getFirst(); - if (firstEvent != null) { - event = new ComplexEventChunk<>(false); + StateEvent firstEvent = event.getFirst(); + if (firstEvent != null) { + event = new ComplexEventChunk<>(false); + } + // Always return an empty event + return event; + } finally { + stateHolder.returnState(state); } - // Always return an empty event - return event; } protected boolean removeOnNoStateChange(StateInputStream.Type stateType) { @@ -258,37 +273,56 @@ public void setScheduler(Scheduler scheduler) { } @Override - public PreStateProcessor cloneProcessor(String key) { - AbsentStreamPreStateProcessor streamPreStateProcessor = new AbsentStreamPreStateProcessor(stateType, - waitingTime); - cloneProperties(streamPreStateProcessor, key); - streamPreStateProcessor.init(siddhiQueryContext); - - // Set the scheduler - siddhiQueryContext.getSiddhiAppContext().addEternalReferencedHolder(streamPreStateProcessor); - EntryValveProcessor entryValveProcessor = new EntryValveProcessor(siddhiQueryContext.getSiddhiAppContext()); - entryValveProcessor.setToLast(streamPreStateProcessor); - Scheduler scheduler = SchedulerParser.parse(entryValveProcessor, siddhiQueryContext.getSiddhiAppContext()); - streamPreStateProcessor.setScheduler(scheduler); - return streamPreStateProcessor; - } - - @Override - public void start() { + public void partitionCreated() { // Start automatically only if it is the start state and 'for' time is defined // Otherwise, scheduler will be started in the addState method - if (isStartState && waitingTime != -1 && active) { - synchronized (this) { - lastScheduledTime = this.siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime() - + waitingTime; - this.scheduler.notifyAt(lastScheduledTime); + LogicalStreamPreState state = (LogicalStreamPreState) stateHolder.getState(); + + try { + if (!state.isStarted()) { + state.started(); + if (isStartState && waitingTime != -1 && state.active) { + state.lastScheduledTime = this.siddhiQueryContext.getSiddhiAppContext() + .getTimestampGenerator().currentTime() + waitingTime; + this.scheduler.notifyAt(state.lastScheduledTime); + } } + } finally { + stateHolder.returnState(state); } } - @Override - public void stop() { - // Scheduler will be stopped automatically - // Nothing to stop here + class LogicalStreamPreState extends StreamPreState { + + /** + * The timestamp of the last event scheduled by this processor. + */ + private long lastScheduledTime; + + /** + * This flag turns to false after processing the first event if 'every' is not used. + * This is used to process only one pattern if 'every' is not used. + */ + private boolean active = true; + + @Override + public boolean canDestroy() { + return super.canDestroy(); + } + + @Override + public Map snapshot() { + Map snapshot = super.snapshot(); + snapshot.put("IsActive", active); + snapshot.put("LastScheduledTime", lastScheduledTime); + return snapshot; + } + + @Override + public void restore(Map state) { + super.restore(state); + active = (Boolean) state.get("IsActive"); + lastScheduledTime = (Long) state.get("LastScheduledTime"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/CountPostStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/CountPostStateProcessor.java index 31d196ec9a..34339d29cf 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/CountPostStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/CountPostStateProcessor.java @@ -36,12 +36,6 @@ public CountPostStateProcessor(int minCount, int maxCount) { this.maxCount = maxCount; } - public PostStateProcessor cloneProcessor(String key) { - CountPostStateProcessor countPostStateProcessor = new CountPostStateProcessor(minCount, maxCount); - cloneProperties(countPostStateProcessor); - return countPostStateProcessor; - } - protected void process(StateEvent stateEvent, ComplexEventChunk complexEventChunk) { StreamEvent streamEvent = stateEvent.getStreamEvent(stateId); int streamEvents = 1; diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/CountPreStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/CountPreStateProcessor.java index 0f592f03be..e8e55618db 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/CountPreStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/CountPreStateProcessor.java @@ -18,12 +18,14 @@ package io.siddhi.core.query.input.stream.state; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.query.api.execution.query.input.stream.StateInputStream; import java.util.Iterator; +import java.util.Map; /** * Created on 1/6/15. @@ -31,9 +33,7 @@ public class CountPreStateProcessor extends StreamPreStateProcessor { private final int minCount; private final int maxCount; - protected volatile boolean successCondition = false; private CountPostStateProcessor countPostStateProcessor; - private volatile boolean startStateReset = false; public CountPreStateProcessor(int minCount, int maxCount, StateInputStream.Type stateType) { super(stateType); @@ -41,12 +41,11 @@ public CountPreStateProcessor(int minCount, int maxCount, StateInputStream.Type this.maxCount = maxCount; } - - public PreStateProcessor cloneProcessor(String key) { - CountPreStateProcessor countPreStateProcessor = new CountPreStateProcessor(minCount, maxCount, stateType); - cloneProperties(countPreStateProcessor, key); - countPreStateProcessor.init(siddhiQueryContext); - return countPreStateProcessor; + public void init(SiddhiQueryContext siddhiQueryContext) { + this.siddhiQueryContext = siddhiQueryContext; + this.stateHolder = siddhiQueryContext.generateStateHolder( + this.getClass().getName(), + false, () -> new CountStreamPreState()); } @Override @@ -54,9 +53,10 @@ public ComplexEventChunk processAndReturn(ComplexEventChunk complexE ComplexEventChunk returnEventChunk = new ComplexEventChunk(false); complexEventChunk.reset(); StreamEvent streamEvent = (StreamEvent) complexEventChunk.next(); //Sure only one will be sent + CountStreamPreState state = (CountStreamPreState) stateHolder.getState(); lock.lock(); try { - for (Iterator iterator = pendingStateEventList.iterator(); iterator.hasNext(); ) { + for (Iterator iterator = state.getPendingStateEventList().iterator(); iterator.hasNext(); ) { StateEvent stateEvent = iterator.next(); if (removeIfNextStateProcessed(stateEvent, iterator, stateId + 1)) { continue; @@ -65,16 +65,16 @@ public ComplexEventChunk processAndReturn(ComplexEventChunk complexE continue; } stateEvent.addEvent(stateId, streamEventCloner.copyStreamEvent(streamEvent)); - successCondition = false; + state.successCondition = false; process(stateEvent); if (this.thisLastProcessor.isEventReturned()) { this.thisLastProcessor.clearProcessedEvent(); returnEventChunk.add(stateEvent); } - if (stateChanged) { + if (state.isStateChanged()) { iterator.remove(); } - if (!successCondition) { + if (!state.successCondition) { switch (stateType) { case PATTERN: stateEvent.removeLastEvent(stateId); @@ -88,6 +88,7 @@ public ComplexEventChunk processAndReturn(ComplexEventChunk complexE } } finally { lock.unlock(); + stateHolder.returnState(state); } return returnEventChunk; } @@ -101,12 +102,15 @@ private boolean removeIfNextStateProcessed(StateEvent stateEvent, Iterator eventChunk = state.getCurrentStateEventChunk(); + eventChunk.clear(); + eventChunk.add(stateEvent); + countPostStateProcessor.processMinCountReached(stateEvent, eventChunk); + eventChunk.clear(); } } @@ -140,18 +145,56 @@ public void setCountPostStateProcessor(CountPostStateProcessor countPostStatePro } public void startStateReset() { - startStateReset = true; - if (thisStatePostProcessor.callbackPreStateProcessor != null) { - ((CountPreStateProcessor) countPostStateProcessor.thisStatePreProcessor).startStateReset(); + CountStreamPreState state = (CountStreamPreState) stateHolder.getState(); + try { + state.startStateReset = true; + if (thisStatePostProcessor.callbackPreStateProcessor != null) { + ((CountPreStateProcessor) countPostStateProcessor.thisStatePreProcessor).startStateReset(); + } + } finally { + stateHolder.returnState(state); } + } @Override public void updateState() { - if (startStateReset) { - startStateReset = false; - init(); + CountStreamPreState state = (CountStreamPreState) stateHolder.getState(); + try { + if (state.startStateReset) { + state.startStateReset = false; + init(); + } + super.updateState(); + } finally { + stateHolder.returnState(state); + } + } + + class CountStreamPreState extends StreamPreState { + + protected volatile boolean successCondition = false; + + private volatile boolean startStateReset = false; + + @Override + public boolean canDestroy() { + return super.canDestroy(); + } + + @Override + public Map snapshot() { + Map snapshot = super.snapshot(); + snapshot.put("SuccessCondition", successCondition); + snapshot.put("StartStateReset", startStateReset); + return snapshot; + } + + @Override + public void restore(Map state) { + super.restore(state); + successCondition = (Boolean) state.get("SuccessCondition"); + startStateReset = (Boolean) state.get("StartStateReset"); } - super.updateState(); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/LogicalPostStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/LogicalPostStateProcessor.java index 7231039d8e..37d916c033 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/LogicalPostStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/LogicalPostStateProcessor.java @@ -118,19 +118,6 @@ public void setToLast(Processor processor) { } } - /** - * Clone a copy of processor - * - * @param key partition key - * @return clone of PostStateProcessor - */ - @Override - public PostStateProcessor cloneProcessor(String key) { - LogicalPostStateProcessor logicalPostStateProcessor = new LogicalPostStateProcessor(type); - cloneProperties(logicalPostStateProcessor); - return logicalPostStateProcessor; - } - public void setNextStatePreProcessor(PreStateProcessor preStateProcessor) { this.nextStatePreProcessor = preStateProcessor; partnerPostStateProcessor.nextStatePreProcessor = preStateProcessor; diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/LogicalPreStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/LogicalPreStateProcessor.java index 0c44624a02..e590f69b8f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/LogicalPreStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/LogicalPreStateProcessor.java @@ -39,35 +39,20 @@ public LogicalPreStateProcessor(LogicalStateElement.Type type, StateInputStream. this.logicalType = type; } - /** - * Clone a copy of processor - * - * @param key partition key - * @return clone of LogicalPreStateProcessor - */ - @Override - public PreStateProcessor cloneProcessor(String key) { - LogicalPreStateProcessor logicalPreStateProcessor = new LogicalPreStateProcessor(logicalType, stateType); - cloneProperties(logicalPreStateProcessor, key); - logicalPreStateProcessor.init(siddhiQueryContext); - return logicalPreStateProcessor; - } - - @Override - public void addState(StateEvent stateEvent) { + protected void addState(StateEvent stateEvent, StreamPreState state) { lock.lock(); try { if (isStartState || stateType == StateInputStream.Type.SEQUENCE) { - if (newAndEveryStateEventList.isEmpty()) { - newAndEveryStateEventList.add(stateEvent); + if (state.getNewAndEveryStateEventList().isEmpty()) { + state.getNewAndEveryStateEventList().add(stateEvent); } - if (partnerStatePreProcessor != null && partnerStatePreProcessor.newAndEveryStateEventList.isEmpty()) { - partnerStatePreProcessor.newAndEveryStateEventList.add(stateEvent); + if (partnerStatePreProcessor != null && partnerStatePreProcessor.isNewAndEveryStateEventListEmpty()) { + partnerStatePreProcessor.addEventToNewAndEveryStateEventList(stateEvent); } } else { - newAndEveryStateEventList.add(stateEvent); + state.getNewAndEveryStateEventList().add(stateEvent); if (partnerStatePreProcessor != null) { - partnerStatePreProcessor.newAndEveryStateEventList.add(stateEvent); + partnerStatePreProcessor.addEventToNewAndEveryStateEventList(stateEvent); } } } finally { @@ -79,32 +64,35 @@ public void addState(StateEvent stateEvent) { public void addEveryState(StateEvent stateEvent) { StateEvent clonedEvent = stateEventCloner.copyStateEvent(stateEvent); clonedEvent.setEvent(stateId, null); + StreamPreState state = stateHolder.getState(); lock.lock(); try { - newAndEveryStateEventList.add(clonedEvent); + state.getNewAndEveryStateEventList().add(clonedEvent); if (partnerStatePreProcessor != null) { clonedEvent.setEvent(partnerStatePreProcessor.stateId, null); - partnerStatePreProcessor.newAndEveryStateEventList.add(clonedEvent); + partnerStatePreProcessor.addEventToNewAndEveryStateEventList(clonedEvent); } } finally { lock.unlock(); + stateHolder.returnState(state); } } @Override public void resetState() { + StreamPreState state = stateHolder.getState(); lock.lock(); try { - if (logicalType == LogicalStateElement.Type.OR || pendingStateEventList.size() == - partnerStatePreProcessor.pendingStateEventList.size()) { - pendingStateEventList.clear(); - partnerStatePreProcessor.pendingStateEventList.clear(); + if (logicalType == LogicalStateElement.Type.OR || state.getPendingStateEventList().size() == + partnerStatePreProcessor.getPendingStateEventList().size()) { + state.getPendingStateEventList().clear(); + partnerStatePreProcessor.getPendingStateEventList().clear(); - if (isStartState && newAndEveryStateEventList.isEmpty()) { + if (isStartState && state.getNewAndEveryStateEventList().isEmpty()) { if (stateType == StateInputStream.Type.SEQUENCE && thisStatePostProcessor.nextEveryStatePreProcessor == null && !((StreamPreStateProcessor) thisStatePostProcessor.nextStatePreProcessor) - .pendingStateEventList.isEmpty()) { + .getPendingStateEventList().isEmpty()) { return; } init(); @@ -112,20 +100,21 @@ public void resetState() { } } finally { lock.unlock(); + stateHolder.returnState(state); } } @Override public void updateState() { + StreamPreState state = stateHolder.getState(); lock.lock(); try { - pendingStateEventList.addAll(newAndEveryStateEventList); - newAndEveryStateEventList.clear(); - - partnerStatePreProcessor.pendingStateEventList.addAll(partnerStatePreProcessor.newAndEveryStateEventList); - partnerStatePreProcessor.newAndEveryStateEventList.clear(); + state.getPendingStateEventList().addAll(state.getNewAndEveryStateEventList()); + state.getNewAndEveryStateEventList().clear(); + partnerStatePreProcessor.moveAllNewAndEveryStateEventListEventsToPendingStateEventList(); } finally { lock.unlock(); + stateHolder.returnState(state); } } @@ -134,9 +123,10 @@ public ComplexEventChunk processAndReturn(ComplexEventChunk complexE ComplexEventChunk returnEventChunk = new ComplexEventChunk(false); complexEventChunk.reset(); StreamEvent streamEvent = (StreamEvent) complexEventChunk.next(); //Sure only one will be sent + StreamPreState state = stateHolder.getState(); lock.lock(); try { - for (Iterator iterator = pendingStateEventList.iterator(); iterator.hasNext(); ) { + for (Iterator iterator = state.getPendingStateEventList().iterator(); iterator.hasNext(); ) { StateEvent stateEvent = iterator.next(); if (isExpired(stateEvent, streamEvent.getTimestamp())) { iterator.remove(); @@ -157,7 +147,7 @@ public ComplexEventChunk processAndReturn(ComplexEventChunk complexE this.thisLastProcessor.clearProcessedEvent(); returnEventChunk.add(stateEvent); } - if (stateChanged) { + if (state.isStateChanged()) { iterator.remove(); } else { switch (stateType) { @@ -173,6 +163,7 @@ public ComplexEventChunk processAndReturn(ComplexEventChunk complexE } } finally { lock.unlock(); + stateHolder.returnState(state); } return returnEventChunk; } @@ -181,4 +172,32 @@ public void setPartnerStatePreProcessor(LogicalPreStateProcessor partnerStatePre this.partnerStatePreProcessor = partnerStatePreProcessor; partnerStatePreProcessor.lock = lock; } + + public void moveAllNewAndEveryStateEventListEventsToPendingStateEventList() { + StreamPreState state = stateHolder.getState(); + try { + state.getPendingStateEventList().addAll(state.getNewAndEveryStateEventList()); + state.getNewAndEveryStateEventList().clear(); + } finally { + stateHolder.returnState(state); + } + } + + public boolean isNewAndEveryStateEventListEmpty() { + StreamPreState state = stateHolder.getState(); + try { + return state.getNewAndEveryStateEventList().isEmpty(); + } finally { + stateHolder.returnState(state); + } + } + + public void addEventToNewAndEveryStateEventList(StateEvent event) { + StreamPreState state = stateHolder.getState(); + try { + state.getNewAndEveryStateEventList().add(event); + } finally { + stateHolder.returnState(state); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/PostStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/PostStateProcessor.java index 487f511132..a5339aed82 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/PostStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/PostStateProcessor.java @@ -33,6 +33,4 @@ public interface PostStateProcessor extends Processor { void setCallbackPreStateProcessor(CountPreStateProcessor callbackPreStateProcessor); - PostStateProcessor cloneProcessor(String key); - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/PreStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/PreStateProcessor.java index f2178cd55f..6898efcaee 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/PreStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/PreStateProcessor.java @@ -55,6 +55,4 @@ public interface PreStateProcessor extends Processor { void setStartStateIds(int[] stateIds); - PreStateProcessor cloneProcessor(String key); - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StateStreamRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StateStreamRuntime.java index 371d29f106..c82c7a00c8 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StateStreamRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StateStreamRuntime.java @@ -22,11 +22,9 @@ import io.siddhi.core.event.MetaComplexEvent; import io.siddhi.core.event.state.MetaStateEvent; import io.siddhi.core.event.state.StateEvent; -import io.siddhi.core.query.input.ProcessStreamReceiver; +import io.siddhi.core.partition.PartitionCreationListener; import io.siddhi.core.query.input.stream.StreamRuntime; import io.siddhi.core.query.input.stream.single.SingleStreamRuntime; -import io.siddhi.core.query.input.stream.state.receiver.SequenceMultiProcessStreamReceiver; -import io.siddhi.core.query.input.stream.state.receiver.SequenceSingleProcessStreamReceiver; import io.siddhi.core.query.input.stream.state.runtime.InnerStateRuntime; import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; @@ -40,10 +38,9 @@ public class StateStreamRuntime implements StreamRuntime { private MetaStateEvent metaStateEvent; private InnerStateRuntime innerStateRuntime; - private SiddhiQueryContext siddhiQueryContext; + private List startupPreStateProcessors; public StateStreamRuntime(SiddhiQueryContext siddhiQueryContext, MetaStateEvent metaStateEvent) { - this.siddhiQueryContext = siddhiQueryContext; this.metaStateEvent = metaStateEvent; } @@ -51,28 +48,10 @@ public List getSingleStreamRuntimes() { return innerStateRuntime.getSingleStreamRuntimeList(); } - @Override - public StreamRuntime clone(String key) { - StateStreamRuntime stateStreamRuntime = new StateStreamRuntime(siddhiQueryContext, metaStateEvent); - stateStreamRuntime.innerStateRuntime = this.innerStateRuntime.clone(key); - for (SingleStreamRuntime singleStreamRuntime : stateStreamRuntime.getSingleStreamRuntimes()) { - ProcessStreamReceiver processStreamReceiver = singleStreamRuntime.getProcessStreamReceiver(); - if (processStreamReceiver instanceof SequenceMultiProcessStreamReceiver) { - ((SequenceMultiProcessStreamReceiver) processStreamReceiver).setStateStreamRuntime(stateStreamRuntime); - } else if (processStreamReceiver instanceof SequenceSingleProcessStreamReceiver) { - ((SequenceSingleProcessStreamReceiver) processStreamReceiver).setStateStreamRuntime(stateStreamRuntime); - } - } - ((StreamPreStateProcessor) stateStreamRuntime.innerStateRuntime.getFirstProcessor()).setThisLastProcessor( - (StreamPostStateProcessor) - stateStreamRuntime.innerStateRuntime.getLastProcessor()); - return stateStreamRuntime; - } - @Override public void setCommonProcessor(Processor commonProcessor) { innerStateRuntime.setQuerySelector(commonProcessor); - innerStateRuntime.init(); + innerStateRuntime.setup(); } @Override @@ -97,4 +76,17 @@ public void resetAndUpdate() { innerStateRuntime.reset(); innerStateRuntime.update(); } + + public void setStartupPreStateProcessors(List startupPreStateProcessors) { + this.startupPreStateProcessors = startupPreStateProcessors; + } + + public void initPartition() { + innerStateRuntime.init(); + for (PreStateProcessor preStateProcessor : startupPreStateProcessors) { + if (preStateProcessor instanceof PartitionCreationListener) { + ((PartitionCreationListener) preStateProcessor).partitionCreated(); + } + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StreamPostStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StreamPostStateProcessor.java index 60285014b3..b797c1fa2b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StreamPostStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StreamPostStateProcessor.java @@ -113,28 +113,6 @@ public void setToLast(Processor processor) { } } - /** - * Clone a copy of processor - * - * @param key partition key - * @return clone of StreamPostStateProcessor - */ - @Override - public PostStateProcessor cloneProcessor(String key) { - StreamPostStateProcessor streamPostStateProcessor = new StreamPostStateProcessor(); - cloneProperties(streamPostStateProcessor); - return streamPostStateProcessor; - } - - @Override - public void clean() { - //ignore - } - - protected void cloneProperties(StreamPostStateProcessor streamPostStateProcessor) { - streamPostStateProcessor.stateId = stateId; - } - public PreStateProcessor getNextStatePreProcessor() { return nextStatePreProcessor; } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StreamPreStateProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StreamPreStateProcessor.java index 1da6c26612..79a522e8d8 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StreamPreStateProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/StreamPreStateProcessor.java @@ -21,50 +21,46 @@ import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.state.StateEventCloner; -import io.siddhi.core.event.state.StateEventPool; +import io.siddhi.core.event.state.StateEventFactory; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.query.processor.Processor; import io.siddhi.core.util.SiddhiConstants; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.query.api.execution.query.input.stream.StateInputStream; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.concurrent.locks.ReentrantLock; /** * The processor the gets executes before checking state conditions. */ -public class StreamPreStateProcessor implements PreStateProcessor, Snapshotable { +public class StreamPreStateProcessor implements PreStateProcessor { protected int stateId; protected boolean isStartState; - protected volatile boolean stateChanged = false; protected StateInputStream.Type stateType; protected long withinTime = SiddhiConstants.UNKNOWN_STATE; protected int[] startStateIds; protected PreStateProcessor withinEveryPreStateProcessor; - protected String elementId; protected StreamPostStateProcessor thisStatePostProcessor; protected StreamPostStateProcessor thisLastProcessor; - protected Processor nextProcessor; - protected ComplexEventChunk currentStateEventChunk = new ComplexEventChunk(false); - protected LinkedList pendingStateEventList = new LinkedList(); - protected LinkedList newAndEveryStateEventList = new LinkedList(); protected ReentrantLock lock = new ReentrantLock(); - protected StateEventPool stateEventPool; + protected StateEventFactory stateEventFactory; protected StreamEventCloner streamEventCloner; protected StateEventCloner stateEventCloner; - protected StreamEventPool streamEventPool; + protected StreamEventFactory streamEventFactory; protected SiddhiQueryContext siddhiQueryContext; - private boolean initialized; + protected StateHolder stateHolder; public StreamPreStateProcessor(StateInputStream.Type stateType) { this.stateType = stateType; @@ -72,12 +68,9 @@ public StreamPreStateProcessor(StateInputStream.Type stateType) { public void init(SiddhiQueryContext siddhiQueryContext) { this.siddhiQueryContext = siddhiQueryContext; - if (elementId == null) { - this.elementId = "StreamPreStateProcessor-" + - siddhiQueryContext.getSiddhiAppContext().getElementIdGenerator().createNewId(); - } - siddhiQueryContext.getSiddhiAppContext().getSnapshotService().addSnapshotable( - siddhiQueryContext.getName(), this); + this.stateHolder = siddhiQueryContext.generateStateHolder( + this.getClass().getName(), + false, () -> new StreamPreState()); } public StreamPostStateProcessor getThisStatePostProcessor() { @@ -113,11 +106,16 @@ protected boolean isExpired(StateEvent pendingStateEvent, long currentTimestamp) } protected void process(StateEvent stateEvent) { - currentStateEventChunk.add(stateEvent); - currentStateEventChunk.reset(); - stateChanged = false; - nextProcessor.process(currentStateEventChunk); - currentStateEventChunk.reset(); + StreamPreState state = stateHolder.getState(); + try { + state.currentStateEventChunk.add(stateEvent); + state.currentStateEventChunk.reset(); + state.stateChanged = false; + nextProcessor.process(state.currentStateEventChunk); + state.currentStateEventChunk.reset(); + } finally { + stateHolder.returnState(state); + } } /** @@ -155,13 +153,20 @@ public void setToLast(Processor processor) { } public void init() { - if (isStartState && (!initialized || this.thisStatePostProcessor.nextEveryStatePreProcessor != null || - (stateType == StateInputStream.Type.SEQUENCE && - this.thisStatePostProcessor.nextStatePreProcessor instanceof AbsentPreStateProcessor))) { - // For 'every' sequence, the 'thisStatePostProcessor.nextEveryStatePreProcessor != null' check is not enough - StateEvent stateEvent = stateEventPool.borrowEvent(); - addState(stateEvent); - initialized = true; + StreamPreState state = stateHolder.getState(); + try { + if (isStartState && (!state.initialized || + this.thisStatePostProcessor.nextEveryStatePreProcessor != null || + (stateType == StateInputStream.Type.SEQUENCE && this.thisStatePostProcessor.nextStatePreProcessor + instanceof AbsentPreStateProcessor))) { + // For 'every' sequence, the 'thisStatePostProcessor.nextEveryStatePreProcessor != null' + // check is not enough + StateEvent stateEvent = stateEventFactory.newInstance(); + addState(stateEvent); + state.initialized = true; + } + } finally { + stateHolder.returnState(state); } } @@ -173,42 +178,25 @@ public void setThisLastProcessor(StreamPostStateProcessor thisLastProcessor) { this.thisLastProcessor = thisLastProcessor; } - /** - * Clone a copy of processor - * - * @param key partition key - * @return clone of StreamPreStateProcessor - */ @Override - public PreStateProcessor cloneProcessor(String key) { - StreamPreStateProcessor streamPreStateProcessor = new StreamPreStateProcessor(stateType); - cloneProperties(streamPreStateProcessor, key); - streamPreStateProcessor.init(siddhiQueryContext); - return streamPreStateProcessor; - } - - protected void cloneProperties(StreamPreStateProcessor streamPreStateProcessor, String key) { - streamPreStateProcessor.stateId = this.stateId; - streamPreStateProcessor.isStartState = this.isStartState; - streamPreStateProcessor.elementId = this.elementId + "-" + key; - streamPreStateProcessor.stateEventPool = this.stateEventPool; - streamPreStateProcessor.streamEventCloner = this.streamEventCloner; - streamPreStateProcessor.stateEventCloner = this.stateEventCloner; - streamPreStateProcessor.streamEventPool = this.streamEventPool; - streamPreStateProcessor.withinTime = this.withinTime; - streamPreStateProcessor.startStateIds = this.startStateIds; + public void addState(StateEvent stateEvent) { + StreamPreState state = stateHolder.getState(); + try { + addState(stateEvent, state); + } finally { + stateHolder.returnState(state); + } } - @Override - public void addState(StateEvent stateEvent) { + protected void addState(StateEvent stateEvent, StreamPreState state) { lock.lock(); try { if (stateType == StateInputStream.Type.SEQUENCE) { - if (newAndEveryStateEventList.isEmpty()) { - newAndEveryStateEventList.add(stateEvent); + if (state.newAndEveryStateEventList.isEmpty()) { + state.newAndEveryStateEventList.add(stateEvent); } } else { - newAndEveryStateEventList.add(stateEvent); + state.newAndEveryStateEventList.add(stateEvent); } } finally { lock.unlock(); @@ -220,7 +208,12 @@ public void addEveryState(StateEvent stateEvent) { lock.lock(); try { StateEvent clonedEvent = stateEventCloner.copyStateEvent(stateEvent); - newAndEveryStateEventList.add(clonedEvent); + StreamPreState state = stateHolder.getState(); + try { + state.newAndEveryStateEventList.add(clonedEvent); + } finally { + stateHolder.returnState(state); + } } finally { lock.unlock(); } @@ -231,7 +224,12 @@ public void setWithinEveryPreStateProcessor(PreStateProcessor withinEveryPreStat } public void stateChanged() { - stateChanged = true; + StreamPreState state = stateHolder.getState(); + try { + state.stateChanged = true; + } finally { + stateHolder.returnState(state); + } } @Override @@ -243,12 +241,12 @@ public void setStartState(boolean isStartState) { this.isStartState = isStartState; } - public void setStateEventPool(StateEventPool stateEventPool) { - this.stateEventPool = stateEventPool; + public void setStateEventFactory(StateEventFactory stateEventFactory) { + this.stateEventFactory = stateEventFactory; } - public void setStreamEventPool(StreamEventPool streamEventPool) { - this.streamEventPool = streamEventPool; + public void setStreamEventFactory(StreamEventFactory streamEventFactory) { + this.streamEventFactory = streamEventFactory; } public void setStreamEventCloner(StreamEventCloner streamEventCloner) { @@ -261,19 +259,21 @@ public void setStateEventCloner(StateEventCloner stateEventCloner) { @Override public void resetState() { + StreamPreState state = stateHolder.getState(); lock.lock(); try { - pendingStateEventList.clear(); - if (isStartState && newAndEveryStateEventList.isEmpty()) { + state.pendingStateEventList.clear(); + if (isStartState && state.newAndEveryStateEventList.isEmpty()) { if (stateType == StateInputStream.Type.SEQUENCE && thisStatePostProcessor.nextEveryStatePreProcessor == - null && !((StreamPreStateProcessor) thisStatePostProcessor.nextStatePreProcessor) - .pendingStateEventList.isEmpty()) { + null && !((StreamPreStateProcessor) thisStatePostProcessor.nextStatePreProcessor). + getPendingStateEventList().isEmpty()) { return; } init(); } } finally { lock.unlock(); + stateHolder.returnState(state); } } @@ -281,8 +281,14 @@ public void resetState() { public void updateState() { lock.lock(); try { - pendingStateEventList.addAll(newAndEveryStateEventList); - newAndEveryStateEventList.clear(); + StreamPreState state = stateHolder.getState(); + try { + state.pendingStateEventList.addAll(state.newAndEveryStateEventList); + state.newAndEveryStateEventList.clear(); + } finally { + stateHolder.returnState(state); + } + } finally { lock.unlock(); } @@ -293,9 +299,10 @@ public ComplexEventChunk processAndReturn(ComplexEventChunk complexE ComplexEventChunk returnEventChunk = new ComplexEventChunk(false); complexEventChunk.reset(); StreamEvent streamEvent = (StreamEvent) complexEventChunk.next(); //Sure only one will be sent + StreamPreState state = stateHolder.getState(); lock.lock(); try { - for (Iterator iterator = pendingStateEventList.iterator(); iterator.hasNext(); ) { + for (Iterator iterator = state.pendingStateEventList.iterator(); iterator.hasNext(); ) { StateEvent stateEvent = iterator.next(); if (isExpired(stateEvent, streamEvent.getTimestamp())) { iterator.remove(); @@ -311,7 +318,7 @@ public ComplexEventChunk processAndReturn(ComplexEventChunk complexE this.thisLastProcessor.clearProcessedEvent(); returnEventChunk.add(stateEvent); } - if (stateChanged) { + if (state.stateChanged) { iterator.remove(); } else { switch (stateType) { @@ -332,6 +339,7 @@ public ComplexEventChunk processAndReturn(ComplexEventChunk complexE } } finally { lock.unlock(); + stateHolder.returnState(state); } return returnEventChunk; } @@ -349,42 +357,86 @@ public void setStateId(int stateId) { this.stateId = stateId; } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("FirstEvent", currentStateEventChunk.getFirst()); - state.put("PendingStateEventList", pendingStateEventList); - state.put("NewAndEveryStateEventList", newAndEveryStateEventList); - return state; + public void setWithinTime(long withinTime) { + this.withinTime = withinTime; } - @Override - public void restoreState(Map state) { - currentStateEventChunk.clear(); - currentStateEventChunk.add((StateEvent) state.get("FirstEvent")); - pendingStateEventList = (LinkedList) state.get("PendingStateEventList"); - newAndEveryStateEventList = (LinkedList) state.get("NewAndEveryStateEventList"); + public void setStartStateIds(int[] stateIds) { + this.startStateIds = stateIds; } - @Override - public String getElementId() { - return elementId; + public List getPendingStateEventList() { + StreamPreState state = stateHolder.getState(); + try { + return state.pendingStateEventList; + } finally { + stateHolder.returnState(state); + } } - @Override - public void clean() { - if (nextProcessor != null) { - nextProcessor.clean(); + class StreamPreState extends State { + private ComplexEventChunk currentStateEventChunk = new ComplexEventChunk(false); + private LinkedList pendingStateEventList = new LinkedList(); + private LinkedList newAndEveryStateEventList = new LinkedList(); + private volatile boolean stateChanged = false; + private boolean initialized; + private boolean started; + + @Override + public boolean canDestroy() { + return currentStateEventChunk.getFirst() == null && + pendingStateEventList.isEmpty() && + newAndEveryStateEventList.isEmpty() && !initialized; } - siddhiQueryContext.getSiddhiAppContext().getSnapshotService().removeSnapshotable( - siddhiQueryContext.getName(), this); - } - public void setWithinTime(long withinTime) { - this.withinTime = withinTime; - } + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("FirstEvent", currentStateEventChunk.getFirst()); + state.put("PendingStateEventList", pendingStateEventList); + state.put("NewAndEveryStateEventList", newAndEveryStateEventList); + state.put("Initialized", initialized); + state.put("Started", started); + return state; + } - public void setStartStateIds(int[] stateIds) { - this.startStateIds = stateIds; + @Override + public void restore(Map state) { + currentStateEventChunk.clear(); + currentStateEventChunk.add((StateEvent) state.get("FirstEvent")); + pendingStateEventList = (LinkedList) state.get("PendingStateEventList"); + newAndEveryStateEventList = (LinkedList) state.get("NewAndEveryStateEventList"); + initialized = (Boolean) state.get("Initialized"); + started = (Boolean) state.get("Started"); + } + + public ComplexEventChunk getCurrentStateEventChunk() { + return currentStateEventChunk; + } + + public LinkedList getPendingStateEventList() { + return pendingStateEventList; + } + + public LinkedList getNewAndEveryStateEventList() { + return newAndEveryStateEventList; + } + + public boolean isStateChanged() { + return stateChanged; + } + + public void setStateChanged(boolean stateChanged) { + this.stateChanged = stateChanged; + } + + public void started() { + started = true; + } + + public boolean isStarted() { + return started; + } } + } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/PatternMultiProcessStreamReceiver.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/PatternMultiProcessStreamReceiver.java index eca517f441..66beb889fe 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/PatternMultiProcessStreamReceiver.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/PatternMultiProcessStreamReceiver.java @@ -19,13 +19,9 @@ package io.siddhi.core.query.input.stream.state.receiver; import io.siddhi.core.config.SiddhiQueryContext; -import io.siddhi.core.event.ComplexEvent; -import io.siddhi.core.event.Event; import io.siddhi.core.query.input.StateMultiProcessStreamReceiver; import io.siddhi.core.query.input.stream.state.PreStateProcessor; -import java.util.List; - /** * {StreamJunction.Receiver} implementation to receive events into pattern queries * with multiple streams. @@ -33,8 +29,8 @@ public class PatternMultiProcessStreamReceiver extends StateMultiProcessStreamReceiver { public PatternMultiProcessStreamReceiver(String streamId, int processCount, - SiddhiQueryContext siddhiQueryContext) { - super(streamId, processCount, siddhiQueryContext); + Object patternSyncObject, SiddhiQueryContext siddhiQueryContext) { + super(streamId, processCount, patternSyncObject, siddhiQueryContext); eventSequence = new int[processCount]; int count = 0; for (int i = eventSequence.length - 1; i >= 0; i--) { @@ -43,11 +39,6 @@ public PatternMultiProcessStreamReceiver(String streamId, int processCount, } } - public PatternMultiProcessStreamReceiver clone(String key) { - return new PatternMultiProcessStreamReceiver(streamId + key, processCount, - siddhiQueryContext); - } - protected void stabilizeStates() { if (stateProcessorsSize != 0) { for (PreStateProcessor preStateProcessor : stateProcessors) { @@ -55,29 +46,4 @@ protected void stabilizeStates() { } } } - - @Override - public void receive(ComplexEvent complexEvent) { - super.receive(complexEvent); - } - - @Override - public void receive(Event event) { - super.receive(event); - } - - @Override - public void receive(Event[] events) { - super.receive(events); - } - - @Override - public void receive(List events) { - super.receive(events); - } - - @Override - public void receive(long timestamp, Object[] data) { - super.receive(timestamp, data); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/PatternSingleProcessStreamReceiver.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/PatternSingleProcessStreamReceiver.java index 731d6585ee..c6bc223d1f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/PatternSingleProcessStreamReceiver.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/PatternSingleProcessStreamReceiver.java @@ -18,25 +18,17 @@ package io.siddhi.core.query.input.stream.state.receiver; import io.siddhi.core.config.SiddhiQueryContext; -import io.siddhi.core.event.ComplexEvent; -import io.siddhi.core.event.Event; import io.siddhi.core.query.input.SingleProcessStreamReceiver; -import java.util.List; - /** * {StreamJunction.Receiver} implementation to receive events into pattern queries * with single stream. */ public class PatternSingleProcessStreamReceiver extends SingleProcessStreamReceiver { - public PatternSingleProcessStreamReceiver(String streamId, String lockKey, SiddhiQueryContext siddhiQueryContext) { - super(streamId, lockKey, siddhiQueryContext); - } - - public PatternSingleProcessStreamReceiver clone(String key) { - return new PatternSingleProcessStreamReceiver(streamId + key, key, - siddhiQueryContext); + public PatternSingleProcessStreamReceiver(String streamId, Object patternSyncObject, + SiddhiQueryContext siddhiQueryContext) { + super(streamId, patternSyncObject, siddhiQueryContext); } protected void stabilizeStates() { @@ -44,29 +36,4 @@ protected void stabilizeStates() { stateProcessors.get(0).updateState(); } } - - @Override - public void receive(ComplexEvent complexEvent) { - super.receive(complexEvent); - } - - @Override - public void receive(Event event) { - super.receive(event); - } - - @Override - public void receive(Event[] events) { - super.receive(events); - } - - @Override - public void receive(List events) { - super.receive(events); - } - - @Override - public void receive(long timestamp, Object[] data) { - super.receive(timestamp, data); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/SequenceMultiProcessStreamReceiver.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/SequenceMultiProcessStreamReceiver.java index c25a308335..58ff205a7a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/SequenceMultiProcessStreamReceiver.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/SequenceMultiProcessStreamReceiver.java @@ -19,13 +19,9 @@ package io.siddhi.core.query.input.stream.state.receiver; import io.siddhi.core.config.SiddhiQueryContext; -import io.siddhi.core.event.ComplexEvent; -import io.siddhi.core.event.Event; import io.siddhi.core.query.input.StateMultiProcessStreamReceiver; import io.siddhi.core.query.input.stream.state.StateStreamRuntime; -import java.util.List; - /** * {StreamJunction.Receiver} implementation to receive events into sequence queries * with multiple streams. @@ -35,8 +31,8 @@ public class SequenceMultiProcessStreamReceiver extends StateMultiProcessStreamR private StateStreamRuntime stateStreamRuntime; public SequenceMultiProcessStreamReceiver(String streamId, int processCount, StateStreamRuntime - stateStreamRuntime, SiddhiQueryContext siddhiQueryContext) { - super(streamId, processCount, siddhiQueryContext); + stateStreamRuntime, Object patternSyncObject, SiddhiQueryContext siddhiQueryContext) { + super(streamId, processCount, patternSyncObject, siddhiQueryContext); this.stateStreamRuntime = stateStreamRuntime; eventSequence = new int[processCount]; int count = 0; @@ -46,41 +42,8 @@ public SequenceMultiProcessStreamReceiver(String streamId, int processCount, Sta } } - public SequenceMultiProcessStreamReceiver clone(String key) { - return new SequenceMultiProcessStreamReceiver(streamId + key, processCount, null, - siddhiQueryContext); - } - - public void setStateStreamRuntime(StateStreamRuntime stateStreamRuntime) { - this.stateStreamRuntime = stateStreamRuntime; - } - protected void stabilizeStates() { stateStreamRuntime.resetAndUpdate(); } - @Override - public void receive(ComplexEvent complexEvent) { - super.receive(complexEvent); - } - - @Override - public void receive(Event event) { - super.receive(event); - } - - @Override - public void receive(Event[] events) { - super.receive(events); - } - - @Override - public void receive(List events) { - super.receive(events); - } - - @Override - public void receive(long timestamp, Object[] data) { - super.receive(timestamp, data); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/SequenceSingleProcessStreamReceiver.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/SequenceSingleProcessStreamReceiver.java index ff7160c357..2205c317ae 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/SequenceSingleProcessStreamReceiver.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/receiver/SequenceSingleProcessStreamReceiver.java @@ -19,13 +19,9 @@ package io.siddhi.core.query.input.stream.state.receiver; import io.siddhi.core.config.SiddhiQueryContext; -import io.siddhi.core.event.ComplexEvent; -import io.siddhi.core.event.Event; import io.siddhi.core.query.input.SingleProcessStreamReceiver; import io.siddhi.core.query.input.stream.state.StateStreamRuntime; -import java.util.List; - /** * {StreamJunction.Receiver} implementation to receive events into sequence queries * with single stream. @@ -35,46 +31,13 @@ public class SequenceSingleProcessStreamReceiver extends SingleProcessStreamRece private StateStreamRuntime stateStreamRuntime; public SequenceSingleProcessStreamReceiver(String streamId, StateStreamRuntime stateStreamRuntime, - String lockKey, SiddhiQueryContext siddhiQueryContext) { - super(streamId, lockKey, siddhiQueryContext); - this.stateStreamRuntime = stateStreamRuntime; - } - - public void setStateStreamRuntime(StateStreamRuntime stateStreamRuntime) { + Object patternSyncObject, SiddhiQueryContext siddhiQueryContext) { + super(streamId, patternSyncObject, siddhiQueryContext); this.stateStreamRuntime = stateStreamRuntime; } - public SequenceSingleProcessStreamReceiver clone(String key) { - return new SequenceSingleProcessStreamReceiver(streamId + key, null, key, - siddhiQueryContext); - } - protected void stabilizeStates() { stateStreamRuntime.resetAndUpdate(); } - @Override - public void receive(ComplexEvent complexEvent) { - super.receive(complexEvent); - } - - @Override - public void receive(Event event) { - super.receive(event); - } - - @Override - public void receive(Event[] events) { - super.receive(events); - } - - @Override - public void receive(List events) { - super.receive(events); - } - - @Override - public void receive(long timestamp, Object[] data) { - super.receive(timestamp, data); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/CountInnerStateRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/CountInnerStateRuntime.java index 2edde77ec7..e57ffbb1dd 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/CountInnerStateRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/CountInnerStateRuntime.java @@ -18,34 +18,17 @@ package io.siddhi.core.query.input.stream.state.runtime; -import io.siddhi.core.query.input.stream.state.CountPostStateProcessor; -import io.siddhi.core.query.input.stream.state.CountPreStateProcessor; - /** * Created on 12/19/14. */ public class CountInnerStateRuntime extends StreamInnerStateRuntime { - protected StreamInnerStateRuntime streamInnerStateRuntime; public CountInnerStateRuntime(StreamInnerStateRuntime streamInnerStateRuntime) { super(streamInnerStateRuntime.getStateType()); - this.streamInnerStateRuntime = streamInnerStateRuntime; singleStreamRuntimeList = streamInnerStateRuntime.singleStreamRuntimeList; firstProcessor = streamInnerStateRuntime.firstProcessor; lastProcessor = streamInnerStateRuntime.lastProcessor; } - @Override - public InnerStateRuntime clone(String key) { - StreamInnerStateRuntime clonedStreamInnerStateRuntime = (StreamInnerStateRuntime) streamInnerStateRuntime - .clone(key); - CountPreStateProcessor countPreStateProcessor = (CountPreStateProcessor) clonedStreamInnerStateRuntime - .getFirstProcessor(); - CountPostStateProcessor countPostStateProcessor = (CountPostStateProcessor) clonedStreamInnerStateRuntime - .getLastProcessor(); - countPreStateProcessor.setCountPostStateProcessor(countPostStateProcessor); - return new CountInnerStateRuntime(clonedStreamInnerStateRuntime); - - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/EveryInnerStateRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/EveryInnerStateRuntime.java index 1bc999316f..ce56101e3c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/EveryInnerStateRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/EveryInnerStateRuntime.java @@ -39,22 +39,13 @@ public void setQuerySelector(Processor commonProcessor) { } @Override - public void init() { - innerStateRuntime.init(); + public void setup() { + innerStateRuntime.setup(); } @Override - public InnerStateRuntime clone(String key) { - InnerStateRuntime clonedStreamInnerStateRuntime = innerStateRuntime.clone(key); - EveryInnerStateRuntime everyInnerStateRuntime = new EveryInnerStateRuntime(clonedStreamInnerStateRuntime, - stateType); - - everyInnerStateRuntime.firstProcessor = clonedStreamInnerStateRuntime.getFirstProcessor(); - everyInnerStateRuntime.lastProcessor = clonedStreamInnerStateRuntime.getLastProcessor(); - everyInnerStateRuntime.singleStreamRuntimeList.addAll(clonedStreamInnerStateRuntime - .getSingleStreamRuntimeList()); - everyInnerStateRuntime.getLastProcessor().setNextEveryStatePreProcessor(everyInnerStateRuntime - .getFirstProcessor()); - return everyInnerStateRuntime; + public void init() { + innerStateRuntime.init(); } + } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/InnerStateRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/InnerStateRuntime.java index a4e45fc761..36422cb93c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/InnerStateRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/InnerStateRuntime.java @@ -43,11 +43,12 @@ public interface InnerStateRuntime { void setQuerySelector(Processor commonProcessor); + void setup(); + void init(); void reset(); void update(); - InnerStateRuntime clone(String key); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/LogicalInnerStateRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/LogicalInnerStateRuntime.java index 216eb59982..603a1d9d09 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/LogicalInnerStateRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/LogicalInnerStateRuntime.java @@ -17,14 +17,9 @@ */ package io.siddhi.core.query.input.stream.state.runtime; -import io.siddhi.core.query.input.stream.single.SingleStreamRuntime; -import io.siddhi.core.query.input.stream.state.LogicalPostStateProcessor; -import io.siddhi.core.query.input.stream.state.LogicalPreStateProcessor; import io.siddhi.core.query.processor.Processor; import io.siddhi.query.api.execution.query.input.stream.StateInputStream; -import java.util.List; - /** * Created on 12/19/14. */ @@ -46,6 +41,12 @@ public void setQuerySelector(Processor commonProcessor) { innerStateRuntime1.setQuerySelector(commonProcessor); } + @Override + public void setup() { + innerStateRuntime2.setup(); + innerStateRuntime1.setup(); + } + @Override public void init() { innerStateRuntime2.init(); @@ -62,46 +63,4 @@ public void update() { innerStateRuntime2.update(); } - @Override - public InnerStateRuntime clone(String key) { - InnerStateRuntime clonedInnerStateRuntime1 = innerStateRuntime1.clone(key); - InnerStateRuntime clonedInnerStateRuntime2 = innerStateRuntime2.clone(key); - - LogicalPreStateProcessor logicalPreStateProcessor1 = (LogicalPreStateProcessor) - clonedInnerStateRuntime1.getFirstProcessor(); - LogicalPostStateProcessor logicalPostStateProcessor1 = (LogicalPostStateProcessor) - clonedInnerStateRuntime1.getLastProcessor(); - LogicalPreStateProcessor logicalPreStateProcessor2 = (LogicalPreStateProcessor) - clonedInnerStateRuntime2.getFirstProcessor(); - LogicalPostStateProcessor logicalPostStateProcessor2 = (LogicalPostStateProcessor) - clonedInnerStateRuntime2.getLastProcessor(); - - logicalPostStateProcessor1.setPartnerPreStateProcessor(logicalPreStateProcessor2); - logicalPostStateProcessor2.setPartnerPreStateProcessor(logicalPreStateProcessor1); - logicalPostStateProcessor1.setPartnerPostStateProcessor(logicalPostStateProcessor2); - logicalPostStateProcessor2.setPartnerPostStateProcessor(logicalPostStateProcessor1); - logicalPreStateProcessor1.setPartnerStatePreProcessor(logicalPreStateProcessor2); - logicalPreStateProcessor2.setPartnerStatePreProcessor(logicalPreStateProcessor1); - - LogicalInnerStateRuntime logicalInnerStateRuntime = new LogicalInnerStateRuntime(clonedInnerStateRuntime1, - clonedInnerStateRuntime2, stateType); - logicalInnerStateRuntime.firstProcessor = clonedInnerStateRuntime1.getFirstProcessor(); - logicalInnerStateRuntime.lastProcessor = clonedInnerStateRuntime2.getLastProcessor(); - logicalInnerStateRuntime.getSingleStreamRuntimeList().addAll - (clonedInnerStateRuntime2.getSingleStreamRuntimeList()); - logicalInnerStateRuntime.getSingleStreamRuntimeList().addAll - (clonedInnerStateRuntime1.getSingleStreamRuntimeList()); - - - List runtimeList = logicalInnerStateRuntime.getSingleStreamRuntimeList(); - for (int i = 0; i < runtimeList.size(); i++) { - String streamId = runtimeList.get(i).getProcessStreamReceiver().getStreamId(); - for (int j = i; j < runtimeList.size(); j++) { - if (streamId.equals(runtimeList.get(j).getProcessStreamReceiver().getStreamId())) { - runtimeList.get(j).setProcessStreamReceiver(runtimeList.get(i).getProcessStreamReceiver()); - } - } - } - return logicalInnerStateRuntime; - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/NextInnerStateRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/NextInnerStateRuntime.java index ff11a7d9c0..7524aa9b08 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/NextInnerStateRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/NextInnerStateRuntime.java @@ -17,12 +17,9 @@ */ package io.siddhi.core.query.input.stream.state.runtime; -import io.siddhi.core.query.input.stream.single.SingleStreamRuntime; import io.siddhi.core.query.processor.Processor; import io.siddhi.query.api.execution.query.input.stream.StateInputStream; -import java.util.List; - /** * Created on 12/19/14. */ @@ -43,6 +40,12 @@ public void setQuerySelector(Processor commonProcessor) { nextInnerStateRuntime.setQuerySelector(commonProcessor); } + @Override + public void setup() { + currentInnerStateRuntime.setup(); + nextInnerStateRuntime.setup(); + } + @Override public void init() { currentInnerStateRuntime.init(); @@ -60,33 +63,4 @@ public void update() { currentInnerStateRuntime.update(); nextInnerStateRuntime.update(); } - - @Override - public InnerStateRuntime clone(String key) { - InnerStateRuntime clonedCurrentInnerStateRuntime = currentInnerStateRuntime.clone(key); - InnerStateRuntime clonedNextInnerStateRuntime = nextInnerStateRuntime.clone(key); - - NextInnerStateRuntime nextInnerStateRuntime = new NextInnerStateRuntime(clonedCurrentInnerStateRuntime, - clonedNextInnerStateRuntime, stateType); - nextInnerStateRuntime.singleStreamRuntimeList.addAll(clonedCurrentInnerStateRuntime - .getSingleStreamRuntimeList()); - nextInnerStateRuntime.singleStreamRuntimeList.addAll(clonedNextInnerStateRuntime.getSingleStreamRuntimeList()); - nextInnerStateRuntime.firstProcessor = clonedCurrentInnerStateRuntime.getFirstProcessor(); - nextInnerStateRuntime.lastProcessor = clonedNextInnerStateRuntime.getLastProcessor(); - - clonedCurrentInnerStateRuntime.getLastProcessor().setNextStatePreProcessor(clonedNextInnerStateRuntime - .getFirstProcessor()); - - List runtimeList = nextInnerStateRuntime.getSingleStreamRuntimeList(); - for (int i = 0; i < runtimeList.size(); i++) { - String streamId = runtimeList.get(i).getProcessStreamReceiver().getStreamId(); - for (int j = i; j < runtimeList.size(); j++) { - if (streamId.equals(runtimeList.get(j).getProcessStreamReceiver().getStreamId())) { - runtimeList.get(j).setProcessStreamReceiver(runtimeList.get(i).getProcessStreamReceiver()); - } - } - } - - return nextInnerStateRuntime; - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/StreamInnerStateRuntime.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/StreamInnerStateRuntime.java index 232b2018e3..aae8901b0b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/StreamInnerStateRuntime.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/input/stream/state/runtime/StreamInnerStateRuntime.java @@ -20,8 +20,6 @@ import io.siddhi.core.query.input.stream.single.SingleStreamRuntime; import io.siddhi.core.query.input.stream.state.PostStateProcessor; import io.siddhi.core.query.input.stream.state.PreStateProcessor; -import io.siddhi.core.query.input.stream.state.StreamPostStateProcessor; -import io.siddhi.core.query.input.stream.state.StreamPreStateProcessor; import io.siddhi.core.query.processor.Processor; import io.siddhi.query.api.execution.query.input.stream.StateInputStream; @@ -79,9 +77,13 @@ public StateInputStream.Type getStateType() { } @Override - public void init() { + public void setup() { singleStreamRuntimeList.get(0).getProcessStreamReceiver().setNext(firstProcessor); singleStreamRuntimeList.get(0).getProcessStreamReceiver().addStatefulProcessor(firstProcessor); + } + + @Override + public void init() { firstProcessor.init(); } @@ -95,31 +97,4 @@ public void update() { firstProcessor.updateState(); } - @Override - public InnerStateRuntime clone(String key) { - StreamInnerStateRuntime streamInnerStateRuntime = new StreamInnerStateRuntime(stateType); - for (SingleStreamRuntime singleStreamRuntime : singleStreamRuntimeList) { - streamInnerStateRuntime.singleStreamRuntimeList.add((SingleStreamRuntime) singleStreamRuntime.clone(key)); - } - - Processor processor = streamInnerStateRuntime.singleStreamRuntimeList.get(0).getProcessorChain(); - streamInnerStateRuntime.firstProcessor = (StreamPreStateProcessor) processor; - - while (processor != null) { - if (processor instanceof StreamPostStateProcessor) { - streamInnerStateRuntime.lastProcessor = (StreamPostStateProcessor) processor; - break; - } else { - processor = processor.getNextProcessor(); - } - } - - ((StreamPostStateProcessor) streamInnerStateRuntime.lastProcessor).setThisStatePreProcessor( - (StreamPreStateProcessor) streamInnerStateRuntime.firstProcessor); - ((StreamPreStateProcessor) streamInnerStateRuntime.firstProcessor).setThisStatePostProcessor( - (StreamPostStateProcessor) streamInnerStateRuntime.lastProcessor); - ((StreamPreStateProcessor) streamInnerStateRuntime.firstProcessor).setThisLastProcessor( - (StreamPostStateProcessor) streamInnerStateRuntime.lastProcessor); - return streamInnerStateRuntime; - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/DeleteTableCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/DeleteTableCallback.java index d5da6c2c0f..ce7f8e2d83 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/DeleteTableCallback.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/DeleteTableCallback.java @@ -21,8 +21,8 @@ import io.siddhi.core.debugger.SiddhiDebugger; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.StateEvent; -import io.siddhi.core.event.state.StateEventPool; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.state.StateEventFactory; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.table.Table; import io.siddhi.core.util.collection.operator.CompiledCondition; @@ -37,26 +37,26 @@ public class DeleteTableCallback extends OutputCallback { private Table table; private CompiledCondition compiledCondition; private boolean convertToStreamEvent; - private StateEventPool stateEventPool; - private StreamEventPool streamEventPool; + private StateEventFactory stateEventFactory; + private StreamEventFactory streamEventFactory; private StreamEventConverter streamEventConverter; public DeleteTableCallback(Table table, CompiledCondition compiledCondition, int matchingStreamIndex, - boolean convertToStreamEvent, StateEventPool stateEventPool, - StreamEventPool streamEventPool, StreamEventConverter streamEventConverter, + boolean convertToStreamEvent, StateEventFactory stateEventFactory, + StreamEventFactory streamEventFactory, StreamEventConverter streamEventConverter, String queryName) { super(queryName); this.matchingStreamIndex = matchingStreamIndex; this.table = table; this.compiledCondition = compiledCondition; this.convertToStreamEvent = convertToStreamEvent; - this.stateEventPool = stateEventPool; - this.streamEventPool = streamEventPool; + this.stateEventFactory = stateEventFactory; + this.streamEventFactory = streamEventFactory; this.streamEventConverter = streamEventConverter; } @Override - public synchronized void send(ComplexEventChunk deletingEventChunk, int noOfEvents) { + public void send(ComplexEventChunk deletingEventChunk, int noOfEvents) { if (getSiddhiDebugger() != null) { getSiddhiDebugger() .checkBreakPoint(getQueryName(), SiddhiDebugger.QueryTerminal.OUT, deletingEventChunk.getFirst()); @@ -64,8 +64,8 @@ public synchronized void send(ComplexEventChunk deletingEventChunk, int noOfEven deletingEventChunk.reset(); if (deletingEventChunk.hasNext()) { ComplexEventChunk deletingStateEventChunk = constructMatchingStateEventChunk( - deletingEventChunk, convertToStreamEvent, stateEventPool, matchingStreamIndex, streamEventPool, - streamEventConverter); + deletingEventChunk, convertToStreamEvent, stateEventFactory, matchingStreamIndex, + streamEventFactory, streamEventConverter); table.deleteEvents(deletingStateEventChunk, compiledCondition, noOfEvents); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoStreamCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoStreamCallback.java index df5fd05d47..592fb08f7c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoStreamCallback.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoStreamCallback.java @@ -28,8 +28,8 @@ * Siddhi queries and put them into {@link StreamJunction}. */ public class InsertIntoStreamCallback extends OutputCallback { - private StreamDefinition outputStreamDefinition; - private StreamJunction.Publisher publisher; + protected StreamDefinition outputStreamDefinition; + protected StreamJunction.Publisher publisher; public InsertIntoStreamCallback(StreamDefinition outputStreamDefinition, String queryName) { super(queryName); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoStreamEndPartitionCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoStreamEndPartitionCallback.java new file mode 100644 index 0000000000..452dafbdf6 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoStreamEndPartitionCallback.java @@ -0,0 +1,61 @@ +/* + * 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.output.callback; + +import io.siddhi.core.config.SiddhiAppContext; +import io.siddhi.core.debugger.SiddhiDebugger; +import io.siddhi.core.event.ComplexEvent; +import io.siddhi.core.event.ComplexEventChunk; +import io.siddhi.core.stream.StreamJunction; +import io.siddhi.query.api.definition.StreamDefinition; + +/** + * Implementation of {@link OutputCallback} to receive processed Siddhi events from partitioned + * Siddhi queries and put them into {@link StreamJunction}. + */ +public class InsertIntoStreamEndPartitionCallback extends InsertIntoStreamCallback { + + public InsertIntoStreamEndPartitionCallback(StreamDefinition outputStreamDefinition, String queryName) { + super(outputStreamDefinition, queryName); + } + + @Override + public void send(ComplexEventChunk complexEventChunk, int noOfEvents) { + if (getSiddhiDebugger() != null) { + getSiddhiDebugger() + .checkBreakPoint(getQueryName(), SiddhiDebugger.QueryTerminal.OUT, complexEventChunk.getFirst()); + } + complexEventChunk.reset(); + if (complexEventChunk.getFirst() != null) { + String flowId = SiddhiAppContext.getPartitionFlowId(); + SiddhiAppContext.stopPartitionFlow(); + try { + while (complexEventChunk.hasNext()) { + ComplexEvent complexEvent = complexEventChunk.next(); + if (complexEvent.getType() == ComplexEvent.Type.EXPIRED) { + complexEvent.setType(ComplexEvent.Type.CURRENT); + } + } + publisher.send(complexEventChunk.getFirst()); + } finally { + SiddhiAppContext.startPartitionFlow(flowId); + } + } + } + +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoTableCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoTableCallback.java index 5e499b4331..ec0856c5b7 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoTableCallback.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoTableCallback.java @@ -21,7 +21,7 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.table.Table; import io.siddhi.query.api.definition.StreamDefinition; @@ -35,17 +35,17 @@ public class InsertIntoTableCallback extends OutputCallback { private Table table; private StreamDefinition outputStreamDefinition; private boolean convertToStreamEvent; - private StreamEventPool streamEventPool; + private StreamEventFactory streamEventFactory; private StreamEventConverter streamEventConverter; public InsertIntoTableCallback(Table table, StreamDefinition outputStreamDefinition, - boolean convertToStreamEvent, StreamEventPool streamEventPool, + boolean convertToStreamEvent, StreamEventFactory streamEventFactory, StreamEventConverter streamEventConverter, String queryName) { super(queryName); this.table = table; this.outputStreamDefinition = outputStreamDefinition; this.convertToStreamEvent = convertToStreamEvent; - this.streamEventPool = streamEventPool; + this.streamEventFactory = streamEventFactory; this.streamEventConverter = streamEventConverter; } @@ -61,7 +61,7 @@ public void send(ComplexEventChunk complexEventChunk, int noOfEvents) { complexEventChunk.reset(); while (complexEventChunk.hasNext()) { ComplexEvent complexEvent = complexEventChunk.next(); - StreamEvent borrowEvent = streamEventPool.borrowEvent(); + StreamEvent borrowEvent = streamEventFactory.newInstance(); streamEventConverter.convertData( complexEvent.getTimestamp(), complexEvent.getOutputData(), diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoWindowCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoWindowCallback.java index d5a19e5e5c..a00f637cb3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoWindowCallback.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoWindowCallback.java @@ -31,12 +31,12 @@ public class InsertIntoWindowCallback extends OutputCallback { /** * Window to which the events have to be inserted. */ - private final Window window; + protected final Window window; /** * StreamDefinition of the window. */ - private final StreamDefinition outputStreamDefinition; + protected final StreamDefinition outputStreamDefinition; public InsertIntoWindowCallback(Window window, StreamDefinition outputStreamDefinition, String queryName) { super(queryName); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoWindowEndPartitionCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoWindowEndPartitionCallback.java new file mode 100644 index 0000000000..e1a562556c --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/InsertIntoWindowEndPartitionCallback.java @@ -0,0 +1,70 @@ +/* + * 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.output.callback; + +import io.siddhi.core.config.SiddhiAppContext; +import io.siddhi.core.debugger.SiddhiDebugger; +import io.siddhi.core.event.ComplexEvent; +import io.siddhi.core.event.ComplexEventChunk; +import io.siddhi.core.stream.StreamJunction; +import io.siddhi.core.window.Window; +import io.siddhi.query.api.definition.StreamDefinition; + +/** + * This callback is an adapter between {@link StreamJunction} and {@link Window}. + * It receives {@link ComplexEventChunk}s from partitioned queries and insert them into {@link Window}. + */ +public class InsertIntoWindowEndPartitionCallback extends InsertIntoWindowCallback { + + public InsertIntoWindowEndPartitionCallback(Window window, StreamDefinition outputStreamDefinition, + String queryName) { + super(window, outputStreamDefinition, queryName); + } + + /** + * Add the event into the {@link Window} + * + * @param complexEventChunk the event to add + * @param noOfEvents number of events + */ + @Override + public void send(ComplexEventChunk complexEventChunk, int noOfEvents) { + if (getSiddhiDebugger() != null) { + getSiddhiDebugger() + .checkBreakPoint(getQueryName(), SiddhiDebugger.QueryTerminal.OUT, complexEventChunk.getFirst()); + } + // If events are inserted directly from another window, expired events can arrive + complexEventChunk.reset(); + if (complexEventChunk.getFirst() != null) { + String flowId = SiddhiAppContext.getCurrentFlowId(); + SiddhiAppContext.stopPartitionFlow(); + try { + while (complexEventChunk.hasNext()) { + ComplexEvent complexEvent = complexEventChunk.next(); + if (complexEvent.getType() == ComplexEvent.Type.EXPIRED) { + complexEvent.setType(ComplexEvent.Type.CURRENT); + } + } + window.add(complexEventChunk); + } finally { + SiddhiAppContext.startPartitionFlow(flowId); + } + } + } +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/OutputCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/OutputCallback.java index 700f101565..49c1a2d462 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/OutputCallback.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/OutputCallback.java @@ -22,9 +22,9 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.StateEvent; -import io.siddhi.core.event.state.StateEventPool; +import io.siddhi.core.event.state.StateEventFactory; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; /** @@ -53,22 +53,19 @@ String getQueryName() { return queryName; } - protected ComplexEventChunk constructMatchingStateEventChunk(ComplexEventChunk - matchingComplexEventChunk, - boolean convertToStreamEvent, - StateEventPool stateEventPool, - int matchingStreamIndex, StreamEventPool - streamEventPool, - StreamEventConverter - streamEventConverter) { - ComplexEventChunk stateEventChunk = new ComplexEventChunk(matchingComplexEventChunk - .isBatch()); + protected ComplexEventChunk constructMatchingStateEventChunk( + ComplexEventChunk matchingComplexEventChunk, boolean convertToStreamEvent, + StateEventFactory stateEventFactory, + int matchingStreamIndex, StreamEventFactory streamEventFactory, + StreamEventConverter streamEventConverter) { + ComplexEventChunk stateEventChunk = new ComplexEventChunk( + matchingComplexEventChunk.isBatch()); while (matchingComplexEventChunk.hasNext()) { ComplexEvent matchingComplexEvent = matchingComplexEventChunk.next(); matchingComplexEventChunk.remove(); - StateEvent stateEvent = stateEventPool.borrowEvent(); + StateEvent stateEvent = stateEventFactory.newInstance(); if (convertToStreamEvent) { - StreamEvent borrowEvent = streamEventPool.borrowEvent(); + StreamEvent borrowEvent = streamEventFactory.newInstance(); streamEventConverter.convertData( matchingComplexEvent.getTimestamp(), matchingComplexEvent.getOutputData(), diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/QueryCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/QueryCallback.java index a28b79406c..3409dfa47d 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/QueryCallback.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/QueryCallback.java @@ -94,14 +94,6 @@ private void bufferEvent(ComplexEvent complexEvent, List eventBuffer) { eventBuffer.add(new Event(complexEvent.getOutputData().length).copyFrom(complexEvent)); } - public synchronized void startProcessing() { - - } - - public synchronized void stopProcessing() { - - } - public abstract void receive(long timestamp, Event[] inEvents, Event[] removeEvents); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/UpdateOrInsertTableCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/UpdateOrInsertTableCallback.java index 5d9edc4293..b8a95fe8e9 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/UpdateOrInsertTableCallback.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/UpdateOrInsertTableCallback.java @@ -21,8 +21,8 @@ import io.siddhi.core.debugger.SiddhiDebugger; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.StateEvent; -import io.siddhi.core.event.state.StateEventPool; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.state.StateEventFactory; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.table.CompiledUpdateSet; import io.siddhi.core.table.Table; @@ -41,14 +41,14 @@ public class UpdateOrInsertTableCallback extends OutputCallback { private CompiledCondition compiledCondition; private CompiledUpdateSet compiledUpdateSet; private boolean convertToStreamEvent; - private StateEventPool stateEventPool; - private StreamEventPool streamEventPool; + private StateEventFactory stateEventFactory; + private StreamEventFactory streamEventFactory; private StreamEventConverter streamEventConverter; public UpdateOrInsertTableCallback(Table table, CompiledCondition compiledCondition, CompiledUpdateSet compiledUpdateSet, int matchingStreamIndex, boolean convertToStreamEvent, - StateEventPool stateEventPool, StreamEventPool streamEventPool, + StateEventFactory stateEventFactory, StreamEventFactory streamEventFactory, StreamEventConverter streamEventConverter, String queryName) { super(queryName); this.matchingStreamIndex = matchingStreamIndex; @@ -56,8 +56,8 @@ public UpdateOrInsertTableCallback(Table table, CompiledCondition compiledCondit this.compiledCondition = compiledCondition; this.compiledUpdateSet = compiledUpdateSet; this.convertToStreamEvent = convertToStreamEvent; - this.stateEventPool = stateEventPool; - this.streamEventPool = streamEventPool; + this.stateEventFactory = stateEventFactory; + this.streamEventFactory = streamEventFactory; this.streamEventConverter = streamEventConverter; this.addingStreamEventExtractor = new AddingStreamEventExtractor(matchingStreamIndex); } @@ -71,10 +71,10 @@ public void send(ComplexEventChunk updateOrAddEventChunk, int noOfEvents) { updateOrAddEventChunk.reset(); if (updateOrAddEventChunk.hasNext()) { ComplexEventChunk updateOrAddStateEventChunk = constructMatchingStateEventChunk - (updateOrAddEventChunk, convertToStreamEvent, stateEventPool, - matchingStreamIndex, streamEventPool, streamEventConverter); - constructMatchingStateEventChunk(updateOrAddEventChunk, convertToStreamEvent, stateEventPool, - matchingStreamIndex, streamEventPool, streamEventConverter); + (updateOrAddEventChunk, convertToStreamEvent, stateEventFactory, + matchingStreamIndex, streamEventFactory, streamEventConverter); + constructMatchingStateEventChunk(updateOrAddEventChunk, convertToStreamEvent, stateEventFactory, + matchingStreamIndex, streamEventFactory, streamEventConverter); table.updateOrAddEvents(updateOrAddStateEventChunk, compiledCondition, compiledUpdateSet, addingStreamEventExtractor, noOfEvents); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/UpdateTableCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/UpdateTableCallback.java index 594a56ec79..a52f6d4d53 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/UpdateTableCallback.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/callback/UpdateTableCallback.java @@ -21,8 +21,8 @@ import io.siddhi.core.debugger.SiddhiDebugger; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.state.StateEvent; -import io.siddhi.core.event.state.StateEventPool; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.state.StateEventFactory; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.table.CompiledUpdateSet; import io.siddhi.core.table.Table; @@ -39,13 +39,15 @@ public class UpdateTableCallback extends OutputCallback { private CompiledCondition compiledCondition; private CompiledUpdateSet compiledUpdateSet; private boolean convertToStreamEvent; - private StateEventPool stateEventPool; - private StreamEventPool streamEventPool; + private StateEventFactory stateEventFactory; + private StreamEventFactory streamEventFactory; private StreamEventConverter streamEventConverter; - public UpdateTableCallback(Table table, CompiledCondition compiledCondition, CompiledUpdateSet compiledUpdateSet, - int matchingStreamIndex, boolean convertToStreamEvent, StateEventPool stateEventPool, - StreamEventPool streamEventPool, StreamEventConverter streamEventConverter, + public UpdateTableCallback(Table table, CompiledCondition compiledCondition, + CompiledUpdateSet compiledUpdateSet, + int matchingStreamIndex, boolean convertToStreamEvent, + StateEventFactory stateEventFactory, StreamEventFactory streamEventFactory, + StreamEventConverter streamEventConverter, String queryName) { super(queryName); this.table = table; @@ -53,21 +55,22 @@ public UpdateTableCallback(Table table, CompiledCondition compiledCondition, Com this.compiledUpdateSet = compiledUpdateSet; this.matchingStreamIndex = matchingStreamIndex; this.convertToStreamEvent = convertToStreamEvent; - this.stateEventPool = stateEventPool; - this.streamEventPool = streamEventPool; + this.stateEventFactory = stateEventFactory; + this.streamEventFactory = streamEventFactory; this.streamEventConverter = streamEventConverter; } @Override - public synchronized void send(ComplexEventChunk updatingEventChunk, int noOfEvents) { + public void send(ComplexEventChunk updatingEventChunk, int noOfEvents) { if (getSiddhiDebugger() != null) { getSiddhiDebugger() .checkBreakPoint(getQueryName(), SiddhiDebugger.QueryTerminal.OUT, updatingEventChunk.getFirst()); } updatingEventChunk.reset(); if (updatingEventChunk.hasNext()) { - ComplexEventChunk updatingStateEventChunk = constructMatchingStateEventChunk(updatingEventChunk, - convertToStreamEvent, stateEventPool, matchingStreamIndex, streamEventPool, streamEventConverter); + ComplexEventChunk updatingStateEventChunk = constructMatchingStateEventChunk( + updatingEventChunk, convertToStreamEvent, stateEventFactory, matchingStreamIndex, + streamEventFactory, streamEventConverter); table.updateEvents(updatingStateEventChunk, compiledCondition, compiledUpdateSet, noOfEvents); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/OutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/OutputRateLimiter.java index 28496d81da..1b25c16783 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/OutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/OutputRateLimiter.java @@ -20,12 +20,14 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; +import io.siddhi.core.partition.PartitionCreationListener; import io.siddhi.core.query.input.MultiProcessStreamReceiver; import io.siddhi.core.query.output.callback.OutputCallback; import io.siddhi.core.query.output.callback.QueryCallback; -import io.siddhi.core.util.extension.holder.EternalReferencedHolder; import io.siddhi.core.util.lock.LockWrapper; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.core.util.statistics.LatencyTracker; import java.util.ArrayList; @@ -34,45 +36,41 @@ /** * Abstract parent implementation of Output Rate Limiting. Output Rate Limiting is used to throttle the output of * Siddhi queries based on various criteria. + * + * @param current state of the RateLimiter */ -public abstract class OutputRateLimiter implements EternalReferencedHolder, Snapshotable { +public abstract class OutputRateLimiter implements PartitionCreationListener { protected List queryCallbacks = new ArrayList(); protected OutputCallback outputCallback = null; protected LatencyTracker latencyTracker; protected SiddhiQueryContext siddhiQueryContext; protected LockWrapper lockWrapper; + protected StateHolder stateHolder; private boolean hasCallBack = false; - private String elementId; - public void init(LockWrapper lockWrapper, SiddhiQueryContext siddhiQueryContext) { + public void init(LockWrapper lockWrapper, boolean groupBy, SiddhiQueryContext siddhiQueryContext) { this.siddhiQueryContext = siddhiQueryContext; if (outputCallback != null) { this.lockWrapper = lockWrapper; } - if (elementId == null) { - elementId = "OutputRateLimiter-" + - siddhiQueryContext.getSiddhiAppContext().getElementIdGenerator().createNewId(); - } - siddhiQueryContext.getSiddhiAppContext().getSnapshotService().addSnapshotable( - siddhiQueryContext.getName(), this); + stateHolder = siddhiQueryContext.generateStateHolder(this.getClass().getName(), groupBy, init()); } + protected abstract StateFactory init(); + public void sendToCallBacks(ComplexEventChunk complexEventChunk) { MultiProcessStreamReceiver.ReturnEventHolder returnEventHolder = MultiProcessStreamReceiver.getMultiProcessReturn().get(); + if (siddhiQueryContext.getSiddhiAppContext().isStatsEnabled() && latencyTracker != null) { + latencyTracker.markOut(); + } if (returnEventHolder != null) { returnEventHolder.setReturnEvents(complexEventChunk); return; } else if (lockWrapper != null) { lockWrapper.unlock(); } - if (siddhiQueryContext.getSiddhiAppContext().isStatsEnabled() && latencyTracker != null) { - latencyTracker.markOut(); - } - if (lockWrapper != null) { - lockWrapper.unlock(); - } if (!queryCallbacks.isEmpty()) { for (QueryCallback callback : queryCallbacks) { callback.receiveStreamEvent(complexEventChunk); @@ -121,20 +119,9 @@ public boolean hasCallBack() { return hasCallBack; } - public abstract OutputRateLimiter clone(String key); - - public String getElementId() { - return elementId; - } - public void setLatencyTracker(LatencyTracker latencyTracker) { this.latencyTracker = latencyTracker; } - @Override - public void clean() { - siddhiQueryContext.getSiddhiAppContext().getSnapshotService().removeSnapshotable( - siddhiQueryContext.getName(), this); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/PassThroughOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/PassThroughOutputRateLimiter.java index 78825e2898..75f1d42ddb 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/PassThroughOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/PassThroughOutputRateLimiter.java @@ -19,10 +19,9 @@ import io.siddhi.core.event.ComplexEventChunk; +import io.siddhi.core.util.snapshot.state.StateFactory; import org.apache.log4j.Logger; -import java.util.Map; - /** * Implementation of {@link OutputRateLimiter} which will pass through events without doing any rate limiting. This * is the default rate limiting strategy used by Siddhi. @@ -35,10 +34,9 @@ public PassThroughOutputRateLimiter(String id) { this.id = id; } - public PassThroughOutputRateLimiter clone(String key) { - PassThroughOutputRateLimiter instance = new PassThroughOutputRateLimiter(id + key); - instance.setLatencyTracker(latencyTracker); - return instance; + @Override + protected StateFactory init() { + return null; } @Override @@ -47,23 +45,8 @@ public void process(ComplexEventChunk complexEventChunk) { } @Override - public void start() { - //Nothing to start - } - - @Override - public void stop() { - //Nothing to stop - } - - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - + public void partitionCreated() { + //Nothing to be done } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/AllPerEventOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/AllPerEventOutputRateLimiter.java index 1d921dd444..7f4ad7ad4f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/AllPerEventOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/AllPerEventOutputRateLimiter.java @@ -21,6 +21,8 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.HashMap; @@ -30,48 +32,46 @@ * Implementation of {@link OutputRateLimiter} which will collect pre-defined number of events and the emit all * collected events as a batch. */ -public class AllPerEventOutputRateLimiter extends OutputRateLimiter { +public class AllPerEventOutputRateLimiter extends OutputRateLimiter { private final Integer value; - private String id; - - private volatile int counter = 0; - private ComplexEventChunk allComplexEventChunk; public AllPerEventOutputRateLimiter(String id, Integer value) { - this.id = id; this.value = value; - allComplexEventChunk = new ComplexEventChunk(false); } @Override - public OutputRateLimiter clone(String key) { - AllPerEventOutputRateLimiter instance = new AllPerEventOutputRateLimiter(id + key, value); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + return () -> new RateLimiterState(); } + @Override public void process(ComplexEventChunk complexEventChunk) { complexEventChunk.reset(); ArrayList> outputEventChunks = new ArrayList>(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type.EXPIRED) { - complexEventChunk.remove(); - allComplexEventChunk.add(event); - counter++; - if (counter == value) { - ComplexEventChunk outputEventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); - outputEventChunk.add(allComplexEventChunk.getFirst()); - allComplexEventChunk.clear(); - counter = 0; - outputEventChunks.add(outputEventChunk); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type.EXPIRED) { + complexEventChunk.remove(); + state.allComplexEventChunk.add(event); + state.counter++; + if (state.counter == value) { + ComplexEventChunk outputEventChunk = new ComplexEventChunk + (complexEventChunk.isBatch()); + outputEventChunk.add(state.allComplexEventChunk.getFirst()); + state.allComplexEventChunk.clear(); + state.counter = 0; + outputEventChunks.add(outputEventChunk); + } } } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); @@ -79,28 +79,34 @@ public void process(ComplexEventChunk complexEventChunk) { } @Override - public void start() { - //Nothing to start + public void partitionCreated() { + //Nothing to be done } - @Override - public void stop() { - //Nothing to stop - } + class RateLimiterState extends State { - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("Counter", counter); - state.put("AllComplexEventChunk", allComplexEventChunk.getFirst()); - return state; - } + private volatile int counter = 0; + private ComplexEventChunk allComplexEventChunk = new ComplexEventChunk(false); - @Override - public void restoreState(Map state) { - allComplexEventChunk.clear(); - allComplexEventChunk.add((ComplexEvent) state.get("AllComplexEventChunk")); - counter = (int) state.get("Counter"); + @Override + public boolean canDestroy() { + return counter == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Counter", counter); + state.put("AllComplexEventChunk", allComplexEventChunk.getFirst()); + return state; + } + + @Override + public void restore(Map state) { + allComplexEventChunk.clear(); + allComplexEventChunk.add((ComplexEvent) state.get("AllComplexEventChunk")); + counter = (int) state.get("Counter"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/FirstGroupByPerEventOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/FirstGroupByPerEventOutputRateLimiter.java index 64f0972cb3..dd3019013a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/FirstGroupByPerEventOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/FirstGroupByPerEventOutputRateLimiter.java @@ -22,101 +22,82 @@ import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.GroupedComplexEvent; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; /** * Implementation of {@link OutputRateLimiter} which will collect pre-defined number of events and the emit only the * first event. This implementation specifically handle queries with group by. */ -public class FirstGroupByPerEventOutputRateLimiter extends OutputRateLimiter { +public class FirstGroupByPerEventOutputRateLimiter + extends OutputRateLimiter { private final Integer value; - private List groupByKeys = new ArrayList(); - private String id; - private ComplexEventChunk allComplexEventChunk; - private volatile int counter = 0; public FirstGroupByPerEventOutputRateLimiter(String id, Integer value) { - this.id = id; this.value = value; - allComplexEventChunk = new ComplexEventChunk(false); } @Override - public OutputRateLimiter clone(String key) { - FirstGroupByPerEventOutputRateLimiter instance = new FirstGroupByPerEventOutputRateLimiter(id + key, value); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { complexEventChunk.reset(); - ArrayList> outputEventChunks = new ArrayList>(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type.EXPIRED) { + ComplexEventChunk outputEventChunk = new ComplexEventChunk<>(complexEventChunk.isBatch()); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); complexEventChunk.remove(); GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); - if (!groupByKeys.contains(groupedComplexEvent.getGroupKey())) { - groupByKeys.add(groupedComplexEvent.getGroupKey()); - allComplexEventChunk.add(groupedComplexEvent.getComplexEvent()); - } - if (++counter == value) { - if (allComplexEventChunk.getFirst() != null) { - ComplexEventChunk outputEventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); - outputEventChunk.add(allComplexEventChunk.getFirst()); - outputEventChunks.add(outputEventChunk); - allComplexEventChunk.clear(); - counter = 0; - groupByKeys.clear(); - } else { - counter = 0; - groupByKeys.clear(); - } - + Integer count = state.groupByOutputTime.get(groupedComplexEvent.getGroupKey()); + if (count == null) { + state.groupByOutputTime.put(groupedComplexEvent.getGroupKey(), 1); + outputEventChunk.add(groupedComplexEvent); + } else if (count.equals(value - 1)) { + state.groupByOutputTime.remove(groupedComplexEvent.getGroupKey()); + } else { + state.groupByOutputTime.put(groupedComplexEvent.getGroupKey(), count + 1); } } } + } finally { + stateHolder.returnState(state); } - for (ComplexEventChunk eventChunk : outputEventChunks) { - sendToCallBacks(eventChunk); + if (outputEventChunk.getFirst() != null) { + sendToCallBacks(outputEventChunk); } } @Override - public void start() { - //Nothing to start + public void partitionCreated() { + //Nothing to be done } - @Override - public void stop() { - //Nothing to stop - } + class RateLimiterState extends State { + private Map groupByOutputTime = new HashMap(); - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("Counter", counter); - state.put("GroupByKeys", groupByKeys); - state.put("AllComplexEventChunk", allComplexEventChunk.getFirst()); + @Override + public boolean canDestroy() { + return groupByOutputTime.isEmpty(); } - return state; - } - @Override - public void restoreState(Map state) { - synchronized (this) { - counter = (int) state.get("Counter"); - groupByKeys = (List) state.get("GroupByKeys"); - allComplexEventChunk.clear(); - allComplexEventChunk.add((ComplexEvent) state.get("AllComplexEventChunk")); + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("GroupByOutputTime", groupByOutputTime); + return state; + } + + @Override + public void restore(Map state) { + groupByOutputTime = (Map) state.get("GroupByOutputTime"); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/FirstPerEventOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/FirstPerEventOutputRateLimiter.java index fe71c7d12d..82c827e21c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/FirstPerEventOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/FirstPerEventOutputRateLimiter.java @@ -21,8 +21,9 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; -import java.util.ArrayList; import java.util.HashMap; import java.util.Map; @@ -30,70 +31,69 @@ * Implementation of {@link OutputRateLimiter} which will collect pre-defined number of events and the emit only the * first event. */ -public class FirstPerEventOutputRateLimiter extends OutputRateLimiter { +public class FirstPerEventOutputRateLimiter + extends OutputRateLimiter { private final Integer value; - private String id; - private volatile int counter = 0; public FirstPerEventOutputRateLimiter(String id, Integer value) { this.value = value; - this.id = id; } @Override - public OutputRateLimiter clone(String key) { - FirstPerEventOutputRateLimiter instance = new FirstPerEventOutputRateLimiter(id + key, value); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { complexEventChunk.reset(); - ArrayList> outputEventChunks = new ArrayList>(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type.EXPIRED) { - if (counter == 0) { - complexEventChunk.remove(); - ComplexEventChunk firstPerEventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); - firstPerEventChunk.add(event); - outputEventChunks.add(firstPerEventChunk); - } - if (++counter == value) { - counter = 0; + ComplexEventChunk outputEventChunk = new ComplexEventChunk<>(complexEventChunk.isBatch()); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + complexEventChunk.remove(); + state.counter++; + if (state.counter == 1) { + outputEventChunk.add(event); + } else if (state.counter == value) { + state.counter = 0; } } } + } finally { + stateHolder.returnState(state); } - for (ComplexEventChunk eventChunk : outputEventChunks) { - sendToCallBacks(eventChunk); + if (outputEventChunk.getFirst() != null) { + sendToCallBacks(outputEventChunk); } - } @Override - public void start() { - //Nothing to start + public void partitionCreated() { + //Nothing to be done } - @Override - public void stop() { - //Nothing to stop - } + class RateLimiterState extends State { - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("Counter", counter); - return state; - } + private volatile int counter = 0; - @Override - public void restoreState(Map state) { - counter = (int) state.get("Counter"); - } + @Override + public boolean canDestroy() { + return counter == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Counter", counter); + return state; + } + @Override + public void restore(Map state) { + counter = (int) state.get("Counter"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/LastGroupByPerEventOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/LastGroupByPerEventOutputRateLimiter.java index 0c117f02bf..d0f7a037e3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/LastGroupByPerEventOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/LastGroupByPerEventOutputRateLimiter.java @@ -23,6 +23,8 @@ import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.GroupedComplexEvent; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.HashMap; @@ -33,50 +35,51 @@ * Implementation of {@link OutputRateLimiter} which will collect pre-defined number of events and the emit only the * last event. This implementation specifically handle queries with group by. */ -public class LastGroupByPerEventOutputRateLimiter extends OutputRateLimiter { +public class LastGroupByPerEventOutputRateLimiter extends + OutputRateLimiter { private final Integer value; - private String id; - private volatile int counter = 0; - private Map allGroupByKeyEvents = new LinkedHashMap(); public LastGroupByPerEventOutputRateLimiter(String id, Integer value) { - this.id = id; this.value = value; } @Override - public OutputRateLimiter clone(String key) { - LastGroupByPerEventOutputRateLimiter instance = new LastGroupByPerEventOutputRateLimiter(id + key, value); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { complexEventChunk.reset(); ArrayList> outputEventChunks = new ArrayList>(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type.EXPIRED) { - complexEventChunk.remove(); - GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); - allGroupByKeyEvents.put(groupedComplexEvent.getGroupKey(), groupedComplexEvent.getComplexEvent()); - if (++counter == value) { - counter = 0; - if (allGroupByKeyEvents.size() != 0) { - ComplexEventChunk outputEventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type.EXPIRED) { + complexEventChunk.remove(); + GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); + state.allGroupByKeyEvents.put(groupedComplexEvent.getGroupKey(), + groupedComplexEvent.getComplexEvent()); + if (++state.counter == value) { + state.counter = 0; + if (state.allGroupByKeyEvents.size() != 0) { + ComplexEventChunk outputEventChunk = new ComplexEventChunk + (complexEventChunk.isBatch()); - for (ComplexEvent complexEvent : allGroupByKeyEvents.values()) { - outputEventChunk.add(complexEvent); + for (ComplexEvent complexEvent : state.allGroupByKeyEvents.values()) { + outputEventChunk.add(complexEvent); + } + state.allGroupByKeyEvents.clear(); + outputEventChunks.add(outputEventChunk); } - allGroupByKeyEvents.clear(); - outputEventChunks.add(outputEventChunk); } } } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); @@ -84,28 +87,32 @@ public void process(ComplexEventChunk complexEventChunk) { } @Override - public void start() { - //Nothing to start + public void partitionCreated() { + //Nothing to be done } - @Override - public void stop() { - //Nothing to stop - } + class RateLimiterState extends State { - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + private volatile int counter = 0; + private Map allGroupByKeyEvents = new LinkedHashMap(); + + @Override + public boolean canDestroy() { + return counter == 0 && allGroupByKeyEvents.isEmpty(); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("Counter", counter); state.put("AllGroupByKeyEvents", allGroupByKeyEvents); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - counter = (int) state.get("Counter"); - allGroupByKeyEvents = (Map) state.get("AllGroupByKeyEvents"); + @Override + public void restore(Map state) { + counter = (int) state.get("Counter"); + allGroupByKeyEvents = (Map) state.get("AllGroupByKeyEvents"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/LastPerEventOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/LastPerEventOutputRateLimiter.java index f57ba1f817..fdc346f5dd 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/LastPerEventOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/event/LastPerEventOutputRateLimiter.java @@ -21,6 +21,8 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.HashMap; @@ -30,41 +32,41 @@ * Implementation of {@link OutputRateLimiter} which will collect pre-defined number of events and the emit only the * last event. */ -public class LastPerEventOutputRateLimiter extends OutputRateLimiter { +public class LastPerEventOutputRateLimiter extends OutputRateLimiter { private final Integer value; - private String id; - private volatile int counter = 0; public LastPerEventOutputRateLimiter(String id, Integer value) { - this.id = id; this.value = value; } @Override - public OutputRateLimiter clone(String key) { - LastPerEventOutputRateLimiter instance = new LastPerEventOutputRateLimiter(id + key, value); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { complexEventChunk.reset(); ArrayList> outputEventChunks = new ArrayList>(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type.EXPIRED) { - if (++counter == value) { - complexEventChunk.remove(); - ComplexEventChunk lastPerEventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); - lastPerEventChunk.add(event); - counter = 0; - outputEventChunks.add(lastPerEventChunk); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type.EXPIRED) { + if (++state.counter == value) { + complexEventChunk.remove(); + ComplexEventChunk lastPerEventChunk = new ComplexEventChunk + (complexEventChunk.isBatch()); + lastPerEventChunk.add(event); + state.counter = 0; + outputEventChunks.add(lastPerEventChunk); + } } } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); @@ -72,25 +74,31 @@ public void process(ComplexEventChunk complexEventChunk) { } @Override - public void start() { - //Nothing to start + public void partitionCreated() { + //Nothing to be done } - @Override - public void stop() { - //Nothing to stop - } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("Counter", counter); - return state; - } + class RateLimiterState extends State { - @Override - public void restoreState(Map state) { - counter = (int) state.get("Counter"); + private volatile int counter = 0; + + @Override + public boolean canDestroy() { + return counter == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Counter", counter); + return state; + } + + @Override + public void restore(Map state) { + counter = (int) state.get("Counter"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AggregationGroupByWindowedPerSnapshotOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AggregationGroupByWindowedPerSnapshotOutputRateLimiter.java index 2da599b077..69cbbf119b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AggregationGroupByWindowedPerSnapshotOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AggregationGroupByWindowedPerSnapshotOutputRateLimiter.java @@ -22,6 +22,9 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.GroupedComplexEvent; +import io.siddhi.core.event.stream.StreamEventFactory; +import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.HashMap; @@ -31,99 +34,112 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link PerSnapshotOutputRateLimiter} for queries with GroupBy, Aggregators and Windows. */ public class AggregationGroupByWindowedPerSnapshotOutputRateLimiter extends AggregationWindowedPerSnapshotOutputRateLimiter { - private List eventList; - private Map> groupByAggregateAttributeValueMap; - protected AggregationGroupByWindowedPerSnapshotOutputRateLimiter(String id, Long value, - ScheduledExecutorService scheduledExecutorService, + protected AggregationGroupByWindowedPerSnapshotOutputRateLimiter(Long value, List aggregateAttributePositionList, WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, + boolean groupBy, SiddhiQueryContext siddhiQueryContext) { - super(id, value, scheduledExecutorService, aggregateAttributePositionList, wrappedSnapshotOutputRateLimiter, - siddhiQueryContext); - groupByAggregateAttributeValueMap = new HashMap>(); - eventList = new LinkedList(); + super(value, aggregateAttributePositionList, wrappedSnapshotOutputRateLimiter, + groupBy, siddhiQueryContext); + } + + @Override + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new AggregationGroupByRateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { complexEventChunk.reset(); List> outputEventChunks = new ArrayList>(); - synchronized (this) { - complexEventChunk.reset(); - String currentGroupByKey = null; - Map currentAggregateAttributeValueMap = null; - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - tryFlushEvents(outputEventChunks, event); - } else { - complexEventChunk.remove(); - tryFlushEvents(outputEventChunks, event); - GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); - if (currentGroupByKey == null || !currentGroupByKey.equals(groupedComplexEvent.getGroupKey())) { - currentGroupByKey = groupedComplexEvent.getGroupKey(); - currentAggregateAttributeValueMap = groupByAggregateAttributeValueMap.get(currentGroupByKey); - if (currentAggregateAttributeValueMap == null) { - currentAggregateAttributeValueMap = new HashMap - (aggregateAttributePositionList.size()); - groupByAggregateAttributeValueMap.put(currentGroupByKey, currentAggregateAttributeValueMap); - } - } - if (groupedComplexEvent.getType() == ComplexEvent.Type.CURRENT) { - eventList.add(groupedComplexEvent); - for (Integer position : aggregateAttributePositionList) { - currentAggregateAttributeValueMap.put(position, event.getOutputData()[position]); + AggregationGroupByRateLimiterState state = (AggregationGroupByRateLimiterState) stateHolder.getState(); + try { + synchronized (state) { + complexEventChunk.reset(); + String currentGroupByKey = null; + Map currentAggregateAttributeValueMap = null; + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.TIMER) { + tryFlushEvents(outputEventChunks, event, state); + } else { + complexEventChunk.remove(); + tryFlushEvents(outputEventChunks, event, state); + GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); + if (currentGroupByKey == null || !currentGroupByKey.equals(groupedComplexEvent.getGroupKey())) { + currentGroupByKey = groupedComplexEvent.getGroupKey(); + currentAggregateAttributeValueMap = + state.groupByAggregateAttributeValueMap.get(currentGroupByKey); + if (currentAggregateAttributeValueMap == null) { + currentAggregateAttributeValueMap = new HashMap + (aggregateAttributePositionList.size()); + state.groupByAggregateAttributeValueMap.put(currentGroupByKey, + currentAggregateAttributeValueMap); + } } - } else if (groupedComplexEvent.getType() == ComplexEvent.Type.EXPIRED) { - for (Iterator iterator = eventList.iterator(); iterator.hasNext(); ) { - GroupedComplexEvent currentEvent = iterator.next(); - if (comparator.compare(currentEvent.getComplexEvent(), groupedComplexEvent - .getComplexEvent()) == 0) { - iterator.remove(); - for (Integer position : aggregateAttributePositionList) { - currentAggregateAttributeValueMap.put(position, - groupedComplexEvent.getOutputData()[position]); + if (groupedComplexEvent.getType() == ComplexEvent.Type.CURRENT) { + state.eventList.add(groupedComplexEvent); + for (Integer position : aggregateAttributePositionList) { + currentAggregateAttributeValueMap.put(position, event.getOutputData()[position]); + } + } else if (groupedComplexEvent.getType() == ComplexEvent.Type.EXPIRED) { + for (Iterator iterator = state.eventList.iterator(); + iterator.hasNext(); ) { + GroupedComplexEvent currentEvent = iterator.next(); + if (comparator.compare(currentEvent.getComplexEvent(), groupedComplexEvent + .getComplexEvent()) == 0) { + iterator.remove(); + for (Integer position : aggregateAttributePositionList) { + currentAggregateAttributeValueMap.put(position, + groupedComplexEvent.getOutputData()[position]); + } + break; } - break; } + } else if (groupedComplexEvent.getType() == ComplexEvent.Type.RESET) { + state.eventList.clear(); + state.groupByAggregateAttributeValueMap.clear(); } - } else if (groupedComplexEvent.getType() == ComplexEvent.Type.RESET) { - eventList.clear(); - groupByAggregateAttributeValueMap.clear(); } } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); } } - private void tryFlushEvents(List> outputEventChunks, ComplexEvent event) { - if (event.getTimestamp() >= scheduledTime) { - constructOutputChunk(outputEventChunks); - scheduledTime = scheduledTime + value; - scheduler.notifyAt(scheduledTime); + private void tryFlushEvents(List> outputEventChunks, ComplexEvent event, + AggregationGroupByRateLimiterState state) { + if (event.getTimestamp() >= state.scheduledTime) { + constructOutputChunk(outputEventChunks, state); + state.scheduledTime = state.scheduledTime + value; + scheduler.notifyAt(state.scheduledTime); } } - private void constructOutputChunk(List> outputEventChunks) { + private void constructOutputChunk(List> outputEventChunks, + AggregationGroupByRateLimiterState state) { ComplexEventChunk outputEventChunk = new ComplexEventChunk(false); Set outputGroupingKeys = new HashSet<>(); - for (GroupedComplexEvent originalComplexEvent : eventList) { + for (GroupedComplexEvent originalComplexEvent : state.eventList) { String currentGroupByKey = originalComplexEvent.getGroupKey(); if (!outputGroupingKeys.contains(currentGroupByKey)) { outputGroupingKeys.add(currentGroupByKey); - Map currentAggregateAttributeValueMap = groupByAggregateAttributeValueMap.get + Map currentAggregateAttributeValueMap = state.groupByAggregateAttributeValueMap.get (currentGroupByKey); ComplexEvent eventCopy = cloneComplexEvent(originalComplexEvent.getComplexEvent()); for (Integer position : aggregateAttributePositionList) { @@ -135,29 +151,36 @@ private void constructOutputChunk(List> outputEv outputEventChunks.add(outputEventChunk); } - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + class AggregationGroupByRateLimiterState extends AggregationRateLimiterState { + + private List eventList; + private Map> groupByAggregateAttributeValueMap; + + public AggregationGroupByRateLimiterState() { + groupByAggregateAttributeValueMap = new HashMap<>(); + eventList = new LinkedList<>(); + } + + @Override + public boolean canDestroy() { + return groupByAggregateAttributeValueMap.isEmpty() && eventList.isEmpty() && scheduledTime == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("EventList", eventList); state.put("GroupByAggregateAttributeValueMap", groupByAggregateAttributeValueMap); + state.put("ScheduledTime", scheduledTime); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - eventList = (List) state.get("EventList"); - groupByAggregateAttributeValueMap = (Map>) state.get - ("GroupByAggregateAttributeValueMap"); - } - - @Override - public SnapshotOutputRateLimiter clone(String key, WrappedSnapshotOutputRateLimiter - wrappedSnapshotOutputRateLimiter) { - return new AggregationGroupByWindowedPerSnapshotOutputRateLimiter(id + key, value, scheduledExecutorService, - aggregateAttributePositionList, - wrappedSnapshotOutputRateLimiter, - siddhiQueryContext); + @Override + public void restore(Map state) { + eventList = (List) state.get("EventList"); + groupByAggregateAttributeValueMap = (Map>) state.get + ("GroupByAggregateAttributeValueMap"); + scheduledTime = (Long) state.get("ScheduledTime"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AggregationWindowedPerSnapshotOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AggregationWindowedPerSnapshotOutputRateLimiter.java index b423ec3b3e..45c83fda5b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AggregationWindowedPerSnapshotOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AggregationWindowedPerSnapshotOutputRateLimiter.java @@ -21,9 +21,11 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.Collections; @@ -33,36 +35,27 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link PerSnapshotOutputRateLimiter} for queries with Windows and Aggregators. */ -public class AggregationWindowedPerSnapshotOutputRateLimiter extends SnapshotOutputRateLimiter { +public class AggregationWindowedPerSnapshotOutputRateLimiter + extends SnapshotOutputRateLimiter + { protected final Long value; - protected final ScheduledExecutorService scheduledExecutorService; - protected String id; protected Comparator comparator; protected List aggregateAttributePositionList; protected Scheduler scheduler; - protected long scheduledTime; - private List eventList; - private Map aggregateAttributeValueMap; - protected AggregationWindowedPerSnapshotOutputRateLimiter(String id, Long value, - ScheduledExecutorService scheduledExecutorService, + protected AggregationWindowedPerSnapshotOutputRateLimiter(Long value, final List aggregateAttributePositionList, WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, - SiddhiQueryContext siddhiQueryContext) { - super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext); - this.id = id; + boolean groupBy, SiddhiQueryContext siddhiQueryContext) { + super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext, groupBy); this.value = value; - this.scheduledExecutorService = scheduledExecutorService; - this.eventList = new LinkedList(); this.aggregateAttributePositionList = aggregateAttributePositionList; Collections.sort(aggregateAttributePositionList); - aggregateAttributeValueMap = new HashMap(aggregateAttributePositionList.size()); this.comparator = new Comparator() { Integer[] aggregateAttributePositions = aggregateAttributePositionList.toArray(new Integer[aggregateAttributePositionList.size()]); @@ -93,99 +86,120 @@ public int compare(ComplexEvent event1, ComplexEvent event2) { }; } + @Override + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new AggregationRateLimiterState(); + } + @Override public void process(ComplexEventChunk complexEventChunk) { complexEventChunk.reset(); ArrayList> outputEventChunks = new ArrayList>(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - tryFlushEvents(outputEventChunks, event); - } else { - complexEventChunk.remove(); - tryFlushEvents(outputEventChunks, event); - if (event.getType() == ComplexEvent.Type.CURRENT) { - eventList.add(event); - for (Integer position : aggregateAttributePositionList) { - aggregateAttributeValueMap.put(position, event.getOutputData()[position]); - } - } else if (event.getType() == ComplexEvent.Type.EXPIRED) { - for (Iterator iterator = eventList.iterator(); iterator.hasNext(); ) { - ComplexEvent complexEvent = iterator.next(); - if (comparator.compare(event, complexEvent) == 0) { - iterator.remove(); - for (Integer position : aggregateAttributePositionList) { - aggregateAttributeValueMap.put(position, event.getOutputData()[position]); + AggregationRateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.TIMER) { + tryFlushEvents(outputEventChunks, event, state); + } else { + complexEventChunk.remove(); + tryFlushEvents(outputEventChunks, event, state); + if (event.getType() == ComplexEvent.Type.CURRENT) { + state.eventList.add(event); + for (Integer position : aggregateAttributePositionList) { + state.aggregateAttributeValueMap.put(position, event.getOutputData()[position]); + } + } else if (event.getType() == ComplexEvent.Type.EXPIRED) { + for (Iterator iterator = state.eventList.iterator(); iterator.hasNext(); ) { + ComplexEvent complexEvent = iterator.next(); + if (comparator.compare(event, complexEvent) == 0) { + iterator.remove(); + for (Integer position : aggregateAttributePositionList) { + state.aggregateAttributeValueMap.put(position, event.getOutputData()[position]); + } + break; } - break; } + } else if (event.getType() == ComplexEvent.Type.RESET) { + state.eventList.clear(); + state.aggregateAttributeValueMap.clear(); } - } else if (event.getType() == ComplexEvent.Type.RESET) { - eventList.clear(); - aggregateAttributeValueMap.clear(); } } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); } } - private void tryFlushEvents(ArrayList> outputEventChunks, ComplexEvent event) { - if (event.getTimestamp() >= scheduledTime) { + private void tryFlushEvents(ArrayList> outputEventChunks, ComplexEvent event, + AggregationRateLimiterState state) { + if (event.getTimestamp() >= state.scheduledTime) { ComplexEventChunk outputEventChunk = new ComplexEventChunk(false); - for (ComplexEvent originalComplexEvent : eventList) { + for (ComplexEvent originalComplexEvent : state.eventList) { ComplexEvent eventCopy = cloneComplexEvent(originalComplexEvent); for (Integer position : aggregateAttributePositionList) { - eventCopy.getOutputData()[position] = aggregateAttributeValueMap.get(position); + eventCopy.getOutputData()[position] = state.aggregateAttributeValueMap.get(position); } outputEventChunk.add(eventCopy); } outputEventChunks.add(outputEventChunk); - scheduledTime += value; - scheduler.notifyAt(scheduledTime); + state.scheduledTime += value; + scheduler.notifyAt(state.scheduledTime); } } @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); + public void partitionCreated() { + AggregationRateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = System.currentTimeMillis(); + state.scheduledTime = currentTime + value; + scheduler.notifyAt(state.scheduledTime); + } + } finally { + stateHolder.returnState(state); + } } - @Override - public void stop() { - //Nothing to stop - } + class AggregationRateLimiterState extends State { - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + private List eventList; + private Map aggregateAttributeValueMap; + protected long scheduledTime; + + public AggregationRateLimiterState() { + this.eventList = new LinkedList<>(); + aggregateAttributeValueMap = new HashMap<>(aggregateAttributePositionList.size()); + } + + @Override + public boolean canDestroy() { + return aggregateAttributeValueMap.isEmpty() && eventList.isEmpty() && scheduledTime == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("EventList", eventList); state.put("AggregateAttributeValueMap", aggregateAttributeValueMap); + state.put("ScheduledTime", scheduledTime); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - eventList = (List) state.get("EventList"); - aggregateAttributeValueMap = (Map) state.get("AdgregateAttributeValueMap"); - } - - @Override - public SnapshotOutputRateLimiter clone(String key, WrappedSnapshotOutputRateLimiter - wrappedSnapshotOutputRateLimiter) { - return new AggregationWindowedPerSnapshotOutputRateLimiter(id + key, value, scheduledExecutorService, - aggregateAttributePositionList, wrappedSnapshotOutputRateLimiter, siddhiQueryContext); + @Override + public void restore(Map state) { + eventList = (List) state.get("EventList"); + aggregateAttributeValueMap = (Map) state.get("AdgregateAttributeValueMap"); + scheduledTime = (Long) state.get("ScheduledTime"); + } } - - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter.java index 5d8f2bd74e..e4fa061b24 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter.java @@ -23,9 +23,11 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.GroupedComplexEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.HashMap; @@ -33,76 +35,77 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link PerSnapshotOutputRateLimiter} for queries with GroupBy, Aggregators and Windows which * will output all events. */ -public class AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter extends SnapshotOutputRateLimiter { +public class AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter + extends SnapshotOutputRateLimiter { private final Long value; - private final ScheduledExecutorService scheduledExecutorService; - private String id; - private Map groupByKeyEvents = new LinkedHashMap(); private Scheduler scheduler; - private long scheduledTime; - public AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter(String id, Long value, - ScheduledExecutorService scheduledExecutorService, + public AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter(Long value, WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, + boolean groupBy, SiddhiQueryContext siddhiQueryContext) { - super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext); - this.id = id; + super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext, groupBy); this.value = value; - this.scheduledExecutorService = scheduledExecutorService; } @Override - public SnapshotOutputRateLimiter clone(String key, WrappedSnapshotOutputRateLimiter - wrappedSnapshotOutputRateLimiter) { - return new AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter(id + key, value, - scheduledExecutorService, wrappedSnapshotOutputRateLimiter, siddhiQueryContext); + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { List> outputEventChunks = new ArrayList>(); complexEventChunk.reset(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - tryFlushEvents(outputEventChunks, event); - } else { - complexEventChunk.remove(); - tryFlushEvents(outputEventChunks, event); - GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); - LastEventHolder lastEventHolder = groupByKeyEvents.get(groupedComplexEvent.getGroupKey()); - if (lastEventHolder == null) { - lastEventHolder = new LastEventHolder(); - groupByKeyEvents.put(groupedComplexEvent.getGroupKey(), lastEventHolder); - } - if (groupedComplexEvent.getType() == ComplexEvent.Type.CURRENT) { - lastEventHolder.addLastInEvent(groupedComplexEvent.getComplexEvent()); - } else if (groupedComplexEvent.getType() == ComplexEvent.Type.EXPIRED) { - lastEventHolder.removeLastInEvent(groupedComplexEvent.getComplexEvent()); - } else if (groupedComplexEvent.getType() == ComplexEvent.Type.RESET) { - groupByKeyEvents.clear(); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.TIMER) { + tryFlushEvents(outputEventChunks, event, state); + } else { + complexEventChunk.remove(); + tryFlushEvents(outputEventChunks, event, state); + GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); + LastEventHolder lastEventHolder = state.groupByKeyEvents.get(groupedComplexEvent.getGroupKey()); + if (lastEventHolder == null) { + lastEventHolder = new LastEventHolder(); + state.groupByKeyEvents.put(groupedComplexEvent.getGroupKey(), lastEventHolder); + } + if (groupedComplexEvent.getType() == ComplexEvent.Type.CURRENT) { + lastEventHolder.addLastInEvent(groupedComplexEvent.getComplexEvent()); + } else if (groupedComplexEvent.getType() == ComplexEvent.Type.EXPIRED) { + lastEventHolder.removeLastInEvent(groupedComplexEvent.getComplexEvent()); + } else if (groupedComplexEvent.getType() == ComplexEvent.Type.RESET) { + state.groupByKeyEvents.clear(); + } } } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); } - } - private void tryFlushEvents(List> outputEventChunks, ComplexEvent event) { - if (event.getTimestamp() >= scheduledTime) { + + private void tryFlushEvents(List> outputEventChunks, ComplexEvent event, + RateLimiterState state) { + if (event.getTimestamp() >= state.scheduledTime) { ComplexEventChunk outputEventChunk = new ComplexEventChunk(false); - for (Iterator> iterator = groupByKeyEvents.entrySet().iterator(); + for (Iterator> iterator = state.groupByKeyEvents.entrySet().iterator(); iterator.hasNext(); ) { Map.Entry lastEventHolderEntry = iterator.next(); @@ -115,38 +118,23 @@ private void tryFlushEvents(List> outputEventChu } } outputEventChunks.add(outputEventChunk); - scheduledTime += value; - scheduler.notifyAt(scheduledTime); + state.scheduledTime += value; + scheduler.notifyAt(state.scheduledTime); } } @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); - } - - @Override - public void stop() { - //Nothing to stop - } - - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("GroupByKeyEvents", groupByKeyEvents); + public void partitionCreated() { + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = System.currentTimeMillis(); + state.scheduledTime = currentTime + value; + scheduler.notifyAt(state.scheduledTime); + } + } finally { + stateHolder.returnState(state); } - return state; - } - - @Override - public synchronized void restoreState(Map state) { - groupByKeyEvents = (Map) state.get("GroupByKeyEvents"); } private class LastEventHolder { @@ -169,4 +157,28 @@ public void checkAndClearLastInEvent() { } } } + + class RateLimiterState extends State { + public long scheduledTime; + private Map groupByKeyEvents = new LinkedHashMap<>(); + + @Override + public boolean canDestroy() { + return groupByKeyEvents.isEmpty() && scheduledTime == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("GroupByKeyEvents", groupByKeyEvents); + state.put("ScheduledTime", scheduledTime); + return state; + } + + @Override + public void restore(Map state) { + groupByKeyEvents = (Map) state.get("GroupByKeyEvents"); + scheduledTime = (Long) state.get("ScheduledTime"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AllAggregationPerSnapshotOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AllAggregationPerSnapshotOutputRateLimiter.java index 67764376b6..f030d47189 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AllAggregationPerSnapshotOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/AllAggregationPerSnapshotOutputRateLimiter.java @@ -21,108 +21,119 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link PerSnapshotOutputRateLimiter} for queries with Aggregators which will output all events. */ -public class AllAggregationPerSnapshotOutputRateLimiter extends SnapshotOutputRateLimiter { +public class AllAggregationPerSnapshotOutputRateLimiter + extends SnapshotOutputRateLimiter { private final Long value; - private final ScheduledExecutorService scheduledExecutorService; - private String id; - private ComplexEvent lastEvent = null; private Scheduler scheduler; - private long scheduledTime; - public AllAggregationPerSnapshotOutputRateLimiter(String id, Long value, ScheduledExecutorService - scheduledExecutorService, WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, - SiddhiQueryContext siddhiQueryContext) { - super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext); - this.id = id; + public AllAggregationPerSnapshotOutputRateLimiter(Long value, + WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, + boolean groupBy, SiddhiQueryContext siddhiQueryContext) { + super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext, groupBy); this.value = value; - this.scheduledExecutorService = scheduledExecutorService; + } + + @Override + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { List> outputEventChunks = new ArrayList>(); complexEventChunk.reset(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - tryFlushEvents(outputEventChunks, event); - } else { - tryFlushEvents(outputEventChunks, event); - if (event.getType() == ComplexEvent.Type.CURRENT) { - complexEventChunk.remove(); - lastEvent = event; + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.TIMER) { + tryFlushEvents(outputEventChunks, event, state); } else { - lastEvent = null; + tryFlushEvents(outputEventChunks, event, state); + if (event.getType() == ComplexEvent.Type.CURRENT) { + complexEventChunk.remove(); + state.lastEvent = event; + } else { + state.lastEvent = null; + } } } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); } } - private void tryFlushEvents(List> outputEventChunks, ComplexEvent event) { - if (event.getTimestamp() >= scheduledTime) { + + private void tryFlushEvents(List> outputEventChunks, ComplexEvent event, + RateLimiterState state) { + if (event.getTimestamp() >= state.scheduledTime) { ComplexEventChunk outputEventChunk = new ComplexEventChunk(false); - if (lastEvent != null) { - outputEventChunk.add(cloneComplexEvent(lastEvent)); + if (state.lastEvent != null) { + outputEventChunk.add(cloneComplexEvent(state.lastEvent)); } outputEventChunks.add(outputEventChunk); - scheduledTime += value; - scheduler.notifyAt(scheduledTime); + state.scheduledTime += value; + scheduler.notifyAt(state.scheduledTime); } } @Override - public SnapshotOutputRateLimiter clone(String key, WrappedSnapshotOutputRateLimiter - wrappedSnapshotOutputRateLimiter) { - return new AllAggregationPerSnapshotOutputRateLimiter(id + key, value, scheduledExecutorService, - wrappedSnapshotOutputRateLimiter, - siddhiQueryContext); + public void partitionCreated() { + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = System.currentTimeMillis(); + state.scheduledTime = currentTime + value; + scheduler.notifyAt(state.scheduledTime); + } + } finally { + stateHolder.returnState(state); + } } - @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); - } + class RateLimiterState extends State { + public long scheduledTime; + private ComplexEvent lastEvent = null; - @Override - public void stop() { - //Nothing to stop - } + @Override + public boolean canDestroy() { + return lastEvent == null && scheduledTime == 0; + } - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("LastEvent", lastEvent); + state.put("ScheduledTime", scheduledTime); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - lastEvent = (ComplexEvent) state.get("LastEvent"); + @Override + public void restore(Map state) { + lastEvent = (ComplexEvent) state.get("LastEvent"); + scheduledTime = (Long) state.get("ScheduledTime"); + } } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/GroupByPerSnapshotOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/GroupByPerSnapshotOutputRateLimiter.java index 7320205d7b..a3fe00f7b4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/GroupByPerSnapshotOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/GroupByPerSnapshotOutputRateLimiter.java @@ -22,36 +22,39 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.GroupedComplexEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link PerSnapshotOutputRateLimiter} for queries with GroupBy. */ -public class GroupByPerSnapshotOutputRateLimiter extends SnapshotOutputRateLimiter { +public class GroupByPerSnapshotOutputRateLimiter + extends SnapshotOutputRateLimiter { private final Long value; - private String id; - private ScheduledExecutorService scheduledExecutorService; - private Map groupByKeyEvents = new LinkedHashMap(); private Scheduler scheduler; - private long scheduledTime; - public GroupByPerSnapshotOutputRateLimiter(String id, Long value, - ScheduledExecutorService scheduledExecutorService, + public GroupByPerSnapshotOutputRateLimiter(Long value, WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, - SiddhiQueryContext siddhiQueryContext) { - super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext); - this.id = id; + boolean groupBy, SiddhiQueryContext siddhiQueryContext) { + super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext, groupBy); this.value = value; - this.scheduledExecutorService = scheduledExecutorService; + } + + @Override + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new RateLimiterState(); } /** @@ -63,71 +66,80 @@ public GroupByPerSnapshotOutputRateLimiter(String id, Long value, public void process(ComplexEventChunk complexEventChunk) { List> outputEventChunks = new ArrayList>(); complexEventChunk.reset(); - synchronized (this) { - complexEventChunk.reset(); - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - tryFlushEvents(outputEventChunks, event); - } else if (event.getType() == ComplexEvent.Type.CURRENT) { - complexEventChunk.remove(); - tryFlushEvents(outputEventChunks, event); - GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); - groupByKeyEvents.put(groupedComplexEvent.getGroupKey(), groupedComplexEvent.getComplexEvent()); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + complexEventChunk.reset(); + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.TIMER) { + tryFlushEvents(outputEventChunks, event, state); + } else if (event.getType() == ComplexEvent.Type.CURRENT) { + complexEventChunk.remove(); + tryFlushEvents(outputEventChunks, event, state); + GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); + state.groupByKeyEvents.put(groupedComplexEvent.getGroupKey(), + groupedComplexEvent.getComplexEvent()); + } } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); } } - private void tryFlushEvents(List> outputEventChunks, ComplexEvent event) { - if (event.getTimestamp() >= scheduledTime) { + + private void tryFlushEvents(List> outputEventChunks, ComplexEvent event, + RateLimiterState state) { + if (event.getTimestamp() >= state.scheduledTime) { ComplexEventChunk outputEventChunk = new ComplexEventChunk(false); - for (ComplexEvent complexEvent : groupByKeyEvents.values()) { + for (ComplexEvent complexEvent : state.groupByKeyEvents.values()) { outputEventChunk.add(cloneComplexEvent(complexEvent)); } outputEventChunks.add(outputEventChunk); - scheduledTime += value; - scheduler.notifyAt(scheduledTime); + state.scheduledTime += value; + scheduler.notifyAt(state.scheduledTime); } } @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); + public void partitionCreated() { + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = System.currentTimeMillis(); + state.scheduledTime = currentTime + value; + scheduler.notifyAt(state.scheduledTime); + } + } finally { + stateHolder.returnState(state); + } } - @Override - public void stop() { - //Nothing to stop - } + class RateLimiterState extends State { + public long scheduledTime; + private Map groupByKeyEvents = new LinkedHashMap<>(); - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("GroupByKeyEvents", groupByKeyEvents); + @Override + public boolean canDestroy() { + return groupByKeyEvents.isEmpty() && scheduledTime == 0; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - groupByKeyEvents = (Map) state.get("groupByKeyEvents"); - } + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("GroupByKeyEvents", groupByKeyEvents); + state.put("ScheduledTime", scheduledTime); + return state; + } - @Override - public SnapshotOutputRateLimiter clone(String key, WrappedSnapshotOutputRateLimiter - wrappedSnapshotOutputRateLimiter) { - return new GroupByPerSnapshotOutputRateLimiter(id + key, value, scheduledExecutorService, - wrappedSnapshotOutputRateLimiter, siddhiQueryContext); + @Override + public void restore(Map state) { + groupByKeyEvents = (Map) state.get("groupByKeyEvents"); + scheduledTime = (Long) state.get("ScheduledTime"); + } } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/PerSnapshotOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/PerSnapshotOutputRateLimiter.java index 5df20c24bc..f260ab1f2e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/PerSnapshotOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/PerSnapshotOutputRateLimiter.java @@ -22,110 +22,124 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Parent implementation for per event periodic snapshot rate limiting. Multiple implementations of this will be * there to represent different queries. Snapshot rate limiting will only emit current events representing the * snapshot period. */ -public class PerSnapshotOutputRateLimiter extends SnapshotOutputRateLimiter { +public class PerSnapshotOutputRateLimiter + extends SnapshotOutputRateLimiter { private final Long value; - private String id; - private ScheduledExecutorService scheduledExecutorService; - private ComplexEventChunk eventChunk = new ComplexEventChunk(false); - private ComplexEvent lastEvent; private Scheduler scheduler; - private long scheduledTime; - public PerSnapshotOutputRateLimiter(String id, Long value, ScheduledExecutorService scheduledExecutorService, + public PerSnapshotOutputRateLimiter(Long value, WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, - SiddhiQueryContext siddhiQueryContext) { - super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext); - this.id = id; + boolean groupBy, SiddhiQueryContext siddhiQueryContext) { + super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext, groupBy); this.value = value; - this.scheduledExecutorService = scheduledExecutorService; + } + + @Override + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { List> outputEventChunks = new ArrayList>(); complexEventChunk.reset(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - tryFlushEvents(outputEventChunks, event); - } else if (event.getType() == ComplexEvent.Type.CURRENT) { - complexEventChunk.remove(); - tryFlushEvents(outputEventChunks, event); - lastEvent = event; - } else { - tryFlushEvents(outputEventChunks, event); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.TIMER) { + tryFlushEvents(outputEventChunks, event, state); + } else if (event.getType() == ComplexEvent.Type.CURRENT) { + complexEventChunk.remove(); + tryFlushEvents(outputEventChunks, event, state); + state.lastEvent = event; + } else { + tryFlushEvents(outputEventChunks, event, state); + } } } - + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); } } - private void tryFlushEvents(List> outputEventChunks, ComplexEvent event) { - if (event.getTimestamp() >= scheduledTime) { + private void tryFlushEvents(List> outputEventChunks, ComplexEvent event, + RateLimiterState state) { + if (event.getTimestamp() >= state.scheduledTime) { ComplexEventChunk outputEventChunk = new ComplexEventChunk(false); - if (lastEvent != null) { - outputEventChunk.add(cloneComplexEvent(lastEvent)); + if (state.lastEvent != null) { + outputEventChunk.add(cloneComplexEvent(state.lastEvent)); } outputEventChunks.add(outputEventChunk); - scheduledTime += value; - scheduler.notifyAt(scheduledTime); + state.scheduledTime += value; + scheduler.notifyAt(state.scheduledTime); } } @Override - public SnapshotOutputRateLimiter clone(String key, WrappedSnapshotOutputRateLimiter - wrappedSnapshotOutputRateLimiter) { - return new PerSnapshotOutputRateLimiter(id + key, value, scheduledExecutorService, - wrappedSnapshotOutputRateLimiter, siddhiQueryContext); + public void partitionCreated() { + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = System.currentTimeMillis(); + state.scheduledTime = currentTime + value; + scheduler.notifyAt(state.scheduledTime); + } + } finally { + stateHolder.returnState(state); + } } - @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); - } + class RateLimiterState extends State { - @Override - public void stop() { - //Nothing to stop - } + public long scheduledTime; + private ComplexEventChunk eventChunk = new ComplexEventChunk(false); + private ComplexEvent lastEvent; - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + @Override + public boolean canDestroy() { + return eventChunk.getFirst() == null && scheduledTime == 0 && lastEvent == null; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("EventChunk", eventChunk.getFirst()); + state.put("ScheduledTime", scheduledTime); + state.put("LastEvent", lastEvent); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - eventChunk.clear(); - eventChunk.add((ComplexEvent) state.get("EventList")); + @Override + public void restore(Map state) { + eventChunk.clear(); + eventChunk.add((ComplexEvent) state.get("EventList")); + scheduledTime = (Long) state.get("ScheduledTime"); + lastEvent = (ComplexEvent) state.get("LastEvent"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/SnapshotOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/SnapshotOutputRateLimiter.java index 596205a2eb..cce558f0bd 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/SnapshotOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/SnapshotOutputRateLimiter.java @@ -28,16 +28,20 @@ import io.siddhi.core.util.Schedulable; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.lock.LockWrapper; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.core.util.snapshot.state.StateHolder; import org.apache.log4j.Logger; -import java.util.Map; - /** * Parent implementation to run the {@link Scheduler} to handle periodic snapshot rate * limiting. + * + * @param current state of the RateLimiter */ -public abstract class SnapshotOutputRateLimiter implements Schedulable { +public abstract class SnapshotOutputRateLimiter implements Schedulable { private static final Logger log = Logger.getLogger(SnapshotOutputRateLimiter.class); + protected final StateHolder stateHolder; protected StreamEventCloner streamEventCloner; protected StateEventCloner stateEventCloner; protected SiddhiQueryContext siddhiQueryContext; @@ -46,15 +50,15 @@ public abstract class SnapshotOutputRateLimiter implements Schedulable { private boolean receiveStreamEvent; protected SnapshotOutputRateLimiter(WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, - SiddhiQueryContext siddhiQueryContext) { + SiddhiQueryContext siddhiQueryContext, boolean groupBy) { this.wrappedSnapshotOutputRateLimiter = wrappedSnapshotOutputRateLimiter; this.siddhiQueryContext = siddhiQueryContext; + stateHolder = siddhiQueryContext.generateStateHolder(this.getClass().getName(), groupBy, init()); } - public abstract void process(ComplexEventChunk complexEventChunk); + protected abstract StateFactory init(); - public abstract SnapshotOutputRateLimiter clone(String key, WrappedSnapshotOutputRateLimiter - wrappedSnapshotOutputRateLimiter); + public abstract void process(ComplexEventChunk complexEventChunk); public void setStreamEventCloner(StreamEventCloner streamEventCloner) { this.streamEventCloner = streamEventCloner; @@ -84,13 +88,7 @@ protected ComplexEvent cloneComplexEvent(ComplexEvent complexEvent) { } } - public abstract void start(); - - public abstract void stop(); - - public abstract Map currentState(); - - public abstract void restoreState(Map state); + public abstract void partitionCreated(); public void setQueryLock(LockWrapper lockWrapper) { this.lockWrapper = lockWrapper; diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/WindowedPerSnapshotOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/WindowedPerSnapshotOutputRateLimiter.java index 0fe5b82a74..f8ab7c4e62 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/WindowedPerSnapshotOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/WindowedPerSnapshotOutputRateLimiter.java @@ -22,9 +22,11 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.GroupedComplexEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.Arrays; @@ -34,126 +36,130 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link PerSnapshotOutputRateLimiter} for queries with Windows. */ -public class WindowedPerSnapshotOutputRateLimiter extends SnapshotOutputRateLimiter { +public class WindowedPerSnapshotOutputRateLimiter + extends SnapshotOutputRateLimiter { private final Long value; - private final ScheduledExecutorService scheduledExecutorService; - private String id; - private List eventList; private Comparator comparator; private Scheduler scheduler; - private long scheduledTime; - public WindowedPerSnapshotOutputRateLimiter(String id, Long value, ScheduledExecutorService - scheduledExecutorService, WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, - SiddhiQueryContext siddhiQueryContext) { - super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext); - this.id = id; + public WindowedPerSnapshotOutputRateLimiter(Long value, + WrappedSnapshotOutputRateLimiter wrappedSnapshotOutputRateLimiter, + boolean groupBy, SiddhiQueryContext siddhiQueryContext) { + super(wrappedSnapshotOutputRateLimiter, siddhiQueryContext, groupBy); this.value = value; - this.scheduledExecutorService = scheduledExecutorService; - this.eventList = new LinkedList(); - this.comparator = new Comparator() { - - @Override - public int compare(ComplexEvent event1, ComplexEvent event2) { - if (Arrays.equals(event1.getOutputData(), event2.getOutputData())) { - return 0; - } else { - return 1; - - } + this.comparator = (Comparator) (event1, event2) -> { + if (Arrays.equals(event1.getOutputData(), event2.getOutputData())) { + return 0; + } else { + return 1; } }; } + @Override + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new RateLimiterState(); + } + @Override public void process(ComplexEventChunk complexEventChunk) { List> outputEventChunks = new ArrayList>(); complexEventChunk.reset(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event instanceof GroupedComplexEvent) { - event = ((GroupedComplexEvent) event).getComplexEvent(); - } - if (event.getType() == ComplexEvent.Type.TIMER) { - tryFlushEvents(outputEventChunks, event); - } else if (event.getType() == ComplexEvent.Type.CURRENT) { - complexEventChunk.remove(); - tryFlushEvents(outputEventChunks, event); - eventList.add(event); - } else if (event.getType() == ComplexEvent.Type.EXPIRED) { - tryFlushEvents(outputEventChunks, event); - for (Iterator iterator = eventList.iterator(); iterator.hasNext(); ) { - ComplexEvent currentEvent = iterator.next(); - if (comparator.compare(currentEvent, event) == 0) { - iterator.remove(); - break; + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event instanceof GroupedComplexEvent) { + event = ((GroupedComplexEvent) event).getComplexEvent(); + } + if (event.getType() == ComplexEvent.Type.TIMER) { + tryFlushEvents(outputEventChunks, event, state); + } else if (event.getType() == ComplexEvent.Type.CURRENT) { + complexEventChunk.remove(); + tryFlushEvents(outputEventChunks, event, state); + state.eventList.add(event); + } else if (event.getType() == ComplexEvent.Type.EXPIRED) { + tryFlushEvents(outputEventChunks, event, state); + for (Iterator iterator = state.eventList.iterator(); iterator.hasNext(); ) { + ComplexEvent currentEvent = iterator.next(); + if (comparator.compare(currentEvent, event) == 0) { + iterator.remove(); + break; + } } + } else if (event.getType() == ComplexEvent.Type.RESET) { + tryFlushEvents(outputEventChunks, event, state); + state.eventList.clear(); } - } else if (event.getType() == ComplexEvent.Type.RESET) { - tryFlushEvents(outputEventChunks, event); - eventList.clear(); } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); } } - private void tryFlushEvents(List> outputEventChunks, ComplexEvent event) { - if (event.getTimestamp() >= scheduledTime) { + private void tryFlushEvents(List> outputEventChunks, ComplexEvent event, + RateLimiterState state) { + if (event.getTimestamp() >= state.scheduledTime) { ComplexEventChunk outputEventChunk = new ComplexEventChunk(false); - for (ComplexEvent complexEvent : eventList) { + for (ComplexEvent complexEvent : state.eventList) { outputEventChunk.add(cloneComplexEvent(complexEvent)); } outputEventChunks.add(outputEventChunk); - scheduledTime = scheduledTime + value; - scheduler.notifyAt(scheduledTime); + state.scheduledTime = state.scheduledTime + value; + scheduler.notifyAt(state.scheduledTime); } } @Override - public SnapshotOutputRateLimiter clone(String key, WrappedSnapshotOutputRateLimiter - wrappedSnapshotOutputRateLimiter) { - return new WindowedPerSnapshotOutputRateLimiter(id + key, value, scheduledExecutorService, - wrappedSnapshotOutputRateLimiter, siddhiQueryContext); + public void partitionCreated() { + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = System.currentTimeMillis(); + state.scheduledTime = currentTime + value; + scheduler.notifyAt(state.scheduledTime); + } + } finally { + stateHolder.returnState(state); + } } - @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, - 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); - } + class RateLimiterState extends State { - @Override - public void stop() { - //Nothing to stop - } + private List eventList = new LinkedList<>(); + private long scheduledTime; - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + @Override + public boolean canDestroy() { + return eventList.isEmpty() && scheduledTime == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("EventList", eventList); + state.put("ScheduledTime", scheduledTime); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - eventList = (List) state.get("EventList"); + @Override + public void restore(Map state) { + eventList = (List) state.get("EventList"); + scheduledTime = (Long) state.get("ScheduledTime"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/WrappedSnapshotOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/WrappedSnapshotOutputRateLimiter.java index 7d5e9c2e84..9a24f4c33c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/WrappedSnapshotOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/snapshot/WrappedSnapshotOutputRateLimiter.java @@ -23,61 +23,51 @@ import io.siddhi.core.event.MetaComplexEvent; import io.siddhi.core.event.state.MetaStateEvent; import io.siddhi.core.event.state.StateEventCloner; -import io.siddhi.core.event.state.StateEventPool; +import io.siddhi.core.event.state.StateEventFactory; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; +import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregatorExecutor; import io.siddhi.core.query.selector.attribute.processor.AttributeProcessor; -import io.siddhi.core.query.selector.attribute.processor.executor.AbstractAggregationAttributeExecutor; import io.siddhi.core.util.lock.LockWrapper; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import java.util.ArrayList; import java.util.List; -import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link OutputRateLimiter} to be used by Snapshot Output Rate Limiting implementations. + * + * @param current state of the RateLimiter */ -public class WrappedSnapshotOutputRateLimiter extends OutputRateLimiter { +public class WrappedSnapshotOutputRateLimiter extends OutputRateLimiter { private final Long value; - private final ScheduledExecutorService scheduledExecutorService; private final boolean groupBy; private final boolean windowed; private SiddhiQueryContext siddhiQueryContext; private SnapshotOutputRateLimiter outputRateLimiter; - private String id; private List aggregateAttributePositionList = new ArrayList(); - public WrappedSnapshotOutputRateLimiter(String id, Long value, ScheduledExecutorService scheduledExecutorService, - boolean isGroupBy, boolean isWindowed, + public WrappedSnapshotOutputRateLimiter(Long value, + boolean groupBy, boolean isWindowed, SiddhiQueryContext siddhiQueryContext) { - this.id = id; this.value = value; - this.scheduledExecutorService = scheduledExecutorService; - this.groupBy = isGroupBy; + this.groupBy = groupBy; this.windowed = isWindowed; this.siddhiQueryContext = siddhiQueryContext; } - @Override - public OutputRateLimiter clone(String key) { - WrappedSnapshotOutputRateLimiter instance = new WrappedSnapshotOutputRateLimiter(id + key, value, - scheduledExecutorService, groupBy, windowed, siddhiQueryContext); - instance.outputRateLimiter = this.outputRateLimiter.clone(key, instance); - return instance; - } - public void init(SiddhiQueryContext siddhiQueryContext, LockWrapper lockWrapper) { - super.init(lockWrapper, siddhiQueryContext); + super.init(lockWrapper, groupBy, siddhiQueryContext); outputRateLimiter.setQueryLock(lockWrapper); } public void init(int outPutAttributeSize, List attributeProcessorList, MetaComplexEvent metaComplexEvent) { for (AttributeProcessor attributeProcessor : attributeProcessorList) { - if (attributeProcessor.getExpressionExecutor() instanceof AbstractAggregationAttributeExecutor) { + if (attributeProcessor.getExpressionExecutor() instanceof AttributeAggregatorExecutor) { aggregateAttributePositionList.add(attributeProcessor.getOutputPosition()); } } @@ -86,66 +76,65 @@ public void init(int outPutAttributeSize, List attributeProc if (groupBy) { if (outPutAttributeSize == aggregateAttributePositionList.size()) { //All Aggregation outputRateLimiter = new AllAggregationGroupByWindowedPerSnapshotOutputRateLimiter( - id, value, scheduledExecutorService, this, + value, this, groupBy, siddhiQueryContext); } else if (aggregateAttributePositionList.size() > 0) { //Some Aggregation outputRateLimiter = new AggregationGroupByWindowedPerSnapshotOutputRateLimiter( - id, value, scheduledExecutorService, aggregateAttributePositionList, - this, siddhiQueryContext); + value, aggregateAttributePositionList, + this, groupBy, siddhiQueryContext); } else { // No aggregation //GroupBy is same as Non GroupBy outputRateLimiter = new WindowedPerSnapshotOutputRateLimiter( - id, value, scheduledExecutorService, this, + value, this, groupBy, siddhiQueryContext); } } else { if (outPutAttributeSize == aggregateAttributePositionList.size()) { //All Aggregation outputRateLimiter = new AllAggregationPerSnapshotOutputRateLimiter( - id, value, scheduledExecutorService, this, + value, this, groupBy, siddhiQueryContext); } else if (aggregateAttributePositionList.size() > 0) { //Some Aggregation outputRateLimiter = new AggregationWindowedPerSnapshotOutputRateLimiter( - id, value, scheduledExecutorService, aggregateAttributePositionList, - this, siddhiQueryContext); + value, aggregateAttributePositionList, + this, groupBy, siddhiQueryContext); } else { // No aggregation outputRateLimiter = new WindowedPerSnapshotOutputRateLimiter( - id, value, scheduledExecutorService, this, - siddhiQueryContext); + value, this, + groupBy, siddhiQueryContext); } } } else { if (groupBy) { - outputRateLimiter = new GroupByPerSnapshotOutputRateLimiter(id, value, scheduledExecutorService, - this, siddhiQueryContext); + outputRateLimiter = new GroupByPerSnapshotOutputRateLimiter(value, + this, groupBy, siddhiQueryContext); } else { - outputRateLimiter = new PerSnapshotOutputRateLimiter(id, value, scheduledExecutorService, - this, siddhiQueryContext); + outputRateLimiter = new PerSnapshotOutputRateLimiter(value, + this, groupBy, siddhiQueryContext); } } if (metaComplexEvent instanceof MetaStateEvent) { - StateEventPool stateEventPool = new StateEventPool((MetaStateEvent) metaComplexEvent, 5); + StateEventFactory stateEventFactory = new StateEventFactory((MetaStateEvent) metaComplexEvent); outputRateLimiter.setStateEventCloner(new StateEventCloner((MetaStateEvent) metaComplexEvent, - stateEventPool)); + stateEventFactory)); } else { - StreamEventPool streamEventPool = new StreamEventPool((MetaStreamEvent) metaComplexEvent, 5); + StreamEventFactory streamEventFactory = new StreamEventFactory((MetaStreamEvent) metaComplexEvent); outputRateLimiter.setStreamEventCloner(new StreamEventCloner((MetaStreamEvent) metaComplexEvent, - streamEventPool)); + streamEventFactory)); } } - @Override - public void start() { - outputRateLimiter.start(); + public void partitionCreated() { + outputRateLimiter.partitionCreated(); } @Override - public void stop() { - outputRateLimiter.stop(); + protected StateFactory init() { + return null; } @Override @@ -157,13 +146,4 @@ public void passToCallBacks(ComplexEventChunk complexEventChunk) { sendToCallBacks(complexEventChunk); } - @Override - public Map currentState() { - return outputRateLimiter.currentState(); - } - - @Override - public void restoreState(Map state) { - outputRateLimiter.restoreState(state); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/AllPerTimeOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/AllPerTimeOutputRateLimiter.java index c162c51b65..ddffcad97a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/AllPerTimeOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/AllPerTimeOutputRateLimiter.java @@ -20,46 +20,43 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; import io.siddhi.core.util.Schedulable; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import org.apache.log4j.Logger; import java.util.ArrayList; import java.util.HashMap; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link OutputRateLimiter} which will collect pre-defined time period and the emit all * collected events as a batch. */ -public class AllPerTimeOutputRateLimiter extends OutputRateLimiter implements Schedulable { +public class AllPerTimeOutputRateLimiter + extends OutputRateLimiter implements Schedulable { private static final Logger log = Logger.getLogger(AllPerTimeOutputRateLimiter.class); private final Long value; private String id; - private ScheduledExecutorService scheduledExecutorService; private Scheduler scheduler; - private ComplexEventChunk allComplexEventChunk; - private long scheduledTime; - public AllPerTimeOutputRateLimiter(String id, Long value, ScheduledExecutorService scheduledExecutorService) { + public AllPerTimeOutputRateLimiter(String id, Long value) { this.id = id; this.value = value; - this.scheduledExecutorService = scheduledExecutorService; - allComplexEventChunk = new ComplexEventChunk(false); } @Override - public OutputRateLimiter clone(String key) { - AllPerTimeOutputRateLimiter instance = new AllPerTimeOutputRateLimiter(id + key, value, - scheduledExecutorService); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new RateLimiterState(); } @Override @@ -67,28 +64,33 @@ public void process(ComplexEventChunk complexEventChunk) { ArrayList> outputEventChunks = new ArrayList>(); complexEventChunk.reset(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - if (event.getTimestamp() >= scheduledTime) { - ComplexEvent first = allComplexEventChunk.getFirst(); - if (first != null) { - allComplexEventChunk.clear(); - ComplexEventChunk outputEventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); - outputEventChunk.add(first); - outputEventChunks.add(outputEventChunk); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.TIMER) { + if (event.getTimestamp() >= state.scheduledTime) { + ComplexEvent first = state.allComplexEventChunk.getFirst(); + if (first != null) { + state.allComplexEventChunk.clear(); + ComplexEventChunk outputEventChunk = new ComplexEventChunk + (complexEventChunk.isBatch()); + outputEventChunk.add(first); + outputEventChunks.add(outputEventChunk); + } + state.scheduledTime = state.scheduledTime + value; + scheduler.notifyAt(state.scheduledTime); } - scheduledTime = scheduledTime + value; - scheduler.notifyAt(scheduledTime); + } else if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type + .EXPIRED) { + complexEventChunk.remove(); + state.allComplexEventChunk.add(event); } - } else if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type - .EXPIRED) { - complexEventChunk.remove(); - allComplexEventChunk.add(event); } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); @@ -96,33 +98,42 @@ public void process(ComplexEventChunk complexEventChunk) { } @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); + public void partitionCreated() { + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = System.currentTimeMillis(); + state.scheduledTime = currentTime + value; + scheduler.notifyAt(state.scheduledTime); + } + } finally { + stateHolder.returnState(state); + } } - @Override - public void stop() { - //Nothing to stop - } + class RateLimiterState extends State { - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + private ComplexEventChunk allComplexEventChunk = new ComplexEventChunk<>(false); + private long scheduledTime; + + @Override + public boolean canDestroy() { + return allComplexEventChunk.getFirst() == null && scheduledTime == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("AllComplexEventChunk", allComplexEventChunk.getFirst()); + state.put("ScheduledTime", scheduledTime); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - allComplexEventChunk.clear(); - allComplexEventChunk.add((ComplexEvent) state.get("AllComplexEventChunk")); + @Override + public void restore(Map state) { + allComplexEventChunk.clear(); + allComplexEventChunk.add((ComplexEvent) state.get("AllComplexEventChunk")); + scheduledTime = (Long) state.get("ScheduledTime"); + } } - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/FirstGroupByPerTimeOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/FirstGroupByPerTimeOutputRateLimiter.java index 584a264bef..944accbc4f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/FirstGroupByPerTimeOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/FirstGroupByPerTimeOutputRateLimiter.java @@ -21,117 +21,87 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.GroupedComplexEvent; -import io.siddhi.core.event.stream.StreamEventPool; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; import io.siddhi.core.util.Schedulable; -import io.siddhi.core.util.Scheduler; -import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import org.apache.log4j.Logger; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link OutputRateLimiter} which will collect pre-defined time period and the emit only first * event. This implementation specifically represent GroupBy queries. */ -public class FirstGroupByPerTimeOutputRateLimiter extends OutputRateLimiter implements Schedulable { +public class FirstGroupByPerTimeOutputRateLimiter + extends OutputRateLimiter implements Schedulable { private static final Logger log = Logger.getLogger(FirstGroupByPerTimeOutputRateLimiter.class); private final Long value; private String id; - private List groupByKeys = new ArrayList(); - private ComplexEventChunk allComplexEventChunk; - private ScheduledExecutorService scheduledExecutorService; - private Scheduler scheduler; - private long scheduledTime; - public FirstGroupByPerTimeOutputRateLimiter(String id, Long value, ScheduledExecutorService - scheduledExecutorService) { + public FirstGroupByPerTimeOutputRateLimiter(String id, Long value) { this.id = id; this.value = value; - this.scheduledExecutorService = scheduledExecutorService; - this.allComplexEventChunk = new ComplexEventChunk(false); } @Override - public OutputRateLimiter clone(String key) { - FirstGroupByPerTimeOutputRateLimiter instance = new FirstGroupByPerTimeOutputRateLimiter(id + key, value, - scheduledExecutorService); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { - ArrayList> outputEventChunks = new ArrayList>(); + ComplexEventChunk outputEventChunk = new ComplexEventChunk<>(complexEventChunk.isBatch()); complexEventChunk.reset(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - if (event.getTimestamp() >= scheduledTime) { - if (allComplexEventChunk.getFirst() != null) { - ComplexEventChunk eventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); - eventChunk.add(allComplexEventChunk.getFirst()); - allComplexEventChunk.clear(); - groupByKeys.clear(); - outputEventChunks.add(eventChunk); - } else { - groupByKeys.clear(); - } - scheduledTime = scheduledTime + value; - scheduler.notifyAt(scheduledTime); - } - } else if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type - .EXPIRED) { + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = siddhiQueryContext.getSiddhiAppContext(). + getTimestampGenerator().currentTime(); + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + complexEventChunk.remove(); GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); - if (!groupByKeys.contains(groupedComplexEvent.getGroupKey())) { - complexEventChunk.remove(); - groupByKeys.add(groupedComplexEvent.getGroupKey()); - allComplexEventChunk.add(groupedComplexEvent.getComplexEvent()); + Long outputTime = state.groupByOutputTime.get(groupedComplexEvent.getGroupKey()); + if (outputTime == null || outputTime + value <= currentTime) { + state.groupByOutputTime.put(groupedComplexEvent.getGroupKey(), currentTime); + outputEventChunk.add(groupedComplexEvent); } } } + } finally { + stateHolder.returnState(state); } - for (ComplexEventChunk eventChunk : outputEventChunks) { - sendToCallBacks(eventChunk); + if (outputEventChunk.getFirst() != null) { + sendToCallBacks(outputEventChunk); } } @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); - } + public void partitionCreated() { - @Override - public void stop() { - //Nothing to stop } - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("AllComplexEventChunk", allComplexEventChunk.getFirst()); - state.put("GroupByKeys", groupByKeys); + class RateLimiterState extends State { + + private Map groupByOutputTime = new HashMap(); + + @Override + public boolean canDestroy() { + return groupByOutputTime.isEmpty(); } - return state; - } - @Override - public synchronized void restoreState(Map state) { - allComplexEventChunk.clear(); - allComplexEventChunk.add((ComplexEvent) state.get("AllComplexEventChunk")); - groupByKeys = (List) state.get("GroupByKeys"); - } + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("GroupByOutputTime", groupByOutputTime); + return state; + } + @Override + public void restore(Map state) { + groupByOutputTime = (Map) state.get("GroupByOutputTime"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/FirstPerTimeOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/FirstPerTimeOutputRateLimiter.java index 116f16b57d..6550846200 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/FirstPerTimeOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/FirstPerTimeOutputRateLimiter.java @@ -20,106 +20,84 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; -import io.siddhi.core.event.stream.StreamEventPool; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; import io.siddhi.core.util.Schedulable; -import io.siddhi.core.util.Scheduler; -import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import org.apache.log4j.Logger; -import java.util.ArrayList; import java.util.HashMap; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link OutputRateLimiter} which will collect pre-defined time period and the emit only first * event. */ -public class FirstPerTimeOutputRateLimiter extends OutputRateLimiter implements Schedulable { +public class FirstPerTimeOutputRateLimiter + extends OutputRateLimiter implements Schedulable { private static final Logger log = Logger.getLogger(FirstPerTimeOutputRateLimiter.class); private final Long value; private String id; - private ComplexEvent firstEvent = null; - private ScheduledExecutorService scheduledExecutorService; - private Scheduler scheduler; - private long scheduledTime; - public FirstPerTimeOutputRateLimiter(String id, Long value, ScheduledExecutorService scheduledExecutorService) { + public FirstPerTimeOutputRateLimiter(String id, Long value) { this.id = id; this.value = value; - this.scheduledExecutorService = scheduledExecutorService; } @Override - public OutputRateLimiter clone(String key) { - FirstPerTimeOutputRateLimiter instance = new FirstPerTimeOutputRateLimiter(id + key, value, - scheduledExecutorService); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { - ArrayList> outputEventChunks = new ArrayList>(); + ComplexEventChunk outputEventChunk = new ComplexEventChunk<>(complexEventChunk.isBatch()); complexEventChunk.reset(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - if (event.getTimestamp() >= scheduledTime) { - if (firstEvent != null) { - firstEvent = null; - } - scheduledTime += value; - scheduler.notifyAt(scheduledTime); - } - } else if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type - .EXPIRED) { - if (firstEvent == null) { - complexEventChunk.remove(); - firstEvent = event; - ComplexEventChunk firstPerEventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); - firstPerEventChunk.add(event); - outputEventChunks.add(firstPerEventChunk); - } + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = siddhiQueryContext.getSiddhiAppContext(). + getTimestampGenerator().currentTime(); + if (state.outputTime == null || state.outputTime + value <= currentTime) { + state.outputTime = currentTime; + ComplexEvent event = complexEventChunk.next(); + complexEventChunk.remove(); + outputEventChunk.add(event); } } + } finally { + stateHolder.returnState(state); } - for (ComplexEventChunk eventChunk : outputEventChunks) { - sendToCallBacks(eventChunk); + if (outputEventChunk.getFirst() != null) { + sendToCallBacks(outputEventChunk); } - } @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); - } + public void partitionCreated() { - @Override - public void stop() { - //Nothing to stop } - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("FirstEvent", firstEvent); + class RateLimiterState extends State { + + private Long outputTime; + + @Override + public boolean canDestroy() { + return outputTime == null; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - firstEvent = (ComplexEvent) state.get("FirstEvent"); + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("OutputTime", outputTime); + return state; + } + + @Override + public void restore(Map state) { + outputTime = (Long) state.get("OutputTime"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/LastGroupByPerTimeOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/LastGroupByPerTimeOutputRateLimiter.java index 7df46bac41..81b4966d00 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/LastGroupByPerTimeOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/LastGroupByPerTimeOutputRateLimiter.java @@ -22,75 +22,78 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.GroupedComplexEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; import io.siddhi.core.util.Schedulable; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import org.apache.log4j.Logger; import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link OutputRateLimiter} which will collect pre-defined time period and the emit only last * event. This implementation specifically represent GroupBy queries. */ -public class LastGroupByPerTimeOutputRateLimiter extends OutputRateLimiter implements Schedulable { +public class LastGroupByPerTimeOutputRateLimiter + extends OutputRateLimiter implements Schedulable { private static final Logger log = Logger.getLogger(LastGroupByPerTimeOutputRateLimiter.class); private final Long value; private String id; - private Map allGroupByKeyEvents = new LinkedHashMap(); - private ScheduledExecutorService scheduledExecutorService; private Scheduler scheduler; - private long scheduledTime; - public LastGroupByPerTimeOutputRateLimiter(String id, Long value, ScheduledExecutorService - scheduledExecutorService) { + public LastGroupByPerTimeOutputRateLimiter(String id, Long value) { this.id = id; this.value = value; - this.scheduledExecutorService = scheduledExecutorService; } @Override - public OutputRateLimiter clone(String key) { - LastGroupByPerTimeOutputRateLimiter instance = new LastGroupByPerTimeOutputRateLimiter(id + key, value, - scheduledExecutorService); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { - ArrayList> outputEventChunks = new ArrayList>(); + ArrayList> outputEventChunks = new ArrayList<>(); complexEventChunk.reset(); - synchronized (this) { - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - if (event.getTimestamp() >= scheduledTime) { - if (allGroupByKeyEvents.size() != 0) { - ComplexEventChunk outputEventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); - for (ComplexEvent complexEvent : allGroupByKeyEvents.values()) { - outputEventChunk.add(complexEvent); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.TIMER) { + if (event.getTimestamp() >= state.scheduledTime) { + if (state.allGroupByKeyEvents.size() != 0) { + ComplexEventChunk outputEventChunk = new ComplexEventChunk + (complexEventChunk.isBatch()); + for (ComplexEvent complexEvent : state.allGroupByKeyEvents.values()) { + outputEventChunk.add(complexEvent); + } + outputEventChunks.add(outputEventChunk); + state.allGroupByKeyEvents.clear(); } - outputEventChunks.add(outputEventChunk); - allGroupByKeyEvents.clear(); + state.scheduledTime = state.scheduledTime + value; + scheduler.notifyAt(state.scheduledTime); } - scheduledTime = scheduledTime + value; - scheduler.notifyAt(scheduledTime); + } else if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type + .EXPIRED) { + complexEventChunk.remove(); + GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); + state.allGroupByKeyEvents.put(groupedComplexEvent.getGroupKey(), + groupedComplexEvent.getComplexEvent()); } - } else if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type - .EXPIRED) { - complexEventChunk.remove(); - GroupedComplexEvent groupedComplexEvent = ((GroupedComplexEvent) event); - allGroupByKeyEvents.put(groupedComplexEvent.getGroupKey(), groupedComplexEvent.getComplexEvent()); } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); @@ -99,32 +102,42 @@ public void process(ComplexEventChunk complexEventChunk) { } @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); + public void partitionCreated() { + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = System.currentTimeMillis(); + state.scheduledTime = currentTime + value; + scheduler.notifyAt(state.scheduledTime); + } + } finally { + stateHolder.returnState(state); + } } - @Override - public void stop() { - //Nothing to stop - } + class RateLimiterState extends State { - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + public long scheduledTime; + private Map allGroupByKeyEvents = new LinkedHashMap(); + + @Override + public boolean canDestroy() { + return allGroupByKeyEvents.size() == 0 && scheduledTime == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("AllGroupByKeyEvents", allGroupByKeyEvents); + state.put("ScheduledTime", scheduledTime); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - allGroupByKeyEvents = (Map) state.get("AllGroupByKeyEvents"); + @Override + public void restore(Map state) { + allGroupByKeyEvents = (Map) state.get("AllGroupByKeyEvents"); + scheduledTime = (Long) state.get("ScheduledTime"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/LastPerTimeOutputRateLimiter.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/LastPerTimeOutputRateLimiter.java index 8e2bdea371..6fde71057e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/LastPerTimeOutputRateLimiter.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/output/ratelimit/time/LastPerTimeOutputRateLimiter.java @@ -20,71 +20,74 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; import io.siddhi.core.util.Schedulable; import io.siddhi.core.util.Scheduler; import io.siddhi.core.util.parser.SchedulerParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import org.apache.log4j.Logger; import java.util.ArrayList; import java.util.HashMap; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** * Implementation of {@link OutputRateLimiter} which will collect pre-defined time period and the emit only last * event. */ -public class LastPerTimeOutputRateLimiter extends OutputRateLimiter implements Schedulable { +public class LastPerTimeOutputRateLimiter extends OutputRateLimiter + implements Schedulable { private static final Logger log = Logger.getLogger(LastPerTimeOutputRateLimiter.class); private final Long value; private String id; - private ComplexEvent lastEvent = null; - private ScheduledExecutorService scheduledExecutorService; private Scheduler scheduler; - private long scheduledTime; - public LastPerTimeOutputRateLimiter(String id, Long value, ScheduledExecutorService scheduledExecutorService) { + public LastPerTimeOutputRateLimiter(String id, Long value) { this.id = id; this.value = value; - this.scheduledExecutorService = scheduledExecutorService; } @Override - public OutputRateLimiter clone(String key) { - LastPerTimeOutputRateLimiter instance = new LastPerTimeOutputRateLimiter(id + key, value, - scheduledExecutorService); - instance.setLatencyTracker(latencyTracker); - return instance; + protected StateFactory init() { + this.scheduler = SchedulerParser.parse(this, siddhiQueryContext); + this.scheduler.setStreamEventFactory(new StreamEventFactory(0, 0, 0)); + this.scheduler.init(lockWrapper, siddhiQueryContext.getName()); + return () -> new RateLimiterState(); } @Override public void process(ComplexEventChunk complexEventChunk) { ArrayList> outputEventChunks = new ArrayList>(); complexEventChunk.reset(); - synchronized (this) { - complexEventChunk.reset(); - while (complexEventChunk.hasNext()) { - ComplexEvent event = complexEventChunk.next(); - if (event.getType() == ComplexEvent.Type.TIMER) { - if (event.getTimestamp() >= scheduledTime) { - if (lastEvent != null) { - ComplexEventChunk outputEventChunk = new ComplexEventChunk - (complexEventChunk.isBatch()); - outputEventChunk.add(lastEvent); - lastEvent = null; - outputEventChunks.add(outputEventChunk); + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + complexEventChunk.reset(); + while (complexEventChunk.hasNext()) { + ComplexEvent event = complexEventChunk.next(); + if (event.getType() == ComplexEvent.Type.TIMER) { + if (event.getTimestamp() >= state.scheduledTime) { + if (state.lastEvent != null) { + ComplexEventChunk outputEventChunk = new ComplexEventChunk + (complexEventChunk.isBatch()); + outputEventChunk.add(state.lastEvent); + state.lastEvent = null; + outputEventChunks.add(outputEventChunk); + } + state.scheduledTime = state.scheduledTime + value; + scheduler.notifyAt(state.scheduledTime); } - scheduledTime = scheduledTime + value; - scheduler.notifyAt(scheduledTime); + } else if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type + .EXPIRED) { + complexEventChunk.remove(); + state.lastEvent = event; } - } else if (event.getType() == ComplexEvent.Type.CURRENT || event.getType() == ComplexEvent.Type - .EXPIRED) { - complexEventChunk.remove(); - lastEvent = event; } } + } finally { + stateHolder.returnState(state); } for (ComplexEventChunk eventChunk : outputEventChunks) { sendToCallBacks(eventChunk); @@ -92,32 +95,42 @@ public void process(ComplexEventChunk complexEventChunk) { } @Override - public void start() { - scheduler = SchedulerParser.parse(this, siddhiQueryContext.getSiddhiAppContext()); - scheduler.setStreamEventPool(new StreamEventPool(0, 0, 0, 5)); - scheduler.init(lockWrapper, siddhiQueryContext.getName()); - long currentTime = System.currentTimeMillis(); - scheduledTime = currentTime + value; - scheduler.notifyAt(scheduledTime); + public void partitionCreated() { + RateLimiterState state = stateHolder.getState(); + try { + synchronized (state) { + long currentTime = System.currentTimeMillis(); + state.scheduledTime = currentTime + value; + scheduler.notifyAt(state.scheduledTime); + } + } finally { + stateHolder.returnState(state); + } } - @Override - public void stop() { - //Nothing to stop - } + class RateLimiterState extends State { - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + private ComplexEvent lastEvent = null; + private long scheduledTime; + + @Override + public boolean canDestroy() { + return lastEvent == null && scheduledTime == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("LastEvent", lastEvent); + state.put("ScheduledTime", scheduledTime); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - lastEvent = (ComplexEvent) state.get("LastEvent"); + @Override + public void restore(Map state) { + lastEvent = (ComplexEvent) state.get("LastEvent"); + scheduledTime = (Long) state.get("ScheduledTime"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/Processor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/Processor.java index 3aee21996e..fc03b84d4f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/Processor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/Processor.java @@ -55,17 +55,4 @@ public interface Processor { */ void setToLast(Processor processor); - /** - * Clone a copy of processor - * - * @param key partition key - * @return cloned processor - */ - Processor cloneProcessor(String key); - - /** - * Used to clean its references - */ - void clean(); - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/filter/FilterProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/filter/FilterProcessor.java index 21d486e844..399ec4cf2a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/filter/FilterProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/filter/FilterProcessor.java @@ -42,15 +42,6 @@ public FilterProcessor(ExpressionExecutor conditionExecutor) { } } - public FilterProcessor cloneProcessor(String key) { - return new FilterProcessor(conditionExecutor.cloneExecutor(key)); - } - - @Override - public void clean() { - conditionExecutor.clean(); - } - @Override public void process(ComplexEventChunk complexEventChunk) { complexEventChunk.reset(); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/AbstractStreamProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/AbstractStreamProcessor.java index 8b8f8d39ba..f77df02d0b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/AbstractStreamProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/AbstractStreamProcessor.java @@ -26,13 +26,14 @@ import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.event.stream.populater.StreamEventPopulaterFactory; import io.siddhi.core.exception.SiddhiAppCreationException; -import io.siddhi.core.exception.SiddhiAppRuntimeException; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; import io.siddhi.core.util.config.ConfigReader; -import io.siddhi.core.util.extension.holder.EternalReferencedHolder; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.util.extension.holder.ExternalReferencedHolder; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.query.api.SiddhiElement; import io.siddhi.query.api.definition.AbstractDefinition; import io.siddhi.query.api.definition.Attribute; @@ -43,50 +44,43 @@ /** * Abstract implementation of {@link Processor} intended to be used by any Stream Processors. + * + * @param current state of the processor */ -public abstract class AbstractStreamProcessor implements Processor, EternalReferencedHolder, Snapshotable { +public abstract class AbstractStreamProcessor implements Processor, ExternalReferencedHolder { private static final Logger log = Logger.getLogger(AbstractStreamProcessor.class); protected Processor nextProcessor; - - protected List additionalAttributes; + private List additionalAttributes; protected MetaStreamEvent metaStreamEvent; protected SiddhiQueryContext siddhiQueryContext; protected StreamEventClonerHolder streamEventClonerHolder = new StreamEventClonerHolder(); - protected StreamEventCloner streamEventCloner; protected AbstractDefinition inputDefinition; protected ExpressionExecutor[] attributeExpressionExecutors; protected int attributeExpressionLength; protected ComplexEventPopulater complexEventPopulater; - protected String elementId = null; - private ConfigReader configReader; - private boolean outputExpectsExpiredEvents; + protected StateHolder stateHolder; public void initProcessor(MetaStreamEvent metaStreamEvent, ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, boolean outputExpectsExpiredEvents, - SiddhiElement siddhiElement, SiddhiQueryContext siddhiQueryContext) { - this.configReader = configReader; - this.outputExpectsExpiredEvents = outputExpectsExpiredEvents; + boolean findToBeExecuted, + boolean groupBy, SiddhiElement siddhiElement, + SiddhiQueryContext siddhiQueryContext) { this.metaStreamEvent = metaStreamEvent; this.siddhiQueryContext = siddhiQueryContext; try { this.inputDefinition = metaStreamEvent.getLastInputDefinition(); this.attributeExpressionExecutors = attributeExpressionExecutors; this.attributeExpressionLength = attributeExpressionExecutors.length; - if (elementId == null) { - elementId = "AbstractStreamProcessor-" + - siddhiQueryContext.getSiddhiAppContext().getElementIdGenerator().createNewId(); - } - siddhiQueryContext.getSiddhiAppContext().getSnapshotService().addSnapshotable( - siddhiQueryContext.getName(), this); - this.additionalAttributes = init(metaStreamEvent, metaStreamEvent.getLastInputDefinition(), - attributeExpressionExecutors, configReader, outputExpectsExpiredEvents, siddhiQueryContext); - + StateFactory stateFactory = init(metaStreamEvent, metaStreamEvent.getLastInputDefinition(), + attributeExpressionExecutors, configReader, streamEventClonerHolder, outputExpectsExpiredEvents, + findToBeExecuted, siddhiQueryContext); + this.additionalAttributes = getReturnAttributes(); + this.stateHolder = siddhiQueryContext.generateStateHolder(this.getClass().getName(), groupBy, stateFactory); siddhiQueryContext.getSiddhiAppContext().addEternalReferencedHolder(this); - if (additionalAttributes.size() > 0) { StreamDefinition outputDefinition = StreamDefinition.id(inputDefinition.getId()); outputDefinition.setQueryContextStartIndex(siddhiElement.getQueryContextStartIndex()); @@ -114,20 +108,31 @@ public void initProcessor(MetaStreamEvent metaStreamEvent, * @param inputDefinition the incoming stream definition * @param attributeExpressionExecutors the executors of each function parameters * @param configReader this hold the {@link AbstractStreamProcessor} extensions configuration - * reader. + * @param streamEventClonerHolder stream event cloner holder * @param outputExpectsExpiredEvents is expired events sent as output + * @param findToBeExecuted find will be executed * @param siddhiQueryContext current siddhi query context - * @return the additional output attributes introduced by the function */ - protected abstract List init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, + protected abstract StateFactory init(MetaStreamEvent metaStreamEvent, + AbstractDefinition inputDefinition, ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, + StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, SiddhiQueryContext siddhiQueryContext); + public abstract List getReturnAttributes(); + public void process(ComplexEventChunk streamEventChunk) { streamEventChunk.reset(); - processEventChunk(streamEventChunk, nextProcessor, streamEventCloner, complexEventPopulater); + S state = stateHolder.getState(); + try { + processEventChunk(streamEventChunk, nextProcessor, streamEventClonerHolder.getStreamEventCloner(), + complexEventPopulater, state); + } finally { + stateHolder.returnState(state); + } + } /** @@ -137,10 +142,13 @@ public void process(ComplexEventChunk streamEventChunk) { * @param nextProcessor the next processor to which the success events need to be passed * @param streamEventCloner helps to clone the incoming event for local storage or modification * @param complexEventPopulater helps to populate the events with the resultant attributes + * @param state current state of the processor */ - protected abstract void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, + protected abstract void processEventChunk(ComplexEventChunk streamEventChunk, + Processor nextProcessor, StreamEventCloner streamEventCloner, - ComplexEventPopulater complexEventPopulater); + ComplexEventPopulater complexEventPopulater, + S state); public Processor getNextProcessor() { return nextProcessor; @@ -150,40 +158,6 @@ public void setNextProcessor(Processor processor) { this.nextProcessor = processor; } - @Override - public Processor cloneProcessor(String key) { - try { - AbstractStreamProcessor abstractStreamProcessor = this.getClass().newInstance(); - abstractStreamProcessor.inputDefinition = inputDefinition; - ExpressionExecutor[] innerExpressionExecutors = new ExpressionExecutor[attributeExpressionLength]; - ExpressionExecutor[] attributeExpressionExecutors1 = this.attributeExpressionExecutors; - for (int i = 0; i < attributeExpressionLength; i++) { - innerExpressionExecutors[i] = attributeExpressionExecutors1[i].cloneExecutor(key); - } - abstractStreamProcessor.attributeExpressionExecutors = innerExpressionExecutors; - abstractStreamProcessor.attributeExpressionLength = attributeExpressionLength; - abstractStreamProcessor.additionalAttributes = additionalAttributes; - abstractStreamProcessor.additionalAttributes = additionalAttributes; - abstractStreamProcessor.complexEventPopulater = complexEventPopulater; - abstractStreamProcessor.siddhiQueryContext = siddhiQueryContext; - abstractStreamProcessor.elementId = elementId + "-" + key; - abstractStreamProcessor.configReader = configReader; - abstractStreamProcessor.outputExpectsExpiredEvents = outputExpectsExpiredEvents; - abstractStreamProcessor.siddhiQueryContext.getSiddhiAppContext().getSnapshotService() - .addSnapshotable(siddhiQueryContext.getName(), abstractStreamProcessor); - abstractStreamProcessor.siddhiQueryContext.getSiddhiAppContext().addEternalReferencedHolder( - abstractStreamProcessor); - - abstractStreamProcessor.init(metaStreamEvent, inputDefinition, attributeExpressionExecutors, configReader, - outputExpectsExpiredEvents, siddhiQueryContext); - abstractStreamProcessor.start(); - return abstractStreamProcessor; - - } catch (Exception e) { - throw new SiddhiAppRuntimeException("Exception in cloning " + this.getClass().getCanonicalName(), e); - } - } - public void constructStreamEventPopulater(MetaStreamEvent metaStreamEvent, int streamEventChainIndex) { if (this.complexEventPopulater == null) { this.complexEventPopulater = StreamEventPopulaterFactory.constructEventPopulator(metaStreamEvent, @@ -193,7 +167,6 @@ public void constructStreamEventPopulater(MetaStreamEvent metaStreamEvent, int s } public void setStreamEventCloner(StreamEventCloner streamEventCloner) { - this.streamEventCloner = streamEventCloner; this.streamEventClonerHolder.setStreamEventCloner(streamEventCloner); } @@ -205,20 +178,6 @@ public void setToLast(Processor processor) { } } - @Override - public String getElementId() { - return elementId; - } - - @Override - public void clean() { - for (ExpressionExecutor expressionExecutor : attributeExpressionExecutors) { - expressionExecutor.clean(); - } - siddhiQueryContext.getSiddhiAppContext().getSnapshotService().removeSnapshotable( - siddhiQueryContext.getName(), this); - } - /** * Defines the behaviour of the processing, will be called after the init * diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/LogStreamProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/LogStreamProcessor.java index e6dc5aa48d..c556160201 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/LogStreamProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/LogStreamProcessor.java @@ -28,12 +28,15 @@ 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.StreamEventClonerHolder; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; 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.AbstractDefinition; import io.siddhi.query.api.definition.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; @@ -41,7 +44,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Map; /** * Input attributes to log is (priority (String), log.message (String), is.event.logged (Bool)) @@ -97,7 +99,7 @@ ) } ) -public class LogStreamProcessor extends StreamProcessor { +public class LogStreamProcessor extends StreamProcessor { private static final Logger log = Logger.getLogger(LogStreamProcessor.class); private ExpressionExecutor isLogEventExpressionExecutor = null; private ExpressionExecutor logMessageExpressionExecutor = null; @@ -112,13 +114,17 @@ public class LogStreamProcessor extends StreamProcessor { * @param inputDefinition the incoming stream definition * @param attributeExpressionExecutors the executors for the function parameters * @param configReader this hold the {@link LogStreamProcessor} configuration reader. + * @param streamEventClonerHolder streamEventCloner Holder + * @param findToBeExecuted find will be executed * @param siddhiQueryContext current siddhi query context * @return the additional output attributes introduced by the function */ @Override - protected List init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, - ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, + ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { int inputExecutorLength = attributeExpressionExecutors.length; if (inputExecutorLength == 1) { if (attributeExpressionExecutors[0].getReturnType() == Attribute.Type.STRING) { @@ -183,12 +189,18 @@ protected List init(MetaStreamEvent metaStreamEvent, AbstractDefiniti "isEventLogged (Bool), but there are " + attributeExpressionExecutors.length + " in the input!"); } logPrefix = siddhiQueryContext.getSiddhiAppContext().getName() + ": "; + return null; + } + + @Override + public List getReturnAttributes() { return new ArrayList(); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater) { + StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, + State state) { while (streamEventChunk.hasNext()) { ComplexEvent complexEvent = streamEventChunk.next(); switch (attributeExpressionLength) { @@ -279,17 +291,6 @@ public void stop() { //Do nothing } - @Override - public Map currentState() { - //No state - return null; - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } - @Override public ProcessingMode getProcessingMode() { return ProcessingMode.BATCH; diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/StreamProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/StreamProcessor.java index 921e90a042..7de60697fd 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/StreamProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/StreamProcessor.java @@ -22,19 +22,23 @@ import io.siddhi.core.event.stream.StreamEventCloner; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.query.processor.Processor; +import io.siddhi.core.util.snapshot.state.State; /** * For Siddhi extensions, extend this class to use the functionality of * AbstractStreamProcessor. This class processes only StreamEvents. Use * StreamFunctionProcessor to process StateEvents. + * + * @param current state of the processor */ -public abstract class StreamProcessor extends AbstractStreamProcessor { +public abstract class StreamProcessor extends AbstractStreamProcessor { @Override protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater) { + StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, + S state) { streamEventChunk.reset(); - process(streamEventChunk, nextProcessor, streamEventCloner, complexEventPopulater); + process(streamEventChunk, nextProcessor, streamEventCloner, complexEventPopulater, state); } @@ -45,8 +49,10 @@ protected void processEventChunk(ComplexEventChunk streamEventChunk * @param nextProcessor the next processor to which the success events need to be passed * @param streamEventCloner helps to clone the incoming event for local storage or modification * @param complexEventPopulater helps to populate the events with the resultant attributes + * @param state current processor state */ protected abstract void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater); + StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, + S state); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/function/Pol2CartStreamFunctionProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/function/Pol2CartStreamFunctionProcessor.java index ac2e0e7fdb..d2b7fe55fb 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/function/Pol2CartStreamFunctionProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/function/Pol2CartStreamFunctionProcessor.java @@ -25,13 +25,13 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.util.config.ConfigReader; +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.exception.SiddhiAppValidationException; import java.util.Arrays; import java.util.List; -import java.util.Map; /** * Created on 1/26/15. @@ -73,6 +73,7 @@ public class Pol2CartStreamFunctionProcessor extends StreamFunctionProcessor { private int inputExecutorLength; + private List retrunAttributes; /** * The init method of the StreamFunction @@ -83,10 +84,10 @@ public class Pol2CartStreamFunctionProcessor extends StreamFunctionProcessor { * @return the additional output attributes introduced by the function */ @Override - protected List init(AbstractDefinition inputDefinition, - ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, boolean outputExpectsExpiredEvents, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(AbstractDefinition inputDefinition, + ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, boolean outputExpectsExpiredEvents, + SiddhiQueryContext siddhiQueryContext) { inputExecutorLength = attributeExpressionExecutors.length; if (inputExecutorLength < 2 || inputExecutorLength > 3) { @@ -109,7 +110,9 @@ protected List init(AbstractDefinition inputDefinition, "not contain attributes with name 'x' or 'y', but found " + attribute); } } - return Arrays.asList(new Attribute("x", Attribute.Type.DOUBLE), new Attribute("y", Attribute.Type.DOUBLE)); + retrunAttributes = Arrays.asList( + new Attribute("x", Attribute.Type.DOUBLE), + new Attribute("y", Attribute.Type.DOUBLE)); } else { for (Attribute attribute : inputDefinition.getAttributeList()) { if (attribute.getName().equals("x") || attribute.getName().equals("y") || attribute.getName().equals @@ -118,9 +121,12 @@ protected List init(AbstractDefinition inputDefinition, "not contain attributes with name 'x' or 'y' or 'z', but found " + attribute); } } - return Arrays.asList(new Attribute("x", Attribute.Type.DOUBLE), new Attribute("y", Attribute.Type.DOUBLE) - , new Attribute("z", Attribute.Type.DOUBLE)); + retrunAttributes = Arrays.asList( + new Attribute("x", Attribute.Type.DOUBLE), + new Attribute("y", Attribute.Type.DOUBLE), + new Attribute("z", Attribute.Type.DOUBLE)); } + return null; } /** @@ -164,15 +170,8 @@ public void stop() { //Do nothing } - - @Override - public Map currentState() { - //No state - return null; - } - @Override - public void restoreState(Map state) { - //Nothing to be done + public List getReturnAttributes() { + return retrunAttributes; } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/function/StreamFunctionProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/function/StreamFunctionProcessor.java index 721f023b00..623f75eca8 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/function/StreamFunctionProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/function/StreamFunctionProcessor.java @@ -23,25 +23,28 @@ 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.StreamEventClonerHolder; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; import io.siddhi.core.query.processor.stream.AbstractStreamProcessor; 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.AbstractDefinition; -import io.siddhi.query.api.definition.Attribute; - -import java.util.List; /** * Stream Processor to handle Stream Functions. + * + * @param current state of the processor */ -public abstract class StreamFunctionProcessor extends AbstractStreamProcessor { +public abstract class StreamFunctionProcessor extends AbstractStreamProcessor { @Override protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater) { + StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, + S state) { while (streamEventChunk.hasNext()) { ComplexEvent complexEvent = streamEventChunk.next(); Object[] outputData; @@ -96,14 +99,16 @@ public ProcessingMode getProcessingMode() { * @param inputDefinition the incoming stream definition * @param attributeExpressionExecutors the executors of each function parameters * @param configReader this hold the {@link StreamFunctionProcessor} extensions configuration - * reader. + * @param streamEventClonerHolder stream event cloner Holder * @param outputExpectsExpiredEvents is expired events sent as output + * @param findToBeExecuted find operation will be performed * @param siddhiQueryContext siddhi query context - * @return the additional output attributes introduced by the function */ - protected List init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, + protected StateFactory init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { return init(inputDefinition, attributeExpressionExecutors, configReader, outputExpectsExpiredEvents, siddhiQueryContext); } @@ -119,7 +124,7 @@ protected List init(MetaStreamEvent metaStreamEvent, AbstractDefiniti * @param siddhiQueryContext the context of the siddhi query * @return the additional output attributes introduced by the function */ - protected abstract List init(AbstractDefinition inputDefinition, + protected abstract StateFactory init(AbstractDefinition inputDefinition, ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/AggregateWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/AggregateWindowProcessor.java index 9ef758709e..925269205e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/AggregateWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/AggregateWindowProcessor.java @@ -22,7 +22,7 @@ import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.exception.SiddhiAppRuntimeException; +import io.siddhi.core.event.stream.holder.StreamEventClonerHolder; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; import io.siddhi.core.query.input.stream.join.JoinProcessor; @@ -31,6 +31,8 @@ import io.siddhi.core.util.collection.operator.CompiledCondition; import io.siddhi.core.util.collection.operator.MatchingMetaInfoHolder; 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.aggregation.Within; import io.siddhi.query.api.expression.Expression; @@ -48,9 +50,6 @@ public class AggregateWindowProcessor extends BatchingWindowProcessor implements private final Within within; private final Expression per; private AggregationRuntime aggregationRuntime; - private ConfigReader configReader; - private boolean outputExpectsExpiredEvents; - private SiddhiQueryContext siddhiQueryContext; public AggregateWindowProcessor(AggregationRuntime aggregationRuntime, Within within, Expression per) { this.aggregationRuntime = aggregationRuntime; @@ -59,24 +58,25 @@ public AggregateWindowProcessor(AggregationRuntime aggregationRuntime, Within wi } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, boolean outputExpectsExpiredEvents, + boolean findToBeExecuted, SiddhiQueryContext siddhiQueryContext) { // nothing to be done - this.configReader = configReader; - this.outputExpectsExpiredEvents = outputExpectsExpiredEvents; - this.siddhiQueryContext = siddhiQueryContext; + return null; } @Override - protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { + protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, + StreamEventCloner streamEventCloner, State state) { // Pass the event to the post JoinProcessor nextProcessor.process(streamEventChunk); } + @Override public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return aggregationRuntime.find(matchingEvent, compiledCondition); + return aggregationRuntime.find(matchingEvent, compiledCondition, siddhiQueryContext); + } @Override @@ -96,39 +96,4 @@ public void start() { public void stop() { //Do nothing } - - @Override - public Processor cloneProcessor(String key) { - try { - AggregateWindowProcessor streamProcessor = new AggregateWindowProcessor(aggregationRuntime, within, per); - streamProcessor.inputDefinition = inputDefinition; - ExpressionExecutor[] innerExpressionExecutors = new ExpressionExecutor[attributeExpressionLength]; - ExpressionExecutor[] attributeExpressionExecutors1 = this.attributeExpressionExecutors; - for (int i = 0; i < attributeExpressionLength; i++) { - innerExpressionExecutors[i] = attributeExpressionExecutors1[i].cloneExecutor(key); - } - streamProcessor.attributeExpressionExecutors = innerExpressionExecutors; - streamProcessor.attributeExpressionLength = attributeExpressionLength; - streamProcessor.additionalAttributes = additionalAttributes; - streamProcessor.complexEventPopulater = complexEventPopulater; - streamProcessor.init(metaStreamEvent, inputDefinition, attributeExpressionExecutors, configReader, - outputExpectsExpiredEvents, siddhiQueryContext); - streamProcessor.start(); - return streamProcessor; - - } catch (Exception e) { - throw new SiddhiAppRuntimeException("Exception in cloning " + this.getClass().getCanonicalName(), e); - } - } - - @Override - public Map currentState() { - //No state - return null; - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchWindowProcessor.java index 6f26822d35..88d4b1ff1a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchWindowProcessor.java @@ -28,6 +28,7 @@ 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.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -39,6 +40,8 @@ import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; 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.exception.SiddhiAppValidationException; import io.siddhi.query.api.expression.Expression; @@ -80,25 +83,22 @@ ) } ) -public class BatchWindowProcessor extends BatchingWindowProcessor implements FindableProcessor { +public class BatchWindowProcessor extends BatchingFindableWindowProcessor { private int length = 0; - private int count = 0; - private SnapshotableStreamEventQueue expiredEventQueue = null; - private SnapshotableStreamEventQueue currentEventQueue; private boolean outputExpectsExpiredEvents; + private boolean findToBeExecuted; private SiddhiQueryContext siddhiQueryContext; - private StreamEvent resetEvent = null; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { this.outputExpectsExpiredEvents = outputExpectsExpiredEvents; + this.findToBeExecuted = findToBeExecuted; this.siddhiQueryContext = siddhiQueryContext; - currentEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); - if (outputExpectsExpiredEvents) { - expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); - } if (attributeExpressionExecutors.length == 1) { length = (Integer) (((ConstantExpressionExecutor) attributeExpressionExecutors[0]).getValue()); } else if (attributeExpressionExecutors.length == 0) { @@ -111,30 +111,32 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea throw new SiddhiAppValidationException("Batch window should have at most one parameter ( " + "chunkLength) greater than zero. But found value 'chunkLength = " + length + " ' "); } + return () -> new WindowState(streamEventClonerHolder, outputExpectsExpiredEvents, findToBeExecuted); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { + StreamEventCloner streamEventCloner, WindowState state) { List> streamEventChunks = new ArrayList>(); ComplexEventChunk currentEventChunk = new ComplexEventChunk(true); - synchronized (this) { + synchronized (state) { long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); if (outputExpectsExpiredEvents) { - if (expiredEventQueue.getFirst() != null) { - while (expiredEventQueue.hasNext()) { - expiredEventQueue.next().setTimestamp(currentTime); + if (state.expiredEventQueue.getFirst() != null) { + while (state.expiredEventQueue.hasNext()) { + state.expiredEventQueue.next().setTimestamp(currentTime); } - currentEventChunk.add(expiredEventQueue.getFirst()); - if (resetEvent != null) { - currentEventChunk.add(resetEvent); - resetEvent = null; + currentEventChunk.add(state.expiredEventQueue.getFirst()); + if (state.resetEvent != null) { + currentEventChunk.add(state.resetEvent); + state.resetEvent = null; } } - expiredEventQueue.clear(); + state.expiredEventQueue.clear(); } - //check whether the streamEventChunk has next event before add into output stream event chunk + ComplexEventChunk currentEventQueue = new ComplexEventChunk<>(true); + int count = 0; if (streamEventChunk.hasNext()) { do { StreamEvent streamEvent = streamEventChunk.next(); @@ -142,17 +144,17 @@ protected void process(ComplexEventChunk streamEventChunk, Processo if (outputExpectsExpiredEvents) { StreamEvent clonedStreamEventToExpire = streamEventCloner.copyStreamEvent(streamEvent); clonedStreamEventToExpire.setType(StreamEvent.Type.EXPIRED); - expiredEventQueue.add(clonedStreamEventToExpire); + state.expiredEventQueue.add(clonedStreamEventToExpire); } currentEventQueue.add(clonedStreamEventToProcess); count++; if (count == length) { if (currentEventQueue.getFirst() != null) { - if (resetEvent != null) { - currentEventChunk.add(resetEvent); + if (state.resetEvent != null) { + currentEventChunk.add(state.resetEvent); } - resetEvent = streamEventCloner.copyStreamEvent(currentEventQueue.getFirst()); - resetEvent.setType(ComplexEvent.Type.RESET); + state.resetEvent = streamEventCloner.copyStreamEvent(currentEventQueue.getFirst()); + state.resetEvent.setType(ComplexEvent.Type.RESET); currentEventChunk.add(currentEventQueue.getFirst()); } count = 0; @@ -164,15 +166,13 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } } while (streamEventChunk.hasNext()); if (currentEventQueue.getFirst() != null) { - if (resetEvent != null) { - currentEventChunk.add(resetEvent); + if (state.resetEvent != null) { + currentEventChunk.add(state.resetEvent); } - resetEvent = streamEventCloner.copyStreamEvent(currentEventQueue.getFirst()); - resetEvent.setType(ComplexEvent.Type.RESET); + state.resetEvent = streamEventCloner.copyStreamEvent(currentEventQueue.getFirst()); + state.resetEvent.setType(ComplexEvent.Type.RESET); currentEventChunk.add(currentEventQueue.getFirst()); } - count = 0; - currentEventQueue.clear(); if (currentEventChunk.getFirst() != null) { streamEventChunks.add(currentEventChunk); } @@ -194,40 +194,53 @@ public void stop() { } @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - if (outputExpectsExpiredEvents) { + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.expiredEventQueue, streamEventCloner); + } + + @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); + } + + class WindowState extends State { + private SnapshotableStreamEventQueue expiredEventQueue; + private StreamEvent resetEvent = null; + + WindowState(StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted) { + if (outputExpectsExpiredEvents || findToBeExecuted) { + expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + } + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + + if (outputExpectsExpiredEvents || findToBeExecuted) { state.put("ExpiredEventQueue", expiredEventQueue.getSnapshot()); } state.put("ResetEvent", resetEvent); + return state; } - return state; - } - @Override - public synchronized void restoreState(Map state) { - if (outputExpectsExpiredEvents) { - expiredEventQueue.clear(); - expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + public void restore(Map state) { + if (outputExpectsExpiredEvents || findToBeExecuted) { + expiredEventQueue.clear(); + expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + } + resetEvent = (StreamEvent) state.get("ResetEvent"); } - resetEvent = (StreamEvent) state.get("ResetEvent"); - } - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, expiredEventQueue, streamEventCloner); - } - - @Override - public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, - List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - if (expiredEventQueue == null) { - expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + @Override + public boolean canDestroy() { + return expiredEventQueue.getFirst() == null && resetEvent == null; } - return OperatorParser.constructOperator(expiredEventQueue, condition, matchingMetaInfoHolder, - variableExpressionExecutors, tableMap, - siddhiQueryContext); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchingFindableWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchingFindableWindowProcessor.java new file mode 100644 index 0000000000..a6ee8b950c --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/BatchingFindableWindowProcessor.java @@ -0,0 +1,98 @@ +/* + * 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.core.config.SiddhiQueryContext; +import io.siddhi.core.event.state.StateEvent; +import io.siddhi.core.event.stream.StreamEvent; +import io.siddhi.core.event.stream.StreamEventCloner; +import io.siddhi.core.executor.VariableExpressionExecutor; +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.snapshot.state.State; +import io.siddhi.query.api.expression.Expression; + +import java.util.List; +import java.util.Map; + +/** + * Window to process events as batches while supporting event search + * + * @param current state of the processor + */ +public abstract class BatchingFindableWindowProcessor extends BatchingWindowProcessor + implements FindableProcessor { + + @Override + public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, + List variableExpressionExecutors, + Map tableMap, SiddhiQueryContext siddhiQueryContext) { + S state = stateHolder.getState(); + try { + return compileCondition(condition, matchingMetaInfoHolder, variableExpressionExecutors, tableMap, + state, siddhiQueryContext); + } finally { + stateHolder.returnState(state); + } + } + + @Override + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { + S state = stateHolder.getState(); + try { + return find(matchingEvent, compiledCondition, streamEventClonerHolder.getStreamEventCloner(), + state); + } finally { + stateHolder.returnState(state); + } + + } + + /** + * To construct a finder having the capability of finding events at the processor that corresponds to the incoming + * matchingEvent and the given matching expression logic. + * + * @param condition the matching condition + * @param matchingMetaInfoHolder the meta structure of the incoming matchingEvent + * @param variableExpressionExecutors the list of variable ExpressionExecutors already created + * @param tableMap map of event tables + * @param state current query state + * @param siddhiQueryContext current siddhi query context + * @return compiled Condition having the capability of matching events against the incoming matchingEvent + */ + public abstract CompiledCondition compileCondition(Expression condition, + MatchingMetaInfoHolder matchingMetaInfoHolder, + List variableExpressionExecutors, + Map tableMap, S state, + SiddhiQueryContext siddhiQueryContext); + + /** + * To find events from the processor event pool, that the matches the matchingEvent based on finder logic. + * + * @param matchingEvent the event to be matched with the events at the processor + * @param compiledCondition the execution element responsible for matching the corresponding events that matches + * the matchingEvent based on pool of events at Processor + * @param streamEventCloner stream event cloner + * @param state current query state @return the matched events + */ + public abstract StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, S state); + +} 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 d4f71acb7f..89056159f2 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 @@ -23,30 +23,31 @@ 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.StreamEventClonerHolder; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; 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.AbstractDefinition; -import io.siddhi.query.api.definition.Attribute; - -import java.util.ArrayList; -import java.util.List; /** * Performs event processing in a tumbling manner + * + * @param current state of the processor */ -public abstract class BatchingWindowProcessor extends WindowProcessor { +public abstract class BatchingWindowProcessor extends WindowProcessor { @Override - protected List init(MetaStreamEvent metaStreamEvent, - AbstractDefinition inputDefinition, - ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { - init(attributeExpressionExecutors, configReader, outputExpectsExpiredEvents, siddhiQueryContext); - return new ArrayList(0); + protected StateFactory init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, + ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { + return init(attributeExpressionExecutors, configReader, streamEventClonerHolder, + outputExpectsExpiredEvents, findToBeExecuted, siddhiQueryContext); } /** @@ -54,17 +55,23 @@ protected List init(MetaStreamEvent metaStreamEvent, * * @param attributeExpressionExecutors the executors of each function parameters * @param configReader the config reader of window + * @param streamEventClonerHolder stream event cloner holder * @param outputExpectsExpiredEvents is expired events sent as output + * @param findToBeExecuted find operation will be performed * @param siddhiQueryContext the context of the siddhi query */ - protected abstract void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext); + protected abstract StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, + boolean findToBeExecuted, SiddhiQueryContext siddhiQueryContext); @Override protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater) { + StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, + S state) { streamEventChunk.reset(); - process(streamEventChunk, nextProcessor, streamEventCloner); + process(streamEventChunk, nextProcessor, streamEventCloner, state); } /** @@ -73,9 +80,10 @@ protected void processEventChunk(ComplexEventChunk streamEventChunk * @param streamEventChunk the stream event chunk that need to be processed * @param nextProcessor the next processor to which the success events need to be passed * @param streamEventCloner helps to clone the incoming event for local storage or modification + * @param state current state of the processor */ protected abstract void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner); + StreamEventCloner streamEventCloner, S state); @Override public ProcessingMode getProcessingMode() { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/CronWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/CronWindowProcessor.java index e291ccaa22..b585baf898 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/CronWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/CronWindowProcessor.java @@ -21,15 +21,21 @@ import io.siddhi.annotation.Extension; import io.siddhi.annotation.Parameter; import io.siddhi.annotation.util.DataType; +import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEventChunk; 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.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.Processor; import io.siddhi.core.util.ExceptionUtil; import io.siddhi.core.util.config.ConfigReader; +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 org.apache.log4j.Logger; import org.quartz.CronScheduleBuilder; import org.quartz.Job; @@ -87,42 +93,49 @@ ) } ) -public class CronWindowProcessor extends BatchingWindowProcessor implements Job { +//todo fix support find and optimize data storage +public class CronWindowProcessor extends BatchingWindowProcessor implements Job { private static final Logger log = Logger.getLogger(CronWindowProcessor.class); private final String jobGroup = "CronWindowGroup"; - private ComplexEventChunk currentEventChunk = new ComplexEventChunk(false); - private ComplexEventChunk expiredEventChunk = new ComplexEventChunk(false); - private SiddhiQueryContext siddhiQueryContext; private Scheduler scheduler; private String jobName; private String cronString; - + private StreamEventClonerHolder streamEventClonerHolder; + private String id; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, boolean - outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { - this.siddhiQueryContext = siddhiQueryContext; + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, + boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { + this.streamEventClonerHolder = streamEventClonerHolder; + this.id = siddhiQueryContext.getName() + "_" + siddhiQueryContext.generateNewId(); if (attributeExpressionExecutors != null) { cronString = (String) (((ConstantExpressionExecutor) attributeExpressionExecutors[0]).getValue()); } + return () -> new WindowState(streamEventClonerHolder); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { - synchronized (this) { + StreamEventCloner streamEventCloner, WindowState state) { + synchronized (state) { while (streamEventChunk.hasNext()) { StreamEvent streamEvent = streamEventChunk.next(); StreamEvent clonedStreamEvent = streamEventCloner.copyStreamEvent(streamEvent); - currentEventChunk.add(clonedStreamEvent); + state.currentEventQueue.add(clonedStreamEvent); streamEventChunk.remove(); } } } @Override - public void start() { - scheduleCronJob(cronString, elementId); + public synchronized void start() { + if (scheduler == null) { + scheduleCronJob(cronString); + } } @Override @@ -137,28 +150,11 @@ public void stop() { } } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("CurrentEventChunk", currentEventChunk.getFirst()); - state.put("ExpiredEventChunk", expiredEventChunk.getFirst()); - return state; - } - - - @Override - public void restoreState(Map state) { - currentEventChunk.clear(); - currentEventChunk.add((StreamEvent) state.get("CurrentEventChunk")); - expiredEventChunk.clear(); - expiredEventChunk.add((StreamEvent) state.get("ExpiredEventChunk")); - } - - private void scheduleCronJob(String cronString, String elementId) { + private void scheduleCronJob(String cronString) { try { SchedulerFactory schedFact = new StdSchedulerFactory(); scheduler = schedFact.getScheduler(); - jobName = "EventRemoverJob_" + elementId; + jobName = siddhiQueryContext.getName() + "_EventRemoverJob_" + siddhiQueryContext.generateNewId(); JobKey jobKey = new JobKey(jobName, jobGroup); if (scheduler.checkExists(jobKey)) { @@ -174,7 +170,7 @@ private void scheduleCronJob(String cronString, String elementId) { .build(); Trigger trigger = org.quartz.TriggerBuilder.newTrigger() - .withIdentity("EventRemoverTrigger_" + elementId, jobGroup) + .withIdentity("EventRemoverTrigger_" + id, jobGroup) .withSchedule(CronScheduleBuilder.cronSchedule(cronString)) .build(); @@ -186,33 +182,50 @@ private void scheduleCronJob(String cronString, String elementId) { } public void dispatchEvents() { + Map> allStates = stateHolder.getAllStates(); + try { + for (Map.Entry> allStatesEntry : allStates.entrySet()) { + for (Map.Entry stateEntry : allStatesEntry.getValue().entrySet()) { + WindowState windowState = stateEntry.getValue(); + ComplexEventChunk streamEventChunk = new ComplexEventChunk(false); + synchronized (windowState) { + if (windowState.currentEventQueue.getFirst() != null) { + long currentTime = siddhiQueryContext.getSiddhiAppContext(). + getTimestampGenerator().currentTime(); + while (windowState.expiredEventQueue.hasNext()) { + StreamEvent expiredEvent = windowState.expiredEventQueue.next(); + expiredEvent.setTimestamp(currentTime); + } + if (windowState.expiredEventQueue.getFirst() != null) { + streamEventChunk.add(windowState.expiredEventQueue.getFirst()); + } + windowState.expiredEventQueue.clear(); + while (windowState.currentEventQueue.hasNext()) { + StreamEvent currentEvent = windowState.currentEventQueue.next(); + StreamEvent toExpireEvent = + streamEventClonerHolder.getStreamEventCloner().copyStreamEvent(currentEvent); + toExpireEvent.setType(StreamEvent.Type.EXPIRED); + windowState.expiredEventQueue.add(toExpireEvent); + } - ComplexEventChunk streamEventChunk = new ComplexEventChunk(false); - synchronized (this) { - if (currentEventChunk.getFirst() != null) { - long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); - while (expiredEventChunk.hasNext()) { - StreamEvent expiredEvent = expiredEventChunk.next(); - expiredEvent.setTimestamp(currentTime); + streamEventChunk.add(windowState.currentEventQueue.getFirst()); + windowState.currentEventQueue.clear(); + } + } + SiddhiAppContext.startPartitionFlow(allStatesEntry.getKey()); + SiddhiAppContext.startGroupByFlow(stateEntry.getKey()); + try { + if (streamEventChunk.getFirst() != null) { + nextProcessor.process(streamEventChunk); + } + } finally { + SiddhiAppContext.stopGroupByFlow(); + SiddhiAppContext.stopPartitionFlow(); + } } - if (expiredEventChunk.getFirst() != null) { - streamEventChunk.add(expiredEventChunk.getFirst()); - } - expiredEventChunk.clear(); - while (currentEventChunk.hasNext()) { - StreamEvent currentEvent = currentEventChunk.next(); - StreamEvent toExpireEvent = streamEventCloner.copyStreamEvent(currentEvent); - toExpireEvent.setType(StreamEvent.Type.EXPIRED); - expiredEventChunk.add(toExpireEvent); - } - - streamEventChunk.add(currentEventChunk.getFirst()); - currentEventChunk.clear(); - } - } - if (streamEventChunk.getFirst() != null) { - nextProcessor.process(streamEventChunk); + } finally { + stateHolder.returnAllStates(allStates); } } @@ -227,4 +240,32 @@ public void execute(JobExecutionContext jobExecutionContext) throws JobExecution windowProcessor.dispatchEvents(); } + + class WindowState extends State { + private SnapshotableStreamEventQueue currentEventQueue; + private SnapshotableStreamEventQueue expiredEventQueue; + + WindowState(StreamEventClonerHolder streamEventClonerHolder) { + currentEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("CurrentEventQueue", currentEventQueue.getSnapshot()); + state.put("ExpiredEventQueue", expiredEventQueue.getSnapshot()); + return state; + } + + public void restore(Map state) { + currentEventQueue.restore((SnapshotStateList) state.get("CurrentEventQueue")); + expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + } + + @Override + public boolean canDestroy() { + return expiredEventQueue.getFirst() == null && currentEventQueue.getFirst() == null; + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/DelayWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/DelayWindowProcessor.java index 2bc79a4dfd..ca5aa9e83b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/DelayWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/DelayWindowProcessor.java @@ -27,6 +27,7 @@ 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.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -38,6 +39,7 @@ import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; import io.siddhi.core.util.snapshot.state.SnapshotStateList; +import io.siddhi.core.util.snapshot.state.StateFactory; import io.siddhi.query.api.definition.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; import io.siddhi.query.api.expression.Expression; @@ -91,14 +93,11 @@ public class DelayWindowProcessor extends TimeWindowProcessor { private long delayInMilliSeconds; private SiddhiQueryContext siddhiQueryContext; - private SnapshotableStreamEventQueue delayedEventQueue; - private volatile long lastTimestamp = Long.MIN_VALUE; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { this.siddhiQueryContext = siddhiQueryContext; - this.delayedEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); if (attributeExpressionExecutors.length == 1) { if (attributeExpressionExecutors[0] instanceof ConstantExpressionExecutor) { if (attributeExpressionExecutors[0].getReturnType() == Attribute.Type.INT || @@ -117,23 +116,24 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea throw new SiddhiAppValidationException("Delay window should only have one parameter ( " + "delayTime), but found " + attributeExpressionExecutors.length + " input attributes"); } + return () -> new DelayedWindowState(streamEventClonerHolder); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { - synchronized (this) { - + StreamEventCloner streamEventCloner, WindowState windowState) { + DelayedWindowState state = ((DelayedWindowState) windowState); + synchronized (state) { while (streamEventChunk.hasNext()) { StreamEvent streamEvent = streamEventChunk.next(); long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); - delayedEventQueue.reset(); - while (delayedEventQueue.hasNext()) { - StreamEvent delayedEvent = delayedEventQueue.next(); + state.delayedEventQueue.reset(); + while (state.delayedEventQueue.hasNext()) { + StreamEvent delayedEvent = state.delayedEventQueue.next(); long timeDiff = delayedEvent.getTimestamp() - currentTime + delayInMilliSeconds; if (timeDiff <= 0) { - delayedEventQueue.remove(); + state.delayedEventQueue.remove(); //insert delayed event before the current event to stream chunk delayedEvent.setTimestamp(currentTime); streamEventChunk.insertBeforeCurrent(delayedEvent); @@ -143,17 +143,16 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } if (streamEvent.getType() == StreamEvent.Type.CURRENT) { - this.delayedEventQueue.add(streamEvent); - - if (lastTimestamp < streamEvent.getTimestamp()) { + state.delayedEventQueue.add(streamEvent); + if (state.lastTimestamp < streamEvent.getTimestamp()) { getScheduler().notifyAt(streamEvent.getTimestamp() + delayInMilliSeconds); - lastTimestamp = streamEvent.getTimestamp(); + state.lastTimestamp = streamEvent.getTimestamp(); } } //current events are not processed, so remove the current event from the stream chunk streamEventChunk.remove(); } - delayedEventQueue.reset(); + state.delayedEventQueue.reset(); } //only pass to next processor if there are any events in the stream chunk if (streamEventChunk.getFirst() != null) { @@ -162,27 +161,47 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, delayedEventQueue, streamEventCloner); + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, ((DelayedWindowState) state).delayedEventQueue, + streamEventCloner); } @Override public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(delayedEventQueue, condition, matchingMetaInfoHolder, - variableExpressionExecutors, tableMap, siddhiQueryContext); + Map tableMap, WindowState state, + SiddhiQueryContext siddhiQueryContext) { + return OperatorParser.constructOperator(((DelayedWindowState) state).delayedEventQueue, + condition, matchingMetaInfoHolder, variableExpressionExecutors, tableMap, siddhiQueryContext); } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("DelayedEventQueue", delayedEventQueue.getSnapshot()); - return state; - } + class DelayedWindowState extends WindowState { + private SnapshotableStreamEventQueue delayedEventQueue; + private volatile long lastTimestamp = Long.MIN_VALUE; - @Override - public void restoreState(Map state) { - delayedEventQueue.restore((SnapshotStateList) state.get("DelayedEventQueue")); + DelayedWindowState(StreamEventClonerHolder streamEventClonerHolder) { + super(streamEventClonerHolder); + this.delayedEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("DelayedEventQueue", delayedEventQueue.getSnapshot()); + state.put("LastTimestamp", lastTimestamp); + return state; + } + + @Override + public void restore(Map state) { + delayedEventQueue.restore((SnapshotStateList) state.get("DelayedEventQueue")); + lastTimestamp = (long) state.get("LastTimestamp"); + } + + @Override + public boolean canDestroy() { + return delayedEventQueue.getFirst() == null; + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExternalTimeBatchWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExternalTimeBatchWindowProcessor.java index c094957ff8..5d89d498ec 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExternalTimeBatchWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExternalTimeBatchWindowProcessor.java @@ -28,6 +28,7 @@ import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; +import io.siddhi.core.event.stream.holder.StreamEventClonerHolder; import io.siddhi.core.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -40,6 +41,8 @@ import io.siddhi.core.util.collection.operator.Operator; import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; +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; import io.siddhi.query.api.expression.Expression; @@ -80,6 +83,7 @@ optional = true, defaultValue = "System waits till an event from next batch arrives to flush current batch") }, + examples = { @Example( syntax = "define window cseEventWindow (symbol string, price float, volume int) " + @@ -109,34 +113,28 @@ ) } ) -public class ExternalTimeBatchWindowProcessor extends BatchingWindowProcessor - implements SchedulingProcessor, FindableProcessor { - private ComplexEventChunk currentEventChunk = new ComplexEventChunk(false); - private ComplexEventChunk expiredEventChunk = null; - private StreamEvent resetEvent = null; +public class ExternalTimeBatchWindowProcessor + extends BatchingFindableWindowProcessor + implements SchedulingProcessor { private VariableExpressionExecutor timestampExpressionExecutor; private ExpressionExecutor startTimeAsVariable; private long timeToKeep; - private long endTime = -1; - private long startTime = 0; private boolean isStartTimeEnabled = false; private long schedulerTimeout = 0; private Scheduler scheduler; - private long lastScheduledTime; - private long lastCurrentEventTime; - private boolean flushed = false; - private boolean storeExpiredEvents = false; + private boolean findToBeExecuted = false; private boolean replaceTimestampWithBatchEndTime = false; private boolean outputExpectsExpiredEvents; + private long commonStartTime = 0; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, boolean - outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { this.outputExpectsExpiredEvents = outputExpectsExpiredEvents; - if (outputExpectsExpiredEvents) { - this.expiredEventChunk = new ComplexEventChunk(false); - this.storeExpiredEvents = true; - } + this.findToBeExecuted = findToBeExecuted; + if (attributeExpressionExecutors.length >= 2 && attributeExpressionExecutors.length <= 5) { if (!(attributeExpressionExecutors[0] instanceof VariableExpressionExecutor)) { @@ -163,10 +161,10 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea isStartTimeEnabled = true; if ((attributeExpressionExecutors[2] instanceof ConstantExpressionExecutor)) { if (attributeExpressionExecutors[2].getReturnType() == Attribute.Type.INT) { - startTime = Integer.parseInt(String.valueOf(((ConstantExpressionExecutor) + commonStartTime = Integer.parseInt(String.valueOf(((ConstantExpressionExecutor) attributeExpressionExecutors[2]).getValue())); } else if (attributeExpressionExecutors[2].getReturnType() == Attribute.Type.LONG) { - startTime = Long.parseLong(String.valueOf(((ConstantExpressionExecutor) + commonStartTime = Long.parseLong(String.valueOf(((ConstantExpressionExecutor) attributeExpressionExecutors[2]).getValue())); } else { throw new SiddhiAppValidationException("ExternalTimeBatch window's 3rd parameter " + @@ -212,11 +210,7 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea "timeout, replaceTimestampWithBatchEndTime), but found " + attributeExpressionExecutors .length + " input attributes"); } - if (schedulerTimeout > 0) { - if (expiredEventChunk == null) { - this.expiredEventChunk = new ComplexEventChunk(false); - } - } + return () -> new WindowState(outputExpectsExpiredEvents, schedulerTimeout, commonStartTime); } /** @@ -227,7 +221,7 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea */ @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { + StreamEventCloner streamEventCloner, WindowState state) { // event incoming trigger process. No events means no action if (streamEventChunk.getFirst() == null) { @@ -235,8 +229,8 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } List> complexEventChunks = new ArrayList>(); - synchronized (this) { - initTiming(streamEventChunk.getFirst()); + synchronized (state) { + initTiming(streamEventChunk.getFirst(), state); StreamEvent nextStreamEvent = streamEventChunk.getFirst(); while (nextStreamEvent != null) { @@ -245,21 +239,23 @@ protected void process(ComplexEventChunk streamEventChunk, Processo nextStreamEvent = nextStreamEvent.getNext(); if (currStreamEvent.getType() == ComplexEvent.Type.TIMER) { - if (lastScheduledTime <= currStreamEvent.getTimestamp()) { + if (state.lastScheduledTime <= currStreamEvent.getTimestamp()) { // implies that there have not been any more events after this schedule has been done. - if (!flushed) { - flushToOutputChunk(streamEventCloner, complexEventChunks, lastCurrentEventTime, true); - flushed = true; + if (!state.flushed) { + flushToOutputChunk(streamEventCloner, complexEventChunks, state.lastCurrentEventTime, + true, state); + state.flushed = true; } else { - if (currentEventChunk.getFirst() != null) { - appendToOutputChunk(streamEventCloner, complexEventChunks, lastCurrentEventTime, true); + if (state.currentEventChunk.getFirst() != null) { + appendToOutputChunk(streamEventCloner, complexEventChunks, state.lastCurrentEventTime, + true, state); } } // rescheduling to emit the current batch after expiring it if no further events arrive. - lastScheduledTime = siddhiQueryContext.getSiddhiAppContext(). + state.lastScheduledTime = siddhiQueryContext.getSiddhiAppContext(). getTimestampGenerator().currentTime() + schedulerTimeout; - scheduler.notifyAt(lastScheduledTime); + scheduler.notifyAt(state.lastScheduledTime); } continue; } else if (currStreamEvent.getType() != ComplexEvent.Type.CURRENT) { @@ -267,27 +263,29 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } long currentEventTime = (Long) timestampExpressionExecutor.execute(currStreamEvent); - if (lastCurrentEventTime < currentEventTime) { - lastCurrentEventTime = currentEventTime; + if (state.lastCurrentEventTime < currentEventTime) { + state.lastCurrentEventTime = currentEventTime; } - if (currentEventTime < endTime) { - cloneAppend(streamEventCloner, currStreamEvent); + if (currentEventTime < state.endTime) { + cloneAppend(streamEventCloner, currStreamEvent, state); } else { - if (flushed) { - appendToOutputChunk(streamEventCloner, complexEventChunks, lastCurrentEventTime, false); - flushed = false; + if (state.flushed) { + appendToOutputChunk(streamEventCloner, complexEventChunks, state.lastCurrentEventTime, + false, state); + state.flushed = false; } else { - flushToOutputChunk(streamEventCloner, complexEventChunks, lastCurrentEventTime, false); + flushToOutputChunk(streamEventCloner, complexEventChunks, state.lastCurrentEventTime, + false, state); } // update timestamp, call next processor - endTime = findEndTime(lastCurrentEventTime, startTime, timeToKeep); - cloneAppend(streamEventCloner, currStreamEvent); + state.endTime = findEndTime(state.lastCurrentEventTime, state.startTime, timeToKeep); + cloneAppend(streamEventCloner, currStreamEvent, state); // triggering the last batch expiration. if (schedulerTimeout > 0) { - lastScheduledTime = siddhiQueryContext.getSiddhiAppContext(). + state.lastScheduledTime = siddhiQueryContext.getSiddhiAppContext(). getTimestampGenerator().currentTime() + schedulerTimeout; - scheduler.notifyAt(lastScheduledTime); + scheduler.notifyAt(state.lastScheduledTime); } } } @@ -297,90 +295,89 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } } - private void initTiming(StreamEvent firstStreamEvent) { + private void initTiming(StreamEvent firstStreamEvent, WindowState state) { // for window beginning, if window is empty, set lastSendTime to incomingChunk first. - if (endTime < 0) { + if (state.endTime < 0) { if (isStartTimeEnabled) { if (startTimeAsVariable == null) { - endTime = findEndTime((Long) timestampExpressionExecutor.execute(firstStreamEvent), startTime, - timeToKeep); + state.endTime = findEndTime((Long) timestampExpressionExecutor.execute(firstStreamEvent), + state.startTime, timeToKeep); } else { - startTime = (Long) startTimeAsVariable.execute(firstStreamEvent); - endTime = startTime + timeToKeep; + state.startTime = (Long) startTimeAsVariable.execute(firstStreamEvent); + state.endTime = state.startTime + timeToKeep; } } else { - startTime = (Long) timestampExpressionExecutor.execute(firstStreamEvent); - endTime = startTime + timeToKeep; + state.startTime = (Long) timestampExpressionExecutor.execute(firstStreamEvent); + state.endTime = state.startTime + timeToKeep; } if (schedulerTimeout > 0) { - lastScheduledTime = siddhiQueryContext.getSiddhiAppContext(). + state.lastScheduledTime = siddhiQueryContext.getSiddhiAppContext(). getTimestampGenerator().currentTime() + schedulerTimeout; - scheduler.notifyAt(lastScheduledTime); + scheduler.notifyAt(state.lastScheduledTime); } } } - private void flushToOutputChunk(StreamEventCloner streamEventCloner, List> - complexEventChunks, - long currentTime, boolean preserveCurrentEvents) { + private void flushToOutputChunk(StreamEventCloner streamEventCloner, + List> complexEventChunks, + long currentTime, boolean preserveCurrentEvents, WindowState state) { ComplexEventChunk newEventChunk = new ComplexEventChunk(true); if (outputExpectsExpiredEvents) { - if (expiredEventChunk.getFirst() != null) { + if (state.expiredEventChunk.getFirst() != null) { // mark the timestamp for the expiredType event - expiredEventChunk.reset(); - while (expiredEventChunk.hasNext()) { - StreamEvent expiredEvent = expiredEventChunk.next(); + state.expiredEventChunk.reset(); + while (state.expiredEventChunk.hasNext()) { + StreamEvent expiredEvent = state.expiredEventChunk.next(); expiredEvent.setTimestamp(currentTime); } // add expired event to newEventChunk. - newEventChunk.add(expiredEventChunk.getFirst()); + newEventChunk.add(state.expiredEventChunk.getFirst()); } } - if (expiredEventChunk != null) { - expiredEventChunk.clear(); + if (state.expiredEventChunk != null) { + state.expiredEventChunk.clear(); } - if (currentEventChunk.getFirst() != null) { + if (state.currentEventChunk.getFirst() != null) { // add reset event in front of current events - resetEvent.setTimestamp(currentTime); - newEventChunk.add(resetEvent); - resetEvent = null; + state.resetEvent.setTimestamp(currentTime); + newEventChunk.add(state.resetEvent); + state.resetEvent = null; // move to expired events - if (preserveCurrentEvents || storeExpiredEvents) { - currentEventChunk.reset(); - while (currentEventChunk.hasNext()) { - StreamEvent currentEvent = currentEventChunk.next(); + if (preserveCurrentEvents || state.expiredEventChunk != null) { + state.currentEventChunk.reset(); + while (state.currentEventChunk.hasNext()) { + StreamEvent currentEvent = state.currentEventChunk.next(); StreamEvent toExpireEvent = streamEventCloner.copyStreamEvent(currentEvent); toExpireEvent.setType(StreamEvent.Type.EXPIRED); - expiredEventChunk.add(toExpireEvent); + state.expiredEventChunk.add(toExpireEvent); } } // add current event chunk to next processor - newEventChunk.add(currentEventChunk.getFirst()); + newEventChunk.add(state.currentEventChunk.getFirst()); } - currentEventChunk.clear(); + state.currentEventChunk.clear(); if (newEventChunk.getFirst() != null) { complexEventChunks.add(newEventChunk); } } - private void appendToOutputChunk(StreamEventCloner streamEventCloner, List> - complexEventChunks, - long currentTime, boolean preserveCurrentEvents) { + private void appendToOutputChunk(StreamEventCloner streamEventCloner, + List> complexEventChunks, + long currentTime, boolean preserveCurrentEvents, WindowState state) { ComplexEventChunk newEventChunk = new ComplexEventChunk(true); ComplexEventChunk sentEventChunk = new ComplexEventChunk(true); - if (currentEventChunk.getFirst() != null) { - - if (expiredEventChunk.getFirst() != null) { + if (state.currentEventChunk.getFirst() != null) { + if (state.expiredEventChunk != null && state.expiredEventChunk.getFirst() != null) { // mark the timestamp for the expiredType event - expiredEventChunk.reset(); - while (expiredEventChunk.hasNext()) { - StreamEvent expiredEvent = expiredEventChunk.next(); + state.expiredEventChunk.reset(); + while (state.expiredEventChunk.hasNext()) { + StreamEvent expiredEvent = state.expiredEventChunk.next(); if (outputExpectsExpiredEvents) { // add expired event to newEventChunk. @@ -396,7 +393,7 @@ private void appendToOutputChunk(StreamEventCloner streamEventCloner, List currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("StartTime", startTime); - state.put("EndTime", endTime); - state.put("LastScheduledTime", lastScheduledTime); - state.put("LastCurrentEventTime", lastCurrentEventTime); - state.put("CurrentEventChunk", currentEventChunk.getFirst()); - state.put("ExpiredEventChunk", expiredEventChunk != null ? expiredEventChunk.getFirst() : null); - state.put("ResetEvent", resetEvent); - state.put("Flushed", flushed); - } - return state; - } - - - @Override - public synchronized void restoreState(Map state) { - startTime = (long) state.get("StartTime"); - endTime = (long) state.get("EndTime"); - lastScheduledTime = (long) state.get("LastScheduledTime"); - lastCurrentEventTime = (long) state.get("LastCurrentEventTime"); - currentEventChunk.clear(); - currentEventChunk.add((StreamEvent) state.get("CurrentEventChunk")); - if (expiredEventChunk != null) { - expiredEventChunk.clear(); - expiredEventChunk.add((StreamEvent) state.get("ExpiredEventChunk")); - } else { - if (outputExpectsExpiredEvents) { - expiredEventChunk = new ComplexEventChunk(false); - } - if (schedulerTimeout > 0) { - expiredEventChunk = new ComplexEventChunk(false); - } + public void stop() { + if (scheduler != null) { + scheduler.stop(); } - resetEvent = (StreamEvent) state.get("ResetEvent"); - flushed = (boolean) state.get("Flushed"); - - } - - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, expiredEventChunk, streamEventCloner); } @Override public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - if (expiredEventChunk == null) { - expiredEventChunk = new ComplexEventChunk(false); - storeExpiredEvents = true; - } - return OperatorParser.constructOperator(expiredEventChunk, condition, matchingMetaInfoHolder, + Map tableMap, WindowState state, + SiddhiQueryContext siddhiQueryContext) { + return OperatorParser.constructOperator(state.expiredEventChunk, condition, matchingMetaInfoHolder, variableExpressionExecutors, tableMap, siddhiQueryContext); } + @Override + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.expiredEventChunk, streamEventCloner); + + } + @Override public Scheduler getScheduler() { return this.scheduler; @@ -517,4 +476,74 @@ public Scheduler getScheduler() { public void setScheduler(Scheduler scheduler) { this.scheduler = scheduler; } + + class WindowState extends State { + + private ComplexEventChunk currentEventChunk = new ComplexEventChunk(false); + private ComplexEventChunk expiredEventChunk = null; + private StreamEvent resetEvent = null; + private long endTime = -1; + private long startTime = 0; + private long lastScheduledTime; + private long lastCurrentEventTime; + private boolean flushed = false; + + + public WindowState(boolean outputExpectsExpiredEvents, long schedulerTimeout, long startTime) { + this.startTime = startTime; + if (outputExpectsExpiredEvents || findToBeExecuted) { + this.expiredEventChunk = new ComplexEventChunk<>(false); + } + if (schedulerTimeout > 0) { + if (expiredEventChunk == null) { + this.expiredEventChunk = new ComplexEventChunk<>(false); + } + } + } + + + @Override + public boolean canDestroy() { + return currentEventChunk.getFirst() == null && + (expiredEventChunk == null || expiredEventChunk.getFirst() == null) && + resetEvent == null && flushed; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("StartTime", startTime); + state.put("EndTime", endTime); + state.put("LastScheduledTime", lastScheduledTime); + state.put("LastCurrentEventTime", lastCurrentEventTime); + state.put("CurrentEventChunk", currentEventChunk.getFirst()); + state.put("ExpiredEventChunk", expiredEventChunk != null ? expiredEventChunk.getFirst() : null); + state.put("ResetEvent", resetEvent); + state.put("Flushed", flushed); + return state; + } + + @Override + public void restore(Map state) { + startTime = (long) state.get("StartTime"); + endTime = (long) state.get("EndTime"); + lastScheduledTime = (long) state.get("LastScheduledTime"); + lastCurrentEventTime = (long) state.get("LastCurrentEventTime"); + currentEventChunk.clear(); + currentEventChunk.add((StreamEvent) state.get("CurrentEventChunk")); + if (expiredEventChunk != null) { + expiredEventChunk.clear(); + expiredEventChunk.add((StreamEvent) state.get("ExpiredEventChunk")); + } else { + if (outputExpectsExpiredEvents || findToBeExecuted) { + expiredEventChunk = new ComplexEventChunk(false); + } + if (schedulerTimeout > 0) { + expiredEventChunk = new ComplexEventChunk(false); + } + } + resetEvent = (StreamEvent) state.get("ResetEvent"); + flushed = (boolean) state.get("Flushed"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExternalTimeWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExternalTimeWindowProcessor.java index 45211bcce3..261dc6d5a1 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExternalTimeWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/ExternalTimeWindowProcessor.java @@ -39,6 +39,8 @@ import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; 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.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; import io.siddhi.query.api.expression.Expression; @@ -81,16 +83,15 @@ "from the eventTime and output expired events." ) ) -public class ExternalTimeWindowProcessor extends SlidingWindowProcessor implements FindableProcessor { +public class ExternalTimeWindowProcessor + extends SlidingFindableWindowProcessor { private static final Logger log = Logger.getLogger(ExternalTimeWindowProcessor.class); private long timeToKeep; - private SnapshotableStreamEventQueue expiredEventQueue; private VariableExpressionExecutor timeStampVariableExpressionExecutor; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { - this.expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length == 2) { if (attributeExpressionExecutors[1].getReturnType() == Attribute.Type.INT) { timeToKeep = Integer.parseInt(String.valueOf(((ConstantExpressionExecutor) @@ -113,12 +114,13 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea "timeStamp, windowTime), but found " + attributeExpressionExecutors.length + " " + "input attributes"); } + return () -> new WindowState(); } @Override - protected synchronized void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { - synchronized (this) { + protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, + StreamEventCloner streamEventCloner, WindowState state) { + synchronized (state) { while (streamEventChunk.hasNext()) { StreamEvent streamEvent = streamEventChunk.next(); @@ -129,25 +131,25 @@ protected synchronized void process(ComplexEventChunk streamEventCh // reset expiredEventQueue to make sure all of the expired events get removed, // otherwise lastReturned.next will always return null and here while check is always false - expiredEventQueue.reset(); - while (expiredEventQueue.hasNext()) { - StreamEvent expiredEvent = expiredEventQueue.next(); + state.expiredEventQueue.reset(); + while (state.expiredEventQueue.hasNext()) { + StreamEvent expiredEvent = state.expiredEventQueue.next(); long expiredEventTime = (Long) timeStampVariableExpressionExecutor.execute(expiredEvent); long timeDiff = expiredEventTime - currentTime + timeToKeep; if (timeDiff <= 0) { - expiredEventQueue.remove(); + state.expiredEventQueue.remove(); expiredEvent.setTimestamp(currentTime); streamEventChunk.insertBeforeCurrent(expiredEvent); } else { - expiredEventQueue.reset(); + state.expiredEventQueue.reset(); break; } } if (streamEvent.getType() == StreamEvent.Type.CURRENT) { - this.expiredEventQueue.add(clonedEvent); + state.expiredEventQueue.add(clonedEvent); } - expiredEventQueue.reset(); + state.expiredEventQueue.reset(); } } nextProcessor.process(streamEventChunk); @@ -164,28 +166,40 @@ public void stop() { } @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("ExpiredEventQueue", expiredEventQueue.getSnapshot()); - return state; + 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 void restoreState(Map state) { - expiredEventQueue.clear(); - expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.expiredEventQueue, streamEventCloner); } - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, expiredEventQueue, streamEventCloner); - } + class WindowState extends State { + private SnapshotableStreamEventQueue expiredEventQueue = + new SnapshotableStreamEventQueue(streamEventClonerHolder); - @Override - public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, - List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(expiredEventQueue, condition, matchingMetaInfoHolder, - variableExpressionExecutors, tableMap, siddhiQueryContext); + @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")); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/FrequentWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/FrequentWindowProcessor.java index ad1141f579..55f101ad32 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/FrequentWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/FrequentWindowProcessor.java @@ -37,6 +37,8 @@ import io.siddhi.core.util.collection.operator.Operator; import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import io.siddhi.query.api.expression.Expression; import java.util.ArrayList; @@ -85,27 +87,28 @@ } ) @Deprecated -public class FrequentWindowProcessor extends SlidingWindowProcessor implements FindableProcessor { - private ConcurrentHashMap countMap = new ConcurrentHashMap(); - private ConcurrentHashMap map = new ConcurrentHashMap(); +public class FrequentWindowProcessor extends SlidingFindableWindowProcessor { private VariableExpressionExecutor[] variableExpressionExecutors; private int mostFrequentCount; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { mostFrequentCount = Integer.parseInt(String.valueOf(((ConstantExpressionExecutor) attributeExpressionExecutors[0]).getValue())); variableExpressionExecutors = new VariableExpressionExecutor[attributeExpressionExecutors.length - 1]; for (int i = 1; i < attributeExpressionExecutors.length; i++) { variableExpressionExecutors[i - 1] = (VariableExpressionExecutor) attributeExpressionExecutors[i]; } + return () -> new WindowState(); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { + StreamEventCloner streamEventCloner, WindowState state) { + synchronized (this) { StreamEvent streamEvent = streamEventChunk.getFirst(); streamEventChunk.clear(); @@ -118,39 +121,39 @@ protected void process(ComplexEventChunk streamEventChunk, Processo clonedEvent.setType(StreamEvent.Type.EXPIRED); String key = generateKey(streamEvent); - StreamEvent oldEvent = map.put(key, clonedEvent); + StreamEvent oldEvent = state.map.put(key, clonedEvent); if (oldEvent != null) { - countMap.put(key, countMap.get(key) + 1); + state.countMap.put(key, state.countMap.get(key) + 1); streamEventChunk.add(streamEvent); } else { // This is a new event - if (map.size() > mostFrequentCount) { - List keys = new ArrayList(countMap.keySet()); + if (state.map.size() > mostFrequentCount) { + List keys = new ArrayList(state.countMap.keySet()); for (int i = 0; i < mostFrequentCount; i++) { - int count = countMap.get(keys.get(i)) - 1; + int count = state.countMap.get(keys.get(i)) - 1; if (count == 0) { - countMap.remove(keys.get(i)); - StreamEvent expiredEvent = map.remove(keys.get(i)); + state.countMap.remove(keys.get(i)); + StreamEvent expiredEvent = state.map.remove(keys.get(i)); expiredEvent.setTimestamp(currentTime); streamEventChunk.add(expiredEvent); } else { - countMap.put(keys.get(i), count); + state.countMap.put(keys.get(i), count); } } // now we have tried to remove one for newly added item - if (map.size() > mostFrequentCount) { + if (state.map.size() > mostFrequentCount) { //nothing happend by the attempt to remove one from the // map so we are ignoring this event - map.remove(key); + state.map.remove(key); // Here we do nothing just drop the message } else { // we got some space, event is already there in map object // we just have to add it to the countMap - countMap.put(key, 1); + state.countMap.put(key, 1); streamEventChunk.add(streamEvent); } } else { - countMap.put(generateKey(streamEvent), 1); + state.countMap.put(generateKey(streamEvent), 1); streamEventChunk.add(streamEvent); } } @@ -170,20 +173,6 @@ public void stop() { //Do nothing } - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("CountMap", countMap); - } - return state; - } - - @Override - public synchronized void restoreState(Map state) { - countMap = (ConcurrentHashMap) state.get("CountMap"); - } - private String generateKey(StreamEvent event) { // for performance reason if its all attribute we don't do // the attribute list check StringBuilder stringBuilder = new StringBuilder(); @@ -199,16 +188,40 @@ private String generateKey(StreamEvent event) { // for performance reason i return stringBuilder.toString(); } - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, map.values(), streamEventCloner); - } - @Override public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(map.values(), condition, matchingMetaInfoHolder, + Map tableMap, WindowState state, + SiddhiQueryContext siddhiQueryContext) { + return OperatorParser.constructOperator(state.map.values(), condition, matchingMetaInfoHolder, variableExpressionExecutors, tableMap, siddhiQueryContext); } + + @Override + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.map.values(), streamEventCloner); + } + + class WindowState extends State { + private ConcurrentHashMap countMap = new ConcurrentHashMap(); + private ConcurrentHashMap map = new ConcurrentHashMap(); + + @Override + public boolean canDestroy() { + return countMap.isEmpty() && map.isEmpty(); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("CountMap", countMap); + return state; + } + + @Override + public void restore(Map state) { + countMap = (ConcurrentHashMap) state.get("CountMap"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/GroupingFindableWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/GroupingFindableWindowProcessor.java new file mode 100644 index 0000000000..11add75eee --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/GroupingFindableWindowProcessor.java @@ -0,0 +1,98 @@ +/* + * 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.core.config.SiddhiQueryContext; +import io.siddhi.core.event.state.StateEvent; +import io.siddhi.core.event.stream.StreamEvent; +import io.siddhi.core.event.stream.StreamEventCloner; +import io.siddhi.core.executor.VariableExpressionExecutor; +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.snapshot.state.State; +import io.siddhi.query.api.expression.Expression; + +import java.util.List; +import java.util.Map; + +/** + * Window that process event processing as a group while supporting event search + * + * @param current state of the processor + */ +public abstract class GroupingFindableWindowProcessor extends GroupingWindowProcessor + implements FindableProcessor { + + @Override + public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, + List variableExpressionExecutors, + Map tableMap, SiddhiQueryContext siddhiQueryContext) { + S state = stateHolder.getState(); + try { + return compileCondition(condition, matchingMetaInfoHolder, variableExpressionExecutors, tableMap, + state, siddhiQueryContext); + } finally { + stateHolder.returnState(state); + } + } + + @Override + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { + S state = stateHolder.getState(); + try { + return find(matchingEvent, compiledCondition, streamEventClonerHolder.getStreamEventCloner(), + state); + } finally { + stateHolder.returnState(state); + } + + } + + /** + * To construct a finder having the capability of finding events at the processor that corresponds to the incoming + * matchingEvent and the given matching expression logic. + * + * @param condition the matching condition + * @param matchingMetaInfoHolder the meta structure of the incoming matchingEvent + * @param variableExpressionExecutors the list of variable ExpressionExecutors already created + * @param tableMap map of event tables + * @param state current query state + * @param siddhiQueryContext current siddhi query context + * @return compiled Condition having the capability of matching events against the incoming matchingEvent + */ + public abstract CompiledCondition compileCondition(Expression condition, + MatchingMetaInfoHolder matchingMetaInfoHolder, + List variableExpressionExecutors, + Map tableMap, S state, + SiddhiQueryContext siddhiQueryContext); + + /** + * To find events from the processor event pool, that the matches the matchingEvent based on finder logic. + * + * @param matchingEvent the event to be matched with the events at the processor + * @param compiledCondition the execution element responsible for matching the corresponding events that matches + * the matchingEvent based on pool of events at Processor + * @param streamEventCloner stream event cloner + * @param state current query state @return the matched events + */ + public abstract StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, S 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 4046ae57a5..9ce8bde753 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 @@ -24,6 +24,7 @@ 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.StreamEventClonerHolder; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.event.stream.populater.SelectiveComplexEventPopulater; import io.siddhi.core.event.stream.populater.StreamEventPopulaterFactory; @@ -31,6 +32,8 @@ import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; 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.AbstractDefinition; import io.siddhi.query.api.definition.Attribute; @@ -39,24 +42,27 @@ /** * Performs event processing with key based event groups + * + * @param current state of the processor */ -public abstract class GroupingWindowProcessor extends WindowProcessor { +public abstract class GroupingWindowProcessor extends WindowProcessor { protected List internalAttributes; protected GroupingKeyPopulator groupingKeyPopulator; @Override - protected List init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, + protected StateFactory init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, - boolean outputExpectsExpiredEvents, + ConfigReader configReader, StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, SiddhiQueryContext siddhiQueryContext) { - init(attributeExpressionExecutors, configReader, outputExpectsExpiredEvents, siddhiQueryContext); + StateFactory stateFactory = init(attributeExpressionExecutors, configReader, outputExpectsExpiredEvents, + siddhiQueryContext); Attribute groupingKey = new Attribute("_groupingKey", Attribute.Type.STRING); internalAttributes = new ArrayList(1); internalAttributes.add(groupingKey); metaStreamEvent.addData(groupingKey); - return new ArrayList<>(0); + return stateFactory; } /** @@ -67,13 +73,14 @@ protected List init(MetaStreamEvent metaStreamEvent, AbstractDefiniti * @param outputExpectsExpiredEvents is expired events sent as output * @param siddhiQueryContext the context of the siddhi query */ - protected abstract void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext); + protected abstract StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, boolean outputExpectsExpiredEvents, + SiddhiQueryContext siddhiQueryContext); protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, StreamEventCloner streamEventCloner, - ComplexEventPopulater complexEventPopulater) { - processEventChunk(streamEventChunk, nextProcessor, streamEventCloner, groupingKeyPopulator); + ComplexEventPopulater complexEventPopulater, S state) { + processEventChunk(streamEventChunk, nextProcessor, streamEventCloner, groupingKeyPopulator, state); } /** @@ -83,10 +90,11 @@ protected void processEventChunk(ComplexEventChunk streamEventChunk * @param nextProcessor the next processor to which the success events need to be passed * @param streamEventCloner helps to clone the incoming event for local storage or modification * @param groupingKeyPopulater helps to populate the events with the grouping key + * @param state current state of the processor */ protected abstract void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, StreamEventCloner streamEventCloner, - GroupingKeyPopulator groupingKeyPopulater); + GroupingKeyPopulator groupingKeyPopulater, S state); public void constructStreamEventPopulater(MetaStreamEvent metaStreamEvent, int streamEventChainIndex) { super.constructStreamEventPopulater(metaStreamEvent, streamEventChainIndex); 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 1f076c8345..1491c3a27d 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 @@ -24,6 +24,7 @@ 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.StreamEventClonerHolder; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.event.stream.populater.SelectiveComplexEventPopulater; import io.siddhi.core.event.stream.populater.StreamEventPopulaterFactory; @@ -31,6 +32,8 @@ import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; 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.AbstractDefinition; import io.siddhi.query.api.definition.Attribute; @@ -39,23 +42,27 @@ /** * Performs event processing in a hopping manner + * + * @param current state of the processor */ -public abstract class HopingWindowProcessor extends WindowProcessor { +public abstract class HopingWindowProcessor extends WindowProcessor { protected List internalAttributes; protected HopingTimestampPopulator hopingTimestampPopulator; @Override - protected List init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, + protected StateFactory init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { - init(attributeExpressionExecutors, configReader, outputExpectsExpiredEvents, siddhiQueryContext); + ConfigReader configReader, StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { + StateFactory stateFactory = init(attributeExpressionExecutors, configReader, outputExpectsExpiredEvents, + siddhiQueryContext); Attribute groupingKey = new Attribute("_hopingTimestamp", Attribute.Type.STRING); internalAttributes = new ArrayList(1); internalAttributes.add(groupingKey); metaStreamEvent.addData(groupingKey); - return new ArrayList<>(0); + return stateFactory; } /** @@ -66,12 +73,14 @@ protected List init(MetaStreamEvent metaStreamEvent, AbstractDefiniti * @param outputExpectsExpiredEvents is expired events sent as output * @param siddhiQueryContext the context of the siddhi query */ - protected abstract void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext); + protected abstract StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + boolean outputExpectsExpiredEvents, + SiddhiQueryContext siddhiQueryContext); protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, StreamEventCloner streamEventCloner, - ComplexEventPopulater complexEventPopulater) { + ComplexEventPopulater complexEventPopulater, S state) { processEventChunk(streamEventChunk, nextProcessor, streamEventCloner, hopingTimestampPopulator); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LengthBatchWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LengthBatchWindowProcessor.java index c58ef383eb..deb0e6b66c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LengthBatchWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LengthBatchWindowProcessor.java @@ -28,6 +28,7 @@ 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.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -39,6 +40,8 @@ import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; 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.exception.SiddhiAppValidationException; import io.siddhi.query.api.expression.Expression; @@ -102,20 +105,18 @@ ) } ) -public class LengthBatchWindowProcessor extends BatchingWindowProcessor implements FindableProcessor { +public class LengthBatchWindowProcessor extends + BatchingFindableWindowProcessor { private int length; - private int count = 0; - private SnapshotableStreamEventQueue currentEventQueue = null; - private SnapshotableStreamEventQueue expiredEventQueue = null; private boolean outputExpectsExpiredEvents; private SiddhiQueryContext siddhiQueryContext; private boolean isStreamCurrentEvents = false; - private StreamEvent resetEvent = null; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, boolean outputExpectsExpiredEvents, + boolean findToBeExecuted, SiddhiQueryContext siddhiQueryContext) { this.outputExpectsExpiredEvents = outputExpectsExpiredEvents; this.siddhiQueryContext = siddhiQueryContext; if (attributeExpressionExecutors.length >= 1) { @@ -140,35 +141,34 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea "window.length) or two parameters ( window.length, stream.current.event), " + "but found " + attributeExpressionExecutors.length + " input parameters."); } - if (!isStreamCurrentEvents) { - currentEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); - } - if (outputExpectsExpiredEvents) { - expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); - } + return () -> new WindowState(streamEventClonerHolder, isStreamCurrentEvents, + outputExpectsExpiredEvents, findToBeExecuted); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { + StreamEventCloner streamEventCloner, WindowState state) { + List> streamEventChunks = new ArrayList>(); - synchronized (this) { + synchronized (state) { ComplexEventChunk outputStreamEventChunk = new ComplexEventChunk(true); long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); while (streamEventChunk.hasNext()) { StreamEvent streamEvent = streamEventChunk.next(); streamEventChunk.remove(); if (length == 0) { - processLengthZeroBatch(streamEvent, outputStreamEventChunk, currentTime); + processLengthZeroBatch(streamEvent, outputStreamEventChunk, currentTime, streamEventCloner); } else { - if (resetEvent == null) { - resetEvent = streamEventCloner.copyStreamEvent(streamEvent); - resetEvent.setType(ComplexEvent.Type.RESET); + if (state.resetEvent == null) { + state.resetEvent = streamEventCloner.copyStreamEvent(streamEvent); + state.resetEvent.setType(ComplexEvent.Type.RESET); } if (isStreamCurrentEvents) { - processStreamCurrentEvents(streamEvent, outputStreamEventChunk, currentTime); + processStreamCurrentEvents(streamEvent, outputStreamEventChunk, currentTime, state, + streamEventCloner); } else { - processFullBatchEvents(streamEvent, outputStreamEventChunk, currentTime); + processFullBatchEvents(streamEvent, outputStreamEventChunk, currentTime, state, + streamEventCloner); } } if (outputStreamEventChunk.getFirst() != null) { @@ -184,7 +184,7 @@ protected void process(ComplexEventChunk streamEventChunk, Processo private void processLengthZeroBatch(StreamEvent streamEvent, ComplexEventChunk outputStreamEventChunk, - long currentTime) { + long currentTime, StreamEventCloner streamEventCloner) { outputStreamEventChunk.add(streamEvent); if (outputExpectsExpiredEvents) { @@ -201,69 +201,70 @@ private void processLengthZeroBatch(StreamEvent streamEvent, private void processFullBatchEvents(StreamEvent streamEvent, ComplexEventChunk outputStreamEventChunk, - long currentTime) { + long currentTime, WindowState state, StreamEventCloner streamEventCloner) { StreamEvent clonedStreamEvent = streamEventCloner.copyStreamEvent(streamEvent); - currentEventQueue.add(clonedStreamEvent); - count++; - if (count == length) { - if (outputExpectsExpiredEvents && expiredEventQueue.getFirst() != null) { - while (expiredEventQueue.hasNext()) { - StreamEvent expiredEvent = expiredEventQueue.next(); + state.currentEventQueue.add(clonedStreamEvent); + state.count++; + if (state.count == length) { + if (outputExpectsExpiredEvents && state.expiredEventQueue.getFirst() != null) { + while (state.expiredEventQueue.hasNext()) { + StreamEvent expiredEvent = state.expiredEventQueue.next(); expiredEvent.setTimestamp(currentTime); } - outputStreamEventChunk.add(expiredEventQueue.getFirst()); - expiredEventQueue.clear(); + outputStreamEventChunk.add(state.expiredEventQueue.getFirst()); + state.expiredEventQueue.clear(); } - if (resetEvent != null) { - resetEvent.setTimestamp(currentTime); - outputStreamEventChunk.add(resetEvent); - resetEvent = null; + if (state.resetEvent != null) { + state.resetEvent.setTimestamp(currentTime); + outputStreamEventChunk.add(state.resetEvent); + state.resetEvent = null; } - if (currentEventQueue.getFirst() != null) { - if (expiredEventQueue != null) { - currentEventQueue.reset(); - while (currentEventQueue.hasNext()) { - StreamEvent currentEvent = currentEventQueue.next(); + if (state.currentEventQueue.getFirst() != null) { + if (state.expiredEventQueue != null) { + state.currentEventQueue.reset(); + while (state.currentEventQueue.hasNext()) { + StreamEvent currentEvent = state.currentEventQueue.next(); StreamEvent toExpireEvent = streamEventCloner.copyStreamEvent(currentEvent); toExpireEvent.setType(StreamEvent.Type.EXPIRED); - expiredEventQueue.add(toExpireEvent); + state.expiredEventQueue.add(toExpireEvent); } } - outputStreamEventChunk.add(currentEventQueue.getFirst()); - currentEventQueue.clear(); + outputStreamEventChunk.add(state.currentEventQueue.getFirst()); + state.currentEventQueue.clear(); } - count = 0; + state.count = 0; } } private void processStreamCurrentEvents(StreamEvent streamEvent, ComplexEventChunk outputStreamEventChunk, - long currentTime) { - count++; - if (count == length + 1) { - if (outputExpectsExpiredEvents && expiredEventQueue.getFirst() != null) { - while (expiredEventQueue.hasNext()) { - StreamEvent expiredEvent = expiredEventQueue.next(); + long currentTime, WindowState state, + StreamEventCloner streamEventCloner) { + state.count++; + if (state.count == length + 1) { + if (outputExpectsExpiredEvents && state.expiredEventQueue.getFirst() != null) { + while (state.expiredEventQueue.hasNext()) { + StreamEvent expiredEvent = state.expiredEventQueue.next(); expiredEvent.setTimestamp(currentTime); } - outputStreamEventChunk.add(expiredEventQueue.getFirst()); - expiredEventQueue.clear(); + outputStreamEventChunk.add(state.expiredEventQueue.getFirst()); + state.expiredEventQueue.clear(); } - if (resetEvent != null) { - resetEvent.setTimestamp(currentTime); - outputStreamEventChunk.add(resetEvent); - resetEvent = null; + if (state.resetEvent != null) { + state.resetEvent.setTimestamp(currentTime); + outputStreamEventChunk.add(state.resetEvent); + state.resetEvent = null; } - count = 1; + state.count = 1; } outputStreamEventChunk.add(streamEvent); - if (expiredEventQueue != null) { + if (state.expiredEventQueue != null) { StreamEvent clonedStreamEvent = streamEventCloner.copyStreamEvent(streamEvent); clonedStreamEvent.setType(StreamEvent.Type.EXPIRED); - expiredEventQueue.add(clonedStreamEvent); + state.expiredEventQueue.add(clonedStreamEvent); } } @@ -278,47 +279,69 @@ public void stop() { //Do nothing } + @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("Count", count); - state.put("CurrentEventQueue", currentEventQueue != null ? currentEventQueue.getSnapshot() : null); - state.put("ExpiredEventQueue", expiredEventQueue != null ? expiredEventQueue.getSnapshot() : null); - state.put("ResetEvent", resetEvent); - } - return state; + 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 synchronized void restoreState(Map state) { - count = (int) state.get("Count"); - if (currentEventQueue != null) { - currentEventQueue.clear(); - currentEventQueue.restore((SnapshotStateList) state.get("CurrentEventQueue")); + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.expiredEventQueue, streamEventCloner); + } + + class WindowState extends State { + + private int count = 0; + private SnapshotableStreamEventQueue currentEventQueue = null; + private SnapshotableStreamEventQueue expiredEventQueue = null; + private StreamEvent resetEvent = null; + + public WindowState(StreamEventClonerHolder streamEventClonerHolder, boolean isStreamCurrentEvents, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted) { + if (!isStreamCurrentEvents) { + currentEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + } + if (outputExpectsExpiredEvents || findToBeExecuted) { + expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + } } - if (expiredEventQueue != null) { - expiredEventQueue.clear(); - expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + @Override + public boolean canDestroy() { + return (currentEventQueue == null || currentEventQueue.getFirst() == null) && + (expiredEventQueue == null || expiredEventQueue.getFirst() == null) && + resetEvent == null && count == 0; } - resetEvent = (StreamEvent) state.get("ResetEvent"); - } - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, expiredEventQueue, streamEventCloner); - } + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Count", count); + state.put("CurrentEventQueue", currentEventQueue != null ? currentEventQueue.getSnapshot() : null); + state.put("ExpiredEventQueue", expiredEventQueue != null ? expiredEventQueue.getSnapshot() : null); + state.put("ResetEvent", resetEvent); + return state; + } - @Override - public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, - List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - if (expiredEventQueue == null) { - expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + @Override + public void restore(Map state) { + count = (int) state.get("Count"); + if (currentEventQueue != null) { + currentEventQueue.clear(); + currentEventQueue.restore((SnapshotStateList) state.get("CurrentEventQueue")); + } + + if (expiredEventQueue != null) { + expiredEventQueue.clear(); + expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + } + resetEvent = (StreamEvent) state.get("ResetEvent"); } - return OperatorParser.constructOperator(expiredEventQueue, condition, matchingMetaInfoHolder, - variableExpressionExecutors, tableMap, siddhiQueryContext); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LengthWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LengthWindowProcessor.java index 674cbb48a7..fa81a64576 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LengthWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LengthWindowProcessor.java @@ -39,6 +39,8 @@ import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; 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.exception.SiddhiAppValidationException; import io.siddhi.query.api.expression.Expression; @@ -72,11 +74,9 @@ description = "This will process last 10 events in a sliding manner." ) ) -public class LengthWindowProcessor extends SlidingWindowProcessor implements FindableProcessor { +public class LengthWindowProcessor extends SlidingFindableWindowProcessor { private int length; - private int count = 0; - private SnapshotableStreamEventQueue expiredEventQueue; public int getLength() { return length; @@ -87,35 +87,35 @@ public void setLength(int length) { } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length == 1) { length = (Integer) ((ConstantExpressionExecutor) attributeExpressionExecutors[0]).getValue(); } else { throw new SiddhiAppValidationException("Length window should only have one parameter ( " + "window.length), but found " + attributeExpressionExecutors.length + " input parameters."); } - expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder, length); + return () -> new WindowState(); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { - synchronized (this) { + StreamEventCloner streamEventCloner, WindowState state) { + synchronized (state) { long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); while (streamEventChunk.hasNext()) { StreamEvent streamEvent = streamEventChunk.next(); StreamEvent clonedEvent = streamEventCloner.copyStreamEvent(streamEvent); clonedEvent.setType(StreamEvent.Type.EXPIRED); - if (count < length) { - count++; - this.expiredEventQueue.add(clonedEvent); + if (state.count < length) { + state.count++; + state.expiredEventQueue.add(clonedEvent); } else { - StreamEvent firstEvent = this.expiredEventQueue.poll(); + StreamEvent firstEvent = state.expiredEventQueue.poll(); if (firstEvent != null) { firstEvent.setTimestamp(currentTime); streamEventChunk.insertBeforeCurrent(firstEvent); - this.expiredEventQueue.add(clonedEvent); + state.expiredEventQueue.add(clonedEvent); } else { StreamEvent resetEvent = streamEventCloner.copyStreamEvent(streamEvent); resetEvent.setType(ComplexEvent.Type.RESET); @@ -137,19 +137,22 @@ protected void process(ComplexEventChunk streamEventChunk, Processo nextProcessor.process(streamEventChunk); } - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, expiredEventQueue, streamEventCloner); - } - @Override public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(expiredEventQueue, condition, matchingMetaInfoHolder, + 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 @@ -161,21 +164,30 @@ public void stop() { } - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { + class WindowState extends State { + + private int count = 0; + private SnapshotableStreamEventQueue expiredEventQueue = + new SnapshotableStreamEventQueue(streamEventClonerHolder, length); + + @Override + public boolean canDestroy() { + return count == 0 && expiredEventQueue.getFirst() == null; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); state.put("Count", count); state.put("ExpiredEventQueue", expiredEventQueue.getSnapshot()); + return state; } - return state; - } - - @Override - public synchronized void restoreState(Map state) { - count = (int) state.get("Count"); - expiredEventQueue.clear(); - expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + @Override + public void restore(Map state) { + count = (int) state.get("Count"); + expiredEventQueue.clear(); + expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LossyFrequentWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LossyFrequentWindowProcessor.java index ad1af1074e..c2ded7a4e4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LossyFrequentWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/LossyFrequentWindowProcessor.java @@ -37,6 +37,8 @@ import io.siddhi.core.util.collection.operator.Operator; import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; import io.siddhi.query.api.expression.Expression; import org.apache.log4j.Logger; @@ -100,21 +102,18 @@ } ) @Deprecated -public class LossyFrequentWindowProcessor extends SlidingWindowProcessor implements FindableProcessor { +public class LossyFrequentWindowProcessor extends + SlidingFindableWindowProcessor { private static final Logger log = Logger.getLogger(LossyFrequentWindowProcessor.class); - private ConcurrentHashMap countMap = new ConcurrentHashMap(); - private ConcurrentHashMap map = new ConcurrentHashMap(); private VariableExpressionExecutor[] variableExpressionExecutors; - private int totalCount = 0; - private double currentBucketId = 1; private double support; // these will be initialize during init private double error; // these will be initialize during init private double windowWidth; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { support = Double.parseDouble(String.valueOf(((ConstantExpressionExecutor) attributeExpressionExecutors[0]) .getValue())); if (attributeExpressionExecutors.length > 1) { @@ -133,14 +132,14 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea } } windowWidth = Math.ceil(1 / error); - currentBucketId = 1; + return () -> new WindowState(); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { + StreamEventCloner streamEventCloner, WindowState state) { - synchronized (this) { + synchronized (state) { long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); StreamEvent streamEvent = streamEventChunk.getFirst(); @@ -152,43 +151,43 @@ protected void process(ComplexEventChunk streamEventChunk, Processo StreamEvent clonedEvent = streamEventCloner.copyStreamEvent(streamEvent); clonedEvent.setType(StreamEvent.Type.EXPIRED); - totalCount++; - if (totalCount != 1) { - currentBucketId = Math.ceil(totalCount / windowWidth); + state.totalCount++; + if (state.totalCount != 1) { + state.currentBucketId = Math.ceil(state.totalCount / windowWidth); } String currentKey = generateKey(streamEvent); - StreamEvent oldEvent = map.put(currentKey, clonedEvent); + StreamEvent oldEvent = state.map.put(currentKey, clonedEvent); if (oldEvent != null) { // this event is already in the store - countMap.put(currentKey, countMap.get(currentKey).incrementCount()); + state.countMap.put(currentKey, state.countMap.get(currentKey).incrementCount()); } else { // This is a new event LossyCount lCount; - lCount = new LossyCount(1, (int) currentBucketId - 1); - countMap.put(currentKey, lCount); + lCount = new LossyCount(1, (int) state.currentBucketId - 1); + state.countMap.put(currentKey, lCount); } // calculating all the events in the system which match the // requirement provided by the user List keys = new ArrayList(); - keys.addAll(countMap.keySet()); + keys.addAll(state.countMap.keySet()); for (String key : keys) { - LossyCount lossyCount = countMap.get(key); - if (lossyCount.getCount() >= ((support - error) * totalCount)) { + LossyCount lossyCount = state.countMap.get(key); + if (lossyCount.getCount() >= ((support - error) * state.totalCount)) { // among the selected events, if the newly arrive event is there we mark it as an inEvent if (key.equals(currentKey)) { streamEventChunk.add(streamEvent); } } } - if (totalCount % windowWidth == 0) { + if (state.totalCount % windowWidth == 0) { // its time to run the data-structure prune code keys = new ArrayList(); - keys.addAll(countMap.keySet()); + keys.addAll(state.countMap.keySet()); for (String key : keys) { - LossyCount lossyCount = countMap.get(key); - if (lossyCount.getCount() + lossyCount.getBucketId() <= currentBucketId) { + LossyCount lossyCount = state.countMap.get(key); + if (lossyCount.getCount() + lossyCount.getBucketId() <= state.currentBucketId) { log.info("Removing the Event: " + key + " from the window"); - countMap.remove(key); - StreamEvent expirtedEvent = map.remove(key); + state.countMap.remove(key); + StreamEvent expirtedEvent = state.map.remove(key); expirtedEvent.setTimestamp(currentTime); streamEventChunk.add(expirtedEvent); } @@ -210,22 +209,6 @@ public void stop() { //Do nothing } - - @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("CountMap", countMap); - } - return state; - } - - - @Override - public synchronized void restoreState(Map state) { - countMap = (ConcurrentHashMap) state.get("CountMap"); - } - private String generateKey(StreamEvent event) { // for performance reason if its all attribute we don't do // the attribute list check StringBuilder stringBuilder = new StringBuilder(); @@ -241,19 +224,21 @@ private String generateKey(StreamEvent event) { // for performance reason i return stringBuilder.toString(); } - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, map.values(), streamEventCloner); - } - @Override public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(map.values(), condition, matchingMetaInfoHolder, + Map tableMap, WindowState state, + SiddhiQueryContext siddhiQueryContext) { + return OperatorParser.constructOperator(state.map.values(), condition, matchingMetaInfoHolder, variableExpressionExecutors, tableMap, siddhiQueryContext); } + @Override + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.map.values(), streamEventCloner); + } + /** * Inner class to keep the lossy count */ @@ -287,4 +272,35 @@ public LossyCount incrementCount() { return this; } } + + class WindowState extends State { + + private int totalCount = 0; + private double currentBucketId = 1; + private ConcurrentHashMap countMap = new ConcurrentHashMap(); + private ConcurrentHashMap map = new ConcurrentHashMap(); + + @Override + public boolean canDestroy() { + return countMap.isEmpty() && map.isEmpty(); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("CountMap", countMap); + state.put("Map", map); + state.put("TotalCount", totalCount); + state.put("CurrentBucketId", currentBucketId); + return state; + } + + @Override + public void restore(Map state) { + countMap = (ConcurrentHashMap) state.get("CountMap"); + countMap = (ConcurrentHashMap) state.get("Map"); + totalCount = (Integer) state.get("TotalCount"); + currentBucketId = (Double) state.get("CurrentBucketId"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SessionWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SessionWindowProcessor.java index 167207b050..8391750a45 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SessionWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SessionWindowProcessor.java @@ -40,11 +40,14 @@ import io.siddhi.core.util.collection.operator.Operator; import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; +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; import io.siddhi.query.api.expression.Expression; import org.apache.log4j.Logger; +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.LinkedHashMap; @@ -52,8 +55,6 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import static java.util.stream.Collectors.toMap; - /** * Implementation of {@link WindowProcessor} which represent a Window operating based on a session. */ @@ -102,7 +103,8 @@ ) } ) -public class SessionWindowProcessor extends GroupingWindowProcessor implements SchedulingProcessor, FindableProcessor { +public class SessionWindowProcessor extends GroupingFindableWindowProcessor + implements SchedulingProcessor { private static final Logger log = Logger.getLogger(SessionWindowProcessor.class); private static final String DEFAULT_KEY = "default-key"; @@ -110,10 +112,6 @@ public class SessionWindowProcessor extends GroupingWindowProcessor implements S private long allowedLatency = 0; private VariableExpressionExecutor sessionKeyExecutor; private Scheduler scheduler; - private Map sessionMap; - private Map sessionKeyEndTimeMap; - private SessionContainer sessionContainer; - private SessionComplexEventChunk expiredEventChunk; @Override public Scheduler getScheduler() { @@ -126,14 +124,9 @@ public void setScheduler(Scheduler scheduler) { } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, boolean outputExpectsExpiredEvents, - SiddhiQueryContext siddhiQueryContext) { - this.sessionMap = new ConcurrentHashMap<>(); - this.sessionKeyEndTimeMap = new HashMap<>(); - this.sessionContainer = new SessionContainer(); - this.expiredEventChunk = new SessionComplexEventChunk<>(); - + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, boolean outputExpectsExpiredEvents, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length >= 1 && attributeExpressionExecutors.length <= 3) { if (attributeExpressionExecutors[0] instanceof ConstantExpressionExecutor) { @@ -209,24 +202,18 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, } - } - - private void validateAllowedLatency(long allowedLatency, long sessionGap) { - if (allowedLatency > sessionGap) { - throw new SiddhiAppValidationException("Session window's allowedLatency parameter value " - + "should not be greater than the session gap parameter value"); - - } + return () -> new WindowState(); } @Override protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner, - GroupingKeyPopulator groupingKeyPopulater) { + StreamEventCloner streamEventCloner, GroupingKeyPopulator groupingKeyPopulater, + WindowState state) { + String key = DEFAULT_KEY; SessionComplexEventChunk currentSession; - synchronized (this) { + synchronized (state) { while (streamEventChunk.hasNext()) { StreamEvent streamEvent = streamEventChunk.next(); long eventTimestamp = streamEvent.getTimestamp(); @@ -246,18 +233,18 @@ protected void processEventChunk(ComplexEventChunk streamEventChunk //get the session configuration based on session key //if the map doesn't contain key, then a new sessionContainer //object needs to be created. - if ((sessionContainer = sessionMap.get(key)) == null) { - sessionContainer = new SessionContainer(key); + if ((state.sessionContainer = state.sessionMap.get(key)) == null) { + state.sessionContainer = new SessionContainer(key); } - sessionMap.put(key, sessionContainer); + state.sessionMap.put(key, state.sessionContainer); StreamEvent clonedStreamEvent = streamEventCloner.copyStreamEvent(streamEvent); clonedStreamEvent.setType(StreamEvent.Type.EXPIRED); - currentSession = sessionContainer.getCurrentSession(); + currentSession = state.sessionContainer.getCurrentSession(); //if current session is empty - if (sessionContainer.getCurrentSession().getFirst() == null) { + if (state.sessionContainer.getCurrentSession().getFirst() == null) { currentSession.add(clonedStreamEvent); currentSession.setTimestamps(eventTimestamp, maxTimestamp, aliveTimestamp); scheduler.notifyAt(maxTimestamp); @@ -272,7 +259,7 @@ protected void processEventChunk(ComplexEventChunk streamEventChunk } else { //when a new session starts if (allowedLatency > 0) { - moveCurrentSessionToPreviousSession(); + moveCurrentSessionToPreviousSession(state); currentSession.clear(); currentSession.setTimestamps(eventTimestamp, maxTimestamp, aliveTimestamp); currentSession.add(clonedStreamEvent); @@ -282,27 +269,36 @@ protected void processEventChunk(ComplexEventChunk streamEventChunk } else { //when a late event arrives - addLateEvent(streamEventChunk, eventTimestamp, clonedStreamEvent); + addLateEvent(streamEventChunk, eventTimestamp, clonedStreamEvent, state); } } } else { - currentSessionTimeout(eventTimestamp); + currentSessionTimeout(eventTimestamp, state); if (allowedLatency > 0) { - previousSessionTimeout(eventTimestamp); + previousSessionTimeout(eventTimestamp, state); } } } } - if (expiredEventChunk != null && expiredEventChunk.getFirst() != null) { - streamEventChunk.add((StreamEvent) expiredEventChunk.getFirst()); - expiredEventChunk.clear(); + if (state.expiredEventChunk != null && state.expiredEventChunk.getFirst() != null) { + streamEventChunk.add((StreamEvent) state.expiredEventChunk.getFirst()); + state.expiredEventChunk.clear(); } nextProcessor.process(streamEventChunk); } + + private void validateAllowedLatency(long allowedLatency, long sessionGap) { + if (allowedLatency > sessionGap) { + throw new SiddhiAppValidationException("Session window's allowedLatency parameter value " + + "should not be greater than the session gap parameter value"); + + } + } + /** * Merge previous window with the next window. */ @@ -324,20 +320,22 @@ private void mergeWindows(SessionComplexEventChunk previousWindow, /** * Moves the events in the current session into previous window. + * + * @param state */ - private void moveCurrentSessionToPreviousSession() { + private void moveCurrentSessionToPreviousSession(WindowState state) { - SessionComplexEventChunk currentSession = sessionContainer.getCurrentSession(); - SessionComplexEventChunk previousSession = sessionContainer.getPreviousSession(); + SessionComplexEventChunk currentSession = state.sessionContainer.getCurrentSession(); + SessionComplexEventChunk previousSession = state.sessionContainer.getPreviousSession(); if (previousSession.getFirst() == null) { previousSession.add(currentSession.getFirst()); } else { - expiredEventChunk.setKey(previousSession.getKey()); - expiredEventChunk.setTimestamps(previousSession.getStartTimestamp(), + state.expiredEventChunk.setKey(previousSession.getKey()); + state.expiredEventChunk.setTimestamps(previousSession.getStartTimestamp(), previousSession.getEndTimestamp(), previousSession.getAliveTimestamp()); - expiredEventChunk.add(previousSession.getFirst()); + state.expiredEventChunk.add(previousSession.getFirst()); previousSession.clear(); previousSession.add(currentSession.getFirst()); @@ -354,10 +352,10 @@ private void moveCurrentSessionToPreviousSession() { * Handles when the late event arrives to the system. */ private void addLateEvent(ComplexEventChunk streamEventChunk, - long eventTimestamp, StreamEvent streamEvent) { + long eventTimestamp, StreamEvent streamEvent, WindowState state) { - SessionComplexEventChunk currentSession = sessionContainer.getCurrentSession(); - SessionComplexEventChunk previousSession = sessionContainer.getPreviousSession(); + SessionComplexEventChunk currentSession = state.sessionContainer.getCurrentSession(); + SessionComplexEventChunk previousSession = state.sessionContainer.getPreviousSession(); if (allowedLatency > 0) { @@ -417,21 +415,26 @@ private void addLateEvent(ComplexEventChunk streamEventChunk, /** * Checks all the sessions and get the expired session. */ - private void currentSessionTimeout(long eventTimestamp) { - Map currentEndTimestamps = findAllCurrentEndTimestamps(sessionMap); + private void currentSessionTimeout(long eventTimestamp, WindowState state) { + Map currentEndTimestamps = findAllCurrentEndTimestamps(state); //sort on endTimestamps if (currentEndTimestamps.size() > 1) { - currentEndTimestamps = currentEndTimestamps.entrySet().stream().sorted(Map.Entry.comparingByValue()) - .collect(toMap(e -> e.getKey(), e -> e.getValue(), (e1, e2) -> e1, - LinkedHashMap::new)); + List> toSort = new ArrayList<>(); + toSort.addAll(currentEndTimestamps.entrySet()); + toSort.sort(Map.Entry.comparingByValue()); + Map map = new LinkedHashMap<>(); + for (Map.Entry e : toSort) { + map.putIfAbsent(e.getKey(), e.getValue()); + } + currentEndTimestamps = map; } for (Map.Entry entry : currentEndTimestamps.entrySet()) { long sessionEndTime = entry.getValue(); - SessionComplexEventChunk currentSession = sessionMap.get(entry.getKey()) + SessionComplexEventChunk currentSession = state.sessionMap.get(entry.getKey()) .getCurrentSession(); - SessionComplexEventChunk previousSession = sessionMap.get(entry.getKey()) + SessionComplexEventChunk previousSession = state.sessionMap.get(entry.getKey()) .getPreviousSession(); if (currentSession.getFirst() != null && eventTimestamp >= sessionEndTime) { @@ -444,14 +447,13 @@ private void currentSessionTimeout(long eventTimestamp) { scheduler.notifyAt(currentSession.getAliveTimestamp()); currentSession.clear(); } else { - expiredEventChunk.setKey(currentSession.getKey()); - expiredEventChunk.setTimestamps(currentSession.getStartTimestamp(), + state.expiredEventChunk.setKey(currentSession.getKey()); + state.expiredEventChunk.setTimestamps(currentSession.getStartTimestamp(), currentSession.getEndTimestamp(), currentSession.getAliveTimestamp()); - expiredEventChunk.add(currentSession.getFirst()); + state.expiredEventChunk.add(currentSession.getFirst()); currentSession.clear(); } - } else { break; } @@ -461,29 +463,36 @@ private void currentSessionTimeout(long eventTimestamp) { /** * Checks all the previous sessions and get the expired sessions. */ - private void previousSessionTimeout(long eventTimestamp) { + private void previousSessionTimeout(long eventTimestamp, WindowState state) { - Map previousEndTimestamps = findAllPreviousEndTimestamps(sessionMap); + Map previousEndTimestamps = findAllPreviousEndTimestamps(state); SessionComplexEventChunk previousSession; //sort on endTimestamps if (previousEndTimestamps.size() > 1) { - previousEndTimestamps = previousEndTimestamps.entrySet().stream().sorted(Map.Entry.comparingByValue()) - .collect(toMap(e -> e.getKey(), e -> e.getValue(), (e1, e2) -> e1, - LinkedHashMap::new)); + List> toSort = new ArrayList<>(); + for (Map.Entry e : previousEndTimestamps.entrySet()) { + toSort.add(e); + } + toSort.sort(Map.Entry.comparingByValue()); + LinkedHashMap map = new LinkedHashMap<>(); + for (Map.Entry e : toSort) { + map.putIfAbsent(e.getKey(), e.getValue()); + } + previousEndTimestamps = map; } for (Map.Entry entry : previousEndTimestamps.entrySet()) { - previousSession = sessionMap.get(entry.getKey()).getPreviousSession(); + previousSession = state.sessionMap.get(entry.getKey()).getPreviousSession(); if (previousSession != null && previousSession.getFirst() != null && eventTimestamp >= previousSession.getAliveTimestamp()) { - expiredEventChunk.setKey(previousSession.getKey()); - expiredEventChunk.setTimestamps(previousSession.getStartTimestamp(), + state.expiredEventChunk.setKey(previousSession.getKey()); + state.expiredEventChunk.setTimestamps(previousSession.getStartTimestamp(), previousSession.getEndTimestamp(), previousSession.getAliveTimestamp()); - expiredEventChunk.add(previousSession.getFirst()); + state.expiredEventChunk.add(previousSession.getFirst()); previousSession.clear(); } else { break; @@ -496,48 +505,51 @@ private void previousSessionTimeout(long eventTimestamp) { /** * Gets all end timestamps of current sessions. * - * @param sessionMap holds all the sessions with the session key + * @param state current state * @return map with the values of each current session's end timestamp and with the key as the session key */ - private Map findAllCurrentEndTimestamps(Map sessionMap) { + private Map findAllCurrentEndTimestamps(WindowState state) { - Collection sessionContainerList = sessionMap.values(); + Collection sessionContainerList = state.sessionMap.values(); - if (!sessionKeyEndTimeMap.isEmpty()) { - sessionKeyEndTimeMap.clear(); + if (!state.sessionKeyEndTimeMap.isEmpty()) { + state.sessionKeyEndTimeMap.clear(); } for (SessionContainer sessionContainer : sessionContainerList) { //not getting empty session details if (sessionContainer.getCurrentSessionEndTimestamp() != -1) { - sessionKeyEndTimeMap.put(sessionContainer.getKey(), sessionContainer.getCurrentSessionEndTimestamp()); + state.sessionKeyEndTimeMap.put(sessionContainer.getKey(), + sessionContainer.getCurrentSessionEndTimestamp()); } } - return sessionKeyEndTimeMap; + return state.sessionKeyEndTimeMap; } /** * Gets all the end timestamps of previous sessions. * + * @param state * @return map with the values of each previous session's end timestamp and with the key as the sesssio key */ - private Map findAllPreviousEndTimestamps(Map sessionMap) { + private Map findAllPreviousEndTimestamps(WindowState state) { - Collection sessionContainerList = sessionMap.values(); + Collection sessionContainerList = state.sessionMap.values(); - if (!sessionKeyEndTimeMap.isEmpty()) { - sessionKeyEndTimeMap.clear(); + if (!state.sessionKeyEndTimeMap.isEmpty()) { + state.sessionKeyEndTimeMap.clear(); } for (SessionContainer sessionContainer : sessionContainerList) { //not getting empty session details if (sessionContainer.getPreviousSessionEndTimestamp() != -1) { - sessionKeyEndTimeMap.put(sessionContainer.getKey(), sessionContainer.getPreviousSessionEndTimestamp()); + state.sessionKeyEndTimeMap.put(sessionContainer.getKey(), + sessionContainer.getPreviousSessionEndTimestamp()); } } - return sessionKeyEndTimeMap; + return state.sessionKeyEndTimeMap; } @Override @@ -547,37 +559,24 @@ public void start() { @Override public void stop() { - //Do nothing - } - - @Override - public synchronized Map currentState() { - Map state = new HashMap<>(); - state.put("sessionMap", sessionMap); - state.put("sessionContainer", sessionContainer); - state.put("expiredEventChunk", expiredEventChunk); - return state; - } - - @Override - public synchronized void restoreState(Map state) { - sessionMap = (ConcurrentHashMap) state.get("sessionMap"); - sessionContainer = (SessionContainer) state.get("sessionContainer"); - expiredEventChunk = (SessionComplexEventChunk) state.get("expiredEventChunk"); - } - - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, expiredEventChunk, streamEventCloner); + if (scheduler != null) { + scheduler.stop(); + } } @Override public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(expiredEventChunk, condition, matchingMetaInfoHolder, + Map tableMap, WindowState state, + SiddhiQueryContext siddhiQueryContext) { + return OperatorParser.constructOperator(state.expiredEventChunk, condition, matchingMetaInfoHolder, variableExpressionExecutors, tableMap, siddhiQueryContext); + } + @Override + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.expiredEventChunk, streamEventCloner); } /** @@ -639,4 +638,42 @@ public void setTimestamps(long startTimestamp, long endTimestamp, long aliveTime this.aliveTimestamp = aliveTimestamp; } } + + class WindowState extends State { + + private Map sessionMap; + private Map sessionKeyEndTimeMap; + private SessionContainer sessionContainer; + private SessionComplexEventChunk expiredEventChunk; + + public WindowState() { + this.sessionMap = new ConcurrentHashMap<>(); + this.sessionKeyEndTimeMap = new HashMap<>(); + this.sessionContainer = new SessionContainer(); + this.expiredEventChunk = new SessionComplexEventChunk<>(); + } + + @Override + public boolean canDestroy() { + return sessionMap.isEmpty() && expiredEventChunk.getFirst() == null + && sessionContainer.getCurrentSession().getFirst() == null + && sessionContainer.getPreviousSession().getFirst() == null; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("sessionMap", sessionMap); + state.put("sessionContainer", sessionContainer); + state.put("expiredEventChunk", expiredEventChunk); + return state; + } + + @Override + public void restore(Map state) { + sessionMap = (ConcurrentHashMap) state.get("sessionMap"); + sessionContainer = (SessionContainer) state.get("sessionContainer"); + expiredEventChunk = (SessionComplexEventChunk) state.get("expiredEventChunk"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SlidingFindableWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SlidingFindableWindowProcessor.java new file mode 100644 index 0000000000..c7a48b9d00 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SlidingFindableWindowProcessor.java @@ -0,0 +1,97 @@ +/* + * 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.core.config.SiddhiQueryContext; +import io.siddhi.core.event.state.StateEvent; +import io.siddhi.core.event.stream.StreamEvent; +import io.siddhi.core.event.stream.StreamEventCloner; +import io.siddhi.core.executor.VariableExpressionExecutor; +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.snapshot.state.State; +import io.siddhi.query.api.expression.Expression; + +import java.util.List; +import java.util.Map; + +/** + * Performs event processing in a sliding manner while supporting event search + * + * @param current state of the processor + */ +public abstract class SlidingFindableWindowProcessor extends SlidingWindowProcessor + implements FindableProcessor { + + @Override + public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, + List variableExpressionExecutors, + Map tableMap, SiddhiQueryContext siddhiQueryContext) { + S state = stateHolder.getState(); + try { + return compileCondition(condition, matchingMetaInfoHolder, variableExpressionExecutors, tableMap, + state, siddhiQueryContext); + } finally { + stateHolder.returnState(state); + } + } + + @Override + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { + S state = stateHolder.getState(); + try { + return find(matchingEvent, compiledCondition, streamEventClonerHolder.getStreamEventCloner(), + state); + } finally { + stateHolder.returnState(state); + } + + } + + /** + * To construct a finder having the capability of finding events at the processor that corresponds to the incoming + * matchingEvent and the given matching expression logic. + * + * @param condition the matching condition + * @param matchingMetaInfoHolder the meta structure of the incoming matchingEvent + * @param variableExpressionExecutors the list of variable ExpressionExecutors already created + * @param tableMap map of event tables + * @param state current query state + * @param siddhiQueryContext current siddhi query context + * @return compiled Condition having the capability of matching events against the incoming matchingEvent + */ + public abstract CompiledCondition compileCondition(Expression condition, + MatchingMetaInfoHolder matchingMetaInfoHolder, + List variableExpressionExecutors, + Map tableMap, S state, + SiddhiQueryContext siddhiQueryContext); + + /** + * To find events from the processor event pool, that the matches the matchingEvent based on finder logic. + * + * @param matchingEvent the event to be matched with the events at the processor + * @param compiledCondition the execution element responsible for matching the corresponding events that matches + * the matchingEvent based on pool of events at Processor + * @param streamEventCloner stream event cloner + * @param state current query state @return the matched events + */ + public abstract StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, S state); +} 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 ae6fb7def9..9bf1a24bb8 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 @@ -23,30 +23,31 @@ 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.StreamEventClonerHolder; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; import io.siddhi.core.query.processor.Processor; 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.AbstractDefinition; -import io.siddhi.query.api.definition.Attribute; - -import java.util.ArrayList; -import java.util.List; /** * Performs event processing in a sliding manner + * + * @param current state of the processor */ -public abstract class SlidingWindowProcessor extends WindowProcessor { +public abstract class SlidingWindowProcessor extends WindowProcessor { @Override - protected List init(MetaStreamEvent metaStreamEvent, + protected StateFactory init(MetaStreamEvent metaStreamEvent, AbstractDefinition inputDefinition, ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { - init(attributeExpressionExecutors, configReader, siddhiQueryContext); - return new ArrayList(0); + ConfigReader configReader, StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { + return init(attributeExpressionExecutors, configReader, siddhiQueryContext); } /** @@ -56,14 +57,16 @@ protected List init(MetaStreamEvent metaStreamEvent, * @param configReader the config reader of window * @param siddhiQueryContext the context of the siddhi query */ - protected abstract void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext); + protected abstract StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext); @Override protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater) { + StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater, + S state) { streamEventChunk.reset(); - process(streamEventChunk, nextProcessor, streamEventCloner); + process(streamEventChunk, nextProcessor, streamEventCloner, state); } /** @@ -72,9 +75,10 @@ protected void processEventChunk(ComplexEventChunk streamEventChunk * @param streamEventChunk the stream event chunk that need to be processed * @param nextProcessor the next processor to which the success events need to be passed * @param streamEventCloner helps to clone the incoming event for local storage or modification + * @param state current query state */ protected abstract void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner); + StreamEventCloner streamEventCloner, S state); @Override public ProcessingMode getProcessingMode() { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SortWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SortWindowProcessor.java index 252b1e0349..1b689d42e5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SortWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/SortWindowProcessor.java @@ -37,6 +37,8 @@ import io.siddhi.core.util.collection.operator.Operator; import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; +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.expression.Expression; @@ -92,17 +94,16 @@ "Therefore, at any given time, the window contains the 5 lowest prices." ) ) -public class SortWindowProcessor extends SlidingWindowProcessor implements FindableProcessor { +public class SortWindowProcessor extends SlidingFindableWindowProcessor { private static final String ASC = "asc"; private static final String DESC = "desc"; private int lengthToKeep; - private List sortedWindow = new ArrayList(); private List parameterInfo; private EventComparator eventComparator; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors[0].getReturnType() == Attribute.Type.INT) { lengthToKeep = Integer.parseInt(String.valueOf(((ConstantExpressionExecutor) attributeExpressionExecutors[0]).getValue())); @@ -138,14 +139,15 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea parameterInfo.add(new Object[]{variableExpressionExecutor, order}); } } - + return () -> new WindowState(); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { + StreamEventCloner streamEventCloner, WindowState state) { + - synchronized (this) { + synchronized (state) { long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); StreamEvent streamEvent = streamEventChunk.getFirst(); @@ -158,10 +160,10 @@ protected void process(ComplexEventChunk streamEventChunk, Processo streamEvent.setNext(null); streamEventChunk.add(streamEvent); - sortedWindow.add(clonedEvent); - if (sortedWindow.size() > lengthToKeep) { - Collections.sort(sortedWindow, eventComparator); - StreamEvent expiredEvent = sortedWindow.remove(sortedWindow.size() - 1); + state.sortedWindow.add(clonedEvent); + if (state.sortedWindow.size() > lengthToKeep) { + Collections.sort(state.sortedWindow, eventComparator); + StreamEvent expiredEvent = state.sortedWindow.remove(state.sortedWindow.size() - 1); expiredEvent.setTimestamp(currentTime); streamEventChunk.add(expiredEvent); } @@ -183,31 +185,19 @@ public void stop() { } @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("SortedWindow", sortedWindow); - } - return state; + public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, + List variableExpressionExecutors, + Map tableMap, WindowState state, + SiddhiQueryContext siddhiQueryContext) { + return OperatorParser.constructOperator(state.sortedWindow, condition, matchingMetaInfoHolder, + variableExpressionExecutors, tableMap, siddhiQueryContext); } - @Override - public synchronized void restoreState(Map state) { - sortedWindow = (List) state.get("SortedWindow"); - } + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.sortedWindow, streamEventCloner); - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, sortedWindow, streamEventCloner); - } - - @Override - public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, - List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(sortedWindow, condition, matchingMetaInfoHolder, - variableExpressionExecutors, tableMap, siddhiQueryContext); } private class EventComparator implements Comparator { @@ -227,4 +217,24 @@ public int compare(StreamEvent e1, StreamEvent e2) { } } + class WindowState extends State { + private List sortedWindow = new ArrayList(); + + @Override + public boolean canDestroy() { + return sortedWindow.isEmpty(); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("SortedWindow", sortedWindow); + return state; + } + + @Override + public void restore(Map state) { + sortedWindow = (List) state.get("SortedWindow"); + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TableWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TableWindowProcessor.java index 2abb9327ae..571362a31b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TableWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TableWindowProcessor.java @@ -22,7 +22,7 @@ import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.exception.SiddhiAppRuntimeException; +import io.siddhi.core.event.stream.holder.StreamEventClonerHolder; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; import io.siddhi.core.query.processor.Processor; @@ -30,6 +30,8 @@ import io.siddhi.core.util.collection.operator.CompiledCondition; import io.siddhi.core.util.collection.operator.MatchingMetaInfoHolder; 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.expression.Expression; import java.util.List; @@ -41,8 +43,6 @@ public class TableWindowProcessor extends BatchingWindowProcessor implements FindableProcessor { private Table table; - private boolean outputExpectsExpiredEvents; - private ConfigReader configReader; public TableWindowProcessor(Table table) { this.table = table; @@ -50,18 +50,19 @@ public TableWindowProcessor(Table table) { @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { - this.configReader = configReader; - this.outputExpectsExpiredEvents = outputExpectsExpiredEvents; + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, boolean outputExpectsExpiredEvents, + boolean findToBeExecuted, SiddhiQueryContext siddhiQueryContext) { + return null; } @Override - protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { + protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, + StreamEventCloner streamEventCloner, State state) { // nothing to be done } + @Override public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { return table.find(matchingEvent, compiledCondition); @@ -84,39 +85,4 @@ public void start() { public void stop() { //Do nothing } - - @Override - public Processor cloneProcessor(String key) { - try { - TableWindowProcessor streamProcessor = new TableWindowProcessor(table); - streamProcessor.inputDefinition = inputDefinition; - ExpressionExecutor[] innerExpressionExecutors = new ExpressionExecutor[attributeExpressionLength]; - ExpressionExecutor[] attributeExpressionExecutors1 = this.attributeExpressionExecutors; - for (int i = 0; i < attributeExpressionLength; i++) { - innerExpressionExecutors[i] = attributeExpressionExecutors1[i].cloneExecutor(key); - } - streamProcessor.attributeExpressionExecutors = innerExpressionExecutors; - streamProcessor.attributeExpressionLength = attributeExpressionLength; - streamProcessor.additionalAttributes = additionalAttributes; - streamProcessor.complexEventPopulater = complexEventPopulater; - streamProcessor.init(metaStreamEvent, inputDefinition, attributeExpressionExecutors, configReader, - outputExpectsExpiredEvents, siddhiQueryContext); - streamProcessor.start(); - return streamProcessor; - - } catch (Exception e) { - throw new SiddhiAppRuntimeException("Exception in cloning " + this.getClass().getCanonicalName(), e); - } - } - - @Override - public Map currentState() { - //No state - return null; - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeBatchWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeBatchWindowProcessor.java index b2c1a9491c..963662b76c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeBatchWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeBatchWindowProcessor.java @@ -28,6 +28,7 @@ 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.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -41,6 +42,8 @@ import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; 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.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; import io.siddhi.query.api.expression.Expression; @@ -110,14 +113,11 @@ ) } ) -public class TimeBatchWindowProcessor extends BatchingWindowProcessor +public class TimeBatchWindowProcessor extends BatchingFindableWindowProcessor implements SchedulingProcessor, FindableProcessor { private long timeInMilliSeconds; private long nextEmitTime = -1; - private SnapshotableStreamEventQueue currentEventQueue = null; - private SnapshotableStreamEventQueue expiredEventQueue = null; - private StreamEvent resetEvent = null; private Scheduler scheduler; private boolean outputExpectsExpiredEvents; private SiddhiQueryContext siddhiQueryContext; @@ -140,16 +140,12 @@ public void setScheduler(Scheduler scheduler) { } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, boolean - outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + StreamEventClonerHolder streamEventClonerHolder, boolean + outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { this.outputExpectsExpiredEvents = outputExpectsExpiredEvents; this.siddhiQueryContext = siddhiQueryContext; - if (!isStreamCurrentEvents) { - this.currentEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); - } - if (outputExpectsExpiredEvents) { - this.expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); - } if (attributeExpressionExecutors.length == 1) { initTimeParameter(attributeExpressionExecutors[0]); } else if (attributeExpressionExecutors.length == 2) { @@ -221,6 +217,8 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea attributeExpressionExecutors.length + " input " + "attributes"); } + return () -> new WindowState(streamEventClonerHolder, outputExpectsExpiredEvents, findToBeExecuted); + } private void initTimeParameter(ExpressionExecutor attributeExpressionExecutor) { @@ -243,11 +241,10 @@ private void initTimeParameter(ExpressionExecutor attributeExpressionExecutor) { } } - @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { - synchronized (this) { + StreamEventCloner streamEventCloner, WindowState state) { + synchronized (state) { if (nextEmitTime == -1) { long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); if (isStartTimeEnabled) { @@ -275,47 +272,47 @@ protected void process(ComplexEventChunk streamEventChunk, Processo continue; } StreamEvent clonedStreamEvent = streamEventCloner.copyStreamEvent(streamEvent); - if (resetEvent == null) { - resetEvent = streamEventCloner.copyStreamEvent(streamEvent); - resetEvent.setType(ComplexEvent.Type.RESET); + if (state.resetEvent == null) { + state.resetEvent = streamEventCloner.copyStreamEvent(streamEvent); + state.resetEvent.setType(ComplexEvent.Type.RESET); } if (!isStreamCurrentEvents) { - currentEventQueue.add(clonedStreamEvent); - } else if (expiredEventQueue != null) { + state.currentEventQueue.add(clonedStreamEvent); + } else if (state.expiredEventQueue != null) { clonedStreamEvent.setType(StreamEvent.Type.EXPIRED); - expiredEventQueue.add(clonedStreamEvent); + state.expiredEventQueue.add(clonedStreamEvent); } } if (!isStreamCurrentEvents) { streamEventChunk.clear(); } if (sendEvents) { - if (outputExpectsExpiredEvents && expiredEventQueue.getFirst() != null) { - while (expiredEventQueue.hasNext()) { - StreamEvent expiredEvent = expiredEventQueue.next(); + if (outputExpectsExpiredEvents && state.expiredEventQueue.getFirst() != null) { + while (state.expiredEventQueue.hasNext()) { + StreamEvent expiredEvent = state.expiredEventQueue.next(); expiredEvent.setTimestamp(currentTime); } - streamEventChunk.add(expiredEventQueue.getFirst()); - expiredEventQueue.clear(); + streamEventChunk.add(state.expiredEventQueue.getFirst()); + state.expiredEventQueue.clear(); } - if (resetEvent != null) { - streamEventChunk.add(resetEvent); - resetEvent = null; + if (state.resetEvent != null) { + streamEventChunk.add(state.resetEvent); + state.resetEvent = null; } - if (currentEventQueue != null && currentEventQueue.getFirst() != null) { - if (expiredEventQueue != null) { - currentEventQueue.reset(); - while (currentEventQueue.hasNext()) { - StreamEvent currentEvent = currentEventQueue.next(); + if (state.currentEventQueue != null && state.currentEventQueue.getFirst() != null) { + if (state.expiredEventQueue != null) { + state.currentEventQueue.reset(); + while (state.currentEventQueue.hasNext()) { + StreamEvent currentEvent = state.currentEventQueue.next(); StreamEvent toExpireEvent = streamEventCloner.copyStreamEvent(currentEvent); toExpireEvent.setType(StreamEvent.Type.EXPIRED); - expiredEventQueue.add(toExpireEvent); + state.expiredEventQueue.add(toExpireEvent); } } - streamEventChunk.add(currentEventQueue.getFirst()); - currentEventQueue.clear(); + streamEventChunk.add(state.currentEventQueue.getFirst()); + state.currentEventQueue.clear(); } } } @@ -341,45 +338,66 @@ public void start() { @Override public void stop() { - //Do nothing + if (scheduler != null) { + scheduler.stop(); + } } @Override - public Map currentState() { - Map state = new HashMap<>(); - synchronized (this) { - state.put("CurrentEventQueue", currentEventQueue != null ? currentEventQueue.getSnapshot() : null); - state.put("ExpiredEventQueue", expiredEventQueue != null ? expiredEventQueue.getSnapshot() : null); - state.put("ResetEvent", resetEvent); - } - return state; + 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 synchronized void restoreState(Map state) { - if (expiredEventQueue != null) { - expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.expiredEventQueue, streamEventCloner); + } + + class WindowState extends State { + private SnapshotableStreamEventQueue currentEventQueue; + private SnapshotableStreamEventQueue expiredEventQueue; + private StreamEvent resetEvent = null; + + WindowState(StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted) { + if (!isStreamCurrentEvents) { + this.currentEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + } + if (outputExpectsExpiredEvents || findToBeExecuted) { + this.expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + } } - if (currentEventQueue != null) { - currentEventQueue.restore((SnapshotStateList) state.get("CurrentEventQueue")); + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("CurrentEventQueue", currentEventQueue != null ? currentEventQueue.getSnapshot() : null); + state.put("ExpiredEventQueue", expiredEventQueue != null ? expiredEventQueue.getSnapshot() : null); + state.put("ResetEvent", resetEvent); + return state; } - resetEvent = (StreamEvent) state.get("ResetEvent"); - } - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, expiredEventQueue, streamEventCloner); - } + public void restore(Map state) { + if (expiredEventQueue != null) { + expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + } + if (currentEventQueue != null) { + currentEventQueue.restore((SnapshotStateList) state.get("CurrentEventQueue")); + } + resetEvent = (StreamEvent) state.get("ResetEvent"); + } - @Override - public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, - List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - if (expiredEventQueue == null) { - expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + @Override + public boolean canDestroy() { + return (currentEventQueue == null || currentEventQueue.getFirst() == null) + && (expiredEventQueue == null || expiredEventQueue.getFirst() == null) + && resetEvent == null; } - return OperatorParser.constructOperator(expiredEventQueue, condition, matchingMetaInfoHolder, - variableExpressionExecutors, tableMap, - siddhiQueryContext); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeLengthWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeLengthWindowProcessor.java index ce56f18a43..bc5b438be2 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeLengthWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeLengthWindowProcessor.java @@ -27,6 +27,8 @@ import io.siddhi.core.event.state.StateEvent; 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.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -39,6 +41,9 @@ import io.siddhi.core.util.collection.operator.Operator; import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; +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.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; import io.siddhi.query.api.expression.Expression; @@ -77,13 +82,11 @@ "last 2 seconds and gets updated for every event arrival and expiry." ) ) -public class TimeLengthWindowProcessor extends SlidingWindowProcessor - implements SchedulingProcessor, FindableProcessor { +public class TimeLengthWindowProcessor extends SlidingFindableWindowProcessor + implements SchedulingProcessor { private long timeInMilliSeconds; private int length; - private int count = 0; - private ComplexEventChunk expiredEventChunk; private Scheduler scheduler; private SiddhiQueryContext siddhiQueryContext; @@ -98,10 +101,9 @@ public void setScheduler(Scheduler scheduler) { } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { this.siddhiQueryContext = siddhiQueryContext; - expiredEventChunk = new ComplexEventChunk(false); if (attributeExpressionExecutors.length == 2) { length = (Integer) ((ConstantExpressionExecutor) attributeExpressionExecutors[1]).getValue(); if (attributeExpressionExecutors[0] instanceof ConstantExpressionExecutor) { @@ -126,24 +128,27 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea "windowTime, windowLength), but found " + attributeExpressionExecutors.length + " input " + "attributes"); } + return () -> new WindowState(streamEventClonerHolder); + } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { - synchronized (this) { + StreamEventCloner streamEventCloner, WindowState state) { + + synchronized (state) { long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); while (streamEventChunk.hasNext()) { StreamEvent streamEvent = streamEventChunk.next(); - expiredEventChunk.reset(); - while (expiredEventChunk.hasNext()) { - StreamEvent expiredEvent = expiredEventChunk.next(); + state.expiredEventQueue.reset(); + while (state.expiredEventQueue.hasNext()) { + StreamEvent expiredEvent = state.expiredEventQueue.next(); long timeDiff = expiredEvent.getTimestamp() - currentTime + timeInMilliSeconds; if (timeDiff <= 0) { - expiredEventChunk.remove(); - count--; + state.expiredEventQueue.remove(); + state.count--; expiredEvent.setTimestamp(currentTime); streamEventChunk.insertBeforeCurrent(expiredEvent); } else { @@ -151,20 +156,20 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } } - expiredEventChunk.reset(); + state.expiredEventQueue.reset(); if (streamEvent.getType() == StreamEvent.Type.CURRENT) { StreamEvent clonedEvent = streamEventCloner.copyStreamEvent(streamEvent); clonedEvent.setType(StreamEvent.Type.EXPIRED); - if (count < length) { - count++; - this.expiredEventChunk.add(clonedEvent); + if (state.count < length) { + state.count++; + state.expiredEventQueue.add(clonedEvent); } else { - StreamEvent firstEvent = this.expiredEventChunk.poll(); + StreamEvent firstEvent = state.expiredEventQueue.poll(); if (firstEvent != null) { firstEvent.setTimestamp(currentTime); streamEventChunk.insertBeforeCurrent(firstEvent); - this.expiredEventChunk.add(clonedEvent); + state.expiredEventQueue.add(clonedEvent); } } scheduler.notifyAt(clonedEvent.getTimestamp() + timeInMilliSeconds); @@ -179,42 +184,58 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } } - - @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, expiredEventChunk, streamEventCloner); - } - @Override public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(expiredEventChunk, condition, matchingMetaInfoHolder, + Map tableMap, WindowState state, + SiddhiQueryContext siddhiQueryContext) { + return OperatorParser.constructOperator(state.expiredEventQueue, condition, matchingMetaInfoHolder, variableExpressionExecutors, tableMap, siddhiQueryContext); } @Override - public void start() { - //Do nothing + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.expiredEventQueue, streamEventCloner); } @Override - public void stop() { + public void start() { //Do nothing } @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("ExpiredEventChunk", expiredEventChunk.getFirst()); - return state; + public void stop() { + if (scheduler != null) { + scheduler.stop(); + } } - @Override - public void restoreState(Map state) { - expiredEventChunk.clear(); - expiredEventChunk.add((StreamEvent) state.get("ExpiredEventChunk")); - } + class WindowState extends State { + private SnapshotableStreamEventQueue expiredEventQueue; + private int count = 0; + WindowState(StreamEventClonerHolder streamEventClonerHolder) { + expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("ExpiredEventQueue", expiredEventQueue.getSnapshot()); + state.put("Count", count); + return state; + } + + public void restore(Map state) { + expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + count = (Integer) state.get("ExpiredEventQueue"); + } + + @Override + public boolean canDestroy() { + return expiredEventQueue.getFirst() == null && count == 0; + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeWindowProcessor.java index 198025d841..fe18749c7e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/TimeWindowProcessor.java @@ -27,6 +27,7 @@ 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.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -40,6 +41,8 @@ import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.OperatorParser; 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.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; import io.siddhi.query.api.expression.Expression; @@ -76,17 +79,12 @@ ) } ) -public class TimeWindowProcessor extends SlidingWindowProcessor implements SchedulingProcessor, FindableProcessor { +public class TimeWindowProcessor extends SlidingFindableWindowProcessor + implements SchedulingProcessor { private long timeInMilliSeconds; - private SnapshotableStreamEventQueue expiredEventQueue; private Scheduler scheduler; private SiddhiQueryContext siddhiQueryContext; - private volatile long lastTimestamp = Long.MIN_VALUE; - - public void setTimeInMilliSeconds(long timeInMilliSeconds) { - this.timeInMilliSeconds = timeInMilliSeconds; - } @Override public Scheduler getScheduler() { @@ -99,10 +97,10 @@ public void setScheduler(Scheduler scheduler) { } @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { this.siddhiQueryContext = siddhiQueryContext; - this.expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + if (attributeExpressionExecutors.length == 1) { if (attributeExpressionExecutors[0] instanceof ConstantExpressionExecutor) { if (attributeExpressionExecutors[0].getReturnType() == Attribute.Type.INT) { @@ -124,13 +122,14 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea throw new SiddhiAppValidationException("Time window should only have one parameter ( " + "windowTime), but found " + attributeExpressionExecutors.length + " input attributes"); } + return () -> new WindowState(streamEventClonerHolder); } @Override protected void process(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner) { - synchronized (this) { - + StreamEventCloner streamEventCloner, WindowState state) { + synchronized (state) { + SnapshotableStreamEventQueue expiredEventQueue = state.expiredEventQueue; while (streamEventChunk.hasNext()) { StreamEvent streamEvent = streamEventChunk.next(); long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); @@ -149,14 +148,12 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } if (streamEvent.getType() == StreamEvent.Type.CURRENT) { - StreamEvent clonedEvent = streamEventCloner.copyStreamEvent(streamEvent); clonedEvent.setType(StreamEvent.Type.EXPIRED); - this.expiredEventQueue.add(clonedEvent); - - if (lastTimestamp < clonedEvent.getTimestamp()) { + expiredEventQueue.add(clonedEvent); + if (state.lastTimestamp < clonedEvent.getTimestamp()) { scheduler.notifyAt(clonedEvent.getTimestamp() + timeInMilliSeconds); - lastTimestamp = clonedEvent.getTimestamp(); + state.lastTimestamp = clonedEvent.getTimestamp(); } } else { streamEventChunk.remove(); @@ -168,16 +165,18 @@ protected void process(ComplexEventChunk streamEventChunk, Processo } @Override - public synchronized StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { - return ((Operator) compiledCondition).find(matchingEvent, expiredEventQueue, streamEventCloner); + 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 CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, - List variableExpressionExecutors, - Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(expiredEventQueue, condition, matchingMetaInfoHolder, - variableExpressionExecutors, tableMap, siddhiQueryContext); + public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition, + StreamEventCloner streamEventCloner, WindowState state) { + return ((Operator) compiledCondition).find(matchingEvent, state.expiredEventQueue, streamEventCloner); } @Override @@ -190,15 +189,31 @@ public void stop() { //Do nothing } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("ExpiredEventQueue", expiredEventQueue.getSnapshot()); - return state; - } + class WindowState extends State { + protected SnapshotableStreamEventQueue expiredEventQueue; + protected volatile long lastTimestamp = Long.MIN_VALUE; - @Override - public void restoreState(Map state) { - expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + + WindowState(StreamEventClonerHolder streamEventClonerHolder) { + expiredEventQueue = new SnapshotableStreamEventQueue(streamEventClonerHolder); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("ExpiredEventQueue", expiredEventQueue.getSnapshot()); + state.put("LastTimestamp", lastTimestamp); + return state; + } + + public void restore(Map state) { + expiredEventQueue.restore((SnapshotStateList) state.get("ExpiredEventQueue")); + lastTimestamp = (long) state.get("LastTimestamp"); + } + + @Override + public boolean canDestroy() { + return expiredEventQueue.getFirst() == null; + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/WindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/WindowProcessor.java index 0b1c9903f3..ee1fb1601f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/WindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/WindowProcessor.java @@ -18,11 +18,23 @@ package io.siddhi.core.query.processor.stream.window; import io.siddhi.core.query.processor.stream.AbstractStreamProcessor; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.query.api.definition.Attribute; + +import java.util.ArrayList; +import java.util.List; /** * Abstract parent implementation of Processor to represent Windows. Common window behaviors will be handled * here through this class and different implementations should extend this + * + * @param current state of the processor */ -public abstract class WindowProcessor extends AbstractStreamProcessor { +public abstract class WindowProcessor extends AbstractStreamProcessor { + + @Override + public List getReturnAttributes() { + return new ArrayList<>(0); + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/WindowWindowProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/WindowWindowProcessor.java index a46d9c7f28..4ddf20208c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/WindowWindowProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/processor/stream/window/WindowWindowProcessor.java @@ -22,8 +22,8 @@ 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.StreamEventClonerHolder; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; -import io.siddhi.core.exception.SiddhiAppRuntimeException; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; import io.siddhi.core.query.input.stream.join.JoinProcessor; @@ -33,12 +33,12 @@ import io.siddhi.core.util.collection.operator.CompiledCondition; import io.siddhi.core.util.collection.operator.MatchingMetaInfoHolder; 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.core.window.Window; import io.siddhi.query.api.definition.AbstractDefinition; -import io.siddhi.query.api.definition.Attribute; import io.siddhi.query.api.expression.Expression; -import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -56,31 +56,30 @@ public class WindowWindowProcessor extends WindowProcessor implements FindablePr * {@link Window} from which the events have to be found. */ private Window window; - private ConfigReader configReader; - private boolean outputExpectsExpiredEvents; public WindowWindowProcessor(Window window) { this.window = window; } @Override - protected List init(MetaStreamEvent metaStreamEvent, - AbstractDefinition inputDefinition, - ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { - this.configReader = configReader; - this.outputExpectsExpiredEvents = outputExpectsExpiredEvents; - return new ArrayList(0); + protected StateFactory init(MetaStreamEvent metaStreamEvent, + AbstractDefinition inputDefinition, + ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, StreamEventClonerHolder streamEventClonerHolder, + boolean outputExpectsExpiredEvents, boolean findToBeExecuted, + SiddhiQueryContext siddhiQueryContext) { + return null; } @Override - protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, - StreamEventCloner streamEventCloner, ComplexEventPopulater complexEventPopulater) { + protected void processEventChunk(ComplexEventChunk streamEventChunk, Processor nextProcessor, + StreamEventCloner streamEventCloner, + ComplexEventPopulater complexEventPopulater, State state) { streamEventChunk.reset(); nextProcessor.process(streamEventChunk); } + @Override public StreamEvent find(StateEvent matchingEvent, CompiledCondition compiledCondition) { return window.find(matchingEvent, compiledCondition); @@ -104,43 +103,9 @@ public void stop() { //Do nothing } - @Override - public Processor cloneProcessor(String key) { - try { - WindowWindowProcessor streamProcessor = new WindowWindowProcessor(window); - streamProcessor.inputDefinition = inputDefinition; - ExpressionExecutor[] innerExpressionExecutors = new ExpressionExecutor[attributeExpressionLength]; - ExpressionExecutor[] attributeExpressionExecutors1 = this.attributeExpressionExecutors; - for (int i = 0; i < attributeExpressionLength; i++) { - innerExpressionExecutors[i] = attributeExpressionExecutors1[i].cloneExecutor(key); - } - streamProcessor.attributeExpressionExecutors = innerExpressionExecutors; - streamProcessor.attributeExpressionLength = attributeExpressionLength; - streamProcessor.additionalAttributes = additionalAttributes; - streamProcessor.complexEventPopulater = complexEventPopulater; - streamProcessor.init(metaStreamEvent, inputDefinition, attributeExpressionExecutors, configReader, - outputExpectsExpiredEvents, siddhiQueryContext); - streamProcessor.start(); - return streamProcessor; - - } catch (Exception e) { - throw new SiddhiAppRuntimeException("Exception in cloning " + this.getClass().getCanonicalName(), e); - } - } - @Override public ProcessingMode getProcessingMode() { return window.getProcessingMode(); } - @Override - public Map currentState() { - //No state - return null; - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/GroupByKeyGenerator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/GroupByKeyGenerator.java index 01141a1864..69c822ef57 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/GroupByKeyGenerator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/GroupByKeyGenerator.java @@ -71,10 +71,4 @@ public String constructEventKey(ComplexEvent event) { return null; } } - - public void clean() { - for (ExpressionExecutor expressionExecutor : groupByExecutors) { - expressionExecutor.clean(); - } - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/QuerySelector.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/QuerySelector.java index 147eb94b7e..91f926747f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/QuerySelector.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/QuerySelector.java @@ -17,6 +17,7 @@ */ package io.siddhi.core.query.selector; +import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; @@ -28,7 +29,6 @@ import io.siddhi.core.query.output.ratelimit.OutputRateLimiter; import io.siddhi.core.query.processor.Processor; import io.siddhi.core.query.selector.attribute.processor.AttributeProcessor; -import io.siddhi.core.query.selector.attribute.processor.executor.GroupByAggregationAttributeExecutor; import io.siddhi.core.util.SiddhiConstants; import io.siddhi.query.api.execution.query.selection.Selector; import org.apache.log4j.Logger; @@ -74,7 +74,6 @@ public QuerySelector(String id, Selector selector, boolean currentOn, boolean ex @Override public void process(ComplexEventChunk complexEventChunk) { - if (log.isTraceEnabled()) { log.trace("event is processed by selector " + id + this); } @@ -97,11 +96,9 @@ public void process(ComplexEventChunk complexEventChunk) { if (outputComplexEventChunk != null) { outputRateLimiter.process(outputComplexEventChunk); } - } public ComplexEventChunk execute(ComplexEventChunk complexEventChunk) { - if (log.isTraceEnabled()) { log.trace("event is executed by selector " + id + this); } @@ -178,31 +175,32 @@ private ComplexEventChunk processGroupBy(ComplexEventChunk complex while (complexEventChunk.hasNext()) { ComplexEvent event = complexEventChunk.next(); switch (event.getType()) { - case CURRENT: case EXPIRED: eventPopulator.populateStateEvent(event); - String groupedByKey = groupByKeyGenerator.constructEventKey(event); - GroupByAggregationAttributeExecutor.getKeyThreadLocal().set(groupedByKey); - - for (AttributeProcessor attributeProcessor : attributeProcessorList) { - attributeProcessor.process(event); - } - if ((event.getType() == StreamEvent.Type.CURRENT && currentOn) || (event.getType() == - StreamEvent.Type.EXPIRED && expiredOn)) { - if (!(havingConditionExecutor != null && !havingConditionExecutor.execute(event))) { - complexEventChunk.remove(); - if (limit == SiddhiConstants.UNKNOWN_STATE) { - currentComplexEventChunk.add(new GroupedComplexEvent(groupedByKey, event)); - } else { - if (limitCount < limit) { - currentComplexEventChunk.add(new GroupedComplexEvent(groupedByKey, event)); - limitCount++; + String groupByKey = groupByKeyGenerator.constructEventKey(event); + SiddhiAppContext.startGroupByFlow(groupByKey); + try { + for (AttributeProcessor attributeProcessor : attributeProcessorList) { + attributeProcessor.process(event); + } + if ((event.getType() == StreamEvent.Type.CURRENT && currentOn) || (event.getType() == + StreamEvent.Type.EXPIRED && expiredOn)) { + if (!(havingConditionExecutor != null && !havingConditionExecutor.execute(event))) { + complexEventChunk.remove(); + if (limit == SiddhiConstants.UNKNOWN_STATE) { + currentComplexEventChunk.add(new GroupedComplexEvent(groupByKey, event)); + } else { + if (limitCount < limit) { + currentComplexEventChunk.add(new GroupedComplexEvent(groupByKey, event)); + limitCount++; + } } } } + } finally { + SiddhiAppContext.stopGroupByFlow(); } - GroupByAggregationAttributeExecutor.getKeyThreadLocal().remove(); break; case TIMER: break; @@ -212,6 +210,7 @@ private ComplexEventChunk processGroupBy(ComplexEventChunk complex } break; } + } } if (isOrderBy) { @@ -287,20 +286,21 @@ private ComplexEventChunk processInBatchGroupBy(ComplexEventChunk complexEventCh case EXPIRED: eventPopulator.populateStateEvent(event); String groupByKey = groupByKeyGenerator.constructEventKey(event); - GroupByAggregationAttributeExecutor.getKeyThreadLocal().set(groupByKey); - - for (AttributeProcessor attributeProcessor : attributeProcessorList) { - attributeProcessor.process(event); - } - - if (!(havingConditionExecutor != null && !havingConditionExecutor.execute(event))) { - if ((event.getType() == StreamEvent.Type.CURRENT && currentOn) || (event.getType() == - StreamEvent.Type.EXPIRED && expiredOn)) { - complexEventChunk.remove(); - groupedEvents.put(groupByKey, event); + SiddhiAppContext.startGroupByFlow(groupByKey); + try { + for (AttributeProcessor attributeProcessor : attributeProcessorList) { + attributeProcessor.process(event); + } + if (!(havingConditionExecutor != null && !havingConditionExecutor.execute(event))) { + if ((event.getType() == StreamEvent.Type.CURRENT && currentOn) || (event.getType() == + StreamEvent.Type.EXPIRED && expiredOn)) { + complexEventChunk.remove(); + groupedEvents.put(groupByKey, event); + } } + } finally { + SiddhiAppContext.stopGroupByFlow(); } - GroupByAggregationAttributeExecutor.getKeyThreadLocal().remove(); break; case TIMER: break; @@ -361,25 +361,6 @@ public void setToLast(Processor processor) { } } - @Override - public Processor cloneProcessor(String key) { - return null; - } - - @Override - public void clean() { - for (AttributeProcessor processor : attributeProcessorList) { - processor.clean(); - } - if (havingConditionExecutor != null) { - havingConditionExecutor.clean(); - } - if (groupByKeyGenerator != null) { - groupByKeyGenerator.clean(); - } - outputRateLimiter.clean(); - } - public List getAttributeProcessorList() { return attributeProcessorList; } @@ -405,27 +386,6 @@ public void setHavingConditionExecutor(ConditionExpressionExecutor havingConditi this.containsAggregator = this.containsAggregator || containsAggregator; } - public QuerySelector clone(String key) { - QuerySelector clonedQuerySelector = new QuerySelector(id + key, selector, currentOn, expiredOn, - siddhiQueryContext); - List clonedAttributeProcessorList = new ArrayList(); - for (AttributeProcessor attributeProcessor : attributeProcessorList) { - clonedAttributeProcessorList.add(attributeProcessor.cloneProcessor(key)); - } - clonedQuerySelector.attributeProcessorList = clonedAttributeProcessorList; - clonedQuerySelector.isGroupBy = isGroupBy; - clonedQuerySelector.containsAggregator = containsAggregator; - clonedQuerySelector.groupByKeyGenerator = groupByKeyGenerator; - clonedQuerySelector.havingConditionExecutor = havingConditionExecutor; - clonedQuerySelector.eventPopulator = eventPopulator; - clonedQuerySelector.batchingEnabled = batchingEnabled; - clonedQuerySelector.isOrderBy = isOrderBy; - clonedQuerySelector.orderByEventComparator = orderByEventComparator; - clonedQuerySelector.limit = limit; - clonedQuerySelector.offset = offset; - return clonedQuerySelector; - } - public void setBatchingEnabled(boolean batchingEnabled) { this.batchingEnabled = batchingEnabled; } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AndAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AndAttributeAggregatorExecutor.java similarity index 59% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AndAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AndAttributeAggregatorExecutor.java index f41a1b6f00..8abd5d7eb7 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AndAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AndAttributeAggregatorExecutor.java @@ -27,13 +27,15 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.HashMap; import java.util.Map; /** - * {@link AttributeAggregator} to calculate sum based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate sum based on an event attribute. */ @Extension( name = "and", @@ -57,11 +59,11 @@ ) } ) -public class AndAttributeAggregator extends AttributeAggregator { +public class AndAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { private static Attribute.Type type = Attribute.Type.BOOL; - private int trueEventsCount = 0; - private int falseEventsCount = 0; + /** * The initialization method for FunctionExecutor @@ -69,18 +71,19 @@ public class AndAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link AndAttributeAggregator} configuration reader. + * @param configReader this hold the {@link AndAttributeAggregatorExecutor} configuration reader. * @param siddhiQueryContext current siddhi query context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("And aggregator has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length + " parameters provided"); } + return () -> new AggregatorState(); } public Attribute.Type getReturnType() { @@ -88,76 +91,82 @@ public Attribute.Type getReturnType() { } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, AggregatorState state) { if ((boolean) data) { - trueEventsCount++; + state.trueEventsCount++; } else { - falseEventsCount++; + state.falseEventsCount++; } - return computeLogicalOperation(); + return computeLogicalOperation(state); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, AggregatorState state) { for (Object object : data) { if ((boolean) object) { - trueEventsCount++; + state.trueEventsCount++; } else { - falseEventsCount++; + state.falseEventsCount++; } } - return computeLogicalOperation(); + return computeLogicalOperation(state); } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, AggregatorState state) { if ((boolean) data) { - trueEventsCount--; + state.trueEventsCount--; } else { - falseEventsCount--; + state.falseEventsCount--; } - return computeLogicalOperation(); + return computeLogicalOperation(state); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, AggregatorState state) { for (Object object : data) { if ((boolean) object) { - trueEventsCount--; + state.trueEventsCount--; } else { - falseEventsCount--; + state.falseEventsCount--; } } - return computeLogicalOperation(); + return computeLogicalOperation(state); } - private boolean computeLogicalOperation() { - return trueEventsCount > 0 && falseEventsCount == 0; + private boolean computeLogicalOperation(AggregatorState state) { + return state.trueEventsCount > 0 && state.falseEventsCount == 0; } @Override - public Object reset() { - trueEventsCount = 0; - falseEventsCount = 0; + public Object reset(AggregatorState state) { + state.trueEventsCount = 0; + state.falseEventsCount = 0; return false; } - @Override - public boolean canDestroy() { - return trueEventsCount == 0 && falseEventsCount == 0; - } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("trueEventsCount", trueEventsCount); - state.put("falseEventsCount", falseEventsCount); - return state; - } + class AggregatorState extends State { + private int trueEventsCount = 0; + private int falseEventsCount = 0; - @Override - public void restoreState(Map state) { - trueEventsCount = (int) state.get("trueEventsCount"); - falseEventsCount = (int) state.get("falseEventsCount"); + @Override + public boolean canDestroy() { + return trueEventsCount == 0 && falseEventsCount == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("trueEventsCount", trueEventsCount); + state.put("falseEventsCount", falseEventsCount); + return state; + } + + @Override + public void restore(Map state) { + trueEventsCount = (int) state.get("trueEventsCount"); + falseEventsCount = (int) state.get("falseEventsCount"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AttributeAggregator.java deleted file mode 100644 index 5559c7dbe7..0000000000 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AttributeAggregator.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package io.siddhi.core.query.selector.attribute.aggregator; - -import io.siddhi.core.config.SiddhiQueryContext; -import io.siddhi.core.event.ComplexEvent; -import io.siddhi.core.exception.SiddhiAppCreationException; -import io.siddhi.core.exception.SiddhiAppRuntimeException; -import io.siddhi.core.executor.ExpressionExecutor; -import io.siddhi.core.query.processor.ProcessingMode; -import io.siddhi.core.util.config.ConfigReader; -import io.siddhi.query.api.definition.Attribute; - -import java.util.Map; - -/** - * Abstract parent class for attribute aggregators. Attribute aggregators are used to perform aggregate operations - * such as count, average, etc. - */ -public abstract class AttributeAggregator { - - protected ExpressionExecutor[] attributeExpressionExecutors; - private int attributeSize; - private ProcessingMode processingMode; - private boolean outputExpectsExpiredEvents; - private SiddhiQueryContext siddhiQueryContext; - private ConfigReader configReader; - - public void initAggregator(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, - ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { - this.processingMode = processingMode; - this.outputExpectsExpiredEvents = outputExpectsExpiredEvents; - this.siddhiQueryContext = siddhiQueryContext; - this.configReader = configReader; - try { - this.attributeExpressionExecutors = attributeExpressionExecutors; - this.attributeSize = attributeExpressionExecutors.length; - init(attributeExpressionExecutors, processingMode, outputExpectsExpiredEvents, - configReader, siddhiQueryContext); - } catch (Throwable t) { - throw new SiddhiAppCreationException(t); - } - } - - public AttributeAggregator cloneAggregator(String key) { - try { - AttributeAggregator attributeAggregator = this.getClass().newInstance(); - ExpressionExecutor[] innerExpressionExecutors = new ExpressionExecutor[attributeSize]; - for (int i = 0; i < attributeSize; i++) { - innerExpressionExecutors[i] = attributeExpressionExecutors[i].cloneExecutor(key); - } - attributeAggregator.initAggregator(innerExpressionExecutors, processingMode, outputExpectsExpiredEvents, - configReader, siddhiQueryContext); - return attributeAggregator; - } catch (Exception e) { - throw new SiddhiAppRuntimeException("Exception in cloning " + this.getClass().getCanonicalName(), e); - } - } - - public synchronized Object process(ComplexEvent event) { - if (attributeSize > 1) { - Object[] data = new Object[attributeSize]; - for (int i = 0; i < attributeSize; i++) { - data[i] = attributeExpressionExecutors[i].execute(event); - } - switch (event.getType()) { - case CURRENT: - return processAdd(data); - case EXPIRED: - return processRemove(data); - case RESET: - return reset(); - } - } else if (attributeSize == 1) { - switch (event.getType()) { - case CURRENT: - return processAdd(attributeExpressionExecutors[0].execute(event)); - case EXPIRED: - return processRemove(attributeExpressionExecutors[0].execute(event)); - case RESET: - return reset(); - } - } else { - switch (event.getType()) { - case CURRENT: - return processAdd(null); - case EXPIRED: - return processRemove(null); - case RESET: - return reset(); - } - } - return null; - } - - /** - * The initialization method for FunctionExecutor - * - * @param attributeExpressionExecutors are the executors of each attributes in the function - * @param processingMode query processing mode - * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link AttributeAggregator} extensions configuration reader. - * @param siddhiQueryContext Siddhi query runtime context - */ - protected abstract void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext); - - public abstract Attribute.Type getReturnType(); - - public abstract Object processAdd(Object data); - - public abstract Object processAdd(Object[] data); - - public abstract Object processRemove(Object data); - - public abstract Object processRemove(Object[] data); - - public abstract boolean canDestroy(); - - public abstract Object reset(); - - public abstract Map currentState(); - - public abstract void restoreState(Map state); - - public void clean() { - for (ExpressionExecutor expressionExecutor : attributeExpressionExecutors) { - expressionExecutor.clean(); - } - } -} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AttributeAggregatorExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AttributeAggregatorExecutor.java new file mode 100644 index 0000000000..505b9a3aa8 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AttributeAggregatorExecutor.java @@ -0,0 +1,177 @@ +/* + * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. + * + * WSO2 Inc. licenses this file to you under the Apache License, + * Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.siddhi.core.query.selector.attribute.aggregator; + +import io.siddhi.core.config.SiddhiQueryContext; +import io.siddhi.core.event.ComplexEvent; +import io.siddhi.core.exception.SiddhiAppCreationException; +import io.siddhi.core.executor.ExpressionExecutor; +import io.siddhi.core.query.processor.ProcessingMode; +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.core.util.snapshot.state.StateHolder; + +/** + * Abstract parent class for attribute aggregators. Attribute aggregators are used to perform aggregate operations + * such as count, average, etc. + * + * @param current state for the Attribute Executor + */ +public abstract class AttributeAggregatorExecutor implements ExpressionExecutor { + + protected ExpressionExecutor[] attributeExpressionExecutors; + private int attributeSize; + private StateHolder stateHolder; + + public void initAggregator(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, + ConfigReader configReader, boolean groupBy, SiddhiQueryContext siddhiQueryContext) { + try { + this.attributeExpressionExecutors = attributeExpressionExecutors; + this.attributeSize = attributeExpressionExecutors.length; + StateFactory stateFactory = init(attributeExpressionExecutors, processingMode, + outputExpectsExpiredEvents, configReader, siddhiQueryContext); + stateHolder = siddhiQueryContext.generateStateHolder(this.getClass().getName(), + groupBy, stateFactory, true); + } catch (Throwable t) { + throw new SiddhiAppCreationException(t); + } + } + + public Object execute(ComplexEvent event) { + if (attributeSize > 1) { + return processAttributeArray(event); + } else if (attributeSize == 1) { + return processAttribute(event); + } else { + return processNoAttribute(event); + } + } + + private Object processAttributeArray(ComplexEvent event) { + Object[] data = new Object[attributeSize]; + for (int i = 0; i < attributeSize; i++) { + data[i] = attributeExpressionExecutors[i].execute(event); + } + switch (event.getType()) { + case CURRENT: + return processAdd(data); + case EXPIRED: + return processRemove(data); + case RESET: + return processReset(); + } + return null; + } + + private Object processAttribute(ComplexEvent event) { + switch (event.getType()) { + case CURRENT: + return processAdd(attributeExpressionExecutors[0].execute(event)); + case EXPIRED: + return processRemove(attributeExpressionExecutors[0].execute(event)); + case RESET: + return processReset(); + } + return null; + } + + private Object processNoAttribute(ComplexEvent event) { + switch (event.getType()) { + case CURRENT: + return processAdd(null); + case EXPIRED: + return processRemove(null); + case RESET: + return processReset(); + } + return null; + } + + private Object processAdd(Object data) { + S state = stateHolder.getState(); + try { + return processAdd(data, state); + } finally { + stateHolder.returnState(state); + } + } + + private Object processRemove(Object data) { + S state = stateHolder.getState(); + try { + return processRemove(data, state); + } finally { + stateHolder.returnState(state); + } + } + + private Object processAdd(Object[] data) { + S state = stateHolder.getState(); + try { + return processAdd(data, state); + } finally { + stateHolder.returnState(state); + } + } + + private Object processRemove(Object[] data) { + S state = stateHolder.getState(); + try { + return processRemove(data, state); + } finally { + stateHolder.returnState(state); + } + } + + private Object processReset() { + S state = stateHolder.cleanGroupByStates(); + if (state != null) { + return reset(state); + } + return null; + } + + /** + * The initialization method for FunctionExecutor + * + * @param attributeExpressionExecutors are the executors of each attributes in the function + * @param processingMode query processing mode + * @param outputExpectsExpiredEvents is expired events sent as output + * @param configReader this hold the {@link AttributeAggregatorExecutor} extensions + * configuration reader. + * @param siddhiQueryContext Siddhi query runtime context + */ + protected abstract StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext); + + public abstract Object processAdd(Object data, S state); + + public abstract Object processAdd(Object[] data, S state); + + public abstract Object processRemove(Object data, S state); + + public abstract Object processRemove(Object[] data, S state); + + public abstract Object reset(S state); + + +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AvgAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AvgAttributeAggregatorExecutor.java similarity index 70% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AvgAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AvgAttributeAggregatorExecutor.java index 55bc20a663..777498a48c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AvgAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/AvgAttributeAggregatorExecutor.java @@ -27,6 +27,8 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.Arrays; @@ -34,7 +36,7 @@ import java.util.Map; /** - * {@link AttributeAggregator} to calculate average based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate average based on an event attribute. */ @Extension( name = "avg", @@ -54,9 +56,10 @@ "arrival and expiry." ) ) -public class AvgAttributeAggregator extends AttributeAggregator { +public class AvgAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { - private AvgAttributeAggregator avgOutputAttributeAggregator; + private Attribute.Type returnType; /** * The initialization method for FunctionExecutor @@ -64,102 +67,79 @@ public class AvgAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link AvgAttributeAggregator} configuration reader. + * @param configReader this hold the {@link AvgAttributeAggregatorExecutor} configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("Avg aggregator has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length + " parameters provided"); } + returnType = Attribute.Type.DOUBLE; Attribute.Type type = attributeExpressionExecutors[0].getReturnType(); - switch (type) { - case FLOAT: - avgOutputAttributeAggregator = new AvgAttributeAggregatorFloat(); - break; - case INT: - avgOutputAttributeAggregator = new AvgAttributeAggregatorInt(); - break; - case LONG: - avgOutputAttributeAggregator = new AvgAttributeAggregatorLong(); - break; - case DOUBLE: - avgOutputAttributeAggregator = new AvgAttributeAggregatorDouble(); - break; - default: - throw new OperationNotSupportedException("Avg not supported for " + type); - } + return () -> { + switch (type) { + case FLOAT: + return new AvgAttributeAggregatorStateFloat(); + case INT: + return new AvgAttributeAggregatorStateInt(); + case LONG: + return new AvgAttributeAggregatorStateLong(); + case DOUBLE: + return new AvgAttributeAggregatorStateDouble(); + default: + throw new OperationNotSupportedException("Avg not supported for " + returnType); + } + }; + } public Attribute.Type getReturnType() { - return avgOutputAttributeAggregator.getReturnType(); + return returnType; } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, AvgAttributeState state) { if (data == null) { - return avgOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return avgOutputAttributeAggregator.processAdd(data); + return state.processAdd(data); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, AvgAttributeState state) { // will not occur return new IllegalStateException("Avg cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, AvgAttributeState state) { if (data == null) { - return avgOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return avgOutputAttributeAggregator.processRemove(data); + return state.processRemove(data); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, AvgAttributeState state) { // will not occur return new IllegalStateException("Avg cannot process data array, but found " + Arrays.deepToString(data)); } - protected Object currentValue() { - return null; - } - - @Override - public boolean canDestroy() { - return avgOutputAttributeAggregator.canDestroy(); - } - - @Override - public Object reset() { - return avgOutputAttributeAggregator.reset(); - } - - @Override - public Map currentState() { - return avgOutputAttributeAggregator.currentState(); - } - @Override - public void restoreState(Map state) { - avgOutputAttributeAggregator.restoreState(state); + public Object reset(AvgAttributeState state) { + return state.reset(); } - class AvgAttributeAggregatorDouble extends AvgAttributeAggregator { + class AvgAttributeAggregatorStateDouble extends AvgAttributeState { - private final Attribute.Type type = Attribute.Type.DOUBLE; private double value = 0.0; private long count = 0; - public Attribute.Type getReturnType() { - return type; - } - @Override public Object processAdd(Object data) { count++; @@ -193,7 +173,7 @@ public boolean canDestroy() { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("Value", value); state.put("Count", count); @@ -201,7 +181,7 @@ public Map currentState() { } @Override - public void restoreState(Map state) { + public void restore(Map state) { value = (double) state.get("Value"); count = (long) state.get("Count"); } @@ -214,16 +194,11 @@ protected Object currentValue() { } } - class AvgAttributeAggregatorFloat extends AvgAttributeAggregator { + class AvgAttributeAggregatorStateFloat extends AvgAttributeState { - private final Attribute.Type type = Attribute.Type.DOUBLE; private double value = 0.0; private long count = 0; - public Attribute.Type getReturnType() { - return this.type; - } - @Override public Object processAdd(Object data) { count++; @@ -257,7 +232,7 @@ public boolean canDestroy() { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("Value", value); state.put("Count", count); @@ -265,7 +240,7 @@ public Map currentState() { } @Override - public void restoreState(Map state) { + public void restore(Map state) { value = (double) state.get("Value"); count = (long) state.get("Count"); } @@ -278,16 +253,11 @@ protected Object currentValue() { } } - class AvgAttributeAggregatorInt extends AvgAttributeAggregator { + class AvgAttributeAggregatorStateInt extends AvgAttributeState { - private final Attribute.Type type = Attribute.Type.DOUBLE; private double value = 0.0; private long count = 0; - public Attribute.Type getReturnType() { - return this.type; - } - @Override public Object processAdd(Object data) { count++; @@ -321,7 +291,7 @@ public boolean canDestroy() { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("Value", value); state.put("Count", count); @@ -329,7 +299,7 @@ public Map currentState() { } @Override - public void restoreState(Map state) { + public void restore(Map state) { value = (double) state.get("Value"); count = (long) state.get("Count"); } @@ -340,19 +310,13 @@ protected Object currentValue() { } return value / count; } - } - class AvgAttributeAggregatorLong extends AvgAttributeAggregator { + class AvgAttributeAggregatorStateLong extends AvgAttributeState { - private final Attribute.Type type = Attribute.Type.DOUBLE; private double value = 0.0; private long count = 0; - public Attribute.Type getReturnType() { - return type; - } - @Override public Object processAdd(Object data) { count++; @@ -386,7 +350,7 @@ public boolean canDestroy() { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("Value", value); state.put("Count", count); @@ -394,7 +358,7 @@ public Map currentState() { } @Override - public void restoreState(Map state) { + public void restore(Map state) { value = (double) state.get("Value"); count = (long) state.get("Count"); } @@ -405,7 +369,16 @@ protected Object currentValue() { } return value / count; } + } + + abstract class AvgAttributeState extends State { + public abstract Object processAdd(Object data); + + public abstract Object processRemove(Object obj); + + public abstract Object reset(); + protected abstract Object currentValue(); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/CountAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/CountAttributeAggregatorExecutor.java similarity index 57% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/CountAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/CountAttributeAggregatorExecutor.java index 390ae68c53..83f55d25e2 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/CountAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/CountAttributeAggregatorExecutor.java @@ -25,13 +25,15 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.HashMap; import java.util.Map; /** - * {@link AttributeAggregator} to calculate count. + * {@link AttributeAggregatorExecutor} to calculate count. */ @Extension( name = "count", @@ -48,10 +50,10 @@ description = "This will return the count of all the events for time batch in 10 seconds." ) ) -public class CountAttributeAggregator extends AttributeAggregator { +public class CountAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { private static Attribute.Type type = Attribute.Type.LONG; - private long count = 0L; /** * The initialization method for FunctionExecutor @@ -59,13 +61,16 @@ public class CountAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link CountAttributeAggregator} configuration reader. + * @param configReader this hold the {@link CountAttributeAggregatorExecutor} configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, + ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { + return () -> new AggregatorState(); } @@ -74,49 +79,54 @@ public Attribute.Type getReturnType() { } @Override - public Object processAdd(Object data) { - count++; - return count; + public Object processAdd(Object data, AggregatorState state) { + state.count++; + return state.count; } @Override - public Object processAdd(Object[] data) { - count++; - return count; + public Object processAdd(Object[] data, AggregatorState state) { + state.count++; + return state.count; } @Override - public Object processRemove(Object data) { - count--; - return count; + public Object processRemove(Object data, AggregatorState state) { + state.count--; + return state.count; } @Override - public Object processRemove(Object[] data) { - count--; - return count; + public Object processRemove(Object[] data, AggregatorState state) { + state.count--; + return state.count; } @Override - public Object reset() { - count = 0L; - return count; + public Object reset(AggregatorState state) { + state.count = 0L; + return state.count; } - @Override - public boolean canDestroy() { - return count == 0; - } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("Count", count); - return state; - } + class AggregatorState extends State { + private long count = 0L; - @Override - public void restoreState(Map state) { - count = (long) state.get("Count"); + @Override + public boolean canDestroy() { + return count == 0L; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Count", count); + return state; + } + + @Override + public void restore(Map state) { + count = (long) state.get("Count"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregatorExecutor.java similarity index 63% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregatorExecutor.java index 7fae74d654..d1cb69fc1f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregatorExecutor.java @@ -28,6 +28,8 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.Arrays; @@ -35,7 +37,7 @@ import java.util.Map; /** - * {@link AttributeAggregator} to calculate distinct count based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate distinct count based on an event attribute. */ @Extension( name = "distinctCount", @@ -64,8 +66,8 @@ " The three distinct occurences identified are 'WEB_PAGE_1', 'WEB_PAGE_2', and 'WEB_PAGE_3'." ) ) -public class DistinctCountAttributeAggregator extends AttributeAggregator { - private Map distinctValues = new HashMap(); +public class DistinctCountAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { /** * The initialization method for FunctionExecutor @@ -73,18 +75,21 @@ public class DistinctCountAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link DistinctCountAttributeAggregator} configuration reader. + * @param configReader this hold the {@link DistinctCountAttributeAggregatorExecutor} + * configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("Distinct count aggregator has to have exactly 1 parameter, " + "currently " + attributeExpressionExecutors.length + " parameters provided"); } + return () -> new AggregatorState(); } public Attribute.Type getReturnType() { @@ -92,64 +97,70 @@ public Attribute.Type getReturnType() { } @Override - public Object processAdd(Object data) { - Long preVal = distinctValues.get(data); + public Object processAdd(Object data, AggregatorState state) { + Long preVal = state.distinctValues.get(data); if (preVal != null) { - distinctValues.put(data, ++preVal); + state.distinctValues.put(data, ++preVal); } else { - distinctValues.put(data, 1L); + state.distinctValues.put(data, 1L); } - return getDistinctCount(); + return state.getDistinctCount(); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, AggregatorState state) { return new IllegalStateException( "Distinct count aggregator cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public Object processRemove(Object data) { - Long preVal = distinctValues.get(data); + public Object processRemove(Object data, AggregatorState state) { + Long preVal = state.distinctValues.get(data); preVal--; if (preVal > 0) { - distinctValues.put(data, preVal); + state.distinctValues.put(data, preVal); } else { - distinctValues.remove(data); + state.distinctValues.remove(data); } - return getDistinctCount(); + return state.getDistinctCount(); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, AggregatorState state) { return new IllegalStateException( "Distinct count aggregator cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public Object reset() { - distinctValues.clear(); - return getDistinctCount(); + public Object reset(AggregatorState state) { + state.distinctValues.clear(); + return state.getDistinctCount(); } - @Override - public boolean canDestroy() { - return distinctValues.size() == 0; - } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("DistinctValues", distinctValues); - return state; - } + class AggregatorState extends State { - @Override - public void restoreState(Map state) { - distinctValues = (Map) state.get("DistinctValues"); - } + private Map distinctValues = new HashMap(); - private long getDistinctCount() { - return distinctValues.size(); + @Override + public boolean canDestroy() { + return distinctValues.isEmpty(); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("DistinctValues", distinctValues); + return state; + } + + @Override + public void restore(Map state) { + distinctValues = (Map) state.get("DistinctValues"); + } + + protected long getDistinctCount() { + return distinctValues.size(); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxAttributeAggregatorExecutor.java similarity index 65% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxAttributeAggregatorExecutor.java index 0d0e2a6382..999c446201 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxAttributeAggregatorExecutor.java @@ -27,6 +27,8 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.Arrays; @@ -37,7 +39,7 @@ import java.util.Map; /** - * {@link AttributeAggregator} to calculate max value based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate max value based on an event attribute. */ @Extension( name = "max", @@ -59,9 +61,10 @@ "arrival and expiry." ) ) -public class MaxAttributeAggregator extends AttributeAggregator { +public class MaxAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { - private MaxAttributeAggregator maxOutputAttributeAggregator; + private Attribute.Type returnType; /** * The initialization method for FunctionExecutor @@ -69,13 +72,15 @@ public class MaxAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link MaxAttributeAggregator} configuration reader. + * @param configReader this hold the {@link MaxAttributeAggregatorExecutor} configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, + ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("Max aggregator has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length + " parameters provided"); @@ -84,99 +89,74 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, Processin if (processingMode == ProcessingMode.SLIDE || outputExpectsExpiredEvents) { trackFutureStates = true; } - Attribute.Type type = attributeExpressionExecutors[0].getReturnType(); - switch (type) { - case FLOAT: - maxOutputAttributeAggregator = new MaxAttributeAggregatorFloat(trackFutureStates); - break; - case INT: - maxOutputAttributeAggregator = new MaxAttributeAggregatorInt(trackFutureStates); - break; - case LONG: - maxOutputAttributeAggregator = new MaxAttributeAggregatorLong(trackFutureStates); - break; - case DOUBLE: - maxOutputAttributeAggregator = new MaxAttributeAggregatorDouble(trackFutureStates); - break; - default: - throw new OperationNotSupportedException("Max not supported for " + type); - } + returnType = attributeExpressionExecutors[0].getReturnType(); + boolean finalTrackFutureStates = trackFutureStates; + return () -> { + switch (returnType) { + case FLOAT: + return new MaxAttributeAggregatorStateFloat(finalTrackFutureStates); + case INT: + return new MaxAttributeAggregatorStateInt(finalTrackFutureStates); + case LONG: + return new MaxAttributeAggregatorStateLong(finalTrackFutureStates); + case DOUBLE: + return new MaxAttributeAggregatorStateDouble(finalTrackFutureStates); + default: + throw new OperationNotSupportedException("Max not supported for " + returnType); + } + }; } public Attribute.Type getReturnType() { - return maxOutputAttributeAggregator.getReturnType(); + return returnType; } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, MaxAggregatorState state) { if (data == null) { - return maxOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return maxOutputAttributeAggregator.processAdd(data); - } - - protected Object currentValue() { - return null; + return state.processAdd(data); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, MaxAggregatorState state) { // will not occur return new IllegalStateException("Max cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, MaxAggregatorState state) { if (data == null) { - return maxOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return maxOutputAttributeAggregator.processRemove(data); + return state.processRemove(data); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, MaxAggregatorState state) { // will not occur return new IllegalStateException("Max cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public boolean canDestroy() { - return maxOutputAttributeAggregator.canDestroy(); + public Object reset(MaxAggregatorState state) { + return state.reset(); } - @Override - public Object reset() { - return maxOutputAttributeAggregator.reset(); - } + class MaxAttributeAggregatorStateDouble extends MaxAggregatorState { - @Override - public Map currentState() { - return maxOutputAttributeAggregator.currentState(); - } - - @Override - public void restoreState(Map state) { - maxOutputAttributeAggregator.restoreState(state); - } - - class MaxAttributeAggregatorDouble extends MaxAttributeAggregator { - - private final Attribute.Type type = Attribute.Type.DOUBLE; private Deque maxDeque = null; private volatile Double maxValue = null; - public MaxAttributeAggregatorDouble(boolean trackFutureStates) { + public MaxAttributeAggregatorStateDouble(boolean trackFutureStates) { if (trackFutureStates) { maxDeque = new LinkedList<>(); } } - public Attribute.Type getReturnType() { - return type; - } - @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Double value = (Double) data; if (maxDeque != null) { for (Iterator iterator = maxDeque.descendingIterator(); iterator.hasNext(); ) { @@ -195,7 +175,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { if (maxDeque != null) { maxDeque.removeFirstOccurrence(data); maxValue = maxDeque.peekFirst(); @@ -208,7 +188,7 @@ public synchronized Object processRemove(Object data) { } @Override - public synchronized Object reset() { + public Object reset() { if (maxDeque != null) { maxDeque.clear(); } @@ -218,21 +198,19 @@ public synchronized Object reset() { @Override public boolean canDestroy() { - return (maxDeque == null || maxDeque.size() == 0) && maxValue == null; + return (maxDeque == null || maxDeque.isEmpty()) && maxValue == null; } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); - synchronized (this) { - state.put("MaxValue", maxValue); - state.put("MaxDeque", maxDeque); - } + state.put("MaxValue", maxValue); + state.put("MaxDeque", maxDeque); return state; } @Override - public synchronized void restoreState(Map state) { + public void restore(Map state) { maxValue = (Double) state.get("MaxValue"); maxDeque = (Deque) state.get("MaxDeque"); } @@ -242,24 +220,19 @@ protected Object currentValue() { } } - class MaxAttributeAggregatorFloat extends MaxAttributeAggregator { + class MaxAttributeAggregatorStateFloat extends MaxAggregatorState { - private final Attribute.Type type = Attribute.Type.FLOAT; private Deque maxDeque = null; private volatile Float maxValue = null; - public MaxAttributeAggregatorFloat(boolean trackFutureStates) { + public MaxAttributeAggregatorStateFloat(boolean trackFutureStates) { if (trackFutureStates) { maxDeque = new LinkedList<>(); } } - public Attribute.Type getReturnType() { - return type; - } - @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Float value = (Float) data; if (maxDeque != null) { for (Iterator iterator = maxDeque.descendingIterator(); iterator.hasNext(); ) { @@ -278,7 +251,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { if (maxDeque != null) { maxDeque.removeFirstOccurrence(data); maxValue = maxDeque.peekFirst(); @@ -291,7 +264,7 @@ public synchronized Object processRemove(Object data) { } @Override - public synchronized Object reset() { + public Object reset() { if (maxDeque != null) { maxDeque.clear(); } @@ -301,21 +274,19 @@ public synchronized Object reset() { @Override public boolean canDestroy() { - return (maxDeque == null || maxDeque.size() == 0) && maxValue == null; + return (maxDeque == null || maxDeque.isEmpty()) && maxValue == null; } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); - synchronized (this) { - state.put("MaxValue", maxValue); - state.put("MaxDeque", maxDeque); - } + state.put("MaxValue", maxValue); + state.put("MaxDeque", maxDeque); return state; } @Override - public synchronized void restoreState(Map state) { + public void restore(Map state) { maxValue = (Float) state.get("MaxValue"); maxDeque = (Deque) state.get("MaxDeque"); } @@ -326,24 +297,19 @@ protected Object currentValue() { } - class MaxAttributeAggregatorInt extends MaxAttributeAggregator { + class MaxAttributeAggregatorStateInt extends MaxAggregatorState { - private final Attribute.Type type = Attribute.Type.INT; private Deque maxDeque = null; private volatile Integer maxValue = null; - public MaxAttributeAggregatorInt(boolean trackFutureStates) { + public MaxAttributeAggregatorStateInt(boolean trackFutureStates) { if (trackFutureStates) { maxDeque = new LinkedList<>(); } } - public Attribute.Type getReturnType() { - return type; - } - @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Integer value = (Integer) data; if (maxDeque != null) { for (Iterator iterator = maxDeque.descendingIterator(); iterator.hasNext(); ) { @@ -362,7 +328,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { if (maxDeque != null) { maxDeque.removeFirstOccurrence(data); maxValue = maxDeque.peekFirst(); @@ -375,7 +341,7 @@ public synchronized Object processRemove(Object data) { } @Override - public synchronized Object reset() { + public Object reset() { if (maxDeque != null) { maxDeque.clear(); } @@ -385,21 +351,19 @@ public synchronized Object reset() { @Override public boolean canDestroy() { - return (maxDeque == null || maxDeque.size() == 0) && maxValue == null; + return (maxDeque == null || maxDeque.isEmpty()) && maxValue == null; } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); - synchronized (this) { - state.put("MaxValue", maxValue); - state.put("MaxDeque", maxDeque); - } + state.put("MaxValue", maxValue); + state.put("MaxDeque", maxDeque); return state; } @Override - public synchronized void restoreState(Map state) { + public void restore(Map state) { maxValue = (Integer) state.get("MaxValue"); maxDeque = (Deque) state.get("MaxDeque"); } @@ -410,24 +374,19 @@ protected Object currentValue() { } - class MaxAttributeAggregatorLong extends MaxAttributeAggregator { + class MaxAttributeAggregatorStateLong extends MaxAggregatorState { - private final Attribute.Type type = Attribute.Type.LONG; private Deque maxDeque = null; private volatile Long maxValue = null; - public MaxAttributeAggregatorLong(boolean trackFutureStates) { + public MaxAttributeAggregatorStateLong(boolean trackFutureStates) { if (trackFutureStates) { maxDeque = new LinkedList<>(); } } - public Attribute.Type getReturnType() { - return type; - } - @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Long value = (Long) data; if (maxDeque != null) { for (Iterator iterator = maxDeque.descendingIterator(); iterator.hasNext(); ) { @@ -446,7 +405,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { if (maxDeque != null) { maxDeque.removeFirstOccurrence(data); maxValue = maxDeque.peekFirst(); @@ -459,7 +418,7 @@ public synchronized Object processRemove(Object data) { } @Override - public synchronized Object reset() { + public Object reset() { if (maxDeque != null) { maxDeque.clear(); } @@ -469,21 +428,19 @@ public synchronized Object reset() { @Override public boolean canDestroy() { - return (maxDeque == null || maxDeque.size() == 0) && maxValue == null; + return (maxDeque == null || maxDeque.isEmpty()) && maxValue == null; } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); - synchronized (this) { - state.put("MaxValue", maxValue); - state.put("MaxDeque", maxDeque); - } + state.put("MaxValue", maxValue); + state.put("MaxDeque", maxDeque); return state; } @Override - public synchronized void restoreState(Map state) { + public void restore(Map state) { maxValue = (Long) state.get("MaxValue"); maxDeque = (Deque) state.get("MaxDeque"); } @@ -494,4 +451,14 @@ protected Object currentValue() { } + abstract class MaxAggregatorState extends State { + public abstract Object processAdd(Object data); + + public abstract Object processRemove(Object data); + + public abstract Object reset(); + + protected abstract Object currentValue(); + } + } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxForeverAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxForeverAttributeAggregatorExecutor.java similarity index 63% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxForeverAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxForeverAttributeAggregatorExecutor.java index a8c80b716d..2743ccad55 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxForeverAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MaxForeverAttributeAggregatorExecutor.java @@ -28,6 +28,8 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.Arrays; @@ -35,7 +37,7 @@ import java.util.Map; /** - * {@link AttributeAggregator} to calculate max value for life time based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate max value for life time based on an event attribute. */ @Extension( name = "maxForever", @@ -58,9 +60,10 @@ " the lifetime of the query." ) ) -public class MaxForeverAttributeAggregator extends AttributeAggregator { +public class MaxForeverAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { - private MaxForeverAttributeAggregator maxForeverAttributeAggregator; + private Attribute.Type returnType; /** * The initialization method for FunctionExecutor @@ -68,106 +71,87 @@ public class MaxForeverAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link MaxForeverAttributeAggregator} configuration reader. + * @param configReader this hold the {@link MaxForeverAttributeAggregatorExecutor} + * configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("MaxForever aggregator has to have exactly 1 parameter, " + "currently " + attributeExpressionExecutors.length + " parameters provided"); } - Attribute.Type type = attributeExpressionExecutors[0].getReturnType(); - switch (type) { - case FLOAT: - maxForeverAttributeAggregator = new MaxForeverAttributeAggregatorFloat(); - break; - case INT: - maxForeverAttributeAggregator = new MaxForeverAttributeAggregatorInt(); - break; - case LONG: - maxForeverAttributeAggregator = new MaxForeverAttributeAggregatorLong(); - break; - case DOUBLE: - maxForeverAttributeAggregator = new MaxForeverAttributeAggregatorDouble(); - break; - default: - throw new OperationNotSupportedException("MaxForever not supported for " + type); - } + returnType = attributeExpressionExecutors[0].getReturnType(); + return new StateFactory() { + @Override + public MaxAggregatorState createNewState() { + switch (returnType) { + case FLOAT: + return new MaxForeverAttributeAggregatorStateFloat(); + case INT: + return new MaxForeverAttributeAggregatorStateInt(); + case LONG: + return new MaxForeverAttributeAggregatorStateLong(); + case DOUBLE: + return new MaxForeverAttributeAggregatorStateDouble(); + default: + throw new OperationNotSupportedException("MaxForever not supported for " + returnType); + } + } + }; + } public Attribute.Type getReturnType() { - return maxForeverAttributeAggregator.getReturnType(); + return returnType; } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, MaxAggregatorState state) { if (data == null) { - return maxForeverAttributeAggregator.currentValue(); + return state.currentValue(); } - return maxForeverAttributeAggregator.processAdd(data); - } - - protected Object currentValue() { - return null; + return state.processAdd(data); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, MaxAggregatorState state) { // will not occur return new IllegalStateException("MaxForever cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, MaxAggregatorState state) { if (data == null) { - return maxForeverAttributeAggregator.currentValue(); + return state.currentValue(); } - return maxForeverAttributeAggregator.processRemove(data); + return state.processRemove(data); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, MaxAggregatorState state) { // will not occur return new IllegalStateException("MaxForever cannot process data array, but found " + Arrays.deepToString(data)); } - @Override - public boolean canDestroy() { - return false; - } @Override - public Object reset() { - return maxForeverAttributeAggregator.reset(); + public Object reset(MaxAggregatorState state) { + return state.reset(); } - @Override - public Map currentState() { - return maxForeverAttributeAggregator.currentState(); - } + class MaxForeverAttributeAggregatorStateDouble extends MaxAggregatorState { - @Override - public void restoreState(Map state) { - maxForeverAttributeAggregator.restoreState(state); - } - - class MaxForeverAttributeAggregatorDouble extends MaxForeverAttributeAggregator { - - private final Attribute.Type type = Attribute.Type.DOUBLE; private volatile Double maxValue = null; - public Attribute.Type getReturnType() { - return type; - } - @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Double value = (Double) data; if (maxValue == null || maxValue < value) { maxValue = value; @@ -176,7 +160,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { Double value = (Double) data; if (maxValue == null || maxValue < value) { maxValue = value; @@ -190,14 +174,19 @@ public Object reset() { } @Override - public Map currentState() { + public boolean canDestroy() { + return maxValue == null; + } + + @Override + public Map snapshot() { Map state = new HashMap<>(); state.put("MaxValue", maxValue); return state; } @Override - public void restoreState(Map state) { + public void restore(Map state) { maxValue = (Double) state.get("MaxValue"); } @@ -207,17 +196,12 @@ protected Object currentValue() { } - class MaxForeverAttributeAggregatorFloat extends MaxForeverAttributeAggregator { + class MaxForeverAttributeAggregatorStateFloat extends MaxAggregatorState { - private final Attribute.Type type = Attribute.Type.FLOAT; private volatile Float maxValue = null; - public Attribute.Type getReturnType() { - return type; - } - @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Float value = (Float) data; if (maxValue == null || maxValue < value) { maxValue = value; @@ -226,7 +210,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { Float value = (Float) data; if (maxValue == null || maxValue < value) { maxValue = value; @@ -240,14 +224,19 @@ public Object reset() { } @Override - public Map currentState() { + public boolean canDestroy() { + return maxValue == null; + } + + @Override + public Map snapshot() { Map state = new HashMap<>(); state.put("MaxValue", maxValue); return state; } @Override - public void restoreState(Map state) { + public void restore(Map state) { maxValue = (Float) state.get("MaxValue"); } @@ -257,17 +246,12 @@ protected Object currentValue() { } - class MaxForeverAttributeAggregatorInt extends MaxForeverAttributeAggregator { + class MaxForeverAttributeAggregatorStateInt extends MaxAggregatorState { - private final Attribute.Type type = Attribute.Type.INT; private volatile Integer maxValue = null; - public Attribute.Type getReturnType() { - return type; - } - @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Integer value = (Integer) data; if (maxValue == null || maxValue < value) { maxValue = value; @@ -276,7 +260,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { Integer value = (Integer) data; if (maxValue == null || maxValue < value) { maxValue = value; @@ -290,14 +274,19 @@ public Object reset() { } @Override - public Map currentState() { + public boolean canDestroy() { + return maxValue == null; + } + + @Override + public Map snapshot() { Map state = new HashMap<>(); state.put("MaxValue", maxValue); return state; } @Override - public void restoreState(Map state) { + public void restore(Map state) { maxValue = (Integer) state.get("MaxValue"); } @@ -306,17 +295,12 @@ protected Object currentValue() { } } - class MaxForeverAttributeAggregatorLong extends MaxForeverAttributeAggregator { + class MaxForeverAttributeAggregatorStateLong extends MaxAggregatorState { - private final Attribute.Type type = Attribute.Type.LONG; private volatile Long maxValue = null; - public Attribute.Type getReturnType() { - return type; - } - @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Long value = (Long) data; if (maxValue == null || maxValue < value) { maxValue = value; @@ -325,7 +309,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { Long value = (Long) data; if (maxValue == null || maxValue < value) { maxValue = value; @@ -339,14 +323,19 @@ public Object reset() { } @Override - public Map currentState() { + public boolean canDestroy() { + return maxValue == null; + } + + @Override + public Map snapshot() { Map state = new HashMap<>(); state.put("MaxValue", maxValue); return state; } @Override - public void restoreState(Map state) { + public void restore(Map state) { maxValue = (Long) state.get("MaxValue"); } @@ -355,4 +344,14 @@ protected Object currentValue() { } } + abstract class MaxAggregatorState extends State { + public abstract Object processAdd(Object data); + + public abstract Object processRemove(Object data); + + public abstract Object reset(); + + protected abstract Object currentValue(); + } + } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinAttributeAggregatorExecutor.java similarity index 69% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinAttributeAggregatorExecutor.java index 00789913c7..b714404a9b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinAttributeAggregatorExecutor.java @@ -27,6 +27,8 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.Arrays; @@ -37,7 +39,7 @@ import java.util.Map; /** - * {@link AttributeAggregator} to calculate min value based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate min value based on an event attribute. */ @Extension( name = "min", @@ -59,9 +61,10 @@ "arrival and expiry." ) ) -public class MinAttributeAggregator extends AttributeAggregator { +public class MinAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { - private MinAttributeAggregator minOutputAttributeAggregator; + private Attribute.Type returnType; public void init(Attribute.Type type) { } @@ -72,13 +75,14 @@ public void init(Attribute.Type type) { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link MinAttributeAggregator} configuration reader. + * @param configReader this hold the {@link MinAttributeAggregatorExecutor} configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("Min aggregator has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length + " parameters provided"); @@ -87,88 +91,72 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, Processin if (processingMode == ProcessingMode.SLIDE || outputExpectsExpiredEvents) { trackFutureStates = true; } - Attribute.Type type = attributeExpressionExecutors[0].getReturnType(); - switch (type) { - case FLOAT: - minOutputAttributeAggregator = new MinAttributeAggregatorFloat(trackFutureStates); - break; - case INT: - minOutputAttributeAggregator = new MinAttributeAggregatorInt(trackFutureStates); - break; - case LONG: - minOutputAttributeAggregator = new MinAttributeAggregatorLong(trackFutureStates); - break; - case DOUBLE: - minOutputAttributeAggregator = new MinAttributeAggregatorDouble(trackFutureStates); - break; - default: - throw new OperationNotSupportedException("Min not supported for " + type); - } + returnType = attributeExpressionExecutors[0].getReturnType(); + boolean finalTrackFutureStates = trackFutureStates; + return new StateFactory() { + @Override + public MinAggregatorState createNewState() { + switch (returnType) { + case FLOAT: + return new MinAttributeAggregatorStateFloat(finalTrackFutureStates); + case INT: + return new MinAttributeAggregatorStateInt(finalTrackFutureStates); + case LONG: + return new MinAttributeAggregatorStateLong(finalTrackFutureStates); + case DOUBLE: + return new MinAttributeAggregatorStateDouble(finalTrackFutureStates); + default: + throw new OperationNotSupportedException("Min not supported for " + returnType); + } + } + }; + } public Attribute.Type getReturnType() { - return minOutputAttributeAggregator.getReturnType(); + return returnType; } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, MinAggregatorState state) { if (data == null) { - return minOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return minOutputAttributeAggregator.processAdd(data); + return state.processAdd(data); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, MinAggregatorState state) { // will not occur return new IllegalStateException("Min cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, MinAggregatorState state) { if (data == null) { - return minOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return minOutputAttributeAggregator.processRemove(data); + return state.processRemove(data); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, MinAggregatorState state) { // will not occur return new IllegalStateException("Min cannot process data array, but found " + Arrays.deepToString(data)); } - protected Object currentValue() { - return null; - } - - @Override - public boolean canDestroy() { - return minOutputAttributeAggregator.canDestroy(); - } - - @Override - public Object reset() { - return minOutputAttributeAggregator.reset(); - } - @Override - public Map currentState() { - return minOutputAttributeAggregator.currentState(); + public Object reset(MinAggregatorState state) { + return state.reset(); } - @Override - public void restoreState(Map state) { - minOutputAttributeAggregator.restoreState(state); - } - - class MinAttributeAggregatorDouble extends MinAttributeAggregator { + class MinAttributeAggregatorStateDouble extends MinAggregatorState { private final Attribute.Type type = Attribute.Type.DOUBLE; private Deque minDeque = null; private volatile Double minValue = null; - public MinAttributeAggregatorDouble(boolean trackFutureStates) { + public MinAttributeAggregatorStateDouble(boolean trackFutureStates) { if (trackFutureStates) { minDeque = new LinkedList<>(); } @@ -179,7 +167,7 @@ public Attribute.Type getReturnType() { } @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Double value = (Double) data; if (minDeque != null) { for (Iterator iterator = minDeque.descendingIterator(); iterator.hasNext(); ) { @@ -198,7 +186,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { if (minDeque != null) { minDeque.removeFirstOccurrence(data); minValue = minDeque.peekFirst(); @@ -211,7 +199,7 @@ public synchronized Object processRemove(Object data) { } @Override - public synchronized Object reset() { + public Object reset() { if (minDeque != null) { minDeque.clear(); } @@ -221,21 +209,19 @@ public synchronized Object reset() { @Override public boolean canDestroy() { - return (minDeque == null || minDeque.size() == 0) && minValue == null; + return (minDeque == null || minDeque.isEmpty()) && minValue == null; } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); - synchronized (this) { - state.put("MinValue", minValue); - state.put("MinDeque", minDeque); - } + state.put("MinValue", minValue); + state.put("MinDeque", minDeque); return state; } @Override - public synchronized void restoreState(Map state) { + public void restore(Map state) { minValue = (Double) state.get("MinValue"); minDeque = (Deque) state.get("MinDeque"); } @@ -246,13 +232,13 @@ protected Object currentValue() { } - class MinAttributeAggregatorFloat extends MinAttributeAggregator { + class MinAttributeAggregatorStateFloat extends MinAggregatorState { private final Attribute.Type type = Attribute.Type.FLOAT; private Deque minDeque = null; private volatile Float minValue = null; - public MinAttributeAggregatorFloat(boolean trackFutureStates) { + public MinAttributeAggregatorStateFloat(boolean trackFutureStates) { if (trackFutureStates) { minDeque = new LinkedList<>(); } @@ -263,7 +249,7 @@ public Attribute.Type getReturnType() { } @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Float value = (Float) data; if (minDeque != null) { for (Iterator iterator = minDeque.descendingIterator(); iterator.hasNext(); ) { @@ -282,7 +268,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { if (minDeque != null) { minDeque.removeFirstOccurrence(data); minValue = minDeque.peekFirst(); @@ -295,7 +281,7 @@ public synchronized Object processRemove(Object data) { } @Override - public synchronized Object reset() { + public Object reset() { if (minDeque != null) { minDeque.clear(); } @@ -305,21 +291,19 @@ public synchronized Object reset() { @Override public boolean canDestroy() { - return (minDeque == null || minDeque.size() == 0) && minValue == null; + return (minDeque == null || minDeque.isEmpty()) && minValue == null; } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); - synchronized (this) { - state.put("MinValue", minValue); - state.put("MinDeque", minDeque); - } + state.put("MinValue", minValue); + state.put("MinDeque", minDeque); return state; } @Override - public synchronized void restoreState(Map state) { + public void restore(Map state) { minValue = (Float) state.get("MinValue"); minDeque = (Deque) state.get("MinDeque"); } @@ -329,13 +313,13 @@ protected Object currentValue() { } } - class MinAttributeAggregatorInt extends MinAttributeAggregator { + class MinAttributeAggregatorStateInt extends MinAggregatorState { private final Attribute.Type type = Attribute.Type.INT; private Deque minDeque = null; private volatile Integer minValue = null; - public MinAttributeAggregatorInt(boolean trackFutureStates) { + public MinAttributeAggregatorStateInt(boolean trackFutureStates) { if (trackFutureStates) { minDeque = new LinkedList<>(); } @@ -346,7 +330,7 @@ public Attribute.Type getReturnType() { } @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Integer value = (Integer) data; if (minDeque != null) { for (Iterator iterator = minDeque.descendingIterator(); iterator.hasNext(); ) { @@ -366,7 +350,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object reset() { + public Object reset() { if (minDeque != null) { minDeque.clear(); } @@ -376,11 +360,11 @@ public synchronized Object reset() { @Override public boolean canDestroy() { - return (minDeque == null || minDeque.size() == 0) && minValue == null; + return (minDeque == null || minDeque.isEmpty()) && minValue == null; } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { if (minDeque != null) { minDeque.removeFirstOccurrence(data); minValue = minDeque.peekFirst(); @@ -393,17 +377,15 @@ public synchronized Object processRemove(Object data) { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); - synchronized (this) { - state.put("MinValue", minValue); - state.put("MinDeque", minDeque); - } + state.put("MinValue", minValue); + state.put("MinDeque", minDeque); return state; } @Override - public synchronized void restoreState(Map state) { + public void restore(Map state) { minValue = (Integer) state.get("MinValue"); minDeque = (Deque) state.get("MinDeque"); } @@ -413,13 +395,13 @@ protected Object currentValue() { } } - class MinAttributeAggregatorLong extends MinAttributeAggregator { + class MinAttributeAggregatorStateLong extends MinAggregatorState { private final Attribute.Type type = Attribute.Type.LONG; private Deque minDeque = null; private volatile Long minValue = null; - public MinAttributeAggregatorLong(boolean trackFutureStates) { + public MinAttributeAggregatorStateLong(boolean trackFutureStates) { if (trackFutureStates) { minDeque = new LinkedList<>(); } @@ -430,7 +412,7 @@ public Attribute.Type getReturnType() { } @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Long value = (Long) data; if (minDeque != null) { for (Iterator iterator = minDeque.descendingIterator(); iterator.hasNext(); ) { @@ -449,7 +431,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object reset() { + public Object reset() { if (minDeque != null) { minDeque.clear(); } @@ -459,11 +441,11 @@ public synchronized Object reset() { @Override public boolean canDestroy() { - return (minDeque == null || minDeque.size() == 0) && minValue == null; + return (minDeque == null || minDeque.isEmpty()) && minValue == null; } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { if (minDeque != null) { minDeque.removeFirstOccurrence(data); minValue = minDeque.peekFirst(); @@ -476,17 +458,15 @@ public synchronized Object processRemove(Object data) { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); - synchronized (this) { - state.put("MinValue", minValue); - state.put("MinDeque", minDeque); - } + state.put("MinValue", minValue); + state.put("MinDeque", minDeque); return state; } @Override - public synchronized void restoreState(Map state) { + public void restore(Map state) { minValue = (Long) state.get("MinValue"); minDeque = (Deque) state.get("MinDeque"); } @@ -497,4 +477,14 @@ protected Object currentValue() { } + abstract class MinAggregatorState extends State { + public abstract Object processAdd(Object data); + + public abstract Object processRemove(Object data); + + public abstract Object reset(); + + protected abstract Object currentValue(); + } + } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinForeverAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinForeverAttributeAggregatorExecutor.java similarity index 65% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinForeverAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinForeverAttributeAggregatorExecutor.java index ebc9af0371..b8da027c94 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinForeverAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/MinForeverAttributeAggregatorExecutor.java @@ -27,6 +27,8 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.Arrays; @@ -34,7 +36,7 @@ import java.util.Map; /** - * {@link AttributeAggregator} to calculate min value for life time based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate min value for life time based on an event attribute. */ @Extension( name = "minForever", @@ -57,9 +59,10 @@ "the lifetime of the query." ) ) -public class MinForeverAttributeAggregator extends AttributeAggregator { +public class MinForeverAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { - private MinForeverAttributeAggregator minForeverAttributeAggregator; + private Attribute.Type returnType; /** * The initialization method for FunctionExecutor @@ -67,95 +70,80 @@ public class MinForeverAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link MinForeverAttributeAggregator} configuration reader. + * @param configReader this hold the {@link MinForeverAttributeAggregatorExecutor} + * configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("MinForever aggregator has to have exactly 1 parameter, " + "currently " + attributeExpressionExecutors.length + " parameters provided"); } - Attribute.Type type = attributeExpressionExecutors[0].getReturnType(); - switch (type) { - case FLOAT: - minForeverAttributeAggregator = new MinForeverAttributeAggregatorFloat(); - break; - case INT: - minForeverAttributeAggregator = new MinForeverAttributeAggregatorInt(); - break; - case LONG: - minForeverAttributeAggregator = new MinForeverAttributeAggregatorLong(); - break; - case DOUBLE: - minForeverAttributeAggregator = new MinForeverAttributeAggregatorDouble(); - break; - default: - throw new OperationNotSupportedException("MinForever not supported for " + type); - } + returnType = attributeExpressionExecutors[0].getReturnType(); + return new StateFactory() { + @Override + public MinAggregatorState createNewState() { + switch (returnType) { + case FLOAT: + return new MinForeverAttributeAggregatorStateFloat(); + case INT: + return new MinForeverAttributeAggregatorStateInt(); + case LONG: + return new MinForeverAttributeAggregatorStateLong(); + case DOUBLE: + return new MinForeverAttributeAggregatorStateDouble(); + default: + throw new OperationNotSupportedException("MinForever not supported for " + returnType); + } + } + }; + } public Attribute.Type getReturnType() { - return minForeverAttributeAggregator.getReturnType(); + return returnType; } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, MinAggregatorState state) { if (data == null) { - return minForeverAttributeAggregator.currentValue(); + return state.currentValue(); } - return minForeverAttributeAggregator.processAdd(data); + return state.processAdd(data); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, MinAggregatorState state) { // will not occur return new IllegalStateException("MinForever cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, MinAggregatorState state) { if (data == null) { - return minForeverAttributeAggregator.currentValue(); + return state.currentValue(); } - return minForeverAttributeAggregator.processRemove(data); + return state.processRemove(data); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, MinAggregatorState state) { // will not occur return new IllegalStateException("MinForever cannot process data array, but found " + Arrays.deepToString(data)); } - protected Object currentValue() { - return null; - } - - @Override - public boolean canDestroy() { - return false; - } - @Override - public Object reset() { - return minForeverAttributeAggregator.reset(); + public Object reset(MinAggregatorState state) { + return state.reset(); } - @Override - public Map currentState() { - return minForeverAttributeAggregator.currentState(); - } - - @Override - public void restoreState(Map state) { - minForeverAttributeAggregator.restoreState(state); - } - - class MinForeverAttributeAggregatorDouble extends MinForeverAttributeAggregator { + class MinForeverAttributeAggregatorStateDouble extends MinAggregatorState { private final Attribute.Type type = Attribute.Type.DOUBLE; private volatile Double minValue = null; @@ -165,7 +153,7 @@ public Attribute.Type getReturnType() { } @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Double value = (Double) data; if (minValue == null || minValue > value) { minValue = value; @@ -174,7 +162,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { Double value = (Double) data; if (minValue == null || minValue > value) { minValue = value; @@ -188,14 +176,14 @@ public Object reset() { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("MinValue", minValue); return state; } @Override - public void restoreState(Map state) { + public void restore(Map state) { minValue = (Double) state.get("MinValue"); } @@ -203,9 +191,14 @@ protected Object currentValue() { return minValue; } + @Override + public boolean canDestroy() { + return minValue == null; + } + } - class MinForeverAttributeAggregatorFloat extends MinForeverAttributeAggregator { + class MinForeverAttributeAggregatorStateFloat extends MinAggregatorState { private final Attribute.Type type = Attribute.Type.FLOAT; private volatile Float minValue = null; @@ -215,7 +208,7 @@ public Attribute.Type getReturnType() { } @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Float value = (Float) data; if (minValue == null || minValue > value) { minValue = value; @@ -224,7 +217,7 @@ public synchronized Object processAdd(Object data) { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { Float value = (Float) data; if (minValue == null || minValue > value) { minValue = value; @@ -238,23 +231,28 @@ public Object reset() { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("MinValue", minValue); return state; } @Override - public void restoreState(Map state) { + public void restore(Map state) { minValue = (Float) state.get("MinValue"); } protected Object currentValue() { return minValue; } + + @Override + public boolean canDestroy() { + return minValue == null; + } } - class MinForeverAttributeAggregatorInt extends MinForeverAttributeAggregator { + class MinForeverAttributeAggregatorStateInt extends MinAggregatorState { private final Attribute.Type type = Attribute.Type.INT; private volatile Integer minValue = null; @@ -264,7 +262,7 @@ public Attribute.Type getReturnType() { } @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Integer value = (Integer) data; if (minValue == null || minValue > value) { minValue = value; @@ -278,7 +276,7 @@ public Object reset() { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { Integer value = (Integer) data; if (minValue == null || minValue > value) { minValue = value; @@ -287,23 +285,28 @@ public synchronized Object processRemove(Object data) { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("MinValue", minValue); return state; } @Override - public void restoreState(Map state) { + public void restore(Map state) { minValue = (Integer) state.get("MinValue"); } protected Object currentValue() { return minValue; } + + @Override + public boolean canDestroy() { + return minValue == null; + } } - class MinForeverAttributeAggregatorLong extends MinForeverAttributeAggregator { + class MinForeverAttributeAggregatorStateLong extends MinAggregatorState { private final Attribute.Type type = Attribute.Type.LONG; private volatile Long minValue = null; @@ -313,7 +316,7 @@ public Attribute.Type getReturnType() { } @Override - public synchronized Object processAdd(Object data) { + public Object processAdd(Object data) { Long value = (Long) data; if (minValue == null || minValue > value) { minValue = value; @@ -327,7 +330,7 @@ public Object reset() { } @Override - public synchronized Object processRemove(Object data) { + public Object processRemove(Object data) { Long value = (Long) data; if (minValue == null || minValue > value) { minValue = value; @@ -336,14 +339,19 @@ public synchronized Object processRemove(Object data) { } @Override - public Map currentState() { + public boolean canDestroy() { + return minValue == null; + } + + @Override + public Map snapshot() { Map state = new HashMap<>(); state.put("MinValue", minValue); return state; } @Override - public void restoreState(Map state) { + public void restore(Map state) { minValue = (Long) state.get("MinValue"); } @@ -352,4 +360,14 @@ protected Object currentValue() { } } + abstract class MinAggregatorState extends State { + public abstract Object processAdd(Object data); + + public abstract Object processRemove(Object data); + + public abstract Object reset(); + + protected abstract Object currentValue(); + } + } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/OrAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/OrAttributeAggregatorExecutor.java similarity index 62% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/OrAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/OrAttributeAggregatorExecutor.java index e47d1f6589..edf021c02b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/OrAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/OrAttributeAggregatorExecutor.java @@ -27,13 +27,15 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.HashMap; import java.util.Map; /** - * {@link AttributeAggregator} to calculate sum based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate sum based on an event attribute. */ @Extension( name = "or", @@ -57,10 +59,10 @@ ) } ) -public class OrAttributeAggregator extends AttributeAggregator { +public class OrAttributeAggregatorExecutor extends + AttributeAggregatorExecutor { private static Attribute.Type type = Attribute.Type.BOOL; - private int trueEventsCount = 0; /** * The initialization method for FunctionExecutor @@ -68,18 +70,20 @@ public class OrAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link OrAttributeAggregator} configuration reader. + * @param configReader this hold the {@link OrAttributeAggregatorExecutor} configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("And aggregator has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length + " parameters provided"); } + return () -> new AggregatorState(); } public Attribute.Type getReturnType() { @@ -87,65 +91,69 @@ public Attribute.Type getReturnType() { } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, AggregatorState state) { if ((boolean) data) { - trueEventsCount++; + state.trueEventsCount++; } - return computeLogicalOperation(); + return computeLogicalOperation(state); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, AggregatorState state) { for (Object object : data) { if ((boolean) object) { - trueEventsCount++; + state.trueEventsCount++; } } - return computeLogicalOperation(); + return computeLogicalOperation(state); } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, AggregatorState state) { if ((boolean) data) { - trueEventsCount--; + state.trueEventsCount--; } - return computeLogicalOperation(); + return computeLogicalOperation(state); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, AggregatorState state) { for (Object object : data) { if ((boolean) object) { - trueEventsCount--; + state.trueEventsCount--; } } - return computeLogicalOperation(); - } - - private boolean computeLogicalOperation() { - return trueEventsCount > 0; + return computeLogicalOperation(state); } @Override - public Object reset() { - trueEventsCount = 0; + public Object reset(AggregatorState state) { + state.trueEventsCount = 0; return false; } - @Override - public boolean canDestroy() { - return trueEventsCount == 0; + private boolean computeLogicalOperation(AggregatorState state) { + return state.trueEventsCount > 0; } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("trueEventsCount", trueEventsCount); - return state; - } + class AggregatorState extends State { + private int trueEventsCount = 0; - @Override - public void restoreState(Map state) { - trueEventsCount = (int) state.get("trueEventsCount"); + @Override + public boolean canDestroy() { + return trueEventsCount == 0; + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("TrueEventsCount", trueEventsCount); + return state; + } + + @Override + public void restore(Map state) { + trueEventsCount = (int) state.get("TrueEventsCount"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregatorExecutor.java similarity index 70% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregatorExecutor.java index 615b6b7f5f..acaea2c1d5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregatorExecutor.java @@ -28,6 +28,8 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.Arrays; @@ -35,7 +37,7 @@ import java.util.Map; /** - * {@link AttributeAggregator} to calculate standard deviation based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate standard deviation based on an event attribute. */ @Extension( name = "stdDev", @@ -57,8 +59,9 @@ "based on their arrival and expiry." ) ) -public class StdDevAttributeAggregator extends AttributeAggregator { - private StdDevAttributeAggregator stdDevOutputAttributeAggregator; +public class StdDevAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { + private Attribute.Type returnType; /** * The initialization method for FunctionExecutor @@ -66,106 +69,85 @@ public class StdDevAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link StdDevAttributeAggregator} configuration reader. + * @param configReader this hold the {@link StdDevAttributeAggregatorExecutor} configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("stdDev aggregator has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length + " parameters provided"); } - + returnType = Attribute.Type.DOUBLE; Attribute.Type type = attributeExpressionExecutors[0].getReturnType(); + return () -> { + switch (type) { + case INT: + return new StdDevAttributeAggregatorStateInt(); + case LONG: + return new StdDevAttributeAggregatorStateLong(); + case FLOAT: + return new StdDevAttributeAggregatorStateFloat(); + case DOUBLE: + return new StdDevAttributeAggregatorStateDouble(); + default: + throw new OperationNotSupportedException("stdDev not supported for " + returnType); + } + }; - switch (type) { - case INT: - stdDevOutputAttributeAggregator = new StdDevAttributeAggregatorInt(); - break; - case LONG: - stdDevOutputAttributeAggregator = new StdDevAttributeAggregatorLong(); - break; - case FLOAT: - stdDevOutputAttributeAggregator = new StdDevAttributeAggregatorFloat(); - break; - case DOUBLE: - stdDevOutputAttributeAggregator = new StdDevAttributeAggregatorDouble(); - break; - default: - throw new OperationNotSupportedException("stdDev not supported for " + type); - } } @Override public Attribute.Type getReturnType() { - return stdDevOutputAttributeAggregator.getReturnType(); + return returnType; } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, AggregatorState state) { if (data == null) { - return stdDevOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return stdDevOutputAttributeAggregator.processAdd(data); + return state.processAdd(data); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, AggregatorState state) { return new IllegalStateException("stdDev cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, AggregatorState state) { if (data == null) { - return stdDevOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return stdDevOutputAttributeAggregator.processRemove(data); + return state.processRemove(data); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, AggregatorState state) { return new IllegalStateException("stdDev cannot process data array, but found " + Arrays.deepToString(data)); } - protected Object currentValue() { - return null; - } - @Override - public boolean canDestroy() { - return stdDevOutputAttributeAggregator.canDestroy(); - } - - @Override - public Object reset() { - return stdDevOutputAttributeAggregator.reset(); - } - - @Override - public Map currentState() { - return stdDevOutputAttributeAggregator.currentState(); - } - - @Override - public void restoreState(Map state) { - stdDevOutputAttributeAggregator.restoreState(state); + public Object reset(AggregatorState state) { + return state.reset(); } /** * Standard deviation abstrct aggregator for Double values */ - private abstract class StdDevAbstractAttributeAggregatorDouble extends StdDevAttributeAggregator { - private final Attribute.Type type = Attribute.Type.DOUBLE; + abstract class AggregatorState extends State { + private double mean, stdDeviation, sum; private int count = 0; - @Override - public Attribute.Type getReturnType() { - return type; - } + public abstract Object processRemove(Object data); + + public abstract Object processAdd(Object data); public Object processAdd(double value) { // See here for the algorithm: http://www.johndcook.com/blog/standard_deviation/ @@ -204,7 +186,6 @@ public Object processRemove(double value) { return Math.sqrt(stdDeviation / count); } - @Override public Object reset() { sum = mean = 0.0; stdDeviation = 0.0; @@ -218,7 +199,7 @@ public boolean canDestroy() { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("Sum", sum); state.put("Mean", mean); @@ -228,7 +209,7 @@ public Map currentState() { } @Override - public void restoreState(Map state) { + public void restore(Map state) { sum = (Long) state.get("Sum"); mean = (Long) state.get("Mean"); stdDeviation = (Long) state.get("stdDeviation"); @@ -248,7 +229,7 @@ protected Object currentValue() { /** * Standard deviation aggregator for Double values */ - private class StdDevAttributeAggregatorDouble extends StdDevAbstractAttributeAggregatorDouble { + private class StdDevAttributeAggregatorStateDouble extends AggregatorState { @Override public Object processAdd(Object data) { return processAdd(((Double) data).doubleValue()); @@ -263,7 +244,7 @@ public Object processRemove(Object data) { /** * Standard deviation aggregator for Float values */ - private class StdDevAttributeAggregatorFloat extends StdDevAbstractAttributeAggregatorDouble { + private class StdDevAttributeAggregatorStateFloat extends AggregatorState { @Override public Object processAdd(Object data) { return processAdd(((Float) data).doubleValue()); @@ -278,7 +259,7 @@ public Object processRemove(Object data) { /** * Standard deviation aggregator for Integer values */ - private class StdDevAttributeAggregatorInt extends StdDevAbstractAttributeAggregatorDouble { + private class StdDevAttributeAggregatorStateInt extends AggregatorState { @Override public Object processAdd(Object data) { return processAdd(((Integer) data).doubleValue()); @@ -293,7 +274,7 @@ public Object processRemove(Object data) { /** * Standard deviation aggregator for Long values */ - private class StdDevAttributeAggregatorLong extends StdDevAbstractAttributeAggregatorDouble { + private class StdDevAttributeAggregatorStateLong extends AggregatorState { @Override public Object processAdd(Object data) { return processAdd(((Long) data).doubleValue()); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/SumAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/SumAttributeAggregatorExecutor.java similarity index 72% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/SumAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/SumAttributeAggregatorExecutor.java index 9aac305b2b..1862f5b30e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/SumAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/SumAttributeAggregatorExecutor.java @@ -27,6 +27,8 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.Arrays; @@ -34,7 +36,7 @@ import java.util.Map; /** - * {@link AttributeAggregator} to calculate sum based on an event attribute. + * {@link AttributeAggregatorExecutor} to calculate sum based on an event attribute. */ @Extension( name = "sum", @@ -59,9 +61,10 @@ ) } ) -public class SumAttributeAggregator extends AttributeAggregator { +public class SumAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { - private SumAttributeAggregator sumOutputAttributeAggregator; + private Attribute.Type returnType; /** * The initialization method for FunctionExecutor @@ -69,13 +72,14 @@ public class SumAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link SumAttributeAggregator} configuration reader. + * @param configReader this hold the {@link SumAttributeAggregatorExecutor} configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("Sum aggregator has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length @@ -83,81 +87,76 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, Processin } Attribute.Type type = attributeExpressionExecutors[0].getReturnType(); switch (type) { - case FLOAT: - sumOutputAttributeAggregator = new SumAttributeAggregatorFloat(); - break; case INT: - sumOutputAttributeAggregator = new SumAttributeAggregatorInt(); - break; case LONG: - sumOutputAttributeAggregator = new SumAttributeAggregatorLong(); + returnType = Attribute.Type.LONG; break; + case FLOAT: case DOUBLE: - sumOutputAttributeAggregator = new SumAttributeAggregatorDouble(); + returnType = Attribute.Type.DOUBLE; break; default: - throw new OperationNotSupportedException("Sum not supported for " + type); + throw new OperationNotSupportedException("Sum not supported for " + returnType); } + return new StateFactory() { + @Override + public AggregatorState createNewState() { + switch (type) { + case FLOAT: + return new AggregatorStateFloat(); + case INT: + return new AggregatorStateInt(); + case LONG: + return new AggregatorStateLong(); + case DOUBLE: + return new AggregatorStateDouble(); + default: + throw new OperationNotSupportedException("Sum not supported for " + returnType); + } + } + }; + } public Attribute.Type getReturnType() { - return sumOutputAttributeAggregator.getReturnType(); + return returnType; } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, AggregatorState state) { if (data == null) { - return sumOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return sumOutputAttributeAggregator.processAdd(data); + return state.processAdd(data); } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, AggregatorState state) { // will not occur return new IllegalStateException("Sin cannot process data array, but found " + Arrays.deepToString(data)); } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, AggregatorState state) { if (data == null) { - return sumOutputAttributeAggregator.currentValue(); + return state.currentValue(); } - return sumOutputAttributeAggregator.processRemove(data); + return state.processRemove(data); } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, AggregatorState state) { // will not occur return new IllegalStateException("Sin cannot process data array, but found " + Arrays.deepToString(data)); } - protected Object currentValue() { - return null; - } - @Override - public boolean canDestroy() { - return sumOutputAttributeAggregator.canDestroy(); + public Object reset(AggregatorState state) { + return state.reset(); } - @Override - public Object reset() { - return sumOutputAttributeAggregator.reset(); - } - - @Override - public Map currentState() { - return sumOutputAttributeAggregator.currentState(); - } - - @Override - public void restoreState(Map state) { - sumOutputAttributeAggregator.restoreState(state); - } - - class SumAttributeAggregatorDouble extends SumAttributeAggregator { + class AggregatorStateDouble extends AggregatorState { private final Attribute.Type type = Attribute.Type.DOUBLE; private double sum = 0.0; @@ -196,6 +195,7 @@ public Object processRemove(double data) { @Override public Object reset() { sum = 0.0; + count = 0; return null; } @@ -205,7 +205,7 @@ public boolean canDestroy() { } @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("Sum", sum); state.put("Count", count); @@ -213,7 +213,7 @@ public Map currentState() { } @Override - public void restoreState(Map state) { + public void restore(Map state) { sum = (double) state.get("Sum"); count = (long) state.get("Count"); } @@ -228,7 +228,7 @@ protected Object currentValue() { } - class SumAttributeAggregatorFloat extends SumAttributeAggregatorDouble { + class AggregatorStateFloat extends AggregatorStateDouble { @Override public Object processAdd(Object data) { @@ -248,7 +248,7 @@ public Object processRemove(Object data) { } - class SumAttributeAggregatorLong extends SumAttributeAggregator { + class AggregatorStateLong extends AggregatorState { private final Attribute.Type type = Attribute.Type.LONG; private long sum = 0L; @@ -288,6 +288,7 @@ public Object processRemove(double data) { public Object reset() { sum = 0L; + count = 0; return sum; } @@ -298,7 +299,7 @@ public boolean canDestroy() { @Override - public Map currentState() { + public Map snapshot() { Map state = new HashMap<>(); state.put("Sum", sum); state.put("Count", count); @@ -306,7 +307,7 @@ public Map currentState() { } @Override - public void restoreState(Map state) { + public void restore(Map state) { sum = (long) state.get("Sum"); count = (long) state.get("Count"); } @@ -321,7 +322,7 @@ protected Object currentValue() { } - class SumAttributeAggregatorInt extends SumAttributeAggregatorLong { + class AggregatorStateInt extends AggregatorStateLong { @Override public Object processAdd(Object data) { @@ -335,4 +336,15 @@ public Object processRemove(Object data) { } + abstract class AggregatorState extends State { + + protected abstract Object currentValue(); + + public abstract Object processAdd(Object data); + + public abstract Object processRemove(Object data); + + public abstract Object reset(); + + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/UnionSetAttributeAggregator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/UnionSetAttributeAggregatorExecutor.java similarity index 65% rename from modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/UnionSetAttributeAggregator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/UnionSetAttributeAggregatorExecutor.java index 447aea5957..9e2ac663bd 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/UnionSetAttributeAggregator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/aggregator/UnionSetAttributeAggregatorExecutor.java @@ -27,6 +27,8 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; 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 java.util.HashMap; @@ -35,7 +37,7 @@ import java.util.Set; /** - * {@link AttributeAggregator} to return a union of an aggregation of sets. + * {@link AttributeAggregatorExecutor} to return a union of an aggregation of sets. */ @Extension( name = "unionSet", @@ -62,13 +64,8 @@ "stock symbols received during a sliding window of 10 seconds." ) ) -public class UnionSetAttributeAggregator extends AttributeAggregator { - - /** - * This map aggregates the count per each distinct element - */ - private Map counter = null; - private Set set = new HashSet(); +public class UnionSetAttributeAggregatorExecutor + extends AttributeAggregatorExecutor { /** * The initialization method for FunctionExecutor @@ -76,13 +73,14 @@ public class UnionSetAttributeAggregator extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link CountAttributeAggregator} configuration reader. + * @param configReader this hold the {@link CountAttributeAggregatorExecutor} configuration reader. * @param siddhiQueryContext Siddhi query runtime context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new OperationNotSupportedException("unionSet aggregator has to have exactly 1 parameter, currently " + attributeExpressionExecutors.length + " parameters provided"); @@ -91,9 +89,8 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, Processin throw new OperationNotSupportedException("Parameter passed to unionSet aggregator should be of type" + " object but found: " + attributeExpressionExecutors[0].getReturnType()); } - if (processingMode == ProcessingMode.SLIDE || outputExpectsExpiredEvents) { - counter = new HashMap<>(); - } + return () -> new AggregatorState(processingMode, outputExpectsExpiredEvents); + } public Attribute.Type getReturnType() { @@ -101,87 +98,104 @@ public Attribute.Type getReturnType() { } @Override - public Object processAdd(Object data) { + public Object processAdd(Object data, AggregatorState state) { Set inputSet = (Set) data; for (Object o : inputSet) { - set.add(o); - if (counter != null) { - Integer currentCount = counter.get(o); + state.set.add(o); + if (state.counter != null) { + Integer currentCount = state.counter.get(o); if (currentCount == null) { - counter.put(o, 1); + state.counter.put(o, 1); } else { - counter.put(o, currentCount + 1); + state.counter.put(o, currentCount + 1); } } } // Creating a new set object as the returned set reference is kept until the aggregated values are // inserted into the store Set returnSet = new HashSet(); - returnSet.addAll(set); + returnSet.addAll(state.set); return returnSet; } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, AggregatorState state) { //UnionSet can have only one input parameter, hence this will not be invoked. return null; } @Override - public Object processRemove(Object data) { + public Object processRemove(Object data, AggregatorState state) { Set newSet = (Set) data; for (Object o : newSet) { - if (counter != null) { - Integer currentCount = counter.get(o); + if (state.counter != null) { + Integer currentCount = state.counter.get(o); if (currentCount == null) { //means o does not exist in the counter map or in the set hence doing nothing } else if (currentCount == 0) { throw new IllegalStateException("Error occurred when removing element from " + "union-set for element: " + o.toString()); } else if (currentCount == 1) { - set.remove(o); + state.set.remove(o); } else { - counter.put(o, currentCount - 1); + state.counter.put(o, currentCount - 1); } } else { - set.remove(o); + state.set.remove(o); } } Set returnSet = new HashSet(); - returnSet.addAll(set); + returnSet.addAll(state.set); return returnSet; } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, AggregatorState state) { //UnionSet can have only one input parameter, hence this will not be invoked. return null; } @Override - public Object reset() { - set.clear(); - counter.clear(); + public Object reset(AggregatorState state) { + state.set.clear(); + if (state.counter != null) { + state.counter.clear(); + } Set returnSet = new HashSet(); return returnSet; // returning an empty set. } - @Override - public boolean canDestroy() { - return set.size() == 0; - } + class AggregatorState extends State { - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("Set", set); - state.put("Counter", counter); - return state; - } + /** + * This map aggregates the count per each distinct element + */ + private Map counter = null; + private Set set = new HashSet(); - @Override - public void restoreState(Map state) { - set = (Set) state.get("Set"); - counter = (Map) state.get("Counter"); + public AggregatorState(ProcessingMode processingMode, boolean outputExpectsExpiredEvents) { + if (processingMode == ProcessingMode.SLIDE || outputExpectsExpiredEvents) { + counter = new HashMap<>(); + } + } + + @Override + public boolean canDestroy() { + return set.isEmpty(); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("Set", set); + state.put("Counter", counter); + return state; + } + + @Override + public void restore(Map state) { + set = (Set) state.get("Set"); + counter = (Map) state.get("Counter"); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/AttributeProcessor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/AttributeProcessor.java index 4b4f9e89f0..35889f221f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/AttributeProcessor.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/AttributeProcessor.java @@ -41,13 +41,6 @@ public void process(ComplexEvent event) { event.setOutputData(expressionExecutor.execute(event), outputPosition); } - public AttributeProcessor cloneProcessor(String key) { - AttributeProcessor attributeProcessor = new AttributeProcessor(expressionExecutor.cloneExecutor(key)); - attributeProcessor.setOutputPosition(this.outputPosition); - return attributeProcessor; - - } - public int getOutputPosition() { return outputPosition; } @@ -60,7 +53,4 @@ public ExpressionExecutor getExpressionExecutor() { return expressionExecutor; } - public void clean() { - expressionExecutor.clean(); - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/executor/AbstractAggregationAttributeExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/executor/AbstractAggregationAttributeExecutor.java deleted file mode 100644 index 429328c09a..0000000000 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/executor/AbstractAggregationAttributeExecutor.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package io.siddhi.core.query.selector.attribute.processor.executor; - -import io.siddhi.core.config.SiddhiQueryContext; -import io.siddhi.core.executor.ExpressionExecutor; -import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator; -import io.siddhi.core.util.snapshot.Snapshotable; -import io.siddhi.query.api.definition.Attribute; - -/** - * Abstract class to represent attribute aggregations. - */ -public abstract class AbstractAggregationAttributeExecutor implements ExpressionExecutor, Snapshotable { - protected AttributeAggregator attributeAggregator; - protected ExpressionExecutor[] attributeExpressionExecutors; - protected SiddhiQueryContext siddhiQueryContext; - protected int size; - private String elementId = null; - - public AbstractAggregationAttributeExecutor(AttributeAggregator attributeAggregator, - ExpressionExecutor[] attributeExpressionExecutors, - SiddhiQueryContext siddhiQueryContext) { - this.siddhiQueryContext = siddhiQueryContext; - this.attributeExpressionExecutors = attributeExpressionExecutors; - this.attributeAggregator = attributeAggregator; - this.size = attributeExpressionExecutors.length; - if (elementId == null) { - elementId = "AbstractAggregationAttributeExecutor-" + - this.siddhiQueryContext.getSiddhiAppContext().getElementIdGenerator().createNewId(); - } - this.siddhiQueryContext.getSiddhiAppContext().getSnapshotService().addSnapshotable( - siddhiQueryContext.getName(), this); - } - - @Override - public Attribute.Type getReturnType() { - return attributeAggregator.getReturnType(); - } - - @Override - public String getElementId() { - return elementId; - } - - @Override - public void clean() { - for (ExpressionExecutor expressionExecutor : attributeExpressionExecutors) { - expressionExecutor.clean(); - } - attributeAggregator.clean(); - siddhiQueryContext.getSiddhiAppContext().getSnapshotService().removeSnapshotable( - siddhiQueryContext.getName(), this); - } -} - diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/executor/AggregationAttributeExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/executor/AggregationAttributeExecutor.java deleted file mode 100644 index cab51caa3c..0000000000 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/executor/AggregationAttributeExecutor.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package io.siddhi.core.query.selector.attribute.processor.executor; - -import io.siddhi.core.config.SiddhiQueryContext; -import io.siddhi.core.event.ComplexEvent; -import io.siddhi.core.executor.ExpressionExecutor; -import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator; - -import java.util.Map; - -/** - * Executor for attribute aggregations. - */ -public class AggregationAttributeExecutor extends AbstractAggregationAttributeExecutor { - - public AggregationAttributeExecutor(AttributeAggregator attributeAggregator, - ExpressionExecutor[] attributeExpressionExecutors, - SiddhiQueryContext siddhiQueryContext) { - super(attributeAggregator, attributeExpressionExecutors, siddhiQueryContext); - } - - @Override - public Object execute(ComplexEvent event) { - return attributeAggregator.process(event); - } - - public ExpressionExecutor cloneExecutor(String key) { - return new AggregationAttributeExecutor(attributeAggregator.cloneAggregator(key), - attributeExpressionExecutors, siddhiQueryContext); - } - - @Override - public Map currentState() { - return attributeAggregator.currentState(); - } - - @Override - public void restoreState(Map state) { - attributeAggregator.restoreState(state); - } -} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/executor/GroupByAggregationAttributeExecutor.java b/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/executor/GroupByAggregationAttributeExecutor.java deleted file mode 100644 index a354950153..0000000000 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/query/selector/attribute/processor/executor/GroupByAggregationAttributeExecutor.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package io.siddhi.core.query.selector.attribute.processor.executor; - -import io.siddhi.core.config.SiddhiQueryContext; -import io.siddhi.core.event.ComplexEvent; -import io.siddhi.core.executor.ExpressionExecutor; -import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator; -import io.siddhi.core.util.config.ConfigReader; -import io.siddhi.core.util.timestamp.TimestampGenerator; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -/** - * Executor class for aggregations with group by configuration. - */ -public class GroupByAggregationAttributeExecutor extends AbstractAggregationAttributeExecutor { - - private static final ThreadLocal keyThreadLocal = new ThreadLocal(); - private final ConfigReader configReader; - private final TimestampGenerator timestampGenerator; - protected Map aggregatorMap = new HashMap(); - protected Set obsoleteAggregatorKeys = new HashSet<>(); - protected long lastCleanupTimestamp = 0; - - public GroupByAggregationAttributeExecutor(AttributeAggregator attributeAggregator, - ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, SiddhiQueryContext siddhiQueryContext) { - super(attributeAggregator, attributeExpressionExecutors, siddhiQueryContext); - this.configReader = configReader; - timestampGenerator = this.siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator(); - lastCleanupTimestamp = timestampGenerator.currentTime(); - } - - public static ThreadLocal getKeyThreadLocal() { - return keyThreadLocal; - } - - @Override - public Object execute(ComplexEvent event) { - - long currentTime = timestampGenerator.currentTime(); - boolean canClean = false; - if (lastCleanupTimestamp + 5000 < currentTime || obsoleteAggregatorKeys.size() > 25) { - lastCleanupTimestamp = currentTime; - canClean = true; - } - - if (event.getType() == ComplexEvent.Type.RESET) { - Object aOutput = null; - if (canClean) { - for (Map.Entry attributeAggregatorEntry : aggregatorMap.entrySet()) { - aOutput = attributeAggregatorEntry.getValue().process(event); - attributeAggregatorEntry.getValue().clean(); - } - aggregatorMap.clear(); - obsoleteAggregatorKeys.clear(); - } else { - for (Map.Entry attributeAggregatorEntry : aggregatorMap.entrySet()) { - aOutput = attributeAggregatorEntry.getValue().process(event); - } - } - return aOutput; - } - - String key = keyThreadLocal.get(); - AttributeAggregator currentAttributeAggregator = aggregatorMap.get(key); - if (currentAttributeAggregator == null) { - currentAttributeAggregator = attributeAggregator.cloneAggregator(key); - aggregatorMap.put(key, currentAttributeAggregator); - } - Object results = currentAttributeAggregator.process(event); - if (event.getType() == ComplexEvent.Type.EXPIRED && currentAttributeAggregator.canDestroy()) { - obsoleteAggregatorKeys.add(key); - } - if (canClean) { - destroyObsoleteAggregators(); - } - return results; - } - - public ExpressionExecutor cloneExecutor(String key) { - return new GroupByAggregationAttributeExecutor(attributeAggregator.cloneAggregator(key), - attributeExpressionExecutors, configReader, - siddhiQueryContext); - } - - @Override - public Map currentState() { - Map state = new HashMap<>(); - for (Map.Entry entry : aggregatorMap.entrySet()) { - state.put(entry.getKey(), entry.getValue().currentState()); - } - return state; - } - - @Override - public void restoreState(Map state) { - for (HashMap.Entry item : state.entrySet()) { - String key = item.getKey(); - AttributeAggregator aAttributeAggregator = attributeAggregator.cloneAggregator(key); - aAttributeAggregator.restoreState((Map) item.getValue()); - aggregatorMap.put(key, aAttributeAggregator); - } - } - - private void destroyObsoleteAggregators() { - for (String obsoleteKey : obsoleteAggregatorKeys) { - AttributeAggregator attributeAggregator = aggregatorMap.get(obsoleteKey); - if (attributeAggregator != null && attributeAggregator.canDestroy()) { - attributeAggregator.clean(); - aggregatorMap.remove(obsoleteKey); - } - } - obsoleteAggregatorKeys.clear(); - } -} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/StreamJunction.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/StreamJunction.java index 79452a5208..64cea27532 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/StreamJunction.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/StreamJunction.java @@ -26,7 +26,7 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.Event; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.FaultStreamEventConverter; import io.siddhi.core.exception.SiddhiAppCreationException; import io.siddhi.core.stream.input.InputProcessor; @@ -47,10 +47,8 @@ import java.beans.ExceptionListener; import java.lang.reflect.Constructor; import java.util.Arrays; -import java.util.Collections; import java.util.LinkedList; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; /** @@ -66,8 +64,8 @@ public class StreamJunction implements EventBufferHolder { private int batchSize; private int workers = -1; private int bufferSize; - private List receivers = new CopyOnWriteArrayList(); - private List publishers = Collections.synchronizedList(new LinkedList<>()); + private List receivers = new LinkedList<>(); + private List publishers = new LinkedList<>(); private ExecutorService executorService; private boolean async = false; private Disruptor disruptor; @@ -75,7 +73,7 @@ public class StreamJunction implements EventBufferHolder { private ThroughputTracker throughputTracker = null; private boolean isTraceEnabled; private StreamJunction faultStreamJunction = null; - private FaultStreamEventConverter faultStreamEventChunk = null; + private FaultStreamEventConverter faultStreamEventConverter = null; private OnErrorAction onErrorAction = OnErrorAction.LOG; private ExceptionListener exceptionListener; @@ -95,11 +93,9 @@ public StreamJunction(StreamDefinition streamDefinition, ExecutorService executo this.faultStreamJunction = faultStreamJunction; if (faultStreamJunction != null) { StreamDefinition faultStreamDefinition = faultStreamJunction.getStreamDefinition(); - StreamEventPool faultStreamEventPool = new StreamEventPool(0, 0, - faultStreamDefinition.getAttributeList().size(), 5); - faultStreamEventPool.borrowEvent(); - faultStreamEventChunk = new FaultStreamEventConverter(faultStreamEventPool); - + StreamEventFactory faultStreamEventFactory = new StreamEventFactory(0, 0, + faultStreamDefinition.getAttributeList().size()); + faultStreamEventConverter = new FaultStreamEventConverter(faultStreamEventFactory); } try { Annotation asyncAnnotation = AnnotationHelper.getAnnotation(SiddhiConstants.ANNOTATION_ASYNC, @@ -277,7 +273,7 @@ private void sendData(long timeStamp, Object[] data) { /** * Create and start disruptor based on annotations given in the streamDefinition. */ - public synchronized void startProcessing() { + public void startProcessing() { if (!receivers.isEmpty() && async) { for (Constructor constructor : Disruptor.class.getConstructors()) { if (constructor.getParameterTypes().length == 5) { // If new disruptor classes available @@ -315,7 +311,7 @@ public synchronized void startProcessing() { } } - public synchronized void stopProcessing() { + public void stopProcessing() { if (disruptor != null) { disruptor.shutdown(); } else { @@ -327,14 +323,14 @@ public synchronized void stopProcessing() { } } - public synchronized Publisher constructPublisher() { + public Publisher constructPublisher() { Publisher publisher = new Publisher(); publisher.setStreamJunction(this); publishers.add(publisher); return publisher; } - public synchronized void subscribe(Receiver receiver) { + public void subscribe(Receiver receiver) { // To have reverse order at the sequence/pattern processors. if (!receivers.contains(receiver)) { receivers.add(receiver); @@ -461,24 +457,16 @@ private void handleError(Object event, Exception e) { if (faultStreamJunction != null) { StreamEvent streamEvent = null; if (event instanceof ComplexEvent) { - synchronized (this) { - streamEvent = faultStreamEventChunk.convert((ComplexEvent) event, e); - } + streamEvent = faultStreamEventConverter.convert((ComplexEvent) event, e); faultStreamJunction.sendEvent(streamEvent); } else if (event instanceof Event) { - synchronized (this) { - streamEvent = faultStreamEventChunk.convert((Event) event, e); - } + streamEvent = faultStreamEventConverter.convert((Event) event, e); faultStreamJunction.sendEvent(streamEvent); } else if (event instanceof Event[]) { - synchronized (this) { - streamEvent = faultStreamEventChunk.convert((Event[]) event, e); - } + streamEvent = faultStreamEventConverter.convert((Event[]) event, e); faultStreamJunction.sendEvent(streamEvent); } else if (event instanceof List) { - synchronized (this) { - streamEvent = faultStreamEventChunk.convert((List) event, e); - } + streamEvent = faultStreamEventConverter.convert((List) event, e); faultStreamJunction.sendEvent(streamEvent); } } else { @@ -505,10 +493,7 @@ private void handleError(long timeStamp, Object[] data, Exception e) { break; case STREAM: if (faultStreamJunction != null) { - StreamEvent streamEvent = null; - synchronized (this) { - streamEvent = faultStreamEventChunk.convert(timeStamp, data, e); - } + StreamEvent streamEvent = faultStreamEventConverter.convert(timeStamp, data, e); faultStreamJunction.sendEvent(streamEvent); } else { log.error("Error in SiddhiApp '" + siddhiAppContext.getName() + diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/InputEntryValve.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/InputEntryValve.java index dbea1ec9d5..884ee5df3f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/InputEntryValve.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/InputEntryValve.java @@ -40,25 +40,41 @@ public InputEntryValve(SiddhiAppContext siddhiAppContext, InputProcessor inputPr @Override public void send(Event event, int streamIndex) { - barrier.pass(); - inputProcessor.send(event, streamIndex); + barrier.enter(); + try { + inputProcessor.send(event, streamIndex); + } finally { + barrier.exit(); + } } @Override public void send(Event[] events, int streamIndex) { - barrier.pass(); - inputProcessor.send(events, streamIndex); + barrier.enter(); + try { + inputProcessor.send(events, streamIndex); + } finally { + barrier.exit(); + } } @Override public void send(List events, int streamIndex) { - barrier.pass(); - inputProcessor.send(events, streamIndex); + barrier.enter(); + try { + inputProcessor.send(events, streamIndex); + } finally { + barrier.exit(); + } } @Override public void send(long timestamp, Object[] data, int streamIndex) { - barrier.pass(); - inputProcessor.send(timestamp, data, streamIndex); + barrier.enter(); + try { + inputProcessor.send(timestamp, data, streamIndex); + } finally { + barrier.exit(); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/InMemorySource.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/InMemorySource.java index cea5c71ec9..ce75fa9dfd 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/InMemorySource.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/InMemorySource.java @@ -29,8 +29,6 @@ import io.siddhi.core.util.transport.OptionHolder; import org.apache.log4j.Logger; -import java.util.Map; - /** * Implementation of {@link Source} to receive events through in-memory transport. */ @@ -103,13 +101,4 @@ public void resume() { InMemoryBroker.subscribe(subscriber); } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - // no state - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/Source.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/Source.java index dabd182183..2b7c2dbfbd 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/Source.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/Source.java @@ -23,7 +23,6 @@ import io.siddhi.core.util.ExceptionUtil; import io.siddhi.core.util.StringUtil; import io.siddhi.core.util.config.ConfigReader; -import io.siddhi.core.util.snapshot.Snapshotable; import io.siddhi.core.util.transport.BackoffRetryCounter; import io.siddhi.core.util.transport.OptionHolder; import io.siddhi.query.api.definition.StreamDefinition; @@ -39,12 +38,11 @@ * transports. Each source represent a transport type. Whenever Siddhi need to support a new transport, a new Event * source should be implemented. */ -public abstract class Source implements Snapshotable { +public abstract class Source { private static final Logger LOG = Logger.getLogger(Source.class); private String type; private SourceMapper mapper; private StreamDefinition streamDefinition; - private String elementId; private SiddhiAppContext siddhiAppContext; private AtomicBoolean isTryingToConnect = new AtomicBoolean(false); @@ -66,7 +64,6 @@ public final void init(String sourceType, OptionHolder transportOptionHolder, So sourceHandler, mapperConfigReader, siddhiAppContext); this.mapper = sourceMapper; this.streamDefinition = streamDefinition; - this.elementId = siddhiAppContext.getElementIdGenerator().createNewId(); this.siddhiAppContext = siddhiAppContext; init(sourceMapper, transportOptionHolder, transportPropertyNames, configReader, siddhiAppContext); scheduledExecutorService = siddhiAppContext.getScheduledExecutorService(); @@ -174,16 +171,6 @@ public void shutdown() { } } - @Override - public final String getElementId() { - return elementId; - } - - @Override - public void clean() { - //ignore - } - public String getType() { return type; } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/SourceHandler.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/SourceHandler.java index aa8f8a595d..0797f99883 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/SourceHandler.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/SourceHandler.java @@ -18,49 +18,64 @@ package io.siddhi.core.stream.input.source; +import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.event.Event; import io.siddhi.core.stream.input.InputHandler; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.query.api.definition.StreamDefinition; /** * SourceHandler is an optional implementable class that wraps {@link InputHandler}. * It will do optional processing to the events before sending the events to the input handler + * + * @param current state for the Source Holder */ -public abstract class SourceHandler implements InputEventHandlerCallback, Snapshotable { +public abstract class SourceHandler implements InputEventHandlerCallback { - private String elementId; private InputHandler inputHandler; + private StateHolder stateHolder; + private String id; - final void initSourceHandler(String siddhiAppName, SourceSyncCallback sourceSyncCallback, String elementId, - StreamDefinition streamDefinition) { - this.elementId = elementId; - init(siddhiAppName, sourceSyncCallback, elementId, streamDefinition); + final void initSourceHandler(String siddhiAppName, SourceSyncCallback sourceSyncCallback, + StreamDefinition streamDefinition, SiddhiAppContext siddhiAppContext) { + StateFactory stateFactory = init(siddhiAppName, sourceSyncCallback, streamDefinition, siddhiAppContext); + id = siddhiAppName + "-" + streamDefinition.getId() + "-" + this.getClass().getName(); + stateHolder = siddhiAppContext.generateStateHolder( + streamDefinition.getId() + "-" + this.getClass().getName(), + stateFactory); } - public abstract void init(String siddhiAppName, SourceSyncCallback sourceSyncCallback, String elementId, - StreamDefinition streamDefinition); + public abstract StateFactory init(String siddhiAppName, SourceSyncCallback sourceSyncCallback, + StreamDefinition streamDefinition, SiddhiAppContext siddhiAppContext); @Override public void sendEvent(Event event, String[] transportSyncProperties) throws InterruptedException { - sendEvent(event, transportSyncProperties, inputHandler); + S state = stateHolder.getState(); + try { + sendEvent(event, transportSyncProperties, state, inputHandler); + } finally { + stateHolder.returnState(state); + } } @Override public void sendEvents(Event[] events, String[] transportSyncProperties) throws InterruptedException { - sendEvent(events, transportSyncProperties, inputHandler); + S state = stateHolder.getState(); + try { + sendEvent(events, transportSyncProperties, state, inputHandler); + } finally { + stateHolder.returnState(state); + } } - public abstract void sendEvent(Event event, String[] transportSyncProperties, InputHandler inputHandler) + public abstract void sendEvent(Event event, String[] transportSyncProperties, S state, InputHandler inputHandler) throws InterruptedException; - public abstract void sendEvent(Event[] events, String[] transportSyncProperties, InputHandler inputHandler) + public abstract void sendEvent(Event[] events, String[] transportSyncProperties, S state, InputHandler inputHandler) throws InterruptedException; - public String getElementId() { - return elementId; - } - public InputHandler getInputHandler() { return inputHandler; } @@ -69,8 +84,7 @@ public void setInputHandler(InputHandler inputHandler) { this.inputHandler = inputHandler; } - @Override - public void clean() { - //ignore + public String getId() { + return id; } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/SourceMapper.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/SourceMapper.java index 915030306d..5d806aa2b7 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/SourceMapper.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/input/source/SourceMapper.java @@ -60,8 +60,8 @@ public final void init(StreamDefinition streamDefinition, String mapType, Option this.sourceType = sourceType; this.transportMappings = transportMappings; if (sourceHandler != null) { - sourceHandler.initSourceHandler(siddhiAppContext.getName(), sourceSyncCallback, - siddhiAppContext.getElementIdGenerator().createNewId(), streamDefinition); + sourceHandler.initSourceHandler(siddhiAppContext.getName(), sourceSyncCallback, streamDefinition, + siddhiAppContext); } this.sourceHandler = sourceHandler; this.siddhiAppContext = siddhiAppContext; @@ -105,7 +105,6 @@ public final void setInputHandler(InputHandler inputHandler) { } else { inputEventHandlerCallback = new PassThroughSourceHandler(inputHandler); } - LatencyTracker mapperLatencyTracker = null; this.inputEventHandler = new InputEventHandler(inputHandler, transportMappings, trpProperties, trpSyncProperties, sourceType, mapperLatencyTracker, siddhiAppContext, inputEventHandlerCallback); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/StreamCallback.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/StreamCallback.java index fcbe603d01..1fec58d8df 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/StreamCallback.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/StreamCallback.java @@ -128,11 +128,11 @@ public void receiveEvents(Event[] events) { public abstract void receive(Event[] events); - public synchronized void startProcessing() { + public void startProcessing() { } - public synchronized void stopProcessing() { + public void stopProcessing() { } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/InMemorySink.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/InMemorySink.java index 87afd4f9a7..55062e8fb3 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/InMemorySink.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/InMemorySink.java @@ -33,8 +33,6 @@ import io.siddhi.query.api.definition.StreamDefinition; import org.apache.log4j.Logger; -import java.util.Map; - /** * Implementation of {@link Sink} which represents in-memory transport. This implementation can send incoming objects * to in-memory transport within JVM. @@ -99,13 +97,4 @@ public void publish(Object payload, DynamicOptions dynamicOptions) throws Connec } } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - // no state - } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/LogSink.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/LogSink.java index afafc24b72..d3dc50225d 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/LogSink.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/LogSink.java @@ -31,7 +31,6 @@ import org.apache.log4j.Logger; import java.util.Arrays; -import java.util.Map; /** * Implementation of {@link Sink} which can be used as a logger. This will log the output events in the output stream @@ -166,16 +165,6 @@ public void destroy() { // do nothing } - @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - // no state - } - private enum LogPriority { INFO, DEBUG, diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/Sink.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/Sink.java index 3f763e27b7..1a07761b9a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/Sink.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/Sink.java @@ -27,7 +27,6 @@ import io.siddhi.core.util.StringUtil; import io.siddhi.core.util.config.ConfigReader; import io.siddhi.core.util.parser.helper.QueryParserHelper; -import io.siddhi.core.util.snapshot.Snapshotable; import io.siddhi.core.util.statistics.LatencyTracker; import io.siddhi.core.util.statistics.ThroughputTracker; import io.siddhi.core.util.transport.BackoffRetryCounter; @@ -46,7 +45,7 @@ * This is a Sink type. these let users to publish events according to * some type. this type can either be local, jms or ws (or any custom extension) */ -public abstract class Sink implements SinkListener, Snapshotable { +public abstract class Sink implements SinkListener { private static final Logger LOG = Logger.getLogger(Sink.class); protected AtomicBoolean isTryingToConnect = new AtomicBoolean(false); @@ -55,7 +54,6 @@ public abstract class Sink implements SinkListener, Snapshotable { private SinkMapper mapper; private SinkHandler handler; private DistributedTransport.ConnectionCallback connectionCallback = null; - private String elementId; private SiddhiAppContext siddhiAppContext; private OnErrorAction onErrorAction; private BackoffRetryCounter backoffRetryCounter = new BackoffRetryCounter(); @@ -72,7 +70,6 @@ public final void init(StreamDefinition streamDefinition, String type, OptionHol ConfigReader mapperConfigReader, SiddhiAppContext siddhiAppContext) { this.streamDefinition = streamDefinition; this.type = type; - this.elementId = siddhiAppContext.getElementIdGenerator().createNewId(); this.siddhiAppContext = siddhiAppContext; this.onErrorAction = OnErrorAction.valueOf(transportOptionHolder .getOrCreateOption(SiddhiConstants.ANNOTATION_ELEMENT_ON_ERROR, "LOG") @@ -93,8 +90,8 @@ public final void init(StreamDefinition streamDefinition, String type, OptionHol this.mapper = sinkMapper; } if (sinkHandler != null) { - sinkHandler.initSinkHandler(siddhiAppContext.getElementIdGenerator().createNewId(), streamDefinition, - new SinkHandlerCallback(sinkMapper)); + sinkHandler.initSinkHandler(siddhiAppContext.getName(), streamDefinition, + new SinkHandlerCallback(sinkMapper), siddhiAppContext); this.handler = sinkHandler; } @@ -112,7 +109,6 @@ public final void initOnlyTransport(StreamDefinition streamDefinition, OptionHol this.type = type; this.streamDefinition = streamDefinition; this.connectionCallback = connectionCallback; - this.elementId = siddhiAppContext.getElementIdGenerator().createNewId(); this.siddhiAppContext = siddhiAppContext; init(streamDefinition, transportOptionHolder, sinkConfigReader, siddhiAppContext); scheduledExecutorService = siddhiAppContext.getScheduledExecutorService(); @@ -249,16 +245,6 @@ public void shutdown() { } } - @Override - public final String getElementId() { - return elementId; - } - - @Override - public void clean() { - //ignore - } - void setTrpDynamicOptions(ThreadLocal trpDynamicOptions) { this.trpDynamicOptions = trpDynamicOptions; } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/SinkHandler.java b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/SinkHandler.java index 05f5e633b9..794861bbc0 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/SinkHandler.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/stream/output/sink/SinkHandler.java @@ -18,47 +18,66 @@ package io.siddhi.core.stream.output.sink; +import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.event.Event; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.query.api.definition.StreamDefinition; /** * SinkHandler is an optional interface before {@link SinkMapper}. * It will do optional processing to the events before sending the events to the desired mapper + * + * @param current state for the Sink Holder */ -public abstract class SinkHandler implements Snapshotable { +public abstract class SinkHandler { private SinkHandlerCallback sinkHandlerCallback; - private String elementId; + private StateHolder stateHolder; + private String id; - final void initSinkHandler(String elementId, StreamDefinition streamDefinition, - SinkHandlerCallback sinkHandlerCallback) { + final void initSinkHandler(String siddhiAppName, StreamDefinition streamDefinition, + SinkHandlerCallback sinkHandlerCallback, + SiddhiAppContext siddhiAppContext) { this.sinkHandlerCallback = sinkHandlerCallback; - this.elementId = elementId; - init(elementId, streamDefinition, sinkHandlerCallback); + StateFactory stateFactory = init(streamDefinition, sinkHandlerCallback); + id = siddhiAppName + "-" + streamDefinition.getId() + "-" + this.getClass().getName(); + stateHolder = siddhiAppContext.generateStateHolder( + streamDefinition.getId() + "-" + this.getClass().getName(), + stateFactory); } - public abstract void init(String elementId, StreamDefinition streamDefinition, - SinkHandlerCallback sinkHandlerCallback); + public abstract StateFactory init(StreamDefinition streamDefinition, + SinkHandlerCallback sinkHandlerCallback); public void handle(Event event) { - handle(event, sinkHandlerCallback); + S state = stateHolder.getState(); + try { + handle(event, sinkHandlerCallback, state); + } finally { + stateHolder.returnState(state); + } } public void handle(Event[] events) { - handle(events, sinkHandlerCallback); + if (stateHolder != null) { + S state = stateHolder.getState(); + try { + handle(events, sinkHandlerCallback, state); + } finally { + stateHolder.returnState(state); + } + } else { + handle(events, sinkHandlerCallback, null); + } } - public abstract void handle(Event event, SinkHandlerCallback sinkHandlerCallback); - - public abstract void handle(Event[] events, SinkHandlerCallback sinkHandlerCallback); + public abstract void handle(Event event, SinkHandlerCallback sinkHandlerCallback, S state); - public String getElementId() { - return elementId; - } + public abstract void handle(Event[] events, SinkHandlerCallback sinkHandlerCallback, S state); - @Override - public void clean() { - //ignore + public String getId() { + return id; } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/table/InMemoryTable.java b/modules/siddhi-core/src/main/java/io/siddhi/core/table/InMemoryTable.java index 9c9fe10367..d34669fa1a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/table/InMemoryTable.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/table/InMemoryTable.java @@ -24,7 +24,7 @@ import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.exception.ConnectionUnavailableException; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -39,8 +39,9 @@ import io.siddhi.core.util.parser.EventHolderPasser; import io.siddhi.core.util.parser.ExpressionParser; import io.siddhi.core.util.parser.OperatorParser; -import io.siddhi.core.util.snapshot.Snapshotable; import io.siddhi.core.util.snapshot.state.SnapshotStateList; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.query.api.definition.TableDefinition; import io.siddhi.query.api.execution.query.output.stream.UpdateSet; import io.siddhi.query.api.expression.Expression; @@ -54,29 +55,23 @@ /** * In-memory event table implementation of SiddhiQL. */ -public class InMemoryTable extends Table implements Snapshotable { +public class InMemoryTable extends Table { private TableDefinition tableDefinition; private StreamEventCloner tableStreamEventCloner; private ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); - private EventHolder eventHolder; - private String elementId; - private SiddhiAppContext siddhiAppContext; + private StateHolder stateHolder; @Override - public void init(TableDefinition tableDefinition, StreamEventPool storeEventPool, + public void init(TableDefinition tableDefinition, StreamEventFactory storeEventPool, StreamEventCloner storeEventCloner, ConfigReader configReader, SiddhiAppContext siddhiAppContext, RecordTableHandler recordTableHandler) { this.tableDefinition = tableDefinition; this.tableStreamEventCloner = storeEventCloner; - this.siddhiAppContext = siddhiAppContext; + EventHolder eventHolder = EventHolderPasser.parse(tableDefinition, storeEventPool, siddhiAppContext); - eventHolder = EventHolderPasser.parse(tableDefinition, storeEventPool, siddhiAppContext); - - if (elementId == null) { - elementId = "InMemoryTable-" + siddhiAppContext.getElementIdGenerator().createNewId(); - } - siddhiAppContext.getSnapshotService().addSnapshotable(tableDefinition.getId(), this); + stateHolder = siddhiAppContext.generateStateHolder(tableDefinition.getId(), + () -> new TableState(eventHolder)); } @Override @@ -86,10 +81,12 @@ public TableDefinition getTableDefinition() { @Override public void add(ComplexEventChunk addingEventChunk) { + readWriteLock.writeLock().lock(); + TableState state = stateHolder.getState(); try { - readWriteLock.writeLock().lock(); - eventHolder.add(addingEventChunk); + state.eventHolder.add(addingEventChunk); } finally { + stateHolder.returnState(state); readWriteLock.writeLock().unlock(); } @@ -97,10 +94,12 @@ public void add(ComplexEventChunk addingEventChunk) { @Override public void delete(ComplexEventChunk deletingEventChunk, CompiledCondition compiledCondition) { + readWriteLock.writeLock().lock(); + TableState state = stateHolder.getState(); try { - readWriteLock.writeLock().lock(); - ((Operator) compiledCondition).delete(deletingEventChunk, eventHolder); + ((Operator) compiledCondition).delete(deletingEventChunk, state.eventHolder); } finally { + stateHolder.returnState(state); readWriteLock.writeLock().unlock(); } } @@ -108,11 +107,13 @@ public void delete(ComplexEventChunk deletingEventChunk, CompiledCon @Override public void update(ComplexEventChunk updatingEventChunk, CompiledCondition compiledCondition, CompiledUpdateSet compiledUpdateSet) { + readWriteLock.writeLock().lock(); + TableState state = stateHolder.getState(); try { - readWriteLock.writeLock().lock(); - ((Operator) compiledCondition).update(updatingEventChunk, eventHolder, + ((Operator) compiledCondition).update(updatingEventChunk, state.eventHolder, (InMemoryCompiledUpdateSet) compiledUpdateSet); } finally { + stateHolder.returnState(state); readWriteLock.writeLock().unlock(); } @@ -123,17 +124,19 @@ public void updateOrAdd(ComplexEventChunk updateOrAddingEventChunk, CompiledCondition compiledCondition, CompiledUpdateSet compiledUpdateSet, AddingStreamEventExtractor addingStreamEventExtractor) { + readWriteLock.writeLock().lock(); + TableState state = stateHolder.getState(); try { - readWriteLock.writeLock().lock(); ComplexEventChunk failedEvents = ((Operator) compiledCondition).tryUpdate( updateOrAddingEventChunk, - eventHolder, + state.eventHolder, (InMemoryCompiledUpdateSet) compiledUpdateSet, addingStreamEventExtractor); if (failedEvents != null) { - eventHolder.add(failedEvents); + state.eventHolder.add(failedEvents); } } finally { + stateHolder.returnState(state); readWriteLock.writeLock().unlock(); } @@ -141,10 +144,12 @@ public void updateOrAdd(ComplexEventChunk updateOrAddingEventChunk, @Override public boolean contains(StateEvent matchingEvent, CompiledCondition compiledCondition) { + readWriteLock.readLock().lock(); + TableState state = stateHolder.getState(); try { - readWriteLock.readLock().lock(); - return ((Operator) compiledCondition).contains(matchingEvent, eventHolder); + return ((Operator) compiledCondition).contains(matchingEvent, state.eventHolder); } finally { + stateHolder.returnState(state); readWriteLock.readLock().unlock(); } @@ -167,10 +172,12 @@ protected void destroy() { @Override public StreamEvent find(CompiledCondition compiledCondition, StateEvent matchingEvent) { + TableState state = stateHolder.getState(); + readWriteLock.readLock().lock(); try { - readWriteLock.readLock().lock(); - return ((Operator) compiledCondition).find(matchingEvent, eventHolder, tableStreamEventCloner); + return ((Operator) compiledCondition).find(matchingEvent, state.eventHolder, tableStreamEventCloner); } finally { + stateHolder.returnState(state); readWriteLock.readLock().unlock(); } } @@ -179,8 +186,13 @@ public StreamEvent find(CompiledCondition compiledCondition, StateEvent matching public CompiledCondition compileCondition(Expression condition, MatchingMetaInfoHolder matchingMetaInfoHolder, List variableExpressionExecutors, Map tableMap, SiddhiQueryContext siddhiQueryContext) { - return OperatorParser.constructOperator(eventHolder, condition, matchingMetaInfoHolder, - variableExpressionExecutors, tableMap, siddhiQueryContext); + TableState state = stateHolder.getState(); + try { + return OperatorParser.constructOperator(state.eventHolder, condition, matchingMetaInfoHolder, + variableExpressionExecutors, tableMap, siddhiQueryContext); + } finally { + stateHolder.returnState(state); + } } @Override @@ -201,26 +213,29 @@ public CompiledUpdateSet compileUpdateSet(UpdateSet updateSet, MatchingMetaInfoH return new InMemoryCompiledUpdateSet(expressionExecutorMap); } + class TableState extends State { - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("EventHolder", eventHolder.getSnapshot()); - return state; - } + private final EventHolder eventHolder; - @Override - public void restoreState(Map state) { - eventHolder.restore((SnapshotStateList) state.get("EventHolder")); - } + public TableState(EventHolder eventHolder) { + this.eventHolder = eventHolder; + } - @Override - public String getElementId() { - return elementId; - } + @Override + public boolean canDestroy() { + return false; + } - @Override - public void clean() { - siddhiAppContext.getSnapshotService().removeSnapshotable(tableDefinition.getId(), this); + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("EventHolder", eventHolder.getSnapshot()); + return state; + } + + @Override + public void restore(Map state) { + eventHolder.restore((SnapshotStateList) state.get("EventHolder")); + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/table/Table.java b/modules/siddhi-core/src/main/java/io/siddhi/core/table/Table.java index 93eff19e89..31f930bd8e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/table/Table.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/table/Table.java @@ -24,7 +24,7 @@ import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.exception.ConnectionUnavailableException; import io.siddhi.core.executor.VariableExpressionExecutor; import io.siddhi.core.query.processor.stream.window.FindableProcessor; @@ -80,7 +80,7 @@ public abstract class Table implements FindableProcessor, MemoryCalculable { private ThroughputTracker throughputTrackerUpdateOrInsert; private ThroughputTracker throughputTrackerContains; - public void initTable(TableDefinition tableDefinition, StreamEventPool storeEventPool, + public void initTable(TableDefinition tableDefinition, StreamEventFactory storeEventPool, StreamEventCloner storeEventCloner, ConfigReader configReader, SiddhiAppContext siddhiAppContext, RecordTableHandler recordTableHandler) { @@ -122,7 +122,7 @@ public void initTable(TableDefinition tableDefinition, StreamEventPool storeEven init(tableDefinition, storeEventPool, storeEventCloner, configReader, siddhiAppContext, recordTableHandler); } - protected abstract void init(TableDefinition tableDefinition, StreamEventPool storeEventPool, + protected abstract void init(TableDefinition tableDefinition, StreamEventFactory storeEventPool, StreamEventCloner storeEventCloner, ConfigReader configReader, SiddhiAppContext siddhiAppContext, RecordTableHandler recordTableHandler); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/EventHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/EventHolder.java index 9814793237..2354b7a268 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/EventHolder.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/EventHolder.java @@ -20,7 +20,7 @@ import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.util.snapshot.state.SnapshotState; +import io.siddhi.core.util.snapshot.state.Snapshot; import io.siddhi.core.util.snapshot.state.SnapshotStateList; /** @@ -30,7 +30,7 @@ public interface EventHolder { void add(ComplexEventChunk addingEventChunk); - SnapshotState getSnapshot(); + Snapshot getSnapshot(); void restore(SnapshotStateList snapshotStatelist); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/IndexEventHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/IndexEventHolder.java index 3bd48d6fcc..779ba4c059 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/IndexEventHolder.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/IndexEventHolder.java @@ -23,12 +23,12 @@ import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.stream.Operation; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.exception.OperationNotSupportedException; import io.siddhi.core.util.SiddhiConstants; import io.siddhi.core.util.snapshot.SnapshotRequest; -import io.siddhi.core.util.snapshot.state.SnapshotState; +import io.siddhi.core.util.snapshot.state.Snapshot; import io.siddhi.core.util.snapshot.state.SnapshotStateList; import io.siddhi.query.api.definition.AbstractDefinition; import io.siddhi.query.api.expression.condition.Compare; @@ -67,7 +67,7 @@ public class IndexEventHolder implements IndexedEventHolder, Serializable { private final String tableName; private final String siddhiAppName; private String primaryKeyAttributes = null; - private StreamEventPool tableStreamEventPool; + private StreamEventFactory tableStreamEventFactory; private StreamEventConverter eventConverter; private Map indexMetaData; private Map multiPrimaryKeyMetaData = new LinkedHashMap<>(); @@ -77,11 +77,11 @@ public class IndexEventHolder implements IndexedEventHolder, Serializable { private boolean forceFullSnapshot = true; private boolean isOperationLogEnabled = true; - public IndexEventHolder(StreamEventPool tableStreamEventPool, StreamEventConverter eventConverter, + public IndexEventHolder(StreamEventFactory tableStreamEventFactory, StreamEventConverter eventConverter, PrimaryKeyReferenceHolder[] primaryKeyReferenceHolders, boolean isPrimaryNumeric, Map indexMetaData, AbstractDefinition tableDefinition, SiddhiAppContext siddhiAppContext) { - this.tableStreamEventPool = tableStreamEventPool; + this.tableStreamEventFactory = tableStreamEventFactory; this.eventConverter = eventConverter; this.primaryKeyReferenceHolders = primaryKeyReferenceHolders; this.indexMetaData = indexMetaData; @@ -157,12 +157,12 @@ public void add(ComplexEventChunk addingEventChunk) { addingEventChunk.reset(); while (addingEventChunk.hasNext()) { ComplexEvent complexEvent = addingEventChunk.next(); - StreamEvent streamEvent = tableStreamEventPool.borrowEvent(); + StreamEvent streamEvent = tableStreamEventFactory.newInstance(); eventConverter.convertComplexEvent(complexEvent, streamEvent); eventsCount++; if (isOperationLogEnabled) { if (!isFullSnapshot()) { - StreamEvent streamEvent2 = tableStreamEventPool.borrowEvent(); + StreamEvent streamEvent2 = tableStreamEventFactory.newInstance(); eventConverter.convertComplexEvent(complexEvent, streamEvent2); operationChangeLog.add(new Operation(ADD, streamEvent2)); } else { @@ -220,7 +220,7 @@ private Object constructPrimaryKey(StreamEvent streamEvent, public void overwrite(StreamEvent streamEvent) { if (isOperationLogEnabled) { if (!isFullSnapshot()) { - StreamEvent streamEvent2 = tableStreamEventPool.borrowEvent(); + StreamEvent streamEvent2 = tableStreamEventFactory.newInstance(); eventConverter.convertComplexEvent(streamEvent, streamEvent2); operationChangeLog.add(new Operation(OVERWRITE, streamEvent2)); } else { @@ -389,7 +389,7 @@ public void deleteAll(Collection storeEventSet) { for (StreamEvent streamEvent : storeEventSet) { if (isOperationLogEnabled) { if (!isFullSnapshot()) { - StreamEvent streamEvent2 = tableStreamEventPool.borrowEvent(); + StreamEvent streamEvent2 = tableStreamEventFactory.newInstance(); eventConverter.convertComplexEvent(streamEvent, streamEvent2); operationChangeLog.add(new Operation(REMOVE, streamEvent)); } else { @@ -617,23 +617,23 @@ private boolean isFullSnapshot() { || SnapshotRequest.isRequestForFullSnapshot(); } - public SnapshotState getSnapshot() { + public Snapshot getSnapshot() { if (isFullSnapshot()) { forceFullSnapshot = false; - return new SnapshotState(this, false); + return new Snapshot(this, false); } else { - SnapshotState snapshot = new SnapshotState(operationChangeLog, true); + Snapshot snapshot = new Snapshot(operationChangeLog, true); operationChangeLog = new ArrayList<>(); return snapshot; } } public void restore(SnapshotStateList snapshotStatelist) { - TreeMap revisions = snapshotStatelist.getSnapshotStates(); - Iterator> itr = revisions.entrySet().iterator(); + TreeMap revisions = snapshotStatelist.getSnapshotStates(); + Iterator> itr = revisions.entrySet().iterator(); this.isOperationLogEnabled = false; while (itr.hasNext()) { - Map.Entry snapshotEntry = itr.next(); + Map.Entry snapshotEntry = itr.next(); if (!snapshotEntry.getValue().isIncrementalSnapshot()) { this.deleteAll(); IndexEventHolder snapshotEventHolder = (IndexEventHolder) snapshotEntry.getValue().getState(); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/ListEventHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/ListEventHolder.java index 4e931f7af0..d7055b2df6 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/ListEventHolder.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/table/holder/ListEventHolder.java @@ -21,7 +21,7 @@ import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.event.stream.holder.SnapshotableStreamEventQueue; import io.siddhi.core.event.stream.holder.StreamEventClonerHolder; @@ -33,13 +33,13 @@ public class ListEventHolder extends SnapshotableStreamEventQueue implements EventHolder { private static final long serialVersionUID = 4695745058501269511L; - private StreamEventPool tableStreamEventPool; + private StreamEventFactory tableStreamEventFactory; private StreamEventConverter eventConverter; - public ListEventHolder(StreamEventPool tableStreamEventPool, StreamEventConverter eventConverter, + public ListEventHolder(StreamEventFactory tableStreamEventFactory, StreamEventConverter eventConverter, StreamEventClonerHolder streamEventClonerHolder) { super(streamEventClonerHolder); - this.tableStreamEventPool = tableStreamEventPool; + this.tableStreamEventFactory = tableStreamEventFactory; this.eventConverter = eventConverter; } @@ -48,7 +48,7 @@ public void add(ComplexEventChunk addingEventChunk) { addingEventChunk.reset(); while (addingEventChunk.hasNext()) { ComplexEvent complexEvent = addingEventChunk.next(); - StreamEvent streamEvent = tableStreamEventPool.borrowEvent(); + StreamEvent streamEvent = tableStreamEventFactory.newInstance(); eventConverter.convertComplexEvent(complexEvent, streamEvent); this.add(streamEvent); } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/AbstractQueryableRecordTable.java b/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/AbstractQueryableRecordTable.java index f6fb142941..8831a59f13 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/AbstractQueryableRecordTable.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/AbstractQueryableRecordTable.java @@ -95,7 +95,7 @@ public StreamEvent query(StateEvent matchingEvent, CompiledCondition compiledCon if (records != null) { while (records.hasNext()) { Object[] record = records.next(); - StreamEvent streamEvent = storeEventPool.borrowEvent(); + StreamEvent streamEvent = storeEventPool.newInstance(); streamEvent.setOutputData(new Object[outputAttributes.length]); System.arraycopy(record, 0, streamEvent.getOutputData(), 0, record.length); streamEventComplexEventChunk.add(streamEvent); @@ -255,14 +255,6 @@ private class RecordStoreCompiledSelection implements CompiledSelection { this.compiledSelection = compiledSelection; } - @Override - public CompiledSelection cloneCompilation(String key) { - Map newVariableExpressionExecutorMap = new HashMap<>(); - for (Map.Entry entry : variableExpressionExecutorMap.entrySet()) { - newVariableExpressionExecutorMap.put(entry.getKey(), entry.getValue().cloneExecutor(key)); - } - return new RecordStoreCompiledSelection(newVariableExpressionExecutorMap, compiledSelection); - } } /** diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/AbstractRecordTable.java b/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/AbstractRecordTable.java index 981d619660..44262e9dfe 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/AbstractRecordTable.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/AbstractRecordTable.java @@ -24,7 +24,7 @@ import io.siddhi.core.event.state.StateEvent; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.exception.ConnectionUnavailableException; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -54,16 +54,16 @@ public abstract class AbstractRecordTable extends Table { private static final Logger log = Logger.getLogger(AbstractRecordTable.class); - protected StreamEventPool storeEventPool; + protected StreamEventFactory storeEventPool; protected RecordTableHandler recordTableHandler; @Override - public void init(TableDefinition tableDefinition, StreamEventPool storeEventPool, + public void init(TableDefinition tableDefinition, StreamEventFactory storeEventPool, StreamEventCloner storeEventCloner, ConfigReader configReader, SiddhiAppContext siddhiAppContext, RecordTableHandler recordTableHandler) { if (recordTableHandler != null) { - recordTableHandler.init(siddhiAppContext.getElementIdGenerator().createNewId(), tableDefinition, - new RecordTableHandlerCallback(this)); + recordTableHandler.init(tableDefinition, new RecordTableHandlerCallback(this), + siddhiAppContext); } this.recordTableHandler = recordTableHandler; this.storeEventPool = storeEventPool; @@ -132,7 +132,7 @@ public StreamEvent find(CompiledCondition compiledCondition, StateEvent matching if (records != null) { while (records.hasNext()) { Object[] record = records.next(); - StreamEvent streamEvent = storeEventPool.borrowEvent(); + StreamEvent streamEvent = storeEventPool.newInstance(); System.arraycopy(record, 0, streamEvent.getOutputData(), 0, record.length); streamEventComplexEventChunk.add(streamEvent); } @@ -402,14 +402,6 @@ protected class RecordStoreCompiledCondition implements CompiledCondition { this.compiledCondition = compiledCondition; } - @Override - public CompiledCondition cloneCompilation(String key) { - Map newVariableExpressionExecutorMap = new HashMap<>(); - for (Map.Entry entry : variableExpressionExecutorMap.entrySet()) { - newVariableExpressionExecutorMap.put(entry.getKey(), entry.getValue().cloneExecutor(key)); - } - return new RecordStoreCompiledCondition(newVariableExpressionExecutorMap, compiledCondition); - } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/RecordTableHandler.java b/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/RecordTableHandler.java index 840a90f1df..e3221fc353 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/RecordTableHandler.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/table/record/RecordTableHandler.java @@ -16,10 +16,14 @@ package io.siddhi.core.table.record; +import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.exception.ConnectionUnavailableException; import io.siddhi.core.util.collection.operator.CompiledCondition; import io.siddhi.core.util.collection.operator.CompiledExpression; import io.siddhi.core.util.collection.operator.CompiledSelection; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateFactory; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.query.api.definition.Attribute; import io.siddhi.query.api.definition.TableDefinition; @@ -31,33 +35,46 @@ /** * RecordTableHandler is an optional handler that can be implemented to do processing on output events before sending * to the AbstractRecordTable. + * + * @param current state */ -public abstract class RecordTableHandler { +public abstract class RecordTableHandler { - private String elementId; + private String id; private RecordTableHandlerCallback recordTableHandlerCallback; + private StateHolder stateHolder; + + protected final void init(TableDefinition tableDefinition, RecordTableHandlerCallback recordTableHandlerCallback, + SiddhiAppContext siddhiAppContext) { + this.recordTableHandlerCallback = recordTableHandlerCallback; + + id = siddhiAppContext.getName() + "-" + tableDefinition.getId() + "-" + this.getClass().getName(); + StateFactory stateFactory = init(id, tableDefinition); + stateHolder = siddhiAppContext.generateStateHolder( + tableDefinition.getId() + "-" + this.getClass().getName(), + stateFactory); - public String getElementId() { - return elementId; } - protected final void init(String elementId, TableDefinition tableDefinition, - RecordTableHandlerCallback recordTableHandlerCallback) { - this.elementId = elementId; - this.recordTableHandlerCallback = recordTableHandlerCallback; - init(elementId, tableDefinition); + public String getId() { + return id; } /** * Initialize the Record Table Handler * - * @param elementId is the generated id for the record table handler + * @param id is the generated id for the record table handler * @param tableDefinition is the definition of the table with annotations if any */ - public abstract void init(String elementId, TableDefinition tableDefinition); + public abstract StateFactory init(String id, TableDefinition tableDefinition); public void add(long timestamp, List records) throws ConnectionUnavailableException { - add(timestamp, records, recordTableHandlerCallback); + S state = stateHolder.getState(); + try { + add(timestamp, records, recordTableHandlerCallback, state); + } finally { + stateHolder.returnState(state); + } } /** @@ -65,15 +82,21 @@ public void add(long timestamp, List records) throws ConnectionUnavail * @param records records that need to be added to the table, each Object[] represent a * record and it will match the attributes of the Table Definition. * @param recordTableHandlerCallback call back to do operations on the record table + * @param state current state * @throws ConnectionUnavailableException */ public abstract void add(long timestamp, List records, - RecordTableHandlerCallback recordTableHandlerCallback) + RecordTableHandlerCallback recordTableHandlerCallback, S state) throws ConnectionUnavailableException; public void delete(long timestamp, List> deleteConditionParameterMaps, CompiledCondition compiledCondition) throws ConnectionUnavailableException { - delete(timestamp, deleteConditionParameterMaps, compiledCondition, recordTableHandlerCallback); + S state = stateHolder.getState(); + try { + delete(timestamp, deleteConditionParameterMaps, compiledCondition, recordTableHandlerCallback, state); + } finally { + stateHolder.returnState(state); + } } /** @@ -82,19 +105,25 @@ public void delete(long timestamp, List> deleteConditionPara * compiled condition * @param compiledCondition the compiledCondition against which records should be matched for deletion * @param recordTableHandlerCallback call back to do operations on the record table + * @param state current state * @throws ConnectionUnavailableException */ public abstract void delete(long timestamp, List> deleteConditionParameterMaps, CompiledCondition compiledCondition, - RecordTableHandlerCallback recordTableHandlerCallback) + RecordTableHandlerCallback recordTableHandlerCallback, S state) throws ConnectionUnavailableException; public void update(long timestamp, CompiledCondition compiledCondition, List> updateConditionParameterMaps, LinkedHashMap updateSetMap, List> updateSetParameterMaps) throws ConnectionUnavailableException { - update(timestamp, compiledCondition, updateConditionParameterMaps, updateSetMap, updateSetParameterMaps, - recordTableHandlerCallback); + S state = stateHolder.getState(); + try { + update(timestamp, compiledCondition, updateConditionParameterMaps, updateSetMap, updateSetParameterMaps, + recordTableHandlerCallback, state); + } finally { + stateHolder.returnState(state); + } } /** @@ -105,13 +134,14 @@ public void update(long timestamp, CompiledCondition compiledCondition, * @param updateSetExpressions the set of updates mappings and related complied expressions * @param updateSetParameterMaps map of matching StreamVariable Ids and their values corresponding to the * @param recordTableHandlerCallback call back to do operations on the record table + * @param state current state * @throws ConnectionUnavailableException */ public abstract void update(long timestamp, CompiledCondition updateCondition, List> updateConditionParameterMaps, LinkedHashMap updateSetExpressions, List> updateSetParameterMaps, - RecordTableHandlerCallback recordTableHandlerCallback) + RecordTableHandlerCallback recordTableHandlerCallback, S state) throws ConnectionUnavailableException; public void updateOrAdd(long timestamp, CompiledCondition compiledCondition, @@ -145,7 +175,12 @@ public abstract void updateOrAdd(long timestamp, CompiledCondition updateConditi public Iterator find(long timestamp, Map findConditionParameterMap, CompiledCondition compiledCondition) throws ConnectionUnavailableException { - return find(timestamp, findConditionParameterMap, compiledCondition, recordTableHandlerCallback); + S state = stateHolder.getState(); + try { + return find(timestamp, findConditionParameterMap, compiledCondition, recordTableHandlerCallback, state); + } finally { + stateHolder.returnState(state); + } } /** @@ -154,17 +189,24 @@ public Iterator find(long timestamp, Map findCondition * corresponding to the compiled condition * @param compiledCondition the compiledCondition against which records should be matched * @param recordTableHandlerCallback call back to do operations on the record table + * @param state * @return RecordIterator of matching records * @throws ConnectionUnavailableException */ public abstract Iterator find(long timestamp, Map findConditionParameterMap, CompiledCondition compiledCondition, - RecordTableHandlerCallback recordTableHandlerCallback) + RecordTableHandlerCallback recordTableHandlerCallback, S state) throws ConnectionUnavailableException; public boolean contains(long timestamp, Map containsConditionParameterMap, CompiledCondition compiledCondition) throws ConnectionUnavailableException { - return contains(timestamp, containsConditionParameterMap, compiledCondition, recordTableHandlerCallback); + S state = stateHolder.getState(); + try { + return contains(timestamp, containsConditionParameterMap, compiledCondition, + recordTableHandlerCallback, state); + } finally { + stateHolder.returnState(state); + } } /** @@ -173,12 +215,13 @@ public boolean contains(long timestamp, Map containsConditionPar * compiled condition * @param compiledCondition the compiledCondition against which records should be matched * @param recordTableHandlerCallback call back to do operations on the record table + * @param state current state * @return if matching record found or not * @throws ConnectionUnavailableException */ public abstract boolean contains(long timestamp, Map containsConditionParameterMap, CompiledCondition compiledCondition, - RecordTableHandlerCallback recordTableHandlerCallback) + RecordTableHandlerCallback recordTableHandlerCallback, S state) throws ConnectionUnavailableException; @@ -186,15 +229,26 @@ public abstract boolean contains(long timestamp, Map containsCon public Iterator query(long timestamp, Map parameterMap, CompiledCondition compiledCondition, CompiledSelection compiledSelection) throws ConnectionUnavailableException { - return query(timestamp, parameterMap, compiledCondition, compiledSelection, recordTableHandlerCallback); + S state = stateHolder.getState(); + try { + return query(timestamp, parameterMap, compiledCondition, compiledSelection, recordTableHandlerCallback, + state); + } finally { + stateHolder.returnState(state); + } } public Iterator query(long timestamp, Map parameterMap, CompiledCondition compiledCondition, CompiledSelection compiledSelection, Attribute[] outputAttributes) throws ConnectionUnavailableException { - return query(timestamp, parameterMap, compiledCondition, compiledSelection, outputAttributes, - recordTableHandlerCallback); + S state = stateHolder.getState(); + try { + return query(timestamp, parameterMap, compiledCondition, compiledSelection, outputAttributes, + recordTableHandlerCallback, state); + } finally { + stateHolder.returnState(state); + } } /** @@ -204,6 +258,7 @@ public Iterator query(long timestamp, Map parameterMap * @param compiledCondition the compiledCondition against which records should be matched * @param compiledSelection the compiledSelection which maps the events based on selection * @param recordTableHandlerCallback call back to do operations on the record table + * @param state current state * @return RecordIterator of matching records * @throws ConnectionUnavailableException */ @@ -211,13 +266,14 @@ public Iterator query(long timestamp, Map parameterMap public abstract Iterator query(long timestamp, Map parameterMap, CompiledCondition compiledCondition, CompiledSelection compiledSelection, - RecordTableHandlerCallback recordTableHandlerCallback) + RecordTableHandlerCallback recordTableHandlerCallback, S state) throws ConnectionUnavailableException; public abstract Iterator query(long timestamp, Map parameterMap, CompiledCondition compiledCondition, CompiledSelection compiledSelection, Attribute[] outputAttributes, - RecordTableHandlerCallback recordTableHandlerCallback) + RecordTableHandlerCallback recordTableHandlerCallback, S state) throws ConnectionUnavailableException; + } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/trigger/Trigger.java b/modules/siddhi-core/src/main/java/io/siddhi/core/trigger/Trigger.java index 5643cf9cad..6abe6ff542 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/trigger/Trigger.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/trigger/Trigger.java @@ -20,14 +20,14 @@ import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.stream.StreamJunction; -import io.siddhi.core.util.extension.holder.EternalReferencedHolder; +import io.siddhi.core.util.extension.holder.ExternalReferencedHolder; import io.siddhi.query.api.definition.TriggerDefinition; /** * Interface class to represent event triggers. Event triggers are used to trigger events within Siddhi itself * according to a user given criteria. */ -public interface Trigger extends EternalReferencedHolder { +public interface Trigger extends ExternalReferencedHolder { void init(TriggerDefinition triggerDefinition, SiddhiAppContext siddhiAppContext, StreamJunction streamJunction); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/ElementIdGenerator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/IdGenerator.java similarity index 79% rename from modules/siddhi-core/src/main/java/io/siddhi/core/util/ElementIdGenerator.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/util/IdGenerator.java index c821e1a9ca..adf4a52213 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/ElementIdGenerator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/IdGenerator.java @@ -23,16 +23,11 @@ /** * unique id generator for elements inside a given siddhi app */ -public class ElementIdGenerator { +public class IdGenerator { - private String siddhiAppName; private AtomicLong id = new AtomicLong(0); - public ElementIdGenerator(String siddhiAppName) { - this.siddhiAppName = siddhiAppName; - } - public String createNewId() { - return siddhiAppName + "-" + id.incrementAndGet(); + return String.valueOf(id.incrementAndGet()); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/Scheduler.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/Scheduler.java index 54375b4126..19a94bc9b2 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/Scheduler.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/Scheduler.java @@ -18,15 +18,16 @@ package io.siddhi.core.util; +import com.google.common.collect.TreeMultimap; import io.siddhi.core.config.SiddhiAppContext; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEventChunk; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; -import io.siddhi.core.event.stream.converter.ConversionStreamEventChunk; -import io.siddhi.core.event.stream.converter.StreamEventConverter; -import io.siddhi.core.query.input.stream.single.EntryValveProcessor; +import io.siddhi.core.event.stream.StreamEventFactory; +import io.siddhi.core.util.extension.holder.ExternalReferencedHolder; import io.siddhi.core.util.lock.LockWrapper; -import io.siddhi.core.util.snapshot.Snapshotable; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateHolder; import io.siddhi.core.util.statistics.LatencyTracker; import io.siddhi.core.util.timestamp.TimestampGeneratorImpl; import org.apache.log4j.Logger; @@ -43,62 +44,98 @@ /** * Scheduler implementation to take periodic snapshots */ -public class Scheduler implements Snapshotable { +public class Scheduler implements ExternalReferencedHolder { private static final Logger log = Logger.getLogger(Scheduler.class); - private final BlockingQueue toNotifyQueue = new LinkedBlockingQueue(); private final ThreadBarrier threadBarrier; private final Schedulable singleThreadEntryValve; private final Semaphore mutex; protected String queryName; - private SiddhiAppContext siddhiAppContext; - private String elementId; + private SiddhiQueryContext siddhiQueryContext; private LockWrapper lockWrapper; private ScheduledExecutorService scheduledExecutorService; - private EventCaller eventCaller; - private StreamEventPool streamEventPool; - private ComplexEventChunk streamEventChunk; + private StreamEventFactory streamEventFactory; private LatencyTracker latencyTracker; - private volatile boolean running = false; - private ScheduledFuture scheduledFuture; + private StateHolder stateHolder; + private boolean stop; - public Scheduler(Schedulable singleThreadEntryValve, SiddhiAppContext siddhiAppContext) { - this.threadBarrier = siddhiAppContext.getThreadBarrier(); - this.siddhiAppContext = siddhiAppContext; + public Scheduler(Schedulable singleThreadEntryValve, SiddhiQueryContext siddhiQueryContext) { + this.threadBarrier = siddhiQueryContext.getSiddhiAppContext().getThreadBarrier(); + this.siddhiQueryContext = siddhiQueryContext; this.singleThreadEntryValve = singleThreadEntryValve; - this.scheduledExecutorService = siddhiAppContext.getScheduledExecutorService(); - this.eventCaller = new EventCaller(); - mutex = new Semaphore(1); + this.scheduledExecutorService = siddhiQueryContext.getSiddhiAppContext().getScheduledExecutorService(); + this.mutex = new Semaphore(1); - siddhiAppContext.getTimestampGenerator() + siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator() .addTimeChangeListener(new TimestampGeneratorImpl.TimeChangeListener() { @Override - public void onTimeChange(long currentTimestamp) { - Long lastTime = toNotifyQueue.peek(); - if (lastTime != null && lastTime <= currentTimestamp) { - // If executed in a separate thread, while it is processing, - // the new event will come into the window. As the result of it, - // the window will emit the new event as an existing current event. - sendTimerEvents(); + public synchronized void onTimeChange(long currentTimestamp) { + Map> allStates = stateHolder.getAllStates(); + try { + TreeMultimap sortedExpires = TreeMultimap.create(); + for (Map.Entry> allStatesEntry : + allStates.entrySet()) { + for (Map.Entry stateEntry : + allStatesEntry.getValue().entrySet()) { + Long lastTime = stateEntry.getValue().toNotifyQueue.peek(); + if (lastTime != null && lastTime <= currentTimestamp) { + sortedExpires.put(lastTime, stateEntry.getValue()); + } + } + } + for (Map.Entry entry : sortedExpires.entries()) { + try { + SiddhiAppContext.startPartitionFlow(entry.getValue().key); + // If executed in a separate thread, while it is processing, + // the new event will come into the window. As the result of it, + // the window will emit the new event as an existing current event. + sendTimerEvents(entry.getValue()); + } finally { + SiddhiAppContext.stopPartitionFlow(); + } + } + } finally { + stateHolder.returnAllStates(allStates); } } }); } - public void schedule(long time) { - if (!siddhiAppContext.isPlayback()) { - if (!running && toNotifyQueue.size() == 1) { + public void init(LockWrapper lockWrapper, String queryName) { + this.lockWrapper = lockWrapper; + this.queryName = queryName; + String id = "Scheduler_" + queryName + "_" + siddhiQueryContext.generateNewId(); + this.stateHolder = siddhiQueryContext.generateStateHolder(id, false, () -> new SchedulerState()); + } + + public void notifyAt(long time) { + SchedulerState state = stateHolder.getState(); + try { + // Insert the time into the queue + state.toNotifyQueue.put(time); + schedule(time, state); // Let the subclasses to schedule the scheduler + } catch (InterruptedException e) { + log.error("Error when adding time:" + time + " to toNotifyQueue at Scheduler", e); + } finally { + stateHolder.returnState(state); + } + } + + public void schedule(long time, SchedulerState state) { + if (!siddhiQueryContext.getSiddhiAppContext().isPlayback()) { + if (!state.running && state.toNotifyQueue.size() == 1) { try { mutex.acquire(); - if (!running) { - running = true; - long timeDiff = time - siddhiAppContext.getTimestampGenerator().currentTime(); + if (!state.running) { + state.running = true; + long timeDiff = time - siddhiQueryContext.getSiddhiAppContext(). + getTimestampGenerator().currentTime(); if (timeDiff > 0) { - scheduledFuture = scheduledExecutorService.schedule(eventCaller, + state.scheduledFuture = scheduledExecutorService.schedule(state.eventCaller, timeDiff, TimeUnit.MILLISECONDS); } else { - scheduledFuture = scheduledExecutorService.schedule(eventCaller, + state.scheduledFuture = scheduledExecutorService.schedule(state.eventCaller, 0, TimeUnit.MILLISECONDS); } } @@ -113,60 +150,9 @@ public void schedule(long time) { } } - public Scheduler clone(String key, EntryValveProcessor entryValveProcessor) { - Scheduler scheduler = new Scheduler(entryValveProcessor, - siddhiAppContext); - scheduler.elementId = elementId + "-" + key; - return scheduler; - } - - public void notifyAt(long time) { - try { - // Insert the time into the queue - toNotifyQueue.put(time); - schedule(time); // Let the subclasses to schedule the scheduler - } catch (InterruptedException e) { - log.error("Error when adding time:" + time + " to toNotifyQueue at Scheduler", e); - } - } - - public void setStreamEventPool(StreamEventPool streamEventPool) { - this.streamEventPool = streamEventPool; - streamEventChunk = new ConversionStreamEventChunk((StreamEventConverter) null, streamEventPool); - } - - public void init(LockWrapper lockWrapper, String queryName) { - this.lockWrapper = lockWrapper; - this.queryName = queryName; - if (elementId == null) { - elementId = "Scheduler-" + siddhiAppContext.getElementIdGenerator().createNewId(); - } - siddhiAppContext.getSnapshotService().addSnapshotable(queryName, this); - } - - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("ToNotifyQueue", toNotifyQueue); - return state; - } - - @Override - public void restoreState(Map state) { - BlockingQueue restoreToNotifyQueue = (BlockingQueue) state.get("ToNotifyQueue"); - for (Long time : restoreToNotifyQueue) { - notifyAt(time); - } - } - - @Override - public String getElementId() { - return elementId; - } - @Override - public void clean() { - siddhiAppContext.getSnapshotService().removeSnapshotable(queryName, this); + public void setStreamEventFactory(StreamEventFactory streamEventFactory) { + this.streamEventFactory = streamEventFactory; } public void setLatencyTracker(LatencyTracker latencyTracker) { @@ -174,24 +160,26 @@ public void setLatencyTracker(LatencyTracker latencyTracker) { } /** - * Go through the timestamps stored in the {@link #toNotifyQueue} and send the TIMER events for the expired events. + * Go through the timestamps stored in the toNotifyQueue and send the TIMER events for the expired events. + * + * @param state current state */ - protected void sendTimerEvents() { - Long toNotifyTime = toNotifyQueue.peek(); - long currentTime = siddhiAppContext.getTimestampGenerator().currentTime(); + protected void sendTimerEvents(SchedulerState state) { + Long toNotifyTime = state.toNotifyQueue.peek(); + long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); while (toNotifyTime != null && toNotifyTime - currentTime <= 0) { - toNotifyQueue.poll(); - - StreamEvent timerEvent = streamEventPool.borrowEvent(); + state.toNotifyQueue.poll(); + StreamEvent timerEvent = streamEventFactory.newInstance(); timerEvent.setType(StreamEvent.Type.TIMER); timerEvent.setTimestamp(toNotifyTime); - streamEventChunk.add(timerEvent); if (lockWrapper != null) { lockWrapper.lock(); } - threadBarrier.pass(); + threadBarrier.enter(); try { - if (siddhiAppContext.isStatsEnabled() && latencyTracker != null) { + ComplexEventChunk streamEventChunk = new ComplexEventChunk<>(false); + streamEventChunk.add(timerEvent); + if (siddhiQueryContext.getSiddhiAppContext().isStatsEnabled() && latencyTracker != null) { try { latencyTracker.markIn(); singleThreadEntryValve.process(streamEventChunk); @@ -201,15 +189,17 @@ protected void sendTimerEvents() { } else { singleThreadEntryValve.process(streamEventChunk); } + } catch (Throwable t) { + log.error("Error while sending timer events, " + t.getMessage(), t); } finally { if (lockWrapper != null) { lockWrapper.unlock(); } + threadBarrier.exit(); } - streamEventChunk.clear(); - toNotifyTime = toNotifyQueue.peek(); - currentTime = siddhiAppContext.getTimestampGenerator().currentTime(); + toNotifyTime = state.toNotifyQueue.peek(); + currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); } } @@ -217,9 +207,25 @@ protected void sendTimerEvents() { * Schedule events which are not scheduled in the queue when switching back from event time to system current time */ public void switchToLiveMode() { - Long toNotifyTime = toNotifyQueue.peek(); - if (toNotifyTime != null) { - schedule(toNotifyTime); + Map> allStates = stateHolder.getAllStates(); + try { + for (Map.Entry> allStatesEntry : allStates.entrySet()) { + for (Map.Entry stateEntry : allStatesEntry.getValue().entrySet()) { + Long toNotifyTime = stateEntry.getValue().toNotifyQueue.peek(); + if (toNotifyTime != null) { + SiddhiAppContext.startPartitionFlow(allStatesEntry.getKey()); + SiddhiAppContext.startGroupByFlow(stateEntry.getKey()); + try { + schedule(toNotifyTime, stateEntry.getValue()); + } finally { + SiddhiAppContext.stopGroupByFlow(); + SiddhiAppContext.stopPartitionFlow(); + } + } + } + } + } finally { + stateHolder.returnAllStates(allStates); } } @@ -228,14 +234,39 @@ public void switchToLiveMode() { * the acquired resources for processing. */ public void switchToPlayBackMode() { - if (scheduledFuture != null) { - scheduledFuture.cancel(true); + Map> allStates = stateHolder.getAllStates(); + try { + for (Map.Entry> allStatesEntry : allStates.entrySet()) { + for (Map.Entry stateEntry : allStatesEntry.getValue().entrySet()) { + if (stateEntry.getValue().scheduledFuture != null) { + stateEntry.getValue().scheduledFuture.cancel(true); + } + //Make the scheduler running flag to false to make sure scheduler will schedule next time starts + stateEntry.getValue().running = false; + } + } + } finally { + stateHolder.returnAllStates(allStates); } - //Make the scheduler running flag to false to make sure scheduler will schedule next time starts - running = false; + } + + public void stop() { + stop = true; + } + + public void start() { + stop = false; } private class EventCaller implements Runnable { + private SchedulerState state; + private String key; + + public EventCaller(SchedulerState state, String key) { + this.state = state; + this.key = key; + } + /** * When an object implementing interface Runnable is used * to create a thread, starting the thread causes the object's @@ -248,24 +279,28 @@ private class EventCaller implements Runnable { * @see Thread#run() */ @Override - public void run() { + public synchronized void run() { + if (stop) { + return; + } + SiddhiAppContext.startPartitionFlow(key); try { - if (!siddhiAppContext.isPlayback()) { - sendTimerEvents(); + if (!siddhiQueryContext.getSiddhiAppContext().isPlayback()) { + sendTimerEvents(state); - Long toNotifyTime = toNotifyQueue.peek(); - long currentTime = siddhiAppContext.getTimestampGenerator().currentTime(); + Long toNotifyTime = state.toNotifyQueue.peek(); + long currentTime = siddhiQueryContext.getSiddhiAppContext().getTimestampGenerator().currentTime(); if (toNotifyTime != null) { - scheduledFuture = scheduledExecutorService. - schedule(eventCaller, toNotifyTime - currentTime, TimeUnit.MILLISECONDS); + state.scheduledFuture = scheduledExecutorService. + schedule(this, toNotifyTime - currentTime, TimeUnit.MILLISECONDS); } else { try { mutex.acquire(); - running = false; - if (toNotifyQueue.peek() != null) { - running = true; - scheduledFuture = scheduledExecutorService.schedule(eventCaller, + state.running = false; + if (state.toNotifyQueue.peek() != null) { + state.running = true; + state.scheduledFuture = scheduledExecutorService.schedule(this, 0, TimeUnit.MILLISECONDS); } } catch (InterruptedException e) { @@ -276,12 +311,53 @@ public void run() { } } } else { - running = false; + state.running = false; } } catch (Throwable t) { - log.error(t); + log.error("Error while executing Scheduled Timer Event Caller, " + t.getMessage(), t); + } finally { + SiddhiAppContext.stopPartitionFlow(); + } + } + + } + + class SchedulerState extends State implements Comparable { + + private final BlockingQueue toNotifyQueue = new LinkedBlockingQueue(); + private final String key; + private volatile boolean running = false; + private EventCaller eventCaller; + private ScheduledFuture scheduledFuture; + + public SchedulerState() { + this.key = SiddhiAppContext.getPartitionFlowId(); + this.eventCaller = new EventCaller(this, key); + } + + @Override + public boolean canDestroy() { + return toNotifyQueue.isEmpty() && (scheduledFuture == null || scheduledFuture.isDone()); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("ToNotifyQueue", toNotifyQueue); + return state; + } + + @Override + public void restore(Map state) { + BlockingQueue restoreToNotifyQueue = (BlockingQueue) state.get("ToNotifyQueue"); + for (Long time : restoreToNotifyQueue) { + notifyAt(time); } } + @Override + public int compareTo(Object o) { + return 0; + } } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/SiddhiAppRuntimeBuilder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/SiddhiAppRuntimeBuilder.java index 575efd90e0..9fd8376fc4 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/SiddhiAppRuntimeBuilder.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/SiddhiAppRuntimeBuilder.java @@ -135,8 +135,6 @@ public void defineWindow(WindowDefinition windowDefinition) { windowDefinition = (WindowDefinition) currentDefinition; } DefinitionParserHelper.addWindow(windowDefinition, windowMap, siddhiAppContext); - // defineStream(windowDefinition); - // DefinitionParserHelper.addStreamJunction(windowDefinition, streamJunctionMap, siddhiQueryContext); } public void defineTrigger(TriggerDefinition triggerDefinition) { @@ -164,7 +162,7 @@ public void defineAggregation(AggregationDefinition aggregationDefinition) { } public void addPartition(PartitionRuntime partitionRuntime) { - partitionMap.put(partitionRuntime.getPartitionId(), partitionRuntime); + partitionMap.put(partitionRuntime.getPartitionName(), partitionRuntime); } public String addQuery(QueryRuntime queryRuntime) { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/SiddhiConstants.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/SiddhiConstants.java index 68b2fc0a94..ccc1538541 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/SiddhiConstants.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/SiddhiConstants.java @@ -77,6 +77,8 @@ public final class SiddhiConstants { public static final String ANNOTATION_IGNORE_EVENTS_OLDER_THAN_BUFFER = "IgnoreEventsOlderThanBuffer"; public static final String ANNOTATION_ELEMENT_REF = "ref"; public static final String ANNOTATION_ELEMENT_ENABLE = "enable"; + public static final String ANNOTATION_ELEMENT_IDLE_PERIOD = "idle.period"; + public static final String ANNOTATION_ELEMENT_INTERVAL = "interval"; public static final String ANNOTATION_ELEMENT_INCLUDE = "include"; public static final String ANNOTATION_PARTITION_BY_ID = "PartitionById"; @@ -119,6 +121,7 @@ public final class SiddhiConstants { public static final String TRANSPORT_CHANNEL_CREATION_IDENTIFIER = "transportChannelCreationEnabled"; public static final String NAMESPACE_PURGE = "purge"; - public static final String NAMESPACE_RETENTION = "retentionPeriod"; - public static final String NAMESPACE_INTERVAL = "interval"; + public static final String NAMESPACE_RETENTION_PERIOD = "retentionPeriod"; + + public static final String PARTITION_ID_DEFAULT = "null"; } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/ThreadBarrier.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/ThreadBarrier.java index a91b6556d0..5c9f078212 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/ThreadBarrier.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/ThreadBarrier.java @@ -18,6 +18,7 @@ package io.siddhi.core.util; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; /** @@ -26,12 +27,23 @@ public class ThreadBarrier { private ReentrantLock lock = new ReentrantLock(); + private AtomicInteger counter = new AtomicInteger(); - public void pass() { + public void enter() { if (lock.isLocked()) { lock.lock(); lock.unlock(); } + counter.incrementAndGet(); + } + + + public void exit() { + counter.decrementAndGet(); + } + + public int getActiveThreads() { + return counter.get(); } public void lock() { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CollectionOperator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CollectionOperator.java index f8a5f797cb..59cc8eaf86 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CollectionOperator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CollectionOperator.java @@ -42,11 +42,6 @@ public CollectionOperator(ExpressionExecutor expressionExecutor, int storeEventP this.storeEventPosition = storeEventPosition; } - @Override - public CompiledCondition cloneCompilation(String key) { - return new CollectionOperator(expressionExecutor.cloneExecutor(key), storeEventPosition); - } - @Override public StreamEvent find(StateEvent matchingEvent, Object storeEvents, StreamEventCloner storeEventCloner) { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledCondition.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledCondition.java index bb2d104885..d80138b394 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledCondition.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledCondition.java @@ -23,6 +23,4 @@ */ public interface CompiledCondition extends CompiledExpression { - CompiledCondition cloneCompilation(String key); - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledExpression.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledExpression.java index 5b3b1a1eae..13c9f0efc0 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledExpression.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledExpression.java @@ -23,6 +23,4 @@ */ public interface CompiledExpression { - CompiledExpression cloneCompilation(String key); - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledSelection.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledSelection.java index ec86ddb896..176b4afe01 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledSelection.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/CompiledSelection.java @@ -23,7 +23,5 @@ */ public interface CompiledSelection extends CompiledExpression { - CompiledSelection cloneCompilation(String key); - } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/EventChunkOperator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/EventChunkOperator.java index 8837734ff9..98141092ed 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/EventChunkOperator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/EventChunkOperator.java @@ -40,11 +40,6 @@ public EventChunkOperator(ExpressionExecutor expressionExecutor, int storeEventP this.storeEventPosition = storeEventPosition; } - @Override - public CompiledCondition cloneCompilation(String key) { - return new EventChunkOperator(expressionExecutor.cloneExecutor(key), storeEventPosition); - } - @Override public StreamEvent find(StateEvent matchingEvent, Object storeEvents, StreamEventCloner storeEventCloner) { ComplexEventChunk storeEventChunk = (ComplexEventChunk) storeEvents; diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/IncrementalAggregateCompileCondition.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/IncrementalAggregateCompileCondition.java index cc2c13bbd4..617ebba51c 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/IncrementalAggregateCompileCondition.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/IncrementalAggregateCompileCondition.java @@ -21,14 +21,14 @@ import io.siddhi.core.aggregation.IncrementalDataAggregator; import io.siddhi.core.aggregation.IncrementalExecutor; import io.siddhi.core.aggregation.IncrementalExternalTimestampDataAggregator; -import io.siddhi.core.config.SiddhiAppContext; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.ComplexEventChunk; 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.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.exception.SiddhiAppRuntimeException; import io.siddhi.core.executor.ExpressionExecutor; @@ -39,10 +39,8 @@ import io.siddhi.query.api.definition.Attribute; import io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import static io.siddhi.query.api.expression.Expression.Time.normalizeDuration; @@ -51,16 +49,15 @@ * based on the logical conditions defined herewith. */ public class IncrementalAggregateCompileCondition implements CompiledCondition { - private final StreamEventPool streamEventPoolForTableMeta; + private final StreamEventFactory streamEventFactoryForTableMeta; private final StreamEventCloner tableEventCloner; - private final StreamEventPool streamEventPoolForAggregateMeta; + private final StreamEventFactory streamEventFactoryForAggregateMeta; private final StreamEventCloner aggregateEventCloner; private final List additionalAttributes; private Map withinTableCompiledConditions; private CompiledCondition inMemoryStoreCompileCondition; private CompiledCondition onCompiledCondition; private MetaStreamEvent tableMetaStreamEvent; - private MetaStreamEvent aggregateMetaStreamEvent; private ComplexEventPopulater complexEventPopulater; private MatchingMetaInfoHolder alteredMatchingMetaInfoHolder; private ExpressionExecutor perExpressionExecutor; @@ -78,13 +75,12 @@ public IncrementalAggregateCompileCondition( this.inMemoryStoreCompileCondition = inMemoryStoreCompileCondition; this.onCompiledCondition = onCompiledCondition; this.tableMetaStreamEvent = tableMetaStreamEvent; - this.aggregateMetaStreamEvent = aggregateMetaSteamEvent; - this.streamEventPoolForTableMeta = new StreamEventPool(tableMetaStreamEvent, 10); - this.tableEventCloner = new StreamEventCloner(tableMetaStreamEvent, streamEventPoolForTableMeta); + this.streamEventFactoryForTableMeta = new StreamEventFactory(tableMetaStreamEvent); + this.tableEventCloner = new StreamEventCloner(tableMetaStreamEvent, streamEventFactoryForTableMeta); - this.streamEventPoolForAggregateMeta = new StreamEventPool(aggregateMetaSteamEvent, 10); - this.aggregateEventCloner = new StreamEventCloner(aggregateMetaSteamEvent, streamEventPoolForAggregateMeta); + this.streamEventFactoryForAggregateMeta = new StreamEventFactory(aggregateMetaSteamEvent); + this.aggregateEventCloner = new StreamEventCloner(aggregateMetaSteamEvent, streamEventFactoryForAggregateMeta); this.additionalAttributes = additionalAttributes; this.alteredMatchingMetaInfoHolder = alteredMatchingMetaInfoHolder; this.perExpressionExecutor = perExpressionExecutor; @@ -92,29 +88,16 @@ public IncrementalAggregateCompileCondition( this.isProcessingOnExternalTime = isProcessingOnExternalTime; } - @Override - public CompiledCondition cloneCompilation(String key) { - Map copyOfWithinTableCompiledConditions = new HashMap<>(); - for (Map.Entry entry : withinTableCompiledConditions.entrySet()) { - copyOfWithinTableCompiledConditions.put(entry.getKey(), entry.getValue().cloneCompilation(key)); - } - return new IncrementalAggregateCompileCondition(copyOfWithinTableCompiledConditions, - inMemoryStoreCompileCondition.cloneCompilation(key), - onCompiledCondition.cloneCompilation(key), tableMetaStreamEvent, aggregateMetaStreamEvent, - additionalAttributes, alteredMatchingMetaInfoHolder, perExpressionExecutor, - startTimeEndTimeExpressionExecutor, isProcessingOnExternalTime); - } - public StreamEvent find(StateEvent matchingEvent, AggregationDefinition aggregationDefinition, Map incrementalExecutorMap, Map aggregationTables, List incrementalDurations, - List baseExecutors, + List baseExecutorsForFind, List outputExpressionExecutors, - SiddhiAppContext siddhiAppContext, - List> aggregateProcessingExecutorsList, + SiddhiQueryContext siddhiQueryContext, + List> aggregateProcessingExecutorsListForFind, List groupbyKeyGeneratorList, - ExpressionExecutor shouldUpdateExpressionExecutor, + ExpressionExecutor shouldUpdateTimestamp, Map incrementalExecutorMapForPartitions) { ComplexEventChunk complexEventChunkToHoldWithinMatches = new ComplexEventChunk<>(true); @@ -156,17 +139,12 @@ public StreamEvent find(StateEvent matchingEvent, AggregationDefinition aggregat long oldestInMemoryEventTimestamp = getOldestInMemoryEventTimestamp(incrementalExecutorMap, incrementalDurations, perValue); - ExpressionExecutor shouldUpdateExpressionExecutorClone = - (shouldUpdateExpressionExecutor == null) ? null : shouldUpdateExpressionExecutor.cloneExecutor(null); - //If processing on external time, the in-memory data also needs to be queried if (isProcessingOnExternalTime || requiresAggregatingInMemoryData(oldestInMemoryEventTimestamp, startTimeEndTime)) { - List clonedBaseExecutors = baseExecutors.stream().map(expressionExecutor -> - expressionExecutor.cloneExecutor("")).collect(Collectors.toList()); IncrementalDataAggregator incrementalDataAggregator = new IncrementalDataAggregator(incrementalDurations, - perValue, oldestInMemoryEventTimestamp, clonedBaseExecutors, tableMetaStreamEvent, - siddhiAppContext, shouldUpdateExpressionExecutorClone); + perValue, oldestInMemoryEventTimestamp, baseExecutorsForFind, tableMetaStreamEvent, + shouldUpdateTimestamp, groupbyKeyGeneratorList.get(0) != null); ComplexEventChunk aggregatedInMemoryEventChunk; // Aggregate in-memory data and create an event chunk out of it if (incrementalExecutorMapForPartitions != null) { @@ -176,7 +154,6 @@ public StreamEvent find(StateEvent matchingEvent, AggregationDefinition aggregat aggregatedInMemoryEventChunk = incrementalDataAggregator .aggregateInMemoryData(incrementalExecutorMap); } - // Get the in-memory aggregate data, which is within given duration StreamEvent withinMatchFromInMemory = ((Operator) inMemoryStoreCompileCondition).find(matchingEvent, aggregatedInMemoryEventChunk, tableEventCloner); @@ -186,17 +163,12 @@ public StreamEvent find(StateEvent matchingEvent, AggregationDefinition aggregat ComplexEventChunk processedEvents; if (isProcessingOnExternalTime) { int durationIndex = incrementalDurations.indexOf(perValue); - List expressionExecutors = aggregateProcessingExecutorsList.get(durationIndex); - List clonedExecutors = expressionExecutors.stream().map(expressionExecutor -> - expressionExecutor.cloneExecutor("")).collect(Collectors.toList()); + List expressionExecutors = aggregateProcessingExecutorsListForFind.get(durationIndex); GroupByKeyGenerator groupByKeyGenerator = groupbyKeyGeneratorList.get(durationIndex); - - ExpressionExecutor shouldUpdateExpressionExecutorCloneExt = - (shouldUpdateExpressionExecutor == null) ? null : - shouldUpdateExpressionExecutor.cloneExecutor(null); IncrementalExternalTimestampDataAggregator incrementalExternalTimestampDataAggregator = - new IncrementalExternalTimestampDataAggregator(clonedExecutors, groupByKeyGenerator, - tableMetaStreamEvent, siddhiAppContext, shouldUpdateExpressionExecutorCloneExt); + new IncrementalExternalTimestampDataAggregator(expressionExecutors, groupByKeyGenerator, + tableMetaStreamEvent, siddhiQueryContext, + shouldUpdateTimestamp); processedEvents = incrementalExternalTimestampDataAggregator .aggregateData(complexEventChunkToHoldWithinMatches); } else { @@ -217,12 +189,12 @@ private ComplexEventChunk createAggregateSelectionEventChunk( ComplexEventChunk complexEventChunkToHoldMatches, List outputExpressionExecutors) { ComplexEventChunk aggregateSelectionComplexEventChunk = new ComplexEventChunk<>(true); - StreamEvent resetEvent = streamEventPoolForTableMeta.borrowEvent(); + StreamEvent resetEvent = streamEventFactoryForTableMeta.newInstance(); resetEvent.setType(ComplexEvent.Type.RESET); while (complexEventChunkToHoldMatches.hasNext()) { StreamEvent streamEvent = complexEventChunkToHoldMatches.next(); - StreamEvent newStreamEvent = streamEventPoolForAggregateMeta.borrowEvent(); + StreamEvent newStreamEvent = streamEventFactoryForAggregateMeta.newInstance(); Object outputData[] = new Object[newStreamEvent.getOutputData().length]; for (int i = 0; i < outputExpressionExecutors.size(); i++) { outputData[i] = outputExpressionExecutors.get(i).execute(streamEvent); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/IndexOperator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/IndexOperator.java index 66d7c986a7..65c53b7013 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/IndexOperator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/IndexOperator.java @@ -48,12 +48,6 @@ public IndexOperator(CollectionExecutor collectionExecutor, String queryName) { this.queryName = queryName; } - @Override - public CompiledCondition cloneCompilation(String key) { - //todo check if there are any issues when not cloning - return new IndexOperator(collectionExecutor, queryName); - } - @Override public StreamEvent find(StateEvent matchingEvent, Object storeEvents, StreamEventCloner storeEventCloner) { return collectionExecutor.find(matchingEvent, (IndexedEventHolder) storeEvents, storeEventCloner); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/MapOperator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/MapOperator.java index bf333816c9..0633d0e305 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/MapOperator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/MapOperator.java @@ -37,11 +37,6 @@ public MapOperator(ExpressionExecutor expressionExecutor, int storeEventPosition super(expressionExecutor, storeEventPosition); } - @Override - public CompiledCondition cloneCompilation(String key) { - return new MapOperator(expressionExecutor.cloneExecutor(key), storeEventPosition); - } - @Override public StreamEvent find(StateEvent matchingEvent, Object storeEvents, StreamEventCloner storeEventCloner) { return super.find(matchingEvent, ((Map) storeEvents).values(), storeEventCloner); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/SnapshotableEventQueueOperator.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/SnapshotableEventQueueOperator.java index 81772a0118..4031de7914 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/SnapshotableEventQueueOperator.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/collection/operator/SnapshotableEventQueueOperator.java @@ -41,11 +41,6 @@ public SnapshotableEventQueueOperator(ExpressionExecutor expressionExecutor, int this.storeEventPosition = storeEventPosition; } - @Override - public CompiledCondition cloneCompilation(String key) { - return new SnapshotableEventQueueOperator(expressionExecutor.cloneExecutor(key), storeEventPosition); - } - @Override public StreamEvent find(StateEvent matchingEvent, Object storeEvents, StreamEventCloner storeEventCloner) { SnapshotableStreamEventQueue storeEventQueue = (SnapshotableStreamEventQueue) storeEvents; diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/AttributeAggregatorExtensionHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/AttributeAggregatorExtensionHolder.java index f0dafb7695..c6964f2da8 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/AttributeAggregatorExtensionHolder.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/AttributeAggregatorExtensionHolder.java @@ -18,15 +18,15 @@ package io.siddhi.core.util.extension.holder; import io.siddhi.core.config.SiddhiAppContext; -import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator; +import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregatorExecutor; import java.util.concurrent.ConcurrentHashMap; /** - * Holder to store {@link AttributeAggregator} Extensions. + * Holder to store {@link AttributeAggregatorExecutor} Extensions. */ public class AttributeAggregatorExtensionHolder extends AbstractExtensionHolder { - private static Class clazz = AttributeAggregator.class; + private static Class clazz = AttributeAggregatorExecutor.class; private AttributeAggregatorExtensionHolder(SiddhiAppContext siddhiAppContext) { super(clazz, siddhiAppContext); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/EternalReferencedHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/ExternalReferencedHolder.java similarity index 97% rename from modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/EternalReferencedHolder.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/ExternalReferencedHolder.java index 0282e0a9f8..19473d2828 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/EternalReferencedHolder.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/extension/holder/ExternalReferencedHolder.java @@ -21,7 +21,7 @@ * Primary interface used to give start and stop method to Siddhi components which will be called once during Siddhi * start up and shutting down so that user can perform actions such as resource allocation and de-allocation. */ -public interface EternalReferencedHolder { +public interface ExternalReferencedHolder { /** * This will be called only once and this can be used to acquire 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 912a18bbac..88b6c852eb 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 @@ -25,8 +25,10 @@ import io.siddhi.core.aggregation.RecreateInMemoryData; import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.config.SiddhiQueryContext; +import io.siddhi.core.event.ComplexEvent; import io.siddhi.core.event.stream.MetaStreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEvent; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.exception.SiddhiAppCreationException; import io.siddhi.core.executor.ConstantExpressionExecutor; import io.siddhi.core.executor.ExpressionExecutor; @@ -177,7 +179,7 @@ public static AggregationRuntime parse(AggregationDefinition aggregationDefiniti if (enablePartioning || shouldPartitionById) { shardId = configManager.extractProperty("shardId"); if (shardId == null) { - throw new SiddhiAppCreationException("Configuration 'shardId' not provided for @partitionbyid " + + throw new SiddhiAppCreationException("Configuration 'shardId' not provided for @partitionById " + "annotation"); } enablePartioning = true; @@ -215,17 +217,22 @@ public static AggregationRuntime parse(AggregationDefinition aggregationDefiniti .map(incrementalDuration -> constructProcessExpressionExecutors( tableMap, siddhiQueryContext, baseAggregatorBeginIndex, finalBaseAggregators, incomingOutputStreamDefinition, processedMetaStreamEvent, - processVariableExpressionExecutors, groupBy, isProcessingOnExternalTime, + processVariableExpressionExecutors, true, isProcessingOnExternalTime, incrementalDuration)) .collect(Collectors.toList()); - ExpressionExecutor shouldUpdateExpressionExecutor = null; + List> processExpressionExecutorsListForFind = incrementalDurations.stream() + .map(incrementalDuration -> constructProcessExpressionExecutors( + tableMap, siddhiQueryContext, baseAggregatorBeginIndex, + finalBaseAggregators, incomingOutputStreamDefinition, processedMetaStreamEvent, + processVariableExpressionExecutors, true, isProcessingOnExternalTime, + incrementalDuration)) + .collect(Collectors.toList()); + + ExpressionExecutor shouldUpdateTimestamp = null; if (isLatestEventAdded) { - Expression shouldUpdateExp = AttributeFunction.function( - "incrementalAggregator", - "shouldUpdate", - new Variable(AGG_LAST_TIMESTAMP_COL)); - shouldUpdateExpressionExecutor = ExpressionParser.parseExpression(shouldUpdateExp, + Expression shouldUpdateTimestampExp = new Variable(AGG_LAST_TIMESTAMP_COL); + shouldUpdateTimestamp = ExpressionParser.parseExpression(shouldUpdateTimestampExp, processedMetaStreamEvent, 0, tableMap, processVariableExpressionExecutors, false, 0, ProcessingMode.BATCH, false, siddhiQueryContext); @@ -268,16 +275,15 @@ public static AggregationRuntime parse(AggregationDefinition aggregationDefiniti LockWrapper lockWrapper = new LockWrapper(aggregatorName); lockWrapper.setLock(new ReentrantLock()); - Scheduler scheduler = SchedulerParser.parse(entryValveExecutor, siddhiAppContext); + Scheduler scheduler = SchedulerParser.parse(entryValveExecutor, siddhiQueryContext); scheduler.init(lockWrapper, aggregatorName); - scheduler.setStreamEventPool(new StreamEventPool(processedMetaStreamEvent, 10)); + scheduler.setStreamEventFactory(new StreamEventFactory(processedMetaStreamEvent)); QueryParserHelper.reduceMetaComplexEvent(incomingMetaStreamEvent); QueryParserHelper.reduceMetaComplexEvent(processedMetaStreamEvent); QueryParserHelper.updateVariablePosition(incomingMetaStreamEvent, incomingVariableExpressionExecutors); QueryParserHelper.updateVariablePosition(processedMetaStreamEvent, processVariableExpressionExecutors); - Map aggregationTables = initDefaultTables(aggregatorName, incrementalDurations, processedMetaStreamEvent.getOutputStreamDefinition(), siddhiAppRuntimeBuilder, aggregationDefinition.getAnnotations(), groupByVariableList, @@ -297,9 +303,9 @@ public static AggregationRuntime parse(AggregationDefinition aggregationDefiniti } Map incrementalExecutorMap = buildIncrementalExecutors( - processedMetaStreamEvent, processExpressionExecutorsList, - groupByKeyGeneratorList, incrementalDurations, - aggregationTables, siddhiAppContext, aggregatorName, shouldUpdateExpressionExecutor); + processedMetaStreamEvent, processExpressionExecutorsList, groupByKeyGeneratorList, + incrementalDurations, aggregationTables, siddhiQueryContext, aggregatorName, + shouldUpdateTimestamp); Map incrementalExecutorMapForPartitions = null; if (shardId != null) { @@ -307,11 +313,10 @@ public static AggregationRuntime parse(AggregationDefinition aggregationDefiniti buildIncrementalExecutors( processedMetaStreamEvent, processExpressionExecutorsList, groupByKeyGeneratorList, incrementalDurations, - aggregationTables, siddhiAppContext, aggregatorName, shouldUpdateExpressionExecutor); + aggregationTables, siddhiQueryContext, aggregatorName, shouldUpdateTimestamp); } - IncrementalDataPurging incrementalDataPurging = new IncrementalDataPurging(); - incrementalDataPurging.init(aggregationDefinition, new StreamEventPool(processedMetaStreamEvent, 10) + incrementalDataPurging.init(aggregationDefinition, new StreamEventFactory(processedMetaStreamEvent) , aggregationTables, isProcessingOnExternalTime, siddhiQueryContext); //Recreate in-memory data from tables @@ -348,16 +353,14 @@ public static AggregationRuntime parse(AggregationDefinition aggregationDefiniti SiddhiConstants.METRIC_INFIX_AGGREGATIONS, SiddhiConstants.METRIC_TYPE_INSERT); } - List baseExecutors = cloneExpressionExecutors(processExpressionExecutorsList.get(0)); - //Remove timestamp executor - baseExecutors.remove(0); AggregationRuntime aggregationRuntime = new AggregationRuntime(aggregationDefinition, incrementalExecutorMap, aggregationTables, ((SingleStreamRuntime) streamRuntime), - incrementalDurations, siddhiAppContext, baseExecutors, processedMetaStreamEvent, + incrementalDurations, processedMetaStreamEvent, outputExpressionExecutors, latencyTrackerFind, throughputTrackerFind, recreateInMemoryData, - isProcessingOnExternalTime, processExpressionExecutorsList, groupByKeyGeneratorList, - incrementalDataPurging, shouldUpdateExpressionExecutor, shardId, - incrementalExecutorMapForPartitions); + isProcessingOnExternalTime, groupByKeyGeneratorList, + incrementalDataPurging, shardId, + incrementalExecutorMapForPartitions, shouldUpdateTimestamp, + processExpressionExecutorsListForFind); streamRuntime.setCommonProcessor(new IncrementalAggregationProcessor(aggregationRuntime, incomingExpressionExecutors, processedMetaStreamEvent, latencyTrackerInsert, @@ -377,8 +380,8 @@ private static Map buildIncrementalExe List groupByKeyGeneratorList, List incrementalDurations, Map aggregationTables, - SiddhiAppContext siddhiAppContext, - String aggregatorName, ExpressionExecutor shouldUpdateExpressionExecutor) { + SiddhiQueryContext siddhiQueryContext, + String aggregatorName, ExpressionExecutor shouldUpdateTimestamp) { Map incrementalExecutorMap = new HashMap<>(); // Create incremental executors IncrementalExecutor child; @@ -392,15 +395,11 @@ private static Map buildIncrementalExe child = root; TimePeriod.Duration duration = incrementalDurations.get(i); - ExpressionExecutor shouldUpdateExpressionExecutorClone = null; - if (shouldUpdateExpressionExecutor != null) { - shouldUpdateExpressionExecutorClone = shouldUpdateExpressionExecutor.cloneExecutor(null); - } IncrementalExecutor incrementalExecutor = new IncrementalExecutor(duration, - cloneExpressionExecutors(processExpressionExecutorsList.get(i)), + new ArrayList<>(processExpressionExecutorsList.get(i)), groupByKeyGeneratorList.get(i), processedMetaStreamEvent, child, isRoot, - aggregationTables.get(duration), siddhiAppContext, aggregatorName, - shouldUpdateExpressionExecutorClone); + aggregationTables.get(duration), siddhiQueryContext, aggregatorName, + shouldUpdateTimestamp); incrementalExecutorMap.put(duration, incrementalExecutor); root = incrementalExecutor; } @@ -650,12 +649,6 @@ private static boolean populateIncomingAggregatorsAndExecutors( return addAggLastEvent; } - private static List cloneExpressionExecutors(List expressionExecutors) { - List arrayList = expressionExecutors.stream().map(expressionExecutor -> - expressionExecutor.cloneExecutor(null)).collect(Collectors.toList()); - return arrayList; - } - private static void validateBaseAggregators(List incrementalAttributeAggregators, IncrementalAttributeAggregator incrementalAttributeAggregator, Attribute[] baseAttributes, Expression[] baseAttributeInitialValues, @@ -843,4 +836,38 @@ private static HashMap initDefaultTables( } return aggregationTableMap; } + + public static StreamEvent createRestEvent(MetaStreamEvent metaStreamEvent, StreamEvent streamEvent) { + streamEvent.setTimestamp(0); + streamEvent.setType(ComplexEvent.Type.RESET); + List outputData = metaStreamEvent.getOutputData(); + for (int i = 0, outputDataSize = outputData.size(); i < outputDataSize; i++) { + Attribute attribute = outputData.get(i); + switch (attribute.getType()) { + + case STRING: + streamEvent.setOutputData("", i); + break; + case INT: + streamEvent.setOutputData(0, i); + break; + case LONG: + streamEvent.setOutputData(0L, i); + break; + case FLOAT: + streamEvent.setOutputData(0f, i); + break; + case DOUBLE: + streamEvent.setOutputData(0.0, i); + break; + case BOOL: + streamEvent.setOutputData(false, i); + break; + case OBJECT: + streamEvent.setOutputData(null, i); + break; + } + } + return streamEvent; + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/EventHolderPasser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/EventHolderPasser.java index 3ba77fcdf8..cfa4222784 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/EventHolderPasser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/EventHolderPasser.java @@ -21,7 +21,7 @@ import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.ZeroStreamEventConverter; import io.siddhi.core.event.stream.holder.StreamEventClonerHolder; import io.siddhi.core.exception.OperationNotSupportedException; @@ -48,7 +48,7 @@ public class EventHolderPasser { private static final Logger log = Logger.getLogger(EventHolderPasser.class); - public static EventHolder parse(AbstractDefinition tableDefinition, StreamEventPool tableStreamEventPool, + public static EventHolder parse(AbstractDefinition tableDefinition, StreamEventFactory tableStreamEventFactory, SiddhiAppContext siddhiAppContext) { ZeroStreamEventConverter eventConverter = new ZeroStreamEventConverter(); @@ -113,15 +113,15 @@ public static EventHolder parse(AbstractDefinition tableDefinition, StreamEventP } } - return new IndexEventHolder(tableStreamEventPool, eventConverter, primaryKeyReferenceHolders, isNumeric, + return new IndexEventHolder(tableStreamEventFactory, eventConverter, primaryKeyReferenceHolders, isNumeric, indexMetaData, tableDefinition, siddhiAppContext); } else { MetaStreamEvent metaStreamEvent = new MetaStreamEvent(); for (Attribute attribute : tableDefinition.getAttributeList()) { metaStreamEvent.addOutputData(attribute); } - StreamEventCloner streamEventCloner = new StreamEventCloner(metaStreamEvent, tableStreamEventPool); - return new ListEventHolder(tableStreamEventPool, eventConverter, + StreamEventCloner streamEventCloner = new StreamEventCloner(metaStreamEvent, tableStreamEventFactory); + return new ListEventHolder(tableStreamEventFactory, eventConverter, new StreamEventClonerHolder(streamEventCloner)); } } 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 c139fac0ff..c6d01dac01 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 @@ -158,10 +158,7 @@ import io.siddhi.core.executor.math.subtract.SubtractExpressionExecutorInt; import io.siddhi.core.executor.math.subtract.SubtractExpressionExecutorLong; import io.siddhi.core.query.processor.ProcessingMode; -import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator; -import io.siddhi.core.query.selector.attribute.processor.executor.AbstractAggregationAttributeExecutor; -import io.siddhi.core.query.selector.attribute.processor.executor.AggregationAttributeExecutor; -import io.siddhi.core.query.selector.attribute.processor.executor.GroupByAggregationAttributeExecutor; +import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregatorExecutor; import io.siddhi.core.table.Table; import io.siddhi.core.util.ExceptionUtil; import io.siddhi.core.util.SiddhiClassLoader; @@ -449,29 +446,29 @@ public static ExpressionExecutor parseExpression(Expression expression, MetaComp processingMode, outputExpectsExpiredEvents, siddhiQueryContext); expressionExecutor.initExecutor(innerExpressionExecutors, - processingMode, configReader, siddhiQueryContext); + processingMode, configReader, groupBy, siddhiQueryContext); if (expressionExecutor.getReturnType() == Attribute.Type.BOOL) { return new BoolConditionExpressionExecutor(expressionExecutor); } return expressionExecutor; } else { - AttributeAggregator attributeAggregator = (AttributeAggregator) executor; + AttributeAggregatorExecutor attributeAggregatorExecutor = (AttributeAggregatorExecutor) executor; Expression[] innerExpressions = ((AttributeFunction) expression).getParameters(); ExpressionExecutor[] innerExpressionExecutors = parseInnerExpression(innerExpressions, metaEvent, currentState, tableMap, executorList, groupBy, defaultStreamEventIndex, processingMode, outputExpectsExpiredEvents, siddhiQueryContext); - attributeAggregator.initAggregator(innerExpressionExecutors, processingMode, - outputExpectsExpiredEvents, configReader, siddhiQueryContext); - AbstractAggregationAttributeExecutor aggregationAttributeProcessor; - if (groupBy) { - aggregationAttributeProcessor = new GroupByAggregationAttributeExecutor(attributeAggregator, - innerExpressionExecutors, configReader, siddhiQueryContext); - } else { - aggregationAttributeProcessor = new AggregationAttributeExecutor(attributeAggregator, - innerExpressionExecutors, siddhiQueryContext); - } + attributeAggregatorExecutor.initAggregator(innerExpressionExecutors, processingMode, + outputExpectsExpiredEvents, configReader, groupBy, siddhiQueryContext); +// AbstractAggregationAttributeExecutor aggregationAttributeProcessor; +// if (groupBy) { +// aggregationAttributeProcessor = new GroupByAggregationAttributeExecutor(attributeAggregatorExecutor, +// innerExpressionExecutors, configReader, siddhiQueryContext); +// } else { +// aggregationAttributeProcessor = new AggregationAttributeExecutor(attributeAggregatorExecutor, +// innerExpressionExecutors, siddhiQueryContext); +// } SelectorParser.getContainsAggregatorThreadLocal().set("true"); - return aggregationAttributeProcessor; + return attributeAggregatorExecutor; } } else if (expression instanceof In) { @@ -1451,7 +1448,8 @@ private static ExpressionExecutor[] parseInnerExpression(Expression[] innerExpre boolean groupBy, int defaultStreamEventIndex, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + boolean outputExpectsExpiredEvents, + SiddhiQueryContext siddhiQueryContext) { ExpressionExecutor[] innerExpressionExecutors; if (innerExpressions != null) { if (innerExpressions.length > 0) { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/InputStreamParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/InputStreamParser.java index cd7d57d096..2f9a2e501b 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/InputStreamParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/InputStreamParser.java @@ -83,7 +83,7 @@ public static StreamRuntime parse(InputStream inputStream, executors, streamDefinitionMap, tableDefinitionMap, windowDefinitionMap, aggregationDefinitionMap, tableMap, new MetaStreamEvent(), processStreamReceiver, true, - outputExpectsExpiredEvents, siddhiQueryContext); + outputExpectsExpiredEvents, false, siddhiQueryContext); } else if (inputStream instanceof JoinInputStream) { return JoinInputStreamParser.parseInputStream(((JoinInputStream) inputStream), streamDefinitionMap, tableDefinitionMap, windowDefinitionMap, diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/JoinInputStreamParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/JoinInputStreamParser.java index ec99867fb9..336ce7dbf6 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/JoinInputStreamParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/JoinInputStreamParser.java @@ -126,12 +126,12 @@ public static StreamRuntime parseInputStream(JoinInputStream joinInputStream, leftMetaStreamEvent.setEventType(WINDOW); rightMetaStreamEvent.setEventType(WINDOW); rightProcessStreamReceiver = new MultiProcessStreamReceiver( - joinInputStream.getAllStreamIds().get(0), 1, siddhiQueryContext); + joinInputStream.getAllStreamIds().get(0), 1, new Object(), siddhiQueryContext); rightProcessStreamReceiver.setBatchProcessingAllowed(true); leftProcessStreamReceiver = rightProcessStreamReceiver; } else if (streamDefinitionMap.containsKey(joinInputStream.getAllStreamIds().get(0))) { rightProcessStreamReceiver = new MultiProcessStreamReceiver( - joinInputStream.getAllStreamIds().get(0), 2, siddhiQueryContext); + joinInputStream.getAllStreamIds().get(0), 2, new Object(), siddhiQueryContext); leftProcessStreamReceiver = rightProcessStreamReceiver; } else { throw new SiddhiAppCreationException("Input of join is from static source " + leftInputStreamId + @@ -146,7 +146,7 @@ public static StreamRuntime parseInputStream(JoinInputStream joinInputStream, leftMetaStreamEvent.getEventType() != WINDOW ? null : windowDefinitionMap, leftMetaStreamEvent.getEventType() != AGGREGATE ? null : aggregationDefinitionMap, tableMap, leftMetaStreamEvent, leftProcessStreamReceiver, true, - outputExpectsExpiredEvents, siddhiQueryContext); + outputExpectsExpiredEvents, true, siddhiQueryContext); for (VariableExpressionExecutor variableExpressionExecutor : executors) { variableExpressionExecutor.getPosition()[SiddhiConstants.STREAM_EVENT_CHAIN_INDEX] = 0; @@ -160,7 +160,7 @@ public static StreamRuntime parseInputStream(JoinInputStream joinInputStream, rightMetaStreamEvent.getEventType() != WINDOW ? null : windowDefinitionMap, rightMetaStreamEvent.getEventType() != AGGREGATE ? null : aggregationDefinitionMap, tableMap, rightMetaStreamEvent, rightProcessStreamReceiver, true, - outputExpectsExpiredEvents, siddhiQueryContext); + outputExpectsExpiredEvents, true, siddhiQueryContext); for (int i = size; i < executors.size(); i++) { VariableExpressionExecutor variableExpressionExecutor = executors.get(i); @@ -298,7 +298,7 @@ private static void setStreamRuntimeProcessorChain( TableWindowProcessor tableWindowProcessor = new TableWindowProcessor(tableMap.get(inputStreamId)); tableWindowProcessor.initProcessor(metaStreamEvent, new ExpressionExecutor[0], null, outputExpectsExpiredEvents, - inputStream, siddhiQueryContext); + true, false, inputStream, siddhiQueryContext); streamRuntime.setProcessorChain(tableWindowProcessor); break; case WINDOW: @@ -306,7 +306,7 @@ private static void setStreamRuntimeProcessorChain( windowMap.get(inputStreamId)); windowWindowProcessor.initProcessor(metaStreamEvent, variableExpressionExecutors.toArray(new ExpressionExecutor[0]), null, - outputExpectsExpiredEvents, inputStream, siddhiQueryContext); + outputExpectsExpiredEvents, true, false, inputStream, siddhiQueryContext); streamRuntime.setProcessorChain(windowWindowProcessor); break; case AGGREGATE: @@ -316,7 +316,7 @@ private static void setStreamRuntimeProcessorChain( aggregationRuntime, within, per); aggregateWindowProcessor.initProcessor(metaStreamEvent, variableExpressionExecutors.toArray(new ExpressionExecutor[0]), null, - outputExpectsExpiredEvents, inputStream, siddhiQueryContext); + outputExpectsExpiredEvents, true, false, inputStream, siddhiQueryContext); streamRuntime.setProcessorChain(aggregateWindowProcessor); break; case DEFAULT: @@ -351,7 +351,7 @@ private static FindableProcessor insertJoinProcessorsAndGetFindable(JoinProcesso windowProcessor.initProcessor( ((MetaStreamEvent) streamRuntime.getMetaComplexEvent()), expressionExecutors, configReader, outputExpectsExpiredEvents, - inputStream, siddhiQueryContext); + true, false, inputStream, siddhiQueryContext); lastProcessor = windowProcessor; } catch (Throwable t) { throw new SiddhiAppCreationException(t); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/OutputParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/OutputParser.java index ef2155ad95..52ef963870 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/OutputParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/OutputParser.java @@ -18,17 +18,19 @@ package io.siddhi.core.util.parser; import io.siddhi.core.config.SiddhiQueryContext; -import io.siddhi.core.event.state.StateEventPool; +import io.siddhi.core.event.state.StateEventFactory; import io.siddhi.core.event.stream.MetaStreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.event.stream.converter.ZeroStreamEventConverter; import io.siddhi.core.exception.OperationNotSupportedException; import io.siddhi.core.exception.SiddhiAppCreationException; import io.siddhi.core.query.output.callback.DeleteTableCallback; import io.siddhi.core.query.output.callback.InsertIntoStreamCallback; +import io.siddhi.core.query.output.callback.InsertIntoStreamEndPartitionCallback; import io.siddhi.core.query.output.callback.InsertIntoTableCallback; import io.siddhi.core.query.output.callback.InsertIntoWindowCallback; +import io.siddhi.core.query.output.callback.InsertIntoWindowEndPartitionCallback; import io.siddhi.core.query.output.callback.OutputCallback; import io.siddhi.core.query.output.callback.UpdateOrInsertTableCallback; import io.siddhi.core.query.output.callback.UpdateTableCallback; @@ -71,7 +73,6 @@ import java.util.Map; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ScheduledExecutorService; /** * Class to parse {@link OutputCallback} @@ -91,7 +92,7 @@ public static OutputCallback constructOutputCallback(final OutputStream outStrea table = tableMap.get(id); window = eventWindowMap.get(id); } - StreamEventPool streamEventPool = null; + StreamEventFactory streamEventFactory = null; StreamEventConverter streamEventConverter = null; MetaStreamEvent tableMetaStreamEvent = null; if (table != null) { @@ -106,20 +107,30 @@ public static OutputCallback constructOutputCallback(final OutputStream outStrea matchingTableDefinition.setQueryContextEndIndex(outStream.getQueryContextEndIndex()); tableMetaStreamEvent.addInputDefinition(matchingTableDefinition); - streamEventPool = new StreamEventPool(tableMetaStreamEvent, 10); + streamEventFactory = new StreamEventFactory(tableMetaStreamEvent); streamEventConverter = new ZeroStreamEventConverter(); } //Construct CallBack if (outStream instanceof InsertIntoStream || outStream instanceof ReturnStream) { if (window != null) { - return new InsertIntoWindowCallback(window, outputStreamDefinition, siddhiQueryContext.getName()); + if (!siddhiQueryContext.isPartitioned()) { + return new InsertIntoWindowCallback(window, outputStreamDefinition, siddhiQueryContext.getName()); + } else { + return new InsertIntoWindowEndPartitionCallback(window, outputStreamDefinition, + siddhiQueryContext.getName()); + } } else if (table != null) { DefinitionParserHelper.validateOutputStream(outputStreamDefinition, table.getTableDefinition()); return new InsertIntoTableCallback(table, outputStreamDefinition, convertToStreamEvent, - streamEventPool, streamEventConverter, siddhiQueryContext.getName()); + streamEventFactory, streamEventConverter, siddhiQueryContext.getName()); } else { - return new InsertIntoStreamCallback(outputStreamDefinition, siddhiQueryContext.getName()); + if (!siddhiQueryContext.isPartitioned() || outputStreamDefinition.getId().startsWith("#")) { + return new InsertIntoStreamCallback(outputStreamDefinition, siddhiQueryContext.getName()); + } else { + return new InsertIntoStreamEndPartitionCallback( + outputStreamDefinition, siddhiQueryContext.getName()); + } } } else if (outStream instanceof DeleteStream || outStream instanceof UpdateStream || outStream instanceof UpdateOrInsertStream) { @@ -157,10 +168,10 @@ public static OutputCallback constructOutputCallback(final OutputStream outStrea CompiledCondition compiledCondition = table.compileCondition( (((DeleteStream) outStream).getOnDeleteExpression()), matchingMetaInfoHolder, null, tableMap, siddhiQueryContext); - StateEventPool stateEventPool = new StateEventPool( - matchingMetaInfoHolder.getMetaStateEvent(), 10); + StateEventFactory stateEventFactory = new StateEventFactory( + matchingMetaInfoHolder.getMetaStateEvent()); return new DeleteTableCallback(table, compiledCondition, matchingMetaInfoHolder. - getMatchingStreamEventIndex(), convertToStreamEvent, stateEventPool, streamEventPool, + getMatchingStreamEventIndex(), convertToStreamEvent, stateEventFactory, streamEventFactory, streamEventConverter, siddhiQueryContext.getName()); } catch (SiddhiAppValidationException e) { throw new SiddhiAppCreationException("Cannot create delete for table '" + outStream.getId() + @@ -186,11 +197,11 @@ public static OutputCallback constructOutputCallback(final OutputStream outStrea } CompiledUpdateSet compiledUpdateSet = table.compileUpdateSet(updateSet, matchingMetaInfoHolder, null, tableMap, siddhiQueryContext); - StateEventPool stateEventPool = new StateEventPool( - matchingMetaInfoHolder.getMetaStateEvent(), 10); + StateEventFactory stateEventFactory = new StateEventFactory( + matchingMetaInfoHolder.getMetaStateEvent()); return new UpdateTableCallback(table, compiledCondition, compiledUpdateSet, matchingMetaInfoHolder.getMatchingStreamEventIndex(), convertToStreamEvent, - stateEventPool, streamEventPool, streamEventConverter, siddhiQueryContext.getName()); + stateEventFactory, streamEventFactory, streamEventConverter, siddhiQueryContext.getName()); } catch (SiddhiAppValidationException e) { throw new SiddhiAppCreationException("Cannot create update for table '" + outStream.getId() + "', " + e.getMessageWithOutContext(), e, e.getQueryContextStartIndex(), @@ -217,10 +228,10 @@ public static OutputCallback constructOutputCallback(final OutputStream outStrea } CompiledUpdateSet compiledUpdateSet = table.compileUpdateSet(updateSet, matchingMetaInfoHolder, null, tableMap, siddhiQueryContext); - StateEventPool stateEventPool = new StateEventPool(matchingMetaInfoHolder.getMetaStateEvent(), 10); + StateEventFactory stateEventFactory = new StateEventFactory(matchingMetaInfoHolder.getMetaStateEvent()); return new UpdateOrInsertTableCallback(table, compiledCondition, compiledUpdateSet, matchingMetaInfoHolder.getMatchingStreamEventIndex(), convertToStreamEvent, - stateEventPool, streamEventPool, streamEventConverter, siddhiQueryContext.getName()); + stateEventFactory, streamEventFactory, streamEventConverter, siddhiQueryContext.getName()); } catch (SiddhiAppValidationException e) { throw new SiddhiAppCreationException("Cannot create update or insert into for table '" + @@ -268,10 +279,10 @@ public static OutputCallback constructOutputCallback(OutputStream outStream, Str } } - public static OutputRateLimiter constructOutputRateLimiter(String id, OutputRate outputRate, boolean isGroupBy, - boolean isWindow, ScheduledExecutorService - scheduledExecutorService, SiddhiQueryContext - siddhiQueryContext) { + public static OutputRateLimiter constructOutputRateLimiter(String id, OutputRate outputRate, + boolean isGroupBy, + boolean isWindow, + SiddhiQueryContext siddhiQueryContext) { if (outputRate == null) { return new PassThroughOutputRateLimiter(id); } else if (outputRate instanceof EventOutputRate) { @@ -297,31 +308,26 @@ public static OutputRateLimiter constructOutputRateLimiter(String id, OutputRate } else if (outputRate instanceof TimeOutputRate) { switch (((TimeOutputRate) outputRate).getType()) { case ALL: - return new AllPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue(), - scheduledExecutorService); + return new AllPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue()); case FIRST: if (isGroupBy) { - return new FirstGroupByPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue(), - scheduledExecutorService); + return new FirstGroupByPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue()); } else { - return new FirstPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue(), - scheduledExecutorService); + return new FirstPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue()); } case LAST: if (isGroupBy) { - return new LastGroupByPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue(), - scheduledExecutorService); + return new LastGroupByPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue()); } else { - return new LastPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue(), - scheduledExecutorService); + return new LastPerTimeOutputRateLimiter(id, ((TimeOutputRate) outputRate).getValue()); } } //never happens throw new OperationNotSupportedException(((TimeOutputRate) outputRate).getType() + " not supported in " + "output rate limiting"); } else { - return new WrappedSnapshotOutputRateLimiter(id, ((SnapshotOutputRate) outputRate).getValue(), - scheduledExecutorService, isGroupBy, isWindow, siddhiQueryContext); + return new WrappedSnapshotOutputRateLimiter(((SnapshotOutputRate) outputRate).getValue(), + isGroupBy, isWindow, siddhiQueryContext); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/PartitionParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/PartitionParser.java index 458c49dbb5..0943e467b6 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/PartitionParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/PartitionParser.java @@ -45,13 +45,13 @@ public class PartitionParser { public static PartitionRuntime parse(SiddhiAppRuntimeBuilder siddhiAppRuntimeBuilder, Partition partition, - SiddhiAppContext siddhiAppContext, int queryIndex) { + SiddhiAppContext siddhiAppContext, int queryIndex, int partitionIndex) { ConcurrentMap streamDefinitionMap = siddhiAppRuntimeBuilder.getStreamDefinitionMap(); ConcurrentMap windowDefinitionMap = siddhiAppRuntimeBuilder.getWindowDefinitionMap(); PartitionRuntime partitionRuntime = new PartitionRuntime(streamDefinitionMap, windowDefinitionMap, - siddhiAppRuntimeBuilder.getStreamJunctions(), partition, siddhiAppContext); + siddhiAppRuntimeBuilder.getStreamJunctions(), partition, partitionIndex, siddhiAppContext); validateStreamPartitions(partition.getPartitionTypeMap(), streamDefinitionMap, windowDefinitionMap); for (Query query : partition.getQueryList()) { List executors = new ArrayList(); @@ -68,7 +68,7 @@ public static PartitionRuntime parse(SiddhiAppRuntimeBuilder siddhiAppRuntimeBui siddhiAppRuntimeBuilder.getAggregationMap(), siddhiAppRuntimeBuilder.getWindowMap(), siddhiAppRuntimeBuilder.getLockSynchronizer(), - String.valueOf(queryIndex)); + String.valueOf(queryIndex), true, partitionRuntime.getPartitionName()); queryIndex++; MetaStateEvent metaStateEvent = createMetaEventForPartitioner(queryRuntime.getMetaComplexEvent()); partitionRuntime.addQuery(queryRuntime); @@ -79,8 +79,8 @@ public static PartitionRuntime parse(SiddhiAppRuntimeBuilder siddhiAppRuntimeBui } else { QueryParserHelper.updateVariablePosition(metaStateEvent.getMetaStreamEvent(0), executors); } - partitionRuntime.init(); } + partitionRuntime.init(); return partitionRuntime; } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/QueryParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/QueryParser.java index e6a3a4c437..dabf6fe133 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/QueryParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/QueryParser.java @@ -78,6 +78,8 @@ public class QueryParser { * @param windowMap keyvalue containing event window map. * @param lockSynchronizer Lock synchronizer for sync the lock across queries. * @param queryIndex query index to identify unknown query by number + * @param partitioned is the query partitioned + * @param partitionId The ID of the partition * @return queryRuntime */ public static QueryRuntime parse(Query query, SiddhiAppContext siddhiAppContext, @@ -88,7 +90,7 @@ public static QueryRuntime parse(Query query, SiddhiAppContext siddhiAppContext, Map tableMap, Map aggregationMap, Map windowMap, LockSynchronizer lockSynchronizer, - String queryIndex) { + String queryIndex, boolean partitioned, String partitionId) { List executors = new ArrayList(); QueryRuntime queryRuntime; Element nameElement = null; @@ -103,7 +105,8 @@ public static QueryRuntime parse(Query query, SiddhiAppContext siddhiAppContext, } else { queryName = "query_" + queryIndex; } - SiddhiQueryContext siddhiQueryContext = new SiddhiQueryContext(siddhiAppContext, queryName); + SiddhiQueryContext siddhiQueryContext = new SiddhiQueryContext(siddhiAppContext, queryName, partitionId); + siddhiQueryContext.setPartitioned(partitioned); latencyTracker = QueryParserHelper.createLatencyTracker(siddhiAppContext, siddhiQueryContext.getName(), SiddhiConstants.METRIC_INFIX_QUERIES, null); siddhiQueryContext.setLatencyTracker(latencyTracker); @@ -204,16 +207,17 @@ public static QueryRuntime parse(Query query, SiddhiAppContext siddhiAppContext, OutputRateLimiter outputRateLimiter = OutputParser.constructOutputRateLimiter( query.getOutputStream().getId(), query.getOutputRate(), query.getSelector().getGroupByList().size() != 0, isWindow, - siddhiAppContext.getScheduledExecutorService(), siddhiQueryContext); + siddhiQueryContext); if (outputRateLimiter instanceof WrappedSnapshotOutputRateLimiter) { selector.setBatchingEnabled(false); } - siddhiAppContext.addEternalReferencedHolder(outputRateLimiter); + + boolean groupBy = !query.getSelector().getGroupByList().isEmpty(); OutputCallback outputCallback = OutputParser.constructOutputCallback(query.getOutputStream(), streamRuntime.getMetaComplexEvent().getOutputStreamDefinition(), tableMap, windowMap, !(streamRuntime instanceof SingleStreamRuntime) || - !query.getSelector().getGroupByList().isEmpty(), siddhiQueryContext); + groupBy, siddhiQueryContext); QueryParserHelper.reduceMetaComplexEvent(streamRuntime.getMetaComplexEvent()); QueryParserHelper.updateVariablePosition(streamRuntime.getMetaComplexEvent(), executors); @@ -222,7 +226,7 @@ public static QueryRuntime parse(Query query, SiddhiAppContext siddhiAppContext, selector.setEventPopulator(StateEventPopulatorFactory.constructEventPopulator(streamRuntime .getMetaComplexEvent())); queryRuntime = new QueryRuntime(query, streamRuntime, selector, outputRateLimiter, outputCallback, - streamRuntime.getMetaComplexEvent(), lockWrapper != null, siddhiQueryContext); + streamRuntime.getMetaComplexEvent(), siddhiQueryContext); if (outputRateLimiter instanceof WrappedSnapshotOutputRateLimiter) { selector.setBatchingEnabled(false); @@ -230,7 +234,7 @@ public static QueryRuntime parse(Query query, SiddhiAppContext siddhiAppContext, .init(streamRuntime.getMetaComplexEvent().getOutputStreamDefinition().getAttributeList().size(), selector.getAttributeProcessorList(), streamRuntime.getMetaComplexEvent()); } - outputRateLimiter.init(lockWrapper, siddhiQueryContext); + outputRateLimiter.init(lockWrapper, groupBy, siddhiQueryContext); } catch (DuplicateDefinitionException e) { if (nameElement != null) { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SchedulerParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SchedulerParser.java index b6497176fc..9aea41a0e5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SchedulerParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SchedulerParser.java @@ -18,7 +18,7 @@ package io.siddhi.core.util.parser; -import io.siddhi.core.config.SiddhiAppContext; +import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.util.Schedulable; import io.siddhi.core.util.Scheduler; @@ -35,14 +35,14 @@ private SchedulerParser() { * Create Scheduler object. * * @param singleThreadEntryValve Schedulable - * @param siddhiAppContext SiddhiAppContext + * @param siddhiQueryContext SiddhiAppContext * @return Scheduler instance */ - public static Scheduler parse(Schedulable - singleThreadEntryValve, SiddhiAppContext siddhiAppContext) { + public static Scheduler parse(Schedulable singleThreadEntryValve, SiddhiQueryContext siddhiQueryContext) { - Scheduler scheduler = new Scheduler(singleThreadEntryValve, siddhiAppContext); - siddhiAppContext.addScheduler(scheduler); + Scheduler scheduler = new Scheduler(singleThreadEntryValve, siddhiQueryContext); + siddhiQueryContext.getSiddhiAppContext().addScheduler(scheduler); + siddhiQueryContext.getSiddhiAppContext().addEternalReferencedHolder(scheduler); return scheduler; } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SelectorParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SelectorParser.java index 0797ced26d..09e4076148 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SelectorParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SelectorParser.java @@ -85,13 +85,13 @@ public static QuerySelector parse(Selector selector, OutputStream outputStream, .getOutputEventType() == OutputStream.OutputEventType.ALL_EVENTS) { expiredOn = true; } - + boolean groupBy = !selector.getGroupByList().isEmpty(); id = outputStream.getId(); containsAggregatorThreadLocal.remove(); QuerySelector querySelector = new QuerySelector(id, selector, currentOn, expiredOn, siddhiQueryContext); List attributeProcessors = getAttributeProcessors(selector, id, metaComplexEvent, tableMap, variableExpressionExecutors, outputStream, metaPosition, - processingMode, outputExpectsExpiredEvents, siddhiQueryContext); + processingMode, outputExpectsExpiredEvents, groupBy, siddhiQueryContext); querySelector.setAttributeProcessorList(attributeProcessors, "true".equals(containsAggregatorThreadLocal.get())); containsAggregatorThreadLocal.remove(); @@ -144,8 +144,8 @@ public static QuerySelector parse(Selector selector, OutputStream outputStream, * @param outputStream output stream * @param processingMode processing mode of the query * @param outputExpectsExpiredEvents is expired events sent as output - * @param siddhiQueryContext current siddhi query context - * @return list of AttributeProcessors + * @param groupBy is Attributes groupBy + * @param siddhiQueryContext current siddhi query context @return list of AttributeProcessors */ private static List getAttributeProcessors(Selector selector, String id, MetaComplexEvent metaComplexEvent, @@ -155,7 +155,9 @@ private static List getAttributeProcessors(Selector selector OutputStream outputStream, int metaPosition, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + boolean outputExpectsExpiredEvents, + boolean groupBy, + SiddhiQueryContext siddhiQueryContext) { List attributeProcessorList = new ArrayList<>(); StreamDefinition outputDefinition = StreamDefinition.id(id); @@ -168,7 +170,13 @@ private static List getAttributeProcessors(Selector selector List attributeList = ((MetaStreamEvent) metaComplexEvent).getLastInputDefinition() .getAttributeList(); for (Attribute attribute : attributeList) { - outputAttributes.add(new OutputAttribute(new Variable(attribute.getName()))); + Variable variable = new Variable(attribute.getName()); + variable.setQueryContextStartIndex(selector.getQueryContextStartIndex()); + variable.setQueryContextEndIndex(selector.getQueryContextEndIndex()); + OutputAttribute outputAttribute = new OutputAttribute(variable); + outputAttribute.setQueryContextStartIndex(selector.getQueryContextStartIndex()); + outputAttribute.setQueryContextEndIndex(selector.getQueryContextEndIndex()); + outputAttributes.add(outputAttribute); } } else { int position = 0; @@ -176,7 +184,12 @@ private static List getAttributeProcessors(Selector selector if (metaPosition == SiddhiConstants.UNKNOWN_STATE || metaPosition == position) { List attributeList = metaStreamEvent.getLastInputDefinition().getAttributeList(); for (Attribute attribute : attributeList) { - OutputAttribute outputAttribute = new OutputAttribute(new Variable(attribute.getName())); + Variable variable = new Variable(attribute.getName()); + variable.setQueryContextStartIndex(selector.getQueryContextStartIndex()); + variable.setQueryContextEndIndex(selector.getQueryContextEndIndex()); + OutputAttribute outputAttribute = new OutputAttribute(variable); + outputAttribute.setQueryContextStartIndex(selector.getQueryContextStartIndex()); + outputAttribute.setQueryContextEndIndex(selector.getQueryContextEndIndex()); if (!outputAttributes.contains(outputAttribute)) { outputAttributes.add(outputAttribute); } else { @@ -201,7 +214,7 @@ private static List getAttributeProcessors(Selector selector ExpressionExecutor expressionExecutor = ExpressionParser.parseExpression(outputAttribute.getExpression(), metaComplexEvent, SiddhiConstants.UNKNOWN_STATE, tableMap, variableExpressionExecutors, - !(selector.getGroupByList().isEmpty()), 0, processingMode, + groupBy, 0, processingMode, outputExpectsExpiredEvents, siddhiQueryContext); if (expressionExecutor instanceof VariableExpressionExecutor) { //for variables we will directly put // value at conversion stage diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SiddhiAppParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SiddhiAppParser.java index c0637bed1b..c577a7723f 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SiddhiAppParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/SiddhiAppParser.java @@ -24,8 +24,8 @@ import io.siddhi.core.partition.PartitionRuntime; import io.siddhi.core.query.QueryRuntime; import io.siddhi.core.stream.StreamJunction; -import io.siddhi.core.util.ElementIdGenerator; import io.siddhi.core.util.ExceptionUtil; +import io.siddhi.core.util.IdGenerator; import io.siddhi.core.util.SiddhiAppRuntimeBuilder; import io.siddhi.core.util.SiddhiConstants; import io.siddhi.core.util.ThreadBarrier; @@ -48,6 +48,11 @@ import io.siddhi.query.api.execution.ExecutionElement; import io.siddhi.query.api.execution.partition.Partition; import io.siddhi.query.api.execution.query.Query; +import io.siddhi.query.api.execution.query.input.handler.StreamHandler; +import io.siddhi.query.api.execution.query.input.stream.JoinInputStream; +import io.siddhi.query.api.execution.query.input.stream.SingleInputStream; +import io.siddhi.query.api.execution.query.input.stream.StateInputStream; +import io.siddhi.query.api.expression.condition.In; import io.siddhi.query.api.util.AnnotationHelper; import io.siddhi.query.compiler.SiddhiCompiler; import io.siddhi.query.compiler.exception.SiddhiParserException; @@ -210,7 +215,7 @@ public static SiddhiAppRuntimeBuilder parse(SiddhiApp siddhiApp, String siddhiAp siddhiAppContext.setTimestampGenerator(new TimestampGeneratorImpl(siddhiAppContext)); } siddhiAppContext.setSnapshotService(new SnapshotService(siddhiAppContext)); - siddhiAppContext.setElementIdGenerator(new ElementIdGenerator(siddhiAppContext.getName())); + siddhiAppContext.setIdGenerator(new IdGenerator()); } catch (DuplicateAnnotationException e) { throw new DuplicateAnnotationException(e.getMessageWithOutContext() + " for the same Siddhi app " + @@ -226,16 +231,21 @@ public static SiddhiAppRuntimeBuilder parse(SiddhiApp siddhiApp, String siddhiAp defineFunctionDefinitions(siddhiAppRuntimeBuilder, siddhiApp.getFunctionDefinitionMap(), siddhiAppContext); defineAggregationDefinitions(siddhiAppRuntimeBuilder, siddhiApp.getAggregationDefinitionMap(), siddhiAppContext); + //todo fix for query API usecase + List findExecutedElements = getFindExecutedElements(siddhiApp); for (Window window : siddhiAppRuntimeBuilder.getWindowMap().values()) { + try { - window.init(siddhiAppRuntimeBuilder.getTableMap(), siddhiAppRuntimeBuilder - .getWindowMap(), window.getWindowDefinition().getId()); + window.init(siddhiAppRuntimeBuilder.getTableMap(), siddhiAppRuntimeBuilder.getWindowMap(), + window.getWindowDefinition().getId(), + findExecutedElements.contains(window.getWindowDefinition().getId())); } catch (Throwable t) { ExceptionUtil.populateQueryContext(t, window.getWindowDefinition(), siddhiAppContext); throw t; } } int queryIndex = 1; + int partitionIndex = 1; for (ExecutionElement executionElement : siddhiApp.getExecutionElementList()) { if (executionElement instanceof Query) { try { @@ -248,8 +258,9 @@ public static SiddhiAppRuntimeBuilder parse(SiddhiApp siddhiApp, String siddhiAp siddhiAppRuntimeBuilder.getAggregationMap(), siddhiAppRuntimeBuilder.getWindowMap(), siddhiAppRuntimeBuilder.getLockSynchronizer(), - String.valueOf(queryIndex)); + String.valueOf(queryIndex), false, SiddhiConstants.PARTITION_ID_DEFAULT); siddhiAppRuntimeBuilder.addQuery(queryRuntime); + siddhiAppContext.addEternalReferencedHolder(queryRuntime); queryIndex++; } catch (Throwable t) { ExceptionUtil.populateQueryContext(t, (Query) executionElement, siddhiAppContext); @@ -258,9 +269,10 @@ public static SiddhiAppRuntimeBuilder parse(SiddhiApp siddhiApp, String siddhiAp } else { try { PartitionRuntime partitionRuntime = PartitionParser.parse(siddhiAppRuntimeBuilder, - (Partition) executionElement, siddhiAppContext, queryIndex); + (Partition) executionElement, siddhiAppContext, queryIndex, partitionIndex); siddhiAppRuntimeBuilder.addPartition(partitionRuntime); queryIndex += ((Partition) executionElement).getQueryList().size(); + partitionIndex++; } catch (Throwable t) { ExceptionUtil.populateQueryContext(t, (Partition) executionElement, siddhiAppContext); throw t; @@ -272,6 +284,50 @@ public static SiddhiAppRuntimeBuilder parse(SiddhiApp siddhiApp, String siddhiAp return siddhiAppRuntimeBuilder; } + private static List getFindExecutedElements(SiddhiApp siddhiApp) { + List findExecutedElements = new ArrayList<>(); + for (ExecutionElement executionElement : siddhiApp.getExecutionElementList()) { + if (executionElement instanceof Query) { + List streamHandlers = new ArrayList<>(); + if (((Query) executionElement).getInputStream() instanceof JoinInputStream) { + findExecutedElements.addAll(((Query) executionElement).getInputStream().getAllStreamIds()); + streamHandlers.addAll(((SingleInputStream) ((JoinInputStream) ((Query) executionElement).getInputStream()).getLeftInputStream()).getStreamHandlers()); + streamHandlers.addAll(((SingleInputStream) ((JoinInputStream) ((Query) executionElement).getInputStream()).getRightInputStream()).getStreamHandlers()); + } else if (((Query) executionElement).getInputStream() instanceof SingleInputStream) { + streamHandlers.addAll(((SingleInputStream) ((Query) executionElement).getInputStream()).getStreamHandlers()); + } else if (((Query) executionElement).getInputStream() instanceof StateInputStream) { + streamHandlers.addAll((((StateInputStream) ((Query) executionElement).getInputStream()).getStreamHandlers())); + } + for (StreamHandler streamHandler : streamHandlers) { + if (streamHandler instanceof In) { + findExecutedElements.add(((In) streamHandler).getSourceId()); + } + } + } else { + List queries = ((Partition) executionElement).getQueryList(); + for (Query query : queries) { + List streamHandlers = new ArrayList<>(); + if (query.getInputStream() instanceof JoinInputStream) { + findExecutedElements.addAll(query.getInputStream().getAllStreamIds()); + streamHandlers.addAll(((SingleInputStream) ((JoinInputStream) query.getInputStream()).getLeftInputStream()).getStreamHandlers()); + streamHandlers.addAll(((SingleInputStream) ((JoinInputStream) query.getInputStream()).getRightInputStream()).getStreamHandlers()); + } else if (query.getInputStream() instanceof SingleInputStream) { + streamHandlers.addAll(((SingleInputStream) query.getInputStream()).getStreamHandlers()); + } else if (query.getInputStream() instanceof StateInputStream) { + streamHandlers.addAll((((StateInputStream) query.getInputStream()).getStreamHandlers())); + } + for (StreamHandler streamHandler : streamHandlers) { + if (streamHandler instanceof In) { + findExecutedElements.add(((In) streamHandler).getSourceId()); + } + } + } + } + + } + return findExecutedElements; + } + private static void defineTriggerDefinitions(SiddhiAppRuntimeBuilder siddhiAppRuntimeBuilder, Map triggerDefinitionMap, SiddhiAppContext siddhiAppContext) { 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 a9c318c46c..528f40d943 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 @@ -75,6 +75,7 @@ public class SingleInputStreamParser { * @param processStreamReceiver ProcessStreamReceiver * @param supportsBatchProcessing supports batch processing * @param outputExpectsExpiredEvents is expired events sent as output + * @param findToBeExecuted find will be executed in the steam stores * @param siddhiQueryContext @return SingleStreamRuntime */ public static SingleStreamRuntime parseInputStream(SingleInputStream inputStream, @@ -88,6 +89,7 @@ public static SingleStreamRuntime parseInputStream(SingleInputStream inputStream ProcessStreamReceiver processStreamReceiver, boolean supportsBatchProcessing, boolean outputExpectsExpiredEvents, + boolean findToBeExecuted, SiddhiQueryContext siddhiQueryContext) { Processor processor = null; EntryValveProcessor entryValveProcessor = null; @@ -120,7 +122,7 @@ public static SingleStreamRuntime parseInputStream(SingleInputStream inputStream for (StreamHandler handler : inputStream.getStreamHandlers()) { Processor currentProcessor = generateProcessor(handler, metaComplexEvent, variableExpressionExecutors, tableMap, supportsBatchProcessing, - outputExpectsExpiredEvents, siddhiQueryContext); + outputExpectsExpiredEvents, findToBeExecuted, siddhiQueryContext); if (currentProcessor instanceof SchedulingProcessor) { if (entryValveProcessor == null) { @@ -132,8 +134,7 @@ public static SingleStreamRuntime parseInputStream(SingleInputStream inputStream processor.setToLast(entryValveProcessor); } } - Scheduler scheduler = SchedulerParser.parse(entryValveProcessor, - siddhiQueryContext.getSiddhiAppContext()); + Scheduler scheduler = SchedulerParser.parse(entryValveProcessor, siddhiQueryContext); ((SchedulingProcessor) currentProcessor).setScheduler(scheduler); } if (currentProcessor instanceof AbstractStreamProcessor) { @@ -158,7 +159,8 @@ public static SingleStreamRuntime parseInputStream(SingleInputStream inputStream public static Processor generateProcessor(StreamHandler streamHandler, MetaComplexEvent metaEvent, List variableExpressionExecutors, Map tableMap, - boolean supportsBatchProcessing, boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + boolean supportsBatchProcessing, boolean outputExpectsExpiredEvents, + boolean findToBeExecuted, SiddhiQueryContext siddhiQueryContext) { Expression[] parameters = streamHandler.getParameters(); MetaStreamEvent metaStreamEvent; int stateIndex = SiddhiConstants.UNKNOWN_STATE; @@ -206,7 +208,7 @@ public static Processor generateProcessor(StreamHandler streamHandler, MetaCompl generateConfigReader(((Window) streamHandler).getNamespace(), ((Window) streamHandler).getName()); windowProcessor.initProcessor(metaStreamEvent, attributeExpressionExecutors, - configReader, outputExpectsExpiredEvents, streamHandler, siddhiQueryContext); + configReader, outputExpectsExpiredEvents, findToBeExecuted, false, streamHandler, siddhiQueryContext); return windowProcessor; } else if (streamHandler instanceof StreamFunction) { @@ -221,7 +223,7 @@ public static Processor generateProcessor(StreamHandler streamHandler, MetaCompl StreamProcessorExtensionHolder.getInstance(siddhiQueryContext.getSiddhiAppContext())); abstractStreamProcessor.initProcessor(metaStreamEvent, attributeExpressionExecutors, configReader, - outputExpectsExpiredEvents, streamHandler, siddhiQueryContext); + outputExpectsExpiredEvents, false, false, streamHandler, siddhiQueryContext); return abstractStreamProcessor; } catch (SiddhiAppCreationException e) { if (!e.isClassLoadingIssue()) { @@ -234,7 +236,7 @@ public static Processor generateProcessor(StreamHandler streamHandler, MetaCompl (Extension) streamHandler, StreamFunctionProcessorExtensionHolder.getInstance(siddhiQueryContext.getSiddhiAppContext())); abstractStreamProcessor.initProcessor(metaStreamEvent, attributeExpressionExecutors, - configReader, outputExpectsExpiredEvents, streamHandler, siddhiQueryContext); + configReader, outputExpectsExpiredEvents, false, false, streamHandler, siddhiQueryContext); return abstractStreamProcessor; } else { throw new SiddhiAppCreationException(streamHandler.getClass().getName() + " is not supported", diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/StateInputStreamParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/StateInputStreamParser.java index af05aef372..2281830f6d 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/StateInputStreamParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/StateInputStreamParser.java @@ -87,39 +87,40 @@ public static StateStreamRuntime parseInputStream(StateInputStream stateInputStr StateStreamRuntime stateStreamRuntime = new StateStreamRuntime(siddhiQueryContext, metaStateEvent); - String defaultLockKey = ""; - + Object patternSyncObject = new Object(); for (String streamId : stateInputStream.getAllStreamIds()) { int streamCount = stateInputStream.getStreamCount(streamId); if (streamCount == 1) { if (stateInputStream.getStateType() == StateInputStream.Type.SEQUENCE) { processStreamReceiverMap.put(streamId, new SequenceSingleProcessStreamReceiver(streamId, - stateStreamRuntime, defaultLockKey, siddhiQueryContext)); + stateStreamRuntime, patternSyncObject, siddhiQueryContext)); } else { processStreamReceiverMap.put(streamId, new PatternSingleProcessStreamReceiver(streamId, - defaultLockKey, siddhiQueryContext)); + patternSyncObject, siddhiQueryContext)); } } else { if (stateInputStream.getStateType() == StateInputStream.Type.SEQUENCE) { processStreamReceiverMap.put(streamId, new SequenceMultiProcessStreamReceiver(streamId, - streamCount, stateStreamRuntime, siddhiQueryContext)); + streamCount, stateStreamRuntime, patternSyncObject, siddhiQueryContext)); } else { processStreamReceiverMap.put(streamId, new PatternMultiProcessStreamReceiver(streamId, - streamCount, siddhiQueryContext)); + streamCount, patternSyncObject, siddhiQueryContext)); } } } StateElement stateElement = stateInputStream.getStateElement(); List preStateProcessors = new ArrayList<>(); + List startupPreStateProcessors = new ArrayList<>(); InnerStateRuntime innerStateRuntime = parse(stateElement, streamDefinitionMap, tableDefinitionMap, windowDefinitionMap, aggregationDefinitionMap, tableMap, metaStateEvent, variableExpressionExecutors, processStreamReceiverMap, null, null, stateInputStream.getStateType(), - preStateProcessors, true, siddhiQueryContext); + preStateProcessors, true, startupPreStateProcessors, siddhiQueryContext); stateStreamRuntime.setInnerStateRuntime(innerStateRuntime); + stateStreamRuntime.setStartupPreStateProcessors(startupPreStateProcessors); if (stateInputStream.getWithinTime() != null) { List startStateIdList = new ArrayList<>(); @@ -153,7 +154,9 @@ private static InnerStateRuntime parse(StateElement stateElement, StreamPostStateProcessor streamPostStateProcessor, StateInputStream.Type stateType, List preStateProcessors, - boolean isStartState, SiddhiQueryContext siddhiQueryContext) { + boolean isStartState, + List startupPreStateProcessors, + SiddhiQueryContext siddhiQueryContext) { if (stateElement instanceof StreamStateElement) { @@ -164,7 +167,7 @@ private static InnerStateRuntime parse(StateElement stateElement, basicSingleInputStream, variableExpressionExecutors, streamDefinitionMap, tableDefinitionMap, windowDefinitionMap, aggregationDefinitionMap, tableMap, metaStateEvent, processStreamReceiverMap.get(basicSingleInputStream.getUniqueStreamIds().get(0)), - false, false, siddhiQueryContext); + false, false, false, siddhiQueryContext); int stateIndex = metaStateEvent.getStreamEventCount() - 1; if (streamPreStateProcessor == null) { @@ -175,13 +178,12 @@ private static InnerStateRuntime parse(StateElement stateElement, ((AbsentStreamStateElement) stateElement).getWaitingTime().value()); // Set the scheduler - siddhiQueryContext.getSiddhiAppContext().addEternalReferencedHolder(absentProcessor); + startupPreStateProcessors.add(absentProcessor); EntryValveProcessor entryValveProcessor = new EntryValveProcessor( siddhiQueryContext.getSiddhiAppContext()); entryValveProcessor.setToLast(absentProcessor); Scheduler scheduler = SchedulerParser.parse( - entryValveProcessor, - siddhiQueryContext.getSiddhiAppContext()); + entryValveProcessor, siddhiQueryContext); absentProcessor.setScheduler(scheduler); // Assign the AbsentStreamPreStateProcessor to streamPreStateProcessor @@ -224,14 +226,14 @@ private static InnerStateRuntime parse(StateElement stateElement, variableExpressionExecutors, processStreamReceiverMap, streamPreStateProcessor, streamPostStateProcessor, - stateType, preStateProcessors, isStartState, siddhiQueryContext); + stateType, preStateProcessors, isStartState, startupPreStateProcessors, siddhiQueryContext); StateElement nextElement = ((NextStateElement) stateElement).getNextStateElement(); InnerStateRuntime nextInnerStateRuntime = parse(nextElement, streamDefinitionMap, tableDefinitionMap, windowDefinitionMap, aggregationDefinitionMap, tableMap, metaStateEvent, variableExpressionExecutors, processStreamReceiverMap, streamPreStateProcessor, streamPostStateProcessor, stateType, preStateProcessors, - false, siddhiQueryContext); + false, startupPreStateProcessors, siddhiQueryContext); currentInnerStateRuntime.getLastProcessor().setNextStatePreProcessor(nextInnerStateRuntime .getFirstProcessor()); @@ -259,7 +261,7 @@ private static InnerStateRuntime parse(StateElement stateElement, windowDefinitionMap, aggregationDefinitionMap, tableMap, metaStateEvent, variableExpressionExecutors, processStreamReceiverMap, streamPreStateProcessor, streamPostStateProcessor, stateType, - withinEveryPreStateProcessors, isStartState, siddhiQueryContext); + withinEveryPreStateProcessors, isStartState, startupPreStateProcessors, siddhiQueryContext); EveryInnerStateRuntime everyInnerStateRuntime = new EveryInnerStateRuntime(innerStateRuntime, stateType); @@ -289,14 +291,12 @@ private static InnerStateRuntime parse(StateElement stateElement, .getStreamStateElement1()).getWaitingTime()); // Set the scheduler - siddhiQueryContext.getSiddhiAppContext().addEternalReferencedHolder((AbsentLogicalPreStateProcessor) - logicalPreStateProcessor1); + startupPreStateProcessors.add(logicalPreStateProcessor1); EntryValveProcessor entryValveProcessor = new EntryValveProcessor( siddhiQueryContext.getSiddhiAppContext()); entryValveProcessor.setToLast(logicalPreStateProcessor1); Scheduler scheduler = SchedulerParser.parse( - entryValveProcessor, - siddhiQueryContext.getSiddhiAppContext()); + entryValveProcessor, siddhiQueryContext); ((SchedulingProcessor) logicalPreStateProcessor1).setScheduler(scheduler); } else { logicalPreStateProcessor1 = new LogicalPreStateProcessor(type, stateType); @@ -314,13 +314,11 @@ private static InnerStateRuntime parse(StateElement stateElement, logicalPreStateProcessor2 = new AbsentLogicalPreStateProcessor(type, stateType, ((AbsentStreamStateElement) ((LogicalStateElement) stateElement).getStreamStateElement2()) .getWaitingTime()); - siddhiQueryContext.getSiddhiAppContext().addEternalReferencedHolder((AbsentLogicalPreStateProcessor) - logicalPreStateProcessor2); + startupPreStateProcessors.add(logicalPreStateProcessor2); EntryValveProcessor entryValveProcessor = new EntryValveProcessor( siddhiQueryContext.getSiddhiAppContext()); entryValveProcessor.setToLast(logicalPreStateProcessor2); - Scheduler scheduler = SchedulerParser.parse(entryValveProcessor, - siddhiQueryContext.getSiddhiAppContext()); + Scheduler scheduler = SchedulerParser.parse(entryValveProcessor, siddhiQueryContext); ((SchedulingProcessor) logicalPreStateProcessor2).setScheduler(scheduler); } else { logicalPreStateProcessor2 = new LogicalPreStateProcessor(type, stateType); @@ -347,14 +345,14 @@ private static InnerStateRuntime parse(StateElement stateElement, windowDefinitionMap, aggregationDefinitionMap, tableMap, metaStateEvent, variableExpressionExecutors, processStreamReceiverMap, logicalPreStateProcessor2, logicalPostStateProcessor2, - stateType, preStateProcessors, isStartState, siddhiQueryContext); + stateType, preStateProcessors, isStartState, startupPreStateProcessors, siddhiQueryContext); StateElement stateElement1 = ((LogicalStateElement) stateElement).getStreamStateElement1(); InnerStateRuntime innerStateRuntime1 = parse(stateElement1, streamDefinitionMap, tableDefinitionMap, windowDefinitionMap, aggregationDefinitionMap, tableMap, metaStateEvent, variableExpressionExecutors, processStreamReceiverMap, logicalPreStateProcessor1, logicalPostStateProcessor1, stateType, - preStateProcessors, isStartState, siddhiQueryContext); + preStateProcessors, isStartState, startupPreStateProcessors, siddhiQueryContext); LogicalInnerStateRuntime logicalInnerStateRuntime = new LogicalInnerStateRuntime( @@ -394,7 +392,7 @@ private static InnerStateRuntime parse(StateElement stateElement, windowDefinitionMap, aggregationDefinitionMap, tableMap, metaStateEvent, variableExpressionExecutors, processStreamReceiverMap, countPreStateProcessor, countPostStateProcessor, stateType, - preStateProcessors, isStartState, siddhiQueryContext); + preStateProcessors, isStartState, startupPreStateProcessors, siddhiQueryContext); return new CountInnerStateRuntime((StreamInnerStateRuntime) innerStateRuntime); } else { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/StoreQueryParser.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/StoreQueryParser.java index aab28804a9..171df37c71 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/StoreQueryParser.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/StoreQueryParser.java @@ -22,7 +22,7 @@ import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.event.state.MetaStateEvent; -import io.siddhi.core.event.state.StateEventPool; +import io.siddhi.core.event.state.StateEventFactory; import io.siddhi.core.event.state.populater.StateEventPopulatorFactory; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.MetaStreamEvent.EventType; @@ -109,7 +109,7 @@ public static StoreQueryRuntime parse(StoreQuery storeQuery, SiddhiAppContext si SiddhiQueryContext siddhiQueryContext; Expression onCondition; - SnapshotService.getSkipSnapshotableThreadLocal().set(true); + SnapshotService.getSkipStateStorageThreadLocal().set(true); switch (storeQuery.getType()) { case FIND: @@ -149,7 +149,8 @@ public static StoreQueryRuntime parse(StoreQuery storeQuery, SiddhiAppContext si table = tableMap.get(inputStore.getStoreId()); if (table != null) { return constructStoreQueryRuntime(table, storeQuery, tableMap, windowMap, - metaPosition, onCondition, metaStreamEvent, variableExpressionExecutors, lockWrapper, siddhiQueryContext); + metaPosition, onCondition, metaStreamEvent, variableExpressionExecutors, lockWrapper, + siddhiQueryContext); } else { AggregationRuntime aggregation = aggregationMap.get(inputStore.getStoreId()); if (aggregation != null) { @@ -170,7 +171,7 @@ public static StoreQueryRuntime parse(StoreQuery storeQuery, SiddhiAppContext si } } finally { - SnapshotService.getSkipSnapshotableThreadLocal().set(null); + SnapshotService.getSkipStateStorageThreadLocal().set(null); } case INSERT: InsertIntoStream inserIntoStreamt = (InsertIntoStream) storeQuery.getOutputStream(); @@ -228,7 +229,7 @@ private static StoreQueryRuntime getStoreQueryRuntime(StoreQuery storeQuery, } } finally { - SnapshotService.getSkipSnapshotableThreadLocal().set(null); + SnapshotService.getSkipStateStorageThreadLocal().set(null); } } @@ -248,7 +249,8 @@ private static StoreQueryRuntime constructStoreQueryRuntime( FindStoreQueryRuntime findStoreQueryRuntime = new FindStoreQueryRuntime(window, compiledCondition, siddhiQueryContext.getName(), metaStreamEvent); populateFindStoreQueryRuntime(findStoreQueryRuntime, metaStreamInfoHolder, storeQuery.getSelector(), - variableExpressionExecutors, tableMap, windowMap, metaPosition, lockWrapper, siddhiQueryContext); + variableExpressionExecutors, tableMap, windowMap, metaPosition, + !storeQuery.getSelector().getGroupByList().isEmpty(), lockWrapper, siddhiQueryContext); return findStoreQueryRuntime; } @@ -272,11 +274,11 @@ private static StoreQueryRuntime constructStoreQueryRuntime(AggregationRuntime a metaStreamInfoHolder = ((IncrementalAggregateCompileCondition) compiledCondition). getAlteredMatchingMetaInfoHolder(); FindStoreQueryRuntime findStoreQueryRuntime = new FindStoreQueryRuntime(aggregation, compiledCondition, - siddhiQueryContext.getName(), metaStreamEvent); + siddhiQueryContext.getName(), metaStreamEvent, siddhiQueryContext); metaPosition = 1; populateFindStoreQueryRuntime(findStoreQueryRuntime, metaStreamInfoHolder, storeQuery.getSelector(), variableExpressionExecutors, tableMap, windowMap, - metaPosition, lockWrapper, siddhiQueryContext); + metaPosition, !storeQuery.getSelector().getGroupByList().isEmpty(), lockWrapper, siddhiQueryContext); ComplexEventPopulater complexEventPopulater = StreamEventPopulaterFactory.constructEventPopulator( metaStreamInfoHolder.getMetaStateEvent().getMetaStreamEvent(0), 0, ((IncrementalAggregateCompileCondition) compiledCondition).getAdditionalAttributes()); @@ -378,7 +380,8 @@ private static StoreQueryRuntime constructRegularStoreQueryRuntime(Table table, siddhiQueryContext.getName(), metaStreamEvent); populateFindStoreQueryRuntime(findStoreQueryRuntime, matchingMetaInfoHolder, storeQuery.getSelector(), variableExpressionExecutors, - tableMap, windowMap, metaPosition, lockWrapper, siddhiQueryContext); + tableMap, windowMap, metaPosition, !storeQuery.getSelector().getGroupByList().isEmpty(), + lockWrapper, siddhiQueryContext); return findStoreQueryRuntime; case INSERT: initMetaStreamEvent(metaStreamEvent, getInputDefinition(storeQuery, table)); @@ -389,8 +392,8 @@ private static StoreQueryRuntime constructRegularStoreQueryRuntime(Table table, InsertStoreQueryRuntime insertStoreQueryRuntime = new InsertStoreQueryRuntime(siddhiQueryContext.getName(), metaStreamEvent); - insertStoreQueryRuntime.setStateEventPool( - new StateEventPool(matchingMetaInfoHolder.getMetaStateEvent(), 5)); + insertStoreQueryRuntime.setStateEventFactory( + new StateEventFactory(matchingMetaInfoHolder.getMetaStateEvent())); insertStoreQueryRuntime.setSelector(querySelector); insertStoreQueryRuntime.setOutputAttributes(matchingMetaInfoHolder.getMetaStateEvent() .getOutputStreamDefinition().getAttributeList()); @@ -405,8 +408,8 @@ private static StoreQueryRuntime constructRegularStoreQueryRuntime(Table table, DeleteStoreQueryRuntime deleteStoreQueryRuntime = new DeleteStoreQueryRuntime(siddhiQueryContext.getName(), metaStreamEvent); - deleteStoreQueryRuntime.setStateEventPool( - new StateEventPool(matchingMetaInfoHolder.getMetaStateEvent(), 5)); + deleteStoreQueryRuntime.setStateEventFactory( + new StateEventFactory(matchingMetaInfoHolder.getMetaStateEvent())); deleteStoreQueryRuntime.setSelector(querySelector); deleteStoreQueryRuntime.setOutputAttributes(matchingMetaInfoHolder.getMetaStateEvent() .getOutputStreamDefinition().getAttributeList()); @@ -421,8 +424,8 @@ private static StoreQueryRuntime constructRegularStoreQueryRuntime(Table table, UpdateStoreQueryRuntime updateStoreQueryRuntime = new UpdateStoreQueryRuntime(siddhiQueryContext.getName(), metaStreamEvent); - updateStoreQueryRuntime.setStateEventPool( - new StateEventPool(matchingMetaInfoHolder.getMetaStateEvent(), 5)); + updateStoreQueryRuntime.setStateEventFactory( + new StateEventFactory(matchingMetaInfoHolder.getMetaStateEvent())); updateStoreQueryRuntime.setSelector(querySelector); updateStoreQueryRuntime.setOutputAttributes(matchingMetaInfoHolder.getMetaStateEvent() .getOutputStreamDefinition().getAttributeList()); @@ -437,8 +440,8 @@ private static StoreQueryRuntime constructRegularStoreQueryRuntime(Table table, UpdateOrInsertStoreQueryRuntime updateOrInsertIntoStoreQueryRuntime = new UpdateOrInsertStoreQueryRuntime(siddhiQueryContext.getName(), metaStreamEvent); - updateOrInsertIntoStoreQueryRuntime.setStateEventPool( - new StateEventPool(matchingMetaInfoHolder.getMetaStateEvent(), 5)); + updateOrInsertIntoStoreQueryRuntime.setStateEventFactory( + new StateEventFactory(matchingMetaInfoHolder.getMetaStateEvent())); updateOrInsertIntoStoreQueryRuntime.setSelector(querySelector); updateOrInsertIntoStoreQueryRuntime.setOutputAttributes(matchingMetaInfoHolder.getMetaStateEvent() .getOutputStreamDefinition().getAttributeList()); @@ -464,14 +467,14 @@ private static void populateFindStoreQueryRuntime(FindStoreQueryRuntime findStor MatchingMetaInfoHolder metaStreamInfoHolder, Selector selector, List variableExpressionExecutors, Map tableMap, Map windowMap, - int metaPosition, LockWrapper lockWrapper, + int metaPosition, boolean groupBy, LockWrapper lockWrapper, SiddhiQueryContext siddhiQueryContext) { ReturnStream returnStream = new ReturnStream(OutputStream.OutputEventType.CURRENT_EVENTS); QuerySelector querySelector = SelectorParser.parse(selector, returnStream, metaStreamInfoHolder.getMetaStateEvent(), tableMap, variableExpressionExecutors, metaPosition, ProcessingMode.BATCH, false, siddhiQueryContext); PassThroughOutputRateLimiter rateLimiter = new PassThroughOutputRateLimiter(siddhiQueryContext.getName()); - rateLimiter.init(lockWrapper, siddhiQueryContext); + rateLimiter.init(lockWrapper, groupBy, siddhiQueryContext); OutputCallback outputCallback = OutputParser.constructOutputCallback(returnStream, metaStreamInfoHolder.getMetaStateEvent().getOutputStreamDefinition(), tableMap, windowMap, true, siddhiQueryContext); @@ -482,7 +485,7 @@ private static void populateFindStoreQueryRuntime(FindStoreQueryRuntime findStor QueryParserHelper.updateVariablePosition(metaStreamInfoHolder.getMetaStateEvent(), variableExpressionExecutors); querySelector.setEventPopulator( StateEventPopulatorFactory.constructEventPopulator(metaStreamInfoHolder.getMetaStateEvent())); - findStoreQueryRuntime.setStateEventPool(new StateEventPool(metaStreamInfoHolder.getMetaStateEvent(), 5)); + findStoreQueryRuntime.setStateEventFactory(new StateEventFactory(metaStreamInfoHolder.getMetaStateEvent())); findStoreQueryRuntime.setSelector(querySelector); findStoreQueryRuntime.setOutputAttributes(metaStreamInfoHolder.getMetaStateEvent(). getOutputStreamDefinition().getAttributeList()); @@ -499,7 +502,7 @@ private static QuerySelector getQuerySelector(MatchingMetaInfoHolder matchingMet ProcessingMode.BATCH, false, siddhiQueryContext); PassThroughOutputRateLimiter rateLimiter = new PassThroughOutputRateLimiter(siddhiQueryContext.getName()); - rateLimiter.init(lockWrapper, siddhiQueryContext); + rateLimiter.init(lockWrapper, !storeQuery.getSelector().getGroupByList().isEmpty(), siddhiQueryContext); OutputCallback outputCallback = OutputParser.constructOutputCallback(storeQuery.getOutputStream(), matchingMetaInfoHolder.getMetaStateEvent().getOutputStreamDefinition(), tableMap, windowMap, true, siddhiQueryContext); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/helper/DefinitionParserHelper.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/helper/DefinitionParserHelper.java index a0bbb92284..7c547998d6 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/helper/DefinitionParserHelper.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/helper/DefinitionParserHelper.java @@ -21,7 +21,7 @@ import io.siddhi.core.config.SiddhiAppContext; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.exception.SiddhiAppCreationException; import io.siddhi.core.function.Script; import io.siddhi.core.stream.StreamJunction; @@ -168,9 +168,9 @@ public static void addTable(TableDefinition tableDefinition, ConcurrentMap eventSources = eventSourceMap.get(streamDefinition.getId()); if (eventSources == null) { @@ -553,9 +551,7 @@ public static void addEventSink(StreamDefinition streamDefinition, } if (sinkHandlerManager != null) { - sinkHandlerManager.registerSinkHandler(sinkHandler.getElementId(), sinkHandler); - siddhiAppContext.getSnapshotService().addSnapshotable(streamDefinition.getId(), - sinkHandler); + sinkHandlerManager.registerSinkHandler(sinkHandler.getId(), sinkHandler); } validateSinkMapperCompatibility(streamDefinition, sinkType, mapType, sink, sinkMapper, @@ -568,8 +564,6 @@ public static void addEventSink(StreamDefinition streamDefinition, sink.getMapper().setGroupDeterminer(groupDeterminer); } - siddhiAppContext.getSnapshotService().addSnapshotable(sink.getStreamDefinition().getId(), sink); - List eventSinks = eventSinkMap.get(streamDefinition.getId()); if (eventSinks == null) { eventSinks = new ArrayList<>(); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/helper/QueryParserHelper.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/helper/QueryParserHelper.java index c053a6f4ab..d80b93465d 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/helper/QueryParserHelper.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/parser/helper/QueryParserHelper.java @@ -23,10 +23,10 @@ import io.siddhi.core.event.state.MetaStateEvent; import io.siddhi.core.event.state.MetaStateEventAttribute; import io.siddhi.core.event.state.StateEventCloner; -import io.siddhi.core.event.state.StateEventPool; +import io.siddhi.core.event.state.StateEventFactory; import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.populater.ComplexEventPopulater; import io.siddhi.core.event.stream.populater.StreamEventPopulaterFactory; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -86,7 +86,7 @@ public static void reduceMetaComplexEvent(MetaComplexEvent metaComplexEvent) { * * @param metaStreamEvent MetaStreamEvent */ - private static synchronized void reduceStreamAttributes(MetaStreamEvent metaStreamEvent) { + private static void reduceStreamAttributes(MetaStreamEvent metaStreamEvent) { for (Attribute attribute : metaStreamEvent.getOutputData()) { if (metaStreamEvent.getBeforeWindowData().contains(attribute)) { metaStreamEvent.getBeforeWindowData().remove(attribute); @@ -159,17 +159,17 @@ public static void initStreamRuntime(StreamRuntime runtime, MetaComplexEvent met null, lockWrapper, queryName); } else { MetaStateEvent metaStateEvent = (MetaStateEvent) metaComplexEvent; - StateEventPool stateEventPool = new StateEventPool(metaStateEvent, 5); + StateEventFactory stateEventFactory = new StateEventFactory(metaStateEvent); MetaStreamEvent[] metaStreamEvents = metaStateEvent.getMetaStreamEvents(); for (int i = 0, metaStreamEventsLength = metaStreamEvents.length; i < metaStreamEventsLength; i++) { - initSingleStreamRuntime(runtime.getSingleStreamRuntimes().get(i), i, metaStateEvent, stateEventPool, + initSingleStreamRuntime(runtime.getSingleStreamRuntimes().get(i), i, metaStateEvent, stateEventFactory, lockWrapper, queryName); } } } private static void initSingleStreamRuntime(SingleStreamRuntime singleStreamRuntime, int streamEventChainIndex, - MetaComplexEvent metaComplexEvent, StateEventPool stateEventPool, + MetaComplexEvent metaComplexEvent, StateEventFactory stateEventFactory, LockWrapper lockWrapper, String queryName) { MetaStreamEvent metaStreamEvent; @@ -178,25 +178,25 @@ private static void initSingleStreamRuntime(SingleStreamRuntime singleStreamRunt } else { metaStreamEvent = (MetaStreamEvent) metaComplexEvent; } - StreamEventPool streamEventPool = new StreamEventPool(metaStreamEvent, 5); + StreamEventFactory streamEventFactory = new StreamEventFactory(metaStreamEvent); ProcessStreamReceiver processStreamReceiver = singleStreamRuntime.getProcessStreamReceiver(); processStreamReceiver.setMetaStreamEvent(metaStreamEvent); - processStreamReceiver.setStreamEventPool(streamEventPool); + processStreamReceiver.setStreamEventFactory(streamEventFactory); processStreamReceiver.setLockWrapper(lockWrapper); processStreamReceiver.init(); Processor processor = singleStreamRuntime.getProcessorChain(); while (processor != null) { if (processor instanceof SchedulingProcessor) { - ((SchedulingProcessor) processor).getScheduler().setStreamEventPool(streamEventPool); + ((SchedulingProcessor) processor).getScheduler().setStreamEventFactory(streamEventFactory); ((SchedulingProcessor) processor).getScheduler().init(lockWrapper, queryName); } if (processor instanceof AbstractStreamProcessor) { ((AbstractStreamProcessor) processor) - .setStreamEventCloner(new StreamEventCloner(metaStreamEvent, streamEventPool)); + .setStreamEventCloner(new StreamEventCloner(metaStreamEvent, streamEventFactory)); ((AbstractStreamProcessor) processor).constructStreamEventPopulater(metaStreamEvent, streamEventChainIndex); } - if (stateEventPool != null && processor instanceof JoinProcessor) { + if (stateEventFactory != null && processor instanceof JoinProcessor) { if (((JoinProcessor) processor) .getCompiledCondition() instanceof IncrementalAggregateCompileCondition) { IncrementalAggregateCompileCondition compiledCondition = (IncrementalAggregateCompileCondition) ( @@ -206,16 +206,16 @@ private static void initSingleStreamRuntime(SingleStreamRuntime singleStreamRunt compiledCondition.setComplexEventPopulater(complexEventPopulater); } - ((JoinProcessor) processor).setStateEventPool(stateEventPool); + ((JoinProcessor) processor).setStateEventFactory(stateEventFactory); } - if (stateEventPool != null && processor instanceof StreamPreStateProcessor) { - ((StreamPreStateProcessor) processor).setStateEventPool(stateEventPool); - ((StreamPreStateProcessor) processor).setStreamEventPool(streamEventPool); + if (stateEventFactory != null && processor instanceof StreamPreStateProcessor) { + ((StreamPreStateProcessor) processor).setStateEventFactory(stateEventFactory); + ((StreamPreStateProcessor) processor).setStreamEventFactory(streamEventFactory); ((StreamPreStateProcessor) processor) - .setStreamEventCloner(new StreamEventCloner(metaStreamEvent, streamEventPool)); + .setStreamEventCloner(new StreamEventCloner(metaStreamEvent, streamEventFactory)); if (metaComplexEvent instanceof MetaStateEvent) { ((StreamPreStateProcessor) processor).setStateEventCloner( - new StateEventCloner(((MetaStateEvent) metaComplexEvent), stateEventPool)); + new StateEventCloner(((MetaStateEvent) metaComplexEvent), stateEventFactory)); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/IncrementalFileSystemPersistenceStore.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/IncrementalFileSystemPersistenceStore.java index 7a00c40b21..929d3c7742 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/IncrementalFileSystemPersistenceStore.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/IncrementalFileSystemPersistenceStore.java @@ -101,7 +101,7 @@ public List getListOfRevisionsToLoad(long restoreTime, IncrementalSnapshotInfo snapshotInfo = PersistenceHelper.convertRevision(fileName); if (snapshotInfo.getTime() <= restoreTime && siddhiAppName.equals(snapshotInfo.getSiddhiAppId()) && - snapshotInfo.getElementId() != null && + snapshotInfo.getId() != null && snapshotInfo.getQueryName() != null) { //Note: Here we discard the (items.length == 2) scenario which is handled // by the full snapshot handling @@ -117,6 +117,7 @@ public List getListOfRevisionsToLoad(long restoreTime, @Override public String getLastRevision(String siddhiAppName) { long restoreTime = -1; + IncrementalSnapshotInfo lastSnapshotInfo = null; File dir = new File(folder + File.separator + siddhiAppName); File[] files = dir.listFiles(); if (files == null || files.length == 0) { @@ -127,11 +128,12 @@ public String getLastRevision(String siddhiAppName) { IncrementalSnapshotInfo snapshotInfo = PersistenceHelper.convertRevision(fileName); if (snapshotInfo.getTime() > restoreTime && siddhiAppName.equals(snapshotInfo.getSiddhiAppId()) && - snapshotInfo.getElementId() != null && + snapshotInfo.getId() != null && snapshotInfo.getQueryName() != null) { //Note: Here we discard the (items.length == 2) scenario which is handled // by the full snapshot handling restoreTime = snapshotInfo.getTime(); + lastSnapshotInfo = snapshotInfo; } } if (restoreTime != -1) { @@ -139,7 +141,7 @@ public String getLastRevision(String siddhiAppName) { log.debug("Latest revision to load: " + restoreTime + PersistenceConstants.REVISION_SEPARATOR + siddhiAppName); } - return restoreTime + PersistenceConstants.REVISION_SEPARATOR + siddhiAppName; + return lastSnapshotInfo.getRevision(); } return null; } @@ -175,9 +177,7 @@ private void cleanOldRevisions(IncrementalSnapshotInfo incrementalSnapshotInfo) String fileName = file.getName(); IncrementalSnapshotInfo snapshotInfo = PersistenceHelper.convertRevision(fileName); if (snapshotInfo.getTime() < baseTimeStamp && - incrementalSnapshotInfo.getSiddhiAppId().equals(snapshotInfo.getSiddhiAppId()) && - incrementalSnapshotInfo.getQueryName().equals(snapshotInfo.getQueryName()) && - incrementalSnapshotInfo.getElementId().equals(snapshotInfo.getElementId())) { + incrementalSnapshotInfo.getId().equals(snapshotInfo.getId())) { if (incrementalSnapshotInfo.getType() == IncrementalSnapshotInfo.SnapshotType.BASE && snapshotInfo.getType() != IncrementalSnapshotInfo.SnapshotType.PERIODIC) { if (file.exists()) { diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/util/IncrementalSnapshotInfo.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/util/IncrementalSnapshotInfo.java index 79ea2b2536..f617fafbb7 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/util/IncrementalSnapshotInfo.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/util/IncrementalSnapshotInfo.java @@ -23,21 +23,33 @@ */ public class IncrementalSnapshotInfo { + private String id; private String siddhiAppId; + private String partitionId; + private String partitionGroupByKey; private String queryName; private String elementId; private long time; private String revision; private SnapshotType type; - public IncrementalSnapshotInfo(String siddhiAppId, String queryName, String elementId, long time, - SnapshotType type) { + public IncrementalSnapshotInfo(String siddhiAppId, String partitionId, String queryName, String elementId, + long time, SnapshotType type, String partitionGroupByKey) { this.siddhiAppId = siddhiAppId; + this.partitionId = partitionId; this.queryName = queryName; this.elementId = elementId; this.time = time; this.type = type; + this.partitionGroupByKey = partitionGroupByKey; + this.id = siddhiAppId + + PersistenceConstants.REVISION_SEPARATOR + partitionId + + PersistenceConstants.REVISION_SEPARATOR + partitionGroupByKey + + PersistenceConstants.REVISION_SEPARATOR + queryName + + PersistenceConstants.REVISION_SEPARATOR + elementId; this.revision = time + PersistenceConstants.REVISION_SEPARATOR + siddhiAppId + + PersistenceConstants.REVISION_SEPARATOR + partitionId + + PersistenceConstants.REVISION_SEPARATOR + partitionGroupByKey + PersistenceConstants.REVISION_SEPARATOR + queryName + PersistenceConstants.REVISION_SEPARATOR + elementId + PersistenceConstants.REVISION_SEPARATOR + type; @@ -87,6 +99,18 @@ public void setTime(long time) { this.time = time; } + public String getPartitionId() { + return partitionId; + } + + public String getPartitionGroupByKey() { + return partitionGroupByKey; + } + + public String getId() { + return id; + } + /** * Type of incremental snapshot types */ diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/util/PersistenceHelper.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/util/PersistenceHelper.java index 88ec36c1a7..b3f739364e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/util/PersistenceHelper.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/persistence/util/PersistenceHelper.java @@ -38,12 +38,12 @@ public final class PersistenceHelper { public static IncrementalSnapshotInfo convertRevision(String revision) { String[] items = revision.split(PersistenceConstants.REVISION_SEPARATOR); //Note: Here we discard the (items.length == 2) scenario which is handled by the full snapshot handling - if (items.length == 5) { - return new IncrementalSnapshotInfo(items[1], items[2], items[3], - Long.parseLong(items[0]), IncrementalSnapshotInfo.SnapshotType.valueOf(items[4])); - } else if (items.length == 2) { - return new IncrementalSnapshotInfo(items[1], null, null, - Long.parseLong(items[0]), IncrementalSnapshotInfo.SnapshotType.PERIODIC); + if (items.length == 7) { + return new IncrementalSnapshotInfo(items[1], items[2], items[4], items[5], + Long.parseLong(items[0]), IncrementalSnapshotInfo.SnapshotType.valueOf(items[6]), items[3]); + } else if (items.length == 4) { + return new IncrementalSnapshotInfo(items[1], items[2], null, null, + Long.parseLong(items[0]), IncrementalSnapshotInfo.SnapshotType.PERIODIC, items[3]); } else { throw new PersistenceStoreException("Invalid revision found '" + revision + "'!"); } @@ -65,13 +65,14 @@ public static PersistenceReference persist(IncrementalSnapshot serializeObj, Sid //Periodic state Map> periodicStateBase = serializeObj.getPeriodicState(); if (periodicStateBase != null) { - periodicStateBase.forEach((queryName, value) -> { - value.forEach((elementId, value1) -> { + periodicStateBase.forEach((partitionId, value) -> { + value.forEach((id, value1) -> { + String[] items = id.split(PersistenceConstants.REVISION_SEPARATOR); AsyncIncrementalSnapshotPersistor asyncIncrementSnapshotPersistor = new AsyncIncrementalSnapshotPersistor(value1, siddhiAppContext.getSiddhiContext().getIncrementalPersistenceStore(), - new IncrementalSnapshotInfo(siddhiAppContext.getName(), queryName, elementId, - revisionTime, IncrementalSnapshotInfo.SnapshotType.PERIODIC)); + new IncrementalSnapshotInfo(siddhiAppContext.getName(), partitionId, items[1], items[2], + revisionTime, IncrementalSnapshotInfo.SnapshotType.PERIODIC, items[0])); Future future = siddhiAppContext.getExecutorService(). submit(asyncIncrementSnapshotPersistor); incrementalFutures.add(future); @@ -81,13 +82,14 @@ public static PersistenceReference persist(IncrementalSnapshot serializeObj, Sid //Incremental base state Map> incrementalStateBase = serializeObj.getIncrementalStateBase(); if (incrementalStateBase != null) { - incrementalStateBase.forEach((queryName, value) -> { - value.forEach((elementId, value1) -> { + incrementalStateBase.forEach((partitionId, value) -> { + value.forEach((id, value1) -> { + String[] items = id.split(PersistenceConstants.REVISION_SEPARATOR); AsyncIncrementalSnapshotPersistor asyncIncrementSnapshotPersistor = new AsyncIncrementalSnapshotPersistor(value1, siddhiAppContext.getSiddhiContext().getIncrementalPersistenceStore(), - new IncrementalSnapshotInfo(siddhiAppContext.getName(), queryName, elementId, - revisionTime, IncrementalSnapshotInfo.SnapshotType.BASE)); + new IncrementalSnapshotInfo(siddhiAppContext.getName(), partitionId, items[1], items[2], + revisionTime, IncrementalSnapshotInfo.SnapshotType.BASE, items[0])); Future future = siddhiAppContext.getExecutorService(). submit(asyncIncrementSnapshotPersistor); incrementalFutures.add(future); @@ -98,13 +100,14 @@ public static PersistenceReference persist(IncrementalSnapshot serializeObj, Sid //Incremental state Map> incrementalState = serializeObj.getIncrementalState(); if (incrementalState != null) { - incrementalState.forEach((queryName, value) -> { - value.forEach((elementId, value1) -> { + incrementalState.forEach((partitionId, value) -> { + value.forEach((id, value1) -> { + String[] items = id.split(PersistenceConstants.REVISION_SEPARATOR); AsyncIncrementalSnapshotPersistor asyncIncrementSnapshotPersistor = new AsyncIncrementalSnapshotPersistor(value1, siddhiAppContext.getSiddhiContext().getIncrementalPersistenceStore(), - new IncrementalSnapshotInfo(siddhiAppContext.getName(), queryName, elementId, - revisionTime, IncrementalSnapshotInfo.SnapshotType.INCREMENT)); + new IncrementalSnapshotInfo(siddhiAppContext.getName(), partitionId, items[1], items[2], + revisionTime, IncrementalSnapshotInfo.SnapshotType.INCREMENT, items[0])); Future future = siddhiAppContext.getExecutorService(). submit(asyncIncrementSnapshotPersistor); incrementalFutures.add(future); diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/SnapshotService.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/SnapshotService.java index bbcb5af7af..abf22de8f5 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/SnapshotService.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/SnapshotService.java @@ -22,13 +22,17 @@ import io.siddhi.core.exception.CannotRestoreSiddhiAppStateException; import io.siddhi.core.exception.NoPersistenceStoreException; import io.siddhi.core.exception.PersistenceStoreException; +import io.siddhi.core.exception.SiddhiAppRuntimeException; import io.siddhi.core.util.ThreadBarrier; import io.siddhi.core.util.persistence.IncrementalPersistenceStore; import io.siddhi.core.util.persistence.PersistenceStore; import io.siddhi.core.util.persistence.util.IncrementalSnapshotInfo; +import io.siddhi.core.util.persistence.util.PersistenceConstants; import io.siddhi.core.util.persistence.util.PersistenceHelper; -import io.siddhi.core.util.snapshot.state.SnapshotState; +import io.siddhi.core.util.snapshot.state.Snapshot; import io.siddhi.core.util.snapshot.state.SnapshotStateList; +import io.siddhi.core.util.snapshot.state.State; +import io.siddhi.core.util.snapshot.state.StateHolder; import org.apache.log4j.Logger; import java.util.Comparator; @@ -41,114 +45,140 @@ /** * Service level implementation to take/restore snapshots of processing elements. + * Memory : PartitionId + QueryName + ElementId + PartitionGroupByKey + Item + * Snapshot : PartitionId + PartitionGroupByKey + QueryName + ElementId + Item */ public class SnapshotService { private static final Logger log = Logger.getLogger(SnapshotService.class); - private static final ThreadLocal skipSnapshotableThreadLocal = new ThreadLocal(); - + private static final ThreadLocal skipStateStorageThreadLocal = new ThreadLocal(); private final ThreadBarrier threadBarrier; - private ConcurrentHashMap> snapshotableMap = new ConcurrentHashMap<>(); + private ConcurrentHashMap partitionIdStates; private SiddhiAppContext siddhiAppContext; public SnapshotService(SiddhiAppContext siddhiAppContext) { this.siddhiAppContext = siddhiAppContext; this.threadBarrier = siddhiAppContext.getThreadBarrier(); + this.partitionIdStates = new ConcurrentHashMap<>(); } - public static ThreadLocal getSkipSnapshotableThreadLocal() { - return skipSnapshotableThreadLocal; + public static ThreadLocal getSkipStateStorageThreadLocal() { + return skipStateStorageThreadLocal; } - public ConcurrentHashMap> getSnapshotableMap() { - return snapshotableMap; + public ConcurrentHashMap getStates() { + return partitionIdStates; } - public synchronized void addSnapshotable(String snapshotableName, Snapshotable snapshotable) { - Boolean skipSnapshotable = skipSnapshotableThreadLocal.get(); + public Map getStateHolderMap(String partitionId, String queryName) { + Boolean skipSnapshotable = skipStateStorageThreadLocal.get(); if (skipSnapshotable == null || !skipSnapshotable) { - Map snapshotableMap = this.snapshotableMap.get(snapshotableName); - - // If List does not exist create it. - if (snapshotableMap == null) { - snapshotableMap = new HashMap(); - snapshotableMap.put(snapshotable.getElementId(), snapshotable); - this.snapshotableMap.put(snapshotableName, snapshotableMap); - } else { - // add if item is not already in list - snapshotableMap.putIfAbsent(snapshotable.getElementId(), snapshotable); + PartitionIdStateHolder partitionIdStateHolder = this.partitionIdStates.get(partitionId); + if (partitionIdStateHolder == null) { + partitionIdStateHolder = new PartitionIdStateHolder(partitionId); + this.partitionIdStates.put(partitionId, partitionIdStateHolder); } - } - } - - public synchronized void removeSnapshotable(String snapshotableName, Snapshotable snapshotable) { - Boolean skipSnapshotable = skipSnapshotableThreadLocal.get(); - if (skipSnapshotable == null || !skipSnapshotable) { - Map snapshotableMap = this.snapshotableMap.get(snapshotableName); - if (snapshotableMap != null) { - snapshotableMap = new HashMap<>(); - snapshotableMap.remove(snapshotable.getElementId(), snapshotable); - if (snapshotableMap.isEmpty()) { - this.snapshotableMap.remove(snapshotableName); - } + ElementStateHolder elementStateHolder = partitionIdStateHolder.getElementState(queryName); + if (elementStateHolder == null) { + elementStateHolder = new ElementStateHolder(queryName, new HashMap<>()); + partitionIdStateHolder.addElementState(queryName, elementStateHolder); } + return elementStateHolder.elementHolderMap; } + return null; } public byte[] fullSnapshot() { try { SnapshotRequest.requestForFullSnapshot(true); - Map> elementSnapshotMapFull = new HashMap<>(); + Map>>>> fullSnapshot = new HashMap<>(); byte[] serializedFullState = null; if (log.isDebugEnabled()) { log.debug("Taking snapshot ..."); } try { threadBarrier.lock(); - for (Map.Entry> entry : snapshotableMap.entrySet()) { - Map elementWiseFullSnapshots = new HashMap<>(); - for (Map.Entry snapshotableEntry : entry.getValue().entrySet()) { - Snapshotable snapshotableObj = ((Snapshotable) snapshotableEntry.getValue()); - Map currentState = ((Snapshotable) snapshotableEntry.getValue()).currentState(); - if (currentState != null) { - Map elementWiseSnapshots = new HashMap<>(); - for (Map.Entry item2 : currentState.entrySet()) { - String key = item2.getKey(); - Object snapShot = item2.getValue(); - if (snapShot instanceof SnapshotState) { - if (((SnapshotState) snapShot).isIncrementalSnapshot()) { - throw new NoPersistenceStoreException("No incremental persistence store " + - "exist to store incremental snapshot of siddhiApp:'" - + siddhiAppContext.getName() + "' subElement:'" + entry.getKey() - + "' elementId:'" + snapshotableObj.getElementId() - + "' and key:'" + key + "'"); - } else { - elementWiseSnapshots.put(key, snapShot); + waitForSystemStabilization(); + for (Map.Entry partitionIdState : partitionIdStates.entrySet()) { + for (Map.Entry queryState : + partitionIdState.getValue().queryStateHolderMap.entrySet()) { + for (Map.Entry elementState : + queryState.getValue().elementHolderMap.entrySet()) { + Map> partitionKeyStates = elementState.getValue().getAllStates(); + try { + for (Map.Entry> partitionKeyState : + partitionKeyStates.entrySet()) { + for (Map.Entry groupByKeyState : + partitionKeyState.getValue().entrySet()) { + String partitionAndGroupByKey = partitionKeyState.getKey() + "--" + + groupByKeyState.getKey(); + State state = groupByKeyState.getValue(); + Map itemStates = state.snapshot(); + if (itemStates != null) { + Map itemSnapshots = new HashMap<>(); + for (Map.Entry itemState : itemStates.entrySet()) { + if (itemState.getValue() instanceof Snapshot) { + if (((Snapshot) itemState.getValue()).isIncrementalSnapshot()) { + throw new NoPersistenceStoreException("No incremental " + + "persistence store exist to store incremental " + + "snapshot of siddhiApp:'" + + siddhiAppContext.getName() + + "' subElement:'" + queryState.getKey() + + "' elementId:'" + elementState.getKey() + + "' partitionKey:'" + partitionKeyState.getKey() + + "' groupByKey:'" + groupByKeyState.getKey() + + "' and itemKey:'" + itemState.getKey() + "'"); + } else { + itemSnapshots.put(itemState.getKey(), itemState.getValue()); + } + } else { + itemSnapshots.put(itemState.getKey(), itemState.getValue()); + } + } + Map>>> + partitionIdSnapshot = fullSnapshot.computeIfAbsent( + partitionIdState.getKey(), + k -> new HashMap<>()); + Map>> partitionGroupByKeySnapshot = + partitionIdSnapshot.computeIfAbsent( + partitionAndGroupByKey, + k -> new HashMap<>()); + Map> querySnapshot = + partitionGroupByKeySnapshot.computeIfAbsent( + queryState.getKey(), + k -> new HashMap<>()); + Map elementSnapshot = + querySnapshot.get(elementState.getKey()); + if (elementSnapshot == null) { + querySnapshot.put(elementState.getKey(), itemSnapshots); + } else { + throw new SiddhiAppRuntimeException("Duplicate state exist for " + + "siddhiApp:'" + siddhiAppContext.getName() + + "' partitionKey:'" + partitionKeyState.getKey() + + "' groupByKey:'" + groupByKeyState.getKey() + + "' subElement:'" + queryState.getKey() + + "' elementId:'" + elementState.getKey() + "'"); + } + } } - } else { - elementWiseSnapshots.put(key, snapShot); } - } - if (!elementWiseSnapshots.isEmpty()) { - elementWiseFullSnapshots.put(snapshotableObj.getElementId(), elementWiseSnapshots); + } finally { + elementState.getValue().returnAllStates(partitionKeyStates); } } } - if (!elementWiseFullSnapshots.isEmpty()) { - elementSnapshotMapFull.put(entry.getKey(), elementWiseFullSnapshots); - } } if (log.isDebugEnabled()) { - log.debug("SnapshotState serialization started ..."); + log.debug("Snapshot serialization started ..."); } - serializedFullState = ByteSerializer.objectToByte(elementSnapshotMapFull, siddhiAppContext); + serializedFullState = ByteSerializer.objectToByte(fullSnapshot, siddhiAppContext); if (log.isDebugEnabled()) { - log.debug("SnapshotState serialization finished."); + log.debug("Snapshot serialization finished."); } } finally { threadBarrier.unlock(); } if (log.isDebugEnabled()) { - log.debug("SnapshotState taken for Siddhi app '" + siddhiAppContext.getName() + "'"); + log.debug("Snapshot taken for Siddhi app '" + siddhiAppContext.getName() + "'"); } return serializedFullState; } finally { @@ -159,84 +189,85 @@ public byte[] fullSnapshot() { public IncrementalSnapshot incrementalSnapshot() { try { SnapshotRequest.requestForFullSnapshot(false); - Map> elementSnapshotMapIncremental = new HashMap<>(); - Map> elementSnapshotMapIncrementalBase = new HashMap<>(); - Map> elementSnapshotMapPeriodic = new HashMap<>(); + Map> incrementalSnapshotMap = new HashMap<>(); + Map> incrementalBaseSnapshotMap = new HashMap<>(); + Map> periodicSnapshotMap = new HashMap<>(); if (log.isDebugEnabled()) { log.debug("Taking snapshot ..."); } try { threadBarrier.lock(); - for (Map.Entry> entry : snapshotableMap.entrySet()) { - Map elementWiseIncrementalSnapshots = new HashMap<>(); - Map elementWiseIncrementalSnapshotsBase = new HashMap<>(); - Map elementWisePeriodicSnapshots = new HashMap<>(); - for (Map.Entry snapshotableEntry : entry.getValue().entrySet()) { - Snapshotable snapshotableObj = ((Snapshotable) snapshotableEntry.getValue()); - Map currentState = snapshotableObj.currentState(); - if (currentState != null) { - Map incrementalSnapshotableMap = new HashMap<>(); - Map incrementalSnapshotableMapBase = new HashMap<>(); - Map periodicSnapshotableMap = new HashMap<>(); - for (Map.Entry stateEntry : currentState.entrySet()) { - String key = stateEntry.getKey(); - Object snapShot = stateEntry.getValue(); - if (snapShot instanceof SnapshotState) { - if (((SnapshotState) snapShot).isIncrementalSnapshot()) { - incrementalSnapshotableMap.put(key, snapShot); - } else { - incrementalSnapshotableMapBase.put(key, snapShot); + waitForSystemStabilization(); + for (Map.Entry partitionIdState : partitionIdStates.entrySet()) { + for (Map.Entry queryState : + partitionIdState.getValue().queryStateHolderMap.entrySet()) { + for (Map.Entry elementState : + queryState.getValue().elementHolderMap.entrySet()) { + Map> partitionKeyStates = elementState.getValue().getAllStates(); + try { + for (Map.Entry> partitionKeyState : + partitionKeyStates.entrySet()) { + for (Map.Entry groupByKeyState : + partitionKeyState.getValue().entrySet()) { + State state = groupByKeyState.getValue(); + Map itemStates = state.snapshot(); + if (itemStates != null) { + Map itemSnapshotsIncremental = new HashMap<>(); + Map itemSnapshotsIncrementalBase = new HashMap<>(); + Map itemSnapshotsPeriodic = new HashMap<>(); + for (Map.Entry itemState : itemStates.entrySet()) { + if (itemState.getValue() instanceof Snapshot) { + if (((Snapshot) itemState.getValue()).isIncrementalSnapshot()) { + itemSnapshotsIncremental.put(itemState.getKey(), + itemState.getValue()); + } else { + itemSnapshotsIncrementalBase.put( + itemState.getKey(), itemState.getValue()); + } + } else { + itemSnapshotsPeriodic.put(itemState.getKey(), itemState.getValue()); + } + } + if (!itemSnapshotsIncremental.isEmpty()) { + addToSnapshotIncrements(incrementalSnapshotMap, partitionIdState, + queryState, elementState, partitionKeyState, groupByKeyState, + itemSnapshotsIncremental); + } + if (!itemSnapshotsIncrementalBase.isEmpty()) { + addToSnapshotIncrements(incrementalBaseSnapshotMap, partitionIdState, + queryState, elementState, partitionKeyState, groupByKeyState, + itemSnapshotsIncrementalBase); + } + if (!itemSnapshotsPeriodic.isEmpty()) { + addToSnapshotIncrements(periodicSnapshotMap, partitionIdState, + queryState, elementState, partitionKeyState, groupByKeyState, + itemSnapshotsPeriodic); + } + } } - } else { - periodicSnapshotableMap.put(key, snapShot); } - } - if (log.isDebugEnabled()) { - log.debug("SnapshotState serialization started ..."); - } - if (!incrementalSnapshotableMap.isEmpty()) { - //Do we need to get and then update? - elementWiseIncrementalSnapshots.put(snapshotableObj.getElementId(), - ByteSerializer.objectToByte(incrementalSnapshotableMap, siddhiAppContext)); - } - if (!incrementalSnapshotableMapBase.isEmpty()) { - elementWiseIncrementalSnapshotsBase.put(snapshotableObj.getElementId(), - ByteSerializer.objectToByte(incrementalSnapshotableMapBase, siddhiAppContext)); - } - if (!periodicSnapshotableMap.isEmpty()) { - elementWisePeriodicSnapshots.put(snapshotableObj.getElementId(), - ByteSerializer.objectToByte(periodicSnapshotableMap, siddhiAppContext)); - } - if (log.isDebugEnabled()) { - log.debug("SnapshotState serialization finished."); + } finally { + elementState.getValue().returnAllStates(partitionKeyStates); } } } - if (!elementWiseIncrementalSnapshots.isEmpty()) { - elementSnapshotMapIncremental.put(entry.getKey(), elementWiseIncrementalSnapshots); - } - if (!elementWiseIncrementalSnapshotsBase.isEmpty()) { - elementSnapshotMapIncrementalBase.put(entry.getKey(), elementWiseIncrementalSnapshotsBase); - } - if (!elementWisePeriodicSnapshots.isEmpty()) { - elementSnapshotMapPeriodic.put(entry.getKey(), elementWisePeriodicSnapshots); - } } + } finally { threadBarrier.unlock(); } if (log.isDebugEnabled()) { - log.debug("SnapshotState taken for Siddhi app '" + siddhiAppContext.getName() + "'"); + log.debug("Snapshot taken for Siddhi app '" + siddhiAppContext.getName() + "'"); } IncrementalSnapshot snapshot = new IncrementalSnapshot(); - if (!elementSnapshotMapIncremental.isEmpty()) { - snapshot.setIncrementalState(elementSnapshotMapIncremental); + if (!incrementalSnapshotMap.isEmpty()) { + snapshot.setIncrementalState(incrementalSnapshotMap); } - if (!elementSnapshotMapIncrementalBase.isEmpty()) { - snapshot.setIncrementalStateBase(elementSnapshotMapIncrementalBase); + if (!incrementalBaseSnapshotMap.isEmpty()) { + snapshot.setIncrementalStateBase(incrementalBaseSnapshotMap); } - if (!elementSnapshotMapPeriodic.isEmpty()) { - snapshot.setPeriodicState(elementSnapshotMapPeriodic); + if (!periodicSnapshotMap.isEmpty()) { + snapshot.setPeriodicState(periodicSnapshotMap); } return snapshot; } finally { @@ -244,19 +275,50 @@ public IncrementalSnapshot incrementalSnapshot() { } } + private void addToSnapshotIncrements(Map> incrementalSnapshotMap, + Map.Entry partitionIdState, + Map.Entry queryState, + Map.Entry elementState, + Map.Entry> partitionKeyState, + Map.Entry groupByKeyState, + Map itemSnapshotsIncremental) { + String id = partitionKeyState.getKey() + "--" + groupByKeyState.getKey() + + PersistenceConstants.REVISION_SEPARATOR + queryState.getKey() + + PersistenceConstants.REVISION_SEPARATOR + elementState.getKey(); + Map partitionIdSnapshot = + incrementalSnapshotMap.computeIfAbsent( + partitionIdState.getKey(), + k -> new HashMap<>()); + partitionIdSnapshot.put(id, ByteSerializer.objectToByte( + itemSnapshotsIncremental, siddhiAppContext)); + } + public Map queryState(String queryName) { - Map state = new HashMap<>(); + Map queryState = new HashMap<>(); try { // Lock the threads in Siddhi threadBarrier.lock(); - Map map = snapshotableMap.get(queryName); - - if (map != null) { - for (Map.Entry entry : map.entrySet()) { - Snapshotable element = (Snapshotable) entry.getValue(); - Map elementState = element.currentState(); - String elementId = element.getElementId(); - state.put(elementId, elementState); + waitForSystemStabilization(); + PartitionIdStateHolder partitionIdStateHolder = partitionIdStates.get(""); + if (partitionIdStateHolder != null) { + ElementStateHolder elementStateHolder = partitionIdStateHolder.queryStateHolderMap.get(queryName); + if (elementStateHolder != null) { + for (Map.Entry elementState : elementStateHolder.elementHolderMap.entrySet()) { + Map> partitionKeyStates = elementState.getValue().getAllStates(); + try { + for (Map.Entry> partitionKeyState : + partitionKeyStates.entrySet()) { + for (Map.Entry groupByKeyState : + partitionKeyState.getValue().entrySet()) { + String id = partitionKeyState.getKey() + "--" + groupByKeyState.getKey() + "_" + + queryName + "_" + elementState.getKey(); + queryState.put(id, groupByKeyState.getValue().snapshot()); + } + } + } finally { + elementState.getValue().returnAllStates(partitionKeyStates); + } + } } } } finally { @@ -265,7 +327,7 @@ public Map queryState(String queryName) { if (log.isDebugEnabled()) { log.debug("Taking snapshot finished."); } - return state; + return queryState; } public void restore(byte[] snapshot) throws CannotRestoreSiddhiAppStateException { @@ -273,167 +335,272 @@ public void restore(byte[] snapshot) throws CannotRestoreSiddhiAppStateException throw new CannotRestoreSiddhiAppStateException("Restoring of Siddhi app " + siddhiAppContext. getName() + " failed due to no snapshot."); } - Map>> snapshotsByQueryName = - (Map>>) ByteSerializer.byteToObject(snapshot, siddhiAppContext); - if (snapshotsByQueryName == null) { + Map>>>> fullSnapshot = + (Map>>>>) + ByteSerializer.byteToObject(snapshot, siddhiAppContext); + if (fullSnapshot == null) { throw new CannotRestoreSiddhiAppStateException("Restoring of Siddhi app " + siddhiAppContext. getName() + " failed due to invalid snapshot."); } try { threadBarrier.lock(); - if (snapshotableMap.containsKey("partition")) { - Map partitionSnapshotables = snapshotableMap.get("partition"); - - try { - if (partitionSnapshotables != null) { - for (Map.Entry entry : partitionSnapshotables.entrySet()) { - Snapshotable snapshotable = (Snapshotable) entry.getValue(); - Map> snapshotsByElementId = - snapshotsByQueryName.get("partition"); - snapshotable.restoreState(snapshotsByElementId.get(snapshotable.getElementId())); - } + waitForSystemStabilization(); + try { + //cleaning old group by states + cleanGroupByStates(); + //restore data + for (Map.Entry>>>> partitionIdSnapshot : + fullSnapshot.entrySet()) { + PartitionIdStateHolder partitionStateHolder = partitionIdStates.get(partitionIdSnapshot.getKey()); + if (partitionStateHolder == null) { + continue; } - } catch (Throwable t) { - throw new CannotRestoreSiddhiAppStateException("Restoring of Siddhi app " + siddhiAppContext. - getName() + " not completed properly. This can occur if the content of Siddhi app has " + - "changed since it was last state persisted, or if the Siddhi app was not given a name. " + - "Make sure to provide a name to the Siddhi app by adding '@app:name('')' " + - "annotation and clean the persistence store if you have done modifications " + - "to the Siddhi app such that it can perform a fresh deployment.", t); - } - } - - for (Map.Entry> entry : snapshotableMap.entrySet()) { - if (!entry.getKey().equals("partition")) { - Map map = entry.getValue(); - try { - for (Map.Entry snapshotableEntry : map.entrySet()) { - Snapshotable snapshotable = (Snapshotable) snapshotableEntry.getValue(); - Map> snapshotsByElementId = - snapshotsByQueryName.get(entry.getKey()); - if (snapshotsByElementId != null) { - Map snapshotsByKey = snapshotsByElementId.get( - snapshotable.getElementId()); - if (snapshotsByKey != null) { - Map snapshotRestoresByKey = new HashMap<>(); - for (Map.Entry snapshotsByKeyEntry : snapshotsByKey.entrySet()) { - if (snapshotsByKeyEntry.getValue() instanceof SnapshotState) { - SnapshotStateList snapshotStateList = new SnapshotStateList(); - snapshotStateList.putSnapshotState(0L, - (SnapshotState) snapshotsByKeyEntry.getValue()); - snapshotRestoresByKey.put(snapshotsByKeyEntry.getKey(), snapshotStateList); - } else { - snapshotRestoresByKey.put(snapshotsByKeyEntry.getKey(), - snapshotsByKeyEntry.getValue()); + for (Map.Entry>>> partitionGroupByKeySnapshot : + partitionIdSnapshot.getValue().entrySet()) { + for (Map.Entry>> querySnapshot : + partitionGroupByKeySnapshot.getValue().entrySet()) { + ElementStateHolder elementStateHolder = + partitionStateHolder.queryStateHolderMap.get(querySnapshot.getKey()); + if (elementStateHolder == null) { + continue; + } + for (Map.Entry> elementSnapshot : + querySnapshot.getValue().entrySet()) { + StateHolder stateHolder = + elementStateHolder.elementHolderMap.get(elementSnapshot.getKey()); + if (stateHolder == null) { + continue; + } + try { + String partitionKey = null; + String groupByKey = null; + if (partitionGroupByKeySnapshot.getKey() != null) { + String[] keys = partitionGroupByKeySnapshot.getKey().split("--"); + if (keys.length == 2) { + if (!keys[0].equals("null")) { + partitionKey = keys[0]; + } + if (!keys[1].equals("null")) { + groupByKey = keys[1]; + } + } + } + SiddhiAppContext.startPartitionFlow(partitionKey); + SiddhiAppContext.startGroupByFlow(groupByKey); + State state = stateHolder.getState(); + try { + if (state == null) { + continue; + } + Map snapshotRestores = new HashMap<>(); + for (Map.Entry itemSnapshot : + elementSnapshot.getValue().entrySet()) { + if (itemSnapshot.getValue() instanceof Snapshot) { + SnapshotStateList snapshotStateList = new SnapshotStateList(); + snapshotStateList.putSnapshotState(0L, + (Snapshot) itemSnapshot.getValue()); + snapshotRestores.put(itemSnapshot.getKey(), snapshotStateList); + } else { + snapshotRestores.put(itemSnapshot.getKey(), + itemSnapshot.getValue()); + } } + state.restore(snapshotRestores); + } finally { + stateHolder.returnState(state); } - snapshotable.restoreState(snapshotRestoresByKey); + } finally { + SiddhiAppContext.stopPartitionFlow(); + SiddhiAppContext.stopGroupByFlow(); } } } - } catch (Throwable t) { - throw new CannotRestoreSiddhiAppStateException("Restoring of Siddhi app " + - siddhiAppContext.getName() + " not completed properly because content of Siddhi " + - "app has changed since last state persistence. Clean persistence store for a " + - "fresh deployment.", t); } + } + } catch (Throwable t) { + throw new CannotRestoreSiddhiAppStateException("Restoring of Siddhi app " + + siddhiAppContext.getName() + " not completed properly because content of Siddhi " + + "app has changed since last state persistence. Clean persistence store for a " + + "fresh deployment.", t); } } finally { threadBarrier.unlock(); } } - public void restore(Map>>> snapshot) + public void restore(Map>>>> + snapshot) throws CannotRestoreSiddhiAppStateException { try { threadBarrier.lock(); - if (snapshotableMap.containsKey("partition")) { - Map partitionSnapshotables = snapshotableMap.get("partition"); - - try { - if (partitionSnapshotables != null) { - for (Map.Entry snapshotableEntry : partitionSnapshotables.entrySet()) { - Snapshotable snapshotable = (Snapshotable) snapshotableEntry.getValue(); - Map>> incrementalStateByElementId - = snapshot.get("partition"); - restoreIncrementalSnapshot(snapshotable, incrementalStateByElementId); - } + waitForSystemStabilization(); + try { + //cleaning old group by states + cleanGroupByStates(); + //restore data + for (Map.Entry>>>> + partitionIdSnapshot : snapshot.entrySet()) { + PartitionIdStateHolder partitionStateHolder = partitionIdStates.get(partitionIdSnapshot.getKey()); + if (partitionStateHolder == null) { + continue; } - } catch (Throwable t) { - throw new CannotRestoreSiddhiAppStateException("Restoring of Siddhi app " + siddhiAppContext. - getName() + " not completed properly because content of Siddhi app has changed since " - + "last state persistence. Clean persistence store for a fresh deployment.", t); - } - } - - for (Map.Entry> entry : snapshotableMap.entrySet()) { - if (!entry.getKey().equals("partition")) { - Map map = entry.getValue(); - try { - for (Map.Entry snapshotableEntry : map.entrySet()) { - Snapshotable snapshotable = (Snapshotable) snapshotableEntry.getValue(); - Map>> incrementalStateByElementId - = snapshot.get(entry.getKey()); - restoreIncrementalSnapshot(snapshotable, incrementalStateByElementId); - } - } catch (Throwable t) { - throw new CannotRestoreSiddhiAppStateException("Restoring of Siddhi app " + - siddhiAppContext.getName() + " not completed properly because content of Siddhi " + - "app has changed since last state persistence. Clean persistence store for a " + - "fresh deployment.", t); + for (Iterator>>>> + iterator = partitionIdSnapshot.getValue().entrySet().iterator(); iterator.hasNext(); ) { + Map.Entry>>> + partitionGroupByKeySnapshot = iterator.next(); + restoreIncrementalSnapshot(partitionStateHolder, partitionGroupByKeySnapshot.getValue()); + iterator.remove(); } + } + } catch (Throwable t) { + throw new CannotRestoreSiddhiAppStateException("Restoring of Siddhi app " + + siddhiAppContext.getName() + " not completed properly because content of Siddhi " + + "app has changed since last state persistence. Clean persistence store for a " + + "fresh deployment.", t); } } finally { threadBarrier.unlock(); } } - private void restoreIncrementalSnapshot(Snapshotable snapshotable, - Map>> - incrementalStateByElementId) { - if (incrementalStateByElementId != null) { - Map> incrementalStateByTime - = incrementalStateByElementId.get(snapshotable.getElementId()); - if (incrementalStateByTime != null) { - Map deserializedElementStateMap = new HashMap<>(); - for (Map.Entry> incrementalStateByTimeEntry : - incrementalStateByTime.entrySet()) { - for (Map.Entry incrementalStateByInfoEntry : - incrementalStateByTimeEntry.getValue().entrySet()) { - Map singleIncrementSnapshot = (Map) - ByteSerializer.byteToObject(incrementalStateByInfoEntry.getValue(), siddhiAppContext); - if (singleIncrementSnapshot != null) { - for (Map.Entry singleIncrementSnapshotEntry : - singleIncrementSnapshot.entrySet()) { - if (singleIncrementSnapshotEntry.getValue() instanceof SnapshotState) { - SnapshotState snapshotState = (SnapshotState) - singleIncrementSnapshotEntry.getValue(); - SnapshotStateList snapshotStateList = (SnapshotStateList) - deserializedElementStateMap.computeIfAbsent( - singleIncrementSnapshotEntry.getKey(), - k -> new SnapshotStateList()); - if (snapshotState.isIncrementalSnapshot()) { - snapshotStateList.putSnapshotState( - incrementalStateByTimeEntry.getKey(), - snapshotState); - } else { - snapshotStateList.getSnapshotStates().clear(); - snapshotStateList.putSnapshotState( - incrementalStateByTimeEntry.getKey(), - snapshotState); + private void restoreIncrementalSnapshot(PartitionIdStateHolder partitionIdStateHolder, + Map>> incrementalStateByTime) { + if (incrementalStateByTime != null) { + String id = null; + State state = null; + StateHolder stateHolder = null; + Map deserializedStateMap = null; + try { + for (Iterator>>> iterator = + incrementalStateByTime.entrySet().iterator(); iterator.hasNext(); ) { + Map.Entry>> incrementalStateByTimeEntry = + iterator.next(); + iterator.remove(); + for (Iterator>> partitionGroupByKeyIterator = + incrementalStateByTimeEntry.getValue().entrySet().iterator(); + partitionGroupByKeyIterator.hasNext(); ) { + Map.Entry> + partitionGroupByKeyStateByTimeEntry = partitionGroupByKeyIterator.next(); + partitionGroupByKeyIterator.remove(); + for (Iterator> iterator1 = + partitionGroupByKeyStateByTimeEntry.getValue().entrySet().iterator(); + iterator1.hasNext(); ) { + Map.Entry incrementalStateByInfoEntry = iterator1.next(); + iterator1.remove(); + IncrementalSnapshotInfo incrementalSnapshotInfo = incrementalStateByInfoEntry.getKey(); + Map singleIncrementSnapshot = (Map) + ByteSerializer.byteToObject( + incrementalStateByInfoEntry.getValue(), + siddhiAppContext); + if (singleIncrementSnapshot != null) { + if (!incrementalSnapshotInfo.getId().equals(id)) { + if (id != null) { + state.restore(deserializedStateMap); + SiddhiAppContext.startPartitionFlow(id); + try { + stateHolder.returnState(state); + } finally { + SiddhiAppContext.stopPartitionFlow(); + } + id = null; + state = null; + stateHolder = null; + deserializedStateMap = null; + } + ElementStateHolder elementStateHolder = partitionIdStateHolder. + queryStateHolderMap.get(incrementalSnapshotInfo.getQueryName()); + if (elementStateHolder == null) { + continue; + } + stateHolder = elementStateHolder.elementHolderMap.get( + incrementalSnapshotInfo.getElementId()); + if (stateHolder == null) { + continue; + } + String partitionKey = null; + String groupByKey = null; + String[] keys = incrementalSnapshotInfo.getPartitionGroupByKey().split("--"); + if (keys.length == 2) { + if (!keys[0].equals("null")) { + partitionKey = keys[0]; + } + if (!keys[1].equals("null")) { + groupByKey = keys[1]; + } + } + SiddhiAppContext.startPartitionFlow(partitionKey); + SiddhiAppContext.startGroupByFlow(groupByKey); + try { + state = stateHolder.getState(); + } finally { + SiddhiAppContext.stopGroupByFlow(); + SiddhiAppContext.stopPartitionFlow(); + } + if (state != null) { + id = incrementalSnapshotInfo.getId(); + deserializedStateMap = new HashMap<>(); + } + } + if (state != null) { + for (Map.Entry singleIncrementSnapshotEntry : + singleIncrementSnapshot.entrySet()) { + if (singleIncrementSnapshotEntry.getValue() instanceof Snapshot) { + Snapshot snapshot = (Snapshot) + singleIncrementSnapshotEntry.getValue(); + SnapshotStateList snapshotStateList = (SnapshotStateList) + deserializedStateMap.computeIfAbsent( + singleIncrementSnapshotEntry.getKey(), + k -> new SnapshotStateList()); + if (snapshot.isIncrementalSnapshot()) { + snapshotStateList.putSnapshotState( + partitionGroupByKeyStateByTimeEntry.getKey(), + snapshot); + } else { + snapshotStateList.getSnapshotStates().clear(); + snapshotStateList.putSnapshotState( + partitionGroupByKeyStateByTimeEntry.getKey(), + snapshot); + } + } else { + deserializedStateMap.put(singleIncrementSnapshotEntry.getKey(), + singleIncrementSnapshotEntry.getValue()); + } } - } else { - deserializedElementStateMap.put(singleIncrementSnapshotEntry.getKey(), - singleIncrementSnapshotEntry.getValue()); } } } } + if (id != null) { + state.restore(deserializedStateMap); + SiddhiAppContext.startPartitionFlow(id); + try { + stateHolder.returnState(state); + } finally { + SiddhiAppContext.stopPartitionFlow(); + } + id = null; + state = null; + stateHolder = null; + deserializedStateMap = null; + } + } + } finally { + if (id != null && stateHolder != null && state != null) { + SiddhiAppContext.startPartitionFlow(id); + try { + stateHolder.returnState(state); + } finally { + SiddhiAppContext.stopPartitionFlow(); + } + id = null; + state = null; + stateHolder = null; } - snapshotable.restoreState(deserializedElementStateMap); } + } } @@ -470,7 +637,7 @@ public void restoreRevision(String revision) throws CannotRestoreSiddhiAppStateE incrementalSnapshotInfos.sort(new Comparator() { @Override public int compare(IncrementalSnapshotInfo o1, IncrementalSnapshotInfo o2) { - int results = o1.getElementId().compareTo(o2.getElementId()); + int results = o1.getId().compareTo(o2.getId()); if (results == 0) { results = Long.compare(o2.getTime(), o1.getTime()); if (results == 0) { @@ -480,13 +647,13 @@ public int compare(IncrementalSnapshotInfo o1, IncrementalSnapshotInfo o2) { return results; } }); - String lastElementId = null; + String lastId = null; boolean baseFound = false; boolean perioicFound = false; for (Iterator iterator = incrementalSnapshotInfos.iterator(); iterator.hasNext(); ) { IncrementalSnapshotInfo snapshotInfo = iterator.next(); - if (snapshotInfo.getElementId().equals(lastElementId)) { + if (snapshotInfo.getId().equals(lastId)) { if (baseFound && (snapshotInfo.getType() == IncrementalSnapshotInfo.SnapshotType.BASE || snapshotInfo.getType() == IncrementalSnapshotInfo.SnapshotType.INCREMENT)) { iterator.remove(); @@ -502,17 +669,21 @@ public int compare(IncrementalSnapshotInfo o1, IncrementalSnapshotInfo o2) { baseFound = snapshotInfo.getType() == IncrementalSnapshotInfo.SnapshotType.BASE; perioicFound = snapshotInfo.getType() == IncrementalSnapshotInfo.SnapshotType.PERIODIC; } - lastElementId = snapshotInfo.getElementId(); + lastId = snapshotInfo.getId(); } - Map>>> incrementalState = - new HashMap<>(); + Map>>>> + incrementalState = new HashMap<>(); for (IncrementalSnapshotInfo snapshotInfo : incrementalSnapshotInfos) { - Map>> incrementalStateByElementId = - incrementalState.computeIfAbsent(snapshotInfo.getQueryName(), k -> new TreeMap<>()); - Map> incrementalStateByTime = - incrementalStateByElementId.computeIfAbsent(snapshotInfo.getElementId(), + Map>>> + incrementalStateByPartitionGroupByKey = incrementalState.computeIfAbsent( + snapshotInfo.getPartitionId(), k -> new TreeMap<>()); + Map>> incrementalStateByTime = + incrementalStateByPartitionGroupByKey.computeIfAbsent(snapshotInfo.getPartitionGroupByKey(), + k -> new TreeMap<>()); + Map> idByTime = + incrementalStateByTime.computeIfAbsent(snapshotInfo.getId(), k -> new TreeMap<>()); - Map incrementalStateByInfo = incrementalStateByTime. + Map incrementalStateByInfo = idByTime. computeIfAbsent(snapshotInfo.getTime(), k -> new HashMap<>()); incrementalStateByInfo.put(snapshotInfo, incrementalPersistenceStore.load(snapshotInfo)); } @@ -568,5 +739,62 @@ public void clearAllRevisions() throws CannotClearSiddhiAppStateException { } } + private void cleanGroupByStates() { + for (Map.Entry partitionIdState : partitionIdStates.entrySet()) { + for (Map.Entry queryState : + partitionIdState.getValue().queryStateHolderMap.entrySet()) { + for (Map.Entry elementState : + queryState.getValue().elementHolderMap.entrySet()) { + elementState.getValue().cleanGroupByStates(); + } + } + } + } + + private void waitForSystemStabilization() { + int retryCount = 100; + int activeThreads = siddhiAppContext.getThreadBarrier().getActiveThreads(); + while (activeThreads != 0 && retryCount > 0) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + throw new SiddhiAppRuntimeException("Stabilization of Siddhi App " + siddhiAppContext.getName() + + " for snapshot/restore interrupted. " + e.getMessage(), e); + } + activeThreads = siddhiAppContext.getThreadBarrier().getActiveThreads(); + retryCount--; + } + if (retryCount == 0) { + throw new SiddhiAppRuntimeException("Siddhi App " + siddhiAppContext.getName() + + " not stabilized for snapshot/restore, Active thread count is " + + activeThreads); + } + } + class PartitionIdStateHolder { + private final String partitionId; + private final Map queryStateHolderMap = new HashMap<>(); + + public PartitionIdStateHolder(String partitionId) { + this.partitionId = partitionId; + } + + public void addElementState(String queryName, ElementStateHolder elementStateHolder) { + queryStateHolderMap.put(queryName, elementStateHolder); + } + + public ElementStateHolder getElementState(String queryName) { + return queryStateHolderMap.get(queryName); + } + } + + class ElementStateHolder { + private final String elementId; + private final Map elementHolderMap; + + public ElementStateHolder(String elementId, Map elementHolderMap) { + this.elementId = elementId; + this.elementHolderMap = elementHolderMap; + } + } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/Snapshotable.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/Snapshotable.java deleted file mode 100644 index eeb14950ec..0000000000 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/Snapshotable.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. - * - * WSO2 Inc. licenses this file to you under the Apache License, - * Version 2.0 (the "License"); you may not use this file except - * in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package io.siddhi.core.util.snapshot; - -import java.util.Map; - -/** - * Parent interface which should be implemented by every processing elements of Siddhi event processing chain so - * that they can save the current state and poll for previous state in case of an issue. - */ -public interface Snapshotable { - - /** - * Used to collect the serializable state of the processing element, that need to be - * persisted for the reconstructing the element to the same state on a different point of time - * - * @return stateful objects of the processing element as an array - */ - Map currentState(); - - /** - * Used to restore serialized state of the processing element, for reconstructing - * the element to the same state as if was on a previous point of time. - * - * @param state the stateful objects of the element as an array on - * the same order provided by currentState(). - */ - void restoreState(Map state); - - String getElementId(); - - /** - * Used to clean its references - */ - void clean(); -} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/EmptyStateHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/EmptyStateHolder.java new file mode 100644 index 0000000000..5b61cc1264 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/EmptyStateHolder.java @@ -0,0 +1,64 @@ +/* + * 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.util.snapshot.state; + +import java.util.HashMap; +import java.util.Map; + +/** + * State holder for non partition use case + */ +public class EmptyStateHolder implements StateHolder { + + private Map emptyMap = new HashMap(0); + + @Override + public State getState() { + return null; + } + + @Override + public void returnState(State state) { + } + + @Override + public Map getAllStates() { + return emptyMap; + } + + @Override + public void returnAllStates(Map partitionKeyStates) { + + } + + @Override + public Map getAllGroupByStates() { + return emptyMap; + } + + @Override + public State cleanGroupByStates() { + return null; + } + + @Override + public void returnGroupByStates(Map states) { + + } +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/PartitionStateHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/PartitionStateHolder.java new file mode 100644 index 0000000000..b2ef1d0b5f --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/PartitionStateHolder.java @@ -0,0 +1,165 @@ +/* + * 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.util.snapshot.state; + +import io.siddhi.core.config.SiddhiAppContext; +import io.siddhi.core.exception.SiddhiAppRuntimeException; +import org.apache.log4j.Logger; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +/** + * State holder for partitioned use case + */ +public class PartitionStateHolder implements StateHolder { + private static final Logger log = Logger.getLogger(PartitionStateHolder.class); + private StateFactory stateFactory; + private Map> states = new HashMap<>(); + + public PartitionStateHolder(StateFactory stateFactory) { + this.stateFactory = stateFactory; + } + + @Override + public State getState() { + String partitionFlowId = SiddhiAppContext.getPartitionFlowId(); + String groupByFlowId = SiddhiAppContext.getGroupByFlowId(); + Map partitionStates = states.computeIfAbsent(partitionFlowId, k -> new HashMap<>()); + return partitionStates.computeIfAbsent(groupByFlowId, s -> stateFactory.createNewState()); + } + + @Override + public void returnState(State state) { + String partitionFlowId = SiddhiAppContext.getPartitionFlowId(); + String groupByFlowId = SiddhiAppContext.getGroupByFlowId(); + if (state.activeUseCount == 0) { + try { + if (state.canDestroy()) { + removeState(partitionFlowId, groupByFlowId); + } + } catch (Throwable t) { + log.error("Dropping partition state for partition key '" + partitionFlowId + + "' and the group by key '" + groupByFlowId + "', due to error! " + t.getMessage(), t); + removeState(partitionFlowId, groupByFlowId); + } + } else if (state.activeUseCount < 0) { + throw new SiddhiAppRuntimeException("State active count has reached less then zero for partition key '" + + partitionFlowId + "' and the group by key '" + groupByFlowId + "', current value is " + + state.activeUseCount); + } + } + + private void removeState(String partitionFlowId, String groupByFlowId) { + Map groupByStates = states.get(partitionFlowId); + if (groupByStates != null) { + groupByStates.remove(groupByFlowId); + if (groupByStates.isEmpty()) { + states.remove(partitionFlowId); + } + } + } + + public Map> getAllStates() { + return states; + } + + @Override + public Map getAllGroupByStates() { + String partitionFlowId = SiddhiAppContext.getPartitionFlowId(); + return states.computeIfAbsent(partitionFlowId, k -> new HashMap<>()); + } + + @Override + public State cleanGroupByStates() { + String partitionFlowId = SiddhiAppContext.getPartitionFlowId(); + Map groupByStates = states.remove(partitionFlowId); + if (groupByStates != null) { + return groupByStates.values().stream().findFirst().orElse(null); + } + return null; + } + + @Override + public void returnGroupByStates(Map states) { + String partitionFlowId = SiddhiAppContext.getPartitionFlowId(); + for (Iterator> iterator = + ((Set>) states.entrySet()).iterator(); + iterator.hasNext(); ) { + Map.Entry stateEntry = iterator.next(); + State state = stateEntry.getValue(); + if (state.activeUseCount == 0) { + try { + if (state.canDestroy()) { + iterator.remove(); + } + } catch (Throwable t) { + log.error("Dropping partition state for partition key '" + partitionFlowId + + "' and the group by key '" + stateEntry.getKey() + "', due to error! " + t.getMessage(), t); + iterator.remove(); + } + } else if (state.activeUseCount < 0) { + throw new SiddhiAppRuntimeException("State active count has reached less then zero for partition key '" + + partitionFlowId + "' and the group by key '" + stateEntry.getKey() + "', current value is " + + state.activeUseCount); + } + + } + if (states.isEmpty()) { + states.remove(partitionFlowId); + } + } + + @Override + public void returnAllStates(Map states) { + for (Iterator>> statesIterator = + ((Set>>) states.entrySet()).iterator(); statesIterator.hasNext(); ) { + Map.Entry> statesEntry = statesIterator.next(); + for (Iterator> stateIterator = statesEntry.getValue().entrySet().iterator(); + stateIterator.hasNext(); ) { + Map.Entry stateEntry = stateIterator.next(); + State state = stateEntry.getValue(); + if (state.activeUseCount == 0) { + try { + if (state.canDestroy()) { + stateIterator.remove(); + } + } catch (Throwable t) { + log.error("Dropping partition state for partition key '" + statesEntry.getKey() + + "' and the group by key '" + stateEntry.getKey() + "', due to error! " + + t.getMessage(), t); + stateIterator.remove(); + } + } else if (state.activeUseCount < 0) { + throw new SiddhiAppRuntimeException("State active count has reached less then zero for " + + "partition key '" + statesEntry.getKey() + "' and the group by key '" + + stateEntry.getKey() + "', current value is " + + state.activeUseCount); + } + } + if (statesEntry.getValue().isEmpty()) { + statesIterator.remove(); + } + } + } + + +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/PartitionSyncStateHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/PartitionSyncStateHolder.java new file mode 100644 index 0000000000..e297c0cb4b --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/PartitionSyncStateHolder.java @@ -0,0 +1,88 @@ +/* + * 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.util.snapshot.state; + +import java.util.Map; + +/** + * State holder for partitioned use case + */ +public class PartitionSyncStateHolder implements StateHolder { + private PartitionStateHolder partitionStateHolder; + + public PartitionSyncStateHolder(StateFactory stateFactory) { + partitionStateHolder = new PartitionStateHolder(stateFactory); + } + + @Override + public synchronized State getState() { + State state = partitionStateHolder.getState(); + state.activeUseCount++; + return state; + } + + @Override + public synchronized void returnState(State state) { + state.activeUseCount--; + partitionStateHolder.returnState(state); + } + + + public synchronized Map> getAllStates() { + Map> states = partitionStateHolder.getAllStates(); + for (Map groupByStates : states.values()) { + for (State state : groupByStates.values()) { + state.activeUseCount++; + } + } + return states; + } + + @Override + public synchronized Map getAllGroupByStates() { + Map groupByStates = partitionStateHolder.getAllGroupByStates(); + for (State state : groupByStates.values()) { + state.activeUseCount++; + } + return groupByStates; + } + + @Override + public synchronized State cleanGroupByStates() { + return partitionStateHolder.cleanGroupByStates(); + } + + @Override + public synchronized void returnGroupByStates(Map states) { + for (State state : ((Map) states).values()) { + state.activeUseCount--; + } + partitionStateHolder.returnGroupByStates(states); + } + + @Override + public synchronized void returnAllStates(Map states) { + for (Map groupByStates : ((Map>) states).values()) { + for (State state : groupByStates.values()) { + state.activeUseCount--; + } + } + partitionStateHolder.returnAllStates(states); + } +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SingleStateHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SingleStateHolder.java new file mode 100644 index 0000000000..ff0594daf5 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SingleStateHolder.java @@ -0,0 +1,91 @@ +/* + * 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.util.snapshot.state; + +import org.apache.log4j.Logger; + +import java.util.HashMap; +import java.util.Map; + +/** + * State holder for non partition use case + */ +public class SingleStateHolder implements StateHolder { + private static final Logger log = Logger.getLogger(SingleStateHolder.class); + + private final StateFactory stateFactory; + private State state = null; + final Map groupByStates = new HashMap<>(1); + final Map> allStates = new HashMap<>(1); + + public SingleStateHolder(StateFactory stateFactory) { + this.stateFactory = stateFactory; + this.allStates.put(null, groupByStates); + } + + @Override + public State getState() { + if (state == null) { + state = stateFactory.createNewState(); + groupByStates.put(null, state); + } + return state; + } + + @Override + public void returnState(State state) { + //ignore + } + + public Map> getAllStates() { + if (state == null) { + state = stateFactory.createNewState(); + groupByStates.put(null, state); + } + return allStates; + } + + @Override + public void returnAllStates(Map states) { + //ignore + + } + + @Override + public Map getAllGroupByStates() { + if (state == null) { + state = stateFactory.createNewState(); + groupByStates.put(null, state); + } + return groupByStates; + } + + @Override + public State cleanGroupByStates() { + State returnValue = state; + state = null; + groupByStates.clear(); + return returnValue; + } + + @Override + public void returnGroupByStates(Map states) { + //ignore + } +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SingleSyncStateHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SingleSyncStateHolder.java new file mode 100644 index 0000000000..5c1a68e2ff --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SingleSyncStateHolder.java @@ -0,0 +1,103 @@ +/* + * 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.util.snapshot.state; + +import org.apache.log4j.Logger; + +import java.util.HashMap; +import java.util.Map; + +/** + * State holder for non partition use case + */ +public class SingleSyncStateHolder implements StateHolder { + private static final Logger log = Logger.getLogger(SingleSyncStateHolder.class); + + private final StateFactory stateFactory; + private State state = null; + final Map groupByStates = new HashMap<>(1); + final Map> allStates = new HashMap<>(1); + + public SingleSyncStateHolder(StateFactory stateFactory) { + this.stateFactory = stateFactory; + this.allStates.put(null, groupByStates); + } + + @Override + public State getState() { + if (state == null) { + synchronized (this) { + if (state == null) { + state = stateFactory.createNewState(); + groupByStates.put(null, state); + } + } + } + return state; + } + + @Override + public void returnState(State state) { + //ignore + } + + public Map> getAllStates() { + if (state == null) { + synchronized (this) { + if (state == null) { + state = stateFactory.createNewState(); + groupByStates.put(null, state); + } + } + } + return allStates; + } + + @Override + public void returnAllStates(Map states) { + //ignore + + } + + @Override + public Map getAllGroupByStates() { + if (state == null) { + synchronized (this) { + if (state == null) { + state = stateFactory.createNewState(); + groupByStates.put(null, state); + } + } + } + return groupByStates; + } + + @Override + public synchronized State cleanGroupByStates() { + State returnValue = state; + state = null; + groupByStates.clear(); + return returnValue; + } + + @Override + public void returnGroupByStates(Map states) { + //ignore + } +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SnapshotState.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/Snapshot.java similarity index 87% rename from modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SnapshotState.java rename to modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/Snapshot.java index 077dd45e3c..f0b45541dc 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SnapshotState.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/Snapshot.java @@ -23,16 +23,16 @@ /** * The class which encloses the state to be serialized. */ -public class SnapshotState implements Serializable { +public class Snapshot implements Serializable { private Object state; private boolean isIncrementalSnapshot; - public SnapshotState(Object state) { + public Snapshot(Object state) { this.state = state; this.isIncrementalSnapshot = false; } - public SnapshotState(Object state, boolean isIncrementalSnapshot) { + public Snapshot(Object state, boolean isIncrementalSnapshot) { this.state = state; this.isIncrementalSnapshot = isIncrementalSnapshot; } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SnapshotStateList.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SnapshotStateList.java index 76f0bf8e6f..609fa0cc9e 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SnapshotStateList.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/SnapshotStateList.java @@ -25,24 +25,24 @@ * The class which encloses multiple snapshot states. */ public class SnapshotStateList implements Serializable { - private TreeMap snapshotStates = new TreeMap<>(); + private TreeMap snapshotStates = new TreeMap<>(); public SnapshotStateList() { } - public SnapshotStateList(TreeMap snapshotStates) { + public SnapshotStateList(TreeMap snapshotStates) { this.snapshotStates = snapshotStates; } - public TreeMap getSnapshotStates() { + public TreeMap getSnapshotStates() { return snapshotStates; } - public void setSnapshotStates(TreeMap snapshotStates) { + public void setSnapshotStates(TreeMap snapshotStates) { this.snapshotStates = snapshotStates; } - public void putSnapshotState(Long id, SnapshotState snapshotState) { - this.snapshotStates.put(id, snapshotState); + public void putSnapshotState(Long id, Snapshot snapshot) { + this.snapshotStates.put(id, snapshot); } } diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/State.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/State.java new file mode 100644 index 0000000000..7456b98f4c --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/State.java @@ -0,0 +1,36 @@ +/* + * 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.util.snapshot.state; + +import java.util.Map; + +/** + * State to hold the data + */ +public abstract class State { + + int activeUseCount = 0; + + public abstract boolean canDestroy(); + + public abstract Map snapshot(); + + public abstract void restore(Map state); + +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/StateFactory.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/StateFactory.java new file mode 100644 index 0000000000..39cb582cb3 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/StateFactory.java @@ -0,0 +1,29 @@ +/* + * 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.util.snapshot.state; + +/** + * State factory that create states that are used to store query and other states + * + * @param state that hold data + */ +public interface StateFactory { + + S createNewState(); +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/StateHolder.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/StateHolder.java new file mode 100644 index 0000000000..77af0285e4 --- /dev/null +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/snapshot/state/StateHolder.java @@ -0,0 +1,43 @@ +/* + * 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.util.snapshot.state; + +import java.util.Map; + +/** + * Holder to have all the states + * + * @param state + */ +public interface StateHolder { + + S getState(); + + void returnState(S state); + + Map> getAllStates(); + + void returnAllStates(Map> states); + + Map getAllGroupByStates(); + + S cleanGroupByStates(); + + void returnGroupByStates(Map states); +} diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/timestamp/TimestampGeneratorImpl.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/timestamp/TimestampGeneratorImpl.java index f34229a116..390cf5abee 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/timestamp/TimestampGeneratorImpl.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/timestamp/TimestampGeneratorImpl.java @@ -65,8 +65,7 @@ public class TimestampGeneratorImpl implements TimestampGenerator { /** * List of listeners listening to this timestamp generator. */ - private List timeChangeListeners = - new ArrayList(); + private List timeChangeListeners = new ArrayList<>(); private SiddhiAppContext siddhiAppContext; diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/transport/MultiClientDistributedSink.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/transport/MultiClientDistributedSink.java index b8798bd874..aa5f893a4a 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/transport/MultiClientDistributedSink.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/transport/MultiClientDistributedSink.java @@ -33,9 +33,7 @@ import org.apache.log4j.Logger; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; /** * This is the distributed transport to publish to multiple endpoints using client/publisher for each endpoint. There @@ -117,39 +115,6 @@ public void destroy() { transports.forEach(Sink::destroy); } - /** - * Used to collect the serializable state of the processing element, that need to be - * persisted for the reconstructing the element to the same state on a different point of time - * - * @return stateful objects of the processing element as an array - */ - @Override - public Map currentState() { - // No state specific to this class. But, fetching and returning state from underlying transports as it's state - Map state = new HashMap<>(); - for (int i = 0; i < transports.size(); i++) { - state.put(Integer.toString(i), transports.get(i).currentState()); - } - return state; - } - - /** - * Used to restore serialized state of the processing element, for reconstructing - * the element to the same state as if was on a previous point of time. - * - * @param state the stateful objects of the element as an array on - * the same order provided by currentState(). - */ - @Override - public void restoreState(Map state) { - if (transports != null) { - for (int i = 0; i < transports.size(); i++) { - Map transportState = (Map) state.get(Integer.toString(i)); - transports.get(i).restoreState(transportState); - } - } - } - /** * Connection callback to notify DistributionStrategy about new connection initiations and failures */ diff --git a/modules/siddhi-core/src/main/java/io/siddhi/core/util/transport/SingleClientDistributedSink.java b/modules/siddhi-core/src/main/java/io/siddhi/core/util/transport/SingleClientDistributedSink.java index 69b871e4e7..e7d6fce596 100644 --- a/modules/siddhi-core/src/main/java/io/siddhi/core/util/transport/SingleClientDistributedSink.java +++ b/modules/siddhi-core/src/main/java/io/siddhi/core/util/transport/SingleClientDistributedSink.java @@ -35,7 +35,6 @@ import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; /** @@ -128,28 +127,6 @@ public void destroy() { sink.destroy(); } - /** - * Used to collect the serializable state of the processing element, that need to be - * persisted for the reconstructing the element to the same state on a different point of time - * - * @return stateful objects of the processing element as an array - */ - @Override - public Map currentState() { - return sink.currentState(); - } - - /** - * Used to restore serialized state of the processing element, for reconstructing - * the element to the same state as if was on a previous point of time. - * - * @param state the stateful objects of the element as an array on - * the same order provided by currentState(). - */ - @Override - public void restoreState(Map state) { - sink.restoreState(state); - } private Set findAllDynamicOptions(List destinationOptionHolders) { Set dynamicOptions = new HashSet<>(); 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 939e5107c1..4a9b5e6d2f 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 @@ -24,7 +24,7 @@ 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.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.ZeroStreamEventConverter; import io.siddhi.core.exception.OperationNotSupportedException; import io.siddhi.core.executor.VariableExpressionExecutor; @@ -44,7 +44,6 @@ import io.siddhi.core.util.parser.SchedulerParser; import io.siddhi.core.util.parser.SingleInputStreamParser; import io.siddhi.core.util.parser.helper.QueryParserHelper; -import io.siddhi.core.util.snapshot.Snapshotable; import io.siddhi.core.util.statistics.LatencyTracker; import io.siddhi.core.util.statistics.MemoryCalculable; import io.siddhi.core.util.statistics.ThroughputTracker; @@ -62,7 +61,7 @@ * Window implementation of SiddhiQL. * It can be seen as a global Window which can be accessed from multiple queries. */ -public class Window implements FindableProcessor, Snapshotable, MemoryCalculable { +public class Window implements FindableProcessor, MemoryCalculable { /** * WindowDefinition used to construct this window. @@ -97,9 +96,9 @@ public class Window implements FindableProcessor, Snapshotable, MemoryCalculable */ private WindowProcessor internalWindowProcessor; /** - * StreamEventPool to create new empty StreamEvent. + * StreamEventFactory to create new empty StreamEvent. */ - private StreamEventPool streamEventPool; + private StreamEventFactory streamEventFactory; /** * window operation latency and throughput trackers @@ -137,11 +136,13 @@ public Window(WindowDefinition windowDefinition, SiddhiAppContext siddhiAppConte /** * Initialize the WindowEvent table by creating {@link WindowProcessor} to handle the events. * - * @param tableMap map of {@link Table}s - * @param eventWindowMap map of EventWindows - * @param windowName name of the query window belongs to. + * @param tableMap map of {@link Table}s + * @param eventWindowMap map of EventWindows + * @param windowName name of the query window belongs to. + * @param findToBeExecuted will find will be executed on the window. */ - public void init(Map tableMap, Map eventWindowMap, String windowName) { + public void init(Map tableMap, Map eventWindowMap, String windowName, + boolean findToBeExecuted) { if (this.windowProcessor != null) { return; } @@ -155,8 +156,8 @@ public void init(Map tableMap, Map eventWindowMap metaStreamEvent.addOutputData(attribute); } - this.streamEventPool = new StreamEventPool(metaStreamEvent, 5); - StreamEventCloner streamEventCloner = new StreamEventCloner(metaStreamEvent, this.streamEventPool); + this.streamEventFactory = new StreamEventFactory(metaStreamEvent); + StreamEventCloner streamEventCloner = new StreamEventCloner(metaStreamEvent, this.streamEventFactory); OutputStream.OutputEventType outputEventType = windowDefinition.getOutputEventType(); boolean outputExpectsExpiredEvents = outputEventType != OutputStream.OutputEventType.CURRENT_EVENTS; @@ -164,16 +165,16 @@ public void init(Map tableMap, Map eventWindowMap WindowProcessor internalWindowProcessor = (WindowProcessor) SingleInputStreamParser.generateProcessor (windowDefinition.getWindow(), metaStreamEvent, new ArrayList(), tableMap, false, - outputExpectsExpiredEvents, siddhiQueryContext); + outputExpectsExpiredEvents, findToBeExecuted, siddhiQueryContext); internalWindowProcessor.setStreamEventCloner(streamEventCloner); internalWindowProcessor.constructStreamEventPopulater(metaStreamEvent, 0); EntryValveProcessor entryValveProcessor = null; if (internalWindowProcessor instanceof SchedulingProcessor) { entryValveProcessor = new EntryValveProcessor(this.siddhiAppContext); - Scheduler scheduler = SchedulerParser.parse(entryValveProcessor, this.siddhiAppContext); + Scheduler scheduler = SchedulerParser.parse(entryValveProcessor, siddhiQueryContext); scheduler.init(this.lockWrapper, windowName); - scheduler.setStreamEventPool(streamEventPool); + scheduler.setStreamEventFactory(streamEventFactory); ((SchedulingProcessor) internalWindowProcessor).setScheduler(scheduler); } if (entryValveProcessor != null) { @@ -219,14 +220,14 @@ public void add(ComplexEventChunk complexEventChunk) { // Convert all events to StreamEvent because StateEvents can be passed if directly received from a join ComplexEvent complexEvents = complexEventChunk.getFirst(); - StreamEvent firstEvent = streamEventPool.borrowEvent(); + StreamEvent firstEvent = streamEventFactory.newInstance(); eventConverter.convertComplexEvent(complexEvents, firstEvent); StreamEvent currentEvent = firstEvent; complexEvents = complexEvents.getNext(); int numberOfEvents = 0; while (complexEvents != null) { numberOfEvents++; - StreamEvent nextEvent = streamEventPool.borrowEvent(); + StreamEvent nextEvent = streamEventFactory.newInstance(); eventConverter.convertComplexEvent(complexEvents, nextEvent); currentEvent.setNext(nextEvent); currentEvent = nextEvent; @@ -290,43 +291,6 @@ public LockWrapper getLock() { return lockWrapper; } - - /** - * Return an object array containing the internal state of the internalWindowProcessor. - * - * @return current state of the Window - */ - @Override - public Map currentState() { - return this.internalWindowProcessor.currentState(); - } - - /** - * Restore the internalWindowProcessor using given state. - * - * @param state the stateful objects of the element as an array on - */ - @Override - public void restoreState(Map state) { - this.internalWindowProcessor.restoreState(state); - } - - - /** - * Return the elementId which may be used for snapshot creation. - * - * @return the element id of this {@link Snapshotable} object - */ - @Override - public String getElementId() { - return this.internalWindowProcessor.getElementId(); - } - - @Override - public void clean() { - internalWindowProcessor.clean(); - } - public ProcessingMode getProcessingMode() { return internalWindowProcessor.getProcessingMode(); } @@ -395,14 +359,5 @@ public void setToLast(Processor processor) { // Do nothing } - - public Processor cloneProcessor(String key) { - return new StreamPublishProcessor(this.outputEventType); - } - - @Override - public void clean() { - //ignore - } } } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/ConcatFunctionExtension.java b/modules/siddhi-core/src/test/java/io/siddhi/core/ConcatFunctionExtension.java index 74e8ea1463..22023e0865 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/ConcatFunctionExtension.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/ConcatFunctionExtension.java @@ -22,11 +22,11 @@ import io.siddhi.core.executor.ExpressionExecutor; 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 io.siddhi.query.api.exception.SiddhiAppValidationException; -import java.util.Map; - /* * concat(string1, string2, ..., stringN) * Returns a string that is the result of concatenating two or more string values. @@ -38,16 +38,17 @@ public class ConcatFunctionExtension extends FunctionExecutor { private Attribute.Type returnType = Attribute.Type.STRING; @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length < 2) { throw new SiddhiAppValidationException("str:concat() function requires at least two arguments, " + "but found only " + attributeExpressionExecutors.length); } + return null; } @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { StringBuilder sb = new StringBuilder(); for (Object aData : data) { if (aData != null) { @@ -58,7 +59,7 @@ protected Object execute(Object[] data) { } @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { return data; } @@ -67,13 +68,4 @@ public Attribute.Type getReturnType() { return returnType; } - @Override - public Map currentState() { - return null; //No states - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/aggregation/Aggregation1TestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/aggregation/Aggregation1TestCase.java index ab5a23e957..33f84e92d2 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/aggregation/Aggregation1TestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/aggregation/Aggregation1TestCase.java @@ -162,7 +162,6 @@ public void incrementalStreamProcessorTest5() throws InterruptedException { stockStreamInputHandler.send(new Object[]{"IBM", 100f, null, 200L, 26, 1496289954000L}); stockStreamInputHandler.send(new Object[]{"IBM", 100f, null, 200L, 96, 1496289954000L}); - Thread.sleep(100); Event[] events = siddhiAppRuntime.query("from stockAggregation " + "within \"2017-06-** **:**:**\" " + @@ -947,7 +946,16 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { "2017-06-01 09:35:52 +05:30", "seconds"}); Thread.sleep(100); - List expected = Arrays.asList( + List expected1 = Arrays.asList( + new Object[]{1496289950000L, 240.0, 60.0, 700f, "WSO2", 70f, 50f}, + new Object[]{1496289951000L, 200.0, 100.0, 9600f, "IBM", 100f, 100f}, + new Object[]{1496289952000L, 1400.0, 700.0, 3500f, "IBM", 900f, 500f}, + new Object[]{1496289953000L, 400.0, 400.0, 3600f, "IBM", 400f, 400f}, + new Object[]{1496289953000L, 300.0, 100.0, 1540f, "WSO2", 140f, 60f}, + new Object[]{1496289954000L, 600.0, 600.0, 3600f, "IBM", 600f, 600f}, + new Object[]{1496290016000L, 1000.0, 1000.0, 9000f, "IBM", 1000f, 1000f} + ); + List expected2 = Arrays.asList( new Object[]{1496289950000L, 240.0, 60.0, 700f, "WSO2", 70f, 50f}, new Object[]{1496289951000L, 200.0, 100.0, 9600f, "IBM", 100f, 100f}, new Object[]{1496289952000L, 1400.0, 700.0, 3500f, "IBM", 900f, 500f}, @@ -959,7 +967,9 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { SiddhiTestHelper.waitForEvents(100, 7, inEventCount, 60000); AssertJUnit.assertEquals("Event arrived", true, eventArrived); AssertJUnit.assertEquals("Number of success events", 7, inEventCount.get()); - AssertJUnit.assertEquals("In events matched", true, SiddhiTestHelper.isEventsMatch(inEventsList, expected)); + AssertJUnit.assertEquals("In events matched", true, + SiddhiTestHelper.isEventsMatch(inEventsList, expected1) || + SiddhiTestHelper.isEventsMatch(inEventsList, expected2)); } finally { siddhiAppRuntime.shutdown(); } @@ -1046,7 +1056,6 @@ public void incrementalStreamProcessorTest16() throws InterruptedException { try { // Thursday, June 1, 2017 4:05:50 AM stockStreamInputHandler.send(new Object[]{"WSO2", 50f, 60f, 90L, 6, "June 1, 2017 4:05:50 AM"}); - AssertJUnit.assertTrue(appender.getMessages().contains("'June 1, 2017 4:05:50 AM' doesn't match the " + "supported formats --
:: (for GMT time zone) or --
" + ":: (for non GMT time zone). The ISO 8601 UTC offset must be provided for " + @@ -2172,8 +2181,7 @@ public void incrementalStreamProcessorTest37() throws InterruptedException { siddhiAppRuntime.shutdown(); } - @Test - //(dependsOnMethods = {"incrementalStreamProcessorTest37"}) + @Test(dependsOnMethods = {"incrementalStreamProcessorTest37"}) public void incrementalStreamProcessorTest38() throws InterruptedException { LOG.info("incrementalStreamProcessorTest38"); SiddhiManager siddhiManager = new SiddhiManager(); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/managment/IncrementalPersistenceTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/managment/IncrementalPersistenceTestCase.java index 8fdb524fba..f61479b821 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/managment/IncrementalPersistenceTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/managment/IncrementalPersistenceTestCase.java @@ -130,6 +130,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { try { siddhiAppRuntime.restoreLastRevision(); } catch (CannotRestoreSiddhiAppStateException e) { + log.error(e.getMessage(), e); Assert.fail("Restoring of Siddhi app " + siddhiAppRuntime.getName() + " failed"); } siddhiAppRuntime.start(); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/managment/SnapshotableEventQueueTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/managment/SnapshotableEventQueueTestCase.java index 284f01280b..f154ead5b7 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/managment/SnapshotableEventQueueTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/managment/SnapshotableEventQueueTestCase.java @@ -22,10 +22,10 @@ import io.siddhi.core.event.stream.Operation; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.event.stream.StreamEventCloner; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.holder.SnapshotableStreamEventQueue; import io.siddhi.core.event.stream.holder.StreamEventClonerHolder; -import io.siddhi.core.util.snapshot.state.SnapshotState; +import io.siddhi.core.util.snapshot.state.Snapshot; import io.siddhi.core.util.snapshot.state.SnapshotStateList; import io.siddhi.query.api.definition.Attribute; import org.apache.log4j.Logger; @@ -84,7 +84,7 @@ public void incrementalPersistenceTest1() throws InterruptedException, IOExcepti metaStreamEvent.addOutputData(new Attribute("volume", Attribute.Type.LONG)); StreamEventCloner streamEventCloner = new StreamEventCloner(metaStreamEvent, - new StreamEventPool(metaStreamEvent, 5)); + new StreamEventFactory(metaStreamEvent)); SnapshotableStreamEventQueue snapshotableStreamEventQueue = new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); @@ -98,7 +98,7 @@ public void incrementalPersistenceTest1() throws InterruptedException, IOExcepti } HashMap snapshots = new HashMap<>(); - SnapshotState snapshot1 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot1 = snapshotableStreamEventQueue.getSnapshot(); StreamEvent streamEvents = (StreamEvent) snapshot1.getState(); Assert.assertTrue(streamEvents != null); snapshots.put(3L, toString(snapshot1)); @@ -108,7 +108,7 @@ public void incrementalPersistenceTest1() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.add(streamEventCloner.copyStreamEvent(streamEvent)); } - SnapshotState snapshot2 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot2 = snapshotableStreamEventQueue.getSnapshot(); ArrayList operationLog = (ArrayList) snapshot2.getState(); Assert.assertTrue(operationLog != null); snapshots.put(4L, toString(snapshot2)); @@ -118,7 +118,7 @@ public void incrementalPersistenceTest1() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.add(streamEventCloner.copyStreamEvent(streamEvent)); } - SnapshotState snapshot3 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot3 = snapshotableStreamEventQueue.getSnapshot(); operationLog = (ArrayList) snapshot3.getState(); Assert.assertTrue(operationLog != null); snapshots.put(5L, toString(snapshot3)); @@ -127,7 +127,7 @@ public void incrementalPersistenceTest1() throws InterruptedException, IOExcepti new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); SnapshotStateList snapshotStateList = new SnapshotStateList(); for (Map.Entry entry : snapshots.entrySet()) { - snapshotStateList.putSnapshotState(entry.getKey(), (SnapshotState) fromString(entry.getValue())); + snapshotStateList.putSnapshotState(entry.getKey(), (Snapshot) fromString(entry.getValue())); } snapshotableStreamEventQueue2.restore(snapshotStateList); @@ -138,7 +138,7 @@ public void incrementalPersistenceTest1() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.add(streamEventCloner.copyStreamEvent(streamEvent)); } - SnapshotState snapshot4 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot4 = snapshotableStreamEventQueue.getSnapshot(); streamEvents = (StreamEvent) snapshot1.getState(); Assert.assertTrue(streamEvents != null); snapshots = new HashMap<>(); @@ -148,7 +148,7 @@ public void incrementalPersistenceTest1() throws InterruptedException, IOExcepti new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); snapshotStateList = new SnapshotStateList(); for (Map.Entry entry : snapshots.entrySet()) { - snapshotStateList.putSnapshotState(entry.getKey(), (SnapshotState) fromString(entry.getValue())); + snapshotStateList.putSnapshotState(entry.getKey(), (Snapshot) fromString(entry.getValue())); } snapshotableStreamEventQueue3.restore(snapshotStateList); @@ -164,7 +164,7 @@ public void incrementalPersistenceTest2() throws InterruptedException, IOExcepti metaStreamEvent.addOutputData(new Attribute("volume", Attribute.Type.LONG)); StreamEventCloner streamEventCloner = new StreamEventCloner(metaStreamEvent, - new StreamEventPool(metaStreamEvent, 5)); + new StreamEventFactory(metaStreamEvent)); SnapshotableStreamEventQueue snapshotableStreamEventQueue = new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); @@ -178,7 +178,7 @@ public void incrementalPersistenceTest2() throws InterruptedException, IOExcepti } HashMap snapshots = new HashMap<>(); - SnapshotState snapshot1 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot1 = snapshotableStreamEventQueue.getSnapshot(); StreamEvent streamEvents = (StreamEvent) snapshot1.getState(); Assert.assertTrue(streamEvents != null); snapshots.put(3L, toString(snapshot1)); @@ -188,7 +188,7 @@ public void incrementalPersistenceTest2() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.remove(); } - SnapshotState snapshot2 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot2 = snapshotableStreamEventQueue.getSnapshot(); ArrayList operationLog = (ArrayList) snapshot2.getState(); Assert.assertTrue(operationLog != null); snapshots.put(4L, toString(snapshot2)); @@ -198,7 +198,7 @@ public void incrementalPersistenceTest2() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.add(streamEventCloner.copyStreamEvent(streamEvent)); } - SnapshotState snapshot3 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot3 = snapshotableStreamEventQueue.getSnapshot(); operationLog = (ArrayList) snapshot3.getState(); Assert.assertTrue(operationLog != null); snapshots.put(5L, toString(snapshot3)); @@ -207,7 +207,7 @@ public void incrementalPersistenceTest2() throws InterruptedException, IOExcepti new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); SnapshotStateList snapshotStateList = new SnapshotStateList(); for (Map.Entry entry : snapshots.entrySet()) { - snapshotStateList.putSnapshotState(entry.getKey(), (SnapshotState) fromString(entry.getValue())); + snapshotStateList.putSnapshotState(entry.getKey(), (Snapshot) fromString(entry.getValue())); } snapshotableStreamEventQueue2.restore(snapshotStateList); @@ -223,7 +223,7 @@ public void incrementalPersistenceTest3() throws InterruptedException, IOExcepti metaStreamEvent.addOutputData(new Attribute("volume", Attribute.Type.LONG)); StreamEventCloner streamEventCloner = new StreamEventCloner(metaStreamEvent, - new StreamEventPool(metaStreamEvent, 5)); + new StreamEventFactory(metaStreamEvent)); SnapshotableStreamEventQueue snapshotableStreamEventQueue = new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); StreamEvent streamEvent = new StreamEvent(metaStreamEvent.getBeforeWindowData().size(), @@ -236,7 +236,7 @@ public void incrementalPersistenceTest3() throws InterruptedException, IOExcepti } HashMap snapshots = new HashMap<>(); - SnapshotState snapshot1 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot1 = snapshotableStreamEventQueue.getSnapshot(); StreamEvent streamEvents = (StreamEvent) snapshot1.getState(); Assert.assertTrue(streamEvents != null); snapshots.put(3L, toString(snapshot1)); @@ -250,7 +250,7 @@ public void incrementalPersistenceTest3() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.remove(); } - SnapshotState snapshot2 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot2 = snapshotableStreamEventQueue.getSnapshot(); ArrayList operationLog = (ArrayList) snapshot2.getState(); Assert.assertTrue(operationLog != null); snapshots.put(4L, toString(snapshot2)); @@ -260,7 +260,7 @@ public void incrementalPersistenceTest3() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.add(streamEventCloner.copyStreamEvent(streamEvent)); } - SnapshotState snapshot3 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot3 = snapshotableStreamEventQueue.getSnapshot(); operationLog = (ArrayList) snapshot3.getState(); Assert.assertTrue(operationLog != null); snapshots.put(5L, toString(snapshot3)); @@ -269,7 +269,7 @@ public void incrementalPersistenceTest3() throws InterruptedException, IOExcepti new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); SnapshotStateList snapshotStateList = new SnapshotStateList(); for (Map.Entry entry : snapshots.entrySet()) { - snapshotStateList.putSnapshotState(entry.getKey(), (SnapshotState) fromString(entry.getValue())); + snapshotStateList.putSnapshotState(entry.getKey(), (Snapshot) fromString(entry.getValue())); } snapshotableStreamEventQueue2.restore(snapshotStateList); Assert.assertEquals(snapshotableStreamEventQueue, snapshotableStreamEventQueue2); @@ -284,7 +284,7 @@ public void incrementalPersistenceTest4() throws InterruptedException, IOExcepti metaStreamEvent.addOutputData(new Attribute("volume", Attribute.Type.LONG)); StreamEventCloner streamEventCloner = new StreamEventCloner(metaStreamEvent, - new StreamEventPool(metaStreamEvent, 5)); + new StreamEventFactory(metaStreamEvent)); SnapshotableStreamEventQueue snapshotableStreamEventQueue = new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); StreamEvent streamEvent = new StreamEvent(metaStreamEvent.getBeforeWindowData().size(), @@ -297,7 +297,7 @@ public void incrementalPersistenceTest4() throws InterruptedException, IOExcepti } HashMap snapshots = new HashMap<>(); - SnapshotState snapshot1 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot1 = snapshotableStreamEventQueue.getSnapshot(); StreamEvent streamEvents = (StreamEvent) snapshot1.getState(); Assert.assertTrue(streamEvents != null); snapshots.put(3L, toString(snapshot1)); @@ -313,7 +313,7 @@ public void incrementalPersistenceTest4() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.poll(); } - SnapshotState snapshot2 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot2 = snapshotableStreamEventQueue.getSnapshot(); ArrayList operationLog = (ArrayList) snapshot2.getState(); Assert.assertTrue(operationLog != null); snapshots.put(4L, toString(snapshot2)); @@ -323,7 +323,7 @@ public void incrementalPersistenceTest4() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.add(streamEventCloner.copyStreamEvent(streamEvent)); } - SnapshotState snapshot3 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot3 = snapshotableStreamEventQueue.getSnapshot(); operationLog = (ArrayList) snapshot3.getState(); Assert.assertTrue(operationLog != null); snapshots.put(5L, toString(snapshot3)); @@ -332,7 +332,7 @@ public void incrementalPersistenceTest4() throws InterruptedException, IOExcepti new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); SnapshotStateList snapshotStateList = new SnapshotStateList(); for (Map.Entry entry : snapshots.entrySet()) { - snapshotStateList.putSnapshotState(entry.getKey(), (SnapshotState) fromString(entry.getValue())); + snapshotStateList.putSnapshotState(entry.getKey(), (Snapshot) fromString(entry.getValue())); } snapshotableStreamEventQueue2.restore(snapshotStateList); Assert.assertEquals(snapshotableStreamEventQueue, snapshotableStreamEventQueue2); @@ -347,7 +347,7 @@ public void incrementalPersistenceTest5() throws InterruptedException, IOExcepti metaStreamEvent.addOutputData(new Attribute("volume", Attribute.Type.LONG)); StreamEventCloner streamEventCloner = new StreamEventCloner(metaStreamEvent, - new StreamEventPool(metaStreamEvent, 5)); + new StreamEventFactory(metaStreamEvent)); SnapshotableStreamEventQueue snapshotableStreamEventQueue = new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); StreamEvent streamEvent = new StreamEvent(metaStreamEvent.getBeforeWindowData().size(), @@ -360,7 +360,7 @@ public void incrementalPersistenceTest5() throws InterruptedException, IOExcepti } HashMap snapshots = new HashMap<>(); - SnapshotState snapshot1 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot1 = snapshotableStreamEventQueue.getSnapshot(); StreamEvent streamEvents = (StreamEvent) snapshot1.getState(); Assert.assertTrue(streamEvents != null); snapshots.put(3L, toString(snapshot1)); @@ -375,7 +375,7 @@ public void incrementalPersistenceTest5() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.overwrite(streamEventCloner.copyStreamEvent(streamEvent)); } - SnapshotState snapshot2 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot2 = snapshotableStreamEventQueue.getSnapshot(); ArrayList operationLog = (ArrayList) snapshot2.getState(); Assert.assertTrue(operationLog != null); snapshots.put(4L, toString(snapshot2)); @@ -385,7 +385,7 @@ public void incrementalPersistenceTest5() throws InterruptedException, IOExcepti snapshotableStreamEventQueue.add(streamEventCloner.copyStreamEvent(streamEvent)); } - SnapshotState snapshot3 = snapshotableStreamEventQueue.getSnapshot(); + Snapshot snapshot3 = snapshotableStreamEventQueue.getSnapshot(); operationLog = (ArrayList) snapshot3.getState(); Assert.assertTrue(operationLog != null); snapshots.put(5L, toString(snapshot3)); @@ -394,7 +394,7 @@ public void incrementalPersistenceTest5() throws InterruptedException, IOExcepti new SnapshotableStreamEventQueue(new StreamEventClonerHolder(streamEventCloner)); SnapshotStateList snapshotStateList = new SnapshotStateList(); for (Map.Entry entry : snapshots.entrySet()) { - snapshotStateList.putSnapshotState(entry.getKey(), (SnapshotState) fromString(entry.getValue())); + snapshotStateList.putSnapshotState(entry.getKey(), (Snapshot) fromString(entry.getValue())); } snapshotableStreamEventQueue2.restore(snapshotStateList); Assert.assertEquals(snapshotableStreamEventQueue, snapshotableStreamEventQueue2); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/FilterTestCase1.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/FilterTestCase1.java index c8ba7e189f..810afee54f 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/FilterTestCase1.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/FilterTestCase1.java @@ -102,7 +102,6 @@ public void receive(long timeStamp, Event[] inEvents, Event[] removeEvents) { }; siddhiAppRuntime.addCallback("query2", queryCallback); - queryCallback.startProcessing(); InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); @@ -112,7 +111,6 @@ public void receive(long timeStamp, Event[] inEvents, Event[] removeEvents) { inputHandler.send(new Object[]{"WSO2", 60.5f, 200L}); SiddhiTestHelper.waitForEvents(10, 1, count, 100); AssertJUnit.assertTrue(eventArrived.get()); - queryCallback.stopProcessing(); siddhiAppRuntime.shutdown(); } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/FilterTestCase2.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/FilterTestCase2.java index f316aaf781..f8be03dc6a 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/FilterTestCase2.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/FilterTestCase2.java @@ -1595,8 +1595,11 @@ public void filterTest120() throws InterruptedException { SiddhiManager siddhiManager = new SiddhiManager(); String cseEventStream = "define stream cseEventStream (symbol string, price float, volume long);"; - String query = "@info(name = 'query1') from cseEventStream select symbol,sum(price) as sumprice group by " + - "symbol having sumprice > 880 insert into outputStream ;"; + String query = "@info(name = 'query1') " + + "from cseEventStream " + + "select symbol, sum(price) as sumprice " + + "group by symbol " + + "having sumprice > 880 insert into outputStream ;"; SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(cseEventStream + query); siddhiAppRuntime.addCallback("query1", new QueryCallback() { diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/PassThroughTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/PassThroughTestCase.java index 93dbf398fa..a5190d50a5 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/PassThroughTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/PassThroughTestCase.java @@ -232,7 +232,6 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { }; siddhiAppRuntime.addCallback("query2", queryCallback); - queryCallback.startProcessing(); InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); @@ -243,7 +242,6 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { SiddhiTestHelper.waitForEvents(10, eventArrived, 100); AssertJUnit.assertEquals(2, count); AssertJUnit.assertTrue(eventArrived.get()); - queryCallback.stopProcessing(); siddhiAppRuntime.shutdown(); } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/ExtensionTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/ExtensionTestCase.java index a3f65e7fbf..727c08e8b8 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/ExtensionTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/ExtensionTestCase.java @@ -21,7 +21,7 @@ import io.siddhi.core.SiddhiManager; import io.siddhi.core.event.Event; import io.siddhi.core.query.extension.util.CustomFunctionExtension; -import io.siddhi.core.query.extension.util.StringConcatAggregatorString; +import io.siddhi.core.query.extension.util.StringConcatAggregatorExecutorString; import io.siddhi.core.query.output.callback.QueryCallback; import io.siddhi.core.stream.input.InputHandler; import io.siddhi.core.util.EventPrinter; @@ -85,7 +85,7 @@ public void extensionTest2() throws InterruptedException, ClassNotFoundException log.info("extension test2"); SiddhiManager siddhiManager = new SiddhiManager(); siddhiManager.setExtension("custom:plus", CustomFunctionExtension.class); - siddhiManager.setExtension("email:getAll", StringConcatAggregatorString.class); + siddhiManager.setExtension("email:getAll", StringConcatAggregatorExecutorString.class); String cseEventStream = "define stream cseEventStream (symbol string, price long, volume long);"; String query = ("@info(name = 'query1') from cseEventStream select symbol , custom:plus(price,volume) as " + @@ -128,7 +128,7 @@ public void extensionTest3() throws InterruptedException { log.info("extension test3"); SiddhiManager siddhiManager = new SiddhiManager(); siddhiManager.setExtension("custom:plus", CustomFunctionExtension.class); - siddhiManager.setExtension("email:getAllNew", StringConcatAggregatorString.class); + siddhiManager.setExtension("email:getAllNew", StringConcatAggregatorExecutorString.class); String cseEventStream = "" + "" + @@ -172,7 +172,7 @@ public void extensionTest4() throws InterruptedException, ClassNotFoundException log.info("extension test4"); SiddhiManager siddhiManager = new SiddhiManager(); siddhiManager.setExtension("custom:plus", CustomFunctionExtension.class); - siddhiManager.setExtension("email:getAll", StringConcatAggregatorString.class); + siddhiManager.setExtension("email:getAll", StringConcatAggregatorExecutorString.class); String cseEventStream = "define stream cseEventStream (price long, volume long);"; @@ -218,7 +218,7 @@ public void extensionTest5() throws InterruptedException, ClassNotFoundException configMap.put("email.getAllNew.append.abc", "true"); siddhiManager.setConfigManager(new InMemoryConfigManager(configMap, null)); siddhiManager.setExtension("custom:plus", CustomFunctionExtension.class); - siddhiManager.setExtension("email:getAllNew", StringConcatAggregatorString.class); + siddhiManager.setExtension("email:getAllNew", StringConcatAggregatorExecutorString.class); String cseEventStream = "" + "" + diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/CustomFunctionExtension.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/CustomFunctionExtension.java index 8bef895603..4f812672f7 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/CustomFunctionExtension.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/CustomFunctionExtension.java @@ -27,10 +27,10 @@ import io.siddhi.core.executor.ExpressionExecutor; 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 java.util.Map; - @Extension( name = "plus", namespace = "custom", @@ -55,9 +55,9 @@ public class CustomFunctionExtension extends FunctionExecutor { private Attribute.Type returnType; @Override - public void init(ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + public StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { for (ExpressionExecutor expressionExecutor : attributeExpressionExecutors) { Attribute.Type attributeType = expressionExecutor.getReturnType(); if (attributeType == Attribute.Type.DOUBLE) { @@ -69,7 +69,7 @@ public void init(ExpressionExecutor[] attributeExpressionExecutors, returnType = Attribute.Type.LONG; } } - + return null; } /** @@ -84,7 +84,7 @@ public Attribute.Type getReturnType() { @Override - protected Object execute(Object[] obj) { + protected Object execute(Object[] obj, State state) { if (returnType == Attribute.Type.DOUBLE) { double total = 0; for (Object aObj : obj) { @@ -103,7 +103,7 @@ protected Object execute(Object[] obj) { } @Override - protected Object execute(Object obj) { + protected Object execute(Object obj, State state) { if (returnType == Attribute.Type.DOUBLE) { double total = 0; if (obj instanceof Object[]) { @@ -123,14 +123,4 @@ protected Object execute(Object obj) { } } - @Override - public Map currentState() { - //No state - return null; - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/StringConcatAggregatorString.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/StringConcatAggregatorExecutorString.java similarity index 55% rename from modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/StringConcatAggregatorString.java rename to modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/StringConcatAggregatorExecutorString.java index 431b1a5bf4..ffae47a80d 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/StringConcatAggregatorString.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/extension/util/StringConcatAggregatorExecutorString.java @@ -26,8 +26,10 @@ import io.siddhi.core.config.SiddhiQueryContext; import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.ProcessingMode; -import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator; +import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregatorExecutor; 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.Type; import java.util.HashMap; @@ -52,9 +54,9 @@ description = "This will concatenate given input values and return 'hello world'." ) ) -public class StringConcatAggregatorString extends AttributeAggregator { +public class StringConcatAggregatorExecutorString extends + AttributeAggregatorExecutor { private static final long serialVersionUID = 1358667438272544590L; - private String aggregatedStringValue = ""; private boolean appendAbc = false; /** @@ -63,15 +65,17 @@ public class StringConcatAggregatorString extends AttributeAggregator { * @param attributeExpressionExecutors are the executors of each attributes in the function * @param processingMode query processing mode * @param outputExpectsExpiredEvents is expired events sent as output - * @param configReader this hold the {@link StringConcatAggregatorString} configuration reader. + * @param configReader this hold the {@link StringConcatAggregatorExecutorString} + * configuration reader. * @param siddhiQueryContext current siddhi query context */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ProcessingMode processingMode, - boolean outputExpectsExpiredEvents, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ProcessingMode processingMode, + boolean outputExpectsExpiredEvents, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { appendAbc = Boolean.parseBoolean(configReader.readConfig("append.abc", "false")); - + return () -> new ExecutorState(); } @Override @@ -81,70 +85,75 @@ public Type getReturnType() { @Override - public Object processAdd(Object data) { - aggregatedStringValue = aggregatedStringValue + data; + public Object processAdd(Object data, ExecutorState state) { + state.aggregatedStringValue = state.aggregatedStringValue + data; if (appendAbc) { - return aggregatedStringValue + "-abc"; + return state.aggregatedStringValue + "-abc"; } else { - return aggregatedStringValue; + return state.aggregatedStringValue; } } @Override - public Object processAdd(Object[] data) { + public Object processAdd(Object[] data, ExecutorState state) { for (Object aData : data) { - aggregatedStringValue = aggregatedStringValue + aData; + state.aggregatedStringValue = state.aggregatedStringValue + aData; } if (appendAbc) { - return aggregatedStringValue + "-abc"; + return state.aggregatedStringValue + "-abc"; } else { - return aggregatedStringValue; + return state.aggregatedStringValue; } } @Override - public Object processRemove(Object data) { - aggregatedStringValue = aggregatedStringValue.replaceFirst(data.toString(), ""); + public Object processRemove(Object data, ExecutorState state) { + state.aggregatedStringValue = state.aggregatedStringValue.replaceFirst(data.toString(), ""); if (appendAbc) { - return aggregatedStringValue + "-abc"; + return state.aggregatedStringValue + "-abc"; } else { - return aggregatedStringValue; + return state.aggregatedStringValue; } } @Override - public Object processRemove(Object[] data) { + public Object processRemove(Object[] data, ExecutorState state) { for (Object aData : data) { - aggregatedStringValue = aggregatedStringValue.replaceFirst(aData.toString(), ""); + state.aggregatedStringValue = state.aggregatedStringValue.replaceFirst(aData.toString(), ""); } if (appendAbc) { - return aggregatedStringValue + "-abc"; + return state.aggregatedStringValue + "-abc"; } else { - return aggregatedStringValue; + return state.aggregatedStringValue; } } @Override - public boolean canDestroy() { - return aggregatedStringValue != null && aggregatedStringValue.equals(""); + public Object reset(ExecutorState state) { + state.aggregatedStringValue = ""; + return state.aggregatedStringValue; } - @Override - public Object reset() { - aggregatedStringValue = ""; - return aggregatedStringValue; - } - @Override - public Map currentState() { - Map state = new HashMap<>(); - state.put("AggregatedStringValue", aggregatedStringValue); - return state; - } + class ExecutorState extends State { + private String aggregatedStringValue = ""; - @Override - public void restoreState(Map state) { - aggregatedStringValue = (String) state.get("AggregatedStringValue"); + @Override + public boolean canDestroy() { + return aggregatedStringValue != null && aggregatedStringValue.equals(""); + } + + @Override + public Map snapshot() { + Map state = new HashMap<>(); + state.put("AggregatedStringValue", aggregatedStringValue); + return state; + } + + @Override + public void restore(Map state) { + aggregatedStringValue = (String) state.get("AggregatedStringValue"); + } } } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/JoinPartitionTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/JoinPartitionTestCase.java index a7f53d0c3a..96619aff0b 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/JoinPartitionTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/JoinPartitionTestCase.java @@ -83,21 +83,25 @@ public void receive(Event[] events) { } - @Test + @Test(dependsOnMethods = "testJoinPartition1") public void testJoinPartition2() throws InterruptedException { log.info("Join partition test2"); SiddhiManager siddhiManager = new SiddhiManager(); - String siddhiApp = "define stream cseEventStream (symbol string, user string,volume int); define stream " + - "twitterStream (user string, tweet string, company string);" - + "partition with (user of cseEventStream, user of twitterStream) begin @info(name = 'query1') " + - "from cseEventStream#window.time(1 sec) join twitterStream#window.time(1 sec) " + - "on cseEventStream.symbol== twitterStream.company " + - "select cseEventStream.symbol as symbol, cseEventStream.user as user,twitterStream.tweet, " + - "cseEventStream.volume " + - "insert all events into outputStream ;" + "" + - "end "; + String siddhiApp = "" + + "define stream cseEventStream (symbol string, user string,volume int); " + + "define stream twitterStream (user string, tweet string, company string); " + + "partition with (user of cseEventStream, user of twitterStream) " + + "begin " + + " @info(name = 'query1') " + + " from cseEventStream#window.time(1 sec) join twitterStream#window.time(1 sec) " + + " on cseEventStream.symbol== twitterStream.company " + + " select cseEventStream.symbol as symbol, cseEventStream.user as user,twitterStream.tweet, " + + " cseEventStream.volume " + + " insert all events into outputStream ;" + + "end " + + ""; SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); @@ -129,21 +133,28 @@ public void receive(Event[] events) { } - @Test + @Test(dependsOnMethods = "testJoinPartition2") public void testJoinPartition3() throws InterruptedException { log.info("Join partition test3"); SiddhiManager siddhiManager = new SiddhiManager(); - String siddhiApp = "define stream cseEventStream (symbol string, user string,volume int); define stream " + - "twitterStream (user string, tweet string, company string);" - + "partition with (user of cseEventStream, user of twitterStream) begin @info(name = 'query1') " + - "from cseEventStream#window.time(1 sec) join twitterStream#window.time(1 sec) " + - "on cseEventStream.symbol== twitterStream.company " + - "select cseEventStream.symbol as symbol, cseEventStream.user as user,twitterStream.tweet, " + - "cseEventStream.volume " + - "insert all events into #outputStream ;" + - "@info(name = 'query2') from #outputStream select symbol,user insert all events into outStream;" + + String siddhiApp = "" + + "define stream cseEventStream (symbol string, user string,volume int); " + + "define stream twitterStream (user string, tweet string, company string);" + + "partition with (user of cseEventStream, user of twitterStream) " + + "begin " + + " @info(name = 'query1') " + + " from cseEventStream#window.time(1 sec) join twitterStream#window.time(1 sec) " + + " on cseEventStream.symbol== twitterStream.company " + + " select cseEventStream.symbol as symbol, cseEventStream.user as user,twitterStream.tweet, " + + " cseEventStream.volume " + + " insert all events into #outputStream ;" + + "" + + " @info(name = 'query2') " + + " from #outputStream " + + " select symbol,user " + + " insert all events into outStream;" + "end "; @@ -167,17 +178,16 @@ public void receive(Event[] events) { cseEventStreamHandler.send(new Object[]{"IBM", "User2", 100}); twitterStreamHandler.send(new Object[]{"User2", "Hello World", "IBM"}); - twitterStreamHandler.send(new Object[]{"User2", "World", "IBM"}); - SiddhiTestHelper.waitForEvents(100, 8, count, 6000); + SiddhiTestHelper.waitForEvents(200, 8, count, 6000); AssertJUnit.assertEquals(8, count.get()); AssertJUnit.assertTrue(eventArrived); siddhiAppRuntime.shutdown(); } - @Test + @Test(dependsOnMethods = "testJoinPartition3") public void testJoinPartition4() throws InterruptedException { log.info("Join partition test4"); @@ -229,7 +239,7 @@ public void receive(Event[] events) { } - @Test + @Test//(dependsOnMethods = "testJoinPartition4") public void testJoinPartition5() throws InterruptedException { log.info("Join partition test5"); @@ -281,7 +291,7 @@ public void receive(Event[] events) { } - @Test + @Test(dependsOnMethods = "testJoinPartition5") public void testJoinPartition6() throws InterruptedException { log.info("Join partition test6"); @@ -328,7 +338,7 @@ public void receive(Event[] events) { } - @Test + @Test(dependsOnMethods = "testJoinPartition6") public void testJoinPartition7() throws InterruptedException { log.info("Join partition test7"); @@ -377,7 +387,7 @@ public void receive(Event[] events) { } - @Test + @Test(dependsOnMethods = "testJoinPartition7") public void testJoinPartition8() throws InterruptedException { log.info("Join partition test8"); @@ -419,7 +429,7 @@ public void receive(Event[] events) { } - @Test + @Test(dependsOnMethods = "testJoinPartition8") public void testJoinPartition9() throws InterruptedException { log.info("Join partition test9"); @@ -467,7 +477,7 @@ public void receive(Event[] events) { } - @Test + @Test(dependsOnMethods = "testJoinPartition9") public void testJoinPartition10() throws InterruptedException { log.info("Join partition test10"); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionDataPurgingTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionDataPurgingTestCase.java new file mode 100644 index 0000000000..7b6f02ef80 --- /dev/null +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionDataPurgingTestCase.java @@ -0,0 +1,129 @@ +/* + * 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.partition; + +import io.siddhi.core.SiddhiAppRuntime; +import io.siddhi.core.SiddhiManager; +import io.siddhi.core.event.Event; +import io.siddhi.core.stream.input.InputHandler; +import io.siddhi.core.stream.output.StreamCallback; +import io.siddhi.core.util.EventPrinter; +import io.siddhi.core.util.SiddhiTestHelper; +import org.apache.log4j.Logger; +import org.testng.Assert; +import org.testng.AssertJUnit; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.concurrent.atomic.AtomicInteger; + +public class PartitionDataPurgingTestCase { + private static final Logger log = Logger.getLogger(PartitionDataPurgingTestCase.class); + private AtomicInteger count = new AtomicInteger(0); + private int stockStreamEventCount; + private boolean eventArrived; + + @BeforeMethod + public void init() { + count.set(0); + eventArrived = false; + stockStreamEventCount = 0; + } + + @Test + public void testPartitionPurgQuery1() throws InterruptedException { + log.info("Partition test"); + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "@app:name('PartitionTest') " + + "" + + "define stream streamA (symbol string, price int);" + + "" + + "" + + "@purge(enable='true', interval='1 sec', idle.period='1 sec') " + + "partition with (symbol of streamA) " + + "begin " + + " @info(name = 'query1') " + + " from streamA#window.length(3) " + + " select symbol, avg(price) as total " + + " insert into StockQuote ; " + + "end "; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + StreamCallback streamCallback = new StreamCallback() { + @Override + public void receive(Event[] events) { + EventPrinter.print(events); + AssertJUnit.assertTrue("IBM".equals(events[0].getData(0)) || + "WSO2".equals(events[0].getData(0))); + for (Event event : events) { + int eventCount = count.incrementAndGet(); + switch (eventCount) { + case 1: + Assert.assertEquals(event.getData(1), 100.0); + break; + case 2: + Assert.assertEquals(event.getData(1), 100.0); + break; + case 3: + Assert.assertEquals(event.getData(1), 200.0); + break; + case 4: + Assert.assertEquals(event.getData(1), 40.0); + break; + case 5: + Assert.assertEquals(event.getData(1), 25.0); + break; + case 6: + Assert.assertEquals(event.getData(1), 20.0); + break; + case 7: + Assert.assertEquals(event.getData(1), 100.0); + break; + case 8: + Assert.assertEquals(event.getData(1), 10.0); + break; + } + + } + eventArrived = true; + } + }; + siddhiAppRuntime.addCallback("StockQuote", streamCallback); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("streamA"); + siddhiAppRuntime.start(); + inputHandler.send(new Object[]{"IBM", 100}); + inputHandler.send(new Object[]{"IBM", 100}); + inputHandler.send(new Object[]{"IBM", 400}); + inputHandler.send(new Object[]{"WSO2", 40}); + inputHandler.send(new Object[]{"WSO2", 10}); + inputHandler.send(new Object[]{"WSO2", 10}); + Thread.sleep(1100); + inputHandler.send(new Object[]{"IBM", 100}); + inputHandler.send(new Object[]{"WSO2", 10}); + + SiddhiTestHelper.waitForEvents(100, 8, count, 60000); + AssertJUnit.assertTrue(eventArrived); + AssertJUnit.assertEquals(8, count.get()); + siddhiAppRuntime.shutdown(); + } + +} diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionTestCase1.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionTestCase1.java index 1862df8efa..def0b81aad 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionTestCase1.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionTestCase1.java @@ -66,8 +66,10 @@ public void testPartitionQuery() throws InterruptedException { "define stream streamA (symbol string, price int);" + "partition with (symbol of streamA) " + "begin " + - "@info(name = 'query1') " + - "from streamA select symbol,price insert into StockQuote ; " + + " @info(name = 'query1') " + + " from streamA " + + " select symbol, price " + + " insert into StockQuote ; " + "end "; SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); @@ -816,20 +818,48 @@ public void testPartitionQuery15() throws InterruptedException { log.info("Partition test15"); SiddhiManager siddhiManager = new SiddhiManager(); - String siddhiApp = "@app:name('PartitionTest15') " + - "define stream cseEventStream (symbol string, price float,volume int);" - + "define stream cseEventStream1 (symbol string, price float,volume int);" - + "define stream StockStream (symbol string, price float,volume int);" - + "partition with (symbol of cseEventStream) begin @info(name = 'query') from cseEventStream select " + - "symbol,price as price,volume insert into #StockStream ;" - + "@info(name = 'query1') from #StockStream select symbol,price,volume insert into OutStockStream ;" - + "@info(name = 'query2') from #StockStream select symbol,price,volume insert into StockStream ; end ;" - + "partition with (symbol of cseEventStream1) begin @info(name = 'query3') from cseEventStream1 " + - "select symbol,price+5 as price,volume insert into #StockStream ;" - + "@info(name = 'query4') from #StockStream select symbol,price,volume insert into OutStockStream ; " + - "end ;" - + "@info(name = 'query5') from StockStream select symbol,price+15 as price,volume group by symbol " + - "insert into OutStockStream ;"; + String siddhiApp = "" + + "@app:name('PartitionTest15') " + + "define stream cseEventStream (symbol string, price float,volume int);" + + "define stream cseEventStream1 (symbol string, price float,volume int);" + + "define stream StockStream (symbol string, price float,volume int);" + + "partition with (symbol of cseEventStream) " + + "begin " + + " @info(name = 'query') " + + " from cseEventStream " + + " select symbol,price as price,volume " + + " insert into #StockStream ;" + + " " + + " @info(name = 'query1') " + + " from #StockStream " + + " select symbol,price,volume " + + " insert into OutStockStream ;" + + "" + + " @info(name = 'query2') " + + " from #StockStream " + + " select symbol,price,volume " + + " insert into StockStream ; " + + "end ;" + + "" + + "partition with (symbol of cseEventStream1) " + + "begin " + + " @info(name = 'query3') " + + " from cseEventStream1 " + + " select symbol, price + 5 as price, volume " + + " insert into #StockStream ;" + + "" + + " @info(name = 'query4') " + + " from #StockStream " + + " select symbol,price,volume " + + " insert into OutStockStream ; " + + "end ;" + + "" + + "@info(name = 'query5') " + + "from StockStream " + + "select symbol,price+15 as price,volume " + + "group by symbol " + + "insert into OutStockStream ;" + + ""; SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); @@ -1463,12 +1493,15 @@ public void testPartitionQuery27() throws InterruptedException { "define stream streamB (symbol string, price int); " + "partition with (symbol of streamA, symbol of streamB) " + "begin " + - "@info(name = 'query1') " + - "from streamA " + - "select symbol, price insert into StockQuote ; " + - "@info(name = 'query2') " + - "from streamB " + - "select symbol, price insert into StockQuote ; " + + " @info(name = 'query1') " + + " from streamA " + + " select symbol, price " + + " insert into StockQuote ; " + + "" + + " @info(name = 'query2') " + + " from streamB " + + " select symbol, price " + + " insert into StockQuote ; " + "end "; SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionTestCase2.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionTestCase2.java index 93f819eab4..f5a433642d 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionTestCase2.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/partition/PartitionTestCase2.java @@ -26,6 +26,7 @@ import io.siddhi.core.util.EventPrinter; import io.siddhi.core.util.SiddhiTestHelper; import org.apache.log4j.Logger; +import org.testng.Assert; import org.testng.AssertJUnit; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -673,4 +674,61 @@ public void partitionStreamValidationTest() throws InterruptedException { } } + + @Test + public void testPartitionQuery() throws InterruptedException { + log.info("Partition test1"); + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "@app:name('PartitionTest') " + + "define stream streamA (ts long, symbol string, price int);" + + "partition with (symbol of streamA) " + + "begin " + + " @info(name = 'query1') " + + " from streamA#window.lengthBatch(2, true) " + + " select symbol, sum(price) as total " + + " insert all events into StockQuote ; " + + "end "; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + StreamCallback streamCallback = new StreamCallback() { + @Override + public void receive(Event[] events) { + EventPrinter.print(events); + AssertJUnit.assertTrue("IBM".equals(events[0].getData(0)) || "WSO2".equals(events[0].getData(0))); + int eventCount = count.addAndGet(events.length); + eventArrived = true; + if (eventCount == 1) { + Assert.assertEquals(events[0].getData(1), 700L); + } else if (eventCount == 2) { + Assert.assertEquals(events[0].getData(1), 60L); + } else if (eventCount == 3) { + Assert.assertEquals(events[0].getData(1), 120L); + } else if (eventCount == 4) { + Assert.assertEquals(events[0].getData(1), 60L); + } else if (eventCount == 5) { + Assert.assertEquals(events[0].getData(1), 1400L); + } else if (eventCount == 6) { + Assert.assertEquals(events[0].getData(1), 700L); + } + } + }; + siddhiAppRuntime.addCallback("StockQuote", streamCallback); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("streamA"); + siddhiAppRuntime.start(); + inputHandler.send(new Object[]{100L, "IBM", 700}); + inputHandler.send(new Object[]{101L, "WSO2", 60}); + inputHandler.send(new Object[]{101L, "WSO2", 60}); + inputHandler.send(new Object[]{1134L, "WSO2", 60}); + inputHandler.send(new Object[]{100L, "IBM", 700}); + inputHandler.send(new Object[]{1145L, "IBM", 700}); + SiddhiTestHelper.waitForEvents(100, 6, count, 60000); + AssertJUnit.assertTrue(eventArrived); + AssertJUnit.assertEquals(6, count.get()); + + siddhiAppRuntime.shutdown(); + } + } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/EventOutputRateLimitTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/EventOutputRateLimitTestCase.java index f16104e48b..238e64dba4 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/EventOutputRateLimitTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/EventOutputRateLimitTestCase.java @@ -26,6 +26,7 @@ import io.siddhi.core.stream.input.InputHandler; import io.siddhi.core.util.EventPrinter; import org.apache.log4j.Logger; +import org.testng.Assert; import org.testng.AssertJUnit; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -470,7 +471,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { Thread.sleep(1000); AssertJUnit.assertEquals("Event arrived", true, eventArrived); - AssertJUnit.assertEquals("Number of output event value", 4, count); + AssertJUnit.assertEquals("Number of output event value", 5, count); siddhiAppRuntime.shutdown(); } @@ -584,7 +585,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { Thread.sleep(1000); AssertJUnit.assertEquals("Event arrived", true, eventArrived); - AssertJUnit.assertEquals("Number of output event value", 6, count); + AssertJUnit.assertEquals("Number of output event value", 5, count); siddhiAppRuntime.shutdown(); } @@ -945,4 +946,123 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { siddhiAppRuntime.shutdown(); } + + @Test + public void testEventOutputRateLimitQuery17() throws InterruptedException { + log.info("EventOutputRateLimit test17"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "@app:name('EventOutputRateLimitTest8') " + + "" + + "define stream LoginEvents (timestamp long, ip string);" + + "" + + "@info(name = 'query1') " + + "from LoginEvents " + + "select ip " + + "group by ip " + + "output first every 2 events " + + "insert into uniqueIps ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + log.info("Running : " + siddhiAppRuntime.getName()); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + if (inEvents != null) { + count += inEvents.length; + } else { + AssertJUnit.fail("Remove events emitted"); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("LoginEvents"); + + siddhiAppRuntime.start(); + + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.3"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.9"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.30"}); + Thread.sleep(1000); + + AssertJUnit.assertEquals("Event arrived", true, eventArrived); + AssertJUnit.assertEquals("Number of output event value", 8, count); + siddhiAppRuntime.shutdown(); + } + + @Test + public void testEventOutputRateLimitQuery18() throws InterruptedException { + log.info("EventOutputRateLimit test18"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "@app:name('EventOutputRateLimitTest8') " + + "" + + "define stream LoginEvents (timestamp long, ip string);" + + "" + + "@info(name = 'query1') " + + "from LoginEvents " + + "select ip " + + "output first every 2 events " + + "insert into uniqueIps ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + log.info("Running : " + siddhiAppRuntime.getName()); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + if (inEvents != null) { + count += inEvents.length; + for (Event event : inEvents) { + Assert.assertTrue(event.getData(0).equals("192.10.1.5") || + event.getData(0).equals("192.10.1.4")); + } + } else { + AssertJUnit.fail("Remove events emitted"); + } + eventArrived = true; + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("LoginEvents"); + + siddhiAppRuntime.start(); + + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.3"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.9"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.30"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + Thread.sleep(1000); + + AssertJUnit.assertEquals("Event arrived", true, eventArrived); + AssertJUnit.assertEquals("Number of output event value", 6, count); + siddhiAppRuntime.shutdown(); + } } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/SnapshotOutputRateLimitTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/SnapshotOutputRateLimitTestCase.java index 345a8892b6..ee6850b19a 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/SnapshotOutputRateLimitTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/SnapshotOutputRateLimitTestCase.java @@ -497,7 +497,7 @@ public void receive(Event[] events) { Thread.sleep(1100); inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5", 3}); inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.3", 6}); - Thread.sleep(2200); + Thread.sleep(2300); inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5", 2}); inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.3", 10}); Thread.sleep(7200); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/TimeOutputRateLimitTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/TimeOutputRateLimitTestCase.java index 4e50f3cda7..fcce8c00f9 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/TimeOutputRateLimitTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/ratelimit/TimeOutputRateLimitTestCase.java @@ -106,7 +106,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery1") public void testTimeOutputRateLimitQuery2() throws InterruptedException { log.info("TimeOutputRateLimit test2"); @@ -163,7 +163,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery2") public void testTimeOutputRateLimitQuery3() throws InterruptedException { log.info("TimeOutputRateLimit test3"); @@ -220,7 +220,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery3") public void testTimeOutputRateLimitQuery4() throws InterruptedException { log.info("TimeOutputRateLimit test4"); @@ -279,7 +279,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery4") public void testTimeOutputRateLimitQuery5() throws InterruptedException { log.info("TimeOutputRateLimit test5"); @@ -338,7 +338,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery5") public void testTimeOutputRateLimitQuery6() throws InterruptedException { log.info("TimeOutputRateLimit test6"); @@ -397,7 +397,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery6") public void testTimeOutputRateLimitQuery7() throws InterruptedException { log.info("TimeOutputRateLimit test7"); @@ -456,7 +456,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery7") public void testTimeOutputRateLimitQuery8() throws InterruptedException { log.info("TimeOutputRateLimit test8"); @@ -515,7 +515,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery8") public void testTimeOutputRateLimitQuery9() throws InterruptedException { log.info("TimeOutputRateLimit test9"); @@ -574,7 +574,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery9") public void testTimeOutputRateLimitQuery10() throws InterruptedException { log.info("TimeOutputRateLimit test10"); @@ -633,7 +633,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery10") public void testTimeOutputRateLimitQuery11() throws InterruptedException { log.info("TimeOutputRateLimit test11"); @@ -694,7 +694,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery11") public void testTimeOutputRateLimitQuery12() throws InterruptedException { log.info("TimeOutputRateLimit test12"); @@ -755,7 +755,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery12") public void testTimeOutputRateLimitQuery13() throws InterruptedException { log.info("TimeOutputRateLimit test13"); @@ -816,7 +816,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { } - @Test + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery13") public void testTimeOutputRateLimitQuery14() throws InterruptedException { log.info("TimeOutputRateLimit test14"); @@ -863,7 +863,7 @@ public void receive(Event[] events) { inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4", "WSO2"}); Thread.sleep(1100); inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.30", "WSO2"}); - Thread.sleep(1000); + Thread.sleep(1100); AssertJUnit.assertEquals("Event arrived", true, eventArrived.get()); AssertJUnit.assertTrue("Number of output event value", 3 == count.get()); @@ -872,4 +872,178 @@ public void receive(Event[] events) { } + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery14") + public void testTimeOutputRateLimitQuery15() throws InterruptedException { + log.info("TimeOutputRateLimit test15"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "@app:name('EventOutputRateLimitTest15') " + + "" + + "define stream LoginEvents (timestamp long, ip string);" + + "" + + "@info(name = 'query1') " + + "from LoginEvents " + + "select ip, count(*) as total " + + "output first every 1 sec " + + "insert all events into uniqueIps ;"; + + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + log.info("Running : " + siddhiAppRuntime.getName()); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + if (inEvents != null) { + inEventCount.addAndGet(inEvents.length); + } + if (removeEvents != null) { + removeEventCount.addAndGet(removeEvents.length); + } + eventArrived.set(true); + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("LoginEvents"); + + siddhiAppRuntime.start(); + + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.3"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.9"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + + AssertJUnit.assertEquals("Event arrived", true, eventArrived.get()); + AssertJUnit.assertEquals("Number of output in event value", 1, inEventCount.get()); + AssertJUnit.assertEquals("Number of output remove event value", 0, removeEventCount.get()); + + siddhiAppRuntime.shutdown(); + + } + + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery15") + public void testTimeOutputRateLimitQuery16() throws InterruptedException { + log.info("TimeOutputRateLimit test16"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "@app:name('EventOutputRateLimitTest16') " + + "" + + "define stream LoginEvents (timestamp long, ip string);" + + "" + + "@info(name = 'query1') " + + "from LoginEvents " + + "select ip, count(*) as total " + + "group by ip " + + "output first every 1 sec " + + "insert all events into uniqueIps ;"; + + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + log.info("Running : " + siddhiAppRuntime.getName()); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + if (inEvents != null) { + inEventCount.addAndGet(inEvents.length); + } + if (removeEvents != null) { + removeEventCount.addAndGet(removeEvents.length); + } + eventArrived.set(true); + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("LoginEvents"); + + siddhiAppRuntime.start(); + + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.3"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.9"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + + AssertJUnit.assertEquals("Event arrived", true, eventArrived.get()); + AssertJUnit.assertEquals("Number of output in event value", 4, inEventCount.get()); + AssertJUnit.assertEquals("Number of output remove event value", 0, removeEventCount.get()); + + siddhiAppRuntime.shutdown(); + + } + + @Test(dependsOnMethods = "testTimeOutputRateLimitQuery16") + public void testTimeOutputRateLimitQuery17() throws InterruptedException { + log.info("TimeOutputRateLimit test17"); + + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "@app:name('EventOutputRateLimitTest11') " + + "" + + "define stream LoginEvents (timestamp long, ip string);" + + "" + + "@info(name = 'query1') " + + "from LoginEvents#window.lengthBatch(2) " + + "select ip, count(*) as total " + + "group by ip " + + "output first every 1 sec " + + "insert into uniqueIps ;"; + + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + + log.info("Running : " + siddhiAppRuntime.getName()); + + siddhiAppRuntime.addCallback("query1", new QueryCallback() { + @Override + public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { + EventPrinter.print(timestamp, inEvents, removeEvents); + if (inEvents != null) { + inEventCount.addAndGet(inEvents.length); + } + if (removeEvents != null) { + removeEventCount.addAndGet(removeEvents.length); + } + eventArrived.set(true); + } + + }); + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("LoginEvents"); + + siddhiAppRuntime.start(); + + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.3"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.9"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + Thread.sleep(1100); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.4"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.5"}); + inputHandler.send(new Object[]{System.currentTimeMillis(), "192.10.1.30"}); + Thread.sleep(1100); + + AssertJUnit.assertEquals("Event arrived", true, eventArrived.get()); + AssertJUnit.assertEquals("Number of output in event value", 6, inEventCount.get()); + AssertJUnit.assertEquals("Number of output remove event value", 0, removeEventCount.get()); + + siddhiAppRuntime.shutdown(); + + } + + + } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregatorTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregatorExecutorTestCase.java similarity index 97% rename from modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregatorTestCase.java rename to modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregatorExecutorTestCase.java index c5b7ccdaf0..787240e12e 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregatorTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/DistinctCountAttributeAggregatorExecutorTestCase.java @@ -28,8 +28,8 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -public class DistinctCountAttributeAggregatorTestCase { - private static final Logger log = Logger.getLogger(DistinctCountAttributeAggregatorTestCase.class); +public class DistinctCountAttributeAggregatorExecutorTestCase { + private static final Logger log = Logger.getLogger(DistinctCountAttributeAggregatorExecutorTestCase.class); private volatile int count; @BeforeMethod diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregatorTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregatorExecutorTestCase.java similarity index 99% rename from modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregatorTestCase.java rename to modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregatorExecutorTestCase.java index 18b2d803c5..5027ece817 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregatorTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/selector/attribute/aggregator/StdDevAttributeAggregatorExecutorTestCase.java @@ -29,8 +29,8 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -public class StdDevAttributeAggregatorTestCase { - private static final Logger log = Logger.getLogger(StdDevAttributeAggregatorTestCase.class); +public class StdDevAttributeAggregatorExecutorTestCase { + private static final Logger log = Logger.getLogger(StdDevAttributeAggregatorExecutorTestCase.class); private final double epsilon = 0.00001; // difference threshold for two doubles to be treated distinct private int inEventCount; // Only used in the Test #1 and #6 diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/streamfunction/AttributeStreamFunction.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/streamfunction/AttributeStreamFunction.java index 027ac614cc..3be31cf8b1 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/streamfunction/AttributeStreamFunction.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/streamfunction/AttributeStreamFunction.java @@ -24,14 +24,16 @@ import io.siddhi.core.executor.ExpressionExecutor; import io.siddhi.core.query.processor.stream.function.StreamFunctionProcessor; import io.siddhi.core.util.config.ConfigReader; +import io.siddhi.core.util.snapshot.state.StateFactory; import io.siddhi.query.api.definition.AbstractDefinition; import io.siddhi.query.api.definition.Attribute; import java.util.ArrayList; import java.util.List; -import java.util.Map; public class AttributeStreamFunction extends StreamFunctionProcessor { + private List newAttributes; + @Override protected Object[] process(Object[] data) { return new Object[]{"test"}; @@ -43,9 +45,9 @@ protected Object[] process(Object data) { } @Override - protected List init(AbstractDefinition inputDefinition, - ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(AbstractDefinition inputDefinition, + ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + boolean outputExpectsExpiredEvents, SiddhiQueryContext siddhiQueryContext) { if (attributeExpressionExecutors.length != 1) { throw new SiddhiAppCreationException("Only one attribute is expected but found " + attributeExpressionExecutors.length); @@ -53,11 +55,11 @@ protected List init(AbstractDefinition inputDefinition, if (!(attributeExpressionExecutors[0] instanceof ConstantExpressionExecutor)) { throw new SiddhiAppCreationException("Attribute is expected to be constant, but its not!"); } - List newAttributes = new ArrayList<>(); + newAttributes = new ArrayList<>(); newAttributes.add( new Attribute(((ConstantExpressionExecutor) attributeExpressionExecutors[0]).getValue().toString(), inputDefinition.getAttributeList().get(0).getType())); - return newAttributes; + return null; } @Override @@ -71,12 +73,7 @@ public void stop() { } @Override - public Map currentState() { - return null; - } - - @Override - public void restoreState(Map state) { - + public List getReturnAttributes() { + return newAttributes; } } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/TimeBatchWindowTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/TimeBatchWindowTestCase.java index 9572519ff4..9831e0be68 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/TimeBatchWindowTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/TimeBatchWindowTestCase.java @@ -362,7 +362,6 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { siddhiAppRuntime.start(); // Start sending events in the beginning of a cycle while (System.currentTimeMillis() % 2000 != 0) { - ; } inputHandler.send(new Object[]{"IBM", 700f, 0}); inputHandler.send(new Object[]{"WSO2", 60.5f, 1}); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/TimeLengthWindowTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/TimeLengthWindowTestCase.java index 2d6f01280c..10d2937369 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/TimeLengthWindowTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/query/window/TimeLengthWindowTestCase.java @@ -273,9 +273,11 @@ public void timeLengthWindowTest6() throws InterruptedException { SiddhiManager siddhiManager = new SiddhiManager(); String sensorStream = "define stream sensorStream (id string, sensorValue int);"; - String query = "@info(name = 'query1') from sensorStream#window.timeLength(3 sec,6)" + - " select id,sum(sensorValue) as sum" + - " insert all events into outputStream ;"; + String query = "" + + "@info(name = 'query1') " + + "from sensorStream#window.timeLength(3 sec, 6) " + + "select id, sum(sensorValue) as sum " + + "insert all events into outputStream ;"; SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(sensorStream + query); siddhiAppRuntime.addCallback("query1", new QueryCallback() { @Override @@ -303,7 +305,7 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { AssertJUnit.assertEquals("5", removeEvents[0].getData(1).toString()); } if (removeEvents[0].getData(0).toString().equals("id3")) { - AssertJUnit.assertEquals("3", removeEvents[0].getData(1).toString()); + AssertJUnit.assertEquals("5", removeEvents[0].getData(1).toString()); } removeEventCount++; } @@ -329,10 +331,10 @@ public void receive(long timestamp, Event[] inEvents, Event[] removeEvents) { Thread.sleep(520); inputHandler.send(new Object[]{"id8", 1}); - Thread.sleep(1000); + Thread.sleep(500); AssertJUnit.assertEquals(8, inEventCount); - AssertJUnit.assertEquals(2, removeEventCount); + AssertJUnit.assertEquals(3, removeEventCount); AssertJUnit.assertTrue(eventArrived); siddhiAppRuntime.shutdown(); } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/stream/FaultFunctionExtension.java b/modules/siddhi-core/src/test/java/io/siddhi/core/stream/FaultFunctionExtension.java index e1f68a208f..4d9ee0cd6c 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/stream/FaultFunctionExtension.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/stream/FaultFunctionExtension.java @@ -25,10 +25,10 @@ import io.siddhi.core.executor.ExpressionExecutor; 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 java.util.Map; - @Extension( name = "fault", namespace = "custom", @@ -50,10 +50,11 @@ public class FaultFunctionExtension extends FunctionExecutor { private Attribute.Type returnType; @Override - public void init(ExpressionExecutor[] attributeExpressionExecutors, - ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + public StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, + ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { returnType = Attribute.Type.DOUBLE; + return null; } /** @@ -68,24 +69,14 @@ public Attribute.Type getReturnType() { @Override - protected Object execute(Object[] obj) { + protected Object execute(Object[] obj, State state) { throw new RuntimeException("Error when running faultAdd()"); } @Override - protected Object execute(Object obj) { + protected Object execute(Object obj, State state) { throw new RuntimeException("Error when running faultAdd()"); } - @Override - public Map currentState() { - //No state - return null; - } - - @Override - public void restoreState(Map state) { - //Nothing to be done - } } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/stream/JunctionTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/stream/JunctionTestCase.java index 88f34740e4..00a92d28bf 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/stream/JunctionTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/stream/JunctionTestCase.java @@ -21,7 +21,6 @@ import io.siddhi.core.config.SiddhiContext; import io.siddhi.core.event.Event; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; import io.siddhi.core.stream.output.StreamCallback; import io.siddhi.query.api.annotation.Annotation; import io.siddhi.query.api.definition.Attribute; @@ -424,186 +423,4 @@ public void receive(Event[] streamEvents) { streamJunctionB.stopProcessing(); streamJunctionC.stopProcessing(); } - - - @Test - public void multiThreadedWithEventPoolTest() throws InterruptedException { - log.info("multi threaded test using event pool"); - - final StreamEventPool streamEventPoolA1 = new StreamEventPool(2, 2, 2, 4); - final StreamEventPool streamEventPoolA2 = new StreamEventPool(2, 2, 2, 4); - final StreamEventPool streamEventPoolA3 = new StreamEventPool(2, 2, 2, 4); - final StreamEventPool streamEventPoolB1 = new StreamEventPool(2, 2, 2, 4); - final StreamEventPool streamEventPoolB2 = new StreamEventPool(2, 2, 2, 4); - - - StreamDefinition streamA = StreamDefinition.id("streamA").attribute("symbol", Attribute.Type.STRING) - .attribute("price", Attribute.Type.INT). - annotation(Annotation.annotation("async")); - StreamJunction streamJunctionA = new StreamJunction(streamA, executorService, 1024, - null, siddhiAppContext); - StreamJunction.Publisher streamPublisherA = streamJunctionA.constructPublisher(); - - StreamDefinition streamB = StreamDefinition.id("streamB").attribute("symbol", Attribute.Type.STRING) - .attribute("price", Attribute.Type.INT). - annotation(Annotation.annotation("async")); - StreamJunction streamJunctionB = new StreamJunction(streamB, executorService, 1024, - null, siddhiAppContext); - final StreamJunction.Publisher streamPublisherB1 = streamJunctionB.constructPublisher(); - final StreamJunction.Publisher streamPublisherB2 = streamJunctionB.constructPublisher(); - final StreamJunction.Publisher streamPublisherB3 = streamJunctionB.constructPublisher(); - - StreamDefinition streamC = StreamDefinition.id("streamC").attribute("symbol", Attribute.Type.STRING) - .attribute("price", Attribute.Type.INT). - annotation(Annotation.annotation("async")); - StreamJunction streamJunctionC = new StreamJunction(streamC, executorService, 1024, - null, siddhiAppContext); - final StreamJunction.Publisher streamPublisherC1 = streamJunctionC.constructPublisher(); - final StreamJunction.Publisher streamPublisherC2 = streamJunctionC.constructPublisher(); - - StreamCallback streamCallbackA1 = new StreamCallback() { - @Override - public void receive(Event[] streamEvents) { - for (Event streamEvent : streamEvents) { - StreamEvent innerStreamEvent = streamEventPoolA1.borrowEvent(); - innerStreamEvent.setTimestamp(streamEvent.getTimestamp()); - Object[] data = new Object[]{streamEvent.getData()[0], streamEvent.getData()[1]}; - data[0] = ((String) data[0]).concat("A1"); - innerStreamEvent.setOutputData(data); - streamPublisherB1.send(innerStreamEvent); - } - } - }; - - StreamCallback streamCallbackA2 = new StreamCallback() { - @Override - public void receive(Event[] streamEvents) { - for (Event streamEvent : streamEvents) { - StreamEvent innerStreamEvent = streamEventPoolA2.borrowEvent(); - innerStreamEvent.setTimestamp(streamEvent.getTimestamp()); - Object[] data = new Object[]{streamEvent.getData()[0], streamEvent.getData()[1]}; - data[0] = ((String) data[0]).concat("A2"); - innerStreamEvent.setOutputData(data); - streamPublisherB2.send(innerStreamEvent); - } - } - }; - - StreamCallback streamCallbackA3 = new StreamCallback() { - @Override - public void receive(Event[] streamEvents) { - for (Event streamEvent : streamEvents) { - StreamEvent innerStreamEvent = streamEventPoolA3.borrowEvent(); - innerStreamEvent.setTimestamp(streamEvent.getTimestamp()); - Object[] data = new Object[]{streamEvent.getData()[0], streamEvent.getData()[1]}; - data[0] = ((String) data[0]).concat("A3"); - innerStreamEvent.setOutputData(data); - streamPublisherB3.send(innerStreamEvent); - } - } - }; - - - StreamCallback streamCallbackB1 = new StreamCallback() { - @Override - public void receive(Event[] streamEvents) { - for (Event streamEvent : streamEvents) { - StreamEvent innerStreamEvent = streamEventPoolB1.borrowEvent(); - innerStreamEvent.setTimestamp(streamEvent.getTimestamp()); - Object[] data = new Object[]{streamEvent.getData()[0], streamEvent.getData()[1]}; - data[0] = ((String) data[0]).concat("B1"); - innerStreamEvent.setOutputData(data); - streamPublisherC1.send(innerStreamEvent); - } - } - }; - - StreamCallback streamCallbackB2 = new StreamCallback() { - @Override - public void receive(Event[] streamEvents) { - for (Event streamEvent : streamEvents) { - StreamEvent innerStreamEvent = streamEventPoolB2.borrowEvent(); - innerStreamEvent.setTimestamp(streamEvent.getTimestamp()); - Object[] data = new Object[]{streamEvent.getData()[0], streamEvent.getData()[1]}; - data[0] = ((String) data[0]).concat("B2"); - innerStreamEvent.setOutputData(data); - streamPublisherC2.send(innerStreamEvent); - } - } - }; - - - final boolean[] eventsArrived = {false, false, false, false, false, false, false, false, false, false, false, - false}; - - - StreamCallback streamCallbackC = new StreamCallback() { - @Override - public void receive(Event[] streamEvents) { - for (Event streamEvent : streamEvents) { - count++; - eventArrived = true; - Object symbol = streamEvent.getData()[0]; - if (symbol.equals("IBMA1B1")) { - eventsArrived[0] = true; - } else if (symbol.equals("IBMA1B2")) { - eventsArrived[1] = true; - } else if (symbol.equals("IBMA2B1")) { - eventsArrived[2] = true; - } else if (symbol.equals("IBMA2B2")) { - eventsArrived[3] = true; - } else if (symbol.equals("IBMA3B1")) { - eventsArrived[4] = true; - } else if (symbol.equals("IBMA3B2")) { - eventsArrived[5] = true; - } - if (symbol.equals("WSO2A1B1")) { - eventsArrived[6] = true; - } else if (symbol.equals("WSO2A1B2")) { - eventsArrived[7] = true; - } else if (symbol.equals("WSO2A2B1")) { - eventsArrived[8] = true; - } else if (symbol.equals("WSO2A2B2")) { - eventsArrived[9] = true; - } else if (symbol.equals("WSO2A3B1")) { - eventsArrived[10] = true; - } else if (symbol.equals("WSO2A3B2")) { - eventsArrived[11] = true; - } - } - } - }; - - streamJunctionA.subscribe(streamCallbackA1); - streamJunctionA.subscribe(streamCallbackA2); - streamJunctionA.subscribe(streamCallbackA3); - streamJunctionA.startProcessing(); - - streamJunctionB.subscribe(streamCallbackB1); - streamJunctionB.subscribe(streamCallbackB2); - streamJunctionB.startProcessing(); - - streamJunctionC.subscribe(streamCallbackC); - streamJunctionC.startProcessing(); - - StreamEvent streamEvent1 = new StreamEvent(2, 2, 2); - streamEvent1.setTimestamp(System.currentTimeMillis()); - streamEvent1.setOutputData(new Object[]{"IBM", 12}); - - StreamEvent streamEvent2 = new StreamEvent(2, 2, 2); - streamEvent2.setTimestamp(System.currentTimeMillis()); - streamEvent2.setOutputData(new Object[]{"WSO2", 112}); - - streamPublisherA.send(streamEvent1); - streamPublisherA.send(streamEvent2); - Thread.sleep(1000); - AssertJUnit.assertTrue(eventArrived); - AssertJUnit.assertEquals(12, count); - for (boolean arrived : eventsArrived) { - AssertJUnit.assertTrue(arrived); - } - streamJunctionA.stopProcessing(); - streamJunctionB.stopProcessing(); - streamJunctionC.stopProcessing(); - } } diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/stream/event/ComplexEventChunkTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/stream/event/ComplexEventChunkTestCase.java index fdb71f95a5..9248152ed8 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/stream/event/ComplexEventChunkTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/stream/event/ComplexEventChunkTestCase.java @@ -19,7 +19,7 @@ package io.siddhi.core.stream.event; import io.siddhi.core.event.stream.StreamEvent; -import io.siddhi.core.event.stream.StreamEventPool; +import io.siddhi.core.event.stream.StreamEventFactory; import io.siddhi.core.event.stream.converter.ConversionStreamEventChunk; import io.siddhi.core.event.stream.converter.StreamEventConverter; import io.siddhi.core.event.stream.converter.ZeroStreamEventConverter; @@ -53,9 +53,9 @@ public void eventChunkTest() { streamEvent1.setNext(streamEvent2); streamEvent2.setNext(streamEvent3); - StreamEventPool streamEventPool = new StreamEventPool(0, 0, 3, 5); + StreamEventFactory streamEventFactory = new StreamEventFactory(0, 0, 3); ConversionStreamEventChunk streamEventChunk = new ConversionStreamEventChunk(streamEventConverter, - streamEventPool); + streamEventFactory); streamEventChunk.convertAndAssign(streamEvent1); while (streamEventChunk.hasNext()) { @@ -80,9 +80,9 @@ public void eventChunkRemoveTest1() { streamEvent1.setNext(streamEvent2); streamEvent2.setNext(streamEvent3); - StreamEventPool streamEventPool = new StreamEventPool(0, 0, 3, 5); + StreamEventFactory streamEventFactory = new StreamEventFactory(0, 0, 3); ConversionStreamEventChunk streamEventChunk = new ConversionStreamEventChunk(streamEventConverter, - streamEventPool); + streamEventFactory); streamEventChunk.convertAndAssign(streamEvent1); while (streamEventChunk.hasNext()) { @@ -113,9 +113,9 @@ public void eventChunkRemoveTest2() { streamEvent2.setNext(streamEvent3); streamEvent3.setNext(streamEvent4); - StreamEventPool streamEventPool = new StreamEventPool(0, 0, 3, 5); + StreamEventFactory streamEventFactory = new StreamEventFactory(0, 0, 3); ConversionStreamEventChunk streamEventChunk = new ConversionStreamEventChunk(streamEventConverter, - streamEventPool); + streamEventFactory); streamEventChunk.convertAndAssign(streamEvent1); while (streamEventChunk.hasNext()) { @@ -148,9 +148,9 @@ public void eventChunkRemoveTest3() { streamEvent2.setNext(streamEvent3); streamEvent3.setNext(streamEvent4); - StreamEventPool streamEventPool = new StreamEventPool(0, 0, 3, 5); + StreamEventFactory streamEventFactory = new StreamEventFactory(0, 0, 3); ConversionStreamEventChunk streamEventChunk = new ConversionStreamEventChunk(streamEventConverter, - streamEventPool); + streamEventFactory); streamEventChunk.convertAndAssign(streamEvent1); while (streamEventChunk.hasNext()) { @@ -179,9 +179,9 @@ public void eventChunkRemoveTest4() { streamEvent2.setNext(streamEvent3); streamEvent3.setNext(streamEvent4); - StreamEventPool streamEventPool = new StreamEventPool(0, 0, 3, 5); + StreamEventFactory streamEventFactory = new StreamEventFactory(0, 0, 3); ConversionStreamEventChunk streamEventChunk = new ConversionStreamEventChunk(streamEventConverter, - streamEventPool); + streamEventFactory); streamEventChunk.convertAndAssign(streamEvent1); while (streamEventChunk.hasNext()) { @@ -207,9 +207,9 @@ public void eventChunkRemoveTest5() { streamEvent1.setNext(streamEvent2); - StreamEventPool streamEventPool = new StreamEventPool(0, 0, 3, 5); + StreamEventFactory streamEventFactory = new StreamEventFactory(0, 0, 3); ConversionStreamEventChunk streamEventChunk = new ConversionStreamEventChunk(streamEventConverter, - streamEventPool); + streamEventFactory); streamEventChunk.convertAndAssign(streamEvent1); streamEventChunk.remove(); 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 76e99d2a02..b4ae47cfa8 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 @@ -25,7 +25,6 @@ import io.siddhi.core.event.stream.MetaStreamEvent; import io.siddhi.core.event.stream.StreamEvent; import io.siddhi.core.event.stream.StreamEventFactory; -import io.siddhi.core.event.stream.StreamEventPool; import io.siddhi.core.event.stream.converter.SelectiveStreamEventConverter; import io.siddhi.core.event.stream.converter.SimpleStreamEventConverter; import io.siddhi.core.event.stream.converter.StreamEventConverter; @@ -44,7 +43,7 @@ import io.siddhi.core.stream.input.source.Source; import io.siddhi.core.stream.output.sink.Sink; import io.siddhi.core.table.Table; -import io.siddhi.core.util.ElementIdGenerator; +import io.siddhi.core.util.IdGenerator; import io.siddhi.core.util.SiddhiConstants; import io.siddhi.core.util.event.handler.EventExchangeHolderFactory; import io.siddhi.core.util.lock.LockSynchronizer; @@ -83,26 +82,6 @@ public void testEventCreation() { AssertJUnit.assertEquals(4, streamEvent.getOutputData().length); } - @Test - public void testEventPool() { - StreamEventPool streamEventPool = new StreamEventPool(2, 3, 1, 4); - - StreamEvent[] streamEvents = new StreamEvent[5]; - for (int i = 0; i < 5; i++) { - streamEvents[i] = streamEventPool.borrowEvent(); - } - AssertJUnit.assertEquals(0, streamEventPool.getBufferedEventsSize()); - - for (int i = 0; i < 5; i++) { - streamEventPool.returnEvents(streamEvents[i]); - } - AssertJUnit.assertEquals(4, streamEventPool.getBufferedEventsSize()); - - streamEventPool.borrowEvent(); - AssertJUnit.assertEquals(3, streamEventPool.getBufferedEventsSize()); - - } - @Test public void testPassThroughStreamEventConverter() { Attribute symbol = new Attribute("symbol", Attribute.Type.STRING); @@ -122,17 +101,17 @@ public void testPassThroughStreamEventConverter() { metaStreamEvent.addInputDefinition(streamDefinition); StreamEventConverter converter = StreamEventConverterFactory.constructEventConverter(metaStreamEvent); - StreamEventPool eventPool = new StreamEventPool(metaStreamEvent, 5); + StreamEventFactory eventPool = new StreamEventFactory(metaStreamEvent); - StreamEvent borrowedEvent = eventPool.borrowEvent(); - converter.convertEvent(event, borrowedEvent); + StreamEvent newEvent = eventPool.newInstance(); + converter.convertEvent(event, newEvent); AssertJUnit.assertTrue(converter instanceof ZeroStreamEventConverter); - AssertJUnit.assertEquals(3, borrowedEvent.getOutputData().length); + AssertJUnit.assertEquals(3, newEvent.getOutputData().length); - AssertJUnit.assertEquals("WSO2", borrowedEvent.getOutputData()[0]); - AssertJUnit.assertEquals(200.0, borrowedEvent.getOutputData()[1]); - AssertJUnit.assertEquals(50, borrowedEvent.getOutputData()[2]); + AssertJUnit.assertEquals("WSO2", newEvent.getOutputData()[0]); + AssertJUnit.assertEquals(200.0, newEvent.getOutputData()[1]); + AssertJUnit.assertEquals(50, newEvent.getOutputData()[2]); } @Test @@ -150,17 +129,17 @@ public void testSimpleStreamEventConverter() { metaStreamEvent.addInputDefinition(streamDefinition); StreamEventConverter converter = StreamEventConverterFactory.constructEventConverter(metaStreamEvent); - StreamEventPool eventPool = new StreamEventPool(metaStreamEvent, 5); - StreamEvent borrowedEvent = eventPool.borrowEvent(); - converter.convertEvent(event, borrowedEvent); + StreamEventFactory eventPool = new StreamEventFactory(metaStreamEvent); + StreamEvent newEvent = eventPool.newInstance(); + converter.convertEvent(event, newEvent); AssertJUnit.assertTrue(converter instanceof SimpleStreamEventConverter); - AssertJUnit.assertNull(borrowedEvent.getBeforeWindowData()); - AssertJUnit.assertNull(borrowedEvent.getOnAfterWindowData()); - AssertJUnit.assertEquals(2, borrowedEvent.getOutputData().length); + AssertJUnit.assertNull(newEvent.getBeforeWindowData()); + AssertJUnit.assertNull(newEvent.getOnAfterWindowData()); + AssertJUnit.assertEquals(2, newEvent.getOutputData().length); - AssertJUnit.assertEquals(200, borrowedEvent.getOutputData()[1]); - AssertJUnit.assertEquals("WSO2", borrowedEvent.getOutputData()[0]); + AssertJUnit.assertEquals(200, newEvent.getOutputData()[1]); + AssertJUnit.assertEquals("WSO2", newEvent.getOutputData()[0]); } @Test @@ -182,19 +161,19 @@ public void testStreamEventConverter() { metaStreamEvent.addInputDefinition(streamDefinition); StreamEventConverter converter = StreamEventConverterFactory.constructEventConverter(metaStreamEvent); - StreamEventPool eventPool = new StreamEventPool(metaStreamEvent, 5); + StreamEventFactory eventPool = new StreamEventFactory(metaStreamEvent); - StreamEvent borrowedEvent = eventPool.borrowEvent(); - converter.convertEvent(event, borrowedEvent); + StreamEvent newEvent = eventPool.newInstance(); + converter.convertEvent(event, newEvent); AssertJUnit.assertTrue(converter instanceof SelectiveStreamEventConverter); - AssertJUnit.assertEquals(1, borrowedEvent.getBeforeWindowData().length); //volume - AssertJUnit.assertEquals(1, borrowedEvent.getOnAfterWindowData().length); //price - AssertJUnit.assertEquals(2, borrowedEvent.getOutputData().length); //symbol and avgPrice + AssertJUnit.assertEquals(1, newEvent.getBeforeWindowData().length); //volume + AssertJUnit.assertEquals(1, newEvent.getOnAfterWindowData().length); //price + AssertJUnit.assertEquals(2, newEvent.getOutputData().length); //symbol and avgPrice - AssertJUnit.assertEquals(50, borrowedEvent.getBeforeWindowData()[0]); - AssertJUnit.assertEquals(200, borrowedEvent.getOnAfterWindowData()[0]); - AssertJUnit.assertEquals("WSO2", borrowedEvent.getOutputData()[0]); + AssertJUnit.assertEquals(50, newEvent.getBeforeWindowData()[0]); + AssertJUnit.assertEquals(200, newEvent.getOnAfterWindowData()[0]); + AssertJUnit.assertEquals("WSO2", newEvent.getOutputData()[0]); } @Test @@ -296,11 +275,11 @@ public void testQueryParser() { SiddhiContext siddhicontext = new SiddhiContext(); SiddhiAppContext context = new SiddhiAppContext(); context.setSiddhiContext(siddhicontext); - context.setElementIdGenerator(new ElementIdGenerator(context.getName())); + context.setIdGenerator(new IdGenerator()); context.setSnapshotService(new SnapshotService(context)); QueryRuntime runtime = QueryParser.parse(query, context, streamDefinitionMap, tableDefinitionMap, windowDefinitionMap, aggregationDefinitionMap, tableMap, aggregationMap, eventWindowMap, - lockSynchronizer, "1"); + lockSynchronizer, "1", false, SiddhiConstants.PARTITION_ID_DEFAULT); AssertJUnit.assertNotNull(runtime); AssertJUnit.assertTrue(runtime.getStreamRuntime() instanceof SingleStreamRuntime); AssertJUnit.assertNotNull(runtime.getSelector()); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/window/DelayWindowTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/window/DelayWindowTestCase.java index cc83264a32..15e70cd035 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/window/DelayWindowTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/window/DelayWindowTestCase.java @@ -374,6 +374,7 @@ public void delayWindowTest7() throws InterruptedException { siddhiAppRuntime.addCallback("outputStream", new StreamCallback() { @Override public void receive(Event[] events) { + EventPrinter.print(events); count.addAndGet(events.length); for (Event event : events) { AssertJUnit.assertTrue(("IBM".equals(event.getData(0)) || "WSO2".equals(event.getData(0)))); diff --git a/modules/siddhi-core/src/test/java/io/siddhi/core/window/SessionWindowTestCase.java b/modules/siddhi-core/src/test/java/io/siddhi/core/window/SessionWindowTestCase.java index 6046643336..66822036c7 100644 --- a/modules/siddhi-core/src/test/java/io/siddhi/core/window/SessionWindowTestCase.java +++ b/modules/siddhi-core/src/test/java/io/siddhi/core/window/SessionWindowTestCase.java @@ -119,8 +119,8 @@ public void testSessionWindow2() { } catch (SiddhiAppCreationException e) { AssertJUnit.assertEquals("Session window's 1st parameter, session gap should be a constant parameter" - + " attribute but found a dynamic attribute " - + "VariableExpressionExecutor", e.getCause().getMessage()); + + " attribute but found a dynamic attribute " + + "io.siddhi.core.executor.VariableExpressionExecutor", e.getCause().getMessage()); throw e; } finally { if (siddhiAppRuntime != null) { @@ -183,9 +183,9 @@ public void testSessionWindow4() { siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(purchaseEventStream + query); } catch (SiddhiAppCreationException e) { - AssertJUnit.assertEquals("Session window's 2nd parameter, session key should be a dynamic parameter" - + " attribute but found a constant attribute " - + "ConstantExpressionExecutor", e.getCause().getMessage()); + AssertJUnit.assertEquals("Session window's 2nd parameter, session key should be a " + + "dynamic parameter attribute but found a constant attribute " + + "io.siddhi.core.executor.ConstantExpressionExecutor", e.getCause().getMessage()); throw e; } finally { if (siddhiAppRuntime != null) { @@ -250,7 +250,7 @@ public void testSessionWindow6() { } catch (SiddhiAppCreationException e) { AssertJUnit.assertEquals("Session window's 3rd parameter, allowedLatency should be a " + "constant parameter attribute but found a dynamic attribute " - + "VariableExpressionExecutor", e.getCause().getMessage()); + + "io.siddhi.core.executor.VariableExpressionExecutor", e.getCause().getMessage()); throw e; } finally { if (siddhiAppRuntime != null) { diff --git a/modules/siddhi-core/src/test/resources/testng.xml b/modules/siddhi-core/src/test/resources/testng.xml index 8e1d17277a..fc7cec5b5a 100644 --- a/modules/siddhi-core/src/test/resources/testng.xml +++ b/modules/siddhi-core/src/test/resources/testng.xml @@ -73,8 +73,8 @@ - - + + @@ -123,6 +123,8 @@ + + diff --git a/modules/siddhi-doc-gen/src/main/java/io/siddhi/doc/gen/metadata/ExtensionType.java b/modules/siddhi-doc-gen/src/main/java/io/siddhi/doc/gen/metadata/ExtensionType.java index cd7e551203..f8af798e25 100644 --- a/modules/siddhi-doc-gen/src/main/java/io/siddhi/doc/gen/metadata/ExtensionType.java +++ b/modules/siddhi-doc-gen/src/main/java/io/siddhi/doc/gen/metadata/ExtensionType.java @@ -22,7 +22,7 @@ import io.siddhi.core.query.processor.stream.StreamProcessor; import io.siddhi.core.query.processor.stream.function.StreamFunctionProcessor; import io.siddhi.core.query.processor.stream.window.WindowProcessor; -import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregator; +import io.siddhi.core.query.selector.attribute.aggregator.AttributeAggregatorExecutor; import io.siddhi.core.stream.input.source.Source; import io.siddhi.core.stream.input.source.SourceMapper; import io.siddhi.core.stream.output.sink.Sink; @@ -65,7 +65,7 @@ public enum ExtensionType { // Populating the processor super class map superClassMap = new HashMap<>(); superClassMap.put(ExtensionType.FUNCTION, FunctionExecutor.class); - superClassMap.put(ExtensionType.ATTRIBUTE_AGGREGATOR, AttributeAggregator.class); + superClassMap.put(ExtensionType.ATTRIBUTE_AGGREGATOR, AttributeAggregatorExecutor.class); superClassMap.put(ExtensionType.WINDOW, WindowProcessor.class); superClassMap.put(ExtensionType.STREAM_FUNCTION, StreamFunctionProcessor.class); superClassMap.put(ExtensionType.STREAM_PROCESSOR, StreamProcessor.class); diff --git a/modules/siddhi-query-api/src/main/java/io/siddhi/query/api/execution/query/input/stream/StateInputStream.java b/modules/siddhi-query-api/src/main/java/io/siddhi/query/api/execution/query/input/stream/StateInputStream.java index c9c2d2e1e5..68e8b056b7 100644 --- a/modules/siddhi-query-api/src/main/java/io/siddhi/query/api/execution/query/input/stream/StateInputStream.java +++ b/modules/siddhi-query-api/src/main/java/io/siddhi/query/api/execution/query/input/stream/StateInputStream.java @@ -17,6 +17,7 @@ */ package io.siddhi.query.api.execution.query.input.stream; +import io.siddhi.query.api.execution.query.input.handler.StreamHandler; import io.siddhi.query.api.execution.query.input.state.CountStateElement; import io.siddhi.query.api.execution.query.input.state.EveryStateElement; import io.siddhi.query.api.execution.query.input.state.LogicalStateElement; @@ -34,6 +35,7 @@ public class StateInputStream extends InputStream { private static final long serialVersionUID = 1L; + private final List streamHandlers; private Type stateType; private StateElement stateElement; private List streamIdList; @@ -42,7 +44,8 @@ public class StateInputStream extends InputStream { public StateInputStream(Type stateType, StateElement stateElement, TimeConstant withinTime) { this.stateType = stateType; this.stateElement = stateElement; - this.streamIdList = collectStreamIds(stateElement, new ArrayList()); + this.streamIdList = collectStreamIds(stateElement, new ArrayList<>()); + this.streamHandlers = collectStreamHanders(stateElement, new ArrayList<>()); this.withinTime = withinTime; } @@ -59,6 +62,10 @@ public List getAllStreamIds() { return streamIdList; } + public List getStreamHandlers() { + return streamHandlers; + } + @Override public List getUniqueStreamIds() { List uniqueStreams = new ArrayList(); @@ -90,6 +97,26 @@ private List collectStreamIds(StateElement stateElement, return streamIds; } + private List collectStreamHanders(StateElement stateElement, + List streamHandlers) { + if (stateElement instanceof LogicalStateElement) { + collectStreamHanders(((LogicalStateElement) stateElement).getStreamStateElement1(), streamHandlers); + collectStreamHanders(((LogicalStateElement) stateElement).getStreamStateElement2(), streamHandlers); + } else if (stateElement instanceof CountStateElement) { + collectStreamHanders(((CountStateElement) stateElement).getStreamStateElement(), streamHandlers); + } else if (stateElement instanceof EveryStateElement) { + collectStreamHanders(((EveryStateElement) stateElement).getStateElement(), streamHandlers); + } else if (stateElement instanceof NextStateElement) { + collectStreamHanders(((NextStateElement) stateElement).getStateElement(), streamHandlers); + collectStreamHanders(((NextStateElement) stateElement).getNextStateElement(), streamHandlers); + } else if (stateElement instanceof StreamStateElement) { + BasicSingleInputStream basicSingleInputStream = ((StreamStateElement) stateElement) + .getBasicSingleInputStream(); + streamHandlers.addAll(basicSingleInputStream.getStreamHandlers()); + } + return streamHandlers; + } + public int getStreamCount(String streamId) { int count = 0; for (String aStreamId : streamIdList) { diff --git a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/GroupByWindowSingleQueryPerformance.java b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/GroupByWindowSingleQueryPerformance.java new file mode 100644 index 0000000000..60752023bf --- /dev/null +++ b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/GroupByWindowSingleQueryPerformance.java @@ -0,0 +1,78 @@ +/* + * 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.performance; + +import io.siddhi.core.SiddhiAppRuntime; +import io.siddhi.core.SiddhiManager; +import io.siddhi.core.event.Event; +import io.siddhi.core.stream.input.InputHandler; +import io.siddhi.core.stream.output.StreamCallback; + +public class GroupByWindowSingleQueryPerformance { + + public static void main(String[] args) throws InterruptedException { + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "define stream cseEventStream (symbol string, price float, volume int, timestamp long);" + + "" + + "@info(name = 'query1') " + + "from cseEventStream#window.lengthBatch(10) " + + "select symbol, sum(price) as total, avg(volume) as avgVolume, timestamp " + + "group by symbol " + + "insert into outputStream ;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + siddhiAppRuntime.addCallback("outputStream", new StreamCallback() { + public int eventCount = 0; + public int timeSpent = 0; + long startTime = System.currentTimeMillis(); + + @Override + public void receive(Event[] events) { + for (Event event : events) { + eventCount++; + timeSpent += (System.currentTimeMillis() - (Long) event.getData(3)); + if (eventCount % 10000000 == 0) { + System.out.println("Throughput : " + (eventCount * 1000) / ((System.currentTimeMillis()) - + startTime)); + System.out.println("Time spent : " + (timeSpent * 1.0 / eventCount)); + startTime = System.currentTimeMillis(); + eventCount = 0; + timeSpent = 0; + } + } + } + }); + + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + while (true) { + inputHandler.send(new Object[]{"WSO2", 55.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"IBM", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"FB", 100f, 80, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"IBM", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"GOOG", 55.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"IBM", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"WSO2", 100f, 80, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"IBM", 75.6f, 100, System.currentTimeMillis()}); + } + + } +} diff --git a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterMultipleQueryPerformance.java b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterMultipleQueryPerformance.java index a48db0da27..db56d47be2 100644 --- a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterMultipleQueryPerformance.java +++ b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterMultipleQueryPerformance.java @@ -53,7 +53,7 @@ public void receive(Event[] events) { for (Event event : events) { eventCount++; timeSpent += (System.currentTimeMillis() - (Long) event.getData(3)); - if (eventCount % 1000000 == 0) { + if (eventCount % 10000000 == 0) { System.out.println("Throughput : " + (eventCount * 1000) / ((System.currentTimeMillis()) - startTime)); System.out.println("Time spent : " + (timeSpent * 1.0 / eventCount)); diff --git a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterSingleQueryPerformance.java b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterSingleQueryPerformance.java index 77938c99ea..ddcce91981 100644 --- a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterSingleQueryPerformance.java +++ b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterSingleQueryPerformance.java @@ -47,7 +47,7 @@ public void receive(Event[] events) { for (Event event : events) { eventCount++; timeSpent += (System.currentTimeMillis() - (Long) event.getData(3)); - if (eventCount % 1000000 == 0) { + if (eventCount % 10000000 == 0) { System.out.println("Throughput : " + (eventCount * 1000) / ((System.currentTimeMillis()) - startTime)); System.out.println("Time spent : " + (timeSpent * 1.0 / eventCount)); diff --git a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimplePartitionedDoubleFilterQueryPerformance.java b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimplePartitionedDoubleFilterQueryPerformance.java new file mode 100644 index 0000000000..39521552f8 --- /dev/null +++ b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimplePartitionedDoubleFilterQueryPerformance.java @@ -0,0 +1,113 @@ +/* + * 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.performance; + +import io.siddhi.core.SiddhiAppRuntime; +import io.siddhi.core.SiddhiManager; +import io.siddhi.core.event.Event; +import io.siddhi.core.stream.input.InputHandler; +import io.siddhi.core.stream.output.StreamCallback; + +import java.util.concurrent.atomic.AtomicInteger; + +public class SimplePartitionedDoubleFilterQueryPerformance { + + public static void main(String[] args) throws InterruptedException { + SiddhiManager siddhiManager = new SiddhiManager(); + + String siddhiApp = "" + + "define stream cseEventStream (symbol string, price float, volume long, timestamp long);" + + "" + + "partition with (symbol of cseEventStream) " + + "begin " + + " @info(name = 'query1') " + + " from cseEventStream[700 > price] " + + " select * " + + " insert into #innerOutputStream ; " + + "" + + " @info(name = 'query2') " + + " from #innerOutputStream[700 > price] " + + " select * " + + " insert into outputStream ;" + + "end;"; + + SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); + siddhiAppRuntime.addCallback("outputStream", new StreamCallback() { + public AtomicInteger eventCount = new AtomicInteger(); + public int timeSpent = 0; + long startTime = System.currentTimeMillis(); + + @Override + public void receive(Event[] events) { + for (Event event : events) { + int count = eventCount.incrementAndGet(); + timeSpent += (System.currentTimeMillis() - (Long) event.getData(3)); + if (count % 10000000 == 0) { + System.out.println("Throughput : " + (count * 1000) / ((System.currentTimeMillis()) - + startTime)); + System.out.println("Time spent : " + (timeSpent * 1.0 / count)); + startTime = System.currentTimeMillis(); + eventCount.set(0); + timeSpent = 0; + } + } + } + }); + + + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); + siddhiAppRuntime.start(); + + for (int i = 0; i <= 10; i++) { + EventPublisher eventPublisher = new EventPublisher(inputHandler); + eventPublisher.start(); + } + //siddhiAppRuntime.shutdown(); + } + + + static class EventPublisher extends Thread { + + InputHandler inputHandler; + + EventPublisher(InputHandler inputHandler) { + this.inputHandler = inputHandler; + } + + @Override + public void run() { + while (true) { + try { + inputHandler.send(new Object[]{"1", 55.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"2", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"3", 100f, 80, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"4", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"5", 55.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"6", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"7", 100f, 80, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"8", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"9", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"10", 75.6f, 100, System.currentTimeMillis()}); + } catch (InterruptedException e) { + e.printStackTrace(); //To change body of catch statement use File | Settings | File Templates. + } + } + + } + } +} diff --git a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterMultipleQueryWithDisruptorPerformance.java b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimplePartitionedFilterQueryPerformance.java similarity index 55% rename from modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterMultipleQueryWithDisruptorPerformance.java rename to modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimplePartitionedFilterQueryPerformance.java index 106a17b2ee..67bac93ad9 100644 --- a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterMultipleQueryWithDisruptorPerformance.java +++ b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimplePartitionedFilterQueryPerformance.java @@ -1,12 +1,12 @@ /* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. + * 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 + * 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 @@ -23,64 +23,60 @@ import io.siddhi.core.stream.input.InputHandler; import io.siddhi.core.stream.output.StreamCallback; -public class SimpleFilterMultipleQueryWithDisruptorPerformance { +import java.util.concurrent.atomic.AtomicInteger; + +public class SimplePartitionedFilterQueryPerformance { public static void main(String[] args) throws InterruptedException { SiddhiManager siddhiManager = new SiddhiManager(); String siddhiApp = "" + - "@app:async" + - " " + - "define stream cseEventStream (symbol string, price float, volume int, timestamp long);" + - "" + - "@info(name = 'query1') " + - "from cseEventStream[70 > price] " + - "select * " + - "insert into outputStream ;" + + "define stream cseEventStream (symbol string, price float, volume long, timestamp long);" + "" + - "@info(name = 'query2') " + - "from cseEventStream[volume > 90] " + - "select * " + - "insert into outputStream ;"; + "partition with (symbol of cseEventStream) " + + "begin " + + " @info(name = 'query1') " + + " from cseEventStream[700 > price] " + + " select * " + + " insert into outputStream ;" + + "end;"; SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); - siddhiAppRuntime.addCallback("outputStream", new StreamCallback() { - public int eventCount = 0; + public AtomicInteger eventCount = new AtomicInteger(); public int timeSpent = 0; long startTime = System.currentTimeMillis(); @Override public void receive(Event[] events) { for (Event event : events) { - eventCount++; + int count = eventCount.incrementAndGet(); timeSpent += (System.currentTimeMillis() - (Long) event.getData(3)); - if (eventCount % 1000000 == 0) { - System.out.println("Throughput : " + (eventCount * 1000) / ((System.currentTimeMillis()) - + if (count % 10000000 == 0) { + System.out.println("Throughput : " + (count * 1000) / ((System.currentTimeMillis()) - startTime)); - System.out.println("Time spent : " + (timeSpent * 1.0 / eventCount)); + System.out.println("Time spent : " + (timeSpent * 1.0 / count)); startTime = System.currentTimeMillis(); - eventCount = 0; + eventCount.set(0); timeSpent = 0; } } } - }); + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); siddhiAppRuntime.start(); - - for (int i = 0; i <= 100; i++) { + for (int i = 0; i <= 10; i++) { EventPublisher eventPublisher = new EventPublisher(inputHandler); - eventPublisher.run(); + eventPublisher.start(); } //siddhiAppRuntime.shutdown(); } - static class EventPublisher implements Runnable { + static class EventPublisher extends Thread { InputHandler inputHandler; @@ -92,18 +88,21 @@ static class EventPublisher implements Runnable { public void run() { while (true) { try { - inputHandler.send(new Object[]{"WSO2", 55.6f, 100, System.currentTimeMillis()}); - inputHandler.send(new Object[]{"IBM", 75.6f, 100, System.currentTimeMillis()}); - inputHandler.send(new Object[]{"WSO2", 100f, 80, System.currentTimeMillis()}); - inputHandler.send(new Object[]{"IBM", 75.6f, 100, System.currentTimeMillis()}); - inputHandler.send(new Object[]{"WSO2", 55.6f, 100, System.currentTimeMillis()}); - inputHandler.send(new Object[]{"IBM", 75.6f, 100, System.currentTimeMillis()}); - inputHandler.send(new Object[]{"WSO2", 100f, 80, System.currentTimeMillis()}); - inputHandler.send(new Object[]{"IBM", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"1", 55.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"2", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"3", 100f, 80, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"4", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"5", 55.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"6", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"7", 100f, 80, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"8", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"9", 75.6f, 100, System.currentTimeMillis()}); + inputHandler.send(new Object[]{"10", 75.6f, 100, System.currentTimeMillis()}); } catch (InterruptedException e) { e.printStackTrace(); //To change body of catch statement use File | Settings | File Templates. } } + } } } diff --git a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterSingleQueryWithDisruptorPerformance.java b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleWindowSingleQueryPerformance.java similarity index 87% rename from modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterSingleQueryWithDisruptorPerformance.java rename to modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleWindowSingleQueryPerformance.java index 804ed92728..852480d7e5 100644 --- a/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleFilterSingleQueryWithDisruptorPerformance.java +++ b/modules/siddhi-samples/performance-samples/src/main/java/io/siddhi/performance/SimpleWindowSingleQueryPerformance.java @@ -1,12 +1,12 @@ /* - * Copyright (c) 2016, WSO2 Inc. (http://www.wso2.org) All Rights Reserved. + * 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 + * 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 @@ -23,23 +23,20 @@ import io.siddhi.core.stream.input.InputHandler; import io.siddhi.core.stream.output.StreamCallback; -public class SimpleFilterSingleQueryWithDisruptorPerformance { +public class SimpleWindowSingleQueryPerformance { public static void main(String[] args) throws InterruptedException { SiddhiManager siddhiManager = new SiddhiManager(); String siddhiApp = "" + - "@app:async " + - "" + - "define stream cseEventStream (symbol string, price float, volume long, timestamp long);" + + "define stream cseEventStream (symbol string, price float, volume int, timestamp long);" + "" + "@info(name = 'query1') " + - "from cseEventStream[700 > price] " + - "select * " + + "from cseEventStream#window.length(10) " + + "select symbol, sum(price) as total, avg(volume) as avgVolume, timestamp " + "insert into outputStream ;"; SiddhiAppRuntime siddhiAppRuntime = siddhiManager.createSiddhiAppRuntime(siddhiApp); - siddhiAppRuntime.addCallback("outputStream", new StreamCallback() { public int eventCount = 0; public int timeSpent = 0; @@ -50,7 +47,7 @@ public void receive(Event[] events) { for (Event event : events) { eventCount++; timeSpent += (System.currentTimeMillis() - (Long) event.getData(3)); - if (eventCount % 1000000 == 0) { + if (eventCount % 10000000 == 0) { System.out.println("Throughput : " + (eventCount * 1000) / ((System.currentTimeMillis()) - startTime)); System.out.println("Time spent : " + (timeSpent * 1.0 / eventCount)); @@ -60,9 +57,9 @@ public void receive(Event[] events) { } } } - }); + InputHandler inputHandler = siddhiAppRuntime.getInputHandler("cseEventStream"); siddhiAppRuntime.start(); while (true) { @@ -75,6 +72,6 @@ public void receive(Event[] events) { inputHandler.send(new Object[]{"WSO2", 100f, 80, System.currentTimeMillis()}); inputHandler.send(new Object[]{"IBM", 75.6f, 100, System.currentTimeMillis()}); } -// siddhiAppRuntime.shutdown(); + } } diff --git a/modules/siddhi-samples/quick-start-samples/src/main/java/io/siddhi/sample/util/CustomFunctionExtension.java b/modules/siddhi-samples/quick-start-samples/src/main/java/io/siddhi/sample/util/CustomFunctionExtension.java index dbe37fd984..616f0fe47b 100644 --- a/modules/siddhi-samples/quick-start-samples/src/main/java/io/siddhi/sample/util/CustomFunctionExtension.java +++ b/modules/siddhi-samples/quick-start-samples/src/main/java/io/siddhi/sample/util/CustomFunctionExtension.java @@ -23,10 +23,10 @@ import io.siddhi.core.executor.ExpressionExecutor; 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 java.util.Map; - public class CustomFunctionExtension extends FunctionExecutor { private Attribute.Type returnType; @@ -39,8 +39,8 @@ public class CustomFunctionExtension extends FunctionExecutor { * @param siddhiQueryContext the context of the siddhi query */ @Override - protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, - SiddhiQueryContext siddhiQueryContext) { + protected StateFactory init(ExpressionExecutor[] attributeExpressionExecutors, ConfigReader configReader, + SiddhiQueryContext siddhiQueryContext) { for (ExpressionExecutor expressionExecutor : attributeExpressionExecutors) { Attribute.Type attributeType = expressionExecutor.getReturnType(); if (attributeType == Attribute.Type.DOUBLE) { @@ -52,17 +52,19 @@ protected void init(ExpressionExecutor[] attributeExpressionExecutors, ConfigRea returnType = Attribute.Type.LONG; } } + return null; } /** * The main execution method which will be called upon event arrival * when there are more then one function parameter * - * @param data the runtime values of function parameters + * @param data the runtime values of function parameters + * @param state * @return the function result */ @Override - protected Object execute(Object[] data) { + protected Object execute(Object[] data, State state) { if (returnType == Attribute.Type.DOUBLE) { double total = 0; for (Object aObj : data) { @@ -83,12 +85,13 @@ protected Object execute(Object[] data) { * The main execution method which will be called upon event arrival * when there are zero or one function parameter * - * @param data null if the function parameter count is zero or - * runtime data value of the function parameter + * @param data null if the function parameter count is zero or + * runtime data value of the function parameter + * @param state Function state * @return the function result */ @Override - protected Object execute(Object data) { + protected Object execute(Object data, State state) { if (returnType == Attribute.Type.DOUBLE) { return Double.parseDouble(String.valueOf(data)); } else { @@ -101,27 +104,4 @@ public Attribute.Type getReturnType() { return returnType; } - /** - * Used to collect the serializable state of the processing element, that need to be - * persisted for the reconstructing the element to the same state on a different point of time - * - * @return stateful objects of the processing element as an array - */ - @Override - public Map currentState() { - return null; - } - - /** - * Used to restore serialized state of the processing element, for reconstructing - * the element to the same state as if was on a previous point of time. - * - * @param state the stateful objects of the element as an array on - * the same order provided by currentState(). - */ - @Override - public void restoreState(Map state) { - - } - } diff --git a/pom.xml b/pom.xml index fb5bfef5c4..dcabda23dc 100644 --- a/pom.xml +++ b/pom.xml @@ -539,7 +539,7 @@ validate - https://raw.githubusercontent.com/wso2/code-quality-tools/master/checkstyle/checkstyle.xml + https://raw.githubusercontent.com/wso2/code-quality-tools/v1.2/checkstyle/checkstyle.xml ${mavan.checkstyle.suppression.file}