This is an automated email from the ASF dual-hosted git repository.
exceptionfactory pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git
The following commit(s) were added to refs/heads/main by this push:
new 2091ecf809 NIFI-14686 Added dependsOn for Kerberos and SSL in
Kafka3ConnectionService (#10043)
2091ecf809 is described below
commit 2091ecf8096a00ddf3f8836223e499408467f581
Author: Peter Turcsanyi <[email protected]>
AuthorDate: Thu Jun 26 15:16:06 2025 +0200
NIFI-14686 Added dependsOn for Kerberos and SSL in Kafka3ConnectionService
(#10043)
- Made Kerberos properties required
- Removed KerberosDelegatingLoginConfigProvider
- Renamed KerberosUserServiceLoginConfigProvider
Signed-off-by: David Handermann <[email protected]>
---
.../shared/component/KafkaClientComponent.java | 14 ++++++--
.../login/DelegatingLoginConfigProvider.java | 2 +-
.../KerberosDelegatingLoginConfigProvider.java | 37 ----------------------
...vider.java => KerberosLoginConfigProvider.java} | 2 +-
4 files changed, 14 insertions(+), 41 deletions(-)
diff --git
a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/component/KafkaClientComponent.java
b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/component/KafkaClientComponent.java
index c8da8e5615..771805870f 100644
---
a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/component/KafkaClientComponent.java
+++
b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/component/KafkaClientComponent.java
@@ -125,15 +125,22 @@ public interface KafkaClientComponent {
.description("Service supporting SSL communication with Kafka
brokers")
.required(false)
.identifiesControllerService(SSLContextService.class)
+ .dependsOn(
+ SECURITY_PROTOCOL,
+ SecurityProtocol.SSL.name(),
+ SecurityProtocol.SASL_SSL.name())
.build();
PropertyDescriptor KERBEROS_SERVICE_NAME = new PropertyDescriptor.Builder()
.name("sasl.kerberos.service.name")
.displayName("Kerberos Service Name")
.description("The service name that matches the primary name of
the Kafka server configured in the broker JAAS configuration")
- .required(false)
+ .required(true)
.addValidator(StandardValidators.NON_BLANK_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
+ .dependsOn(
+ SASL_MECHANISM,
+ SaslMechanism.GSSAPI)
.build();
PropertyDescriptor SELF_CONTAINED_KERBEROS_USER_SERVICE = new
PropertyDescriptor.Builder()
@@ -141,7 +148,10 @@ public interface KafkaClientComponent {
.displayName("Kerberos User Service")
.description("Service supporting user authentication with
Kerberos")
.identifiesControllerService(SelfContainedKerberosUserService.class)
- .required(false)
+ .required(true)
+ .dependsOn(
+ SASL_MECHANISM,
+ SaslMechanism.GSSAPI)
.build();
PropertyDescriptor OAUTH2_ACCESS_TOKEN_PROVIDER_SERVICE = new
PropertyDescriptor.Builder()
diff --git
a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/DelegatingLoginConfigProvider.java
b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/DelegatingLoginConfigProvider.java
index 1d596544a5..9927592369 100644
---
a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/DelegatingLoginConfigProvider.java
+++
b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/DelegatingLoginConfigProvider.java
@@ -29,7 +29,7 @@ public class DelegatingLoginConfigProvider implements
LoginConfigProvider {
private static final LoginConfigProvider SCRAM_PROVIDER = new
ScramLoginConfigProvider();
private static final Map<SaslMechanism, LoginConfigProvider> PROVIDERS =
Map.of(
- SaslMechanism.GSSAPI, new KerberosDelegatingLoginConfigProvider(),
+ SaslMechanism.GSSAPI, new KerberosLoginConfigProvider(),
SaslMechanism.PLAIN, new PlainLoginConfigProvider(),
SaslMechanism.SCRAM_SHA_256, SCRAM_PROVIDER,
SaslMechanism.SCRAM_SHA_512, SCRAM_PROVIDER,
diff --git
a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/KerberosDelegatingLoginConfigProvider.java
b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/KerberosDelegatingLoginConfigProvider.java
deleted file mode 100644
index 88d3afecc7..0000000000
---
a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/KerberosDelegatingLoginConfigProvider.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.kafka.shared.login;
-
-import org.apache.nifi.context.PropertyContext;
-
-/**
- * Kerberos Delegating Login Module implementation of configuration provider
- */
-public class KerberosDelegatingLoginConfigProvider implements
LoginConfigProvider {
- private static final LoginConfigProvider USER_SERVICE_PROVIDER = new
KerberosUserServiceLoginConfigProvider();
-
- /**
- * Get JAAS configuration using configured Kerberos credentials
- *
- * @param context Property Context
- * @return JAAS configuration with Kerberos Login Module
- */
- @Override
- public String getConfiguration(final PropertyContext context) {
- return USER_SERVICE_PROVIDER.getConfiguration(context);
- }
-}
diff --git
a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/KerberosUserServiceLoginConfigProvider.java
b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/KerberosLoginConfigProvider.java
similarity index 96%
rename from
nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/KerberosUserServiceLoginConfigProvider.java
rename to
nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/KerberosLoginConfigProvider.java
index e82207be91..574e892e8e 100644
---
a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/KerberosUserServiceLoginConfigProvider.java
+++
b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-shared/src/main/java/org/apache/nifi/kafka/shared/login/KerberosLoginConfigProvider.java
@@ -29,7 +29,7 @@ import static
org.apache.nifi.kafka.shared.component.KafkaClientComponent.SELF_C
/**
* Kerberos User Service Login Module implementation of configuration provider
*/
-public class KerberosUserServiceLoginConfigProvider implements
LoginConfigProvider {
+public class KerberosLoginConfigProvider implements LoginConfigProvider {
/**
* Get JAAS configuration using configured Kerberos credentials