exceptionfactory commented on code in PR #10214:
URL: https://github.com/apache/nifi/pull/10214#discussion_r2290991711


##########
nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/StandardProtobufReader.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.Schema;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.DescribedValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.MessageName;
+import org.apache.nifi.schemaregistry.services.MessageNameResolver;
+import org.apache.nifi.schemaregistry.services.MessageNameResolverService;
+import org.apache.nifi.schemaregistry.services.SchemaDefinition;
+import org.apache.nifi.schemaregistry.services.SchemaReferenceReader;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.schemaregistry.services.StandardMessageName;
+import org.apache.nifi.schemaregistry.services.StandardSchemaDefinition;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+import org.apache.nifi.services.protobuf.schema.ProtoSchemaParser;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.commons.codec.digest.DigestUtils.sha256Hex;
+import static 
org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_BRANCH_NAME;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME_PROPERTY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REGISTRY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_VERSION;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_PROPERTY;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_RESOLVER_SERVICE;
+
+@Tags({"protobuf", "record", "reader", "parser"})
+@CapabilityDescription("""
+    Parses Protocol Buffers messages from binary format into NiFi Records. \
+    Supports multiple schema access strategies including inline schema text, 
schema registry lookup, \
+    and schema reference readers.
+    Protobuf reader needs to know the Proto schema message name in order to 
deserialize the binary payload correctly. \
+    The name of this message can be determined statically using 'Message Name' 
property, \
+    or dynamically, using a Message Name Resolver service.""")
+
+public class StandardProtobufReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+    public static final PropertyDescriptor MESSAGE_NAME_RESOLVER_STRATEGY = 
new PropertyDescriptor.Builder()
+        .name("Message Name Resolver Strategy")
+        .displayName("Message Name Resolver Strategy")
+        .description("Strategy for determining the Protocol Buffers message 
name for processing")
+        .required(true)
+        .allowableValues(MESSAGE_NAME_PROPERTY, MESSAGE_NAME_RESOLVER_SERVICE)
+        .defaultValue(MESSAGE_NAME_PROPERTY)
+        .build();
+
+    public static final PropertyDescriptor MESSAGE_NAME = new 
PropertyDescriptor.Builder()
+        .name("Message Name")
+        .displayName("Message Name")
+        .description("Fully qualified name of the Protocol Buffers message 
including its package (eg. mypackage.MyMessage).")
+        .required(true)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .dependsOn(MESSAGE_NAME_RESOLVER_STRATEGY, MESSAGE_NAME_PROPERTY)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .build();
+
+    public static final PropertyDescriptor 
MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE = new PropertyDescriptor.Builder()
+        .name("Message Name Resolver Service")
+        .displayName("Message Name Resolver Service")
+        .description("Controller service that dynamically resolves Protocol 
Buffer message names from FlowFile content or attributes")
+        .required(true)
+        .identifiesControllerService(MessageNameResolverService.class)
+        .dependsOn(MESSAGE_NAME_RESOLVER_STRATEGY, 
MESSAGE_NAME_RESOLVER_SERVICE)
+        .build();
+
+    private volatile ProtobufSchemaCompiler schemaCompiler;
+    private volatile MessageNameResolver messageNameResolver;
+    private volatile SchemaReferenceReader schemaReferenceReader;
+    private volatile SchemaRegistry schemaRegistry;
+    private volatile String schemaAccessStrategyValue;
+    private volatile PropertyValue schemaText;
+    private volatile PropertyValue schemaName;
+    private volatile PropertyValue schemaBranchName;
+    private volatile PropertyValue schemaVersion;
+
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        super.storeSchemaAccessStrategy(context);
+        setupMessageNameResolver(context);
+        schemaAccessStrategyValue = 
context.getProperty(SCHEMA_ACCESS_STRATEGY).getValue();
+        schemaReferenceReader = 
context.getProperty(SCHEMA_REFERENCE_READER).asControllerService(SchemaReferenceReader.class);
+        schemaRegistry = 
context.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class);
+        schemaName = context.getProperty(SCHEMA_NAME);
+        schemaText = context.getProperty(SCHEMA_TEXT);
+        schemaBranchName = context.getProperty(SCHEMA_BRANCH_NAME);
+        schemaVersion = context.getProperty(SCHEMA_VERSION);
+        schemaCompiler = new ProtobufSchemaCompiler(getLogger());
+
+    }
+
+    @OnDisabled
+    public void onDisabled(final ConfigurationContext context) {
+        schemaCompiler = null;
+    }
+
+    @Override
+    public RecordReader createRecordReader(final Map<String, String> 
variables, final InputStream in, final long inputLength, final ComponentLog 
logger) throws IOException, SchemaNotFoundException {
+        if (SCHEMA_TEXT_PROPERTY.getValue().equals(schemaAccessStrategyValue)) 
{
+            final SchemaDefinition schemaDefinition = 
createSchemaDefinitionFromText(variables);
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        } else if 
(SCHEMA_NAME_PROPERTY.getValue().equals(schemaAccessStrategyValue)) {
+            final SchemaDefinition schemaDefinition = 
createSchemaDefinitionFromRegistry(variables);
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        } else if 
(SCHEMA_REFERENCE_READER_PROPERTY.getValue().equals(schemaAccessStrategyValue)) 
{
+            final SchemaIdentifier schemaIdentifier = 
schemaReferenceReader.getSchemaIdentifier(variables, in);
+            final SchemaDefinition schemaDefinition = 
schemaRegistry.retrieveSchemaDefinition(schemaIdentifier);
+            if (logger.isDebugEnabled()) {

Review Comment:
   This conditional check is not needed.



##########
nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/StandardProtobufReader.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.Schema;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.DescribedValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.MessageName;
+import org.apache.nifi.schemaregistry.services.MessageNameResolver;
+import org.apache.nifi.schemaregistry.services.MessageNameResolverService;
+import org.apache.nifi.schemaregistry.services.SchemaDefinition;
+import org.apache.nifi.schemaregistry.services.SchemaReferenceReader;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.schemaregistry.services.StandardMessageName;
+import org.apache.nifi.schemaregistry.services.StandardSchemaDefinition;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+import org.apache.nifi.services.protobuf.schema.ProtoSchemaParser;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.commons.codec.digest.DigestUtils.sha256Hex;
+import static 
org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_BRANCH_NAME;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME_PROPERTY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REGISTRY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_VERSION;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_PROPERTY;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_RESOLVER_SERVICE;
+
+@Tags({"protobuf", "record", "reader", "parser"})
+@CapabilityDescription("""
+    Parses Protocol Buffers messages from binary format into NiFi Records. \
+    Supports multiple schema access strategies including inline schema text, 
schema registry lookup, \
+    and schema reference readers.
+    Protobuf reader needs to know the Proto schema message name in order to 
deserialize the binary payload correctly. \
+    The name of this message can be determined statically using 'Message Name' 
property, \
+    or dynamically, using a Message Name Resolver service.""")
+
+public class StandardProtobufReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+    public static final PropertyDescriptor MESSAGE_NAME_RESOLVER_STRATEGY = 
new PropertyDescriptor.Builder()
+        .name("Message Name Resolver Strategy")
+        .displayName("Message Name Resolver Strategy")
+        .description("Strategy for determining the Protocol Buffers message 
name for processing")
+        .required(true)
+        .allowableValues(MESSAGE_NAME_PROPERTY, MESSAGE_NAME_RESOLVER_SERVICE)
+        .defaultValue(MESSAGE_NAME_PROPERTY)
+        .build();
+
+    public static final PropertyDescriptor MESSAGE_NAME = new 
PropertyDescriptor.Builder()
+        .name("Message Name")
+        .displayName("Message Name")

Review Comment:
   The duplicative `displayName` is not needed, this should be removed from all 
Property Descriptors.



##########
nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/ProtobufSchemaCompiler.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+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.commons.io.FileUtils;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schemaregistry.services.SchemaDefinition;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+
+import java.io.IOException;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static 
org.apache.nifi.services.protobuf.ProtobufSchemaValidator.validateSchemaDefinitionIdentifiers;
+
+/**
+ * Handles Protocol Buffer schema compilation, caching, and temporary 
directory operations.
+ * This class is responsible for compiling schema definitions into Wire Schema 
objects,
+ * managing a cache of compiled schemas, and handling temporary directory 
operations
+ * required during the compilation process.
+ */
+final class ProtobufSchemaCompiler {
+
+    private static final List<Location> STANDARD_PROTOBUF_LOCATIONS = 
Arrays.asList(
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/any.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/duration.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/empty.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/struct.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/timestamp.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/wrappers.proto")
+    );
+    private static final int CACHE_EXPIRE_HOURS = 1;
+    private static final int COMPILED_SCHEMAS_CACHE_SIZE = 200;
+
+    private final Cache<SchemaIdentifier, Schema> compiledSchemaCache;
+    private final ComponentLog logger;
+
+    /**
+     * Creates a new ProtobufSchemaCompiler with default cache settings.
+     *
+     * @param logger the component logger for logging compilation activities
+     */
+    public ProtobufSchemaCompiler(final ComponentLog logger) {
+        this.logger = logger;
+        this.compiledSchemaCache = Caffeine.newBuilder()
+            .expireAfterAccess(CACHE_EXPIRE_HOURS, TimeUnit.HOURS)
+            .maximumSize(COMPILED_SCHEMAS_CACHE_SIZE)
+            .build();
+    }
+
+    /**
+     * Compiles a schema definition or retrieves it from cache.
+     *
+     * @param schemaDefinition the schema definition to compile
+     * @return the compiled Schema
+     */
+    public Schema compileOrGetFromCache(final SchemaDefinition 
schemaDefinition) {
+        return compiledSchemaCache.get(schemaDefinition.getIdentifier(),
+            identifier -> {
+                try {
+                    return compileSchemaDefinition(schemaDefinition);
+                } catch (final IOException e) {
+                    throw new RuntimeException("Could not compile schema for 
identifier: " + identifier, e);

Review Comment:
   `UncheckedIOException` should be preferred over the general 
`RuntimeException` when handling an `IOException`
   ```suggestion
                       throw new UncheckedIOException("Could not compile schema 
for identifier: " + identifier, e);
   ```



##########
nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/StandardProtobufReader.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.Schema;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.DescribedValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.MessageName;
+import org.apache.nifi.schemaregistry.services.MessageNameResolver;
+import org.apache.nifi.schemaregistry.services.MessageNameResolverService;
+import org.apache.nifi.schemaregistry.services.SchemaDefinition;
+import org.apache.nifi.schemaregistry.services.SchemaReferenceReader;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.schemaregistry.services.StandardMessageName;
+import org.apache.nifi.schemaregistry.services.StandardSchemaDefinition;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+import org.apache.nifi.services.protobuf.schema.ProtoSchemaParser;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.commons.codec.digest.DigestUtils.sha256Hex;
+import static 
org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_BRANCH_NAME;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME_PROPERTY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REGISTRY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_VERSION;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_PROPERTY;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_RESOLVER_SERVICE;
+
+@Tags({"protobuf", "record", "reader", "parser"})
+@CapabilityDescription("""
+    Parses Protocol Buffers messages from binary format into NiFi Records. \
+    Supports multiple schema access strategies including inline schema text, 
schema registry lookup, \
+    and schema reference readers.
+    Protobuf reader needs to know the Proto schema message name in order to 
deserialize the binary payload correctly. \
+    The name of this message can be determined statically using 'Message Name' 
property, \
+    or dynamically, using a Message Name Resolver service.""")
+
+public class StandardProtobufReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+    public static final PropertyDescriptor MESSAGE_NAME_RESOLVER_STRATEGY = 
new PropertyDescriptor.Builder()
+        .name("Message Name Resolver Strategy")
+        .displayName("Message Name Resolver Strategy")
+        .description("Strategy for determining the Protocol Buffers message 
name for processing")
+        .required(true)
+        .allowableValues(MESSAGE_NAME_PROPERTY, MESSAGE_NAME_RESOLVER_SERVICE)
+        .defaultValue(MESSAGE_NAME_PROPERTY)
+        .build();
+
+    public static final PropertyDescriptor MESSAGE_NAME = new 
PropertyDescriptor.Builder()
+        .name("Message Name")
+        .displayName("Message Name")
+        .description("Fully qualified name of the Protocol Buffers message 
including its package (eg. mypackage.MyMessage).")
+        .required(true)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .dependsOn(MESSAGE_NAME_RESOLVER_STRATEGY, MESSAGE_NAME_PROPERTY)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .build();
+
+    public static final PropertyDescriptor 
MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE = new PropertyDescriptor.Builder()
+        .name("Message Name Resolver Service")
+        .displayName("Message Name Resolver Service")
+        .description("Controller service that dynamically resolves Protocol 
Buffer message names from FlowFile content or attributes")
+        .required(true)
+        .identifiesControllerService(MessageNameResolverService.class)
+        .dependsOn(MESSAGE_NAME_RESOLVER_STRATEGY, 
MESSAGE_NAME_RESOLVER_SERVICE)
+        .build();
+
+    private volatile ProtobufSchemaCompiler schemaCompiler;
+    private volatile MessageNameResolver messageNameResolver;
+    private volatile SchemaReferenceReader schemaReferenceReader;
+    private volatile SchemaRegistry schemaRegistry;
+    private volatile String schemaAccessStrategyValue;
+    private volatile PropertyValue schemaText;
+    private volatile PropertyValue schemaName;
+    private volatile PropertyValue schemaBranchName;
+    private volatile PropertyValue schemaVersion;
+
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        super.storeSchemaAccessStrategy(context);
+        setupMessageNameResolver(context);
+        schemaAccessStrategyValue = 
context.getProperty(SCHEMA_ACCESS_STRATEGY).getValue();
+        schemaReferenceReader = 
context.getProperty(SCHEMA_REFERENCE_READER).asControllerService(SchemaReferenceReader.class);
+        schemaRegistry = 
context.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class);
+        schemaName = context.getProperty(SCHEMA_NAME);
+        schemaText = context.getProperty(SCHEMA_TEXT);
+        schemaBranchName = context.getProperty(SCHEMA_BRANCH_NAME);
+        schemaVersion = context.getProperty(SCHEMA_VERSION);
+        schemaCompiler = new ProtobufSchemaCompiler(getLogger());
+
+    }
+
+    @OnDisabled
+    public void onDisabled(final ConfigurationContext context) {
+        schemaCompiler = null;
+    }
+
+    @Override
+    public RecordReader createRecordReader(final Map<String, String> 
variables, final InputStream in, final long inputLength, final ComponentLog 
logger) throws IOException, SchemaNotFoundException {
+        if (SCHEMA_TEXT_PROPERTY.getValue().equals(schemaAccessStrategyValue)) 
{
+            final SchemaDefinition schemaDefinition = 
createSchemaDefinitionFromText(variables);
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        } else if 
(SCHEMA_NAME_PROPERTY.getValue().equals(schemaAccessStrategyValue)) {
+            final SchemaDefinition schemaDefinition = 
createSchemaDefinitionFromRegistry(variables);
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        } else if 
(SCHEMA_REFERENCE_READER_PROPERTY.getValue().equals(schemaAccessStrategyValue)) 
{
+            final SchemaIdentifier schemaIdentifier = 
schemaReferenceReader.getSchemaIdentifier(variables, in);
+            final SchemaDefinition schemaDefinition = 
schemaRegistry.retrieveSchemaDefinition(schemaIdentifier);
+            if (logger.isDebugEnabled()) {
+                logger.debug("Using message name for schema identifier: {}", 
schemaDefinition.getIdentifier());
+            }
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        }
+
+        throw new SchemaNotFoundException("Unsupported schema access strategy: 
" + schemaAccessStrategyValue);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new 
ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(MESSAGE_NAME_RESOLVER_STRATEGY);
+        properties.add(MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE);
+        properties.add(MESSAGE_NAME);
+        return properties;
+    }
+
+    @Override
+    protected PropertyDescriptor buildSchemaTextProperty() {
+        return new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(SCHEMA_TEXT)
+            .required(true)
+            .clearValidators()
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .defaultValue("${proto.schema}")
+            .description("The text of a Proto 3 formatted Schema")
+            .build();
+    }
+
+    private RecordReader createProtobufRecordReader(final Map<String, String> 
variables, final InputStream in, final SchemaDefinition schemaDefinition) 
throws IOException {
+        final Schema schema = 
schemaCompiler.compileOrGetFromCache(schemaDefinition);
+        final ProtoSchemaParser schemaParser = new ProtoSchemaParser(schema);
+        final MessageName messageName = 
messageNameResolver.getMessageName(variables, schemaDefinition, in);
+        final RecordSchema recordSchema = 
schemaParser.createSchema(messageName.getFullyQualifiedName());
+        return new ProtobufRecordReader(schema, 
messageName.getFullyQualifiedName(), in, recordSchema);
+    }
+
+
+    private void setupMessageNameResolver(final ConfigurationContext context) {
+        final MessageNameResolverStrategyName messageNameResolverStrategyName 
= 
context.getProperty(MESSAGE_NAME_RESOLVER_STRATEGY).asAllowableValue(MessageNameResolverStrategyName.class);
+        messageNameResolver = switch (messageNameResolverStrategyName) {
+            case MESSAGE_NAME_PROPERTY -> new 
PropertyMessageNameResolver(context);
+            case MESSAGE_NAME_RESOLVER_SERVICE -> 
context.getProperty(MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE).asControllerService(MessageNameResolverService.class);
+        };
+    }
+
+    private SchemaDefinition createSchemaDefinitionFromText(final Map<String, 
String> variables) throws SchemaNotFoundException {
+        final String schemaTextString = 
schemaText.evaluateAttributeExpressions(variables).getValue();
+        validateSchemaText(schemaTextString);
+
+        final String sha256hex = sha256Hex(schemaTextString);
+        final SchemaIdentifier schemaIdentifier = SchemaIdentifier.builder()
+            .name(sha256hex + ".proto")

Review Comment:
   It looks like the `.proto` extension is reused in several places, so it may 
be useful to declare a `FileExtension` enum and reuse it across multiple 
classes. Either that, or just declaring a private static variable is the 
appropriate classes should help.



##########
nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/StandardProtobufReader.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.Schema;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.DescribedValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.MessageName;
+import org.apache.nifi.schemaregistry.services.MessageNameResolver;
+import org.apache.nifi.schemaregistry.services.MessageNameResolverService;
+import org.apache.nifi.schemaregistry.services.SchemaDefinition;
+import org.apache.nifi.schemaregistry.services.SchemaReferenceReader;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.schemaregistry.services.StandardMessageName;
+import org.apache.nifi.schemaregistry.services.StandardSchemaDefinition;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+import org.apache.nifi.services.protobuf.schema.ProtoSchemaParser;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.commons.codec.digest.DigestUtils.sha256Hex;
+import static 
org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_BRANCH_NAME;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME_PROPERTY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REGISTRY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_VERSION;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_PROPERTY;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_RESOLVER_SERVICE;
+
+@Tags({"protobuf", "record", "reader", "parser"})
+@CapabilityDescription("""
+    Parses Protocol Buffers messages from binary format into NiFi Records. \
+    Supports multiple schema access strategies including inline schema text, 
schema registry lookup, \
+    and schema reference readers.
+    Protobuf reader needs to know the Proto schema message name in order to 
deserialize the binary payload correctly. \
+    The name of this message can be determined statically using 'Message Name' 
property, \
+    or dynamically, using a Message Name Resolver service.""")
+
+public class StandardProtobufReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+    public static final PropertyDescriptor MESSAGE_NAME_RESOLVER_STRATEGY = 
new PropertyDescriptor.Builder()
+        .name("Message Name Resolver Strategy")
+        .displayName("Message Name Resolver Strategy")
+        .description("Strategy for determining the Protocol Buffers message 
name for processing")
+        .required(true)
+        .allowableValues(MESSAGE_NAME_PROPERTY, MESSAGE_NAME_RESOLVER_SERVICE)
+        .defaultValue(MESSAGE_NAME_PROPERTY)
+        .build();
+
+    public static final PropertyDescriptor MESSAGE_NAME = new 
PropertyDescriptor.Builder()
+        .name("Message Name")
+        .displayName("Message Name")
+        .description("Fully qualified name of the Protocol Buffers message 
including its package (eg. mypackage.MyMessage).")
+        .required(true)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .dependsOn(MESSAGE_NAME_RESOLVER_STRATEGY, MESSAGE_NAME_PROPERTY)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .build();
+
+    public static final PropertyDescriptor 
MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE = new PropertyDescriptor.Builder()
+        .name("Message Name Resolver Service")
+        .displayName("Message Name Resolver Service")
+        .description("Controller service that dynamically resolves Protocol 
Buffer message names from FlowFile content or attributes")
+        .required(true)
+        .identifiesControllerService(MessageNameResolverService.class)
+        .dependsOn(MESSAGE_NAME_RESOLVER_STRATEGY, 
MESSAGE_NAME_RESOLVER_SERVICE)
+        .build();
+
+    private volatile ProtobufSchemaCompiler schemaCompiler;
+    private volatile MessageNameResolver messageNameResolver;
+    private volatile SchemaReferenceReader schemaReferenceReader;
+    private volatile SchemaRegistry schemaRegistry;
+    private volatile String schemaAccessStrategyValue;
+    private volatile PropertyValue schemaText;
+    private volatile PropertyValue schemaName;
+    private volatile PropertyValue schemaBranchName;
+    private volatile PropertyValue schemaVersion;
+
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        super.storeSchemaAccessStrategy(context);
+        setupMessageNameResolver(context);
+        schemaAccessStrategyValue = 
context.getProperty(SCHEMA_ACCESS_STRATEGY).getValue();
+        schemaReferenceReader = 
context.getProperty(SCHEMA_REFERENCE_READER).asControllerService(SchemaReferenceReader.class);
+        schemaRegistry = 
context.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class);
+        schemaName = context.getProperty(SCHEMA_NAME);
+        schemaText = context.getProperty(SCHEMA_TEXT);
+        schemaBranchName = context.getProperty(SCHEMA_BRANCH_NAME);
+        schemaVersion = context.getProperty(SCHEMA_VERSION);
+        schemaCompiler = new ProtobufSchemaCompiler(getLogger());
+
+    }
+
+    @OnDisabled
+    public void onDisabled(final ConfigurationContext context) {
+        schemaCompiler = null;
+    }
+
+    @Override
+    public RecordReader createRecordReader(final Map<String, String> 
variables, final InputStream in, final long inputLength, final ComponentLog 
logger) throws IOException, SchemaNotFoundException {
+        if (SCHEMA_TEXT_PROPERTY.getValue().equals(schemaAccessStrategyValue)) 
{
+            final SchemaDefinition schemaDefinition = 
createSchemaDefinitionFromText(variables);
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        } else if 
(SCHEMA_NAME_PROPERTY.getValue().equals(schemaAccessStrategyValue)) {
+            final SchemaDefinition schemaDefinition = 
createSchemaDefinitionFromRegistry(variables);
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        } else if 
(SCHEMA_REFERENCE_READER_PROPERTY.getValue().equals(schemaAccessStrategyValue)) 
{
+            final SchemaIdentifier schemaIdentifier = 
schemaReferenceReader.getSchemaIdentifier(variables, in);
+            final SchemaDefinition schemaDefinition = 
schemaRegistry.retrieveSchemaDefinition(schemaIdentifier);
+            if (logger.isDebugEnabled()) {
+                logger.debug("Using message name for schema identifier: {}", 
schemaDefinition.getIdentifier());
+            }
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        }
+
+        throw new SchemaNotFoundException("Unsupported schema access strategy: 
" + schemaAccessStrategyValue);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new 
ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(MESSAGE_NAME_RESOLVER_STRATEGY);
+        properties.add(MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE);
+        properties.add(MESSAGE_NAME);
+        return properties;
+    }
+
+    @Override
+    protected PropertyDescriptor buildSchemaTextProperty() {
+        return new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(SCHEMA_TEXT)
+            .required(true)
+            .clearValidators()
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .defaultValue("${proto.schema}")
+            .description("The text of a Proto 3 formatted Schema")
+            .build();
+    }
+
+    private RecordReader createProtobufRecordReader(final Map<String, String> 
variables, final InputStream in, final SchemaDefinition schemaDefinition) 
throws IOException {
+        final Schema schema = 
schemaCompiler.compileOrGetFromCache(schemaDefinition);
+        final ProtoSchemaParser schemaParser = new ProtoSchemaParser(schema);
+        final MessageName messageName = 
messageNameResolver.getMessageName(variables, schemaDefinition, in);
+        final RecordSchema recordSchema = 
schemaParser.createSchema(messageName.getFullyQualifiedName());
+        return new ProtobufRecordReader(schema, 
messageName.getFullyQualifiedName(), in, recordSchema);
+    }
+
+
+    private void setupMessageNameResolver(final ConfigurationContext context) {
+        final MessageNameResolverStrategyName messageNameResolverStrategyName 
= 
context.getProperty(MESSAGE_NAME_RESOLVER_STRATEGY).asAllowableValue(MessageNameResolverStrategyName.class);
+        messageNameResolver = switch (messageNameResolverStrategyName) {
+            case MESSAGE_NAME_PROPERTY -> new 
PropertyMessageNameResolver(context);
+            case MESSAGE_NAME_RESOLVER_SERVICE -> 
context.getProperty(MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE).asControllerService(MessageNameResolverService.class);
+        };
+    }
+
+    private SchemaDefinition createSchemaDefinitionFromText(final Map<String, 
String> variables) throws SchemaNotFoundException {
+        final String schemaTextString = 
schemaText.evaluateAttributeExpressions(variables).getValue();
+        validateSchemaText(schemaTextString);
+
+        final String sha256hex = sha256Hex(schemaTextString);
+        final SchemaIdentifier schemaIdentifier = SchemaIdentifier.builder()
+            .name(sha256hex + ".proto")
+            .build();
+
+        return new StandardSchemaDefinition(schemaIdentifier, 
schemaTextString, SchemaDefinition.SchemaType.PROTOBUF);
+    }
+
+    private SchemaDefinition createSchemaDefinitionFromRegistry(final 
Map<String, String> variables) throws SchemaNotFoundException, IOException {
+        final String schemaNameValue = 
schemaName.evaluateAttributeExpressions(variables).getValue();
+        validateSchemaName(schemaNameValue);
+
+        final String schemaBranchNameValue = 
schemaBranchName.evaluateAttributeExpressions(variables).getValue();
+        final String schemaVersionValue = 
schemaVersion.evaluateAttributeExpressions(variables).getValue();
+
+        final SchemaIdentifier schemaIdentifier = 
buildSchemaIdentifier(schemaNameValue, schemaBranchNameValue, 
schemaVersionValue);
+        return schemaRegistry.retrieveSchemaDefinition(schemaIdentifier);
+    }
+
+    private SchemaIdentifier buildSchemaIdentifier(final String 
schemaNameValue, final String schemaBranchNameValue, final String 
schemaVersionValue) throws SchemaNotFoundException {
+        final SchemaIdentifier.Builder identifierBuilder = 
SchemaIdentifier.builder().name(schemaNameValue);
+
+        if (schemaBranchNameValue != null && !schemaBranchNameValue.isBlank()) 
{
+            identifierBuilder.branch(schemaBranchNameValue);
+        }
+
+        if (schemaVersionValue != null && !schemaVersionValue.isBlank()) {
+            try {
+                identifierBuilder.version(Integer.valueOf(schemaVersionValue));
+            } catch (final NumberFormatException nfe) {
+                throw new SchemaNotFoundException("Could not retrieve schema 
with name '" + schemaNameValue
+                    + "' because a non-numeric version was supplied '" + 
schemaVersionValue + "'", nfe);

Review Comment:
   Recommend using a string with placeholders and `.formatted()` instead of 
concatenation.



##########
nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/ProtobufSchemaCompiler.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+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.commons.io.FileUtils;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schemaregistry.services.SchemaDefinition;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+
+import java.io.IOException;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static 
org.apache.nifi.services.protobuf.ProtobufSchemaValidator.validateSchemaDefinitionIdentifiers;
+
+/**
+ * Handles Protocol Buffer schema compilation, caching, and temporary 
directory operations.
+ * This class is responsible for compiling schema definitions into Wire Schema 
objects,
+ * managing a cache of compiled schemas, and handling temporary directory 
operations
+ * required during the compilation process.
+ */
+final class ProtobufSchemaCompiler {
+
+    private static final List<Location> STANDARD_PROTOBUF_LOCATIONS = 
Arrays.asList(
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/any.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/duration.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/empty.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/struct.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/timestamp.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/wrappers.proto")
+    );
+    private static final int CACHE_EXPIRE_HOURS = 1;
+    private static final int COMPILED_SCHEMAS_CACHE_SIZE = 200;
+
+    private final Cache<SchemaIdentifier, Schema> compiledSchemaCache;
+    private final ComponentLog logger;
+
+    /**
+     * Creates a new ProtobufSchemaCompiler with default cache settings.
+     *
+     * @param logger the component logger for logging compilation activities
+     */
+    public ProtobufSchemaCompiler(final ComponentLog logger) {
+        this.logger = logger;
+        this.compiledSchemaCache = Caffeine.newBuilder()
+            .expireAfterAccess(CACHE_EXPIRE_HOURS, TimeUnit.HOURS)
+            .maximumSize(COMPILED_SCHEMAS_CACHE_SIZE)
+            .build();
+    }
+
+    /**
+     * Compiles a schema definition or retrieves it from cache.
+     *
+     * @param schemaDefinition the schema definition to compile
+     * @return the compiled Schema
+     */
+    public Schema compileOrGetFromCache(final SchemaDefinition 
schemaDefinition) {
+        return compiledSchemaCache.get(schemaDefinition.getIdentifier(),
+            identifier -> {
+                try {
+                    return compileSchemaDefinition(schemaDefinition);
+                } catch (final IOException e) {
+                    throw new RuntimeException("Could not compile schema for 
identifier: " + identifier, e);
+                }
+            });
+    }
+
+    /**
+     * Compiles a SchemaDefinition structure into a Schema using the wire 
library.
+     * Creates a temporary directory structure that mirrors the package 
structure and
+     * places all schemas in their appropriate directories.
+     *
+     * @param schemaDefinition the main schema definition to compile
+     * @return the compiled Schema
+     * @throws IOException if unable to create temporary files or compile 
schema
+     */
+    private Schema compileSchemaDefinition(final SchemaDefinition 
schemaDefinition) throws IOException {
+        logger.debug("Starting schema compilation for identifier: {}", 
schemaDefinition.getIdentifier());
+
+        // Validate that all schema identifiers end with .proto extension
+        validateSchemaDefinitionIdentifiers(schemaDefinition, true);
+
+        return executeWithTemporaryDirectory(tempDir -> {
+            try {
+                // Process main schema definition
+                writeSchemaToTempDirectory(tempDir, schemaDefinition);
+
+                // Process all referenced schemas recursively
+                processSchemaReferences(tempDir, 
schemaDefinition.getReferences());
+
+                // Create and configure schema loader
+                final Schema compiledSchema = createAndLoadSchema(tempDir);
+                logger.debug("Successfully compiled schema for identifier: 
{}", schemaDefinition.getIdentifier());
+                return compiledSchema;
+
+            } catch (final Exception e) {
+                throw new RuntimeException("Failed to compile Protobuf schema 
for identifier: " + schemaDefinition.getIdentifier(), e);
+            }
+        });
+    }
+
+    /**
+     * Executes a function with a temporary directory, ensuring proper cleanup.
+     *
+     * @param function the function to execute with the temporary directory
+     * @return the result of the function
+     * @throws IOException if unable to create or manage temporary directory
+     */
+    private <T> T executeWithTemporaryDirectory(final 
WithTemporaryDirectory<T> function) throws IOException {
+        final Path tempDir = Files.createTempDirectory("nifi-protobuf-schema");
+        logger.debug("Created temporary directory for schema compilation: {}", 
tempDir);
+
+        try {
+            return function.apply(tempDir);
+        } catch (final Exception e) {
+            throw new RuntimeException(e);
+        } finally {
+            safeDeleteDirectory(tempDir);
+        }
+    }
+
+    private Schema createAndLoadSchema(final Path tempDir) {
+        final SchemaLoader schemaLoader = new 
SchemaLoader(FileSystems.getDefault());
+
+        final List<Location> roots = new ArrayList<>();
+        roots.add(Location.get(tempDir.toString()));
+
+        // Add standard protobuf libraries
+        roots.addAll(STANDARD_PROTOBUF_LOCATIONS);
+
+        schemaLoader.initRoots(roots, Collections.emptyList());
+
+        // Load and return the compiled schema
+        return schemaLoader.loadSchema();
+    }
+
+    private void safeDeleteDirectory(final Path directory) {
+        if (Files.exists(directory)) {
+            try {
+                FileUtils.deleteDirectory(directory.toFile());
+            } catch (final IOException | IllegalArgumentException e) {
+                logger.warn("Failed to delete temporary directory: {}", 
directory, e);
+            }
+        }
+    }
+
+
+    /**
+     * Writes a schema definition to the temporary directory structure.
+     * If package name is present, creates the appropriate directory structure.
+     *
+     * @param tempDir          the temporary directory root
+     * @param schemaDefinition the schema definition to write
+     * @throws IOException if unable to create directories or write files
+     */
+    private void writeSchemaToTempDirectory(final Path tempDir, final 
SchemaDefinition schemaDefinition) throws IOException {
+        logger.debug("Writing schema definition to temporary directory. 
Identifier: {}", schemaDefinition.getIdentifier());
+
+        final String schemaFileName = generateSchemaFileName(schemaDefinition);
+        final Path schemaFile = tempDir.resolve(schemaFileName);
+
+        // Write schema text to file
+        Files.write(schemaFile, schemaDefinition.getText().getBytes(), CREATE, 
WRITE, TRUNCATE_EXISTING);
+        logger.debug("Successfully wrote schema to file: {} (string length: 
{})",
+            schemaFile, schemaDefinition.getText().length());
+    }
+
+    /**
+     * Generates a filename for a schema definition, ensuring it has a .proto 
extension.
+     *
+     * @param schemaDefinition the schema definition
+     * @return the generated filename
+     */
+    private String generateSchemaFileName(final SchemaDefinition 
schemaDefinition) {
+        String schemaFileName = 
schemaDefinition.getIdentifier().getName().orElseGet(
+            () -> 
String.valueOf(schemaDefinition.getIdentifier().getSchemaVersionId().orElse(0L))
+        );
+
+        if (!schemaFileName.endsWith(".proto")) {
+            schemaFileName += ".proto"; // Ensure the file ends with .proto, 
otherwise the wire library will not recognize it
+        }
+
+        return schemaFileName;
+    }
+
+    private void processSchemaReferences(final Path tempDir, final Map<String, 
SchemaDefinition> references) throws IOException {
+        logger.debug("Processing schema references. Count: {}, Temp directory: 
{}",
+            references.size(), tempDir);
+
+        for (final Map.Entry<String, SchemaDefinition> entry : 
references.entrySet()) {
+            final String referenceKey = entry.getKey();
+            final SchemaDefinition referencedSchema = entry.getValue();
+
+            logger.debug("Processing schema reference - Key: {}, Identifier: 
{}",

Review Comment:
   ```suggestion
               logger.debug("Processing schema reference [{}] Identifier [{}]",
   ```



##########
nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/StandardProtobufReader.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.Schema;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.DescribedValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.schemaregistry.services.MessageName;
+import org.apache.nifi.schemaregistry.services.MessageNameResolver;
+import org.apache.nifi.schemaregistry.services.MessageNameResolverService;
+import org.apache.nifi.schemaregistry.services.SchemaDefinition;
+import org.apache.nifi.schemaregistry.services.SchemaReferenceReader;
+import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.schemaregistry.services.StandardMessageName;
+import org.apache.nifi.schemaregistry.services.StandardSchemaDefinition;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SchemaRegistryService;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+import org.apache.nifi.services.protobuf.schema.ProtoSchemaParser;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.commons.codec.digest.DigestUtils.sha256Hex;
+import static 
org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_BRANCH_NAME;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME_PROPERTY;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REFERENCE_READER_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REGISTRY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT;
+import static 
org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPERTY;
+import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_VERSION;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_PROPERTY;
+import static 
org.apache.nifi.services.protobuf.StandardProtobufReader.MessageNameResolverStrategyName.MESSAGE_NAME_RESOLVER_SERVICE;
+
+@Tags({"protobuf", "record", "reader", "parser"})
+@CapabilityDescription("""
+    Parses Protocol Buffers messages from binary format into NiFi Records. \
+    Supports multiple schema access strategies including inline schema text, 
schema registry lookup, \
+    and schema reference readers.
+    Protobuf reader needs to know the Proto schema message name in order to 
deserialize the binary payload correctly. \
+    The name of this message can be determined statically using 'Message Name' 
property, \
+    or dynamically, using a Message Name Resolver service.""")
+
+public class StandardProtobufReader extends SchemaRegistryService implements 
RecordReaderFactory {
+
+    public static final PropertyDescriptor MESSAGE_NAME_RESOLVER_STRATEGY = 
new PropertyDescriptor.Builder()
+        .name("Message Name Resolver Strategy")
+        .displayName("Message Name Resolver Strategy")
+        .description("Strategy for determining the Protocol Buffers message 
name for processing")
+        .required(true)
+        .allowableValues(MESSAGE_NAME_PROPERTY, MESSAGE_NAME_RESOLVER_SERVICE)
+        .defaultValue(MESSAGE_NAME_PROPERTY)
+        .build();
+
+    public static final PropertyDescriptor MESSAGE_NAME = new 
PropertyDescriptor.Builder()
+        .name("Message Name")
+        .displayName("Message Name")
+        .description("Fully qualified name of the Protocol Buffers message 
including its package (eg. mypackage.MyMessage).")
+        .required(true)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .dependsOn(MESSAGE_NAME_RESOLVER_STRATEGY, MESSAGE_NAME_PROPERTY)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .build();
+
+    public static final PropertyDescriptor 
MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE = new PropertyDescriptor.Builder()
+        .name("Message Name Resolver Service")
+        .displayName("Message Name Resolver Service")
+        .description("Controller service that dynamically resolves Protocol 
Buffer message names from FlowFile content or attributes")
+        .required(true)
+        .identifiesControllerService(MessageNameResolverService.class)
+        .dependsOn(MESSAGE_NAME_RESOLVER_STRATEGY, 
MESSAGE_NAME_RESOLVER_SERVICE)
+        .build();
+
+    private volatile ProtobufSchemaCompiler schemaCompiler;
+    private volatile MessageNameResolver messageNameResolver;
+    private volatile SchemaReferenceReader schemaReferenceReader;
+    private volatile SchemaRegistry schemaRegistry;
+    private volatile String schemaAccessStrategyValue;
+    private volatile PropertyValue schemaText;
+    private volatile PropertyValue schemaName;
+    private volatile PropertyValue schemaBranchName;
+    private volatile PropertyValue schemaVersion;
+
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        super.storeSchemaAccessStrategy(context);
+        setupMessageNameResolver(context);
+        schemaAccessStrategyValue = 
context.getProperty(SCHEMA_ACCESS_STRATEGY).getValue();
+        schemaReferenceReader = 
context.getProperty(SCHEMA_REFERENCE_READER).asControllerService(SchemaReferenceReader.class);
+        schemaRegistry = 
context.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class);
+        schemaName = context.getProperty(SCHEMA_NAME);
+        schemaText = context.getProperty(SCHEMA_TEXT);
+        schemaBranchName = context.getProperty(SCHEMA_BRANCH_NAME);
+        schemaVersion = context.getProperty(SCHEMA_VERSION);
+        schemaCompiler = new ProtobufSchemaCompiler(getLogger());
+
+    }
+
+    @OnDisabled
+    public void onDisabled(final ConfigurationContext context) {
+        schemaCompiler = null;
+    }
+
+    @Override
+    public RecordReader createRecordReader(final Map<String, String> 
variables, final InputStream in, final long inputLength, final ComponentLog 
logger) throws IOException, SchemaNotFoundException {
+        if (SCHEMA_TEXT_PROPERTY.getValue().equals(schemaAccessStrategyValue)) 
{
+            final SchemaDefinition schemaDefinition = 
createSchemaDefinitionFromText(variables);
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        } else if 
(SCHEMA_NAME_PROPERTY.getValue().equals(schemaAccessStrategyValue)) {
+            final SchemaDefinition schemaDefinition = 
createSchemaDefinitionFromRegistry(variables);
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        } else if 
(SCHEMA_REFERENCE_READER_PROPERTY.getValue().equals(schemaAccessStrategyValue)) 
{
+            final SchemaIdentifier schemaIdentifier = 
schemaReferenceReader.getSchemaIdentifier(variables, in);
+            final SchemaDefinition schemaDefinition = 
schemaRegistry.retrieveSchemaDefinition(schemaIdentifier);
+            if (logger.isDebugEnabled()) {
+                logger.debug("Using message name for schema identifier: {}", 
schemaDefinition.getIdentifier());
+            }
+            return createProtobufRecordReader(variables, in, schemaDefinition);
+        }
+
+        throw new SchemaNotFoundException("Unsupported schema access strategy: 
" + schemaAccessStrategyValue);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> properties = new 
ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(MESSAGE_NAME_RESOLVER_STRATEGY);
+        properties.add(MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE);
+        properties.add(MESSAGE_NAME);
+        return properties;
+    }
+
+    @Override
+    protected PropertyDescriptor buildSchemaTextProperty() {
+        return new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(SCHEMA_TEXT)
+            .required(true)
+            .clearValidators()
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .defaultValue("${proto.schema}")
+            .description("The text of a Proto 3 formatted Schema")
+            .build();
+    }
+
+    private RecordReader createProtobufRecordReader(final Map<String, String> 
variables, final InputStream in, final SchemaDefinition schemaDefinition) 
throws IOException {
+        final Schema schema = 
schemaCompiler.compileOrGetFromCache(schemaDefinition);
+        final ProtoSchemaParser schemaParser = new ProtoSchemaParser(schema);
+        final MessageName messageName = 
messageNameResolver.getMessageName(variables, schemaDefinition, in);
+        final RecordSchema recordSchema = 
schemaParser.createSchema(messageName.getFullyQualifiedName());
+        return new ProtobufRecordReader(schema, 
messageName.getFullyQualifiedName(), in, recordSchema);
+    }
+
+
+    private void setupMessageNameResolver(final ConfigurationContext context) {
+        final MessageNameResolverStrategyName messageNameResolverStrategyName 
= 
context.getProperty(MESSAGE_NAME_RESOLVER_STRATEGY).asAllowableValue(MessageNameResolverStrategyName.class);
+        messageNameResolver = switch (messageNameResolverStrategyName) {
+            case MESSAGE_NAME_PROPERTY -> new 
PropertyMessageNameResolver(context);
+            case MESSAGE_NAME_RESOLVER_SERVICE -> 
context.getProperty(MESSAGE_NAME_RESOLVER_CONTROLLER_SERVICE).asControllerService(MessageNameResolverService.class);
+        };
+    }
+
+    private SchemaDefinition createSchemaDefinitionFromText(final Map<String, 
String> variables) throws SchemaNotFoundException {
+        final String schemaTextString = 
schemaText.evaluateAttributeExpressions(variables).getValue();
+        validateSchemaText(schemaTextString);
+
+        final String sha256hex = sha256Hex(schemaTextString);

Review Comment:
   If this is the only use of Commons Codec, I recommend using the 
MessageDigest API directly.



##########
nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/ProtobufSchemaCompiler.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+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.commons.io.FileUtils;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schemaregistry.services.SchemaDefinition;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+
+import java.io.IOException;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static 
org.apache.nifi.services.protobuf.ProtobufSchemaValidator.validateSchemaDefinitionIdentifiers;
+
+/**
+ * Handles Protocol Buffer schema compilation, caching, and temporary 
directory operations.
+ * This class is responsible for compiling schema definitions into Wire Schema 
objects,
+ * managing a cache of compiled schemas, and handling temporary directory 
operations
+ * required during the compilation process.
+ */
+final class ProtobufSchemaCompiler {
+
+    private static final List<Location> STANDARD_PROTOBUF_LOCATIONS = 
Arrays.asList(
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/any.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/duration.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/empty.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/struct.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/timestamp.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/wrappers.proto")
+    );
+    private static final int CACHE_EXPIRE_HOURS = 1;
+    private static final int COMPILED_SCHEMAS_CACHE_SIZE = 200;
+
+    private final Cache<SchemaIdentifier, Schema> compiledSchemaCache;
+    private final ComponentLog logger;
+
+    /**
+     * Creates a new ProtobufSchemaCompiler with default cache settings.
+     *
+     * @param logger the component logger for logging compilation activities
+     */
+    public ProtobufSchemaCompiler(final ComponentLog logger) {
+        this.logger = logger;
+        this.compiledSchemaCache = Caffeine.newBuilder()
+            .expireAfterAccess(CACHE_EXPIRE_HOURS, TimeUnit.HOURS)
+            .maximumSize(COMPILED_SCHEMAS_CACHE_SIZE)
+            .build();
+    }
+
+    /**
+     * Compiles a schema definition or retrieves it from cache.
+     *
+     * @param schemaDefinition the schema definition to compile
+     * @return the compiled Schema
+     */
+    public Schema compileOrGetFromCache(final SchemaDefinition 
schemaDefinition) {
+        return compiledSchemaCache.get(schemaDefinition.getIdentifier(),
+            identifier -> {
+                try {
+                    return compileSchemaDefinition(schemaDefinition);
+                } catch (final IOException e) {
+                    throw new RuntimeException("Could not compile schema for 
identifier: " + identifier, e);
+                }
+            });
+    }
+
+    /**
+     * Compiles a SchemaDefinition structure into a Schema using the wire 
library.
+     * Creates a temporary directory structure that mirrors the package 
structure and
+     * places all schemas in their appropriate directories.
+     *
+     * @param schemaDefinition the main schema definition to compile
+     * @return the compiled Schema
+     * @throws IOException if unable to create temporary files or compile 
schema
+     */
+    private Schema compileSchemaDefinition(final SchemaDefinition 
schemaDefinition) throws IOException {
+        logger.debug("Starting schema compilation for identifier: {}", 
schemaDefinition.getIdentifier());
+
+        // Validate that all schema identifiers end with .proto extension
+        validateSchemaDefinitionIdentifiers(schemaDefinition, true);
+
+        return executeWithTemporaryDirectory(tempDir -> {
+            try {
+                // Process main schema definition
+                writeSchemaToTempDirectory(tempDir, schemaDefinition);
+
+                // Process all referenced schemas recursively
+                processSchemaReferences(tempDir, 
schemaDefinition.getReferences());
+
+                // Create and configure schema loader
+                final Schema compiledSchema = createAndLoadSchema(tempDir);
+                logger.debug("Successfully compiled schema for identifier: 
{}", schemaDefinition.getIdentifier());
+                return compiledSchema;
+
+            } catch (final Exception e) {
+                throw new RuntimeException("Failed to compile Protobuf schema 
for identifier: " + schemaDefinition.getIdentifier(), e);
+            }
+        });
+    }
+
+    /**
+     * Executes a function with a temporary directory, ensuring proper cleanup.
+     *
+     * @param function the function to execute with the temporary directory
+     * @return the result of the function
+     * @throws IOException if unable to create or manage temporary directory
+     */
+    private <T> T executeWithTemporaryDirectory(final 
WithTemporaryDirectory<T> function) throws IOException {
+        final Path tempDir = Files.createTempDirectory("nifi-protobuf-schema");
+        logger.debug("Created temporary directory for schema compilation: {}", 
tempDir);
+
+        try {
+            return function.apply(tempDir);
+        } catch (final Exception e) {
+            throw new RuntimeException(e);
+        } finally {
+            safeDeleteDirectory(tempDir);
+        }
+    }
+
+    private Schema createAndLoadSchema(final Path tempDir) {
+        final SchemaLoader schemaLoader = new 
SchemaLoader(FileSystems.getDefault());
+
+        final List<Location> roots = new ArrayList<>();
+        roots.add(Location.get(tempDir.toString()));
+
+        // Add standard protobuf libraries
+        roots.addAll(STANDARD_PROTOBUF_LOCATIONS);
+
+        schemaLoader.initRoots(roots, Collections.emptyList());
+
+        // Load and return the compiled schema
+        return schemaLoader.loadSchema();
+    }
+
+    private void safeDeleteDirectory(final Path directory) {
+        if (Files.exists(directory)) {
+            try {
+                FileUtils.deleteDirectory(directory.toFile());
+            } catch (final IOException | IllegalArgumentException e) {
+                logger.warn("Failed to delete temporary directory: {}", 
directory, e);
+            }
+        }
+    }
+
+
+    /**
+     * Writes a schema definition to the temporary directory structure.
+     * If package name is present, creates the appropriate directory structure.
+     *
+     * @param tempDir          the temporary directory root
+     * @param schemaDefinition the schema definition to write
+     * @throws IOException if unable to create directories or write files
+     */
+    private void writeSchemaToTempDirectory(final Path tempDir, final 
SchemaDefinition schemaDefinition) throws IOException {
+        logger.debug("Writing schema definition to temporary directory. 
Identifier: {}", schemaDefinition.getIdentifier());
+
+        final String schemaFileName = generateSchemaFileName(schemaDefinition);
+        final Path schemaFile = tempDir.resolve(schemaFileName);
+
+        // Write schema text to file
+        Files.write(schemaFile, schemaDefinition.getText().getBytes(), CREATE, 
WRITE, TRUNCATE_EXISTING);
+        logger.debug("Successfully wrote schema to file: {} (string length: 
{})",
+            schemaFile, schemaDefinition.getText().length());
+    }
+
+    /**
+     * Generates a filename for a schema definition, ensuring it has a .proto 
extension.
+     *
+     * @param schemaDefinition the schema definition
+     * @return the generated filename
+     */
+    private String generateSchemaFileName(final SchemaDefinition 
schemaDefinition) {
+        String schemaFileName = 
schemaDefinition.getIdentifier().getName().orElseGet(
+            () -> 
String.valueOf(schemaDefinition.getIdentifier().getSchemaVersionId().orElse(0L))
+        );
+
+        if (!schemaFileName.endsWith(".proto")) {

Review Comment:
   Recommend declaring a static variable named `PROTO_EXTENSION = ".proto"` and 
reusing through this class.



##########
nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/ProtobufSchemaCompiler.java:
##########
@@ -0,0 +1,245 @@
+/*
+ * 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.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+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.commons.io.FileUtils;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.schemaregistry.services.SchemaDefinition;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
+
+import java.io.IOException;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static 
org.apache.nifi.services.protobuf.ProtobufSchemaValidator.validateSchemaDefinitionIdentifiers;
+
+/**
+ * Handles Protocol Buffer schema compilation, caching, and temporary 
directory operations.
+ * This class is responsible for compiling schema definitions into Wire Schema 
objects,
+ * managing a cache of compiled schemas, and handling temporary directory 
operations
+ * required during the compilation process.
+ */
+final class ProtobufSchemaCompiler {
+
+    private static final List<Location> STANDARD_PROTOBUF_LOCATIONS = 
Arrays.asList(
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/any.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/duration.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/empty.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/struct.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/timestamp.proto"),
+        Location.get(CoreLoaderKt.WIRE_RUNTIME_JAR, 
"google/protobuf/wrappers.proto")
+    );
+    private static final int CACHE_EXPIRE_HOURS = 1;
+    private static final int COMPILED_SCHEMAS_CACHE_SIZE = 200;
+
+    private final Cache<SchemaIdentifier, Schema> compiledSchemaCache;
+    private final ComponentLog logger;
+
+    /**
+     * Creates a new ProtobufSchemaCompiler with default cache settings.
+     *
+     * @param logger the component logger for logging compilation activities
+     */
+    public ProtobufSchemaCompiler(final ComponentLog logger) {
+        this.logger = logger;
+        this.compiledSchemaCache = Caffeine.newBuilder()
+            .expireAfterAccess(CACHE_EXPIRE_HOURS, TimeUnit.HOURS)
+            .maximumSize(COMPILED_SCHEMAS_CACHE_SIZE)
+            .build();
+    }
+
+    /**
+     * Compiles a schema definition or retrieves it from cache.
+     *
+     * @param schemaDefinition the schema definition to compile
+     * @return the compiled Schema
+     */
+    public Schema compileOrGetFromCache(final SchemaDefinition 
schemaDefinition) {
+        return compiledSchemaCache.get(schemaDefinition.getIdentifier(),
+            identifier -> {
+                try {
+                    return compileSchemaDefinition(schemaDefinition);
+                } catch (final IOException e) {
+                    throw new RuntimeException("Could not compile schema for 
identifier: " + identifier, e);
+                }
+            });
+    }
+
+    /**
+     * Compiles a SchemaDefinition structure into a Schema using the wire 
library.
+     * Creates a temporary directory structure that mirrors the package 
structure and
+     * places all schemas in their appropriate directories.
+     *
+     * @param schemaDefinition the main schema definition to compile
+     * @return the compiled Schema
+     * @throws IOException if unable to create temporary files or compile 
schema
+     */
+    private Schema compileSchemaDefinition(final SchemaDefinition 
schemaDefinition) throws IOException {
+        logger.debug("Starting schema compilation for identifier: {}", 
schemaDefinition.getIdentifier());
+
+        // Validate that all schema identifiers end with .proto extension
+        validateSchemaDefinitionIdentifiers(schemaDefinition, true);
+
+        return executeWithTemporaryDirectory(tempDir -> {
+            try {
+                // Process main schema definition
+                writeSchemaToTempDirectory(tempDir, schemaDefinition);
+
+                // Process all referenced schemas recursively
+                processSchemaReferences(tempDir, 
schemaDefinition.getReferences());
+
+                // Create and configure schema loader
+                final Schema compiledSchema = createAndLoadSchema(tempDir);
+                logger.debug("Successfully compiled schema for identifier: 
{}", schemaDefinition.getIdentifier());
+                return compiledSchema;
+
+            } catch (final Exception e) {
+                throw new RuntimeException("Failed to compile Protobuf schema 
for identifier: " + schemaDefinition.getIdentifier(), e);
+            }
+        });
+    }
+
+    /**
+     * Executes a function with a temporary directory, ensuring proper cleanup.
+     *
+     * @param function the function to execute with the temporary directory
+     * @return the result of the function
+     * @throws IOException if unable to create or manage temporary directory
+     */
+    private <T> T executeWithTemporaryDirectory(final 
WithTemporaryDirectory<T> function) throws IOException {
+        final Path tempDir = Files.createTempDirectory("nifi-protobuf-schema");
+        logger.debug("Created temporary directory for schema compilation: {}", 
tempDir);
+
+        try {
+            return function.apply(tempDir);
+        } catch (final Exception e) {
+            throw new RuntimeException(e);
+        } finally {
+            safeDeleteDirectory(tempDir);
+        }
+    }
+
+    private Schema createAndLoadSchema(final Path tempDir) {
+        final SchemaLoader schemaLoader = new 
SchemaLoader(FileSystems.getDefault());
+
+        final List<Location> roots = new ArrayList<>();
+        roots.add(Location.get(tempDir.toString()));
+
+        // Add standard protobuf libraries
+        roots.addAll(STANDARD_PROTOBUF_LOCATIONS);
+
+        schemaLoader.initRoots(roots, Collections.emptyList());
+
+        // Load and return the compiled schema
+        return schemaLoader.loadSchema();
+    }
+
+    private void safeDeleteDirectory(final Path directory) {
+        if (Files.exists(directory)) {
+            try {
+                FileUtils.deleteDirectory(directory.toFile());
+            } catch (final IOException | IllegalArgumentException e) {
+                logger.warn("Failed to delete temporary directory: {}", 
directory, e);
+            }
+        }
+    }
+
+
+    /**
+     * Writes a schema definition to the temporary directory structure.
+     * If package name is present, creates the appropriate directory structure.
+     *
+     * @param tempDir          the temporary directory root
+     * @param schemaDefinition the schema definition to write
+     * @throws IOException if unable to create directories or write files
+     */
+    private void writeSchemaToTempDirectory(final Path tempDir, final 
SchemaDefinition schemaDefinition) throws IOException {
+        logger.debug("Writing schema definition to temporary directory. 
Identifier: {}", schemaDefinition.getIdentifier());
+
+        final String schemaFileName = generateSchemaFileName(schemaDefinition);
+        final Path schemaFile = tempDir.resolve(schemaFileName);
+
+        // Write schema text to file
+        Files.write(schemaFile, schemaDefinition.getText().getBytes(), CREATE, 
WRITE, TRUNCATE_EXISTING);
+        logger.debug("Successfully wrote schema to file: {} (string length: 
{})",
+            schemaFile, schemaDefinition.getText().length());
+    }
+
+    /**
+     * Generates a filename for a schema definition, ensuring it has a .proto 
extension.
+     *
+     * @param schemaDefinition the schema definition
+     * @return the generated filename
+     */
+    private String generateSchemaFileName(final SchemaDefinition 
schemaDefinition) {
+        String schemaFileName = 
schemaDefinition.getIdentifier().getName().orElseGet(
+            () -> 
String.valueOf(schemaDefinition.getIdentifier().getSchemaVersionId().orElse(0L))
+        );
+
+        if (!schemaFileName.endsWith(".proto")) {
+            schemaFileName += ".proto"; // Ensure the file ends with .proto, 
otherwise the wire library will not recognize it
+        }
+
+        return schemaFileName;
+    }
+
+    private void processSchemaReferences(final Path tempDir, final Map<String, 
SchemaDefinition> references) throws IOException {
+        logger.debug("Processing schema references. Count: {}, Temp directory: 
{}",

Review Comment:
   Recommend adjusting the wording:
   ```suggestion
           logger.debug("Processing [{}] schema references in [{}]",
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to