diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..9985ffb
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,30 @@
+.cache
+scalastyle-output.xml
+.classpath
+.idea
+.metadata
+.settings
+.project
+.version.properties
+filter.properties
+logs.zip
+target
+tmp
+*.class
+*.iml
+*.swp
+*.jar
+*.log
+.DS_Store
+build-target
+flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/generated/
+flink-runtime-web/web-dashboard/assets/fonts/
+flink-runtime-web/web-dashboard/node_modules/
+flink-runtime-web/web-dashboard/bower_components/
+atlassian-ide-plugin.xml
+out/
+/docs/api
+/docs/content
+/docs/Gemfile.lock
+/docs/.bundle
+/docs/.rubydeps
diff --git a/LICENSE b/LICENSE
new file mode 100644
index 0000000..9c8f3ea
--- /dev/null
+++ b/LICENSE
@@ -0,0 +1,201 @@
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "{}"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright {yyyy} {name of copyright owner}
+
+ Licensed 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.
\ No newline at end of file
diff --git a/README.md b/README.md
new file mode 100644
index 0000000..ba195bb
--- /dev/null
+++ b/README.md
@@ -0,0 +1,88 @@
+
+flink-siddhi
+============
+
+> A light-weight library to run siddhi cep within flink streaming application.
+
+__Version:`1.2-SNAPSHOT`__
+
+## About
+
+Siddhi CEP is a lightweight and easy-to-use Open Source Complex Event Processing Engine (CEP) released as a Java Library under `Apache Software License v2.0`.
+Siddhi CEP processes events which are generated by various event sources, analyses them and notifies appropriate complex events according to the user specified queries.
+
+This project is mainly to provide a light-weight library to easily run Siddhi CEP within flink streaming application.
+
+## How to Use
+
+* Add `flink-siddhi` in maven dependency:
+
+
+ com.github.haoch
+ flink-siddhi
+ 1.2-SNAPSHOT
+
+
+* Start using API from `SiddhiCEP`, for example:
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ SiddhiCEP cep = SiddhiCEP.getSiddhiEnvironment(env);
+
+ cep.registerExtension("custom:plus",CustomPlusFunctionExtension.class);
+
+ cep.registerStream("inputStream1", input1, "id", "name", "price","timestamp");
+ cep.registerStream("inputStream2", input2, "id", "name", "price","timestamp");
+
+ DataStream> output = cep
+ .from("inputStream1").union("inputStream2")
+ .sql(
+ "from every s1 = inputStream1[id == 2] "
+ + " -> s2 = inputStream2[id == 3] "
+ + "select s1.id as id_1, s1.name as name_1, s2.id as id_2, s2.name as name_2 , custom:plus(s1.price,s2.price) as price"
+ + "insert into outputStream"
+ )
+ .returns("outputStream");
+
+ env.execute();
+
+ > See more examples at `org.apache.flink.contrib.siddhi.SiddhiCEPITCase`
+
+## How to Build
+
+ mvn clean install -DskipTests
+
+## How to Test
+
+ mvn clean test
+
+## Main Features
+
+* Integrate Siddhi CEP as an stream operator (i.e. `TupleStreamSiddhiOperator`), supporting rich CEP features like
+ * Filter
+ * Join
+ * Aggregation
+ * Group by
+ * Having
+ * Window
+ * Conditions and Expressions
+ * Pattern processing
+ * Sequence processing
+ * Event Tables
+ * ...
+
+* Provide easy-to-use Siddhi CEP API to integrate Flink DataStream API (See `SiddhiCEP` and `SiddhiStream`)
+ * Register Flink DataStream associating native type information with Siddhi Stream Schema, supporting POJO,Tuple, Primitive Type, etc.
+ * Connect with single or multiple Flink DataStreams with Siddhi CEP Execution Plan
+ * Return output stream as DataStream with type intelligently inferred from Siddhi Stream Schema
+
+* Integrate siddhi runtime state management with Flink state (See `AbstractSiddhiOperator`)
+
+* Support siddhi plugin management to extend CEP functions. (See `SiddhiCEP#registerExtension`)
+
+## Get Help and Contact
+
+* [@haoch](http://github.com/haoch) (hao AT apache DOT org)
+
+## License
+
+Licensed under the [Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0). More details, please refer to [LICENSE](LICENSE) file.
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
new file mode 100644
index 0000000..391568f
--- /dev/null
+++ b/pom.xml
@@ -0,0 +1,133 @@
+
+
+
+
+ 4.0.0
+
+
+ org.apache.flink
+ flink-parent
+ 1.2-SNAPSHOT
+
+
+ com.github.haoch
+ flink-siddhi_2.10
+ flink-siddhi
+
+ jar
+
+
+ 3.0.5
+
+
+
+ central
+
+ Maven Repository
+ https://repo1.maven.org/maven2
+
+ true
+
+
+ false
+
+
+
+ snapshot
+ Maven Snapshot Repository
+ https://repository.apache.org/snapshots
+
+ true
+
+
+ false
+
+
+
+ Wso2 Repository
+ http://maven.wso2.org/nexus/content/groups/wso2-public
+
+ true
+
+
+ false
+
+
+
+
+
+ central
+ https://repo1.maven.org/maven2
+
+ true
+
+
+ false
+
+
+
+
+
+
+
+ org.wso2.siddhi
+ siddhi-core
+ ${siddhi.version}
+
+
+
+
+ com.esotericsoftware.kryo
+ kryo
+ provided
+
+
+
+
+ org.apache.flink
+ flink-streaming-java_2.10
+ ${project.version}
+ provided
+
+
+
+
+
+ org.apache.flink
+ flink-runtime_2.10
+ ${project.version}
+ test-jar
+ test
+
+
+ org.apache.flink
+ flink-test-utils_2.10
+ ${project.version}
+ test
+
+
+ org.apache.flink
+ flink-scala_2.10
+ ${project.version}
+
+
+
diff --git a/src/main/java/org/apache/flink/contrib/siddhi/SiddhiCEP.java b/src/main/java/org/apache/flink/contrib/siddhi/SiddhiCEP.java
new file mode 100644
index 0000000..14ee580
--- /dev/null
+++ b/src/main/java/org/apache/flink/contrib/siddhi/SiddhiCEP.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.contrib.siddhi;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.contrib.siddhi.exception.DuplicatedStreamException;
+import org.apache.flink.contrib.siddhi.exception.UndefinedStreamException;
+import org.apache.flink.contrib.siddhi.schema.SiddhiStreamSchema;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Siddhi CEP Execution Environment
+ */
+@PublicEvolving
+public class SiddhiCEP {
+ private final StreamExecutionEnvironment executionEnvironment;
+ private final Map> dataStreams = new HashMap<>();
+ private final Map> dataStreamSchemas = new HashMap<>();
+ private final Map> extensions = new HashMap<>();
+
+ public Map> getDataStreams() {
+ return this.dataStreams;
+ }
+
+ public Map> getDataStreamSchemas() {
+ return this.dataStreamSchemas;
+ }
+
+ public boolean isStreamDefined(String streamId) {
+ return dataStreams.containsKey(streamId);
+ }
+
+ public Map> getExtensions() {
+ return this.extensions;
+ }
+
+ public void checkStreamDefined(String streamId) throws UndefinedStreamException {
+ if (!isStreamDefined(streamId)) {
+ throw new UndefinedStreamException("Stream (streamId: " + streamId + ") not defined");
+ }
+ }
+
+ public SiddhiCEP(StreamExecutionEnvironment streamExecutionEnvironment) {
+ this.executionEnvironment = streamExecutionEnvironment;
+ }
+
+ public static SiddhiStream.SingleSiddhiStream define(String streamId, DataStream inStream, String... fieldNames) {
+ SiddhiCEP environment = SiddhiCEP.getSiddhiEnvironment(inStream.getExecutionEnvironment());
+ return environment.from(streamId, inStream, fieldNames);
+ }
+
+ public SiddhiStream.SingleSiddhiStream from(String streamId, DataStream inStream, String... fieldNames) {
+ this.registerStream(streamId, inStream, fieldNames);
+ return new SiddhiStream.SingleSiddhiStream<>(streamId, this);
+ }
+
+ public SiddhiStream.SingleSiddhiStream from(String streamId) {
+ return new SiddhiStream.SingleSiddhiStream<>(streamId, this);
+ }
+
+ public SiddhiStream.UnionSiddhiStream union(String firstStreamId, String... unionStreamIds) {
+ return new SiddhiStream.SingleSiddhiStream(firstStreamId, this).union(unionStreamIds);
+ }
+
+ public void registerStream(final String streamId, DataStream dataStream, String... fieldNames) {
+ if (isStreamDefined(streamId)) {
+ throw new DuplicatedStreamException("Input stream: " + streamId + " already exists");
+ }
+ dataStreams.put(streamId, dataStream);
+ SiddhiStreamSchema schema = new SiddhiStreamSchema<>(dataStream.getType(), fieldNames);
+ schema.setTypeSerializer(schema.getTypeInfo().createSerializer(dataStream.getExecutionConfig()));
+ dataStreamSchemas.put(streamId, schema);
+ }
+
+ public StreamExecutionEnvironment getExecutionEnvironment() {
+ return executionEnvironment;
+ }
+
+ public void registerExtension(String extensionName, Class> extensionClass) {
+ if (extensions.containsKey(extensionName)) {
+ throw new IllegalArgumentException("Extension named " + extensionName + " already registered");
+ }
+ extensions.put(extensionName, extensionClass);
+ }
+
+ public DataStream getDataStream(String streamId) {
+ if (this.dataStreams.containsKey(streamId)) {
+ return (DataStream) this.dataStreams.get(streamId);
+ } else {
+ throw new UndefinedStreamException("Undefined stream " + streamId);
+ }
+ }
+
+ public static SiddhiCEP getSiddhiEnvironment(StreamExecutionEnvironment streamExecutionEnvironment) {
+ return new SiddhiCEP(streamExecutionEnvironment);
+ }
+}
diff --git a/src/main/java/org/apache/flink/contrib/siddhi/SiddhiStream.java b/src/main/java/org/apache/flink/contrib/siddhi/SiddhiStream.java
new file mode 100644
index 0000000..f10cf1b
--- /dev/null
+++ b/src/main/java/org/apache/flink/contrib/siddhi/SiddhiStream.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.contrib.siddhi;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.contrib.siddhi.operator.SiddhiOperatorContext;
+import org.apache.flink.contrib.siddhi.utils.SiddhiStreamFactory;
+import org.apache.flink.contrib.siddhi.utils.SiddhiTypeFactory;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Siddhi CEP API Interface
+ */
+@PublicEvolving
+public abstract class SiddhiStream {
+ private final SiddhiCEP environment;
+
+ public SiddhiStream(SiddhiCEP environment) {
+ this.environment = environment;
+ }
+
+ protected SiddhiCEP getEnvironment() {
+ return this.environment;
+ }
+
+ protected abstract DataStream> toDataStream();
+
+ /**
+ * Convert DataStream<T> to DataStream<Tuple2<String,T>>.
+ * If it's KeyedStream. pass through original keySelector
+ */
+ protected DataStream> convertDataStream(DataStream dataStream, String streamId) {
+ final String streamIdInClosure = streamId;
+ DataStream> resultStream = dataStream.map(new MapFunction>() {
+ @Override
+ public Tuple2 map(T value) throws Exception {
+ return Tuple2.of(streamIdInClosure, (Object) value);
+ }
+ });
+ if (dataStream instanceof KeyedStream) {
+ final KeySelector keySelector = ((KeyedStream) dataStream).getKeySelector();
+ final KeySelector, Object> keySelectorInClosure = new KeySelector, Object>() {
+ @Override
+ public Object getKey(Tuple2 value) throws Exception {
+ return keySelector.getKey((T) value.f1);
+ }
+ };
+ return resultStream.keyBy(keySelectorInClosure);
+ } else {
+ return resultStream;
+ }
+ }
+
+ public static abstract class ExecutableStream extends SiddhiStream {
+ public ExecutableStream(SiddhiCEP environment) {
+ super(environment);
+ }
+
+ public ExecutionSiddhiStream sql(String executionPlan) {
+ return new ExecutionSiddhiStream(this.toDataStream(), executionPlan, getEnvironment());
+ }
+ }
+
+ public static class SingleSiddhiStream extends ExecutableStream {
+ private final String streamId;
+
+ public SingleSiddhiStream(String streamId, SiddhiCEP environment) {
+ super(environment);
+ environment.checkStreamDefined(streamId);
+ this.streamId = streamId;
+ }
+
+ public UnionSiddhiStream union(String streamId, DataStream dataStream, String... fieldNames) {
+ getEnvironment().registerStream(streamId, dataStream, fieldNames);
+ return union(streamId);
+ }
+
+ public UnionSiddhiStream union(String... streamIds) {
+ Preconditions.checkNotNull(streamIds);
+ return new UnionSiddhiStream(this.streamId, Arrays.asList(streamIds), this.getEnvironment());
+ }
+
+ @Override
+ protected DataStream> toDataStream() {
+ return convertDataStream(getEnvironment().getDataStream(this.streamId), this.streamId);
+ }
+ }
+
+ public static class UnionSiddhiStream extends ExecutableStream {
+ private String firstStreamId;
+ private List unionStreamIds;
+
+ public UnionSiddhiStream(String firstStreamId, List unionStreamIds, SiddhiCEP environment) {
+ super(environment);
+ environment.checkStreamDefined(firstStreamId);
+ for (String unionStreamId : unionStreamIds) {
+ environment.checkStreamDefined(unionStreamId);
+ }
+ this.firstStreamId = firstStreamId;
+ this.unionStreamIds = unionStreamIds;
+ }
+
+ public UnionSiddhiStream union(String streamId, DataStream dataStream, String... fieldNames) {
+ getEnvironment().registerStream(streamId, dataStream, fieldNames);
+ return union(streamId);
+ }
+
+ public UnionSiddhiStream union(String... streamId) {
+ List newUnionStreamIds = new LinkedList<>();
+ newUnionStreamIds.addAll(unionStreamIds);
+ newUnionStreamIds.addAll(Arrays.asList(streamId));
+ return new UnionSiddhiStream(this.firstStreamId, newUnionStreamIds, this.getEnvironment());
+ }
+
+ @Override
+ protected DataStream> toDataStream() {
+ final String localFirstStreamId = firstStreamId;
+ final List localUnionStreamIds = this.unionStreamIds;
+ DataStream> dataStream = convertDataStream(getEnvironment().getDataStream(localFirstStreamId), this.firstStreamId);
+ for (String unionStreamId : localUnionStreamIds) {
+ dataStream = dataStream.union(convertDataStream(getEnvironment().getDataStream(unionStreamId), unionStreamId));
+ }
+ return dataStream;
+ }
+ }
+
+ public static class ExecutionSiddhiStream {
+ private final DataStream> dataStream;
+ private final SiddhiCEP environment;
+ private final String executionPlan;
+
+ public ExecutionSiddhiStream(DataStream> dataStream, String executionPlan, SiddhiCEP environment) {
+ this.executionPlan = executionPlan;
+ this.dataStream = dataStream;
+ this.environment = environment;
+ }
+
+ /**
+ * @param outStreamId The streamId to return as data stream.
+ * @param Type information should match with stream definition.
+ * During execution phase, it will automatically build type information based on stream definition.
+ * @return Return output stream as Tuple
+ * @see SiddhiTypeFactory
+ */
+ public DataStream returns(String outStreamId) {
+ SiddhiOperatorContext siddhiContext = new SiddhiOperatorContext();
+ siddhiContext.setExecutionPlan(executionPlan);
+ siddhiContext.setInputStreamSchemas(environment.getDataStreamSchemas());
+ siddhiContext.setTimeCharacteristic(environment.getExecutionEnvironment().getStreamTimeCharacteristic());
+ siddhiContext.setOutputStreamId(outStreamId);
+ siddhiContext.setExtensions(environment.getExtensions());
+ siddhiContext.setExecutionConfig(environment.getExecutionEnvironment().getConfig());
+ TypeInformation typeInformation =
+ SiddhiTypeFactory.getTupleTypeInformation(siddhiContext.getFinalExecutionPlan(), outStreamId);
+ siddhiContext.setOutputStreamType(typeInformation);
+ return returnsInternal(siddhiContext);
+ }
+
+ /**
+ * @return Return output stream as DataStream<Map<String,Object>>,
+ * out type is LinkedHashMap<String,Object> and guarantee field order
+ * as defined in siddhi execution plan
+ * @see java.util.LinkedHashMap
+ */
+ public DataStream