You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by mattyb149 <gi...@git.apache.org> on 2018/03/21 01:51:34 UTC

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

GitHub user mattyb149 opened a pull request:

    https://github.com/apache/nifi/pull/2570

    NIFI-4857: Support String<->byte[] conversion

    Thank you for submitting a contribution to Apache NiFi.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [x] 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.
    
    - [x] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    - [x] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [x] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
    - [x] Have you written or updated unit tests to verify your changes?
    - [ ] 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:
    - [x] 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.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mattyb149/nifi NIFI-4857_2

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/2570.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2570
    
----
commit 424bd82ff670ccaed2622cdfb9f517d6a3960947
Author: Matthew Burgess <ma...@...>
Date:   2018-03-21T01:49:11Z

    NIFI-4857: Support String<->byte[] conversion

----


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi/pull/2570


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176437278
  
    --- Diff: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java ---
    @@ -474,6 +555,14 @@ public static String toString(final Object value, final String format) {
                 return Arrays.toString((Object[]) value);
             }
     
    +        if (value instanceof byte[]) {
    +            return new String((byte[]) value, charset);
    +        }
    +        if (value instanceof ByteBuffer) {
    --- End diff --
    
    Same as above.


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176482056
  
    --- Diff: nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java ---
    @@ -609,6 +623,9 @@ private static Object convertToAvroObject(final Object rawValue, final Schema fi
                     if (rawValue instanceof byte[]) {
                         return ByteBuffer.wrap((byte[]) rawValue);
                     }
    +                if (rawValue instanceof String) {
    +                    return ByteBuffer.wrap(((String) rawValue).getBytes(charset));
    --- End diff --
    
    In the clause above, a byte[] is wrapped in a ByteBuffer as well (that's where I got the code from), won't we be returning two different objects in that case?


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176440529
  
    --- Diff: nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java ---
    @@ -609,6 +623,9 @@ private static Object convertToAvroObject(final Object rawValue, final Schema fi
                     if (rawValue instanceof byte[]) {
                         return ByteBuffer.wrap((byte[]) rawValue);
                     }
    +                if (rawValue instanceof String) {
    +                    return ByteBuffer.wrap(((String) rawValue).getBytes(charset));
    --- End diff --
    
    I would prefer to avoid ByteBuffer here and instead use just byte[]


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176484111
  
    --- Diff: nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/ToBytes.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.record.path.functions;
    +
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPathEvaluationContext;
    +import org.apache.nifi.record.path.StandardFieldValue;
    +import org.apache.nifi.record.path.paths.RecordPathSegment;
    +import org.apache.nifi.record.path.util.RecordPathUtils;
    +import org.apache.nifi.serialization.record.RecordFieldType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import java.nio.charset.Charset;
    +import java.util.stream.Stream;
    +
    +public class ToBytes extends RecordPathSegment {
    +
    +    private final RecordPathSegment recordPath;
    +    private final RecordPathSegment charsetSegment;
    +
    +    public ToBytes(final RecordPathSegment recordPath, final RecordPathSegment charsetSegment, final boolean absolute) {
    +        super("toBytes", null, absolute);
    +        this.recordPath = recordPath;
    +        this.charsetSegment = charsetSegment;
    +    }
    +
    +    @Override
    +    public Stream<FieldValue> evaluate(RecordPathEvaluationContext context) {
    +        final Stream<FieldValue> fieldValues = recordPath.evaluate(context);
    +        return fieldValues.filter(fv -> fv.getValue() != null)
    +                .map(fv -> {
    +
    +                    if (!(fv.getValue() instanceof String)) {
    +                        return fv;
    --- End diff --
    
    Makes sense to me, but I copied that from ToDate, seemed like since it's a Stream, the top-level caller is expecting a new object back rather than catching an exception?


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176437061
  
    --- Diff: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java ---
    @@ -432,6 +478,37 @@ public static String toString(final Object value, final Supplier<DateFormat> for
                 return formatDate((java.util.Date) value, format);
             }
     
    +        if (value instanceof byte[]) {
    +            return new String((byte[])value, charset);
    +        }
    +
    +        if (value instanceof ByteBuffer) {
    --- End diff --
    
    Same as above, I think we should avoid the use of ByteBuffer here


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176439185
  
    --- Diff: nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/ToBytes.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.record.path.functions;
    +
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPathEvaluationContext;
    +import org.apache.nifi.record.path.StandardFieldValue;
    +import org.apache.nifi.record.path.paths.RecordPathSegment;
    +import org.apache.nifi.record.path.util.RecordPathUtils;
    +import org.apache.nifi.serialization.record.RecordFieldType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import java.nio.charset.Charset;
    +import java.util.stream.Stream;
    +
    +public class ToBytes extends RecordPathSegment {
    +
    +    private final RecordPathSegment recordPath;
    +    private final RecordPathSegment charsetSegment;
    +
    +    public ToBytes(final RecordPathSegment recordPath, final RecordPathSegment charsetSegment, final boolean absolute) {
    +        super("toBytes", null, absolute);
    +        this.recordPath = recordPath;
    +        this.charsetSegment = charsetSegment;
    +    }
    +
    +    @Override
    +    public Stream<FieldValue> evaluate(RecordPathEvaluationContext context) {
    +        final Stream<FieldValue> fieldValues = recordPath.evaluate(context);
    +        return fieldValues.filter(fv -> fv.getValue() != null)
    +                .map(fv -> {
    +
    +                    if (!(fv.getValue() instanceof String)) {
    +                        return fv;
    --- End diff --
    
    We should probably be throwing an Exception in this case? The user in this case is attempting to coerce a type that is not valid to coerce. Or otherwise filter it out from the results. It seems wrong to me to just ignore the conversion.


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176438047
  
    --- Diff: nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/ToString.java ---
    @@ -0,0 +1,95 @@
    +/*
    + * 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.record.path.functions;
    +
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPathEvaluationContext;
    +import org.apache.nifi.record.path.StandardFieldValue;
    +import org.apache.nifi.record.path.paths.RecordPathSegment;
    +import org.apache.nifi.record.path.util.RecordPathUtils;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import java.nio.charset.Charset;
    +import java.util.stream.Stream;
    +
    +public class ToString extends RecordPathSegment {
    +
    +    private final RecordPathSegment recordPath;
    +    private final RecordPathSegment charsetSegment;
    +
    +    public ToString(final RecordPathSegment recordPath, final RecordPathSegment charsetSegment, final boolean absolute) {
    +        super("toString", null, absolute);
    +        this.recordPath = recordPath;
    +        this.charsetSegment = charsetSegment;
    +    }
    +
    +    @Override
    +    public Stream<FieldValue> evaluate(RecordPathEvaluationContext context) {
    +        final Stream<FieldValue> fieldValues = recordPath.evaluate(context);
    +        return fieldValues.filter(fv -> fv.getValue() != null)
    +                .map(fv -> {
    +                    final Charset charset = getCharset(this.charsetSegment, context);
    +                    Object value = fv.getValue();
    +                    final String stringValue;
    +
    +                    if (value instanceof Object[]) {
    +                        Object[] o = (Object[]) value;
    +                        if (o.length > 0) {
    +
    +                            byte[] dest = new byte[o.length];
    +                            for (int i = 0; i < o.length; i++) {
    +                                dest[i] = (byte) o[i];
    +                            }
    +                            stringValue = new String(dest, charset);
    +                        } else {
    +                            stringValue = ""; // Empty array = empty string
    +                        }
    +                    } else if (!(fv.getValue() instanceof byte[])) {
    +                        return fv;
    --- End diff --
    
    This probably warrants throwing an Exception. It seems wrong to me to have the user explicitly indicating that they want a conversion to a String and then return something different, like an Integer.


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
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[].


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176531476
  
    --- Diff: nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/ToBytes.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.record.path.functions;
    +
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPathEvaluationContext;
    +import org.apache.nifi.record.path.StandardFieldValue;
    +import org.apache.nifi.record.path.paths.RecordPathSegment;
    +import org.apache.nifi.record.path.util.RecordPathUtils;
    +import org.apache.nifi.serialization.record.RecordFieldType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import java.nio.charset.Charset;
    +import java.util.stream.Stream;
    +
    +public class ToBytes extends RecordPathSegment {
    +
    +    private final RecordPathSegment recordPath;
    +    private final RecordPathSegment charsetSegment;
    +
    +    public ToBytes(final RecordPathSegment recordPath, final RecordPathSegment charsetSegment, final boolean absolute) {
    +        super("toBytes", null, absolute);
    +        this.recordPath = recordPath;
    +        this.charsetSegment = charsetSegment;
    +    }
    +
    +    @Override
    +    public Stream<FieldValue> evaluate(RecordPathEvaluationContext context) {
    +        final Stream<FieldValue> fieldValues = recordPath.evaluate(context);
    +        return fieldValues.filter(fv -> fv.getValue() != null)
    +                .map(fv -> {
    +
    +                    if (!(fv.getValue() instanceof String)) {
    +                        return fv;
    --- End diff --
    
    Agreed, the top-level caller is expecting that... but in this case, we cannot give the caller the correct type of data. So I think it's best to just throw instead of giving the caller the wrong data... if that's happening in ToDate then it's either a bug there as well, or perhaps there's some undocumented assumption being made about what else the type could be??


---

[GitHub] nifi issue #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on the issue:

    https://github.com/apache/nifi/pull/2570
  
    @mattyb149 thanks for the update! Sorry about the delay in getting back to this. All looks good now from my POV. There was a checkstyle violation (unused import) but I addressed that and all else looks good so merged to master.


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176439422
  
    --- Diff: nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/ToBytes.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.record.path.functions;
    +
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPathEvaluationContext;
    +import org.apache.nifi.record.path.StandardFieldValue;
    +import org.apache.nifi.record.path.paths.RecordPathSegment;
    +import org.apache.nifi.record.path.util.RecordPathUtils;
    +import org.apache.nifi.serialization.record.RecordFieldType;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import java.nio.charset.Charset;
    +import java.util.stream.Stream;
    +
    +public class ToBytes extends RecordPathSegment {
    +
    +    private final RecordPathSegment recordPath;
    +    private final RecordPathSegment charsetSegment;
    +
    +    public ToBytes(final RecordPathSegment recordPath, final RecordPathSegment charsetSegment, final boolean absolute) {
    +        super("toBytes", null, absolute);
    +        this.recordPath = recordPath;
    +        this.charsetSegment = charsetSegment;
    +    }
    +
    +    @Override
    +    public Stream<FieldValue> evaluate(RecordPathEvaluationContext context) {
    +        final Stream<FieldValue> fieldValues = recordPath.evaluate(context);
    +        return fieldValues.filter(fv -> fv.getValue() != null)
    +                .map(fv -> {
    +
    +                    if (!(fv.getValue() instanceof String)) {
    +                        return fv;
    +                    }
    +
    +                    final Charset charset = getCharset(this.charsetSegment, context);
    +
    +                    final byte[] bytesValue;
    +                    try {
    +                        Byte[] src = (Byte[]) DataTypeUtils.toArray(fv.getValue(), fv.getField().getFieldName(), RecordFieldType.BYTE.getDataType(), charset);
    +                        bytesValue = new byte[src.length];
    +                        for(int i=0;i<src.length;i++) {
    +                            bytesValue[i] = src[i];
    +                        }
    +                    } catch (final Exception e) {
    +                        return fv;
    --- End diff --
    
    Should probably let the Exception fly - as-is, it is just silently swallowing the Exception and then returning an object of the wrong type.


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176437638
  
    --- Diff: nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java ---
    @@ -1100,4 +1189,16 @@ public static boolean isScalarValue(final DataType dataType, final Object value)
     
             return true;
         }
    +
    +    public static Charset getCharset(String charsetName) {
    +        if(charsetName == null) {
    +            return StandardCharsets.UTF_8;
    +        } else {
    +            try {
    +                return Charset.forName(charsetName);
    +            } catch(Exception e) {
    --- End diff --
    
    If given an invalid character set, I think I would prefer to just throw the Exception. If there is a typo somewhere, this can lead to some very unexpected results that are difficult to track down.


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176530763
  
    --- Diff: nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java ---
    @@ -609,6 +623,9 @@ private static Object convertToAvroObject(final Object rawValue, final Schema fi
                     if (rawValue instanceof byte[]) {
                         return ByteBuffer.wrap((byte[]) rawValue);
                     }
    +                if (rawValue instanceof String) {
    +                    return ByteBuffer.wrap(((String) rawValue).getBytes(charset));
    --- End diff --
    
    Whoops - my bad on this one. This is #convertToAvroObject, and I was thinking of #normalizeValue. In this case, we are converting into the object that Avro wants, so a ByteBuffer is the correct thing to do.


---

[GitHub] nifi pull request #2570: NIFI-4857: Support String<->byte[] conversion

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2570#discussion_r176438643
  
    --- Diff: nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/functions/ToString.java ---
    @@ -0,0 +1,95 @@
    +/*
    + * 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.record.path.functions;
    +
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPathEvaluationContext;
    +import org.apache.nifi.record.path.StandardFieldValue;
    +import org.apache.nifi.record.path.paths.RecordPathSegment;
    +import org.apache.nifi.record.path.util.RecordPathUtils;
    +import org.apache.nifi.serialization.record.util.DataTypeUtils;
    +
    +import java.nio.charset.Charset;
    +import java.util.stream.Stream;
    +
    +public class ToString extends RecordPathSegment {
    +
    +    private final RecordPathSegment recordPath;
    +    private final RecordPathSegment charsetSegment;
    +
    +    public ToString(final RecordPathSegment recordPath, final RecordPathSegment charsetSegment, final boolean absolute) {
    +        super("toString", null, absolute);
    +        this.recordPath = recordPath;
    +        this.charsetSegment = charsetSegment;
    +    }
    +
    +    @Override
    +    public Stream<FieldValue> evaluate(RecordPathEvaluationContext context) {
    +        final Stream<FieldValue> fieldValues = recordPath.evaluate(context);
    +        return fieldValues.filter(fv -> fv.getValue() != null)
    +                .map(fv -> {
    +                    final Charset charset = getCharset(this.charsetSegment, context);
    +                    Object value = fv.getValue();
    +                    final String stringValue;
    +
    +                    if (value instanceof Object[]) {
    +                        Object[] o = (Object[]) value;
    +                        if (o.length > 0) {
    +
    +                            byte[] dest = new byte[o.length];
    +                            for (int i = 0; i < o.length; i++) {
    +                                dest[i] = (byte) o[i];
    +                            }
    +                            stringValue = new String(dest, charset);
    +                        } else {
    +                            stringValue = ""; // Empty array = empty string
    +                        }
    +                    } else if (!(fv.getValue() instanceof byte[])) {
    +                        return fv;
    +                    } else {
    +                        try {
    +                            stringValue = DataTypeUtils.toString(fv.getValue(), (String) null, charset);
    +                        } catch (final Exception e) {
    +                            return fv;
    --- End diff --
    
    If any RuntimeException is thrown here, I don't think we want to silently ignore it. Should probably let it fly.


---