pvillard31 commented on code in PR #10400:
URL: https://github.com/apache/nifi/pull/10400#discussion_r2452624402


##########
nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/PutIcebergRecord.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.processors.iceberg;
+
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.nifi.processors.iceberg.record.DelegatedRecord;
+import org.apache.nifi.services.iceberg.IcebergCatalog;
+import org.apache.nifi.services.iceberg.IcebergRowWriter;
+import org.apache.nifi.services.iceberg.IcebergWriter;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.time.Clock;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+@Tags({"iceberg", "analytics", "polaris", "s3"})
+@CapabilityDescription("Store records in Iceberg Table using configurable 
Catalog for managing namespaces and tables.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class PutIcebergRecord extends AbstractProcessor {
+
+    static final PropertyDescriptor ICEBERG_CATALOG = new 
PropertyDescriptor.Builder()
+            .name("Iceberg Catalog")
+            .description("Provider Service for Iceberg Catalog")
+            .required(true)
+            .identifiesControllerService(IcebergCatalog.class)
+            .build();
+
+    static final PropertyDescriptor ICEBERG_WRITER = new 
PropertyDescriptor.Builder()
+            .name("Iceberg Writer")
+            .description("Provider Service for Iceberg Row Writers responsible 
for producing formatted Iceberg Data Files")
+            .required(true)
+            .identifiesControllerService(IcebergWriter.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_READER = new 
PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .description("Record Reader for incoming FlowFiles")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor NAMESPACE = new 
PropertyDescriptor.Builder()
+            .name("Namespace")
+            .description("Iceberg Namespace containing Tables")
+            .required(true)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_NAME = new 
PropertyDescriptor.Builder()
+            .name("Table Name")
+            .description("Iceberg Table Name")
+            .required(true)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles transferred to Iceberg")
+            .build();
+
+    static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles not transferred to Iceberg")
+            .build();
+
+    static final String RECORDS_PROCESSED_COUNTER = "Records Processed";
+
+    static final String DATA_FILES_PROCESSED_COUNTER = "Data Files Processed";
+
+    private static final List<PropertyDescriptor> properties = List.of(
+            ICEBERG_CATALOG,
+            ICEBERG_WRITER,
+            RECORD_READER,
+            NAMESPACE,
+            TABLE_NAME
+    );
+
+    private static final Set<Relationship> relationships = Set.of(SUCCESS, 
FAILURE);
+
+    private static final long MAXIMUM_BYTES = 536870912;
+
+    private final Clock clock = Clock.systemDefaultZone();
+
+    private volatile Catalog catalog;
+
+    private volatile IcebergWriter icebergWriter;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final IcebergCatalog icebergCatalog = 
context.getProperty(ICEBERG_CATALOG).asControllerService(IcebergCatalog.class);
+        catalog = icebergCatalog.getCatalog();
+        icebergWriter = 
context.getProperty(ICEBERG_WRITER).asControllerService(IcebergWriter.class);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession 
session) throws ProcessException {
+        final TableIdentifierFlowFileFilter flowFileFilter = new 
TableIdentifierFlowFileFilter(context, MAXIMUM_BYTES);
+        final List<FlowFile> flowFiles = session.get(flowFileFilter);
+        if (flowFiles.isEmpty()) {
+            return;
+        }
+
+        final TableIdentifier tableIdentifier = 
flowFileFilter.getTableIdentifier();
+        final RecordReaderFactory recordReaderFactory = 
context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        processFlowFiles(session, flowFiles, tableIdentifier, 
recordReaderFactory);
+    }
+
+    private void processFlowFiles(final ProcessSession session, final 
List<FlowFile> flowFiles, final TableIdentifier tableIdentifier, final 
RecordReaderFactory recordReaderFactory) {
+        final long started = clock.millis();
+
+        final AtomicReference<Relationship> relationship = new 
AtomicReference<>(SUCCESS);
+
+        final Table table = getTable(tableIdentifier);
+        final Schema schema = table.schema();
+        final Types.StructType struct = schema.asStruct();
+        final IcebergRowWriter rowWriter = icebergWriter.getRowWriter(table);
+
+        for (final FlowFile flowFile : flowFiles) {
+            try (
+                    InputStream inputStream = session.read(flowFile);
+                    RecordReader recordReader = 
recordReaderFactory.createRecordReader(flowFile, inputStream, getLogger())
+            ) {
+                final AtomicLong recordsProcessed = new AtomicLong();
+                try {
+                    writeRecords(recordReader, rowWriter, struct, 
recordsProcessed);
+                    session.adjustCounter(RECORDS_PROCESSED_COUNTER, 
recordsProcessed.get(), false);
+                } catch (final Exception e) {
+                    getLogger().error("Write Rows to Table [{}] failed {}", 
tableIdentifier, flowFile, e);
+                    abortWriter(rowWriter, tableIdentifier);
+                    relationship.set(FAILURE);
+                }
+            } catch (final Exception e) {
+                getLogger().error("Processing Records for Table [{}] failed 
{}", tableIdentifier, flowFile, e);
+                relationship.set(FAILURE);

Review Comment:
   Shouldn't we abort the writer here too?
   
   ```suggestion
                   getLogger().error("Processing Records for Table [{}] failed 
{}", tableIdentifier, flowFile, e);
                   abortWriter(rowWriter, tableIdentifier);
                   relationship.set(FAILURE);
   ```



##########
nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-rest-catalog/src/main/java/org/apache/nifi/services/iceberg/catalog/RESTIcebergCatalog.java:
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.iceberg.catalog;
+
+import org.apache.iceberg.rest.RESTSessionCatalog;
+import org.apache.iceberg.rest.auth.AuthProperties;
+import org.apache.nifi.services.iceberg.IcebergCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.SessionCatalog;
+import org.apache.iceberg.metrics.LoggingMetricsReporter;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.AuthSession;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.nifi.annotation.behavior.SupportsSensitiveDynamicProperties;
+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.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.VerifiableControllerService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.iceberg.IcebergFileIOProvider;
+import org.apache.nifi.services.iceberg.ProviderContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import static 
org.apache.nifi.components.ConfigVerificationResult.Outcome.FAILED;
+import static 
org.apache.nifi.components.ConfigVerificationResult.Outcome.SUCCESSFUL;
+
+@SupportsSensitiveDynamicProperties
+@Tags({"iceberg", "catalog", "polaris"})
+@CapabilityDescription("Provides Apache Iceberg integration with REST Catalogs 
such as Apache Polaris")
+public class RESTIcebergCatalog extends AbstractControllerService implements 
IcebergCatalog, VerifiableControllerService {
+    static final PropertyDescriptor CATALOG_URI = new 
PropertyDescriptor.Builder()
+            .name("Catalog URI")
+            .description("Apache Iceberg Catalog REST URI")
+            .required(true)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FILE_IO_PROVIDER = new 
PropertyDescriptor.Builder()
+            .name("File IO Provider")
+            .description("Provider for Iceberg File Input and Output 
operations")
+            .required(true)
+            .identifiesControllerService(IcebergFileIOProvider.class)
+            .build();
+
+    static final PropertyDescriptor AUTHENTICATION_STRATEGY = new 
PropertyDescriptor.Builder()
+            .name("Authentication Strategy")
+            .description("Strategy for authenticating with the Apache Iceberg 
Catalog over HTTP")
+            .required(true)
+            .allowableValues(AuthenticationStrategy.class)
+            .defaultValue(AuthenticationStrategy.OAUTH2)
+            .build();
+
+    static final PropertyDescriptor BEARER_TOKEN = new 
PropertyDescriptor.Builder()
+            .name("Bearer Token")
+            .description("Bearer Token for authentication to Apache Iceberg 
Catalog")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(AUTHENTICATION_STRATEGY, AuthenticationStrategy.BEARER)
+            .build();
+
+    static final PropertyDescriptor AUTHORIZATION_SERVER_URI = new 
PropertyDescriptor.Builder()
+            .name("Authorization Server URI")
+            .description("Authorization Server URI supporting OAuth 2")
+            .required(true)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .dependsOn(AUTHENTICATION_STRATEGY, AuthenticationStrategy.OAUTH2)
+            .build();
+
+    static final PropertyDescriptor AUTHORIZATION_GRANT_TYPE = new 
PropertyDescriptor.Builder()
+            .name("Authorization Grant Type")
+            .description("OAuth 2.0 Authorization Grant Type for obtaining 
Access Tokens")
+            .required(true)
+            .allowableValues(AuthorizationGrantType.class)
+            .defaultValue(AuthorizationGrantType.CLIENT_CREDENTIALS)
+            .dependsOn(AUTHENTICATION_STRATEGY, AuthenticationStrategy.OAUTH2)
+            .build();
+
+    static final PropertyDescriptor CLIENT_ID = new 
PropertyDescriptor.Builder()
+            .name("Client ID")
+            .description("Client ID for OAuth 2 Client Credentials")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(AUTHORIZATION_GRANT_TYPE, 
AuthorizationGrantType.CLIENT_CREDENTIALS)
+            .build();
+
+    static final PropertyDescriptor CLIENT_SECRET = new 
PropertyDescriptor.Builder()
+            .name("Client Secret")
+            .description("Client Secret for OAuth 2 Client Credentials")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(AUTHORIZATION_GRANT_TYPE, 
AuthorizationGrantType.CLIENT_CREDENTIALS)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN_SCOPES = new 
PropertyDescriptor.Builder()
+            .name("Access Token Scopes")
+            .description("Comma-separated list of one or more OAuth 2 scopes 
requested for Access Tokens")
+            .required(true)
+            .defaultValue(OAuth2Properties.CATALOG_SCOPE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(AUTHENTICATION_STRATEGY, AuthenticationStrategy.OAUTH2)
+            .build();
+
+    static final PropertyDescriptor WAREHOUSE_LOCATION = new 
PropertyDescriptor.Builder()
+            .name("Warehouse Location")
+            .description("Apache Iceberg Catalog Warehouse location or 
identifier")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = 
List.of(
+            CATALOG_URI,
+            FILE_IO_PROVIDER,
+            AUTHENTICATION_STRATEGY,
+            BEARER_TOKEN,
+            AUTHORIZATION_SERVER_URI,
+            AUTHORIZATION_GRANT_TYPE,
+            CLIENT_ID,
+            CLIENT_SECRET,
+            ACCESS_TOKEN_SCOPES,
+            WAREHOUSE_LOCATION
+    );
+
+    private static final String CONFIGURATION_STEP = "Catalog Configuration";
+
+    private static final String INITIALIZED_STATUS = "Initialized";
+
+    private static final String CLIENT_CREDENTIALS_FORMAT = "%s:%s";
+
+    private static final String SPACE_SEPARATOR = " ";
+
+    private RESTSessionCatalog sessionCatalog;
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final 
String propertyName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyName)
+                .dynamic(true)
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                .build();
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        sessionCatalog = getInitializedCatalog(context);
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        try {
+            sessionCatalog.close();

Review Comment:
   Null check? I believe that if onEnabled fails and sessionCatalog is not set, 
we would still expect the user to disable the controller service and we would 
get a NPE, no?



##########
nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/PutIcebergRecord.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.processors.iceberg;
+
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.nifi.processors.iceberg.record.DelegatedRecord;
+import org.apache.nifi.services.iceberg.IcebergCatalog;
+import org.apache.nifi.services.iceberg.IcebergRowWriter;
+import org.apache.nifi.services.iceberg.IcebergWriter;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.time.Clock;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+@Tags({"iceberg", "analytics", "polaris", "s3"})
+@CapabilityDescription("Store records in Iceberg Table using configurable 
Catalog for managing namespaces and tables.")
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+public class PutIcebergRecord extends AbstractProcessor {
+
+    static final PropertyDescriptor ICEBERG_CATALOG = new 
PropertyDescriptor.Builder()
+            .name("Iceberg Catalog")
+            .description("Provider Service for Iceberg Catalog")
+            .required(true)
+            .identifiesControllerService(IcebergCatalog.class)
+            .build();
+
+    static final PropertyDescriptor ICEBERG_WRITER = new 
PropertyDescriptor.Builder()
+            .name("Iceberg Writer")
+            .description("Provider Service for Iceberg Row Writers responsible 
for producing formatted Iceberg Data Files")
+            .required(true)
+            .identifiesControllerService(IcebergWriter.class)
+            .build();
+
+    static final PropertyDescriptor RECORD_READER = new 
PropertyDescriptor.Builder()
+            .name("Record Reader")
+            .description("Record Reader for incoming FlowFiles")
+            .required(true)
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    static final PropertyDescriptor NAMESPACE = new 
PropertyDescriptor.Builder()
+            .name("Namespace")
+            .description("Iceberg Namespace containing Tables")
+            .required(true)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor TABLE_NAME = new 
PropertyDescriptor.Builder()
+            .name("Table Name")
+            .description("Iceberg Table Name")
+            .required(true)
+            
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    static final Relationship SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles transferred to Iceberg")
+            .build();
+
+    static final Relationship FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles not transferred to Iceberg")
+            .build();
+
+    static final String RECORDS_PROCESSED_COUNTER = "Records Processed";
+
+    static final String DATA_FILES_PROCESSED_COUNTER = "Data Files Processed";
+
+    private static final List<PropertyDescriptor> properties = List.of(
+            ICEBERG_CATALOG,
+            ICEBERG_WRITER,
+            RECORD_READER,
+            NAMESPACE,
+            TABLE_NAME
+    );
+
+    private static final Set<Relationship> relationships = Set.of(SUCCESS, 
FAILURE);
+
+    private static final long MAXIMUM_BYTES = 536870912;
+
+    private final Clock clock = Clock.systemDefaultZone();
+
+    private volatile Catalog catalog;
+
+    private volatile IcebergWriter icebergWriter;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final IcebergCatalog icebergCatalog = 
context.getProperty(ICEBERG_CATALOG).asControllerService(IcebergCatalog.class);
+        catalog = icebergCatalog.getCatalog();
+        icebergWriter = 
context.getProperty(ICEBERG_WRITER).asControllerService(IcebergWriter.class);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession 
session) throws ProcessException {
+        final TableIdentifierFlowFileFilter flowFileFilter = new 
TableIdentifierFlowFileFilter(context, MAXIMUM_BYTES);
+        final List<FlowFile> flowFiles = session.get(flowFileFilter);
+        if (flowFiles.isEmpty()) {
+            return;
+        }
+
+        final TableIdentifier tableIdentifier = 
flowFileFilter.getTableIdentifier();
+        final RecordReaderFactory recordReaderFactory = 
context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        processFlowFiles(session, flowFiles, tableIdentifier, 
recordReaderFactory);
+    }
+
+    private void processFlowFiles(final ProcessSession session, final 
List<FlowFile> flowFiles, final TableIdentifier tableIdentifier, final 
RecordReaderFactory recordReaderFactory) {
+        final long started = clock.millis();
+
+        final AtomicReference<Relationship> relationship = new 
AtomicReference<>(SUCCESS);
+
+        final Table table = getTable(tableIdentifier);
+        final Schema schema = table.schema();
+        final Types.StructType struct = schema.asStruct();
+        final IcebergRowWriter rowWriter = icebergWriter.getRowWriter(table);
+
+        for (final FlowFile flowFile : flowFiles) {
+            try (
+                    InputStream inputStream = session.read(flowFile);
+                    RecordReader recordReader = 
recordReaderFactory.createRecordReader(flowFile, inputStream, getLogger())
+            ) {
+                final AtomicLong recordsProcessed = new AtomicLong();
+                try {
+                    writeRecords(recordReader, rowWriter, struct, 
recordsProcessed);
+                    session.adjustCounter(RECORDS_PROCESSED_COUNTER, 
recordsProcessed.get(), false);
+                } catch (final Exception e) {
+                    getLogger().error("Write Rows to Table [{}] failed {}", 
tableIdentifier, flowFile, e);
+                    abortWriter(rowWriter, tableIdentifier);
+                    relationship.set(FAILURE);
+                }
+            } catch (final Exception e) {
+                getLogger().error("Processing Records for Table [{}] failed 
{}", tableIdentifier, flowFile, e);
+                relationship.set(FAILURE);
+            }
+        }
+
+        try {
+            final DataFile[] dataFiles = rowWriter.dataFiles();
+            appendDataFiles(table, dataFiles);
+            session.adjustCounter(DATA_FILES_PROCESSED_COUNTER, 
dataFiles.length, false);
+        } catch (final Exception e) {
+            getLogger().error("Appending Data Files to Table [{}] failed", 
tableIdentifier, e);
+            relationship.set(FAILURE);
+        }

Review Comment:
   Shouldn't we do the append only in case of success?
   
   ```suggestion
           if (SUCCESS.equals(relationship.get())) {
               try {
                   final DataFile[] dataFiles = rowWriter.dataFiles();
                   appendDataFiles(table, dataFiles);
                   session.adjustCounter(DATA_FILES_PROCESSED_COUNTER, 
dataFiles.length, false);
               } catch (final Exception e) {
                   getLogger().error("Appending Data Files to Table [{}] 
failed", tableIdentifier, e);
                   relationship.set(FAILURE);
                   abortWriter(rowWriter, tableIdentifier);
               }
           } else {
               abortWriter(rowWriter, tableIdentifier);
           }
   ```



##########
nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-rest-catalog/src/main/java/org/apache/nifi/services/iceberg/catalog/RESTIcebergCatalog.java:
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.iceberg.catalog;
+
+import org.apache.iceberg.rest.RESTSessionCatalog;
+import org.apache.iceberg.rest.auth.AuthProperties;
+import org.apache.nifi.services.iceberg.IcebergCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.SessionCatalog;
+import org.apache.iceberg.metrics.LoggingMetricsReporter;
+import org.apache.iceberg.rest.HTTPClient;
+import org.apache.iceberg.rest.RESTClient;
+import org.apache.iceberg.rest.auth.AuthSession;
+import org.apache.iceberg.rest.auth.OAuth2Properties;
+import org.apache.nifi.annotation.behavior.SupportsSensitiveDynamicProperties;
+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.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.VerifiableControllerService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.services.iceberg.IcebergFileIOProvider;
+import org.apache.nifi.services.iceberg.ProviderContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import static 
org.apache.nifi.components.ConfigVerificationResult.Outcome.FAILED;
+import static 
org.apache.nifi.components.ConfigVerificationResult.Outcome.SUCCESSFUL;
+
+@SupportsSensitiveDynamicProperties
+@Tags({"iceberg", "catalog", "polaris"})
+@CapabilityDescription("Provides Apache Iceberg integration with REST Catalogs 
such as Apache Polaris")
+public class RESTIcebergCatalog extends AbstractControllerService implements 
IcebergCatalog, VerifiableControllerService {
+    static final PropertyDescriptor CATALOG_URI = new 
PropertyDescriptor.Builder()
+            .name("Catalog URI")
+            .description("Apache Iceberg Catalog REST URI")
+            .required(true)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .build();
+
+    static final PropertyDescriptor FILE_IO_PROVIDER = new 
PropertyDescriptor.Builder()
+            .name("File IO Provider")
+            .description("Provider for Iceberg File Input and Output 
operations")
+            .required(true)
+            .identifiesControllerService(IcebergFileIOProvider.class)
+            .build();
+
+    static final PropertyDescriptor AUTHENTICATION_STRATEGY = new 
PropertyDescriptor.Builder()
+            .name("Authentication Strategy")
+            .description("Strategy for authenticating with the Apache Iceberg 
Catalog over HTTP")
+            .required(true)
+            .allowableValues(AuthenticationStrategy.class)
+            .defaultValue(AuthenticationStrategy.OAUTH2)
+            .build();
+
+    static final PropertyDescriptor BEARER_TOKEN = new 
PropertyDescriptor.Builder()
+            .name("Bearer Token")
+            .description("Bearer Token for authentication to Apache Iceberg 
Catalog")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(AUTHENTICATION_STRATEGY, AuthenticationStrategy.BEARER)
+            .build();
+
+    static final PropertyDescriptor AUTHORIZATION_SERVER_URI = new 
PropertyDescriptor.Builder()
+            .name("Authorization Server URI")
+            .description("Authorization Server URI supporting OAuth 2")
+            .required(true)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .dependsOn(AUTHENTICATION_STRATEGY, AuthenticationStrategy.OAUTH2)
+            .build();
+
+    static final PropertyDescriptor AUTHORIZATION_GRANT_TYPE = new 
PropertyDescriptor.Builder()
+            .name("Authorization Grant Type")
+            .description("OAuth 2.0 Authorization Grant Type for obtaining 
Access Tokens")
+            .required(true)
+            .allowableValues(AuthorizationGrantType.class)
+            .defaultValue(AuthorizationGrantType.CLIENT_CREDENTIALS)
+            .dependsOn(AUTHENTICATION_STRATEGY, AuthenticationStrategy.OAUTH2)
+            .build();
+
+    static final PropertyDescriptor CLIENT_ID = new 
PropertyDescriptor.Builder()
+            .name("Client ID")
+            .description("Client ID for OAuth 2 Client Credentials")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(AUTHORIZATION_GRANT_TYPE, 
AuthorizationGrantType.CLIENT_CREDENTIALS)
+            .build();
+
+    static final PropertyDescriptor CLIENT_SECRET = new 
PropertyDescriptor.Builder()
+            .name("Client Secret")
+            .description("Client Secret for OAuth 2 Client Credentials")
+            .required(true)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(AUTHORIZATION_GRANT_TYPE, 
AuthorizationGrantType.CLIENT_CREDENTIALS)
+            .build();
+
+    static final PropertyDescriptor ACCESS_TOKEN_SCOPES = new 
PropertyDescriptor.Builder()
+            .name("Access Token Scopes")
+            .description("Comma-separated list of one or more OAuth 2 scopes 
requested for Access Tokens")
+            .required(true)
+            .defaultValue(OAuth2Properties.CATALOG_SCOPE)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .dependsOn(AUTHENTICATION_STRATEGY, AuthenticationStrategy.OAUTH2)
+            .build();
+
+    static final PropertyDescriptor WAREHOUSE_LOCATION = new 
PropertyDescriptor.Builder()
+            .name("Warehouse Location")
+            .description("Apache Iceberg Catalog Warehouse location or 
identifier")
+            .required(false)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = 
List.of(
+            CATALOG_URI,
+            FILE_IO_PROVIDER,
+            AUTHENTICATION_STRATEGY,
+            BEARER_TOKEN,
+            AUTHORIZATION_SERVER_URI,
+            AUTHORIZATION_GRANT_TYPE,
+            CLIENT_ID,
+            CLIENT_SECRET,
+            ACCESS_TOKEN_SCOPES,
+            WAREHOUSE_LOCATION
+    );
+
+    private static final String CONFIGURATION_STEP = "Catalog Configuration";
+
+    private static final String INITIALIZED_STATUS = "Initialized";
+
+    private static final String CLIENT_CREDENTIALS_FORMAT = "%s:%s";
+
+    private static final String SPACE_SEPARATOR = " ";
+
+    private RESTSessionCatalog sessionCatalog;
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final 
String propertyName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyName)
+                .dynamic(true)
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                .build();
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        sessionCatalog = getInitializedCatalog(context);
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        try {
+            sessionCatalog.close();
+        } catch (final IOException e) {
+            getLogger().warn("Close Catalog failed", e);
+        }
+    }
+
+    @Override
+    public List<ConfigVerificationResult> verify(final ConfigurationContext 
context, final ComponentLog componentLog, final Map<String, String> attributes) 
{
+        final List<ConfigVerificationResult> results = new ArrayList<>();
+
+        try {
+            final RESTSessionCatalog initializedCatalog = 
getInitializedCatalog(context);
+            final String name = initializedCatalog.name();
+            componentLog.info("REST Catalog Initialized [{}]", name);
+
+            results.add(new ConfigVerificationResult.Builder()
+                    .verificationStepName(CONFIGURATION_STEP)
+                    .outcome(SUCCESSFUL)
+                    .explanation(INITIALIZED_STATUS)
+                    .build()
+            );
+        } catch (final Exception e) {
+            componentLog.warn("Catalog Configuration failed", e);
+            final String explanation = getExplanation(e);
+            results.add(new ConfigVerificationResult.Builder()
+                    .verificationStepName(CONFIGURATION_STEP)
+                    .outcome(FAILED)
+                    .explanation(explanation)
+                    .build()
+            );
+        }
+
+        return results;
+    }
+
+    @Override
+    public Catalog getCatalog() {
+        final SessionCatalog.SessionContext sessionContext = 
SessionCatalog.SessionContext.createEmpty();
+        return sessionCatalog.asCatalog(sessionContext);
+    }
+
+    private RESTSessionCatalog getInitializedCatalog(final 
ConfigurationContext context) {
+        final Map<String, String> properties = new HashMap<>();
+
+        // Set default implementations for Metrics
+        properties.put(CatalogProperties.METRICS_REPORTER_IMPL, 
LoggingMetricsReporter.class.getName());
+
+        final Map<String, String> dynamicProperties = 
getDynamicProperties(context);
+        properties.putAll(dynamicProperties);
+
+        final String catalogUri = context.getProperty(CATALOG_URI).getValue();
+        properties.put(CatalogProperties.URI, catalogUri);
+
+        final PropertyValue warehouseLocationProperty = 
context.getProperty(WAREHOUSE_LOCATION);
+        if (warehouseLocationProperty.isSet()) {
+            final String warehouseLocation = 
warehouseLocationProperty.getValue();
+            properties.put(CatalogProperties.WAREHOUSE_LOCATION, 
warehouseLocation);
+        }
+
+        final AuthenticationStrategy authenticationStrategy = 
context.getProperty(AUTHENTICATION_STRATEGY).asAllowableValue(AuthenticationStrategy.class);
+        final Function<Map<String, String>, RESTClient> restClientBuilder;
+
+        if (AuthenticationStrategy.BEARER == authenticationStrategy) {
+            final String bearerToken = 
context.getProperty(BEARER_TOKEN).getValue();
+            properties.put(OAuth2Properties.TOKEN, bearerToken);
+            properties.put(AuthProperties.AUTH_TYPE, 
AuthProperties.AUTH_TYPE_OAUTH2);

Review Comment:
   Should be `AuthProperties.AUTH_TYPE_BEARER`, no?



-- 
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