mark-bathori commented on code in PR #8250:
URL: https://github.com/apache/nifi/pull/8250#discussion_r1501870173


##########
nifi-nar-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/converter/ProtobufDataConverter.java:
##########
@@ -0,0 +1,380 @@
+/*
+ * 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.services.protobuf.converter;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.UnknownFieldSet;
+import com.squareup.wire.schema.EnumType;
+import com.squareup.wire.schema.Field;
+import com.squareup.wire.schema.MessageType;
+import com.squareup.wire.schema.OneOf;
+import com.squareup.wire.schema.ProtoType;
+import com.squareup.wire.schema.Schema;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+import org.apache.nifi.services.protobuf.FieldType;
+import org.apache.nifi.services.protobuf.schema.ProtoSchemaParser;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.function.Function;
+
+import static com.google.protobuf.CodedInputStream.decodeZigZag32;
+import static com.google.protobuf.TextFormat.unsignedToString;
+
+/**
+ * The class is responsible for creating Record by mapping the provided proto 
schema fields with the list of Unknown fields parsed from encoded proto data.
+ */
+public class ProtobufDataConverter {
+
+    public static final String MAP_KEY_FIELD_NAME = "key";
+    public static final String MAP_VALUE_FIELD_NAME = "value";
+    public static final String ANY_TYPE_URL_FIELD_NAME = "type_url";
+    public static final String ANY_VALUE_FIELD_NAME = "value";
+    public static final String ANY_MESSAGE_TYPE = "google.protobuf.Any";
+
+    private final Schema schema;
+    private final String message;
+    private final RecordSchema rootRecordSchema;
+    private final boolean coerceTypes;
+    private final boolean dropUnknownFields;
+
+    private boolean containsAnyField = false;
+
+    public ProtobufDataConverter(Schema schema, String message, RecordSchema 
recordSchema, boolean coerceTypes, boolean dropUnknownFields) {
+        this.schema = schema;
+        this.message = message;
+        this.rootRecordSchema = recordSchema;
+        this.coerceTypes = coerceTypes;
+        this.dropUnknownFields = dropUnknownFields;
+    }
+
+    /**
+     * Creates a record from the root message.
+     *
+     * @return created record
+     * @throws IOException failed to read input stream
+     */
+    public MapRecord createRecord(InputStream data) throws IOException {
+        final MessageType rootType = (MessageType) schema.getType(message);
+        Objects.requireNonNull(rootType, String.format("Message with name [%s] 
not found in the provided proto files", message));
+
+        MapRecord record = createRecord(rootType, ByteString.readFrom(data), 
rootRecordSchema);
+        if (containsAnyField) {
+            record.regenerateSchema();
+        }
+
+        return record;
+    }
+
+    /**
+     * Creates a record for the provided message.
+     *
+     * @param messageType  message to create a record from
+     * @param data         proto message data
+     * @param recordSchema record schema for the created record
+     * @return created record
+     * @throws InvalidProtocolBufferException failed to parse input data
+     */
+    private MapRecord createRecord(MessageType messageType, ByteString data, 
RecordSchema recordSchema) throws InvalidProtocolBufferException {
+        final UnknownFieldSet unknownFieldSet = 
UnknownFieldSet.parseFrom(data);
+
+        if ((ANY_MESSAGE_TYPE).equals(messageType.getType().toString())) {
+            containsAnyField = true;
+            return handleAnyField(unknownFieldSet);
+        }
+
+        return new MapRecord(recordSchema, processMessageFields(messageType, 
unknownFieldSet), false, dropUnknownFields);
+    }
+
+    /**
+     * Process declared, extension and oneOf fields in the provided message.
+     *
+     * @param messageType     message with fields to be processed
+     * @param unknownFieldSet received proto data fields
+     * @return Map of processed fields
+     */
+    private Map<String, Object> processMessageFields(MessageType messageType, 
UnknownFieldSet unknownFieldSet) throws InvalidProtocolBufferException {
+        Map<String, Object> recordValues = new HashMap<>();
+
+        for (final Field field : messageType.getDeclaredFields()) {
+            getField(new ProtoField(field), 
unknownFieldSet.getField(field.getTag()), recordValues);
+        }
+
+        for (final Field field : messageType.getExtensionFields()) {
+            getField(new ProtoField(field), 
unknownFieldSet.getField(field.getTag()), recordValues);
+        }
+
+        for (final OneOf oneOf : messageType.getOneOfs()) {
+            for (Field field : oneOf.getFields()) {
+                getField(new ProtoField(field), 
unknownFieldSet.getField(field.getTag()), recordValues);
+            }
+        }
+        return recordValues;
+    }
+
+    /**
+     * Checks the field value's presence and sets it into the result Map.
+     *
+     * @param protoField   proto field's properties
+     * @param unknownField field's value
+     * @param values       Map of values
+     */
+    private void getField(ProtoField protoField, UnknownFieldSet.Field 
unknownField, Map<String, Object> values) throws InvalidProtocolBufferException 
{
+        Optional<Object> fieldValue = convertFieldValues(protoField, 
unknownField);
+        fieldValue.ifPresent(o -> values.put(protoField.getFieldName(), o));
+    }
+
+    private Optional<Object> convertFieldValues(ProtoField protoField, 
UnknownFieldSet.Field unknownField) throws InvalidProtocolBufferException {
+        if (!unknownField.getLengthDelimitedList().isEmpty()) {
+            return Optional.of(convertLengthDelimitedFields(protoField, 
unknownField.getLengthDelimitedList()));
+        }
+        if (!unknownField.getFixed32List().isEmpty()) {
+            return Optional.of(convertFixed32Fields(protoField, 
unknownField.getFixed32List()));
+        }
+        if (!unknownField.getFixed64List().isEmpty()) {
+            return Optional.of(convertFixed64Fields(protoField, 
unknownField.getFixed64List()));
+        }
+        if (!unknownField.getVarintList().isEmpty()) {
+            return Optional.of(convertVarintFields(protoField, 
unknownField.getVarintList()));
+        }
+
+        return Optional.empty();
+    }
+
+    /**
+     * Converts a Length-Delimited field value into it's suitable data type.
+     *
+     * @param protoField proto field's properties
+     * @param values     field's unprocessed values
+     * @return converted field values
+     * @throws InvalidProtocolBufferException failed to parse input data
+     */
+    private Object convertLengthDelimitedFields(ProtoField protoField, 
List<ByteString> values) throws InvalidProtocolBufferException {
+        final ProtoType protoType = protoField.getProtoType();
+        if (protoType.isScalar()) {
+            switch (FieldType.findValue(protoType.getSimpleName())) {
+                case STRING:
+                    return resolveFieldValue(protoField, values, 
ByteString::toStringUtf8);
+                case BYTES:
+                    return resolveFieldValue(protoField, values, 
ByteString::toByteArray);
+                default:
+                    throw new 
IllegalStateException(String.format("Incompatible value was received for field 
[%s]," +
+                            " [%s] is not LengthDelimited field type", 
protoField.getFieldName(), protoType.getSimpleName()));
+            }

Review Comment:
   The `FieldType` enum cannot be removed since it is being used in the 
`ProtoSchemaParser` for DataType matching. If I remove it I need to define the 
scalar proto field types in two different place which is not efficient. Instead 
I changed the `Enum` matching to assign only the '`valueConverter`' function so 
the multiple '`return`' were removed.



-- 
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.

To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org

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

Reply via email to