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 2016/10/24 14:34:08 UTC

[2/4] nifi git commit: NIFI-1662 adding Expression Language decimal support

http://git-wip-us.apache.org/repos/asf/nifi/blob/94ab9990/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
index 7c9278f..be275e5 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java
@@ -25,27 +25,32 @@ import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 
 public class PlusEvaluator extends NumberEvaluator {
 
-    private final Evaluator<Long> subject;
-    private final Evaluator<Long> plusValue;
+    private final Evaluator<Number> subject;
+    private final Evaluator<Number> plusValue;
 
-    public PlusEvaluator(final Evaluator<Long> subject, final Evaluator<Long> plusValue) {
+    public PlusEvaluator(final Evaluator<Number> subject, final Evaluator<Number> plusValue) {
         this.subject = subject;
         this.plusValue = plusValue;
     }
 
     @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final Long subjectValue = subject.evaluate(attributes).getValue();
+    public QueryResult<Number> evaluate(final Map<String, String> attributes) {
+        final Number subjectValue = subject.evaluate(attributes).getValue();
         if (subjectValue == null) {
             return new NumberQueryResult(null);
         }
 
-        final Long plus = plusValue.evaluate(attributes).getValue();
+        final Number plus = plusValue.evaluate(attributes).getValue();
         if (plus == null) {
             return new NumberQueryResult(null);
         }
 
-        final long result = subjectValue + plus;
+        final Number result;
+        if (subjectValue instanceof Double || plus instanceof Double){
+            result = subjectValue.doubleValue() + plus.doubleValue();
+        } else {
+            result = subjectValue.longValue() + plus.longValue();
+        }
         return new NumberQueryResult(result);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/94ab9990/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/RandomNumberGeneratorEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/RandomNumberGeneratorEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/RandomNumberGeneratorEvaluator.java
index 84be971..7272c04 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/RandomNumberGeneratorEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/RandomNumberGeneratorEvaluator.java
@@ -20,18 +20,18 @@ import java.util.Map;
 import java.util.Random;
 
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.WholeNumberEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.WholeNumberQueryResult;
 
-public class RandomNumberGeneratorEvaluator extends NumberEvaluator {
+public class RandomNumberGeneratorEvaluator extends WholeNumberEvaluator {
 
     private static final Random RNG = new Random();
 
 
     @Override
     public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        return new NumberQueryResult(Math.abs(RNG.nextLong()));
+        return new WholeNumberQueryResult(Math.abs(RNG.nextLong()));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/94ab9990/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java
deleted file mode 100644
index 736f0b9..0000000
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java
+++ /dev/null
@@ -1,45 +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.nifi.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class ToNumberEvaluator extends NumberEvaluator {
-
-    private final Evaluator<String> subject;
-
-    public ToNumberEvaluator(final Evaluator<String> subject) {
-        this.subject = subject;
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        final String subjectValue = subject.evaluate(attributes).getValue();
-        return new NumberQueryResult(subjectValue == null || subjectValue.trim().isEmpty() ? null : Long.valueOf(subjectValue));
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return subject;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/94ab9990/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java
deleted file mode 100644
index d7569e0..0000000
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java
+++ /dev/null
@@ -1,44 +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.nifi.attribute.expression.language.evaluation.literals;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-
-public class NumberLiteralEvaluator extends NumberEvaluator {
-
-    private final long literal;
-
-    public NumberLiteralEvaluator(final String value) {
-        this.literal = Long.parseLong(value);
-    }
-
-    @Override
-    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
-        return new NumberQueryResult(literal);
-    }
-
-    @Override
-    public Evaluator<?> getSubjectEvaluator() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/94ab9990/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/WholeNumberLiteralEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/WholeNumberLiteralEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/WholeNumberLiteralEvaluator.java
new file mode 100644
index 0000000..7ca6255
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/WholeNumberLiteralEvaluator.java
@@ -0,0 +1,44 @@
+/*
+ * 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.attribute.expression.language.evaluation.literals;
+
+import java.util.Map;
+
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.WholeNumberEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.WholeNumberQueryResult;
+
+public class WholeNumberLiteralEvaluator extends WholeNumberEvaluator {
+
+    private final long literal;
+
+    public WholeNumberLiteralEvaluator(final String value) {
+        this.literal = Long.parseLong(value);
+    }
+
+    @Override
+    public QueryResult<Long> evaluate(final Map<String, String> attributes) {
+        return new WholeNumberQueryResult(literal);
+    }
+
+    @Override
+    public Evaluator<?> getSubjectEvaluator() {
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/94ab9990/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
index f3fb21d..5026846 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/reduce/CountEvaluator.java
@@ -19,12 +19,12 @@ package org.apache.nifi.attribute.expression.language.evaluation.reduce;
 import java.util.Map;
 
 import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.WholeNumberEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.WholeNumberQueryResult;
 import org.apache.nifi.expression.AttributeExpression.ResultType;
 
-public class CountEvaluator extends NumberEvaluator implements ReduceEvaluator<Long> {
+public class CountEvaluator extends WholeNumberEvaluator implements ReduceEvaluator<Long> {
 
     private final Evaluator<?> subjectEvaluator;
     private long count = 0L;
@@ -37,15 +37,15 @@ public class CountEvaluator extends NumberEvaluator implements ReduceEvaluator<L
     public QueryResult<Long> evaluate(final Map<String, String> attributes) {
         final QueryResult<?> result = subjectEvaluator.evaluate(attributes);
         if (result.getValue() == null) {
-            return new NumberQueryResult(count);
+            return new WholeNumberQueryResult(count);
         }
 
         if (result.getResultType() == ResultType.BOOLEAN && ((Boolean) result.getValue()).equals(Boolean.FALSE)) {
-            return new NumberQueryResult(count);
+            return new WholeNumberQueryResult(count);
         }
 
         count++;
-        return new NumberQueryResult(count);
+        return new WholeNumberQueryResult(count);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi/blob/94ab9990/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/util/NumberParsing.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/util/NumberParsing.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/util/NumberParsing.java
new file mode 100644
index 0000000..c79c922
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/util/NumberParsing.java
@@ -0,0 +1,84 @@
+/*
+ * 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.attribute.expression.language.evaluation.util;
+
+import java.util.regex.Pattern;
+
+public class NumberParsing {
+
+    private static final Pattern NUMBER_PATTERN = Pattern.compile("-?\\d+");
+
+    public static enum ParseResultType {
+        NOT_NUMBER, WHOLE_NUMBER, DECIMAL;
+    }
+    private static final String Digits     = "(\\p{Digit}+)";
+
+    // Double regex according to Oracle documentation: http://docs.oracle.com/javase/6/docs/api/java/lang/Double.html#valueOf%28java.lang.String%29
+    private static final String HexDigits  = "(\\p{XDigit}+)";
+    // an exponent is 'e' or 'E' followed by an optionally
+    // signed decimal integer.
+    private static final String Exp        = "[eE][+-]?"+Digits;
+    private static final String fpRegex    =
+            ("[\\x00-\\x20]*"+  // Optional leading "whitespace"
+                    "[+-]?(" + // Optional sign character
+                    "NaN|" +           // "NaN" string
+                    "Infinity|" +      // "Infinity" string
+
+                    // A decimal floating-point string representing a finite positive
+                    // number without a leading sign has at most five basic pieces:
+                    // Digits . Digits ExponentPart FloatTypeSuffix
+                    //
+                    // Since this method allows integer-only strings as input
+                    // in addition to strings of floating-point literals, the
+                    // two sub-patterns below are simplifications of the grammar
+                    // productions from the Java Language Specification, 2nd
+                    // edition, section 3.10.2.
+
+                    // Digits ._opt Digits_opt ExponentPart_opt FloatTypeSuffix_opt
+                    "((("+Digits+"(\\.)?("+Digits+"?)("+Exp+")?)|"+
+
+                    // . Digits ExponentPart_opt FloatTypeSuffix_opt
+                    "(\\.("+Digits+")("+Exp+")?)|"+
+
+                    // Hexadecimal strings
+                    "((" +
+                    // 0[xX] HexDigits ._opt BinaryExponent FloatTypeSuffix_opt
+                    "(0[xX]" + HexDigits + "(\\.)?)|" +
+
+                    // 0[xX] HexDigits_opt . HexDigits BinaryExponent FloatTypeSuffix_opt
+                    "(0[xX]" + HexDigits + "?(\\.)" + HexDigits + ")" +
+
+                    ")[pP][+-]?" + Digits + "))" +
+                    "[fFdD]?))" +
+                    "[\\x00-\\x20]*");// Optional trailing "whitespace"
+
+    private static final Pattern DOUBLE_PATTERN = Pattern.compile(fpRegex);
+
+    private NumberParsing(){
+    }
+
+    public static ParseResultType parse(String input){
+        if (NUMBER_PATTERN.matcher(input).matches()) {
+            return ParseResultType.WHOLE_NUMBER;
+        } else if (DOUBLE_PATTERN.matcher(input).matches()) {
+            return ParseResultType.DECIMAL;
+        } else {
+            return ParseResultType.NOT_NUMBER;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/94ab9990/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
index 3b22e13..0eefab5 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
@@ -37,6 +37,7 @@ import java.util.Locale;
 import java.util.Map;
 
 import org.apache.nifi.attribute.expression.language.Query.Range;
+import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
 import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
 import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
@@ -194,7 +195,7 @@ public class TestQuery {
         attributes.put("dateTime", "2013/11/18 10:22:27.678");
 
         final QueryResult<?> result = query.evaluate(attributes);
-        assertEquals(ResultType.NUMBER, result.getResultType());
+        assertEquals(ResultType.WHOLE_NUMBER, result.getResultType());
         assertEquals(1384788147678L, result.getValue());
     }
 
@@ -491,7 +492,7 @@ public class TestQuery {
         assertEquals(3, types.size());
         assertEquals(ResultType.BOOLEAN, types.get(0));
         assertEquals(ResultType.STRING, types.get(1));
-        assertEquals(ResultType.NUMBER, types.get(2));
+        assertEquals(ResultType.WHOLE_NUMBER, types.get(2));
     }
 
     @Test
@@ -768,7 +769,7 @@ public class TestQuery {
     }
 
     @Test
-    public void testMathOperations() {
+    public void testMathWholeNumberOperations() {
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("one", "1");
         attributes.put("two", "2");
@@ -777,7 +778,46 @@ public class TestQuery {
         attributes.put("five", "5");
         attributes.put("hundred", "100");
 
-        verifyEquals("${hundred:toNumber():multiply(2):divide(3):plus(1):mod(5)}", attributes, 2L);
+        verifyEquals("${hundred:toNumber():multiply(${two}):divide(${three}):plus(${one}):mod(${five})}", attributes, 2L);
+    }
+
+    @Test
+    public void testMathDecimalOperations() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("first", "1.5");
+        attributes.put("second", "12.3");
+        attributes.put("third", "3");
+        attributes.put("fourth", "4.201");
+        attributes.put("fifth", "5.1");
+        attributes.put("hundred", "100");
+
+        // The expected resulted is calculated instead of a set number due to the inaccuracy of double arithmetic
+        verifyEquals("${hundred:toNumber():multiply(${second}):divide(${third}):plus(${first}):mod(${fifth})}", attributes, (((100 * 12.3) / 3) + 1.5) %5.1);
+    }
+
+    @Test
+    public void testMathResultInterpretation() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("ten", "10.1");
+        attributes.put("two", "2.2");
+
+        // The expected resulted is calculated instead of a set number due to the inaccuracy of double arithmetic
+        verifyEquals("${ten:divide(${two:plus(3)}):toNumber()}", attributes, (Double.valueOf(10.1 / (2.2 + 3)).longValue()));
+
+        // The expected resulted is calculated instead of a set number due to the inaccuracy of double arithmetic
+        verifyEquals("${ten:divide(${two:plus(3)}):toDecimal()}", attributes, (10.1 / (2.2 + 3)));
+
+        verifyEquals("${ten:divide(${two:plus(3)}):toDate():format(\"SSS\")}", attributes, "001");
+    }
+
+    @Test
+    public void testMathLiteralOperations() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("ten", "10.1");
+        attributes.put("two", "2.2");
+
+        // The expected resulted is calculated instead of a set number due to the inaccuracy of double arithmetic
+        verifyEquals("${literal(5):toNumber():multiply(${two:plus(1)})}", attributes, 5*3.2);
     }
 
     @Test
@@ -900,7 +940,7 @@ public class TestQuery {
     }
 
     @Test
-    public void testMathOperators() {
+    public void testMathWholeNumberOperators() {
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("abc", "1234");
         attributes.put("xyz", "4132");
@@ -910,6 +950,62 @@ public class TestQuery {
     }
 
     @Test
+    public void testMathDecimalOperators() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("one", "1.1");
+        attributes.put("two", "2.2");
+        attributes.put("one_2", "1.1");
+
+        verifyEquals("${one:lt(${two})}", attributes, true);
+        verifyEquals("${one:lt(${one_2})}", attributes, false);
+        verifyEquals("${two:lt(${one})}", attributes, false);
+
+        verifyEquals("${one:le(${two})}", attributes, true);
+        verifyEquals("${one:le(${one_2})}", attributes, true);
+        verifyEquals("${two:le(${one_2})}", attributes, false);
+
+        verifyEquals("${one:ge(${two})}", attributes, false);
+        verifyEquals("${one:ge(${one_2})}", attributes, true);
+        verifyEquals("${two:ge(${one_2})}", attributes, true);
+
+        verifyEquals("${one:gt(${two})}", attributes, false);
+        verifyEquals("${one:gt(${one_2})}", attributes, false);
+        verifyEquals("${two:gt(${one})}", attributes, true);
+    }
+
+    @Test
+    public void testMathNumberDecimalConversion() {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("xyz", "1.332");
+        attributes.put("hello", "world!");
+
+        verifyEquals("${xyz:toNumber()}", attributes, 1L);
+
+        attributes.put("xyz", "2");
+        attributes.put("hello", "world!");
+
+        verifyEquals("${xyz:toDecimal()}", attributes, 2D);
+    }
+
+    @Test
+    public void testLiteral() {
+        final Map<String, String> attributes = new HashMap<>();
+
+        verifyEquals("${literal(5)}", attributes, "5");
+
+        verifyEquals("${literal(\"5\")}", attributes, "5");
+
+        verifyEquals("${literal(5):toNumber()}", attributes, 5L);
+        verifyEquals("${literal(5):toDecimal()}", attributes, 5D);
+
+        // Unquoted doubles are not due to more complicated parsing
+        verifyEquals("${literal(\"5.5\")}", attributes, "5.5");
+
+        verifyEquals("${literal(\"5.5\"):toNumber()}", attributes, 5L);
+        verifyEquals("${literal(\"5.5\"):toDecimal()}", attributes, 5.5D);
+    }
+
+    @Test
     public void testAllMatchingAttributes() {
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("abc", "1234");
@@ -1170,12 +1266,12 @@ public class TestQuery {
 
     @Test
     public void testToNumberFunctionReturnsNumberType() {
-        assertEquals(ResultType.NUMBER, Query.getResultType("${header.size:toNumber()}"));
+        assertEquals(ResultType.WHOLE_NUMBER, Query.getResultType("${header.size:toNumber()}"));
     }
 
     @Test
     public void testRandomFunctionReturnsNumberType() {
-        assertEquals(ResultType.NUMBER, Query.getResultType("${random()}"));
+        assertEquals(ResultType.WHOLE_NUMBER, Query.getResultType("${random()}"));
     }
 
     @Test
@@ -1377,8 +1473,20 @@ public class TestQuery {
         final Query query = Query.compile(expression);
         final QueryResult<?> result = query.evaluate(attributes);
 
-        if (expectedResult instanceof Number) {
-            assertEquals(ResultType.NUMBER, result.getResultType());
+        if (expectedResult instanceof Long) {
+            if (ResultType.NUMBER.equals(result.getResultType())) {
+                final Number resultNumber = ((NumberQueryResult) result).getValue();
+                assertTrue(resultNumber instanceof Long);
+            } else {
+                assertEquals(ResultType.WHOLE_NUMBER, result.getResultType());
+            }
+        } else if(expectedResult instanceof Double) {
+            if (ResultType.NUMBER.equals(result.getResultType())) {
+                final Number resultNumber = ((NumberQueryResult) result).getValue();
+                assertTrue(resultNumber instanceof Double);
+            } else {
+                assertEquals(ResultType.DECIMAL, result.getResultType());
+            }
         } else if (expectedResult instanceof Boolean) {
             assertEquals(ResultType.BOOLEAN, result.getResultType());
         } else {

http://git-wip-us.apache.org/repos/asf/nifi/blob/94ab9990/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/expression-language-guide.adoc b/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
index 98778e4..75290f6 100644
--- a/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/expression-language-guide.adoc
@@ -189,16 +189,20 @@ Language supports four different data types:
 
 - *String*: A String is a sequence of characters that can consist of numbers, letters, white space, and
 	special characters.
-- *Number*: A Number is an integer comprised of one or more digits (`0` through `9`). The Expression Language
-	does not provide support for fractional numbers. When converting to numbers from Date data types, they are represented as
+- *Number*: A Number is an whole number comprised of one or more digits (`0` through `9`). When converting to numbers from Date data types, they are represented as
 	the number of milliseconds since midnight GMT on January 1, 1970.
+- *Decimal*: A Decimal is a numeric value that can support decimals and larger values with minimal loss of precision. More precisely it
+    is a double-precision 64-bit IEEE 754 floating point. Due to this minimal loss of precision this data type should not be used for
+    very precise values, such as currency. For more documentation on the range of values stored in this data type
+    refer to this https://docs.oracle.com/javase/specs/jls/se7/html/jls-4.html#jls-4.2.3[link]. Decimals cannot be expressed as un-quoted characters
+    when inputting a literal Decimal to an Expression Language function. They must be input as Strings using quotes, like so: "1.1".
 - *Date*: A Date is an object that holds a Date and Time. Utilizing the <<dates>> and <<type_cast>> functions this data
 	type can be converted to/from Strings and numbers. If the whole Expression Language expression is evaluated to be a
 	date then it will be converted to a String with the format: "<Day of Week> <Month> <Day of Month> <Hour>:<Minute>:<Second> <Time Zone> <Year>".
 	Also expressed as "E MMM dd HH:mm:ss z yyyy" in Java SimpleDateFormat format. For example: "Wed Dec 31 12:00:04 UTC 2016".
 - *Boolean*: A Boolean is one of either `true` or `false`.
 
-All attributes are considered to be of type String.
+After evaluating expression language functions, all attributes are stored as of type String.
 
 The Expression Language is generally able to automatically coerce a value of one data type to the appropriate
 data type for a function. However, functions do exist to manually coerce a value into a specific data type.
@@ -1496,6 +1500,10 @@ An empty subject value or a subject value with an invalid JSON document results
 [[numbers]]
 == Mathematical Operations and Numeric Manipulation
 
+For those functions that support Decimal and Number (whole number) types, the return value type depends on the input types. If either the
+subject or argument are a Decimal then the result will be a Decimal. If both values are Numbers then the result will be a Number. This includes
+Divide. This is to preserve backwards compatibility and to not force rounding errors.
+
 
 [.function]
 === plus
@@ -1503,13 +1511,13 @@ An empty subject value or a subject value with an invalid JSON document results
 *Description*: [.description]#Adds a numeric value to the Subject. If either the argument or the Subject cannot be
 	coerced into a Number, returns `null`.#
 
-*Subject Type*: [.subject]#Number#
+*Subject Type*: [.subject]#Number or Decimal#
 
 *Arguments*:
 
 	- [.argName]#_Operand_# : [.argDesc]#The value to add to the Subject#
 
-*Return Type*: [.returnType]#Number#
+*Return Type*: [.returnType]#Number or Decimal (depending on input types)#
 
 *Examples*: If the "fileSize" attribute has a value of 100, then the Expression `${fileSize:plus(1000)}`
 	will return the value `1100`.
@@ -1523,13 +1531,13 @@ An empty subject value or a subject value with an invalid JSON document results
 
 *Description*: [.description]#Subtracts a numeric value from the Subject.#
 
-*Subject Type*: [.subject]#Number#
+*Subject Type*: [.subject]#Number or Decimal#
 
 *Arguments*:
 
 	- [.argName]#_Operand_# : [.argDesc]#The value to subtract from the Subject#
 
-*Return Type*: [.returnType]#Number#
+*Return Type*: [.returnType]#Number or Decimal (depending on input types)#
 
 *Examples*: If the "fileSize" attribute has a value of 100, then the Expression `${fileSize:minus(100)}`
 	will return the value `0`.
@@ -1543,13 +1551,13 @@ An empty subject value or a subject value with an invalid JSON document results
 
 *Description*: [.description]#Multiplies a numeric value by the Subject and returns the product.#
 
-*Subject Type*: [.subject]#Number#
+*Subject Type*: [.subject]#Number or Decimal#
 
 *Arguments*:
 
 	- [.argName]#_Operand_# : [.argDesc]#The value to multiple the Subject by#
 
-*Return Type*: [.returnType]#Number#
+*Return Type*: [.returnType]#Number or Decimal (depending on input types)#
 
 *Examples*: If the "fileSize" attribute has a value of 100, then the Expression `${fileSize:multiply(1024)}`
 	will return the value `102400`.
@@ -1560,15 +1568,15 @@ An empty subject value or a subject value with an invalid JSON document results
 [.function]
 === divide
 
-*Description*: [.description]#Divides a numeric value by the Subject and returns the result, rounded down to the nearest integer.#
+*Description*: [.description]#Divides the Subject by a numeric value and returns the result.#
 
-*Subject Type*: [.subject]#Number#
+*Subject Type*: [.subject]#Number or Decimal#
 
 *Arguments*:
 
-	- [.argName]#_Operand_# : [.argDesc]#The value to add divide the Subject by#
+	- [.argName]#_Operand_# : [.argDesc]#The value to divide the Subject by#
 
-*Return Type*: [.returnType]#Number#
+*Return Type*: [.returnType]#Number or Decimal (depending on input types)#
 
 *Examples*: If the "fileSize" attribute has a value of 100, then the Expression `${fileSize:divide(12)}`
 	will return the value `8`.
@@ -1582,13 +1590,13 @@ An empty subject value or a subject value with an invalid JSON document results
 *Description*: [.description]#Performs a modular division of the Subject by the argument. That is, this function will divide
 	the Subject by the value of the argument and return not the quotient but rather the remainder.#
 
-*Subject Type*: [.subject]#Number#
+*Subject Type*: [.subject]#Number or Decimal#
 
 *Arguments*:
 
 	- [.argName]#_Operand_# : [.argDesc]#The value to divide the Subject by#
 
-*Return Type*: [.returnType]#Number#
+*Return Type*: [.returnType]#Number or Decimal (depending on input types)#
 
 *Examples*: If the "fileSize" attribute has a value of 100, then the Expression `${fileSize:mod(12)}`
 	will return the value `4`.
@@ -1602,7 +1610,9 @@ An empty subject value or a subject value with an invalid JSON document results
 
 *Description*: [.description]#Converts the Subject from a Base 10 number to a different Radix (or number base). An optional
 	second argument can be used to indicate the minimum number of characters to be used. If the converted value
-	has fewer than this number of characters, the number will be padded with leading zeroes.#
+	has fewer than this number of characters, the number will be padded with leading zeroes.
+
+	If a decimal is passed as the subject, it will first be converted to a whole number and then processed.#
 
 *Subject Type*: [.subject]#Number#
 
@@ -1632,7 +1642,7 @@ An empty subject value or a subject value with an invalid JSON document results
 [.function]
 === random
 
-*Description*: [.description]#Returns a random number ( 0 to 2^63 - 1) using an insecure random number generator.#
+*Description*: [.description]#Returns a random whole number ( 0 to 2^63 - 1) using an insecure random number generator.#
 
 *Subject Type*: [.subjectless]#No subject#
 
@@ -1761,22 +1771,32 @@ manipulate the value.
 
 
 
-
-
 [.function]
 === toNumber
 
 *Description*: [.description]#Coerces the Subject into a Number#
 
-*Subject Type*: [.subject]#String#
+*Subject Type*: [.subject]#String, Decimal, or Date#
 
 *Arguments*: No arguments
 
 *Return Type*: [.returnType]#Number#
 
-*Examples*: The Expression `${fileSize:toNumber()}` converts the String attribute value of "fileSize" to a number.
+*Examples*: The Expression `${fileSize:toNumber()}` converts the attribute value of "fileSize" to a number.
+
+
+[.function]
+=== toDecimal
+
+*Description*: [.description]#Coerces the Subject into a Decimal#
+
+*Subject Type*: [.subject]#String, Whole Number or Date#
+
+*Arguments*: No arguments
 
+*Return Type*: [.returnType]#Decimal#
 
+*Examples*: The Expression `${fileSize:toDecimal()}` converts the attribute value of "fileSize" to a decimal.