Skip to content

Commit

Permalink
Kafka Connect: Record converters and delta writers
Browse files Browse the repository at this point in the history
  • Loading branch information
bryanck committed Feb 4, 2024
1 parent fb02bd2 commit fefe257
Show file tree
Hide file tree
Showing 16 changed files with 2,171 additions and 45 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,9 @@
*/
package org.apache.iceberg.connect.events;

import static java.util.stream.Collectors.toList;

import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.util.Utf8;
Expand Down Expand Up @@ -96,7 +95,9 @@ public void put(int i, Object v) {
return;
case NAMESPACE:
this.namespace =
v == null ? null : ((List<Utf8>) v).stream().map(Utf8::toString).collect(toList());
v == null
? null
: ((List<Utf8>) v).stream().map(Utf8::toString).collect(Collectors.toList());
return;
case NAME:
this.name = v == null ? null : v.toString();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,6 @@
*/
package org.apache.iceberg.connect;

import static java.util.stream.Collectors.toList;

import java.io.InputStream;
import java.nio.file.Files;
import java.nio.file.Paths;
Expand All @@ -28,6 +26,7 @@
import java.util.Map;
import java.util.Properties;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.iceberg.IcebergBuild;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
Expand Down Expand Up @@ -72,7 +71,9 @@ public class IcebergSinkConfig extends AbstractConfig {
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";
// FIXME: add config for CDC and upsert mode
private static final String TABLES_CDC_FIELD_PROP = "iceberg.tables.cdc-field";
private static final String TABLES_UPSERT_MODE_ENABLED_PROP =
"iceberg.tables.upsert-mode-enabled";
private static final String TABLES_AUTO_CREATE_ENABLED_PROP =
"iceberg.tables.auto-create-enabled";
private static final String TABLES_EVOLVE_SCHEMA_ENABLED_PROP =
Expand Down Expand Up @@ -151,6 +152,18 @@ private static ConfigDef newConfigDef() {
null,
Importance.MEDIUM,
"Default partition spec to use when creating tables, comma-separated");
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_UPSERT_MODE_ENABLED_PROP,
ConfigDef.Type.BOOLEAN,
false,
Importance.MEDIUM,
"Set to true to treat all appends as upserts, false otherwise");
configDef.define(
TABLES_AUTO_CREATE_ENABLED_PROP,
ConfigDef.Type.BOOLEAN,
Expand Down Expand Up @@ -365,7 +378,7 @@ static List<String> stringToList(String value, String regex) {
return ImmutableList.of();
}

return Arrays.stream(value.split(regex)).map(String::trim).collect(toList());
return Arrays.stream(value.split(regex)).map(String::trim).collect(Collectors.toList());
}

public String controlTopic() {
Expand Down Expand Up @@ -409,6 +422,14 @@ public String hadoopConfDir() {
return getString(HADDOP_CONF_DIR_PROP);
}

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

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

public boolean autoCreateEnabled() {
return getBoolean(TABLES_AUTO_CREATE_ENABLED_PROP);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,10 @@
*/
package org.apache.iceberg.connect;

import static java.util.stream.Collectors.toList;

import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.kafka.common.config.ConfigDef;
Expand Down Expand Up @@ -60,7 +59,7 @@ public List<Map<String, String>> taskConfigs(int maxTasks) {
map.put(IcebergSinkConfig.INTERNAL_TRANSACTIONAL_SUFFIX_PROP, txnSuffix + i);
return map;
})
.collect(toList());
.collect(Collectors.toList());
}

@Override
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> {

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
Expand Up @@ -38,8 +38,7 @@ public class IcebergWriter implements RecordWriter {
private final IcebergSinkConfig config;
private final List<WriterResult> writerResults;

// FIXME: update this when the record converter is added
// private RecordConverter recordConverter;
private RecordConverter recordConverter;
private TaskWriter<Record> writer;

public IcebergWriter(Table table, String tableName, IcebergSinkConfig config) {
Expand All @@ -52,20 +51,23 @@ public IcebergWriter(Table table, String tableName, IcebergSinkConfig config) {

private void initNewWriter() {
this.writer = Utilities.createTableWriter(table, tableName, config);
// FIXME: update this when the record converter is added
// this.recordConverter = new RecordConverter(table, config);
this.recordConverter = new RecordConverter(table, config);
}

@Override
public void write(SinkRecord record) {
try {
// TODO: config to handle tombstones instead of always ignoring?
// TODO: config to handle tombstones instead of always ignoring?
if (record.value() != null) {
Record row = convertToRow(record);

// FIXME: add CDC operation support

writer.write(row);
String cdcField = config.tablesCdcField();
if (cdcField == null) {
writer.write(row);
} else {
Operation op = extractCdcOperation(record.value(), cdcField);
writer.write(new RecordWrapper(row, op));
}
}
} catch (Exception e) {
throw new DataException(
Expand All @@ -77,8 +79,49 @@ public void write(SinkRecord record) {
}

private Record convertToRow(SinkRecord record) {
// FIXME: update this when the record converter is added
return null;
if (!config.evolveSchemaEnabled()) {
return recordConverter.convert(record.value());
}

SchemaUpdate.Consumer updates = new SchemaUpdate.Consumer();
Record row = recordConverter.convert(record.value(), updates);

if (!updates.empty()) {
// complete the current file
flush();
// apply the schema updates, this will refresh the table
SchemaUtils.applySchemaUpdates(table, updates);
// initialize a new writer with the new schema
initNewWriter();
// convert the row again, this time using the new table schema
row = recordConverter.convert(record.value(), null);
}

return row;
}

private Operation extractCdcOperation(Object recordValue, String cdcField) {
Object opValue = Utilities.extractFromRecordValue(recordValue, cdcField);

if (opValue == null) {
return Operation.INSERT;
}

String opStr = opValue.toString().trim().toUpperCase();
if (opStr.isEmpty()) {
return Operation.INSERT;
}

// TODO: define value mapping in config?

switch (opStr.charAt(0)) {
case 'U':
return Operation.UPDATE;
case 'D':
return Operation.DELETE;
default:
return Operation.INSERT;
}
}

private void flush() {
Expand Down
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
}
Loading

0 comments on commit fefe257

Please sign in to comment.