You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by tp...@apache.org on 2023/02/03 13:06:02 UTC

[nifi] branch commit created (now 0c171bbb08)

This is an automated email from the ASF dual-hosted git repository.

tpalfy pushed a change to branch commit
in repository https://gitbox.apache.org/repos/asf/nifi.git


      at 0c171bbb08 NIFI-10846: Added SMB Dialect and Use Encryption properties in SMB processors

This branch includes the following new commits:

     new 0c171bbb08 NIFI-10846: Added SMB Dialect and Use Encryption properties in SMB processors

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[nifi] 01/01: NIFI-10846: Added SMB Dialect and Use Encryption properties in SMB processors

Posted by tp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tpalfy pushed a commit to branch commit
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 0c171bbb08160709d47890b6cd786a0c0575af3b
Author: Peter Turcsanyi <tu...@apache.org>
AuthorDate: Wed Feb 1 16:31:31 2023 +0100

    NIFI-10846: Added SMB Dialect and Use Encryption properties in SMB processors
    
    This closes #6917.
    
    Signed-off-by: Tamas Palfy <tp...@apache.org>
---
 .../nifi-smb-bundle/nifi-smb-processors/pom.xml    |  5 ++
 .../org/apache/nifi/processors/smb/GetSmbFile.java | 29 +++++---
 .../org/apache/nifi/processors/smb/PutSmbFile.java | 76 +++++++++++---------
 .../apache/nifi/processors/smb/GetSmbFileTest.java | 12 ++--
 .../apache/nifi/processors/smb/PutSmbFileTest.java | 12 ++--
 .../nifi-smb-bundle/nifi-smb-smbj-client/pom.xml   |  5 ++
 .../services/smb/SmbjClientProviderService.java    | 58 ++++++++--------
 .../apache/nifi/services/smb/NiFiSmbjClientIT.java |  2 +-
 .../pom.xml                                        | 34 ++-------
 .../org/apache/nifi/smb/common/SmbDialect.java     | 58 ++++++++++++++++
 .../org/apache/nifi/smb/common/SmbProperties.java  | 55 +++++++++++++++
 .../java/org/apache/nifi/smb/common/SmbUtils.java  | 59 ++++++++++++++++
 .../org/apache/nifi/smb/common/SmbUtilsTest.java   | 80 ++++++++++++++++++++++
 nifi-nar-bundles/nifi-smb-bundle/pom.xml           |  1 +
 14 files changed, 374 insertions(+), 112 deletions(-)

diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/pom.xml b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/pom.xml
index ccb53d2a87..31e9efdaca 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/pom.xml
@@ -49,6 +49,11 @@
             <artifactId>nifi-utils</artifactId>
             <version>1.20.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-smb-smbj-common</artifactId>
+            <version>1.20.0-SNAPSHOT</version>
+        </dependency>
         <dependency>
             <groupId>com.hierynomus</groupId>
             <artifactId>smbj</artifactId>
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/GetSmbFile.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/GetSmbFile.java
index c03a846ab4..244ae4639f 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/GetSmbFile.java
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/GetSmbFile.java
@@ -40,6 +40,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
@@ -79,6 +80,10 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.regex.Pattern;
 
+import static org.apache.nifi.smb.common.SmbProperties.SMB_DIALECT;
+import static org.apache.nifi.smb.common.SmbProperties.USE_ENCRYPTION;
+import static org.apache.nifi.smb.common.SmbUtils.buildSmbClient;
+
 @TriggerWhenEmpty
 @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
 @Tags({"samba, smb, cifs, files, get"})
@@ -249,15 +254,13 @@ public class GetSmbFile extends AbstractProcessor {
         descriptors.add(RECURSE);
         descriptors.add(POLLING_INTERVAL);
         descriptors.add(IGNORE_HIDDEN_FILES);
+        descriptors.add(SMB_DIALECT);
+        descriptors.add(USE_ENCRYPTION);
         this.descriptors = Collections.unmodifiableList(descriptors);
 
         final Set<Relationship> relationships = new HashSet<Relationship>();
         relationships.add(REL_SUCCESS);
         this.relationships = Collections.unmodifiableSet(relationships);
-
-        if (this.smbClient == null) {
-            initSmbClient();
-        }
     }
 
     @Override
