You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2019/11/08 16:25:18 UTC

[nifi] branch master updated: NIFI-6854 - Added option to ignore condition errors in rules. Test correction

This is an automated email from the ASF dual-hosted git repository.

mattyb149 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new a37b57e  NIFI-6854 - Added option to ignore condition errors in rules. Test correction
a37b57e is described below

commit a37b57e96b7e0d1d048f0d4344b1cec692216617
Author: Yolanda M. Davis <yo...@gmail.com>
AuthorDate: Thu Nov 7 17:21:48 2019 -0500

    NIFI-6854 - Added option to ignore condition errors in rules. Test correction
    
    NIFI-6854 - change warning to debug
    
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #3876
---
 .../nifi-easyrules-service/pom.xml                 |   1 +
 .../org/apache/nifi/rules/RulesMVELCondition.java  |  54 ++++++++
 .../org/apache/nifi/rules/RulesSPELCondition.java  |  75 +++++++++++
 .../nifi/rules/engine/EasyRulesEngineService.java  |  21 ++-
 .../org/apache/nifi/rules/TestRulesFactory.java    |   2 +-
 .../rules/engine/TestEasyRulesEngineService.java   | 148 ++++++++++++++++++++-
 ...st_spel_rules.json => test_bad_spel_rules.json} |   2 +-
 .../src/test/resources/test_mvel_rules.json        |   2 +-
 .../src/test/resources/test_mvel_rules.yml         |   2 +-
 .../src/test/resources/test_nifi_rules.json        |   2 +-
 .../src/test/resources/test_nifi_rules.yml         |   2 +-
 .../src/test/resources/test_spel_rules.json        |   2 +-
 .../src/test/resources/test_spel_rules.yml         |   2 +-
 13 files changed, 297 insertions(+), 18 deletions(-)

diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml
index 7e98f34..801cbb3 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml
@@ -79,6 +79,7 @@
                         <exclude>src/test/resources/test_mvel_rules.json</exclude>
                         <exclude>src/test/resources/test_mvel_rules.yml</exclude>
                         <exclude>src/test/resources/test_spel_rules.json</exclude>
