Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -123,14 +124,25 @@ 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<PropertyDescriptor> PROPERTY_DESCRIPTORS = List.of(
AUTHENTICATION_STRATEGY,
ACCESS_KEY_ID,
SECRET_ACCESS_KEY,
SESSION_TOKEN,
CLIENT_REGION,
ENDPOINT_URL,
PATH_STYLE_ACCESS
PATH_STYLE_ACCESS,
STORAGE_CLASS
);

private final Map<String, String> standardProperties = new ConcurrentHashMap<>();
Expand Down Expand Up @@ -189,6 +201,8 @@ private Map<String, String> 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);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,11 @@
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-parquet</artifactId>
</dependency>
<!-- Iceberg Data dependencies -->
<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-data</artifactId>
</dependency>
<!-- Override Parquet version from iceberg-parquet -->
<dependency>
<groupId>org.apache.parquet</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -32,6 +33,7 @@
public class ParquetPartitionedWriter extends PartitionedFanoutWriter<Record> {

private final PartitionKey partitionKey;
private final InternalRecordWrapper recordWrapper;

public ParquetPartitionedWriter(
final PartitionSpec spec,
Expand All @@ -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;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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<Class<?>, Function<Object, Object>> 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
Expand All @@ -54,7 +62,7 @@ public Types.StructType struct() {
*/
@Override
public Object getField(final String fieldName) {
return record.getValue(fieldName);
return convertRecord(record.getValue(fieldName));
}

/**
Expand All @@ -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;
}

/**
Expand Down
Loading