You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@eagle.apache.org by ra...@apache.org on 2016/08/19 03:18:30 UTC

incubator-eagle git commit: Refactor : Move siddhi extension to alert project

Repository: incubator-eagle
Updated Branches:
  refs/heads/develop 2c63fdf0c -> 34e77cafc


Refactor : Move siddhi extension to alert project

add loop back test


Project: http://git-wip-us.apache.org/repos/asf/incubator-eagle/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-eagle/commit/34e77caf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-eagle/tree/34e77caf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-eagle/diff/34e77caf

Branch: refs/heads/develop
Commit: 34e77cafc4ef73802b9a5716dca6a6f7b54e10c5
Parents: 2c63fdf
Author: Ralph, Su <su...@gmail.com>
Authored: Fri Aug 19 11:18:58 2016 +0800
Committer: Ralph, Su <su...@gmail.com>
Committed: Fri Aug 19 11:18:58 2016 +0800

----------------------------------------------------------------------
 .../extension/ContainsIgnoreCaseExtension.java  |  88 +++++++++
 .../extension/EqualsIgnoreCaseExtension.java    |  92 +++++++++
 .../RegexpIgnoreCaseFunctionExtension.java      |  90 +++++++++
 .../src/main/resources/str.siddhiext            |  40 ++++
 .../alert/engine/siddhi/SiddhiPolicyTest.java   |  30 +++
 .../test/resources/loopback/datasources.json    |  36 ++++
 .../src/test/resources/loopback/policies.json   |  50 +++++
 .../test/resources/loopback/publishments.json   |  15 ++
 .../resources/loopback/streamdefinitions.json   | 195 +++++++++++++++++++
 .../common/agg/TestSiddhiExternalTimeBatch.java |   3 +
 .../extension/ContainsIgnoreCaseExtension.java  |  88 ---------
 .../extension/EqualsIgnoreCaseExtension.java    |  92 ---------
 .../RegexpIgnoreCaseFunctionExtension.java      |  90 ---------
 .../src/main/resources/str.siddhiext            |  40 ----
 14 files changed, 639 insertions(+), 310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/ContainsIgnoreCaseExtension.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/ContainsIgnoreCaseExtension.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/ContainsIgnoreCaseExtension.java
