diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenTCP.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenTCP.java index 70a5223da9d1..c5cb88da41c5 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenTCP.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenTCP.java @@ -37,6 +37,7 @@ import org.apache.nifi.event.transport.netty.ByteArrayMessageNettyEventServerFactory; import org.apache.nifi.event.transport.netty.NettyEventServerFactory; import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.migration.PropertyConfiguration; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; @@ -112,16 +113,6 @@ public class ListenTCP extends AbstractProcessor { .defaultValue(ClientAuth.REQUIRED.name()) .build(); - // Deprecated - public static final PropertyDescriptor MAX_RECV_THREAD_POOL_SIZE = new PropertyDescriptor.Builder() - .name("max-receiving-threads") - .displayName("Max Number of Receiving Message Handler Threads") - .description( - "This property is deprecated and no longer used.") - .addValidator(StandardValidators.createLongValidator(1, 65535, true)) - .required(false) - .build(); - protected static final PropertyDescriptor POOL_RECV_BUFFERS = new PropertyDescriptor.Builder() .name("pool-receive-buffers") .displayName("Pool Receive Buffers") @@ -152,8 +143,6 @@ public class ListenTCP extends AbstractProcessor { ListenerProperties.MAX_BATCH_SIZE, ListenerProperties.MESSAGE_DELIMITER, IDLE_CONNECTION_TIMEOUT, - // Deprecated - MAX_RECV_THREAD_POOL_SIZE, POOL_RECV_BUFFERS, SSL_CONTEXT_SERVICE, CLIENT_AUTH @@ -177,6 +166,11 @@ public class ListenTCP extends AbstractProcessor { protected volatile byte[] messageDemarcatorBytes; protected volatile EventBatcher eventBatcher; + @Override + public void migrateProperties(PropertyConfiguration config) { + config.removeProperty("max-receiving-threads"); + } + @OnScheduled public void onScheduled(ProcessContext context) throws IOException { int workerThreads = context.getProperty(ListenerProperties.WORKER_THREADS).asInteger();