Skip to content
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

Kafka Connect: Add delta writer support #12070

Draft
wants to merge 12 commits into
base: main
Choose a base branch
from
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,8 @@ public class IcebergSinkConfig extends AbstractConfig {
private static final String TABLES_PROP = "iceberg.tables";
private static final String TABLES_DYNAMIC_PROP = "iceberg.tables.dynamic-enabled";
private static final String TABLES_ROUTE_FIELD_PROP = "iceberg.tables.route-field";
private static final String TABLES_CDC_FIELD_PROP = "iceberg.tables.cdcField";
private static final String TABLES_UPSERT_MODE_ENABLED_PROP = "iceberg.tables.upsertModeEnabled";
Comment on lines +71 to +72
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i believe naming should change to dash-case-naming ? example cdc-field?

private static final String TABLES_DEFAULT_COMMIT_BRANCH = "iceberg.tables.default-commit-branch";
private static final String TABLES_DEFAULT_ID_COLUMNS = "iceberg.tables.default-id-columns";
private static final String TABLES_DEFAULT_PARTITION_BY = "iceberg.tables.default-partition-by";
Expand Down Expand Up @@ -127,6 +129,12 @@ private static ConfigDef newConfigDef() {
null,
Importance.MEDIUM,
"Source record field for routing records to tables");
configDef.define(
TABLES_CDC_FIELD_PROP,
ConfigDef.Type.STRING,
null,
Importance.MEDIUM,
"Source record field that identifies the type of operation (insert, update, or delete)");
configDef.define(
TABLES_DEFAULT_COMMIT_BRANCH,
ConfigDef.Type.STRING,
Expand Down Expand Up @@ -157,6 +165,12 @@ private static ConfigDef newConfigDef() {
false,
Importance.MEDIUM,
"Set to true to set columns as optional during table create and evolution, false to respect schema");
configDef.define(
TABLES_UPSERT_MODE_ENABLED_PROP,
ConfigDef.Type.BOOLEAN,
false,
Importance.MEDIUM,
"Set to true to treat all appends as upserts, false otherwise");
configDef.define(
TABLES_SCHEMA_CASE_INSENSITIVE_PROP,
ConfigDef.Type.BOOLEAN,
Expand Down Expand Up @@ -353,6 +367,10 @@ public TableSinkConfig tableConfig(String tableName) {
});
}

public String tablesCdcField() {
return getString(TABLES_CDC_FIELD_PROP);
}

@VisibleForTesting
static List<String> stringToList(String value, String regex) {
if (value == null || value.isEmpty()) {
Expand Down Expand Up @@ -389,6 +407,10 @@ public int commitTimeoutMs() {
return getInt(COMMIT_TIMEOUT_MS_PROP);
}

public boolean isUpsertMode() {
return getBoolean(TABLES_UPSERT_MODE_ENABLED_PROP);
}

public int commitThreads() {
return getInt(COMMIT_THREADS_PROP);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
/*
* 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.connect.data;

import java.io.IOException;
import java.util.Set;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.PartitionKey;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.data.InternalRecordWrapper;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.io.BaseTaskWriter;
import org.apache.iceberg.io.FileAppenderFactory;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.OutputFileFactory;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.types.TypeUtil;

abstract class BaseDeltaTaskWriter extends BaseTaskWriter<Record> {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@bryanck should we add this and other DeltaWriter* classes to core, next to AppendWriter classes? for downstream projects to reuse it?

using RecordWrapper + Operation it becomes generic enough to be in core? what do you think?

example refactored version:
BaseDeltaTaskWriter

Operation


private final Schema schema;
private final Schema deleteSchema;
private final InternalRecordWrapper wrapper;
private final InternalRecordWrapper keyWrapper;
private final RecordProjection keyProjection;
private final boolean upsertMode;

BaseDeltaTaskWriter(
PartitionSpec spec,
FileFormat format,
FileAppenderFactory<Record> appenderFactory,
OutputFileFactory fileFactory,
FileIO io,
long targetFileSize,
Schema schema,
Set<Integer> identifierFieldIds,
boolean upsertMode) {
super(spec, format, appenderFactory, fileFactory, io, targetFileSize);
this.schema = schema;
this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(identifierFieldIds));
this.wrapper = new InternalRecordWrapper(schema.asStruct());
this.keyWrapper = new InternalRecordWrapper(deleteSchema.asStruct());
this.keyProjection = RecordProjection.create(schema, deleteSchema);
this.upsertMode = upsertMode;
}

abstract RowDataDeltaWriter route(Record row);

InternalRecordWrapper wrapper() {
return wrapper;
}

@Override
public void write(Record row) throws IOException {
Operation op =
row instanceof RecordWrapper
? ((RecordWrapper) row).op()
: upsertMode ? Operation.UPDATE : Operation.INSERT;
RowDataDeltaWriter writer = route(row);
if (op == Operation.UPDATE || op == Operation.DELETE) {
writer.deleteKey(keyProjection.wrap(row));
}
if (op == Operation.UPDATE || op == Operation.INSERT) {
writer.write(row);
}
}

class RowDataDeltaWriter extends BaseEqualityDeltaWriter {

RowDataDeltaWriter(PartitionKey partition) {
super(partition, schema, deleteSchema);
}

@Override
protected StructLike asStructLike(Record data) {
return wrapper.wrap(data);
}

@Override
protected StructLike asStructLikeKey(Record data) {
return keyWrapper.wrap(data);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.connect.data;

public enum Operation {
INSERT,
UPDATE,
DELETE
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
/*
* 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.connect.data;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Map;
import java.util.Set;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.PartitionKey;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.io.FileAppenderFactory;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.OutputFileFactory;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.Tasks;

public class PartitionedDeltaWriter extends BaseDeltaTaskWriter {
private final PartitionKey partitionKey;

private final Map<PartitionKey, RowDataDeltaWriter> writers = Maps.newHashMap();

PartitionedDeltaWriter(
PartitionSpec spec,
FileFormat format,
FileAppenderFactory<Record> appenderFactory,
OutputFileFactory fileFactory,
FileIO io,
long targetFileSize,
Schema schema,
Set<Integer> identifierFieldIds,
boolean upsertMode) {
super(
spec,
format,
appenderFactory,
fileFactory,
io,
targetFileSize,
schema,
identifierFieldIds,
upsertMode);
this.partitionKey = new PartitionKey(spec, schema);
}

@Override
RowDataDeltaWriter route(Record row) {
partitionKey.partition(wrapper().wrap(row));

RowDataDeltaWriter writer = writers.get(partitionKey);
if (writer == null) {
// NOTICE: we need to copy a new partition key here, in case of messing up the keys in
// writers.
PartitionKey copiedKey = partitionKey.copy();
writer = new RowDataDeltaWriter(copiedKey);
writers.put(copiedKey, writer);
}

return writer;
}

@Override
public void close() {
try {
Tasks.foreach(writers.values())
.throwFailureWhenFinished()
.noRetry()
.run(RowDataDeltaWriter::close, IOException.class);

writers.clear();
} catch (IOException e) {
throw new UncheckedIOException("Failed to close equality delta writer", e);
}
}
}
Loading