new file mode 100644
index 0000000..1bd24ed
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/ContainsIgnoreCaseExtension.java
@@ -0,0 +1,88 @@
+/*
+ * 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.eagle.alert.engine.siddhi.extension;
+
+import org.wso2.siddhi.core.config.ExecutionPlanContext;
+import org.wso2.siddhi.core.exception.ExecutionPlanRuntimeException;
+import org.wso2.siddhi.core.executor.ExpressionExecutor;
+import org.wso2.siddhi.core.executor.function.FunctionExecutor;
+import org.wso2.siddhi.query.api.definition.Attribute;
+import org.wso2.siddhi.query.api.exception.ExecutionPlanValidationException;
+
+public class ContainsIgnoreCaseExtension extends FunctionExecutor {
+
+    Attribute.Type returnType = Attribute.Type.BOOL;
+
+    @Override
+    protected void init(ExpressionExecutor[] attributeExpressionExecutors, ExecutionPlanContext executionPlanContext) {
+        if (attributeExpressionExecutors.length != 2) {
+            throw new ExecutionPlanValidationException("Invalid no of arguments passed to str:containsIgnoreCase() function, required 2, " +
+                    "but found " + attributeExpressionExecutors.length);
+        }
+        if (attributeExpressionExecutors[0].getReturnType() != Attribute.Type.STRING) {
+            throw new ExecutionPlanValidationException("Invalid parameter type found for the first argument of str:containsIgnoreCase() function, " +
+                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[0].getReturnType().toString());
+        }
+        if (attributeExpressionExecutors[1].getReturnType() != Attribute.Type.STRING) {
+            throw new ExecutionPlanValidationException("Invalid parameter type found for the second argument of str:containsIgnoreCase() function, " +
+                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[1].getReturnType().toString());
+        }
+    }
+
+    @Override
+    protected Object execute(Object[] data) {
+        if (data[0] == null) {
+            throw new ExecutionPlanRuntimeException("Invalid input given to str:containsIgnoreCase() function. First argument cannot be null");
+        }
+        if (data[1] == null) {
+            throw new ExecutionPlanRuntimeException("Invalid input given to str:containsIgnoreCase() function. Second argument cannot be null");
+        }
+        String str1 = (String)data[0];
+        String str2 = (String)data[1];
+        return str1.toUpperCase().contains(str2.toUpperCase());
+    }
+
+    @Override
+    protected Object execute(Object data) {
+        return null; //Since the containsIgnoreCase function takes in 2 parameters, this method does not get called. Hence, not implemented.
+    }
+
+    @Override
+    public void start() {
+        //Nothing to start
+    }
+
+    @Override
+    public void stop() {
+        //Nothing to stop
+    }
+
+    @Override
+    public Attribute.Type getReturnType() {
+        return returnType;
+    }
+
+    @Override
+    public Object[] currentState() {
+        return new Object[]{};
+    }
+
+    @Override
+    public void restoreState(Object[] state) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/EqualsIgnoreCaseExtension.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/EqualsIgnoreCaseExtension.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/EqualsIgnoreCaseExtension.java
new file mode 100644
index 0000000..e99c4b9
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/EqualsIgnoreCaseExtension.java
@@ -0,0 +1,92 @@
+/*
+ * 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.eagle.alert.engine.siddhi.extension;
+
+import org.wso2.siddhi.core.config.ExecutionPlanContext;
+import org.wso2.siddhi.core.exception.ExecutionPlanRuntimeException;
+import org.wso2.siddhi.core.executor.ConstantExpressionExecutor;
+import org.wso2.siddhi.core.executor.ExpressionExecutor;
+import org.wso2.siddhi.core.executor.function.FunctionExecutor;
+import org.wso2.siddhi.query.api.definition.Attribute;
+import org.wso2.siddhi.query.api.exception.ExecutionPlanValidationException;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class EqualsIgnoreCaseExtension extends FunctionExecutor {
+
+    Attribute.Type returnType = Attribute.Type.BOOL;
+
+    @Override
+    protected void init(ExpressionExecutor[] attributeExpressionExecutors, ExecutionPlanContext executionPlanContext) {
+        if (attributeExpressionExecutors.length != 2) {
+            throw new ExecutionPlanValidationException("Invalid no of arguments passed to str:equalsIgnoreCase() function, required 2, " +
+                    "but found " + attributeExpressionExecutors.length);
+        }
+        if (attributeExpressionExecutors[0].getReturnType() != Attribute.Type.STRING) {
+            throw new ExecutionPlanValidationException("Invalid parameter type found for the first argument of str:equalsIgnoreCase() function, " +
+                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[0].getReturnType().toString());
+        }
+        if (attributeExpressionExecutors[1].getReturnType() != Attribute.Type.STRING) {
+            throw new ExecutionPlanValidationException("Invalid parameter type found for the second argument of str:equalsIgnoreCase() function, " +
+                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[1].getReturnType().toString());
+        }
+    }
+
+    @Override
+    protected Object execute(Object[] data) {
+        if (data[0] == null) {
+            throw new ExecutionPlanRuntimeException("Invalid input given to str:equalsIgnoreCase() function. First argument cannot be null");
+        }
+        if (data[1] == null) {
+            throw new ExecutionPlanRuntimeException("Invalid input given to str:equalsIgnoreCase() function. Second argument cannot be null");
+        }
+        String str1 = (String)data[0];
+        String str2 = (String)data[1];
+        return str1.equalsIgnoreCase(str2);
+    }
+
+    @Override
+    protected Object execute(Object data) {
+        return null; //Since the equalsIgnoreCase function takes in 2 parameters, this method does not get called. Hence, not implemented.
+    }
+
+    @Override
+    public void start() {
+        //Nothing to start
+    }
+
+    @Override
+    public void stop() {
+        //Nothing to stop
+    }
+
+    @Override
+    public Attribute.Type getReturnType() {
+        return returnType;
+    }
+
+    @Override
+    public Object[] currentState() {
+        return new Object[]{};
+    }
+
+    @Override
+    public void restoreState(Object[] state) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/RegexpIgnoreCaseFunctionExtension.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/RegexpIgnoreCaseFunctionExtension.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/RegexpIgnoreCaseFunctionExtension.java
new file mode 100644
index 0000000..d384d47
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/java/org/apache/eagle/alert/engine/siddhi/extension/RegexpIgnoreCaseFunctionExtension.java
@@ -0,0 +1,90 @@
+/*
+ * 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.eagle.alert.engine.siddhi.extension;
+
+import org.wso2.siddhi.core.config.ExecutionPlanContext;
+import org.wso2.siddhi.core.exception.ExecutionPlanRuntimeException;
+import org.wso2.siddhi.core.executor.ConstantExpressionExecutor;
+import org.wso2.siddhi.core.executor.ExpressionExecutor;
+import org.wso2.siddhi.extension.string.RegexpFunctionExtension;
+import org.wso2.siddhi.query.api.definition.Attribute;
+import org.wso2.siddhi.query.api.exception.ExecutionPlanValidationException;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * regexpIgnoreCase(string, regex)
+ * Tells whether or not this 'string' matches the given regular expression 'regex'.
+ * Accept Type(s): (STRING,STRING)
+ * Return Type(s): BOOLEAN
+ */
+public class RegexpIgnoreCaseFunctionExtension extends RegexpFunctionExtension {
+
+    //state-variables
+    boolean isRegexConstant = false;
+    String regexConstant;
+    Pattern patternConstant;
+
+    @Override
+    protected void init(ExpressionExecutor[] attributeExpressionExecutors, ExecutionPlanContext executionPlanContext) {
+        if (attributeExpressionExecutors.length != 2) {
+            throw new ExecutionPlanValidationException("Invalid no of arguments passed to str:regexpIgnoreCase() function, required 2, " +
+                    "but found " + attributeExpressionExecutors.length);
+        }
+        if (attributeExpressionExecutors[0].getReturnType() != Attribute.Type.STRING) {
+            throw new ExecutionPlanValidationException("Invalid parameter type found for the first argument of str:regexpIgnoreCase() function, " +
+                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[0].getReturnType().toString());
+        }
+        if (attributeExpressionExecutors[1].getReturnType() != Attribute.Type.STRING) {
+            throw new ExecutionPlanValidationException("Invalid parameter type found for the second argument of str:regexpIgnoreCase() function, " +
+                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[1].getReturnType().toString());
+        }
+        if(attributeExpressionExecutors[1] instanceof ConstantExpressionExecutor){
+            isRegexConstant = true;
+            regexConstant = (String) ((ConstantExpressionExecutor) attributeExpressionExecutors[1]).getValue();
+            patternConstant = Pattern.compile(regexConstant, Pattern.CASE_INSENSITIVE);
+        }
+    }
+
+    @Override
+    protected Object execute(Object[] data) {
+        String regex;
+        Pattern pattern;
+        Matcher matcher;
+
+        if (data[0] == null) {
+            throw new ExecutionPlanRuntimeException("Invalid input given to str:regexpIgnoreCase() function. First argument cannot be null");
+        }
+        if (data[1] == null) {
+            throw new ExecutionPlanRuntimeException("Invalid input given to str:regexpIgnoreCase() function. Second argument cannot be null");
+        }
+        String source = (String) data[0];
+
+        if(!isRegexConstant){
+            regex = (String) data[1];
+            pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
+            matcher = pattern.matcher(source);
+            return matcher.matches();
+
+        } else {
+            matcher = patternConstant.matcher(source);
+            return matcher.matches();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/str.siddhiext
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/str.siddhiext b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/str.siddhiext
new file mode 100644
index 0000000..7569989
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/main/resources/str.siddhiext
@@ -0,0 +1,40 @@
+#
+# 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.
+#
+
+charAt=org.wso2.siddhi.extension.string.CharAtFunctionExtension
+coalesce=org.wso2.siddhi.extension.string.CoalesceFunctionExtension
+concat=org.wso2.siddhi.extension.string.ConcatFunctionExtension
+length=org.wso2.siddhi.extension.string.LengthFunctionExtension
+lower=org.wso2.siddhi.extension.string.LowerFunctionExtension
+regexp=org.wso2.siddhi.extension.string.RegexpFunctionExtension
+repeat=org.wso2.siddhi.extension.string.RepeatFunctionExtension
+replaceAll=org.wso2.siddhi.extension.string.ReplaceAllFunctionExtension
+replaceFirst=org.wso2.siddhi.extension.string.ReplaceFirstFunctionExtension
+reverse=org.wso2.siddhi.extension.string.ReverseFunctionExtension
+strcmp=org.wso2.siddhi.extension.string.StrcmpFunctionExtension
+substr=org.wso2.siddhi.extension.string.SubstrFunctionExtension
+trim=org.wso2.siddhi.extension.string.TrimFunctionExtension
+upper=org.wso2.siddhi.extension.string.UpperFunctionExtension
+hex=org.wso2.siddhi.extension.string.HexFunctionExtension
+unhex=org.wso2.siddhi.extension.string.UnhexFunctionExtension
+contains=org.wso2.siddhi.extension.string.ContainsFunctionExtension
+
+# Eagle Siddhi Extension
+equalsIgnoreCase=org.apache.eagle.alert.engine.siddhi.extension.EqualsIgnoreCaseExtension
+containsIgnoreCase=org.apache.eagle.alert.engine.siddhi.extension.ContainsIgnoreCaseExtension
+regexpIgnoreCase=org.apache.eagle.alert.engine.siddhi.extension.RegexpIgnoreCaseFunctionExtension
+

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/siddhi/SiddhiPolicyTest.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/siddhi/SiddhiPolicyTest.java b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/siddhi/SiddhiPolicyTest.java
index cb60a46..3d373b6 100644
--- a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/siddhi/SiddhiPolicyTest.java
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/java/org/apache/eagle/alert/engine/siddhi/SiddhiPolicyTest.java
@@ -24,6 +24,7 @@ import org.wso2.siddhi.core.SiddhiManager;
 import org.wso2.siddhi.core.event.Event;
 import org.wso2.siddhi.core.stream.input.InputHandler;
 import org.wso2.siddhi.core.stream.output.StreamCallback;
+import org.wso2.siddhi.core.util.EventPrinter;
 
 import java.util.HashSet;
 import java.util.Set;
@@ -240,4 +241,33 @@ public class SiddhiPolicyTest {
         e.setData(new Object[] {"facitliy", "SEVERITY_EMERG", "HOSTNAME-" + 11 , "MSGID-...", "Timestamp", "conn-sss", "op-msg", "msgId..", "command-...", "name-", "namespace", System.currentTimeMillis()});
         handler.send(e);
     }
+
+
+    @Test
+    public void testStrConcat() throws Exception {
+        String ql = " define stream log(timestamp long, switchLabel string, port string, message string); " +
+                " from log select timestamp, str:concat(switchLabel, '===', port) as alertKey, message insert into output; ";
+        SiddhiManager manager = new SiddhiManager();
+        ExecutionPlanRuntime runtime = manager.createExecutionPlanRuntime(ql);
+        runtime.addCallback("output", new StreamCallback() {
+            @Override
+            public void receive(Event[] events) {
+                EventPrinter.print(events);
+            }
+        });
+
+        runtime.start();
+
+        InputHandler logInput = runtime.getInputHandler("log");
+
+        Event e = new Event();
+        e.setTimestamp(System.currentTimeMillis());
+        e.setData(new Object[] {System.currentTimeMillis(), "switch-ra-slc-01", "port01", "log-message...."});
+        logInput.send(e);
+
+        Thread.sleep(1000);
+        runtime.shutdown();
+
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/datasources.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/datasources.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/datasources.json
new file mode 100644
index 0000000..4162ed6
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/datasources.json
@@ -0,0 +1,36 @@
+[
+  {
+    "name": "network_syslog_datasource",
+    "type": "KAFKA",
+    "properties": {
+    },
+    "topic": "syslog_events",
+    "schemeCls": "org.apache.eagle.alert.engine.scheme.JsonScheme",
+    "codec": {
+      "streamNameSelectorProp": {
+        "userProvidedStreamName": "syslog_stream",
+        "streamNameFormat": "%s"
+      },
+      "streamNameSelectorCls": "org.apache.eagle.alert.engine.scheme.JsonStringStreamNameSelector",
+      "timestampColumn": "timestamp",
+      "timestampFormat": ""
+    }
+  },
+  {
+    "name": "network_syslog_alert_datasource",
+    "type": "KAFKA",
+    "properties": {
+    },
+    "topic": "syslog_alerts",
+    "schemeCls": "org.apache.eagle.alert.engine.scheme.JsonScheme",
+    "codec": {
+      "streamNameSelectorProp": {
+        "userProvidedStreamName": "syslog_stream_subalert",
+        "streamNameFormat": "%s"
+      },
+      "streamNameSelectorCls": "org.apache.eagle.alert.engine.scheme.JsonStringStreamNameSelector",
+      "timestampColumn": "timestamp",
+      "timestampFormat": ""
+    }
+  }
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/policies.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/policies.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/policies.json
new file mode 100644
index 0000000..05743f5
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/policies.json
@@ -0,0 +1,50 @@
+[
+  {
+    "name": "syslog_severity_critical_check",
+    "description": "syslog.severity > 3 then error",
+    "inputStreams": [
+      "syslog_stream"
+    ],
+    "outputStreams": [
+      "syslog_severity_critical_output"
+    ],
+    "definition": {
+      "type": "siddhi",
+      "value": "from syslog_stream[(severity_name == \"EMERG\" or severity_name == \"ALERT\" or severity_name ==\"CRIT\") and (regex:find(\"The system is overheating\", msg) or regex:find(\"Power Supply is not responding\",msg) or regex:find(\"Memory inconsistency detected\", msg))] select 'alert' as name, 'CRITICAL' as alertSeverity, namespace, facility_name, facility_code, severity_code,severity_name ,dims_hostname, dims_appname, msgid,  msg, epochMillis, str:concat('ab', 'c') as alertKey insert into syslog_severity_critical_output;"
+    },
+    "partitionSpec": [
+      {
+        "streamId": "syslog_stream",
+        "type": "GROUPBY",
+        "columns": [
+          "dims_hostname"
+        ]
+      }
+    ],
+    "parallelismHint": 10
+  },
+  {
+    "name": "syslog_parent_alert",
+    "description": "aggregate alerts into parent alerts",
+    "inputStreams": [
+      "syslog_severity_critical_output"
+    ],
+    "outputStreams": [
+      "syslog_parent_alert_aggregate_out"
+    ],
+    "definition": {
+      "type": "siddhi",
+      "value": "from syslog_severity_critical_output#window.externalTimeBatch(timestamp, 3min) select 'AggregateAlert' as name, 'CRITICAL' as alertSeverity, namespace, facility_name, facility_code, severity_code,severity_name ,dims_hostname, dims_appname, msgid,  msg, epochMillis, str:subStr(alertKey, 0, 20) as alertKey insert into syslog_parent_alert_aggregate_out;"
+    },
+    "partitionSpec": [
+      {
+        "streamId": "syslog_severity_critical_output",
+        "type": "GROUPBY",
+        "columns": [
+          "dims_hostname"
+        ]
+      }
+    ],
+    "parallelismHint": 10
+  }
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/publishments.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/publishments.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/publishments.json
new file mode 100644
index 0000000..6d0fbf4
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/publishments.json
@@ -0,0 +1,15 @@
+[
+
+{
+  "type": "org.apache.eagle.alert.engine.publisher.impl.AlertKafkaPublisher",
+  "name":"network-syslog-publish",
+  "policyIds": ["syslog_severity_critical_check", "syslog_aggregation_alert"],
+  "dedupIntervalMin": "PT0M",
+  "properties":{
+    "kafka_broker":"localhost:9092",
+    "topic":"syslog_alerts"
+  },
+  "serializer" : "org.apache.eagle.alert.engine.publisher.impl.JsonEventSerializer"
+}
+
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/streamdefinitions.json
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/streamdefinitions.json b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/streamdefinitions.json
new file mode 100644
index 0000000..abe1f13
--- /dev/null
+++ b/eagle-core/eagle-alert-parent/eagle-alert/alert-engine/src/test/resources/loopback/streamdefinitions.json
@@ -0,0 +1,195 @@
+[
+  {
+    "streamId": "syslog_stream",
+    "dataSource": "network_syslog_datasource",
+    "description": "the data stream for syslog events",
+    "validate": false,
+    "timeseries": false,
+    "columns": [
+      {
+        "name": "dims_hostname",
+        "type": "STRING",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "facility_code",
+        "type": "INT",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "facility_name",
+        "type": "STRING",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "severity_code",
+        "type": "INT",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "severity_name",
+        "type": "STRING",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "msg",
+        "type": "STRING",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "msgid",
+        "type": "STRING",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "procid",
+        "type": "STRING",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "dims_appname",
+        "type": "STRING",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "name",
+        "type": "STRING",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "namespace",
+        "type": "STRING",
+        "defaultValue": "",
+        "required": true,
+        "description": null
+      },
+      {
+        "name": "epochMillis",
+        "type": "LONG",
+        "defaultValue": 0,
+        "required": true,
+        "description": null
+      }
+    ]
+  },
+  {
+    "streamId":"syslog_severity_critical_check",
+    "dataSource":"network_syslog_alert_datasource",
+    "description":"the data stream for syslog events",
+    "validate":false,
+    "timeseries":false,
+    "columns":[
+      {
+        "name":"dims_hostname",
+        "type":"STRING",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"facility_code",
+        "type":"INT",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"facility_name",
+        "type":"STRING",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"severity_code",
+        "type":"INT",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"severity_name",
+        "type":"STRING",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"msg",
+        "type":"STRING",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"msgid",
+        "type":"STRING",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"procid",
+        "type":"STRING",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"dims_appname",
+        "type":"STRING",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"name",
+        "type":"STRING",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"namespace",
+        "type":"STRING",
+        "defaultValue":"",
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"epochMillis",
+        "type":"LONG",
+        "defaultValue":0,
+        "required":true,
+        "description":null
+      },
+      {
+        "name":"alertKey",
+        "type":"STRING",
+        "defaultValue": "",
+        "required":true,
+        "description":null
+      }
+    ]
+  }
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-common/src/test/java/org/apache/eagle/common/agg/TestSiddhiExternalTimeBatch.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-common/src/test/java/org/apache/eagle/common/agg/TestSiddhiExternalTimeBatch.java b/eagle-core/eagle-common/src/test/java/org/apache/eagle/common/agg/TestSiddhiExternalTimeBatch.java
index e59c0c4..d664ab3 100644
--- a/eagle-core/eagle-common/src/test/java/org/apache/eagle/common/agg/TestSiddhiExternalTimeBatch.java
+++ b/eagle-core/eagle-common/src/test/java/org/apache/eagle/common/agg/TestSiddhiExternalTimeBatch.java
@@ -25,6 +25,8 @@ import org.wso2.siddhi.core.SiddhiManager;
 import org.wso2.siddhi.core.event.Event;
 import org.wso2.siddhi.core.query.output.callback.QueryCallback;
 import org.wso2.siddhi.core.stream.input.InputHandler;
+import org.wso2.siddhi.core.stream.output.StreamCallback;
+import org.wso2.siddhi.core.util.EventPrinter;
 
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -111,4 +113,5 @@ public class TestSiddhiExternalTimeBatch {
         e.setData(new Object[]{host, timestamp, "missingblocks", "site1", 14.0});
         return e;
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/ContainsIgnoreCaseExtension.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/ContainsIgnoreCaseExtension.java b/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/ContainsIgnoreCaseExtension.java
deleted file mode 100644
index d80888e..0000000
--- a/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/ContainsIgnoreCaseExtension.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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.eagle.policy.siddhi.extension;
-
-import org.wso2.siddhi.core.config.ExecutionPlanContext;
-import org.wso2.siddhi.core.exception.ExecutionPlanRuntimeException;
-import org.wso2.siddhi.core.executor.ExpressionExecutor;
-import org.wso2.siddhi.core.executor.function.FunctionExecutor;
-import org.wso2.siddhi.query.api.definition.Attribute;
-import org.wso2.siddhi.query.api.exception.ExecutionPlanValidationException;
-
-public class ContainsIgnoreCaseExtension extends FunctionExecutor {
-
-    Attribute.Type returnType = Attribute.Type.BOOL;
-
-    @Override
-    protected void init(ExpressionExecutor[] attributeExpressionExecutors, ExecutionPlanContext executionPlanContext) {
-        if (attributeExpressionExecutors.length != 2) {
-            throw new ExecutionPlanValidationException("Invalid no of arguments passed to str:containsIgnoreCase() function, required 2, " +
-                    "but found " + attributeExpressionExecutors.length);
-        }
-        if (attributeExpressionExecutors[0].getReturnType() != Attribute.Type.STRING) {
-            throw new ExecutionPlanValidationException("Invalid parameter type found for the first argument of str:containsIgnoreCase() function, " +
-                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[0].getReturnType().toString());
-        }
-        if (attributeExpressionExecutors[1].getReturnType() != Attribute.Type.STRING) {
-            throw new ExecutionPlanValidationException("Invalid parameter type found for the second argument of str:containsIgnoreCase() function, " +
-                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[1].getReturnType().toString());
-        }
-    }
-
-    @Override
-    protected Object execute(Object[] data) {
-        if (data[0] == null) {
-            throw new ExecutionPlanRuntimeException("Invalid input given to str:containsIgnoreCase() function. First argument cannot be null");
-        }
-        if (data[1] == null) {
-            throw new ExecutionPlanRuntimeException("Invalid input given to str:containsIgnoreCase() function. Second argument cannot be null");
-        }
-        String str1 = (String)data[0];
-        String str2 = (String)data[1];
-        return str1.toUpperCase().contains(str2.toUpperCase());
-    }
-
-    @Override
-    protected Object execute(Object data) {
-        return null; //Since the containsIgnoreCase function takes in 2 parameters, this method does not get called. Hence, not implemented.
-    }
-
-    @Override
-    public void start() {
-        //Nothing to start
-    }
-
-    @Override
-    public void stop() {
-        //Nothing to stop
-    }
-
-    @Override
-    public Attribute.Type getReturnType() {
-        return returnType;
-    }
-
-    @Override
-    public Object[] currentState() {
-        return new Object[]{};
-    }
-
-    @Override
-    public void restoreState(Object[] state) {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/EqualsIgnoreCaseExtension.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/EqualsIgnoreCaseExtension.java b/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/EqualsIgnoreCaseExtension.java
deleted file mode 100644
index b0c1662..0000000
--- a/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/EqualsIgnoreCaseExtension.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.eagle.policy.siddhi.extension;
-
-import org.wso2.siddhi.core.config.ExecutionPlanContext;
-import org.wso2.siddhi.core.exception.ExecutionPlanRuntimeException;
-import org.wso2.siddhi.core.executor.ConstantExpressionExecutor;
-import org.wso2.siddhi.core.executor.ExpressionExecutor;
-import org.wso2.siddhi.core.executor.function.FunctionExecutor;
-import org.wso2.siddhi.query.api.definition.Attribute;
-import org.wso2.siddhi.query.api.exception.ExecutionPlanValidationException;
-
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public class EqualsIgnoreCaseExtension extends FunctionExecutor {
-
-    Attribute.Type returnType = Attribute.Type.BOOL;
-
-    @Override
-    protected void init(ExpressionExecutor[] attributeExpressionExecutors, ExecutionPlanContext executionPlanContext) {
-        if (attributeExpressionExecutors.length != 2) {
-            throw new ExecutionPlanValidationException("Invalid no of arguments passed to str:equalsIgnoreCase() function, required 2, " +
-                    "but found " + attributeExpressionExecutors.length);
-        }
-        if (attributeExpressionExecutors[0].getReturnType() != Attribute.Type.STRING) {
-            throw new ExecutionPlanValidationException("Invalid parameter type found for the first argument of str:equalsIgnoreCase() function, " +
-                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[0].getReturnType().toString());
-        }
-        if (attributeExpressionExecutors[1].getReturnType() != Attribute.Type.STRING) {
-            throw new ExecutionPlanValidationException("Invalid parameter type found for the second argument of str:equalsIgnoreCase() function, " +
-                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[1].getReturnType().toString());
-        }
-    }
-
-    @Override
-    protected Object execute(Object[] data) {
-        if (data[0] == null) {
-            throw new ExecutionPlanRuntimeException("Invalid input given to str:equalsIgnoreCase() function. First argument cannot be null");
-        }
-        if (data[1] == null) {
-            throw new ExecutionPlanRuntimeException("Invalid input given to str:equalsIgnoreCase() function. Second argument cannot be null");
-        }
-        String str1 = (String)data[0];
-        String str2 = (String)data[1];
-        return str1.equalsIgnoreCase(str2);
-    }
-
-    @Override
-    protected Object execute(Object data) {
-        return null; //Since the equalsIgnoreCase function takes in 2 parameters, this method does not get called. Hence, not implemented.
-    }
-
-    @Override
-    public void start() {
-        //Nothing to start
-    }
-
-    @Override
-    public void stop() {
-        //Nothing to stop
-    }
-
-    @Override
-    public Attribute.Type getReturnType() {
-        return returnType;
-    }
-
-    @Override
-    public Object[] currentState() {
-        return new Object[]{};
-    }
-
-    @Override
-    public void restoreState(Object[] state) {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/RegexpIgnoreCaseFunctionExtension.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/RegexpIgnoreCaseFunctionExtension.java b/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/RegexpIgnoreCaseFunctionExtension.java
deleted file mode 100644
index fa2d03e..0000000
--- a/eagle-core/eagle-policy/eagle-policy-base/src/main/java/org/apache/eagle/policy/siddhi/extension/RegexpIgnoreCaseFunctionExtension.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.eagle.policy.siddhi.extension;
-
-import org.wso2.siddhi.core.config.ExecutionPlanContext;
-import org.wso2.siddhi.core.exception.ExecutionPlanRuntimeException;
-import org.wso2.siddhi.core.executor.ConstantExpressionExecutor;
-import org.wso2.siddhi.core.executor.ExpressionExecutor;
-import org.wso2.siddhi.extension.string.RegexpFunctionExtension;
-import org.wso2.siddhi.query.api.definition.Attribute;
-import org.wso2.siddhi.query.api.exception.ExecutionPlanValidationException;
-
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-/**
- * regexpIgnoreCase(string, regex)
- * Tells whether or not this 'string' matches the given regular expression 'regex'.
- * Accept Type(s): (STRING,STRING)
- * Return Type(s): BOOLEAN
- */
-public class RegexpIgnoreCaseFunctionExtension extends RegexpFunctionExtension {
-
-    //state-variables
-    boolean isRegexConstant = false;
-    String regexConstant;
-    Pattern patternConstant;
-
-    @Override
-    protected void init(ExpressionExecutor[] attributeExpressionExecutors, ExecutionPlanContext executionPlanContext) {
-        if (attributeExpressionExecutors.length != 2) {
-            throw new ExecutionPlanValidationException("Invalid no of arguments passed to str:regexpIgnoreCase() function, required 2, " +
-                    "but found " + attributeExpressionExecutors.length);
-        }
-        if (attributeExpressionExecutors[0].getReturnType() != Attribute.Type.STRING) {
-            throw new ExecutionPlanValidationException("Invalid parameter type found for the first argument of str:regexpIgnoreCase() function, " +
-                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[0].getReturnType().toString());
-        }
-        if (attributeExpressionExecutors[1].getReturnType() != Attribute.Type.STRING) {
-            throw new ExecutionPlanValidationException("Invalid parameter type found for the second argument of str:regexpIgnoreCase() function, " +
-                    "required "+ Attribute.Type.STRING+", but found "+attributeExpressionExecutors[1].getReturnType().toString());
-        }
-        if(attributeExpressionExecutors[1] instanceof ConstantExpressionExecutor){
-            isRegexConstant = true;
-            regexConstant = (String) ((ConstantExpressionExecutor) attributeExpressionExecutors[1]).getValue();
-            patternConstant = Pattern.compile(regexConstant, Pattern.CASE_INSENSITIVE);
-        }
-    }
-
-    @Override
-    protected Object execute(Object[] data) {
-        String regex;
-        Pattern pattern;
-        Matcher matcher;
-
-        if (data[0] == null) {
-            throw new ExecutionPlanRuntimeException("Invalid input given to str:regexpIgnoreCase() function. First argument cannot be null");
-        }
-        if (data[1] == null) {
-            throw new ExecutionPlanRuntimeException("Invalid input given to str:regexpIgnoreCase() function. Second argument cannot be null");
-        }
-        String source = (String) data[0];
-
-        if(!isRegexConstant){
-            regex = (String) data[1];
-            pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
-            matcher = pattern.matcher(source);
-            return matcher.matches();
-
-        } else {
-            matcher = patternConstant.matcher(source);
-            return matcher.matches();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/34e77caf/eagle-core/eagle-policy/eagle-policy-base/src/main/resources/str.siddhiext
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-policy/eagle-policy-base/src/main/resources/str.siddhiext b/eagle-core/eagle-policy/eagle-policy-base/src/main/resources/str.siddhiext
deleted file mode 100644
index 479cdb0..0000000
--- a/eagle-core/eagle-policy/eagle-policy-base/src/main/resources/str.siddhiext
+++ /dev/null
@@ -1,40 +0,0 @@
-#
-# 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.
-#
-
-charAt=org.wso2.siddhi.extension.string.CharAtFunctionExtension
-coalesce=org.wso2.siddhi.extension.string.CoalesceFunctionExtension
-concat=org.wso2.siddhi.extension.string.ConcatFunctionExtension
-length=org.wso2.siddhi.extension.string.LengthFunctionExtension
-lower=org.wso2.siddhi.extension.string.LowerFunctionExtension
-regexp=org.wso2.siddhi.extension.string.RegexpFunctionExtension
-repeat=org.wso2.siddhi.extension.string.RepeatFunctionExtension
-replaceAll=org.wso2.siddhi.extension.string.ReplaceAllFunctionExtension
-replaceFirst=org.wso2.siddhi.extension.string.ReplaceFirstFunctionExtension
-reverse=org.wso2.siddhi.extension.string.ReverseFunctionExtension
-strcmp=org.wso2.siddhi.extension.string.StrcmpFunctionExtension
-substr=org.wso2.siddhi.extension.string.SubstrFunctionExtension
-trim=org.wso2.siddhi.extension.string.TrimFunctionExtension
-upper=org.wso2.siddhi.extension.string.UpperFunctionExtension
-hex=org.wso2.siddhi.extension.string.HexFunctionExtension
-unhex=org.wso2.siddhi.extension.string.UnhexFunctionExtension
-contains=org.wso2.siddhi.extension.string.ContainsFunctionExtension
-
-# Eagle Siddhi Extension
-equalsIgnoreCase=org.apache.eagle.policy.siddhi.extension.EqualsIgnoreCaseExtension
-containsIgnoreCase=org.apache.eagle.policy.siddhi.extension.ContainsIgnoreCaseExtension
-regexpIgnoreCase=org.apache.eagle.policy.siddhi.extension.RegexpIgnoreCaseFunctionExtension
-