@@ -272,6 +275,8 @@ public class GetSmbFile extends AbstractProcessor {
 
     @OnScheduled
     public void onScheduled(final ProcessContext context) {
+        smbClient = initSmbClient(context);
+
         initiateFilterFile(context);
         fileQueue.clear();
 
@@ -291,6 +296,14 @@ public class GetSmbFile extends AbstractProcessor {
         }
     }
 
+    @OnStopped
+    public void onStopped() {
+        if (smbClient != null) {
+            smbClient.close();
+            smbClient = null;
+        }
+    }
+
     @Override
     protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
         Collection<ValidationResult> set = new ArrayList<>();
@@ -300,12 +313,8 @@ public class GetSmbFile extends AbstractProcessor {
         return set;
     }
 
-    private void initSmbClient() {
-        initSmbClient(new SMBClient());
-    }
-
-    public void initSmbClient(SMBClient smbClient) {
-        this.smbClient = smbClient;
+    SMBClient initSmbClient(final ProcessContext context) {
+        return buildSmbClient(context);
     }
 
     private void initiateFilterFile(final ProcessContext context) {
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/PutSmbFile.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/PutSmbFile.java
index 43196d85a1..b7ee444ab9 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/PutSmbFile.java
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/PutSmbFile.java
@@ -16,52 +16,57 @@
  */
 package org.apache.nifi.processors.smb;
 
+
+import com.hierynomus.msdtyp.AccessMask;
+import com.hierynomus.msfscc.FileAttributes;
+import com.hierynomus.mssmb2.SMB2CreateDisposition;
+import com.hierynomus.mssmb2.SMB2CreateOptions;
+import com.hierynomus.mssmb2.SMB2ShareAccess;
+import com.hierynomus.smbj.SMBClient;
+import com.hierynomus.smbj.auth.AuthenticationContext;
+import com.hierynomus.smbj.connection.Connection;
+import com.hierynomus.smbj.session.Session;
+import com.hierynomus.smbj.share.DiskEntry;
+import com.hierynomus.smbj.share.DiskShare;
+import com.hierynomus.smbj.share.File;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.annotation.behavior.ReadsAttribute;
 import org.apache.nifi.annotation.behavior.ReadsAttributes;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.SeeAlso;
 import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.ProcessorInitializationContext;
 import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.logging.ComponentLog;
 
+import java.io.OutputStream;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.net.URI;
 
-import com.hierynomus.smbj.SMBClient;
-import com.hierynomus.smbj.connection.Connection;
-import com.hierynomus.smbj.auth.AuthenticationContext;
-import com.hierynomus.smbj.share.DiskEntry;
-import com.hierynomus.smbj.share.DiskShare;
-import com.hierynomus.smbj.session.Session;
-import com.hierynomus.msfscc.FileAttributes;
-import com.hierynomus.msdtyp.AccessMask;
-import com.hierynomus.mssmb2.SMB2ShareAccess;
-import com.hierynomus.mssmb2.SMB2CreateDisposition;
-import com.hierynomus.mssmb2.SMB2CreateOptions;
-import com.hierynomus.smbj.share.File;
-import java.io.OutputStream;
-import java.util.EnumSet;
+import static org.apache.nifi.smb.common.SmbProperties.SMB_DIALECT;
+import static org.apache.nifi.smb.common.SmbProperties.USE_ENCRYPTION;
+import static org.apache.nifi.smb.common.SmbUtils.buildSmbClient;
 
 @InputRequirement(Requirement.INPUT_REQUIRED)
 @Tags({"samba, smb, cifs, files, put"})
@@ -186,16 +191,14 @@ public class PutSmbFile extends AbstractProcessor {
         descriptors.add(CONFLICT_RESOLUTION);
         descriptors.add(BATCH_SIZE);
         descriptors.add(RENAME_SUFFIX);
+        descriptors.add(SMB_DIALECT);
+        descriptors.add(USE_ENCRYPTION);
         this.descriptors = Collections.unmodifiableList(descriptors);
 
         final Set<Relationship> relationships = new HashSet<Relationship>();
         relationships.add(REL_SUCCESS);
         relationships.add(REL_FAILURE);
         this.relationships = Collections.unmodifiableSet(relationships);
-
-        if (this.smbClient == null) {
-            initSmbClient();
-        }
     }
 
     @Override
@@ -210,6 +213,7 @@ public class PutSmbFile extends AbstractProcessor {
 
     @OnScheduled
     public void onScheduled(final ProcessContext context) {
+        smbClient = initSmbClient(context);
 
         switch (context.getProperty(SHARE_ACCESS).getValue()) {
             case SHARE_ACCESS_NONE:
@@ -227,6 +231,14 @@ public class PutSmbFile extends AbstractProcessor {
         }
     }
 
+    @OnStopped
+    public void onStopped() {
+        if (smbClient != null) {
+            smbClient.close();
+            smbClient = null;
+        }
+    }
+
     @Override
     protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
         Collection<ValidationResult> set = new ArrayList<>();
@@ -236,12 +248,8 @@ public class PutSmbFile extends AbstractProcessor {
         return set;
     }
 
-    private void initSmbClient() {
-        initSmbClient(new SMBClient());
-    }
-
-    void initSmbClient(SMBClient smbClient) {
-        this.smbClient = smbClient;
+    SMBClient initSmbClient(final ProcessContext context) {
+        return buildSmbClient(context);
     }
 
     private void createMissingDirectoriesRecursevly(ComponentLog logger, DiskShare share, String pathToCreate) {
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/test/java/org/apache/nifi/processors/smb/GetSmbFileTest.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/test/java/org/apache/nifi/processors/smb/GetSmbFileTest.java
index da209a06bb..b9584f6aec 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/test/java/org/apache/nifi/processors/smb/GetSmbFileTest.java
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/test/java/org/apache/nifi/processors/smb/GetSmbFileTest.java
@@ -30,6 +30,7 @@ import com.hierynomus.smbj.session.Session;
 import com.hierynomus.smbj.share.DiskShare;
 import com.hierynomus.smbj.share.File;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
@@ -90,10 +91,6 @@ public class GetSmbFileTest {
         testRunner.setProperty(GetSmbFile.DIRECTORY, DIRECTORY);
         testRunner.setProperty(GetSmbFile.USERNAME, USERNAME);
         testRunner.setProperty(GetSmbFile.PASSWORD, PASSWORD);
-
-        GetSmbFile GetSmbFile = (GetSmbFile) testRunner.getProcessor();
-        GetSmbFile.initSmbClient(smbClient);
-
     }
 
     private FileIdBothDirectoryInformation mockFile(String path, String filename, String fileContent, long fileAttributes) {
@@ -161,7 +158,12 @@ public class GetSmbFileTest {
 
     @BeforeEach
     public void init() throws IOException {
-        testRunner = TestRunners.newTestRunner(GetSmbFile.class);
+        testRunner = TestRunners.newTestRunner(new GetSmbFile() {
+            @Override
+            SMBClient initSmbClient(ProcessContext context) {
+                return smbClient;
+            }
+        });
         setupSmbProcessor();
     }
 
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/test/java/org/apache/nifi/processors/smb/PutSmbFileTest.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/test/java/org/apache/nifi/processors/smb/PutSmbFileTest.java
index e6192caf7d..01b416f07c 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/test/java/org/apache/nifi/processors/smb/PutSmbFileTest.java
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/test/java/org/apache/nifi/processors/smb/PutSmbFileTest.java
@@ -26,6 +26,7 @@ import com.hierynomus.smbj.session.Session;
 import com.hierynomus.smbj.share.DiskEntry;
 import com.hierynomus.smbj.share.DiskShare;
 import com.hierynomus.smbj.share.File;
+import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.jupiter.api.BeforeEach;
@@ -114,10 +115,6 @@ public class PutSmbFileTest {
         testRunner.setProperty(PutSmbFile.DOMAIN, DOMAIN);
         testRunner.setProperty(PutSmbFile.USERNAME, USERNAME);
         testRunner.setProperty(PutSmbFile.PASSWORD, PASSWORD);
-
-
-        PutSmbFile PutSmbFile = (PutSmbFile) testRunner.getProcessor();
-        PutSmbFile.initSmbClient(smbClient);
     }
 
     private void testDirectoryCreation(String dirFlag, int times) throws IOException {
@@ -147,7 +144,12 @@ public class PutSmbFileTest {
 
     @BeforeEach
     public void init() throws IOException {
-        testRunner = TestRunners.newTestRunner(PutSmbFile.class);
+        testRunner = TestRunners.newTestRunner(new PutSmbFile() {
+            @Override
+            SMBClient initSmbClient(ProcessContext context) {
+                return smbClient;
+            }
+        });
         MockitoAnnotations.initMocks(this);
         setupSmbProcessor();
     }
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/pom.xml b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/pom.xml
index bba7ddddcc..9c45b5365f 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/pom.xml
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/pom.xml
@@ -45,6 +45,11 @@
             <artifactId>nifi-utils</artifactId>
             <version>1.20.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-smb-smbj-common</artifactId>
+            <version>1.20.0-SNAPSHOT</version>
+        </dependency>
         <dependency>
             <groupId>com.hierynomus</groupId>
             <artifactId>smbj</artifactId>
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/src/main/java/org/apache/nifi/services/smb/SmbjClientProviderService.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/src/main/java/org/apache/nifi/services/smb/SmbjClientProviderService.java
index 70c9260a62..080be06199 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/src/main/java/org/apache/nifi/services/smb/SmbjClientProviderService.java
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/src/main/java/org/apache/nifi/services/smb/SmbjClientProviderService.java
@@ -16,21 +16,8 @@
  */
 package org.apache.nifi.services.smb;
 
-import static java.util.Arrays.asList;
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.nifi.processor.util.StandardValidators.NON_BLANK_VALIDATOR;
-import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
-import static org.apache.nifi.processor.util.StandardValidators.PORT_VALIDATOR;
-import static org.apache.nifi.processor.util.StandardValidators.TIME_PERIOD_VALIDATOR;
-
 import com.hierynomus.smbj.SMBClient;
-import com.hierynomus.smbj.SmbConfig;
 import com.hierynomus.smbj.auth.AuthenticationContext;
-import java.io.IOException;
-import java.net.URI;
-import java.util.Collections;
-import java.util.List;
-
 import com.hierynomus.smbj.connection.Connection;
 import com.hierynomus.smbj.session.Session;
 import com.hierynomus.smbj.share.DiskShare;
@@ -43,6 +30,20 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 
+import java.io.IOException;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+
+import static java.util.Arrays.asList;
+import static org.apache.nifi.processor.util.StandardValidators.NON_BLANK_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.PORT_VALIDATOR;
+import static org.apache.nifi.smb.common.SmbProperties.SMB_DIALECT;
+import static org.apache.nifi.smb.common.SmbProperties.TIMEOUT;
+import static org.apache.nifi.smb.common.SmbProperties.USE_ENCRYPTION;
+import static org.apache.nifi.smb.common.SmbUtils.buildSmbClient;
+
 @Tags({"samba, smb, cifs, files"})
 @CapabilityDescription("Provides access to SMB Sessions with shared authentication credentials.")
 public class SmbjClientProviderService extends AbstractControllerService implements SmbClientProviderService {
@@ -54,6 +55,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
             .required(true)
             .addValidator(NON_BLANK_VALIDATOR)
             .build();
+
     public static final PropertyDescriptor DOMAIN = new PropertyDescriptor.Builder()
             .displayName("Domain")
             .name("domain")
@@ -62,6 +64,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
             .required(false)
             .addValidator(NON_EMPTY_VALIDATOR)
             .build();
+
     public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
             .displayName("Username")
             .name("username")
@@ -71,6 +74,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
             .defaultValue("Guest")
             .addValidator(NON_EMPTY_VALIDATOR)
             .build();
+
     public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
             .displayName("Password")
             .name("password")
@@ -79,6 +83,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
             .addValidator(NON_EMPTY_VALIDATOR)
             .sensitive(true)
             .build();
+
     public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
             .displayName("Port")
             .name("port")
@@ -87,6 +92,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
             .addValidator(PORT_VALIDATOR)
             .defaultValue("445")
             .build();
+
     public static final PropertyDescriptor SHARE = new PropertyDescriptor.Builder()
             .displayName("Share")
             .name("share")
@@ -95,14 +101,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
             .required(true)
             .addValidator(NON_BLANK_VALIDATOR)
             .build();
-    public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
-            .displayName("Timeout")
-            .name("timeout")
-            .description("Timeout for read and write operations.")
-            .required(true)
-            .defaultValue("5 sec")
-            .addValidator(TIME_PERIOD_VALIDATOR)
-            .build();
+
     private static final List<PropertyDescriptor> PROPERTIES = Collections
             .unmodifiableList(asList(
                     HOSTNAME,
@@ -111,8 +110,11 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
                     USERNAME,
                     PASSWORD,
                     DOMAIN,
+                    SMB_DIALECT,
+                    USE_ENCRYPTION,
                     TIMEOUT
             ));
+
     private SMBClient smbClient;
     private AuthenticationContext authenticationContext;
     private String hostname;
@@ -137,7 +139,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
         }
     }
 
-    private SmbjClientService connectToShare(Connection connection) throws IOException {
+    private SmbjClientService connectToShare(final Connection connection) throws IOException {
         final Session session;
         final Share share;
 
@@ -166,7 +168,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
         smbClient.getServerList().unregister(hostname);
     }
 
-    private void closeConnection(Connection connection) {
+    private void closeConnection(final Connection connection) {
         try {
             if (connection != null) {
                 connection.close(true);
@@ -176,7 +178,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
         }
     }
 
-    private void closeSession(Session session) {
+    private void closeSession(final Session session) {
         try {
             if (session != null) {
                 session.close();
@@ -192,13 +194,11 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
     }
 
     @OnEnabled
-    public void onEnabled(ConfigurationContext context) {
+    public void onEnabled(final ConfigurationContext context) {
         this.hostname = context.getProperty(HOSTNAME).getValue();
         this.port = context.getProperty(PORT).asInteger();
         this.shareName = context.getProperty(SHARE).getValue();
-        this.smbClient = new SMBClient(SmbConfig.builder()
-                .withTimeout(context.getProperty(TIMEOUT).asTimePeriod(MILLISECONDS), MILLISECONDS)
-                .build());
+        this.smbClient = buildSmbClient(context);
         createAuthenticationContext(context);
     }
 
@@ -216,7 +216,7 @@ public class SmbjClientProviderService extends AbstractControllerService impleme
         return PROPERTIES;
     }
 
-    private void createAuthenticationContext(ConfigurationContext context) {
+    private void createAuthenticationContext(final ConfigurationContext context) {
         if (context.getProperty(USERNAME).isSet()) {
             final String userName = context.getProperty(USERNAME).getValue();
             final String password =
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/src/test/java/org/apache/nifi/services/smb/NiFiSmbjClientIT.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/src/test/java/org/apache/nifi/services/smb/NiFiSmbjClientIT.java
index f3de70ce71..a030abc363 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/src/test/java/org/apache/nifi/services/smb/NiFiSmbjClientIT.java
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/src/test/java/org/apache/nifi/services/smb/NiFiSmbjClientIT.java
@@ -22,8 +22,8 @@ import static org.apache.nifi.services.smb.SmbjClientProviderService.HOSTNAME;
 import static org.apache.nifi.services.smb.SmbjClientProviderService.PASSWORD;
 import static org.apache.nifi.services.smb.SmbjClientProviderService.PORT;
 import static org.apache.nifi.services.smb.SmbjClientProviderService.SHARE;
-import static org.apache.nifi.services.smb.SmbjClientProviderService.TIMEOUT;
 import static org.apache.nifi.services.smb.SmbjClientProviderService.USERNAME;
+import static org.apache.nifi.smb.common.SmbProperties.TIMEOUT;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 import static org.mockito.Mockito.mock;
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/pom.xml b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/pom.xml
similarity index 59%
copy from nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/pom.xml
copy to nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/pom.xml
index bba7ddddcc..7f9f63d5e8 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-client/pom.xml
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/pom.xml
@@ -13,7 +13,9 @@
   See the License for the specific language governing permissions and
   limitations under the License.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -22,24 +24,13 @@
         <version>1.20.0-SNAPSHOT</version>
     </parent>
 
-    <artifactId>nifi-smb-smbj-client</artifactId>
-    <packaging>jar</packaging>
+    <artifactId>nifi-smb-smbj-common</artifactId>
 
     <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-smb-client-api</artifactId>
-            <version>1.20.0-SNAPSHOT</version>
-            <scope>provided</scope>
-        </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
         </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-record</artifactId>
-        </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
@@ -49,25 +40,12 @@
             <groupId>com.hierynomus</groupId>
             <artifactId>smbj</artifactId>
         </dependency>
-        <dependency>
-            <groupId>org.bouncycastle</groupId>
-            <artifactId>bcprov-jdk18on</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.testcontainers</groupId>
-            <artifactId>testcontainers</artifactId>
-            <scope>test</scope>
-        </dependency>
+
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
             <version>1.20.0-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>org.testcontainers</groupId>
-            <artifactId>toxiproxy</artifactId>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
-</project>
+</project>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/main/java/org/apache/nifi/smb/common/SmbDialect.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/main/java/org/apache/nifi/smb/common/SmbDialect.java
new file mode 100644
index 0000000000..b446ac8fb7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/main/java/org/apache/nifi/smb/common/SmbDialect.java
@@ -0,0 +1,58 @@
+/*
+ * 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.smb.common;
+
+import com.hierynomus.mssmb2.SMB2Dialect;
+import org.apache.nifi.components.DescribedValue;
+
+public enum SmbDialect implements DescribedValue {
+
+    AUTO("AUTO", null),
+    SMB_2_0_2("SMB 2.0.2", SMB2Dialect.SMB_2_0_2),
+    SMB_2_1("SMB 2.1", SMB2Dialect.SMB_2_1),
+    SMB_3_0("SMB 3.0", SMB2Dialect.SMB_3_0),
+    SMB_3_0_2("SMB 3.0.2", SMB2Dialect.SMB_3_0_2),
+    SMB_3_1_1("SMB 3.1.1", SMB2Dialect.SMB_3_1_1);
+
+    private final String displayName;
+
+    private final SMB2Dialect smbjDialect;
+
+    SmbDialect(String displayName, SMB2Dialect smbjDialect) {
+        this.displayName = displayName;
+        this.smbjDialect = smbjDialect;
+    }
+
+    @Override
+    public String getValue() {
+        return name();
+    }
+
+    @Override
+    public String getDisplayName() {
+        return displayName;
+    }
+
+    @Override
+    public String getDescription() {
+        return null;
+    }
+
+    public SMB2Dialect getSmbjDialect() {
+        return smbjDialect;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/main/java/org/apache/nifi/smb/common/SmbProperties.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/main/java/org/apache/nifi/smb/common/SmbProperties.java
new file mode 100644
index 0000000000..5b474fcc94
--- /dev/null
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/main/java/org/apache/nifi/smb/common/SmbProperties.java
@@ -0,0 +1,55 @@
+/*
+ * 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.smb.common;
+
+import org.apache.nifi.components.PropertyDescriptor;
+
+import static org.apache.nifi.processor.util.StandardValidators.TIME_PERIOD_VALIDATOR;
+
+public class SmbProperties {
+
+    public static final PropertyDescriptor SMB_DIALECT = new PropertyDescriptor.Builder()
+            .name("smb-dialect")
+            .displayName("SMB Dialect")
+            .description("The SMB dialect is negotiated between the client and the server by default to the highest common version supported by both end. " +
+                    "In some rare cases, the client-server communication may fail with the automatically negotiated dialect. This property can be used to set the dialect explicitly " +
+                    "(e.g. to downgrade to a lower version), when those situations would occur.")
+            .required(true)
+            .allowableValues(SmbDialect.class)
+            .defaultValue(SmbDialect.AUTO.getValue())
+            .build();
+
+    public static final PropertyDescriptor USE_ENCRYPTION = new PropertyDescriptor.Builder()
+            .name("use-encryption")
+            .displayName("Use Encryption")
+            .description("Turns on/off encrypted communication between the client and the server. The property's behavior is SMB dialect dependent: " +
+                    "SMB 2.x does not support encryption and the property has no effect. " +
+                    "In case of SMB 3.x, it is a hint/request to the server to turn encryption on if the server also supports it.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Timeout")
+            .name("timeout")
+            .description("Timeout for read and write operations.")
+            .required(true)
+            .defaultValue("5 sec")
+            .addValidator(TIME_PERIOD_VALIDATOR)
+            .build();
+}
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/main/java/org/apache/nifi/smb/common/SmbUtils.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/main/java/org/apache/nifi/smb/common/SmbUtils.java
new file mode 100644
index 0000000000..0895abfae0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/main/java/org/apache/nifi/smb/common/SmbUtils.java
@@ -0,0 +1,59 @@
+/*
+ * 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.smb.common;
+
+import com.hierynomus.smbj.SMBClient;
+import com.hierynomus.smbj.SmbConfig;
+import org.apache.nifi.context.PropertyContext;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.nifi.smb.common.SmbProperties.SMB_DIALECT;
+import static org.apache.nifi.smb.common.SmbProperties.TIMEOUT;
+import static org.apache.nifi.smb.common.SmbProperties.USE_ENCRYPTION;
+
+public final class SmbUtils {
+
+    private SmbUtils() {
+        // util class' constructor
+    }
+
+    public static SMBClient buildSmbClient(final PropertyContext context) {
+        return new SMBClient(buildSmbConfig(context));
+    }
+
+    static SmbConfig buildSmbConfig(final PropertyContext context) {
+        final SmbConfig.Builder configBuilder = SmbConfig.builder();
+
+        if (context.getProperty(SMB_DIALECT).isSet()) {
+            final SmbDialect dialect = SmbDialect.valueOf(context.getProperty(SMB_DIALECT).getValue());
+
+            if (dialect != SmbDialect.AUTO) {
+                configBuilder.withDialects(dialect.getSmbjDialect());
+            }
+        }
+
+        if (context.getProperty(USE_ENCRYPTION).isSet()) {
+            configBuilder.withEncryptData(context.getProperty(USE_ENCRYPTION).asBoolean());
+        }
+
+        if (context.getProperty(TIMEOUT).isSet()) {
+            configBuilder.withTimeout(context.getProperty(TIMEOUT).asTimePeriod(MILLISECONDS), MILLISECONDS);
+        }
+
+        return configBuilder.build();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/test/java/org/apache/nifi/smb/common/SmbUtilsTest.java b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/test/java/org/apache/nifi/smb/common/SmbUtilsTest.java
new file mode 100644
index 0000000000..4ee0cad4bd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-smb-bundle/nifi-smb-smbj-common/src/test/java/org/apache/nifi/smb/common/SmbUtilsTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.smb.common;
+
+import com.hierynomus.mssmb2.SMB2Dialect;
+import com.hierynomus.smbj.SmbConfig;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.util.MockPropertyContext;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static com.hierynomus.mssmb2.SMB2Dialect.SMB_2_0_2;
+import static com.hierynomus.mssmb2.SMB2Dialect.SMB_2_1;
+import static com.hierynomus.mssmb2.SMB2Dialect.SMB_3_0;
+import static com.hierynomus.mssmb2.SMB2Dialect.SMB_3_0_2;
+import static com.hierynomus.mssmb2.SMB2Dialect.SMB_3_1_1;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class SmbUtilsTest {
+
+    @Test
+    public void testSmbConfigDefault() {
+        Map<PropertyDescriptor, String> properties = new HashMap<>();
+
+        properties.put(SmbProperties.SMB_DIALECT, null);
+        properties.put(SmbProperties.USE_ENCRYPTION, null);
+        properties.put(SmbProperties.TIMEOUT, null);
+
+        MockPropertyContext propertyContext = new MockPropertyContext(properties);
+
+        SmbConfig config = SmbUtils.buildSmbConfig(propertyContext);
+
+        assertSmbConfig(config, new HashSet<>(Arrays.asList(SMB_3_1_1, SMB_3_0_2, SMB_3_0, SMB_2_1, SMB_2_0_2)), false, 5_000);
+    }
+
+    @Test
+    public void testSmbConfigNonDefault() {
+        Map<PropertyDescriptor, String> properties = new HashMap<>();
+
+        properties.put(SmbProperties.SMB_DIALECT, SmbDialect.SMB_3_1_1.getValue());
+        properties.put(SmbProperties.USE_ENCRYPTION, "true");
+        properties.put(SmbProperties.TIMEOUT, "30 s");
+
+        MockPropertyContext propertyContext = new MockPropertyContext(properties);
+
+        SmbConfig config = SmbUtils.buildSmbConfig(propertyContext);
+
+        assertSmbConfig(config, Collections.singleton(SMB_3_1_1), true, 30_000);
+    }
+
+    private void assertSmbConfig(SmbConfig config, Set<SMB2Dialect> expectedDialects, boolean expectedEncryption, long expectedTimeout) {
+        assertEquals(expectedDialects, config.getSupportedDialects());
+
+        assertEquals(expectedEncryption, config.isEncryptData());
+
+        assertEquals(expectedTimeout, config.getReadTimeout());
+        assertEquals(expectedTimeout, config.getWriteTimeout());
+        assertEquals(expectedTimeout, config.getTransactTimeout());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-smb-bundle/pom.xml b/nifi-nar-bundles/nifi-smb-bundle/pom.xml
index c7d8c7e531..c0391fd482 100644
--- a/nifi-nar-bundles/nifi-smb-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-smb-bundle/pom.xml
@@ -32,6 +32,7 @@
         <module>nifi-smb-smbj-client-nar</module>
         <module>nifi-smb-processors</module>
         <module>nifi-smb-nar</module>
+        <module>nifi-smb-smbj-common</module>
     </modules>
 
     <dependencyManagement>