You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/04/22 15:12:03 UTC

[GitHub] [nifi] simonbence opened a new pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

simonbence opened a new pull request #4223:
URL: https://github.com/apache/nifi/pull/4223


   [NIFI-7369](https://issues.apache.org/jira/browse/NIFI-7369)
   
   My original issue was that when reading in Avro record containing decimal (logical) type, the incoming floating point data might loose precision due to the record reader parses this data type into a double value. In order to avoid this I added BigDecimal support next to the BigInteger support in a similar fashion. With this, decimal (logical) types from Avro input are handled as BigDecimal internally. Adding it as supported type in record correctly came with touching parts not strictly bounded to my particular issue but needed to be done to not leave loose ends.
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `master`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on both JDK 8 and JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 closed pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
markap14 closed pull request #4223:
URL: https://github.com/apache/nifi/pull/4223


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r413232172



##########
File path: nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java
##########
@@ -460,7 +459,7 @@ private void verifyORCUsers(final Path orcUsers, final int numExpectedUsers, BiF
         RecordReader recordReader = reader.rows();
 
         TypeInfo typeInfo =
-                TypeInfoUtils.getTypeInfoFromTypeString("struct<name:string,favorite_number:int,favorite_color:string,scale:double>");
+                TypeInfoUtils.getTypeInfoFromTypeString("struct<name:string,favorite_number:int,favorite_color:string,scale:decimal>");

Review comment:
       Something's strange. This change doesn't seem to be necessary.
   Even if left as `scale:double` all tests pass.
   Note the assertion bellow expects `double`:
   ```java
                   assertEquals(10.0 * currUser, ((DoubleWritable) x.get(3)).get(), Double.MIN_VALUE);
   ```

##########
File path: nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java
##########
@@ -285,7 +284,7 @@ public void testWriteORCWithAvroLogicalTypes() throws IOException, Initializatio
                     final DateFormat noTimeOfDayDateFormat = new SimpleDateFormat("yyyy-MM-dd");
                     noTimeOfDayDateFormat.setTimeZone(TimeZone.getTimeZone("gmt"));
                     assertEquals(noTimeOfDayDateFormat.format(dt), ((DateWritableV2) x.get(3)).get().toString());
-                    assertEquals(dec, ((DoubleWritable) x.get(4)).get(), Double.MIN_VALUE);
+                    assertEquals(dec.doubleValue(), ((HiveDecimalWritable) x.get(4)).doubleValue(), Double.MIN_VALUE);

Review comment:
       Can't we depend on the accuracy of `BigDecimal`?
   ```suggestion
                       assertEquals(dec, ((HiveDecimalWritable) x.get(4)).getHiveDecimal().bigDecimalValue());
   ```

##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
##########
@@ -1266,6 +1279,46 @@ public static boolean isBooleanTypeCompatible(final Object value) {
         return false;
     }
 
+    public static BigDecimal toBigDecimal(final Object value, final String fieldName) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof BigDecimal) {
+            return (BigDecimal) value;
+        }
+
+        if (value instanceof Number) {
+            final Number number = (Number) value;
+
+            if (number instanceof Byte
+                    || number instanceof Short
+                    || number instanceof Integer
+                    || number instanceof Long) {
+                return BigDecimal.valueOf(number.longValue());
+            }
+
+            if (number instanceof BigInteger) {
+                return new BigDecimal((BigInteger) number);
+            }
+
+            if (number instanceof Float || number instanceof Double) {
+                return BigDecimal.valueOf(number.doubleValue());

Review comment:
       We should probably use
   ```java
   new BigDecimal(Float.toString((Float)number))
   ```
   and
   ```java
   new BigDecimal(Double.toString((Double)number))
   ```
   According to the javadoc for this construcor:
   
   > Note: For values other than float and double NaN and ±Infinity, this constructor is compatible with the values returned by Float.toString(float) and Double.toString(double). This is generally the preferred way to convert a float or double into a BigDecimal, as it doesn't suffer from the unpredictability of the BigDecimal(double) constructor.

##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
##########
@@ -1759,6 +1812,8 @@ public static int getSQLTypeValue(final DataType dataType) {
                 return Types.DOUBLE;
             case FLOAT:
                 return Types.FLOAT;
+            case BIGDECIMAL:
+                return Types.NUMERIC;

Review comment:
       Wondering if it can be used for `DECIMAL` columns as well?

##########
File path: nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java
##########
@@ -265,6 +268,92 @@ public void testBytesToBytes() {
         assertEquals("Conversion from byte[] to String failed at char 0", (Object) "Hello".getBytes(StandardCharsets.UTF_16)[0], ((Byte[]) b)[0]);
     }
 
+    @Test
+    public void testConvertToBigDecimalWhenInputIsValid() {
+        // given
+        final BigDecimal expectedValue = BigDecimal.valueOf(12L);
+
+        // when & then
+        whenExpectingValidBigDecimalConversion(expectedValue, BigDecimal.valueOf(12L));
+        whenExpectingValidBigDecimalConversion(expectedValue, (byte) 12);
+        whenExpectingValidBigDecimalConversion(expectedValue, (short) 12);
+        whenExpectingValidBigDecimalConversion(expectedValue, 12);
+        whenExpectingValidBigDecimalConversion(expectedValue, 12L);
+        whenExpectingValidBigDecimalConversion(expectedValue, BigInteger.valueOf(12L));
+        whenExpectingValidBigDecimalConversion(expectedValue, 12F);
+        whenExpectingValidBigDecimalConversion(expectedValue, 12.000F);
+        whenExpectingValidBigDecimalConversion(expectedValue, 12D);
+        whenExpectingValidBigDecimalConversion(expectedValue, 12.000D);
+        whenExpectingValidBigDecimalConversion(expectedValue, "12");
+        whenExpectingValidBigDecimalConversion(expectedValue, "12.000");
+    }
+
+    @Test
+    public void testConvertToBigDecimalWhenNullInput() {
+        assertNull(DataTypeUtils.convertType(null, RecordFieldType.BIGDECIMAL.getDataType(), null, StandardCharsets.UTF_8));
+    }
+
+    @Test(expected = IllegalTypeConversionException.class)
+    public void testConvertToBigDecimalWhenInputStringIsInvalid() {
+        DataTypeUtils.convertType("test", RecordFieldType.BIGDECIMAL.getDataType(), null, StandardCharsets.UTF_8);
+    }
+
+    @Test(expected = IllegalTypeConversionException.class)
+    public void testConvertToBigDecimalWhenUnsupportedType() {
+        DataTypeUtils.convertType(new ArrayList<Double>(), RecordFieldType.BIGDECIMAL.getDataType(), null, StandardCharsets.UTF_8);
+    }
+
+    @Test(expected = IllegalTypeConversionException.class)
+    public void testConvertToBigDecimalWhenUnsupportedNumberType() {
+        DataTypeUtils.convertType(new DoubleAdder(), RecordFieldType.BIGDECIMAL.getDataType(), null, StandardCharsets.UTF_8);
+    }
+
+    @Test
+    public void testCompatibleDataTypeBigDecimal() {
+        // given
+        final DataType dataType = RecordFieldType.BIGDECIMAL.getDataType();
+
+        // when & then
+        assertTrue(DataTypeUtils.isCompatibleDataType(new BigDecimal("1.2345678901234567890"), dataType));
+        assertTrue(DataTypeUtils.isCompatibleDataType(new BigInteger("12345678901234567890"), dataType));
+        assertTrue(DataTypeUtils.isCompatibleDataType(1234567890123456789L, dataType));
+        assertTrue(DataTypeUtils.isCompatibleDataType(1, dataType));
+        assertTrue(DataTypeUtils.isCompatibleDataType((byte) 1, dataType));
+        assertTrue(DataTypeUtils.isCompatibleDataType((short) 1, dataType));
+        assertTrue(DataTypeUtils.isCompatibleDataType("1.2345678901234567890", dataType));
+        assertTrue(DataTypeUtils.isCompatibleDataType(3.1F, dataType));
+        assertTrue(DataTypeUtils.isCompatibleDataType(3.0D, dataType));
+        assertFalse(DataTypeUtils.isCompatibleDataType("1234567XYZ", dataType));
+        assertFalse(DataTypeUtils.isCompatibleDataType(new Long[]{1L, 2L}, dataType));
+    }
+
+    @Test
+    public void testInferDataTypeWithBigDecimal() {
+        assertEquals(RecordFieldType.BIGDECIMAL.getDataType(), DataTypeUtils.inferDataType(BigDecimal.valueOf(12L), RecordFieldType.BIGDECIMAL.getDataType()));

Review comment:
       The 2nd parameter for `DataTypeUtils.inferDataType` (default return value) is probable better to left as `null`.

##########
File path: nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java
##########
@@ -265,6 +268,92 @@ public void testBytesToBytes() {
         assertEquals("Conversion from byte[] to String failed at char 0", (Object) "Hello".getBytes(StandardCharsets.UTF_16)[0], ((Byte[]) b)[0]);
     }
 
+    @Test

Review comment:
       Can we please add a test like this?
   ```java
       @Test
       public void testChooseDataTypeWhenBigDecimal_vs_FLOAT_DOUBLE_BIGDECIMAL_ThenShouldReturnBIGDECIMAL() {
           // GIVEN
           List<DataType> dataTypes = Arrays.asList(
               RecordFieldType.FLOAT.getDataType(),
               RecordFieldType.DOUBLE.getDataType(),
               RecordFieldType.BIGDECIMAL.getDataType()
           );
   
           Object value = new BigDecimal("1.2");
           DataType expected = RecordFieldType.BIGDECIMAL.getDataType();
   
           // WHEN
           // THEN
           testChooseDataTypeAlsoReverseTypes(value, dataTypes, expected);
       }
   ```

##########
File path: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java
##########
@@ -376,6 +377,9 @@ private void writeValue(final JsonGenerator generator, final Object value, final
             case STRING:
                 generator.writeString(coercedValue.toString());
                 break;
+            case BIGDECIMAL:
+                generator.writeNumber((BigDecimal) coercedValue);

Review comment:
       We might want
   ```java
                   generator.writeNumber(DataTypeUtils.toBigDecimal(coercedValue, fieldName));
   ```
   just to follow conventions. I think `coercedValue` can only be `BigDecimal` at this point but not sure if that is by design or because of the current implementation.

##########
File path: nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
##########
@@ -463,6 +463,8 @@ private static void writeValue(final SolrInputDocument inputDocument, final Obje
                     addFieldToSolrDocument(inputDocument,fieldName,(BigInteger)coercedValue,fieldsToIndex);
                 }
                 break;
+            case BIGDECIMAL:
+                addFieldToSolrDocument(inputDocument, fieldName, coercedValue, fieldsToIndex);

Review comment:
       I'd use `DataTypeUtils.toBigDecimal(coercedValue, fieldName)` to be safe and follow conventions.

##########
File path: nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/test/java/org/apache/nifi/schema/validation/TestStandardSchemaValidator.java
##########
@@ -145,6 +148,9 @@ public void testValidateCorrectSimpleTypesStrictValidation() throws ParseExcepti
 
         valueMap.put("float_as_double", 8.0F);
 
+        valueMap.put("float_as_bigdecimal", 8.0F);

Review comment:
       Minor: we might want to add `"byte/short/int/long/bigint_as_bigdecimal"` as well.
   Kind of redundant with the ones below but still better to have this stay a simplified universal test.

##########
File path: nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/test/java/org/apache/nifi/processors/solr/SolrUtilsTest.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.processors.solr;
+
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+@RunWith(MockitoJUnitRunner.class)
+public class SolrUtilsTest {
+
+    @Mock
+    private SolrInputDocument inputDocument;
+
+    @Test
+    public void test() throws Exception {
+        // given
+        final String value = "12345678901234567890.123456789012345678901234567890";
+        final BigDecimal bigDecimalValue = new BigDecimal(value);
+        final List<RecordField> fields = Collections.singletonList(new RecordField("test", RecordFieldType.BIGDECIMAL.getDataType()));
+
+        final Map<String, Object> values = new HashMap<>();
+        values.put("test", bigDecimalValue);
+
+        final Record record = new MapRecord(new SimpleRecordSchema(fields), values);
+        final List<String> fieldsToIndex = Collections.singletonList("parent_test");
+
+        // when
+        SolrUtils.writeRecord(record, inputDocument, fieldsToIndex, "parent");
+
+        // then
+        Mockito.verify(inputDocument, Mockito.times(1)).addField(Mockito.eq("parent_test"), Mockito.eq(bigDecimalValue));

Review comment:
       Minor: could be simplified to
   ```java
           Mockito.verify(inputDocument).addField("parent_test", bigDecimalValue);
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r420181813



##########
File path: nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/test/java/org/apache/nifi/schema/validation/TestStandardSchemaValidator.java
##########
@@ -145,6 +148,9 @@ public void testValidateCorrectSimpleTypesStrictValidation() throws ParseExcepti
 
         valueMap.put("float_as_double", 8.0F);
 
+        valueMap.put("float_as_bigdecimal", 8.0F);

Review comment:
       The focus for this and the ones below is different: all of the ones added here is about the validation based on "narrow type", and the other cases cover the more specific situations, like allowing double as float in some cases.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r414812833



##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
##########
@@ -1266,6 +1279,46 @@ public static boolean isBooleanTypeCompatible(final Object value) {
         return false;
     }
 
+    public static BigDecimal toBigDecimal(final Object value, final String fieldName) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof BigDecimal) {
+            return (BigDecimal) value;
+        }
+
+        if (value instanceof Number) {
+            final Number number = (Number) value;
+
+            if (number instanceof Byte
+                    || number instanceof Short
+                    || number instanceof Integer
+                    || number instanceof Long) {
+                return BigDecimal.valueOf(number.longValue());
+            }
+
+            if (number instanceof BigInteger) {
+                return new BigDecimal((BigInteger) number);
+            }
+
+            if (number instanceof Float || number instanceof Double) {
+                return BigDecimal.valueOf(number.doubleValue());

Review comment:
       `BigDecimal.valueOf(double)` seems to me fine.
   It calls `new BigDecimal(Double.toString(val))` in the background, which is the proposed solution.
   I think the problematic way would be the `new BigDecimal(double)` constructor, not `BigDecimal.valueOf(double)`.
   
   `new BigDecimal(number.toString())` could also be used.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r420179686



##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
##########
@@ -1759,6 +1812,8 @@ public static int getSQLTypeValue(final DataType dataType) {
                 return Types.DOUBLE;
             case FLOAT:
                 return Types.FLOAT;
+            case BIGDECIMAL:
+                return Types.NUMERIC;

Review comment:
       The best I found in the matter is [this](https://stackoverflow.com/questions/759401/is-there-any-difference-between-decimal-and-numeric-in-sql-server). I made my decision based on this as NUMERIC has less implementational freedom for different databases




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r430518456



##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
##########
@@ -1232,6 +1242,10 @@ public static boolean isBigIntTypeCompatible(final Object value) {
         return isNumberTypeCompatible(value, DataTypeUtils::isIntegral);
     }
 
+    public static boolean isBigDecimalTypeCompatible(final Object value) {
+        return isNumberTypeCompatible(value, DataTypeUtils::isFloatingPoint);

Review comment:
       This could be an unarmed landmine.
   The `DataTypeUtils::isFloatingPoint` has a `Float.parse()` with the comment above it: `// Just to ensure that the exponents are in range, etc.`
   It ensures _no_ such thing in fact (if the number is out of range the parse still succeeds and returns infinitiy), but this might get fixed later ("arming" the landmine), after which for `BigDecimal` it might start to cause issues.

##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
##########
@@ -34,6 +34,7 @@
 import java.io.InputStream;
 import java.io.Reader;
 import java.lang.reflect.Array;
+import java.math.BigDecimal;

Review comment:
       I think `public static Optional<DataType> getWiderType(final DataType thisDataType, final DataType otherDataType)` should be updated as well.
   
   Could add along a new test in `TestFieldTypeInference`:
   ```java
       @Test
       public void test() {
           // GIVEN
           List<DataType> dataTypes = Arrays.asList(
               RecordFieldType.DECIMAL.getDecimalDataType(10, 1),
               RecordFieldType.DECIMAL.getDecimalDataType(10, 3),
               RecordFieldType.DECIMAL.getDecimalDataType(7, 3),
               RecordFieldType.DECIMAL.getDecimalDataType(7, 5),
               RecordFieldType.DECIMAL.getDecimalDataType(7, 7),
               RecordFieldType.FLOAT.getDataType(),
               RecordFieldType.DOUBLE.getDataType()
           );
   
           DataType expected = RecordFieldType.DECIMAL.getDecimalDataType(10, 7);
   
           // WHEN
           // THEN
           runWithAllPermutations(this::testToDataTypeShouldReturnSingleType, dataTypes, expected);
       }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r414816704



##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java
##########
@@ -354,6 +358,7 @@ private static RecordFieldType getFieldType(final int sqlType, final String valu
             case Types.DATE:
                 return RecordFieldType.DATE;
             case Types.DECIMAL:
+                return RecordFieldType.BIGDECIMAL;
             case Types.DOUBLE:
             case Types.NUMERIC:

Review comment:
       The `NUMERIC` SQL type is a fixed-point decimal too.
   I think it should be handled together with `DECIMAL` and return BigDecimal here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r420161745



##########
File path: nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/orc/PutORCTest.java
##########
@@ -460,7 +459,7 @@ private void verifyORCUsers(final Path orcUsers, final int numExpectedUsers, BiF
         RecordReader recordReader = reader.rows();
 
         TypeInfo typeInfo =
-                TypeInfoUtils.getTypeInfoFromTypeString("struct<name:string,favorite_number:int,favorite_color:string,scale:double>");
+                TypeInfoUtils.getTypeInfoFromTypeString("struct<name:string,favorite_number:int,favorite_color:string,scale:decimal>");

Review comment:
       You are right with the double, It was a mistake on my side, I did change it back.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r422971080



##########
File path: nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java
##########
@@ -284,6 +290,11 @@ public static TypeInfo getOrcField(DataType dataType, boolean hiveFieldNames) th
                 || RecordFieldType.STRING.equals(fieldType)) {
             return getPrimitiveOrcTypeFromPrimitiveFieldType(dataType);
         }
+
+        if (RecordFieldType.BIGDECIMAL.equals(fieldType)) {
+            // 38 is the maximum allowed precision and 19 digit is needed to represent long values

Review comment:
       The Hive libraries the NiFi currently depends on limit the precision to 38. Also it expects user to specify the precision and scale. The idea of having fix numbers came from this. As for the 19 digits the aim was to be able to represent longs without information loss, but I am open for any opinion. That was an arbitrary choice from my side as it looked the best fitting.
   
   Also, passing down this extra type information via DataType within RecordField looks possible, thus avoiding predefined values here, however it would come with a lot of other changes. In longer term it might be better but for initial support, I think it covers a lot of possible use cases even with limited representation capabilities.
   
   Other than the code of TypeInfoFactory I found the [following](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types#LanguageManualTypes-DecimalsdecimalDecimals) about the Hive decimal support and the limit.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
markap14 commented on pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#issuecomment-637753053


   Thanks @simonbence! I looked over the code and all looks good. Did some tests. The only thing that I think needed to be changed was a minor tweak to FlowFileTable, where it considers which types are numeric for a CHOICE data type. That was due to a commit of mine that was just recently merged in, so it makes sense that it was overlooked. Given that it was just a 1-liner to add the type, I just updated that before pushing the commit.
   
   And a big thanks to @tpalfy for reviewing. +1 merged to master.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r420180641



##########
File path: nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
##########
@@ -463,6 +463,8 @@ private static void writeValue(final SolrInputDocument inputDocument, final Obje
                     addFieldToSolrDocument(inputDocument,fieldName,(BigInteger)coercedValue,fieldsToIndex);
                 }
                 break;
+            case BIGDECIMAL:
+                addFieldToSolrDocument(inputDocument, fieldName, coercedValue, fieldsToIndex);

Review comment:
       Sadly In a number of places I followed the BigInt approach, which missed some of these safe(r) handling.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r420183429



##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
##########
@@ -72,6 +72,11 @@
      */
     DOUBLE("double", FLOAT),
 
+    /**
+     * A big decimal field type. Fields of this type use a {@code java.math.BigDecimal} value.
+     */
+    BIGDECIMAL("bigdecimal", FLOAT, DOUBLE),

Review comment:
       I was unluckily looking for BigInt enum usage instead of usage of the whole RecordFieldType. I added them, thank you for finding this. Also, it's a question for me if BigInt should be explicitly handled in these cases. (I will raise a story to check and fix if needed)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r420183429



##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
##########
@@ -72,6 +72,11 @@
      */
     DOUBLE("double", FLOAT),
 
+    /**
+     * A big decimal field type. Fields of this type use a {@code java.math.BigDecimal} value.
+     */
+    BIGDECIMAL("bigdecimal", FLOAT, DOUBLE),

Review comment:
       I was unluckily looking for BigInt enum usage instead of usage of the whole RecordFieldType. I added them, thank you for finding this. Also, it's a question for me if BigInt should be explicitly handled in these cases.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r422237710



##########
File path: nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
##########
@@ -256,6 +256,12 @@ private static Schema buildAvroSchema(final DataType dataType, final String fiel
             case LONG:
                 schema = Schema.create(Type.LONG);
                 break;
+            case BIGDECIMAL:
+                // One more byte than below to allow the dot in the string representation
+                schema = Schema.createFixed(fieldName + "Type", null,  "org.apache.nifi",39);

Review comment:
       Not sure where the concept that this should be a fixed sized type (with namespace) comes from.
   Also where did the 38 maximum come from? I remember seeing such a max in ORC, but not sure about Avro.

##########
File path: nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java
##########
@@ -284,6 +290,11 @@ public static TypeInfo getOrcField(DataType dataType, boolean hiveFieldNames) th
                 || RecordFieldType.STRING.equals(fieldType)) {
             return getPrimitiveOrcTypeFromPrimitiveFieldType(dataType);
         }
+
+        if (RecordFieldType.BIGDECIMAL.equals(fieldType)) {
+            // 38 is the maximum allowed precision and 19 digit is needed to represent long values

Review comment:
       Not sure I understand these constraints. With this, no bigger than 19 digit numbers (apart from decimal part) can be used.
   All-in-all I'm wondering if the `RecordFieldType.BIGDECIMAL` can stay as it is, without being able to convey precision and scale information.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r413766080



##########
File path: nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
##########
@@ -463,6 +463,8 @@ private static void writeValue(final SolrInputDocument inputDocument, final Obje
                     addFieldToSolrDocument(inputDocument,fieldName,(BigInteger)coercedValue,fieldsToIndex);
                 }
                 break;
+            case BIGDECIMAL:
+                addFieldToSolrDocument(inputDocument, fieldName, coercedValue, fieldsToIndex);

Review comment:
       I'd use `DataTypeUtils.toBigDecimal(coercedValue, fieldName)` to be safe and follow conventions.
   Also, curse of the switch statement: `break;` seems to be missing.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r420183429



##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
##########
@@ -72,6 +72,11 @@
      */
     DOUBLE("double", FLOAT),
 
+    /**
+     * A big decimal field type. Fields of this type use a {@code java.math.BigDecimal} value.
+     */
+    BIGDECIMAL("bigdecimal", FLOAT, DOUBLE),

Review comment:
       I was unluckily looking for BigInt enum usage instead of usage of the whole RecordFieldType. I added them, thank you for finding this. Also, it's a question for me if BigInt should be explicitly handled in these cases. (I will raise a ticket to check and fix if needed)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r420179889



##########
File path: nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java
##########
@@ -265,6 +268,92 @@ public void testBytesToBytes() {
         assertEquals("Conversion from byte[] to String failed at char 0", (Object) "Hello".getBytes(StandardCharsets.UTF_16)[0], ((Byte[]) b)[0]);
     }
 
+    @Test

Review comment:
       Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r413947216



##########
File path: nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
##########
@@ -341,9 +341,7 @@ public static DataType determineDataType(final Schema avroSchema, Map<String, Da
                 case LOGICAL_TYPE_TIMESTAMP_MICROS:
                     return RecordFieldType.TIMESTAMP.getDataType();
                 case LOGICAL_TYPE_DECIMAL:
-                    // We convert Decimal to Double.
-                    // Alternatively we could convert it to String, but numeric type is generally more preferable by users.
-                    return RecordFieldType.DOUBLE.getDataType();
+                    return RecordFieldType.BIGDECIMAL.getDataType();

Review comment:
       Shouldn't we also handle `RecordFieldType.BIGDECIMA` in the `buildAvroSchema` method?

##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
##########
@@ -72,6 +72,11 @@
      */
     DOUBLE("double", FLOAT),
 
+    /**
+     * A big decimal field type. Fields of this type use a {@code java.math.BigDecimal} value.
+     */
+    BIGDECIMAL("bigdecimal", FLOAT, DOUBLE),

Review comment:
       Maybe we should consider the following methods as well:
   `InferenceSchemaStrategy.convertSchema`
   `PutElasticsearchHttpRecord.writeValue`
   `PutHBaseRecord.asBytes`
   `AbstractKuduProcessor.toKuduType`
   `ResultSetRecordSet.getFieldType:382 case Types.JAVA_OBJECT`
   `DataTypeUtils.getWiderType:1744,1749 case FLOAT,case DOUBLE`
   `AbstractCSVRecordReader.convertSimpleIfPossible`
   `CSVSchemaInference.getDataType`
   `JsonTreeRowRecordReader.convertField`
   `WriteXMLResult.writeFieldForType`
   `XMLRecordReader.parseFieldForType`
   `XMLRecordReader.parseStringForType`
   `XmlSchemaInference.inferTextualDataType`
   `RecordSinkHandler.getDataType`
   `KuduLookupService.kuduSchemaToNiFiSchema:319 case DECIMAL`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r422978152



##########
File path: nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
##########
@@ -256,6 +256,12 @@ private static Schema buildAvroSchema(final DataType dataType, final String fiel
             case LONG:
                 schema = Schema.create(Type.LONG);
                 break;
+            case BIGDECIMAL:
+                // One more byte than below to allow the dot in the string representation
+                schema = Schema.createFixed(fieldName + "Type", null,  "org.apache.nifi",39);

Review comment:
       Based what I saw within org.apache.avro.Schema, other than creating schema of a composite type or one based on the Type enum within (which does not support BigDecimal), creating a fixed one is supported by it. As we do use it in [convertToAvroObject](https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java#L691-L698) as well it seems like a working approach.
   
   As for the 38, the idea was to be consistent with ORC, but you are right, we do not need to bound ourselves for the smallest common set but we might say in case of Avro, we pick a bigger limit.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] simonbence commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r420164345



##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
##########
@@ -1266,6 +1279,46 @@ public static boolean isBooleanTypeCompatible(final Object value) {
         return false;
     }
 
+    public static BigDecimal toBigDecimal(final Object value, final String fieldName) {
+        if (value == null) {
+            return null;
+        }
+
+        if (value instanceof BigDecimal) {
+            return (BigDecimal) value;
+        }
+
+        if (value instanceof Number) {
+            final Number number = (Number) value;
+
+            if (number instanceof Byte
+                    || number instanceof Short
+                    || number instanceof Integer
+                    || number instanceof Long) {
+                return BigDecimal.valueOf(number.longValue());
+            }
+
+            if (number instanceof BigInteger) {
+                return new BigDecimal((BigInteger) number);
+            }
+
+            if (number instanceof Float || number instanceof Double) {
+                return BigDecimal.valueOf(number.doubleValue());

Review comment:
       I did update as it is not worse than the original one, but honestly I could not reproduce any issue with that (behaves the same way for NaN and Infinity). However, I cannot close out a situation where it might happen so it is better be on the safe side.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #4223: NIFI-7369 Adding big decimal support for record handling in order to avoid missing precision when reading in records

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4223:
URL: https://github.com/apache/nifi/pull/4223#discussion_r432585859



##########
File path: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
##########
@@ -1691,15 +1765,37 @@ public static DataType mergeDataTypes(final DataType thisDataType, final DataTyp
             case FLOAT:
                 if (otherFieldType == RecordFieldType.DOUBLE) {
                     return Optional.of(otherDataType);
+                } else if (otherFieldType == RecordFieldType.DECIMAL) {
+                    return Optional.of(otherDataType);
                 }
                 break;
             case DOUBLE:
                 if (otherFieldType == RecordFieldType.FLOAT) {
                     return Optional.of(thisDataType);
+                } else if (otherFieldType == RecordFieldType.DECIMAL) {
+                    return Optional.of(otherDataType);
                 }
                 break;
+            case DECIMAL:
+                if (otherFieldType == RecordFieldType.DOUBLE) {
+                    return Optional.of(thisDataType);
+                } else if (otherFieldType == RecordFieldType.FLOAT) {
+                    return Optional.of(thisDataType);
+                } else if (otherFieldType == RecordFieldType.DECIMAL) {
+                    final DecimalDataType thisDecimalDataType = (DecimalDataType) thisDataType;
+                    final DecimalDataType otherDecimalDataType = (DecimalDataType) otherDataType;
+
+                    final int precision = thisDecimalDataType.getPrecision() > otherDecimalDataType.getPrecision()

Review comment:
       Minor: `Math.max` would be nicer imo.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org