+                        <exclude>src/test/resources/test_bad_spel_rules.json</exclude>
                         <exclude>src/test/resources/test_spel_rules.yml</exclude>
                     </excludes>
                 </configuration>
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java
new file mode 100644
index 0000000..7bd615f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java
@@ -0,0 +1,54 @@
+/*
+ * 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.nifi.rules;
+
+import org.jeasy.rules.api.Condition;
+import org.jeasy.rules.api.Facts;
+import org.jeasy.rules.mvel.MVELCondition;
+import org.mvel2.MVEL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+
+public class RulesMVELCondition implements Condition {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(MVELCondition.class);
+    private String expression;
+    private Serializable compiledExpression;
+    private boolean ignoreConditionErrors;
+
+    public RulesMVELCondition(String expression, boolean ignoreConditionErrors) {
+        this.expression = expression;
+        this.compiledExpression = MVEL.compileExpression(expression);
+        this.ignoreConditionErrors = ignoreConditionErrors;
+    }
+
+    public boolean evaluate(Facts facts) {
+        try {
+            return (Boolean)MVEL.executeExpression(this.compiledExpression, facts.asMap());
+        } catch (Exception ex) {
+            if(ignoreConditionErrors) {
+                LOGGER.debug("Unable to evaluate expression: '" + this.expression + "' on facts: " + facts, ex);
+                return false;
+            } else{
+                throw ex;
+            }
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesSPELCondition.java b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesSPELCondition.java
new file mode 100644
index 0000000..e753f93
--- /dev/null
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesSPELCondition.java
@@ -0,0 +1,75 @@
+/*
+ * 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.nifi.rules;
+
+import org.jeasy.rules.api.Condition;
+import org.jeasy.rules.api.Facts;
+import org.jeasy.rules.spel.SpELCondition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.expression.Expression;
+import org.springframework.expression.ExpressionParser;
+import org.springframework.expression.ParserContext;
+import org.springframework.expression.spel.standard.SpelExpressionParser;
+import org.springframework.expression.spel.support.StandardEvaluationContext;
+
+public class RulesSPELCondition implements Condition {
+    private static final Logger LOGGER = LoggerFactory.getLogger(SpELCondition.class);
+    private final ExpressionParser parser = new SpelExpressionParser();
+    private String expression;
+    private Expression compiledExpression;
+    private boolean ignoreConditionErrors;
+
+
+    public RulesSPELCondition(String expression) {
+        this.expression = expression;
+        this.compiledExpression = this.parser.parseExpression(expression);
+    }
+
+    public RulesSPELCondition(String expression, ParserContext parserContext) {
+        this.expression = expression;
+        this.compiledExpression = this.parser.parseExpression(expression, parserContext);
+    }
+
+    public RulesSPELCondition(String expression, boolean ignoreConditionErrors) {
+        this.expression = expression;
+        this.compiledExpression =  this.parser.parseExpression(expression);
+        this.ignoreConditionErrors = ignoreConditionErrors;
+    }
+
+    public RulesSPELCondition(String expression, ParserContext parserContext, boolean ignoreConditionErrors) {
+        this.expression = expression;
+        this.compiledExpression =  this.parser.parseExpression(expression, parserContext);
+        this.ignoreConditionErrors = ignoreConditionErrors;
+    }
+
+    public boolean evaluate(Facts facts) {
+        try {
+            StandardEvaluationContext context = new StandardEvaluationContext();
+            context.setRootObject(facts.asMap());
+            context.setVariables(facts.asMap());
+            return this.compiledExpression.getValue(context, Boolean.class);
+        } catch (Exception ex) {
+            if(ignoreConditionErrors) {
+                LOGGER.debug("Unable to evaluate expression: '" + this.expression + "' on facts: " + facts, ex);
+                return false;
+            } else{
+                throw ex;
+            }
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/engine/EasyRulesEngineService.java b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/engine/EasyRulesEngineService.java
index 1318009..66f69ae 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/engine/EasyRulesEngineService.java
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/engine/EasyRulesEngineService.java
@@ -31,14 +31,14 @@ import org.apache.nifi.rules.Action;
 import org.apache.nifi.rules.ActionHandler;
 import org.apache.nifi.rules.Rule;
 import org.apache.nifi.rules.RulesFactory;
+import org.apache.nifi.rules.RulesMVELCondition;
+import org.apache.nifi.rules.RulesSPELCondition;
 import org.jeasy.rules.api.Condition;
 import org.jeasy.rules.api.Facts;
 import org.jeasy.rules.api.RuleListener;
 import org.jeasy.rules.api.Rules;
 import org.jeasy.rules.core.DefaultRulesEngine;
 import org.jeasy.rules.core.RuleBuilder;
-import org.jeasy.rules.mvel.MVELCondition;
-import org.jeasy.rules.spel.SpELCondition;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -89,10 +89,21 @@ public class EasyRulesEngineService  extends AbstractControllerService implement
             .defaultValue(NIFI.getValue())
             .build();
 
+    static final PropertyDescriptor IGNORE_CONDITION_ERRORS = new PropertyDescriptor.Builder()
+            .name("rules-ignore-condition-errors")
+            .displayName("Ignore Condition Errors")
+            .description("When set to true, rules engine will ignore errors for any rule that encounters issues " +
+                    "when compiling rule conditions (including syntax errors and/or missing facts). Rule will simply return as false " +
+                    "and engine will continue with execution.")
+            .required(true)
+            .defaultValue("false")
+            .allowableValues("true", "false")
+            .build();
 
     protected List<PropertyDescriptor> properties;
     protected volatile List<Rule> rules;
     protected volatile String rulesFileFormat;
+    private boolean ignoreConditionErrors;
 
     @Override
     protected void init(ControllerServiceInitializationContext config) throws InitializationException {
@@ -101,6 +112,7 @@ public class EasyRulesEngineService  extends AbstractControllerService implement
         properties.add(RULES_FILE_TYPE);
         properties.add(RULES_FILE_PATH);
         properties.add(RULES_FILE_FORMAT);
+        properties.add(IGNORE_CONDITION_ERRORS);
         this.properties = Collections.unmodifiableList(properties);
     }
 
@@ -114,6 +126,7 @@ public class EasyRulesEngineService  extends AbstractControllerService implement
         final String rulesFile = context.getProperty(RULES_FILE_PATH).getValue();
         final String rulesFileType = context.getProperty(RULES_FILE_TYPE).getValue();
         rulesFileFormat = context.getProperty(RULES_FILE_FORMAT).getValue();
+        ignoreConditionErrors = context.getProperty(IGNORE_CONDITION_ERRORS).asBoolean();
         try{
             rules = RulesFactory.createRules(rulesFile, rulesFileType, rulesFileFormat);
         } catch (Exception fex){
@@ -129,7 +142,7 @@ public class EasyRulesEngineService  extends AbstractControllerService implement
     @Override
     public List<Action> fireRules(Map<String, Object> facts) {
         final List<Action> actions = new ArrayList<>();
-        if(rules == null){
+        if (rules == null || facts == null || facts.isEmpty()) {
             return null;
         }else {
             org.jeasy.rules.api.Rules easyRules = convertToEasyRules(rules, (action, eventFacts) ->
@@ -149,7 +162,7 @@ public class EasyRulesEngineService  extends AbstractControllerService implement
         rules.forEach(rule -> {
             RuleBuilder ruleBuilder = new RuleBuilder();
             Condition condition = rulesFileFormat.equalsIgnoreCase(SPEL.getValue())
-                                 ? new SpELCondition(rule.getCondition()): new MVELCondition(rule.getCondition());
+                                 ? new RulesSPELCondition(rule.getCondition(), ignoreConditionErrors): new RulesMVELCondition(rule.getCondition(), ignoreConditionErrors);
             ruleBuilder.name(rule.getName())
                     .description(rule.getDescription())
                     .priority(rule.getPriority())
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/java/org/apache/nifi/rules/TestRulesFactory.java b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/java/org/apache/nifi/rules/TestRulesFactory.java
index 9cc8afd..ded5ea6 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/java/org/apache/nifi/rules/TestRulesFactory.java
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/java/org/apache/nifi/rules/TestRulesFactory.java
@@ -128,7 +128,7 @@ public class TestRulesFactory {
                 && rule1.getPriority() == 1 && rule1.getCondition().equals("predictedQueuedCount > 50");
 
         checkDiagnostic = rule2.getName().equals("Time To Back Pressure") && rule2.getDescription().equals("Back pressure time less than 5 minutes")
-                && rule2.getPriority() == 2 && rule2.getCondition().equals("predictedTimeToBytesBackpressureMillis >= 300000") && checkDiagnostic;
+                && rule2.getPriority() == 2 && rule2.getCondition().equals("predictedTimeToBytesBackpressureMillis < 300000 && predictedTimeToBytesBackpressureMillis >= 0") && checkDiagnostic;
 
         return checkDiagnostic;
 
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/java/org/apache/nifi/rules/engine/TestEasyRulesEngineService.java b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/java/org/apache/nifi/rules/engine/TestEasyRulesEngineService.java
index 223c1b4..0adad8b 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/java/org/apache/nifi/rules/engine/TestEasyRulesEngineService.java
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/java/org/apache/nifi/rules/engine/TestEasyRulesEngineService.java
@@ -21,6 +21,7 @@ import org.apache.nifi.rules.Action;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Test;
+import org.mvel2.PropertyAccessException;
 
 import java.io.IOException;
 import java.util.HashMap;
@@ -29,6 +30,7 @@ import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
 
 public class TestEasyRulesEngineService {
 
@@ -44,7 +46,7 @@ public class TestEasyRulesEngineService {
         runner.assertValid(service);
         Map<String, Object> facts = new HashMap<>();
         facts.put("predictedQueuedCount",60);
-        facts.put("predictedTimeToBytesBackpressureMillis",300000);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
         List<Action> actions = service.fireRules(facts);
         assertNotNull(actions);
         assertEquals(actions.size(), 3);
@@ -62,7 +64,7 @@ public class TestEasyRulesEngineService {
         runner.assertValid(service);
         Map<String, Object> facts = new HashMap<>();
         facts.put("predictedQueuedCount",60);
-        facts.put("predictedTimeToBytesBackpressureMillis",300000);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
         List<Action> actions = service.fireRules(facts);
         assertNotNull(actions);
         assertEquals(actions.size(), 2);
@@ -80,7 +82,7 @@ public class TestEasyRulesEngineService {
         runner.assertValid(service);
         Map<String, Object> facts = new HashMap<>();
         facts.put("predictedQueuedCount",60);
-        facts.put("predictedTimeToBytesBackpressureMillis",300000);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
         List<Action> actions = service.fireRules(facts);
         assertNotNull(actions);
         assertEquals(actions.size(), 2);
@@ -98,7 +100,7 @@ public class TestEasyRulesEngineService {
         runner.assertValid(service);
         Map<String, Object> facts = new HashMap<>();
         facts.put("predictedQueuedCount",60);
-        facts.put("predictedTimeToBytesBackpressureMillis",300000);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
         List<Action> actions = service.fireRules(facts);
         assertNotNull(actions);
         assertEquals(actions.size(), 2);
@@ -116,7 +118,7 @@ public class TestEasyRulesEngineService {
         runner.assertValid(service);
         Map<String, Object> facts = new HashMap<>();
         facts.put("predictedQueuedCount",60);
-        facts.put("predictedTimeToBytesBackpressureMillis",300000);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
         List<Action> actions = service.fireRules(facts);
         assertNotNull(actions);
         assertEquals(actions.size(), 2);
@@ -134,12 +136,146 @@ public class TestEasyRulesEngineService {
         runner.assertValid(service);
         Map<String, Object> facts = new HashMap<>();
         facts.put("predictedQueuedCount",60);
-        facts.put("predictedTimeToBytesBackpressureMillis",300000);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
         List<Action> actions = service.fireRules(facts);
         assertNotNull(actions);
         assertEquals(actions.size(), 2);
     }
 
+    @Test
+    public void testIgnoreConditionErrorsFalseNIFI() throws InitializationException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final RulesEngineService service = new MockEasyRulesEngineService();
+        runner.addControllerService("easy-rules-engine-service-test",service);
+        runner.setProperty(service, EasyRulesEngineService.RULES_FILE_PATH, "src/test/resources/test_nifi_rules.json");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_TYPE, "JSON");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_FORMAT, "NIFI");
+        runner.setProperty(service,EasyRulesEngineService.IGNORE_CONDITION_ERRORS,"false");
+        runner.enableControllerService(service);
+        runner.assertValid(service);
+        Map<String, Object> facts = new HashMap<>();
+        facts.put("fakeMetric",60);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
+        try {
+            service.fireRules(facts);
+            fail("Expected exception to be thrown");
+        }catch (PropertyAccessException pae){
+            assert true;
+        }
+    }
+
+    @Test
+    public void testIgnoreConditionErrorsTrueNIFI() throws InitializationException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final RulesEngineService service = new MockEasyRulesEngineService();
+        runner.addControllerService("easy-rules-engine-service-test",service);
+        runner.setProperty(service, EasyRulesEngineService.RULES_FILE_PATH, "src/test/resources/test_nifi_rules.json");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_TYPE, "JSON");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_FORMAT, "NIFI");
+        runner.setProperty(service,EasyRulesEngineService.IGNORE_CONDITION_ERRORS,"true");
+        runner.enableControllerService(service);
+        runner.assertValid(service);
+        Map<String, Object> facts = new HashMap<>();
+        facts.put("fakeMetric",60);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
+        try {
+            List<Action> actions = service.fireRules(facts);
+            assertNotNull(actions);
+            assertEquals(actions.size(), 1);
+        }catch (PropertyAccessException pae){
+            fail();
+        }
+    }
+
+    @Test
+    public void testIgnoreConditionErrorsFalseMVEL() throws InitializationException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final RulesEngineService service = new MockEasyRulesEngineService();
+        runner.addControllerService("easy-rules-engine-service-test",service);
+        runner.setProperty(service, EasyRulesEngineService.RULES_FILE_PATH, "src/test/resources/test_mvel_rules.json");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_TYPE, "JSON");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_FORMAT, "MVEL");
+        runner.setProperty(service,EasyRulesEngineService.IGNORE_CONDITION_ERRORS,"false");
+        runner.enableControllerService(service);
+        runner.assertValid(service);
+        Map<String, Object> facts = new HashMap<>();
+        facts.put("fakeMetric",60);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
+        try {
+            service.fireRules(facts);
+            fail("Expected exception to be thrown");
+        }catch (PropertyAccessException pae){
+            assert true;
+        }
+    }
+
+    @Test
+    public void testIgnoreConditionErrorsTrueMVEL() throws InitializationException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final RulesEngineService service = new MockEasyRulesEngineService();
+        runner.addControllerService("easy-rules-engine-service-test",service);
+        runner.setProperty(service, EasyRulesEngineService.RULES_FILE_PATH, "src/test/resources/test_mvel_rules.json");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_TYPE, "JSON");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_FORMAT, "MVEL");
+        runner.setProperty(service,EasyRulesEngineService.IGNORE_CONDITION_ERRORS,"true");
+        runner.enableControllerService(service);
+        runner.assertValid(service);
+        Map<String, Object> facts = new HashMap<>();
+        facts.put("fakeMetric",60);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
+        try {
+            List<Action> actions = service.fireRules(facts);
+            assertNotNull(actions);
+            assertEquals(actions.size(), 1);
+        }catch (PropertyAccessException pae){
+            fail();
+        }
+    }
+
+    @Test
+    public void testIgnoreConditionErrorsFalseSPEL() throws InitializationException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final RulesEngineService service = new MockEasyRulesEngineService();
+        runner.addControllerService("easy-rules-engine-service-test",service);
+        runner.setProperty(service, EasyRulesEngineService.RULES_FILE_PATH, "src/test/resources/test_bad_spel_rules.json");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_TYPE, "JSON");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_FORMAT, "SPEL");
+        runner.setProperty(service,EasyRulesEngineService.IGNORE_CONDITION_ERRORS,"false");
+        runner.enableControllerService(service);
+        runner.assertValid(service);
+        Map<String, Object> facts = new HashMap<>();
+        facts.put("fakeMetric",60);
+        facts.put("fakeMetric2",299999);
+        try {
+            service.fireRules(facts);
+            fail("Expected exception to be thrown");
+        }catch (Exception pae){
+            assert true;
+        }
+    }
+
+    @Test
+    public void testIgnoreConditionErrorsTrueSPEL() throws InitializationException, IOException {
+        final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+        final RulesEngineService service = new MockEasyRulesEngineService();
+        runner.addControllerService("easy-rules-engine-service-test",service);
+        runner.setProperty(service, EasyRulesEngineService.RULES_FILE_PATH, "src/test/resources/test_bad_spel_rules.json");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_TYPE, "JSON");
+        runner.setProperty(service,EasyRulesEngineService.RULES_FILE_FORMAT, "SPEL");
+        runner.setProperty(service,EasyRulesEngineService.IGNORE_CONDITION_ERRORS,"true");
+        runner.enableControllerService(service);
+        runner.assertValid(service);
+        Map<String, Object> facts = new HashMap<>();
+        facts.put("predictedQueuedCount",60);
+        facts.put("predictedTimeToBytesBackpressureMillis",299999);
+        try {
+            List<Action> actions = service.fireRules(facts);
+            assertNotNull(actions);
+            assertEquals(actions.size(), 1);
+        }catch (Exception pae){
+            fail();
+        }
+    }
     private class MockEasyRulesEngineService extends EasyRulesEngineService {
 
     }
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.json b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_bad_spel_rules.json
similarity index 79%
copy from nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.json
copy to nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_bad_spel_rules.json
index 4e50bbd..9f8df70 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.json
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_bad_spel_rules.json
@@ -10,7 +10,7 @@
     "name": "Time To Back Pressure",
     "description": "Back pressure time less than 5 minutes",
     "priority": 2,
-    "condition": "#predictedTimeToBytesBackpressureMillis >= 300000",
+    "condition": "predictedTimeToBytesBackpressureMillis < 300000 && #predictedTimeToBytesBackpressureMillis >= 0",
     "actions": ["'System is approaching backpressure! Predicted time left: ' + #predictedTimeToBytesBackpressureMillis"]
   }
 ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_mvel_rules.json b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_mvel_rules.json
index 190d5e7..a3fc01c 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_mvel_rules.json
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_mvel_rules.json
@@ -10,7 +10,7 @@
     "name": "Time To Back Pressure",
     "description": "Back pressure time less than 5 minutes",
     "priority": 2,
-    "condition": "predictedTimeToBytesBackpressureMillis >= 300000",
+    "condition": "predictedTimeToBytesBackpressureMillis < 300000 && predictedTimeToBytesBackpressureMillis >= 0",
     "actions": ["System.out.println(\"Back Pressure prediction less than 5 minutes!\")"]
   }
 ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_mvel_rules.yml b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_mvel_rules.yml
index f3d9150..0face66 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_mvel_rules.yml
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_mvel_rules.yml
@@ -9,6 +9,6 @@ actions:
 name: "Time To Back Pressure"
 description: "Back pressure time less than 5 minutes"
 priority: 2
-condition: "predictedTimeToBytesBackpressureMillis >= 300000"
+condition: "predictedTimeToBytesBackpressureMillis < 300000 && predictedTimeToBytesBackpressureMillis >= 0"
 actions:
   - "System.out.println(\"Back Pressure prediction less than 5 minutes!\")"
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_nifi_rules.json b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_nifi_rules.json
index 33554af..2486fd9 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_nifi_rules.json
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_nifi_rules.json
@@ -19,7 +19,7 @@
 "name": "Time To Back Pressure",
 "description": "Back pressure time less than 5 minutes",
 "priority": 2,
-"condition": "predictedTimeToBytesBackpressureMillis >= 300000",
+"condition": "predictedTimeToBytesBackpressureMillis < 300000 && predictedTimeToBytesBackpressureMillis >= 0",
 "actions": [
 {
 "type": "LOG",
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_nifi_rules.yml b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_nifi_rules.yml
index ebcec61..3d0d0dd 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_nifi_rules.yml
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_nifi_rules.yml
@@ -14,7 +14,7 @@ facts:
 name: "Time To Back Pressure"
 description: "Back pressure time less than 5 minutes"
 priority: 2
-condition: "predictedTimeToBytesBackpressureMillis >= 300000"
+condition: "predictedTimeToBytesBackpressureMillis < 300000 && predictedTimeToBytesBackpressureMillis >= 0"
 actions:
   - type: "LOG"
     attributes:
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.json b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.json
index 4e50bbd..f041413 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.json
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.json
@@ -10,7 +10,7 @@
     "name": "Time To Back Pressure",
     "description": "Back pressure time less than 5 minutes",
     "priority": 2,
-    "condition": "#predictedTimeToBytesBackpressureMillis >= 300000",
+    "condition": "#predictedTimeToBytesBackpressureMillis < 300000 && #predictedTimeToBytesBackpressureMillis >= 0",
     "actions": ["'System is approaching backpressure! Predicted time left: ' + #predictedTimeToBytesBackpressureMillis"]
   }
 ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.yml b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.yml
index ecbf612..f13a32a 100644
--- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.yml
+++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/test/resources/test_spel_rules.yml
@@ -9,6 +9,6 @@ actions:
 name: "Time To Back Pressure"
 description: "Back pressure time less than 5 minutes"
 priority: 2
-condition: "#predictedTimeToBytesBackpressureMillis >= 300000"
+condition: "#predictedTimeToBytesBackpressureMillis < 300000 && #predictedTimeToBytesBackpressureMillis >= 0"
 actions:
   - "System.out.println(\"Back Pressure prediction less than 5 minutes!\")"
\ No newline at end of file