diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-aws/src/main/java/org/apache/nifi/services/iceberg/aws/S3IcebergFileIOProvider.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-aws/src/main/java/org/apache/nifi/services/iceberg/aws/S3IcebergFileIOProvider.java index 58feed85180f..faa275b56e0d 100644 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-aws/src/main/java/org/apache/nifi/services/iceberg/aws/S3IcebergFileIOProvider.java +++ b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-aws/src/main/java/org/apache/nifi/services/iceberg/aws/S3IcebergFileIOProvider.java @@ -32,6 +32,7 @@ import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.services.iceberg.IcebergFileIOProvider; import org.apache.nifi.services.iceberg.ProviderContext; +import software.amazon.awssdk.services.s3.model.StorageClass; import java.util.HashMap; import java.util.List; @@ -123,6 +124,16 @@ public class S3IcebergFileIOProvider extends AbstractControllerService implement .addValidator(StandardValidators.BOOLEAN_VALIDATOR) .build(); + static final PropertyDescriptor STORAGE_CLASS = new PropertyDescriptor.Builder() + .name("Storage Class") + .description(""" + Specifies the S3 storage class to use when writing objects. + Primarily intended for on-premises or S3-compatible object storage implementations.""") + .required(true) + .allowableValues(StorageClass.values()) + .defaultValue(String.valueOf(StorageClass.STANDARD)) + .build(); + private static final List PROPERTY_DESCRIPTORS = List.of( AUTHENTICATION_STRATEGY, ACCESS_KEY_ID, @@ -130,7 +141,8 @@ public class S3IcebergFileIOProvider extends AbstractControllerService implement SESSION_TOKEN, CLIENT_REGION, ENDPOINT_URL, - PATH_STYLE_ACCESS + PATH_STYLE_ACCESS, + STORAGE_CLASS ); private final Map standardProperties = new ConcurrentHashMap<>(); @@ -189,6 +201,8 @@ private Map getConfiguredProperties(final ConfigurationContext c final String pathStyleAccess = context.getProperty(PATH_STYLE_ACCESS).getValue(); contextProperties.put(S3FileIOProperties.PATH_STYLE_ACCESS, pathStyleAccess); + final String storageClass = context.getProperty(STORAGE_CLASS).getValue(); + contextProperties.put(S3FileIOProperties.WRITE_STORAGE_CLASS, storageClass); // HttpURLConnection Client Type avoids additional dependencies contextProperties.put(HttpClientProperties.CLIENT_TYPE, HttpClientProperties.CLIENT_TYPE_URLCONNECTION); diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-parquet-writer/pom.xml b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-parquet-writer/pom.xml index 37e3f4d26c31..2393c344297a 100644 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-parquet-writer/pom.xml +++ b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-parquet-writer/pom.xml @@ -50,6 +50,11 @@ org.apache.iceberg iceberg-parquet + + + org.apache.iceberg + iceberg-data + org.apache.parquet diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-parquet-writer/src/main/java/org/apache/nifi/services/iceberg/parquet/io/ParquetPartitionedWriter.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-parquet-writer/src/main/java/org/apache/nifi/services/iceberg/parquet/io/ParquetPartitionedWriter.java index e5bc9a186578..1c5f25830d3d 100644 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-parquet-writer/src/main/java/org/apache/nifi/services/iceberg/parquet/io/ParquetPartitionedWriter.java +++ b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-parquet-writer/src/main/java/org/apache/nifi/services/iceberg/parquet/io/ParquetPartitionedWriter.java @@ -20,6 +20,7 @@ import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.data.InternalRecordWrapper; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.io.FileIO; @@ -32,6 +33,7 @@ public class ParquetPartitionedWriter extends PartitionedFanoutWriter { private final PartitionKey partitionKey; + private final InternalRecordWrapper recordWrapper; public ParquetPartitionedWriter( final PartitionSpec spec, @@ -43,11 +45,12 @@ public ParquetPartitionedWriter( ) { super(spec, FileFormat.PARQUET, appenderFactory, fileFactory, io, targetFileSize); this.partitionKey = new PartitionKey(spec, schema); + this.recordWrapper = new InternalRecordWrapper(schema.asStruct()); } @Override protected PartitionKey partition(final Record record) { - partitionKey.partition(record); + partitionKey.partition(recordWrapper.wrap(record)); return partitionKey; } } diff --git a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/record/DelegatedRecord.java b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/record/DelegatedRecord.java index c9ab79c42873..81541e2bf960 100644 --- a/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/record/DelegatedRecord.java +++ b/nifi-extension-bundles/nifi-iceberg-bundle/nifi-iceberg-processors/src/main/java/org/apache/nifi/processors/iceberg/record/DelegatedRecord.java @@ -21,18 +21,26 @@ import org.apache.nifi.serialization.record.MapRecord; import org.apache.nifi.serialization.record.RecordField; +import java.sql.Timestamp; import java.util.Collections; import java.util.Map; import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; /** * Standard implementation of Iceberg Record wrapping NiFi Record */ public class DelegatedRecord implements Record { private final org.apache.nifi.serialization.record.Record record; - private final Types.StructType struct; - + private final Map, Function> typeHandlers = + new ConcurrentHashMap<>( + Map.of( + java.sql.Timestamp.class, + (timestamp) -> ((Timestamp) timestamp).toLocalDateTime() + ) + ); public DelegatedRecord( final org.apache.nifi.serialization.record.Record record, final Types.StructType struct @@ -54,7 +62,7 @@ public Types.StructType struct() { */ @Override public Object getField(final String fieldName) { - return record.getValue(fieldName); + return convertRecord(record.getValue(fieldName)); } /** @@ -77,7 +85,14 @@ public void setField(final String fieldName, final Object fieldValue) { @Override public Object get(final int position) { final RecordField recordField = record.getSchema().getField(position); - return record.getValue(recordField); + return convertRecord(record.getValue(recordField)); + } + + private Object convertRecord(Object recordValue) { + if (recordValue != null && typeHandlers.containsKey(recordValue.getClass())) { + return typeHandlers.get(recordValue.getClass()).apply(recordValue); + } + return record; } /**