Skip to content

Flink: Dynamic Iceberg Sink Contribution #12424

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 6 commits into
base: main
Choose a base branch
from
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 @@ -26,7 +26,7 @@
public class NamedReference<T> implements UnboundTerm<T>, Reference<T> {
private final String name;

NamedReference(String name) {
public NamedReference(String name) {
Preconditions.checkNotNull(name, "Name cannot be null");
this.name = name;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ public class UnboundTransform<S, T> implements UnboundTerm<T>, Term {
private final NamedReference<S> ref;
private final Transform<S, T> transform;

UnboundTransform(NamedReference<S> ref, Transform<S, T> transform) {
public UnboundTransform(NamedReference<S> ref, Transform<S, T> transform) {
this.ref = ref;
this.transform = transform;
}
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/java/org/apache/iceberg/SchemaUpdate.java
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@
import org.slf4j.LoggerFactory;

/** Schema evolution API implementation. */
class SchemaUpdate implements UpdateSchema {
public class SchemaUpdate implements UpdateSchema {
private static final Logger LOG = LoggerFactory.getLogger(SchemaUpdate.class);
private static final int TABLE_ROOT_ID = -1;

Expand All @@ -71,7 +71,7 @@ class SchemaUpdate implements UpdateSchema {
}

/** For testing only. */
SchemaUpdate(Schema schema, int lastColumnId) {
public SchemaUpdate(Schema schema, int lastColumnId) {
this(null, null, schema, lastColumnId);
}

Expand Down
3 changes: 3 additions & 0 deletions flink/v1.20/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,9 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") {

implementation libs.datasketches

// for caching in DynamicSink
implementation libs.caffeine

testImplementation libs.flink120.connector.test.utils
testImplementation libs.flink120.core
testImplementation libs.flink120.runtime
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
/*
* 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.iceberg.flink.sink.dynamic;

import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.flink.api.common.serialization.SerializerConfig;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputDeserializer;
import org.apache.flink.core.memory.DataOutputSerializer;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.data.RandomGenericData;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.flink.CatalogLoader;
import org.apache.iceberg.flink.RowDataConverter;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.types.Types;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Threads;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
import org.openjdk.jmh.runner.Runner;
import org.openjdk.jmh.runner.RunnerException;
import org.openjdk.jmh.runner.options.Options;
import org.openjdk.jmh.runner.options.OptionsBuilder;

@Fork(1)
@State(Scope.Benchmark)
@Warmup(iterations = 3)
@Measurement(iterations = 5)
@BenchmarkMode(Mode.SingleShotTime)
public class DynamicRecordSerializerDeserializerBenchmark {
private static final int SAMPLE_SIZE = 100_000;
private static final Schema SCHEMA =
new Schema(
Types.NestedField.required(1, "id", Types.IntegerType.get()),
Types.NestedField.required(2, "name2", Types.StringType.get()),
Types.NestedField.required(3, "name3", Types.StringType.get()),
Types.NestedField.required(4, "name4", Types.StringType.get()),
Types.NestedField.required(5, "name5", Types.StringType.get()),
Types.NestedField.required(6, "name6", Types.StringType.get()),
Types.NestedField.required(7, "name7", Types.StringType.get()),
Types.NestedField.required(8, "name8", Types.StringType.get()),
Types.NestedField.required(9, "name9", Types.StringType.get()));

private List<DynamicRecordInternal> rows = Lists.newArrayListWithExpectedSize(SAMPLE_SIZE);
private DynamicRecordInternalType type;

public static void main(String[] args) throws RunnerException {
Options options =
new OptionsBuilder()
.include(DynamicRecordSerializerDeserializerBenchmark.class.getSimpleName())
.build();
new Runner(options).run();
}

@Setup
public void setupBenchmark() throws IOException {
List<Record> records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L);
this.rows =
records.stream()
.map(
r ->
new DynamicRecordInternal(
"t",
"main",
SCHEMA,
PartitionSpec.unpartitioned(),
1,
RowDataConverter.convert(SCHEMA, r),
false,
List.of()))
.collect(Collectors.toList());

File warehouse = Files.createTempFile("perf-bench", null).toFile();
CatalogLoader catalogLoader =
CatalogLoader.hadoop(
"hadoop",
new Configuration(),
ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse.getPath()));
this.type = new DynamicRecordInternalType(catalogLoader, true, 100);
}

@Benchmark
@Threads(1)
public void testSerialize(Blackhole blackhole) throws IOException {
TypeSerializer<DynamicRecordInternal> serializer =
type.createSerializer((SerializerConfig) null);
DataOutputSerializer outputView = new DataOutputSerializer(1024);
for (int i = 0; i < SAMPLE_SIZE; ++i) {
serializer.serialize(rows.get(i), outputView);
}
}

@Benchmark
@Threads(1)
public void testSerializeAndDeserialize(Blackhole blackhole) throws IOException {
TypeSerializer<DynamicRecordInternal> serializer =
type.createSerializer((SerializerConfig) null);

DataOutputSerializer outputView = new DataOutputSerializer(1024);
for (int i = 0; i < SAMPLE_SIZE; ++i) {
serializer.serialize(rows.get(i), outputView);
serializer.deserialize(new DataInputDeserializer(outputView.getSharedBuffer()));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.flink.util.TimeUtils;
import org.apache.iceberg.Table;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;

class FlinkConfParser {
Expand All @@ -41,6 +42,12 @@ class FlinkConfParser {
this.readableConfig = readableConfig;
}

FlinkConfParser(Map<String, String> options, ReadableConfig readableConfig) {
this.tableProperties = ImmutableMap.of();
this.options = options;
this.readableConfig = readableConfig;
}

public BooleanConfParser booleanConf() {
return new BooleanConfParser();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,10 @@ public FlinkWriteConf(
this.confParser = new FlinkConfParser(table, writeOptions, readableConfig);
}

public FlinkWriteConf(Map<String, String> writeOptions, ReadableConfig readableConfig) {
this.confParser = new FlinkConfParser(writeOptions, readableConfig);
}

public boolean overwriteMode() {
return confParser
.booleanConf()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.transforms.PartitionSpecVisitor;

final class BucketPartitionerUtil {
public final class BucketPartitionerUtil {
static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE =
"Invalid number of buckets: %s (must be 1)";

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,14 @@
package org.apache.iceberg.flink.sink;

import java.util.Arrays;
import java.util.List;
import java.util.NavigableMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.iceberg.io.WriteResult;
import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
import org.apache.iceberg.util.ScanTaskUtil;

class CommitSummary {
public class CommitSummary {

private final AtomicLong dataFilesCount = new AtomicLong();
private final AtomicLong dataFilesRecordCount = new AtomicLong();
Expand All @@ -34,30 +35,35 @@ class CommitSummary {
private final AtomicLong deleteFilesRecordCount = new AtomicLong();
private final AtomicLong deleteFilesByteCount = new AtomicLong();

CommitSummary(NavigableMap<Long, WriteResult> pendingResults) {
pendingResults
.values()
public CommitSummary() {}

public CommitSummary(NavigableMap<Long, WriteResult> pendingResults) {
pendingResults.values().forEach(this::addWriteResult);
}

public void addAll(NavigableMap<Long, List<WriteResult>> pendingResults) {
pendingResults.values().forEach(writeResults -> writeResults.forEach(this::addWriteResult));
}

private void addWriteResult(WriteResult writeResult) {
dataFilesCount.addAndGet(writeResult.dataFiles().length);
Arrays.stream(writeResult.dataFiles())
.forEach(
dataFile -> {
dataFilesRecordCount.addAndGet(dataFile.recordCount());
dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes());
});
deleteFilesCount.addAndGet(writeResult.deleteFiles().length);
Arrays.stream(writeResult.deleteFiles())
.forEach(
writeResult -> {
dataFilesCount.addAndGet(writeResult.dataFiles().length);
Arrays.stream(writeResult.dataFiles())
.forEach(
dataFile -> {
dataFilesRecordCount.addAndGet(dataFile.recordCount());
dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes());
});
deleteFilesCount.addAndGet(writeResult.deleteFiles().length);
Arrays.stream(writeResult.deleteFiles())
.forEach(
deleteFile -> {
deleteFilesRecordCount.addAndGet(deleteFile.recordCount());
long deleteBytes = ScanTaskUtil.contentSizeInBytes(deleteFile);
deleteFilesByteCount.addAndGet(deleteBytes);
});
deleteFile -> {
deleteFilesRecordCount.addAndGet(deleteFile.recordCount());
long deleteBytes = ScanTaskUtil.contentSizeInBytes(deleteFile);
deleteFilesByteCount.addAndGet(deleteBytes);
});
}

long dataFilesCount() {
public long dataFilesCount() {
return dataFilesCount.get();
}

Expand All @@ -69,7 +75,7 @@ long dataFilesByteCount() {
return dataFilesByteCount.get();
}

long deleteFilesCount() {
public long deleteFilesCount() {
return deleteFilesCount.get();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;

class DeltaManifests {
public class DeltaManifests {

private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0];

Expand Down Expand Up @@ -56,7 +56,7 @@ CharSequence[] referencedDataFiles() {
return referencedDataFiles;
}

List<ManifestFile> manifests() {
public List<ManifestFile> manifests() {
List<ManifestFile> manifests = Lists.newArrayListWithCapacity(2);
if (dataManifest != null) {
manifests.add(dataManifest);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,12 +28,12 @@
import org.apache.iceberg.ManifestFiles;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;

class DeltaManifestsSerializer implements SimpleVersionedSerializer<DeltaManifests> {
public class DeltaManifestsSerializer implements SimpleVersionedSerializer<DeltaManifests> {
private static final int VERSION_1 = 1;
private static final int VERSION_2 = 2;
private static final byte[] EMPTY_BINARY = new byte[0];

static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer();
public static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer();

@Override
public int getVersion() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
package org.apache.iceberg.flink.sink;

import java.util.List;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.Schema;
Expand All @@ -30,10 +29,10 @@
import org.apache.iceberg.util.StructProjection;

/**
* Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record
* will be emitted to same writer in order.
* Create a {@link NonThrowingKeySelector} to shuffle by equality fields, to ensure same equality
* fields record will be emitted to same writer in order.
*/
class EqualityFieldKeySelector implements KeySelector<RowData, Integer> {
public class EqualityFieldKeySelector implements NonThrowingKeySelector<RowData, Integer> {

private final Schema schema;
private final RowType flinkSchema;
Expand All @@ -43,7 +42,8 @@ class EqualityFieldKeySelector implements KeySelector<RowData, Integer> {
private transient StructProjection structProjection;
private transient StructLikeWrapper structLikeWrapper;

EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List<Integer> equalityFieldIds) {
public EqualityFieldKeySelector(
Schema schema, RowType flinkSchema, List<Integer> equalityFieldIds) {
this.schema = schema;
this.flinkSchema = flinkSchema;
this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds));
Expand Down
Loading