[ 
https://issues.apache.org/jira/browse/NIFI-4857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16409615#comment-16409615
 ] 

ASF GitHub Bot commented on NIFI-4857:
--------------------------------------

Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176436923
  
    --- Diff: 
nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java
 ---
    @@ -270,11 +290,33 @@ public static boolean isRecordTypeCompatible(final 
Object value) {
                 return (Object[]) value;
             }
     
    +        if (value instanceof String && 
RecordFieldType.BYTE.getDataType().equals(elementDataType)) {
    +            byte[] src = ((String) value).getBytes(charset);
    +            Byte[] dest = new Byte[src.length];
    +            for (int i = 0; i < src.length; i++) {
    +                dest[i] = src[i];
    +            }
    +            return dest;
    +        }
    +
    +        if (value instanceof byte[]) {
    +            byte[] src = (byte[]) value;
    +            Byte[] dest = new Byte[src.length];
    +            for (int i = 0; i < src.length; i++) {
    +                dest[i] = src[i];
    +            }
    +            return dest;
    +        }
    +
             throw new IllegalTypeConversionException("Cannot convert value [" 
+ value + "] of type " + value.getClass() + " to Object Array for field " + 
fieldName);
         }
     
    -    public static boolean isArrayTypeCompatible(final Object value) {
    -        return value != null && value instanceof Object[];
    +    public static boolean isArrayTypeCompatible(final Object value, final 
DataType elementDataType) {
    +        return value != null
    +                // Either an object array or a String to be converted to 
byte[] or a ByteBuffer (from Avro, e.g.)
    +                && (value instanceof Object[]
    +                || (value instanceof String && 
RecordFieldType.BYTE.getDataType().equals(elementDataType))
    +                || value instanceof ByteBuffer);
    --- End diff --
    
    I don't think we should be supporting ByteBuffer here, just byte[]. The 
more we allow for, the more complex this gets and the more error-prone and less 
consistent it will become. While Avro may use ByteBuffers, when we use an Avro 
Reader to create a Record, we should be doing the conversion there from 
ByteBuffer to byte[].


> Record components do not support String <-> byte[] conversions
> --------------------------------------------------------------
>
>                 Key: NIFI-4857
>                 URL: https://issues.apache.org/jira/browse/NIFI-4857
>             Project: Apache NiFi
>          Issue Type: Improvement
>          Components: Extensions
>            Reporter: Matt Burgess
>            Assignee: Matt Burgess
>            Priority: Major
>
> When trying to perform a conversion of a field between a String and a byte 
> array, various errors are reporting (depending on where the conversion is 
> taking place). Here are some examples:
> 1) CSVReader, if a column with String values is specified in the schema as 
> "bytes"
> 2) ConvertRecord, if an input field is of type String and the output field is 
> of type "bytes"
> 3) ConvertRecord, if an input field is of type "bytes" and the output field 
> is of type "String"
> Many/most/all of the record components use utility methods to convert values, 
> I believe these methods need to be updated to support conversion between 
> String and byte[] values.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to