Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-04-10 Thread via GitHub


tpalfy commented on PR #8250:
URL: https://github.com/apache/nifi/pull/8250#issuecomment-2048073797

   LGTM
   Thank your for work @mark-bathori and for the review @dan-s1 !
   Merged into main.


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



Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-04-10 Thread via GitHub


asfgit closed pull request #8250: NIFI-12614: Create record reader service for 
Protobuf messages
URL: https://github.com/apache/nifi/pull/8250


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



Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-02-25 Thread via GitHub


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())) {
+

Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-02-25 Thread via GitHub


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


##
nifi-nar-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/ProtobufReader.java:
##
@@ -0,0 +1,136 @@
+/*
+ * 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;
+
+import com.squareup.wire.schema.CoreLoaderKt;
+import com.squareup.wire.schema.Location;
+import com.squareup.wire.schema.Schema;
+import com.squareup.wire.schema.SchemaLoader;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.services.protobuf.schema.ProtoSchemaStrategy;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileSystems;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"protobuf", "record", "reader", "parser"})
+@CapabilityDescription("Parses a Protocol Buffers message from binary format.")
+public class ProtobufReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+private static final String ANY_PROTO = "google/protobuf/any.proto";
+private static final String DURATION_PROTO = 
"google/protobuf/duration.proto";
+private static final String EMPTY_PROTO = "google/protobuf/empty.proto";
+private static final String STRUCT_PROTO = "google/protobuf/struct.proto";
+private static final String TIMESTAMP_PROTO = 
"google/protobuf/timestamp.proto";
+private static final String WRAPPERS_PROTO = 
"google/protobuf/wrappers.proto";
+
+private static final AllowableValue GENERATE_FROM_PROTO_FILE = new 
AllowableValue("generate-from-proto-file",
+"Generate from Proto file", "The record schema is generated from 
the provided proto file");
+
+private String message;
+private Schema protoSchema;
+
+public static final PropertyDescriptor PROTOBUF_DIRECTORY = new 
PropertyDescriptor.Builder()
+.name("Proto Directory")
+.displayName("Proto Directory")
+.description("Directory containing Protocol Buffers message 
definition (.proto) file(s).")
+.required(true)
+
.addValidator(StandardValidators.createDirectoryExistsValidator(true, false))
+.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+.build();
+
+public static final PropertyDescriptor MESSAGE_TYPE = new 
PropertyDescriptor.Builder()
+.name("Message Type")
+.displayName("Message Type")
+.description("Fully qualified name of the Protocol Buffers message 
type including its package (eg. mypackage.MyMessage). " +
+"The .proto files configured in '" + 
PROTOBUF_DIRECTORY.getDisplayName() + "' must contain the definition of this 
message type.")
+.required(true)
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+.build();
+
+@Override
+protected List getSupportedPropertyDescriptors() {
+final List properties = new 
ArrayList<>(super.getSupportedPropertyDescriptors());
+properties.add(PROTOBUF_DIRECTORY);
+ 

Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-02-20 Thread via GitHub


tpalfy commented on code in PR #8250:
URL: https://github.com/apache/nifi/pull/8250#discussion_r1495827976


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

Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-02-20 Thread via GitHub


tpalfy commented on code in PR #8250:
URL: https://github.com/apache/nifi/pull/8250#discussion_r1491461479


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

Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-02-15 Thread via GitHub


tpalfy commented on code in PR #8250:
URL: https://github.com/apache/nifi/pull/8250#discussion_r1491426957


##
nifi-nar-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/TestProtobufRecordReader.java:
##
@@ -0,0 +1,151 @@
+/*
+ * 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;
+
+import com.google.protobuf.Descriptors;
+import com.squareup.wire.schema.Location;
+import com.squareup.wire.schema.Schema;
+import com.squareup.wire.schema.SchemaLoader;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileSystems;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.nifi.services.protobuf.ProtoTestUtil.BASE_TEST_PATH;
+import static 
org.apache.nifi.services.protobuf.ProtoTestUtil.generateInputDataForProto3;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+public class TestProtobufRecordReader {
+
+private static Schema protoSchema;
+
+@BeforeAll
+public static void setup(){
+final SchemaLoader schemaLoader = new 
SchemaLoader(FileSystems.getDefault());
+
schemaLoader.initRoots(Collections.singletonList(Location.get(BASE_TEST_PATH + 
"test_proto3.proto")), Collections.emptyList());
+protoSchema = schemaLoader.loadSchema();

Review Comment:
   We are creating the schema around 9 times across 3 test classes in very 
similar if not outright the same way. We could probably move this logic into 
`ProtoTestUtil`.



##
nifi-nar-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/schema/TestProtoSchemaParser.java:
##
@@ -0,0 +1,115 @@
+/*
+ * 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.schema;
+
+import com.google.common.collect.Sets;
+import com.squareup.wire.schema.CoreLoaderKt;
+import com.squareup.wire.schema.Location;
+import com.squareup.wire.schema.SchemaLoader;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.type.EnumDataType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+import org.junit.jupiter.api.Test;
+
+import java.nio.file.FileSystems;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.nifi.services.protobuf.ProtoTestUtil.BASE_TEST_PATH;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestProtoSchemaParser {
+
+@Test
+public void testSchemaParserForProto3() {

Review Comment:
   Checking a prebuilt expected 

Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-02-14 Thread via GitHub


tpalfy commented on code in PR #8250:
URL: https://github.com/apache/nifi/pull/8250#discussion_r1487902665


##
nifi-nar-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/ProtobufReader.java:
##
@@ -0,0 +1,136 @@
+/*
+ * 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;
+
+import com.squareup.wire.schema.CoreLoaderKt;
+import com.squareup.wire.schema.Location;
+import com.squareup.wire.schema.Schema;
+import com.squareup.wire.schema.SchemaLoader;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaAccessStrategy;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.services.protobuf.schema.ProtoSchemaStrategy;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileSystems;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+@Tags({"protobuf", "record", "reader", "parser"})
+@CapabilityDescription("Parses a Protocol Buffers message from binary format.")
+public class ProtobufReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+private static final String ANY_PROTO = "google/protobuf/any.proto";
+private static final String DURATION_PROTO = 
"google/protobuf/duration.proto";
+private static final String EMPTY_PROTO = "google/protobuf/empty.proto";
+private static final String STRUCT_PROTO = "google/protobuf/struct.proto";
+private static final String TIMESTAMP_PROTO = 
"google/protobuf/timestamp.proto";
+private static final String WRAPPERS_PROTO = 
"google/protobuf/wrappers.proto";
+
+private static final AllowableValue GENERATE_FROM_PROTO_FILE = new 
AllowableValue("generate-from-proto-file",
+"Generate from Proto file", "The record schema is generated from 
the provided proto file");
+
+private String message;
+private Schema protoSchema;
+
+public static final PropertyDescriptor PROTOBUF_DIRECTORY = new 
PropertyDescriptor.Builder()
+.name("Proto Directory")
+.displayName("Proto Directory")
+.description("Directory containing Protocol Buffers message 
definition (.proto) file(s).")
+.required(true)
+
.addValidator(StandardValidators.createDirectoryExistsValidator(true, false))
+.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+.build();
+
+public static final PropertyDescriptor MESSAGE_TYPE = new 
PropertyDescriptor.Builder()
+.name("Message Type")
+.displayName("Message Type")
+.description("Fully qualified name of the Protocol Buffers message 
type including its package (eg. mypackage.MyMessage). " +
+"The .proto files configured in '" + 
PROTOBUF_DIRECTORY.getDisplayName() + "' must contain the definition of this 
message type.")
+.required(true)
+.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+.build();
+
+@Override
+protected List getSupportedPropertyDescriptors() {
+final List properties = new 
ArrayList<>(super.getSupportedPropertyDescriptors());
+properties.add(PROTOBUF_DIRECTORY);
+

Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-02-05 Thread via GitHub


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


##
nifi-nar-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/resources/docs/org.apache.nifi.services.protobuf.ProtobufReader/additionalDetails.html:
##
@@ -0,0 +1,197 @@
+
+
+
+
+
+ProtobufReader
+
+
+
+
+
+The ProtobufReader Controller Service reads and parses a Protocol 
Buffers Message from binary format and creates a Record object.
+The Controller Service must be configured with the same '.proto' 
file that was used for the Message encoding, and the fully qualified
+name of the Message type including its package (e.g. 
mypackage.MyMessage). The Reader will always generate one record from the input
+data which represents the provided Protocol Buffers Message type.
+Further information about Protocol Buffers can be found here:
+https://protobuf.dev/;>protobuf.dev

Review Comment:
   The Reader supports both proto2 and proto3 versions, only exception is the 
`GROUPS` feature since the schema reader doesn't support it. This feature was 
already deprecated in proto2 and not included in proto3. 



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



Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-02-02 Thread via GitHub


dan-s1 commented on code in PR #8250:
URL: https://github.com/apache/nifi/pull/8250#discussion_r1476509298


##
nifi-nar-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/resources/docs/org.apache.nifi.services.protobuf.ProtobufReader/additionalDetails.html:
##
@@ -0,0 +1,197 @@
+
+
+
+
+
+ProtobufReader
+
+
+
+
+
+The ProtobufReader Controller Service reads and parses a Protocol 
Buffers Message from binary format and creates a Record object.
+The Controller Service must be configured with the same '.proto' 
file that was used for the Message encoding, and the fully qualified
+name of the Message type including its package (e.g. 
mypackage.MyMessage). The Reader will always generate one record from the input
+data which represents the provided Protocol Buffers Message type.
+Further information about Protocol Buffers can be found here:
+https://protobuf.dev/;>protobuf.dev

Review Comment:
   @mark-bathori Thanks for adding these additional details. I did have one 
question as I see on this referenced site there is Protobuf versions 2 and 3. 
Does the reader you created support both versions or only one? 



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



Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-02-02 Thread via GitHub


dan-s1 commented on code in PR #8250:
URL: https://github.com/apache/nifi/pull/8250#discussion_r1476509298


##
nifi-nar-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/resources/docs/org.apache.nifi.services.protobuf.ProtobufReader/additionalDetails.html:
##
@@ -0,0 +1,197 @@
+
+
+
+
+
+ProtobufReader
+
+
+
+
+
+The ProtobufReader Controller Service reads and parses a Protocol 
Buffers Message from binary format and creates a Record object.
+The Controller Service must be configured with the same '.proto' 
file that was used for the Message encoding, and the fully qualified
+name of the Message type including its package (e.g. 
mypackage.MyMessage). The Reader will always generate one record from the input
+data which represents the provided Protocol Buffers Message type.
+Further information about Protocol Buffers can be found here:
+https://protobuf.dev/;>protobuf.dev

Review Comment:
   @mark-bathori Thanks for adding these additional details. I did have one 
question as  see on this referenced site there is Protobuf versions 2 and 3. 
Does the reader you created support both versions or only one? 



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



Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-01-31 Thread via GitHub


dan-s1 commented on PR #8250:
URL: https://github.com/apache/nifi/pull/8250#issuecomment-1919646204

   @exceptionfactory Can you please restart the `dependency-check`? The 
exception does not seem to relate to the changes. 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.

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

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



Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-01-18 Thread via GitHub


mark-bathori commented on PR #8250:
URL: https://github.com/apache/nifi/pull/8250#issuecomment-1899211820

   Thanks @dan-s1 for the comment. I've added additionDeatils page to the 
Reader in my latest commit.


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



Re: [PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-01-17 Thread via GitHub


dan-s1 commented on PR #8250:
URL: https://github.com/apache/nifi/pull/8250#issuecomment-1896558818

   @mark-bathori With a big addition such as this, I would recommend some more 
documentation for end users so they are clear on how this works. This would go 
in your case under 
   
`nifi-nar-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/resources/org.apache.nifi.services.protobuf.ProtobufReader/additionalDetails.html`
   
   To get ideas on how to format and what to include, take a look at the other 
readers and the documentation they have.
   


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



[PR] NIFI-12614: Create record reader service for Protobuf messages [nifi]

2024-01-16 Thread via GitHub


mark-bathori opened a new pull request, #8250:
URL: https://github.com/apache/nifi/pull/8250

   
   
   
   
   
   
   
   
   
   
   
   
   
   # Summary
   
   The implementation does not generate message specific code with builders, 
instead it reads the schema from the provided proto file and maps it with the 
Unknown field list parsed from encoded proto data.
   
   [NIFI-12614](https://issues.apache.org/jira/browse/NIFI-12614)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue 
created
   
   ### Pull Request Tracking
   
   - [x] Pull Request title starts with Apache NiFi Jira issue number, such as 
`NIFI-0`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, 
as such `NIFI-0`
   
   ### Pull Request Formatting
   
   - [x] Pull Request based on current revision of the `main` branch
   - [x] Pull Request refers to a feature branch with one commit containing 
changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request 
creation.
   
   ### Build
   
   - [x] Build completed using `mvn clean install -P contrib-check`
 - [x] JDK 21
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 
2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License 
Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` 
files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


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