[GitHub] [nifi] turcsanyip commented on a diff in pull request #7449: NIFI-11334: PutIceberg processor instance interference due same class loader usage

2023-06-29 Thread via GitHub


turcsanyip commented on code in PR #7449:
URL: https://github.com/apache/nifi/pull/7449#discussion_r1246952906


##
nifi-nar-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/AbstractIcebergProcessor.java:
##
@@ -117,6 +122,15 @@ public void onTrigger(ProcessContext context, 
ProcessSession session) throws Pro
 }
 }
 
+@Override
+public String getClassloaderIsolationKey(PropertyContext context) {
+final KerberosUserService kerberosUserService = 
context.getProperty(KERBEROS_USER_SERVICE).asControllerService(KerberosUserService.class);
+if (kerberosUserService != null) {
+return kerberosUserService.getIdentifier();

Review Comment:
   I would suggest using the same classloader isolation key as the one in the 
other hadoop related components: kerberos principal.
   ```
   final KerberosUser kerberosUser = 
kerberosUserService.createKerberosUser();
   return kerberosUser.getPrincipal();
   ```
   The controller service identifier also works but it may be too restrictive. 
It creates separate classloaders for controller services having the same 
principal but they could share the classloader.
   E.g. the user can create a process group with the kerberos service and the 
iceberg processor in it and then copy it multiple times. Not the best design 
because the kerberos service should be extracted in the parent process group in 
this case but I can imagine it happening.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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



[GitHub] [nifi] turcsanyip commented on a diff in pull request #7449: NIFI-11334: PutIceberg processor instance interference due same class loader usage

2023-06-29 Thread via GitHub


turcsanyip commented on code in PR #7449:
URL: https://github.com/apache/nifi/pull/7449#discussion_r1246952906


##
nifi-nar-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/AbstractIcebergProcessor.java:
##
@@ -117,6 +122,15 @@ public void onTrigger(ProcessContext context, 
ProcessSession session) throws Pro
 }
 }
 
+@Override
+public String getClassloaderIsolationKey(PropertyContext context) {
+final KerberosUserService kerberosUserService = 
context.getProperty(KERBEROS_USER_SERVICE).asControllerService(KerberosUserService.class);
+if (kerberosUserService != null) {
+return kerberosUserService.getIdentifier();

Review Comment:
   I would suggest using the same classloader isolation key as the one in the 
other hadoop related components: kerberos principal.
   ```
   final KerberosUser kerberosUser = 
kerberosUserService.createKerberosUser();
   return kerberosUser.getPrincipal();
   ```
   The controller service identifier also works but it may be too restrictive. 
It creates separate classloaders for controller service having the same 
principal but they could share the classloader. The user can create a process 
group with the kerberos service and the iceberg processor in it and then copy 
it multiple times. Not the best design because the kerberos service should be 
extracted in the parent process group in this case but I can imagine it 
happening.



##
nifi-nar-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogService.java:
##
@@ -17,16 +17,18 @@
  */
 package org.apache.nifi.services.iceberg;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.iceberg.catalog.Catalog;
 import org.apache.nifi.controller.ControllerService;
 
+import java.util.Map;
+
 /**
  * Provides a basic connector to Iceberg catalog services.
  */
 public interface IcebergCatalogService extends ControllerService {
 
-Catalog getCatalog();
+IcebergCatalogServiceType getCatalogServiceType();
+
+Map getAdditionalParameters();

Review Comment:
   "additionalParameters" are quite generic and the callers also use 
"properties" for the return value.
   `getCatalogProperties()` would be more descriptive.
   Also, the return type would be `Map` (see 
the comment about `IcebergCatalogProperty` enum above).



##
nifi-nar-bundles/nifi-iceberg-bundle/nifi-iceberg-services-api/src/main/java/org/apache/nifi/services/iceberg/IcebergCatalogService.java:
##
@@ -17,16 +17,18 @@
  */
 package org.apache.nifi.services.iceberg;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.iceberg.catalog.Catalog;
 import org.apache.nifi.controller.ControllerService;
 
+import java.util.Map;
+
 /**
  * Provides a basic connector to Iceberg catalog services.
  */
 public interface IcebergCatalogService extends ControllerService {
 
-Catalog getCatalog();
+IcebergCatalogServiceType getCatalogServiceType();
+
+Map getAdditionalParameters();
 
-Configuration getConfiguration();
+String getConfigFiles();

Review Comment:
   It returns a comma separated list of file paths which is not obvious at 
first. The method could parse the raw property value and return the paths in a 
list:
   ```
   List getConfigFilePaths();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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