Skip to content
Open
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
@@ -0,0 +1,24 @@
/*
* 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.parquet.io;

import org.apache.iceberg.data.Record;


public interface IcebergRecordConverter {
void convertRecord(Record record);
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.iceberg.data.Record;
import org.apache.iceberg.io.TaskWriter;
import org.apache.nifi.services.iceberg.IcebergRowWriter;
import org.apache.nifi.services.iceberg.parquet.io.recordconverters.IcebergParquetRecordWriterConverter;

import java.io.IOException;
import java.util.Objects;
Expand All @@ -29,13 +30,16 @@
*/
public class ParquetIcebergRowWriter implements IcebergRowWriter {
private final TaskWriter<Record> writer;
private final IcebergRecordConverter icebergParquetRecordWriterConverter =
new IcebergParquetRecordWriterConverter();

public ParquetIcebergRowWriter(final TaskWriter<Record> writer) {
this.writer = Objects.requireNonNull(writer, "Writer required");
}

@Override
public void write(final Record row) throws IOException {
this.icebergParquetRecordWriterConverter.convertRecord(row);
writer.write(row);
}

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
@@ -0,0 +1,87 @@
/*
* 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.parquet.io.recordconverters;

import org.apache.iceberg.data.Record;
import org.apache.nifi.services.iceberg.parquet.io.IcebergRecordConverter;

import java.util.List;
import java.util.Map;
import java.util.function.Function;

public class IcebergParquetRecordWriterConverter implements IcebergRecordConverter {

private final Map<Class<?>, Function<Object, Object>> typeHandlers = Map.of(
java.sql.Timestamp.class, (ts) -> ((java.sql.Timestamp) ts).toLocalDateTime(),
Record.class, (rec) -> {
convertRecord((Record) rec);
return rec;
},
List.class, (list) -> {
convertList((List<?>) list);
return list;
}
);

@Override
public void convertRecord(Record record) {
if (record == null) {
return;
}

for (int i = 0; i < record.struct().fields().size(); i++) {
Object value = record.get(i);
if (value == null) {
continue;
}

Function<Object, Object> handler = findHandler(value.getClass());
if (handler != null) {
record.set(i, handler.apply(value));
}
}
}

public void convertList(List<?> list) {
if (list == null) {
return;
}

for (Object element : list) {
if (element == null) {
continue;
}

Function<Object, Object> handler = findHandler(element.getClass());
if (handler != null) {
handler.apply(element);
}
}
}

private Function<Object, Object> findHandler(Class<?> clazz) {
if (typeHandlers.containsKey(clazz)) {
return typeHandlers.get(clazz);
}

return typeHandlers.entrySet().stream()
.filter(entry -> entry.getKey().isAssignableFrom(clazz))
.map(Map.Entry::getValue)
.findFirst()
.orElse(null);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* 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.parquet.io.recordconverters;

import org.apache.iceberg.Schema;
import org.apache.iceberg.data.GenericRecord;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.types.Types;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.junit.jupiter.MockitoExtension;

import java.sql.Timestamp;
import java.time.LocalDateTime;
import java.util.Collections;
import java.util.List;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNull;

@ExtendWith(MockitoExtension.class)
class IcebergParquetRecordWriterConverterTest {

private IcebergParquetRecordWriterConverter converter;

@BeforeEach
void setUp() {
converter = new IcebergParquetRecordWriterConverter();
}

@Test
void testConvertTimestampToLocalDateTime() {
final Schema schema = new Schema(
Types.NestedField.required(1, "ts_field", Types.TimestampType.withoutZone())
);
final Timestamp timestamp = Timestamp.valueOf("2026-06-06 06:06:06");
final Record record = GenericRecord.create(schema);
record.set(0, timestamp);

converter.convertRecord(record);

Object result = record.get(0);
assertInstanceOf(LocalDateTime.class, result);
assertEquals(timestamp.toLocalDateTime(), result);
}

@Test
void testConvertNestedRecord() {
final Schema innerSchema = new Schema(
Types.NestedField.required(1, "inner_ts", Types.TimestampType.withoutZone())
);
final Schema outerSchema = new Schema(
Types.NestedField.required(2, "nested_record", innerSchema.asStruct())
);

final Record innerRecord = GenericRecord.create(innerSchema);
final Timestamp timestamp = Timestamp.valueOf("2026-06-06 06:06:06");
innerRecord.set(0, timestamp);

final Record outerRecord = GenericRecord.create(outerSchema);
outerRecord.set(0, innerRecord);

converter.convertRecord(outerRecord);

Record resultInner = (Record) outerRecord.get(0);
assertInstanceOf(LocalDateTime.class, resultInner.get(0));
}

@Test
void testConvertListWithRecords() {
final Schema elementSchema = new Schema(
Types.NestedField.required(1, "ts", Types.TimestampType.withoutZone())
);
final Record recordInList = GenericRecord.create(elementSchema);
recordInList.set(0, Timestamp.valueOf("2026-06-06 06:06:06"));

final List<Record> list = Collections.singletonList(recordInList);

converter.convertList(list);

assertInstanceOf(LocalDateTime.class, recordInList.get(0));
}

@Test
void testNullValueHandling() {
final Schema schema = new Schema(
Types.NestedField.optional(1, "nullable_field", Types.TimestampType.withoutZone())
);
final Record record = GenericRecord.create(schema);
record.set(0, null);

converter.convertRecord(record);

assertNull(record.get(0));
}
}
Loading