Skip to content

Commit

Permalink
merge conflicts, tests, docs
Browse files Browse the repository at this point in the history
  • Loading branch information
tabmatfournier committed May 10, 2024
1 parent eeea1d9 commit 8c46cb8
Show file tree
Hide file tree
Showing 10 changed files with 144 additions and 74 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,13 @@ public Event(Schema avroSchema) {
this.avroSchema = avroSchema;
}

/**
* @deprecated
* <p>This class is required for a fallback decoder that can decode the legacy iceberg 1.4.x avro schemas in the case where
* the coordinator topic was not fully drained during the upgrade to 1.5.2. This entire module should be removed
* in later releases.</p>
*/
@Deprecated
public Event(String groupId, EventType type, Payload payload) {
this.id = UUID.randomUUID();
this.type = type;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,13 @@
import org.apache.avro.io.EncoderFactory;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;

/**
* Iceberg 1.5.0 introduced a breaking change to Avro serialization that the connector uses when encoding
* messages for the control topic, requiring a way to fall back to decoding 1.4.x series messages that may
* be left behind on a control topic when upgrading.
*
* This class should be removed in later releases.
*/
public class DeprecatedAvroEncoderUtil {

private DeprecatedAvroEncoderUtil() {}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,13 @@
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;

/**
* Iceberg 1.5.0 introduced a breaking change to Avro serialization that the connector uses when encoding
* messages for the control topic, requiring a way to fall back to decoding 1.4.x series messages that may
* be left behind on a control topic when upgrading.
*
* This class should be removed in later releases.
*/
public class DeprecatedAvroSchemaUtil {

private DeprecatedAvroSchemaUtil() {}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,13 @@
import org.apache.iceberg.common.DynClasses;
import org.apache.iceberg.data.avro.DecoderResolver;

/**
* Iceberg 1.5.0 introduced a breaking change to Avro serialization that the connector uses when encoding
* messages for the control topic, requiring a way to fall back to decoding 1.4.x series messages that may
* be left behind on a control topic when upgrading.
*
* This class should be removed in later releases.
*/
public class DeprecatedGenericAvroReader<T> implements DatumReader<T>, SupportsRowPosition {

private final Schema readSchema;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,13 @@
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;

/**
* Iceberg 1.5.0 introduced a breaking change to Avro serialization that the connector uses when encoding
* messages for the control topic, requiring a way to fall back to decoding 1.4.x series messages that may
* be left behind on a control topic when upgrading.
*
* This class should be removed in later releases.
*/
class DeprecatedTypeToSchema extends TypeUtil.SchemaVisitor<Schema> {
private static final Schema BOOLEAN_SCHEMA = Schema.create(Schema.Type.BOOLEAN);
private static final Schema INTEGER_SCHEMA = Schema.create(Schema.Type.INT);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,14 @@
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.function.Function;

import org.apache.iceberg.connect.events.AvroUtil;
import org.apache.iceberg.connect.events.Event;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.Pair;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
Expand All @@ -47,7 +50,6 @@ public abstract class Channel {
private static final Logger LOG = LoggerFactory.getLogger(Channel.class);

private final String controlTopic;
private final String controlGroupId;
private final String groupId;
private final Producer<String, byte[]> producer;
private final Consumer<String, byte[]> consumer;
Expand All @@ -63,17 +65,16 @@ public Channel(
IcebergSinkConfig config,
KafkaClientFactory clientFactory) {
this.controlTopic = config.controlTopic();
this.controlGroupId = config.controlGroupId();
this.groupId = config.controlGroupId();

String transactionalId = name + config.transactionalSuffix();
this.producer = clientFactory.createProducer(transactionalId);
Pair<UUID, Producer<String, byte[]>> pair = clientFactory.createProducer(transactionalId);
this.producer = pair.second();
this.consumer = clientFactory.createConsumer(consumerGroupId);
consumer.subscribe(ImmutableList.of(controlTopic));
this.admin = clientFactory.createAdmin();

this.producerId = UUID.randomUUID().toString();
this.eventDecoder = new EventDecoder(config.catalogName());
this.producerId = pair.first().toString();
this.eventDecoder = new EventDecoder(config.catalogName());
}

protected void send(Event event) {
Expand Down Expand Up @@ -129,13 +130,13 @@ record -> {

Event event = eventDecoder.decode(record.value());
if (event != null) {
if (event.groupId().equals(groupId)) {
LOG.debug("Received event of type: {}", event.type().name());
if (receiveFn.apply(new Envelope(event, record.partition(), record.offset()))) {
LOG.debug("Handled event of type: {}", event.type().name());
if (event.groupId().equals(groupId)) {
LOG.debug("Received event of type: {}", event.type().name());
if (receiveFn.apply(new Envelope(event, record.partition(), record.offset()))) {
LOG.debug("Handled event of type: {}", event.type().name());
}
}
}
}
});
records = consumer.poll(pollDuration);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,6 @@

import io.tabular.iceberg.connect.IcebergSinkConfig;
import io.tabular.iceberg.connect.data.Offset;
import io.tabular.iceberg.connect.events.CommitReadyPayload;
import io.tabular.iceberg.connect.events.CommitRequestPayload;
import io.tabular.iceberg.connect.events.CommitResponsePayload;
import io.tabular.iceberg.connect.events.Event;
import io.tabular.iceberg.connect.events.EventType;
import io.tabular.iceberg.connect.events.TableName;
import io.tabular.iceberg.connect.events.TopicPartitionOffset;
import java.io.IOException;
import java.time.Duration;
import java.util.List;
Expand All @@ -38,6 +31,13 @@
import java.util.UUID;
import java.util.concurrent.ExecutionException;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.connect.events.DataComplete;
import org.apache.iceberg.connect.events.DataWritten;
import org.apache.iceberg.connect.events.Event;
import org.apache.iceberg.connect.events.PayloadType;
import org.apache.iceberg.connect.events.StartCommit;
import org.apache.iceberg.connect.events.TableReference;
import org.apache.iceberg.connect.events.TopicPartitionOffset;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
Expand Down Expand Up @@ -129,8 +129,8 @@ private void throwExceptionIfCoordinatorIsTerminated() {
}

private boolean receive(Envelope envelope, CommittableSupplier committableSupplier) {
if (envelope.event().type() == EventType.COMMIT_REQUEST) {
UUID commitId = ((CommitRequestPayload) envelope.event().payload()).commitId();
if (envelope.event().type() == PayloadType.START_COMMIT) {
UUID commitId = ((StartCommit) envelope.event().payload()).commitId();
sendCommitResponse(commitId, committableSupplier);
return true;
}
Expand All @@ -149,11 +149,10 @@ private void sendCommitResponse(UUID commitId, CommittableSupplier committableSu
Event commitResponse =
new Event(
config.controlGroupId(),
EventType.COMMIT_RESPONSE,
new CommitResponsePayload(
new DataWritten(
writerResult.partitionStruct(),
commitId,
TableName.of(writerResult.tableIdentifier()),
TableReference.of(config.catalogName(), writerResult.tableIdentifier()),
writerResult.dataFiles(),
writerResult.deleteFiles()));

Expand All @@ -180,8 +179,7 @@ private void sendCommitResponse(UUID commitId, CommittableSupplier committableSu
Event commitReady =
new Event(
config.controlGroupId(),
EventType.COMMIT_READY,
new CommitReadyPayload(commitId, assignments));
new DataComplete(commitId, assignments));
events.add(commitReady);

Map<TopicPartition, Offset> offsets = committable.offsetsByTopicPartition();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,13 @@
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;

/**
* Iceberg 1.5.0 introduced a breaking change to Avro serialization that the connector uses when encoding
* messages for the control topic, requiring a way to fall back to decoding 1.4.x series messages that may
* be left behind on a control topic when upgrading.
*
* This class should be removed in later revisions.
*/
public class EventDecoder {

private final String catalogName;
Expand All @@ -55,6 +62,12 @@ public EventDecoder(String catalogName) {
this.catalogName = catalogName;
}

/**
* @deprecated
* <p>This provides a fallback decoder that can decode the legacy iceberg 1.4.x avro schemas in the case where
* the coordinator topic was not fully drained during the upgrade to 1.5.2</p>
*/
@Deprecated
public Event decode(byte[] value) {
try {
return AvroUtil.decode(value);
Expand Down
Loading

0 comments on commit 8c46cb8

Please sign in to comment.