From f63ca073bb6d162f3e5e6bec5aa5b6f52e4ae1a0 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Thu, 11 Apr 2024 16:50:51 -0600 Subject: [PATCH 01/13] upgrade-to-iceberg-1.5.0 --- kafka-connect-events/build.gradle | 28 --- .../connect/events/CommitCompletePayload.java | 97 ---------- .../connect/events/CommitReadyPayload.java | 99 ---------- .../connect/events/CommitRequestPayload.java | 79 -------- .../connect/events/CommitResponsePayload.java | 154 ---------------- .../connect/events/CommitTablePayload.java | 129 ------------- .../iceberg/connect/events/Element.java | 37 ---- .../tabular/iceberg/connect/events/Event.java | 174 ------------------ .../iceberg/connect/events/EventType.java | 37 ---- .../iceberg/connect/events/Payload.java | 21 --- .../iceberg/connect/events/TableName.java | 108 ----------- .../connect/events/TopicPartitionOffset.java | 129 ------------- .../events/EventSerializationTest.java | 132 ------------- .../connect/fixtures}/EventTestUtil.java | 0 14 files changed, 1224 deletions(-) delete mode 100644 kafka-connect-events/build.gradle delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java delete mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java delete mode 100644 kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java rename {kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events => kafka-connect/src/test/java/io/tabular/iceberg/connect/fixtures}/EventTestUtil.java (100%) diff --git a/kafka-connect-events/build.gradle b/kafka-connect-events/build.gradle deleted file mode 100644 index 9817c844..00000000 --- a/kafka-connect-events/build.gradle +++ /dev/null @@ -1,28 +0,0 @@ -plugins { - id "java-test-fixtures" -} - -dependencies { - implementation libs.iceberg.core - implementation libs.iceberg.common - implementation libs.iceberg.guava - implementation libs.avro - - testImplementation libs.junit.api - testRuntimeOnly libs.junit.engine - - testImplementation libs.mockito - testImplementation libs.assertj - - testFixturesImplementation libs.iceberg.common - testFixturesImplementation libs.iceberg.core - testFixturesImplementation libs.avro -} - -publishing { - publications { - mavenJava(MavenPublication) { - from components.java - } - } -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java deleted file mode 100644 index 53015a93..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import java.util.UUID; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; - -public class CommitCompletePayload implements Payload { - - private UUID commitId; - private Long vtts; - private final Schema avroSchema; - - private static final Schema AVRO_SCHEMA = - SchemaBuilder.builder() - .record(CommitCompletePayload.class.getName()) - .fields() - .name("commitId") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type(UUID_SCHEMA) - .noDefault() - .name("vtts") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .nullable() - .longType() - .noDefault() - .endRecord(); - - // Used by Avro reflection to instantiate this class when reading events - public CommitCompletePayload(Schema avroSchema) { - this.avroSchema = avroSchema; - } - - public CommitCompletePayload(UUID commitId, Long vtts) { - this.commitId = commitId; - this.vtts = vtts; - this.avroSchema = AVRO_SCHEMA; - } - - public UUID commitId() { - return commitId; - } - - public Long vtts() { - return vtts; - } - - @Override - public Schema getSchema() { - return avroSchema; - } - - @Override - @SuppressWarnings("unchecked") - public void put(int i, Object v) { - switch (i) { - case 0: - this.commitId = (UUID) v; - return; - case 1: - this.vtts = (Long) v; - return; - default: - // ignore the object, it must be from a newer version of the format - } - } - - @Override - public Object get(int i) { - switch (i) { - case 0: - return commitId; - case 1: - return vtts; - default: - throw new UnsupportedOperationException("Unknown field ordinal: " + i); - } - } -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java deleted file mode 100644 index fe766ac9..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import java.util.List; -import java.util.UUID; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; - -public class CommitReadyPayload implements Payload { - - private UUID commitId; - private List assignments; - private final Schema avroSchema; - - private static final Schema AVRO_SCHEMA = - SchemaBuilder.builder() - .record(CommitReadyPayload.class.getName()) - .fields() - .name("commitId") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type(UUID_SCHEMA) - .noDefault() - .name("assignments") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .nullable() - .array() - .items(TopicPartitionOffset.AVRO_SCHEMA) - .noDefault() - .endRecord(); - - // Used by Avro reflection to instantiate this class when reading events - public CommitReadyPayload(Schema avroSchema) { - this.avroSchema = avroSchema; - } - - public CommitReadyPayload(UUID commitId, List assignments) { - this.commitId = commitId; - this.assignments = assignments; - this.avroSchema = AVRO_SCHEMA; - } - - public UUID commitId() { - return commitId; - } - - public List assignments() { - return assignments; - } - - @Override - public Schema getSchema() { - return avroSchema; - } - - @Override - @SuppressWarnings("unchecked") - public void put(int i, Object v) { - switch (i) { - case 0: - this.commitId = (UUID) v; - return; - case 1: - this.assignments = (List) v; - return; - default: - // ignore the object, it must be from a newer version of the format - } - } - - @Override - public Object get(int i) { - switch (i) { - case 0: - return commitId; - case 1: - return assignments; - default: - throw new UnsupportedOperationException("Unknown field ordinal: " + i); - } - } -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java deleted file mode 100644 index bc0ced46..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import java.util.UUID; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; - -public class CommitRequestPayload implements Payload { - - private UUID commitId; - private final Schema avroSchema; - - public static final Schema AVRO_SCHEMA = - SchemaBuilder.builder() - .record(CommitRequestPayload.class.getName()) - .fields() - .name("commitId") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type(UUID_SCHEMA) - .noDefault() - .endRecord(); - - // Used by Avro reflection to instantiate this class when reading events - public CommitRequestPayload(Schema avroSchema) { - this.avroSchema = avroSchema; - } - - public CommitRequestPayload(UUID commitId) { - this.commitId = commitId; - this.avroSchema = AVRO_SCHEMA; - } - - public UUID commitId() { - return commitId; - } - - @Override - public Schema getSchema() { - return avroSchema; - } - - @Override - public void put(int i, Object v) { - switch (i) { - case 0: - this.commitId = (UUID) v; - return; - default: - // ignore the object, it must be from a newer version of the format - } - } - - @Override - public Object get(int i) { - switch (i) { - case 0: - return commitId; - default: - throw new UnsupportedOperationException("Unknown field ordinal: " + i); - } - } -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java deleted file mode 100644 index bef7a75b..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import java.util.List; -import java.util.Map; -import java.util.UUID; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types.StructType; - -public class CommitResponsePayload implements Payload { - - private UUID commitId; - private TableName tableName; - private List dataFiles; - private List deleteFiles; - private final Schema avroSchema; - - // Used by Avro reflection to instantiate this class when reading events - public CommitResponsePayload(Schema avroSchema) { - this.avroSchema = avroSchema; - } - - public CommitResponsePayload( - StructType partitionType, - UUID commitId, - TableName tableName, - List dataFiles, - List deleteFiles) { - this.commitId = commitId; - this.tableName = tableName; - this.dataFiles = dataFiles; - this.deleteFiles = deleteFiles; - - StructType dataFileStruct = DataFile.getType(partitionType); - - Map dataFileNames = Maps.newHashMap(); - dataFileNames.put(dataFileStruct, "org.apache.iceberg.GenericDataFile"); - dataFileNames.put(partitionType, "org.apache.iceberg.PartitionData"); - Schema dataFileSchema = AvroSchemaUtil.convert(dataFileStruct, dataFileNames); - - Map deleteFileNames = Maps.newHashMap(); - deleteFileNames.put(dataFileStruct, "org.apache.iceberg.GenericDeleteFile"); - deleteFileNames.put(partitionType, "org.apache.iceberg.PartitionData"); - Schema deleteFileSchema = AvroSchemaUtil.convert(dataFileStruct, deleteFileNames); - - this.avroSchema = - SchemaBuilder.builder() - .record(getClass().getName()) - .fields() - .name("commitId") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type(UUID_SCHEMA) - .noDefault() - .name("tableName") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type(TableName.AVRO_SCHEMA) - .noDefault() - .name("dataFiles") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .nullable() - .array() - .items(dataFileSchema) - .noDefault() - .name("deleteFiles") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .nullable() - .array() - .items(deleteFileSchema) - .noDefault() - .endRecord(); - } - - public UUID commitId() { - return commitId; - } - - public TableName tableName() { - return tableName; - } - - public List dataFiles() { - return dataFiles; - } - - public List deleteFiles() { - return deleteFiles; - } - - @Override - public Schema getSchema() { - return avroSchema; - } - - @Override - @SuppressWarnings("unchecked") - public void put(int i, Object v) { - switch (i) { - case 0: - this.commitId = (UUID) v; - return; - case 1: - this.tableName = (TableName) v; - return; - case 2: - this.dataFiles = (List) v; - return; - case 3: - this.deleteFiles = (List) v; - return; - default: - // ignore the object, it must be from a newer version of the format - } - } - - @Override - public Object get(int i) { - switch (i) { - case 0: - return commitId; - case 1: - return tableName; - case 2: - return dataFiles; - case 3: - return deleteFiles; - default: - throw new UnsupportedOperationException("Unknown field ordinal: " + i); - } - } -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java deleted file mode 100644 index 9e6d3257..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import java.util.UUID; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; - -public class CommitTablePayload implements Payload { - - private UUID commitId; - private TableName tableName; - private Long snapshotId; - private Long vtts; - private final Schema avroSchema; - - private static final Schema AVRO_SCHEMA = - SchemaBuilder.builder() - .record(CommitTablePayload.class.getName()) - .fields() - .name("commitId") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type(UUID_SCHEMA) - .noDefault() - .name("tableName") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type(TableName.AVRO_SCHEMA) - .noDefault() - .name("snapshotId") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .nullable() - .longType() - .noDefault() - .name("vtts") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .nullable() - .longType() - .noDefault() - .endRecord(); - - // Used by Avro reflection to instantiate this class when reading events - public CommitTablePayload(Schema avroSchema) { - this.avroSchema = avroSchema; - } - - public CommitTablePayload(UUID commitId, TableName tableName, Long snapshotId, Long vtts) { - this.commitId = commitId; - this.tableName = tableName; - this.snapshotId = snapshotId; - this.vtts = vtts; - this.avroSchema = AVRO_SCHEMA; - } - - public UUID commitId() { - return commitId; - } - - public TableName tableName() { - return tableName; - } - - public Long snapshotId() { - return snapshotId; - } - - public Long vtts() { - return vtts; - } - - @Override - public Schema getSchema() { - return avroSchema; - } - - @Override - @SuppressWarnings("unchecked") - public void put(int i, Object v) { - switch (i) { - case 0: - this.commitId = (UUID) v; - return; - case 1: - this.tableName = (TableName) v; - return; - case 2: - this.snapshotId = (Long) v; - return; - case 3: - this.vtts = (Long) v; - return; - default: - // ignore the object, it must be from a newer version of the format - } - } - - @Override - public Object get(int i) { - switch (i) { - case 0: - return commitId; - case 1: - return tableName; - case 2: - return snapshotId; - case 3: - return vtts; - default: - throw new UnsupportedOperationException("Unknown field ordinal: " + i); - } - } -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java deleted file mode 100644 index 79f72f02..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.specific.SpecificData.SchemaConstructable; -import org.apache.iceberg.avro.AvroSchemaUtil; - -public interface Element extends IndexedRecord, SchemaConstructable { - // this is required by Iceberg's Avro deserializer to check for special metadata - // fields, but we aren't using any - String DUMMY_FIELD_ID = "-1"; - - String FIELD_ID_PROP = AvroSchemaUtil.FIELD_ID_PROP; - - Schema UUID_SCHEMA = - LogicalTypes.uuid().addToSchema(SchemaBuilder.builder().fixed("uuid").size(16)); -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java deleted file mode 100644 index cfca858a..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java +++ /dev/null @@ -1,174 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Map; -import java.util.UUID; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.iceberg.avro.AvroEncoderUtil; -import org.apache.iceberg.common.DynFields; -import org.apache.iceberg.data.avro.DecoderResolver; - -public class Event implements Element { - - private UUID id; - private EventType type; - private Long timestamp; - private String groupId; - private Payload payload; - private final Schema avroSchema; - - private static final ThreadLocal> DECODER_CACHES = decoderCaches(); - - public static byte[] encode(Event event) { - try { - return AvroEncoderUtil.encode(event, event.getSchema()); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - public static Event decode(byte[] bytes) { - try { - Event event = AvroEncoderUtil.decode(bytes); - // workaround for memory leak, until this is addressed upstream - DECODER_CACHES.get().clear(); - return event; - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - // Used by Avro reflection to instantiate this class when reading events - public Event(Schema avroSchema) { - this.avroSchema = avroSchema; - } - - public Event(String groupId, EventType type, Payload payload) { - this.id = UUID.randomUUID(); - this.type = type; - this.timestamp = System.currentTimeMillis(); - this.groupId = groupId; - this.payload = payload; - - this.avroSchema = - SchemaBuilder.builder() - .record(getClass().getName()) - .fields() - .name("id") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type(UUID_SCHEMA) - .noDefault() - .name("type") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .intType() - .noDefault() - .name("timestamp") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .longType() - .noDefault() - .name("payload") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type(payload.getSchema()) - .noDefault() - .name("groupId") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .stringType() - .noDefault() - .endRecord(); - } - - public UUID id() { - return id; - } - - public EventType type() { - return type; - } - - public Long timestamp() { - return timestamp; - } - - public Payload payload() { - return payload; - } - - public String groupId() { - return groupId; - } - - @Override - public Schema getSchema() { - return avroSchema; - } - - @Override - public void put(int i, Object v) { - switch (i) { - case 0: - this.id = (UUID) v; - return; - case 1: - this.type = v == null ? null : EventType.values()[(Integer) v]; - return; - case 2: - this.timestamp = (Long) v; - return; - case 3: - this.payload = (Payload) v; - return; - case 4: - this.groupId = v == null ? null : v.toString(); - return; - default: - // ignore the object, it must be from a newer version of the format - } - } - - @Override - public Object get(int i) { - switch (i) { - case 0: - return id; - case 1: - return type == null ? null : type.id(); - case 2: - return timestamp; - case 3: - return payload; - case 4: - return groupId; - default: - throw new UnsupportedOperationException("Unknown field ordinal: " + i); - } - } - - @SuppressWarnings("unchecked") - private static ThreadLocal> decoderCaches() { - return (ThreadLocal>) - DynFields.builder().hiddenImpl(DecoderResolver.class, "DECODER_CACHES").buildStatic().get(); - } -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java deleted file mode 100644 index a76ea9fe..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -public enum EventType { - COMMIT_REQUEST(0), - COMMIT_RESPONSE(1), - COMMIT_READY(2), - COMMIT_TABLE(3), - COMMIT_COMPLETE(4); - - private final int id; - - EventType(int id) { - this.id = id; - } - - public int id() { - return id; - } -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java deleted file mode 100644 index 2aedbb55..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -public interface Payload extends Element {} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java deleted file mode 100644 index 309ce49a..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import static java.util.stream.Collectors.toList; - -import java.util.Arrays; -import java.util.List; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.util.Utf8; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; - -public class TableName implements Element { - - private List namespace; - private String name; - private final Schema avroSchema; - - public static final Schema AVRO_SCHEMA = - SchemaBuilder.builder() - .record(TableName.class.getName()) - .fields() - .name("namespace") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .array() - .items() - .stringType() - .noDefault() - .name("name") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .stringType() - .noDefault() - .endRecord(); - - public static TableName of(TableIdentifier tableIdentifier) { - return new TableName( - Arrays.asList(tableIdentifier.namespace().levels()), tableIdentifier.name()); - } - - // Used by Avro reflection to instantiate this class when reading events - public TableName(Schema avroSchema) { - this.avroSchema = avroSchema; - } - - public TableName(List namespace, String name) { - this.namespace = namespace; - this.name = name; - this.avroSchema = AVRO_SCHEMA; - } - - public TableIdentifier toIdentifier() { - Namespace icebergNamespace = Namespace.of(namespace.toArray(new String[0])); - return TableIdentifier.of(icebergNamespace, name); - } - - @Override - public Schema getSchema() { - return avroSchema; - } - - @Override - @SuppressWarnings("unchecked") - public void put(int i, Object v) { - switch (i) { - case 0: - this.namespace = - v == null ? null : ((List) v).stream().map(Utf8::toString).collect(toList()); - return; - case 1: - this.name = v == null ? null : v.toString(); - return; - default: - // ignore the object, it must be from a newer version of the format - } - } - - @Override - public Object get(int i) { - switch (i) { - case 0: - return namespace; - case 1: - return name; - default: - throw new UnsupportedOperationException("Unknown field ordinal: " + i); - } - } -} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java deleted file mode 100644 index 2d989de4..00000000 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; - -public class TopicPartitionOffset implements Element { - - private String topic; - private Integer partition; - private Long offset; - private Long timestamp; - private final Schema avroSchema; - - public static final Schema AVRO_SCHEMA = - SchemaBuilder.builder() - .record(TopicPartitionOffset.class.getName()) - .fields() - .name("topic") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .stringType() - .noDefault() - .name("partition") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .intType() - .noDefault() - .name("offset") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .nullable() - .longType() - .noDefault() - .name("timestamp") - .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) - .type() - .nullable() - .longType() - .noDefault() - .endRecord(); - - // Used by Avro reflection to instantiate this class when reading events - public TopicPartitionOffset(Schema avroSchema) { - this.avroSchema = avroSchema; - } - - public TopicPartitionOffset(String topic, int partition, Long offset, Long timestamp) { - this.topic = topic; - this.partition = partition; - this.offset = offset; - this.timestamp = timestamp; - this.avroSchema = AVRO_SCHEMA; - } - - public String topic() { - return topic; - } - - public Integer partition() { - return partition; - } - - public Long offset() { - return offset; - } - - public Long timestamp() { - return timestamp; - } - - @Override - public Schema getSchema() { - return avroSchema; - } - - @Override - public void put(int i, Object v) { - switch (i) { - case 0: - this.topic = v == null ? null : v.toString(); - return; - case 1: - this.partition = (Integer) v; - return; - case 2: - this.offset = (Long) v; - return; - case 3: - this.timestamp = (Long) v; - return; - default: - // ignore the object, it must be from a newer version of the format - } - } - - @Override - public Object get(int i) { - switch (i) { - case 0: - return topic; - case 1: - return partition; - case 2: - return offset; - case 3: - return timestamp; - default: - throw new UnsupportedOperationException("Unknown field ordinal: " + i); - } - } -} diff --git a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java deleted file mode 100644 index 50029d00..00000000 --- a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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 io.tabular.iceberg.connect.events; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Arrays; -import java.util.Collections; -import java.util.UUID; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types.StructType; -import org.junit.jupiter.api.Test; - -public class EventSerializationTest { - - @Test - public void testCommitRequestSerialization() { - UUID commitId = UUID.randomUUID(); - Event event = - new Event("cg-connector", EventType.COMMIT_REQUEST, new CommitRequestPayload(commitId)); - - byte[] data = Event.encode(event); - Event result = Event.decode(data); - - assertThat(result.type()).isEqualTo(event.type()); - CommitRequestPayload payload = (CommitRequestPayload) result.payload(); - assertThat(payload.commitId()).isEqualTo(commitId); - } - - @Test - public void testCommitResponseSerialization() { - UUID commitId = UUID.randomUUID(); - Event event = - new Event( - "cg-connector", - EventType.COMMIT_RESPONSE, - new CommitResponsePayload( - StructType.of(), - commitId, - new TableName(Collections.singletonList("db"), "tbl"), - Arrays.asList(EventTestUtil.createDataFile(), EventTestUtil.createDataFile()), - Arrays.asList(EventTestUtil.createDeleteFile(), EventTestUtil.createDeleteFile()))); - - byte[] data = Event.encode(event); - Event result = Event.decode(data); - - assertThat(result.type()).isEqualTo(event.type()); - CommitResponsePayload payload = (CommitResponsePayload) result.payload(); - assertThat(payload.commitId()).isEqualTo(commitId); - assertThat(payload.tableName().toIdentifier()).isEqualTo(TableIdentifier.parse("db.tbl")); - assertThat(payload.dataFiles()).hasSize(2); - assertThat(payload.dataFiles()).allMatch(f -> f.specId() == 0); - assertThat(payload.deleteFiles()).hasSize(2); - assertThat(payload.deleteFiles()).allMatch(f -> f.specId() == 0); - } - - @Test - public void testCommitReadySerialization() { - UUID commitId = UUID.randomUUID(); - Event event = - new Event( - "cg-connector", - EventType.COMMIT_READY, - new CommitReadyPayload( - commitId, - Arrays.asList( - new TopicPartitionOffset("topic", 1, 1L, 1L), - new TopicPartitionOffset("topic", 2, null, null)))); - - byte[] data = Event.encode(event); - Event result = Event.decode(data); - - assertThat(result.type()).isEqualTo(event.type()); - CommitReadyPayload payload = (CommitReadyPayload) result.payload(); - assertThat(payload.commitId()).isEqualTo(commitId); - assertThat(payload.assignments()).hasSize(2); - assertThat(payload.assignments()).allMatch(tp -> tp.topic().equals("topic")); - } - - @Test - public void testCommitTableSerialization() { - UUID commitId = UUID.randomUUID(); - Event event = - new Event( - "cg-connector", - EventType.COMMIT_TABLE, - new CommitTablePayload( - commitId, new TableName(Collections.singletonList("db"), "tbl"), 1L, 2L)); - - byte[] data = Event.encode(event); - Event result = Event.decode(data); - - assertThat(result.type()).isEqualTo(event.type()); - CommitTablePayload payload = (CommitTablePayload) result.payload(); - assertThat(payload.commitId()).isEqualTo(commitId); - assertThat(payload.tableName().toIdentifier()).isEqualTo(TableIdentifier.parse("db.tbl")); - assertThat(payload.snapshotId()).isEqualTo(1L); - assertThat(payload.vtts()).isEqualTo(2L); - } - - @Test - public void testCommitCompleteSerialization() { - UUID commitId = UUID.randomUUID(); - Event event = - new Event( - "cg-connector", EventType.COMMIT_COMPLETE, new CommitCompletePayload(commitId, 2L)); - - byte[] data = Event.encode(event); - Event result = Event.decode(data); - - assertThat(result.type()).isEqualTo(event.type()); - CommitCompletePayload payload = (CommitCompletePayload) result.payload(); - assertThat(payload.commitId()).isEqualTo(commitId); - assertThat(payload.vtts()).isEqualTo(2L); - } -} diff --git a/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/EventTestUtil.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/fixtures/EventTestUtil.java similarity index 100% rename from kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/EventTestUtil.java rename to kafka-connect/src/test/java/io/tabular/iceberg/connect/fixtures/EventTestUtil.java From 42dd045b38aa335294b9b87b20f59dd49beb68d2 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Thu, 11 Apr 2024 16:52:04 -0600 Subject: [PATCH 02/13] porting --- gradle/libs.versions.toml | 5 +- kafka-connect/build.gradle | 3 +- .../iceberg/connect/channel/Channel.java | 7 +- .../iceberg/connect/channel/CommitState.java | 52 ++++--- .../iceberg/connect/channel/Coordinator.java | 45 +++---- .../iceberg/connect/channel/Deduplicated.java | 18 +-- .../iceberg/connect/channel/Envelope.java | 2 +- .../iceberg/connect/channel/Worker.java | 29 ++-- .../tabular/iceberg/connect/data/Offset.java | 14 +- .../connect/channel/CommitStateTest.java | 36 +++-- .../connect/channel/CoordinatorTest.java | 127 +++++++++--------- .../connect/channel/DeduplicatedTest.java | 12 +- .../iceberg/connect/channel/WorkerTest.java | 34 ++--- .../connect/fixtures/EventTestUtil.java | 2 +- 14 files changed, 211 insertions(+), 175 deletions(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 2420248d..991537e7 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -5,7 +5,7 @@ awaitility-ver = "4.2.0" hadoop-ver = "3.3.6" hive-ver = "2.3.9" http-client-ver = "5.2.1" -iceberg-ver = "1.4.2" +iceberg-ver = "1.5.0" jackson-ver = "2.14.2" junit-ver = "5.10.0" kafka-ver = "3.5.1" @@ -30,6 +30,7 @@ iceberg-gcp = { module = "org.apache.iceberg:iceberg-gcp", version.ref = "iceber iceberg-gcp-bundle = { module = "org.apache.iceberg:iceberg-gcp-bundle", version.ref = "iceberg-ver" } iceberg-guava = { module = "org.apache.iceberg:iceberg-bundled-guava", version.ref = "iceberg-ver" } iceberg-hive-metastore = { module = "org.apache.iceberg:iceberg-hive-metastore", version.ref = "iceberg-ver" } +iceberg-kafka-connect-events = {module = "org.apache.iceberg:iceberg-kafka-connect-events", version.ref = "iceberg-ver"} iceberg-nessie = { module = "org.apache.iceberg:iceberg-nessie", version.ref = "iceberg-ver" } iceberg-orc = { module = "org.apache.iceberg:iceberg-orc", version.ref = "iceberg-ver" } iceberg-parquet = { module = "org.apache.iceberg:iceberg-parquet", version.ref = "iceberg-ver" } @@ -57,7 +58,7 @@ palantir-gradle = "com.palantir.baseline:gradle-baseline-java:4.42.0" [bundles] -iceberg = ["iceberg-api", "iceberg-common", "iceberg-core", "iceberg-data", "iceberg-guava", "iceberg-orc", "iceberg-parquet"] +iceberg = ["iceberg-api", "iceberg-common", "iceberg-core", "iceberg-data", "iceberg-guava", "iceberg-orc", "iceberg-parquet", "iceberg-kafka-connect-events"] iceberg-ext = ["iceberg-aws", "iceberg-aws-bundle", "iceberg-azure", "iceberg-azure-bundle", "iceberg-gcp","iceberg-gcp-bundle", "iceberg-nessie"] jackson = ["jackson-core", "jackson-databind"] kafka-connect = ["kafka-clients", "kafka-connect-api", "kafka-connect-json", "kafka-connect-transforms"] diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 4f6337c4..756dbc6e 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -1,12 +1,10 @@ dependencies { - implementation project(":iceberg-kafka-connect-events") implementation libs.bundles.iceberg implementation libs.bundles.jackson implementation libs.avro implementation libs.slf4j compileOnly libs.bundles.kafka.connect - testImplementation(testFixtures(project(":iceberg-kafka-connect-events"))) testImplementation libs.hadoop.client testImplementation libs.junit.api @@ -15,6 +13,7 @@ dependencies { testImplementation libs.mockito testImplementation libs.assertj + testImplementation libs.iceberg.kafka.connect.events testImplementation 'ch.qos.logback:logback-classic:1.5.3' } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java index 38367b15..4efbfd6c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java @@ -22,11 +22,12 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.data.Offset; -import io.tabular.iceberg.connect.events.Event; import java.time.Duration; import java.util.List; import java.util.Map; import java.util.UUID; +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; @@ -84,7 +85,7 @@ protected void send(List events, Map sourceOffset .map( event -> { LOG.info("Sending event of type: {}", event.type().name()); - byte[] data = Event.encode(event); + byte[] data = AvroUtil.encode(event); // key by producer ID to keep event order return new ProducerRecord<>(controlTopic, producerId, data); }) @@ -122,7 +123,7 @@ record -> { // so increment the record offset by one controlTopicOffsets.put(record.partition(), record.offset() + 1); - Event event = Event.decode(record.value()); + Event event = AvroUtil.decode(record.value()); if (event.groupId().equals(groupId)) { LOG.debug("Received event of type: {}", event.type().name()); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java index d027846e..c195c91e 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java @@ -21,14 +21,18 @@ import static java.util.stream.Collectors.groupingBy; import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.events.CommitReadyPayload; -import io.tabular.iceberg.connect.events.CommitResponsePayload; -import io.tabular.iceberg.connect.events.TopicPartitionOffset; +import java.time.OffsetDateTime; +import java.util.Comparator; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; import java.util.UUID; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.TopicPartitionOffset; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,7 +40,7 @@ public class CommitState { private static final Logger LOG = LoggerFactory.getLogger(CommitState.class); private final List commitBuffer = new LinkedList<>(); - private final List readyBuffer = new LinkedList<>(); + private final List readyBuffer = new LinkedList<>(); private long startTime; private UUID currentCommitId; private final IcebergSinkConfig config; @@ -49,17 +53,17 @@ public void addResponse(Envelope envelope) { commitBuffer.add(envelope); if (!isCommitInProgress()) { LOG.warn( - "Received commit response with commit-id={} when no commit in progress, this can happen during recovery", - ((CommitResponsePayload) envelope.event().payload()).commitId()); + "Received data written with commit-id={} when no commit in progress, this can happen during recovery", + ((DataWritten) envelope.event().payload()).commitId()); } } public void addReady(Envelope envelope) { - readyBuffer.add((CommitReadyPayload) envelope.event().payload()); + readyBuffer.add((DataComplete) envelope.event().payload()); if (!isCommitInProgress()) { LOG.warn( - "Received commit ready for commit-id={} when no commit in progress, this can happen during recovery", - ((CommitReadyPayload) envelope.event().payload()).commitId()); + "Received data complete for commit-id={} when no commit in progress, this can happen during recovery", + ((DataComplete) envelope.event().payload()).commitId()); } } @@ -139,26 +143,38 @@ public Map> tableCommitMap() { .collect( groupingBy( envelope -> - ((CommitResponsePayload) envelope.event().payload()) - .tableName() - .toIdentifier())); + ((DataWritten) envelope.event().payload()).tableReference().identifier())); } - public Long vtts(boolean partialCommit) { + public OffsetDateTime vtts(boolean partialCommit) { + + Comparator comparator = + new Comparator() { + + @Override + public int compare(OffsetDateTime o1, OffsetDateTime o2) { + return o1.compareTo(o2); + } + }; + boolean validVtts = !partialCommit && readyBuffer.stream() .flatMap(event -> event.assignments().stream()) .allMatch(offset -> offset.timestamp() != null); - Long result; + OffsetDateTime result; if (validVtts) { - result = + Optional maybeResult = readyBuffer.stream() .flatMap(event -> event.assignments().stream()) - .mapToLong(TopicPartitionOffset::timestamp) - .min() - .getAsLong(); + .map(TopicPartitionOffset::timestamp) + .min(comparator); + if (maybeResult.isPresent()) { + result = maybeResult.get(); + } else { + throw new NoSuchElementException("no vtts found"); + } } else { result = null; } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 439ba4d2..63469bd7 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -23,15 +23,10 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.events.CommitCompletePayload; -import io.tabular.iceberg.connect.events.CommitRequestPayload; -import io.tabular.iceberg.connect.events.CommitTablePayload; -import io.tabular.iceberg.connect.events.Event; -import io.tabular.iceberg.connect.events.EventType; -import io.tabular.iceberg.connect.events.TableName; import java.io.IOException; import java.io.UncheckedIOException; import java.time.Duration; +import java.time.OffsetDateTime; import java.util.Collection; import java.util.List; import java.util.Map; @@ -47,6 +42,11 @@ import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.events.CommitComplete; +import org.apache.iceberg.connect.events.CommitToTable; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.StartCommit; +import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -95,10 +95,7 @@ public void process() { // send out begin commit commitState.startNewCommit(); Event event = - new Event( - config.controlGroupId(), - EventType.COMMIT_REQUEST, - new CommitRequestPayload(commitState.currentCommitId())); + new Event(config.controlGroupId(), new StartCommit(commitState.currentCommitId())); send(event); LOG.info("Started new commit with commit-id={}", commitState.currentCommitId().toString()); } @@ -113,10 +110,10 @@ public void process() { @Override protected boolean receive(Envelope envelope) { switch (envelope.event().type()) { - case COMMIT_RESPONSE: + case DATA_WRITTEN: commitState.addResponse(envelope); return true; - case COMMIT_READY: + case DATA_COMPLETE: commitState.addReady(envelope); if (commitState.isCommitReady(totalPartitionCount)) { commit(false); @@ -140,7 +137,7 @@ private void doCommit(boolean partialCommit) { Map> commitMap = commitState.tableCommitMap(); String offsetsJson = offsetsJson(); - Long vtts = commitState.vtts(partialCommit); + OffsetDateTime vtts = commitState.vtts(partialCommit); Tasks.foreach(commitMap.entrySet()) .executeWith(exec) @@ -155,10 +152,7 @@ private void doCommit(boolean partialCommit) { commitState.clearResponses(); Event event = - new Event( - config.controlGroupId(), - EventType.COMMIT_COMPLETE, - new CommitCompletePayload(commitState.currentCommitId(), vtts)); + new Event(config.controlGroupId(), new CommitComplete(commitState.currentCommitId(), vtts)); send(event); LOG.info( @@ -177,7 +171,10 @@ private String offsetsJson() { } private void commitToTable( - TableIdentifier tableIdentifier, List envelopeList, String offsetsJson, Long vtts) { + TableIdentifier tableIdentifier, + List envelopeList, + String offsetsJson, + OffsetDateTime vtts) { Table table; try { table = catalog.loadTable(tableIdentifier); @@ -233,7 +230,7 @@ private void commitToTable( if (i == lastIdx) { appendOp.set(snapshotOffsetsProp, offsetsJson); if (vtts != null) { - appendOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts)); + appendOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } } @@ -247,7 +244,7 @@ private void commitToTable( deltaOp.set(snapshotOffsetsProp, offsetsJson); deltaOp.set(COMMIT_ID_SNAPSHOT_PROP, commitState.currentCommitId().toString()); if (vtts != null) { - deltaOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts)); + deltaOp.set(VTTS_SNAPSHOT_PROP, Long.toString(vtts.toInstant().toEpochMilli())); } dataFiles.forEach(deltaOp::addRows); deleteFiles.forEach(deltaOp::addDeletes); @@ -258,9 +255,11 @@ private void commitToTable( Event event = new Event( config.controlGroupId(), - EventType.COMMIT_TABLE, - new CommitTablePayload( - commitState.currentCommitId(), TableName.of(tableIdentifier), snapshotId, vtts)); + new CommitToTable( + commitState.currentCommitId(), + TableReference.of(catalog.name(), tableIdentifier), + snapshotId, + vtts)); send(event); LOG.info( diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Deduplicated.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Deduplicated.java index 67f46f79..fc81030e 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Deduplicated.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Deduplicated.java @@ -20,7 +20,7 @@ import static java.util.stream.Collectors.toList; -import io.tabular.iceberg.connect.events.CommitResponsePayload; +import java.time.OffsetDateTime; import java.util.List; import java.util.Objects; import java.util.UUID; @@ -31,6 +31,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.events.DataWritten; import org.apache.iceberg.util.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,7 +83,7 @@ public static List dataFiles( tableIdentifier, envelopes, "data", - CommitResponsePayload::dataFiles, + DataWritten::dataFiles, dataFile -> dataFile.path().toString()); } @@ -97,7 +98,7 @@ public static List deleteFiles( tableIdentifier, envelopes, "delete", - CommitResponsePayload::deleteFiles, + DataWritten::deleteFiles, deleteFile -> deleteFile.path().toString()); } @@ -106,14 +107,13 @@ private static List deduplicatedFiles( TableIdentifier tableIdentifier, List envelopes, String fileType, - Function> extractFilesFromPayload, + Function> extractFilesFromPayload, Function extractPathFromFile) { List> filesAndEnvelopes = envelopes.stream() .flatMap( envelope -> { - CommitResponsePayload payload = - (CommitResponsePayload) envelope.event().payload(); + DataWritten payload = (DataWritten) envelope.event().payload(); List files = extractFilesFromPayload.apply(payload); if (files == null) { return Stream.empty(); @@ -207,7 +207,7 @@ private static class SimpleEnvelope { private final long offset; private final UUID eventId; private final String eventGroupId; - private final Long eventTimestamp; + private final OffsetDateTime eventTimestamp; private final UUID payloadCommitId; SimpleEnvelope(Envelope envelope) { @@ -216,7 +216,7 @@ private static class SimpleEnvelope { eventId = envelope.event().id(); eventGroupId = envelope.event().groupId(); eventTimestamp = envelope.event().timestamp(); - payloadCommitId = ((CommitResponsePayload) envelope.event().payload()).commitId(); + payloadCommitId = ((DataWritten) envelope.event().payload()).commitId(); } @Override @@ -255,7 +255,7 @@ public String toString() { + eventGroupId + '\'' + ", eventTimestamp=" - + eventTimestamp + + eventTimestamp.toInstant().toEpochMilli() + ", payloadCommitId=" + payloadCommitId + '}'; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Envelope.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Envelope.java index 3458a3b7..27939897 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Envelope.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Envelope.java @@ -18,7 +18,7 @@ */ package io.tabular.iceberg.connect.channel; -import io.tabular.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.Event; public class Envelope { private final Event event; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java index 512471c7..aaf8c404 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Worker.java @@ -27,13 +27,6 @@ import io.tabular.iceberg.connect.data.RecordWriter; import io.tabular.iceberg.connect.data.Utilities; import io.tabular.iceberg.connect.data.WriterResult; -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.time.Duration; import java.util.Collection; import java.util.List; @@ -41,6 +34,13 @@ import java.util.Map.Entry; import java.util.UUID; import java.util.concurrent.ExecutionException; +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.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult; @@ -104,7 +104,7 @@ public void process() { @Override protected boolean receive(Envelope envelope) { Event event = envelope.event(); - if (event.type() != EventType.COMMIT_REQUEST) { + if (event.type() != PayloadType.START_COMMIT) { return false; } @@ -131,7 +131,7 @@ protected boolean receive(Envelope envelope) { }) .collect(toList()); - UUID commitId = ((CommitRequestPayload) event.payload()).commitId(); + UUID commitId = ((StartCommit) event.payload()).commitId(); List events = writeResults.stream() @@ -139,20 +139,15 @@ protected boolean receive(Envelope envelope) { writeResult -> new Event( config.controlGroupId(), - EventType.COMMIT_RESPONSE, - new CommitResponsePayload( + new DataWritten( writeResult.partitionStruct(), commitId, - TableName.of(writeResult.tableIdentifier()), + TableReference.of(config.catalogName(), writeResult.tableIdentifier()), writeResult.dataFiles(), writeResult.deleteFiles()))) .collect(toList()); - Event readyEvent = - new Event( - config.controlGroupId(), - EventType.COMMIT_READY, - new CommitReadyPayload(commitId, assignments)); + Event readyEvent = new Event(config.controlGroupId(), new DataComplete(commitId, assignments)); events.add(readyEvent); send(events, offsets); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Offset.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Offset.java index 0ea6261c..c04a7bec 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Offset.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Offset.java @@ -18,6 +18,9 @@ */ package io.tabular.iceberg.connect.data; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; import java.util.Objects; public class Offset implements Comparable { @@ -36,8 +39,15 @@ public Long offset() { return offset; } - public Long timestamp() { - return timestamp; + // public Long timestamp() { + // return timestamp; + // } + + public OffsetDateTime timestamp() { + if (timestamp == null) { + return null; + } + return OffsetDateTime.ofInstant(Instant.ofEpochMilli(timestamp), ZoneOffset.UTC); } @Override diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java index e06e7f1e..9ee165d9 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitStateTest.java @@ -23,15 +23,23 @@ import static org.mockito.Mockito.when; import io.tabular.iceberg.connect.IcebergSinkConfig; -import io.tabular.iceberg.connect.events.CommitReadyPayload; -import io.tabular.iceberg.connect.events.Event; -import io.tabular.iceberg.connect.events.Payload; -import io.tabular.iceberg.connect.events.TopicPartitionOffset; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; import java.util.UUID; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.Payload; +import org.apache.iceberg.connect.events.TopicPartitionOffset; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Test; public class CommitStateTest { + + private OffsetDateTime offsetDateTime(Long ts) { + return OffsetDateTime.ofInstant(Instant.ofEpochMilli(ts), ZoneOffset.UTC); + } + @Test public void testIsCommitReady() { TopicPartitionOffset tp = mock(TopicPartitionOffset.class); @@ -39,15 +47,15 @@ public void testIsCommitReady() { CommitState commitState = new CommitState(mock(IcebergSinkConfig.class)); commitState.startNewCommit(); - CommitReadyPayload payload1 = mock(CommitReadyPayload.class); + DataComplete payload1 = mock(DataComplete.class); when(payload1.commitId()).thenReturn(commitState.currentCommitId()); when(payload1.assignments()).thenReturn(ImmutableList.of(tp, tp)); - CommitReadyPayload payload2 = mock(CommitReadyPayload.class); + DataComplete payload2 = mock(DataComplete.class); when(payload2.commitId()).thenReturn(commitState.currentCommitId()); when(payload2.assignments()).thenReturn(ImmutableList.of(tp)); - CommitReadyPayload payload3 = mock(CommitReadyPayload.class); + DataComplete payload3 = mock(DataComplete.class); when(payload3.commitId()).thenReturn(UUID.randomUUID()); when(payload3.assignments()).thenReturn(ImmutableList.of(tp)); @@ -61,16 +69,16 @@ public void testIsCommitReady() { @Test public void testGetVtts() { - CommitReadyPayload payload1 = mock(CommitReadyPayload.class); + DataComplete payload1 = mock(DataComplete.class); TopicPartitionOffset tp1 = mock(TopicPartitionOffset.class); - when(tp1.timestamp()).thenReturn(3L); + when(tp1.timestamp()).thenReturn(offsetDateTime(3L)); TopicPartitionOffset tp2 = mock(TopicPartitionOffset.class); - when(tp2.timestamp()).thenReturn(2L); + when(tp2.timestamp()).thenReturn(offsetDateTime(2L)); when(payload1.assignments()).thenReturn(ImmutableList.of(tp1, tp2)); - CommitReadyPayload payload2 = mock(CommitReadyPayload.class); + DataComplete payload2 = mock(DataComplete.class); TopicPartitionOffset tp3 = mock(TopicPartitionOffset.class); - when(tp3.timestamp()).thenReturn(1L); + when(tp3.timestamp()).thenReturn(offsetDateTime(1L)); when(payload2.assignments()).thenReturn(ImmutableList.of(tp3)); CommitState commitState = new CommitState(mock(IcebergSinkConfig.class)); @@ -79,11 +87,11 @@ public void testGetVtts() { commitState.addReady(wrapInEnvelope(payload1)); commitState.addReady(wrapInEnvelope(payload2)); - assertThat(commitState.vtts(false)).isEqualTo(1L); + assertThat(commitState.vtts(false)).isEqualTo(offsetDateTime(1L)); assertThat(commitState.vtts(true)).isNull(); // null timestamp for one, so should not set a vtts - CommitReadyPayload payload3 = mock(CommitReadyPayload.class); + DataComplete payload3 = mock(DataComplete.class); TopicPartitionOffset tp4 = mock(TopicPartitionOffset.class); when(tp4.timestamp()).thenReturn(null); when(payload3.assignments()).thenReturn(ImmutableList.of(tp4)); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java index 353036fd..09233acf 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CoordinatorTest.java @@ -21,16 +21,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.when; -import io.tabular.iceberg.connect.events.CommitCompletePayload; -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.CommitTablePayload; -import io.tabular.iceberg.connect.events.Event; -import io.tabular.iceberg.connect.events.EventTestUtil; -import io.tabular.iceberg.connect.events.EventType; -import io.tabular.iceberg.connect.events.TableName; -import io.tabular.iceberg.connect.events.TopicPartitionOffset; +import io.tabular.iceberg.connect.fixtures.EventTestUtil; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; import java.util.List; import java.util.Map; import java.util.Set; @@ -46,6 +40,16 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.CommitComplete; +import org.apache.iceberg.connect.events.CommitToTable; +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.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; @@ -64,7 +68,7 @@ public class CoordinatorTest extends ChannelTestBase { public void testCommitAppend() { Assertions.assertEquals(0, ImmutableList.copyOf(table.snapshots().iterator()).size()); - long ts = System.currentTimeMillis(); + OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); UUID commitId = coordinatorTest(ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(), ts); table.refresh(); @@ -84,12 +88,13 @@ public void testCommitAppend() { Map summary = snapshot.summary(); Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); - Assertions.assertEquals(Long.toString(ts), summary.get(VTTS_SNAPSHOT_PROP)); + Assertions.assertEquals( + Long.toString(ts.toInstant().toEpochMilli()), summary.get(VTTS_SNAPSHOT_PROP)); } @Test public void testCommitDelta() { - long ts = System.currentTimeMillis(); + OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); UUID commitId = coordinatorTest( ImmutableList.of(EventTestUtil.createDataFile()), @@ -111,12 +116,13 @@ public void testCommitDelta() { Map summary = snapshot.summary(); Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); - Assertions.assertEquals(Long.toString(ts), summary.get(VTTS_SNAPSHOT_PROP)); + Assertions.assertEquals( + Long.toString(ts.toInstant().toEpochMilli()), summary.get(VTTS_SNAPSHOT_PROP)); } @Test public void testCommitNoFiles() { - long ts = System.currentTimeMillis(); + OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); UUID commitId = coordinatorTest(ImmutableList.of(), ImmutableList.of(), ts); assertThat(producer.history()).hasSize(2); @@ -139,7 +145,10 @@ public void testCommitError() { .withRecordCount(5) .build(); - coordinatorTest(ImmutableList.of(badDataFile), ImmutableList.of(), 0L); + coordinatorTest( + ImmutableList.of(badDataFile), + ImmutableList.of(), + OffsetDateTime.ofInstant(Instant.ofEpochMilli(0L), ZoneOffset.UTC)); // no commit messages sent assertThat(producer.history()).hasSize(1); @@ -150,7 +159,7 @@ public void testCommitError() { @Test public void testShouldDeduplicateDataFilesBeforeAppending() { - long ts = System.currentTimeMillis(); + OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); DataFile dataFile = EventTestUtil.createDataFile(); UUID commitId = @@ -159,11 +168,10 @@ public void testShouldDeduplicateDataFilesBeforeAppending() { Event commitResponse = new Event( config.controlGroupId(), - EventType.COMMIT_RESPONSE, - new CommitResponsePayload( + new DataWritten( StructType.of(), currentCommitId, - new TableName(ImmutableList.of("db"), "tbl"), + new TableReference("catalog", ImmutableList.of("db"), "tbl"), ImmutableList.of(dataFile, dataFile), // duplicated data files ImmutableList.of())); @@ -172,8 +180,7 @@ public void testShouldDeduplicateDataFilesBeforeAppending() { commitResponse, // duplicate commit response new Event( config.controlGroupId(), - EventType.COMMIT_READY, - new CommitReadyPayload( + new DataComplete( currentCommitId, ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts))))); }); @@ -192,7 +199,7 @@ public void testShouldDeduplicateDataFilesBeforeAppending() { @Test public void testShouldDeduplicateDeleteFilesBeforeAppending() { - long ts = System.currentTimeMillis(); + OffsetDateTime ts = OffsetDateTime.ofInstant(Instant.now(), ZoneOffset.UTC); DeleteFile deleteFile = EventTestUtil.createDeleteFile(); UUID commitId = @@ -201,11 +208,10 @@ public void testShouldDeduplicateDeleteFilesBeforeAppending() { Event duplicateCommitResponse = new Event( config.controlGroupId(), - EventType.COMMIT_RESPONSE, - new CommitResponsePayload( + new DataWritten( StructType.of(), currentCommitId, - new TableName(ImmutableList.of("db"), "tbl"), + new TableReference("catalog", ImmutableList.of("db"), "tbl"), ImmutableList.of(), ImmutableList.of(deleteFile, deleteFile))); // duplicate delete files @@ -214,8 +220,7 @@ public void testShouldDeduplicateDeleteFilesBeforeAppending() { duplicateCommitResponse, // duplicate commit response new Event( config.controlGroupId(), - EventType.COMMIT_READY, - new CommitReadyPayload( + new DataComplete( currentCommitId, ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts))))); }); @@ -294,9 +299,9 @@ public void testCommitMultiPartitionSpecAppendDataFiles() { // retrieve commitId from commit request produced by coordinator final byte[] bytes = producer.history().get(0).value(); - final Event commitRequest = Event.decode(bytes); - assert commitRequest.type().equals(EventType.COMMIT_REQUEST); - final UUID commitId = ((CommitRequestPayload) commitRequest.payload()).commitId(); + final Event commitRequest = AvroUtil.decode(bytes); + assert commitRequest.type().equals(PayloadType.START_COMMIT); + final UUID commitId = ((StartCommit) commitRequest.payload()).commitId(); // each worker sends its responses for the commit request Map workerIdToSpecMap = @@ -323,14 +328,13 @@ public void testCommitMultiPartitionSpecAppendDataFiles() { 0, currentControlTopicOffset, "key", - Event.encode( + AvroUtil.encode( new Event( config.controlGroupId(), - EventType.COMMIT_RESPONSE, - new CommitResponsePayload( + new DataWritten( spec.partitionType(), commitId, - TableName.of(TABLE_IDENTIFIER), + TableReference.of("catalog", TABLE_IDENTIFIER), ImmutableList.of(dataFile), ImmutableList.of()))))); currentControlTopicOffset += 1; @@ -341,14 +345,18 @@ public void testCommitMultiPartitionSpecAppendDataFiles() { 0, currentControlTopicOffset, "key", - Event.encode( + AvroUtil.encode( new Event( config.controlGroupId(), - EventType.COMMIT_READY, - new CommitReadyPayload( + new DataComplete( commitId, ImmutableList.of( - new TopicPartitionOffset(SRC_TOPIC_NAME, 0, 100L, 100L))))))); + new TopicPartitionOffset( + SRC_TOPIC_NAME, + 0, + 100L, + OffsetDateTime.ofInstant( + Instant.ofEpochMilli(100L), ZoneOffset.UTC)))))))); currentControlTopicOffset += 1; } @@ -391,45 +399,44 @@ public void testCommitMultiPartitionSpecAppendDataFiles() { "Only the most recent snapshot should include vtts in it's summary"); } - private void assertCommitTable(int idx, UUID commitId, long ts) { + private void assertCommitTable(int idx, UUID commitId, OffsetDateTime ts) { byte[] bytes = producer.history().get(idx).value(); - Event commitTable = Event.decode(bytes); - assertThat(commitTable.type()).isEqualTo(EventType.COMMIT_TABLE); - CommitTablePayload commitTablePayload = (CommitTablePayload) commitTable.payload(); + Event commitTable = AvroUtil.decode(bytes); + assertThat(commitTable.type()).isEqualTo(PayloadType.COMMIT_TO_TABLE); + CommitToTable commitTablePayload = (CommitToTable) commitTable.payload(); assertThat(commitTablePayload.commitId()).isEqualTo(commitId); - assertThat(commitTablePayload.tableName().toIdentifier().toString()) + assertThat(commitTablePayload.tableReference().identifier().toString()) .isEqualTo(TABLE_IDENTIFIER.toString()); - assertThat(commitTablePayload.vtts()).isEqualTo(ts); + assertThat(commitTablePayload.validThroughTs()).isEqualTo(ts); } - private void assertCommitComplete(int idx, UUID commitId, long ts) { + private void assertCommitComplete(int idx, UUID commitId, OffsetDateTime ts) { byte[] bytes = producer.history().get(idx).value(); - Event commitComplete = Event.decode(bytes); - assertThat(commitComplete.type()).isEqualTo(EventType.COMMIT_COMPLETE); - CommitCompletePayload commitCompletePayload = (CommitCompletePayload) commitComplete.payload(); + Event commitComplete = AvroUtil.decode(bytes); + assertThat(commitComplete.type()).isEqualTo(PayloadType.COMMIT_COMPLETE); + CommitComplete commitCompletePayload = (CommitComplete) commitComplete.payload(); assertThat(commitCompletePayload.commitId()).isEqualTo(commitId); - assertThat(commitCompletePayload.vtts()).isEqualTo(ts); + assertThat(commitCompletePayload.validThroughTs()).isEqualTo(ts); } - private UUID coordinatorTest(List dataFiles, List deleteFiles, long ts) { + private UUID coordinatorTest( + List dataFiles, List deleteFiles, OffsetDateTime ts) { return coordinatorTest( currentCommitId -> { Event commitResponse = new Event( config.controlGroupId(), - EventType.COMMIT_RESPONSE, - new CommitResponsePayload( + new DataWritten( StructType.of(), currentCommitId, - new TableName(ImmutableList.of("db"), "tbl"), + new TableReference("catalog", ImmutableList.of("db"), "tbl"), dataFiles, deleteFiles)); Event commitReady = new Event( config.controlGroupId(), - EventType.COMMIT_READY, - new CommitReadyPayload( + new DataComplete( currentCommitId, ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts)))); @@ -453,14 +460,14 @@ private UUID coordinatorTest(Function> eventsFn) { assertThat(producer.history()).hasSize(1); byte[] bytes = producer.history().get(0).value(); - Event commitRequest = Event.decode(bytes); - assertThat(commitRequest.type()).isEqualTo(EventType.COMMIT_REQUEST); + Event commitRequest = AvroUtil.decode(bytes); + assertThat(commitRequest.type()).isEqualTo(PayloadType.START_COMMIT); - UUID commitId = ((CommitRequestPayload) commitRequest.payload()).commitId(); + UUID commitId = ((StartCommit) commitRequest.payload()).commitId(); int currentOffset = 1; for (Event event : eventsFn.apply(commitId)) { - bytes = Event.encode(event); + bytes = AvroUtil.encode(event); consumer.addRecord(new ConsumerRecord<>(CTL_TOPIC_NAME, 0, currentOffset, "key", bytes)); currentOffset += 1; } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/DeduplicatedTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/DeduplicatedTest.java index 24685b04..3b066ce8 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/DeduplicatedTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/DeduplicatedTest.java @@ -18,12 +18,8 @@ */ package io.tabular.iceberg.connect.channel; -import static io.tabular.iceberg.connect.events.EventType.COMMIT_RESPONSE; import static org.assertj.core.api.Assertions.assertThat; -import io.tabular.iceberg.connect.events.CommitResponsePayload; -import io.tabular.iceberg.connect.events.Event; -import io.tabular.iceberg.connect.events.TableName; import java.util.List; import java.util.Set; import java.util.UUID; @@ -35,6 +31,9 @@ import org.apache.iceberg.FileMetadata; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -54,7 +53,7 @@ class DeduplicatedTest { private static final UUID PAYLOAD_COMMIT_ID = UUID.fromString("4142add7-7c92-4bbe-b864-21ce8ac4bf53"); private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("db", "tbl"); - private static final TableName TABLE_NAME = TableName.of(TABLE_IDENTIFIER); + private static final TableReference TABLE_NAME = TableReference.of("catalog", TABLE_IDENTIFIER); private static final String GROUP_ID = "some-group"; private static final DataFile DATA_FILE_1 = createDataFile("1"); private static final DataFile DATA_FILE_2 = createDataFile("2"); @@ -124,8 +123,7 @@ private void assertWarnOrHigherLogsContainsEntryMatching(String expectedMessages private Event commitResponseEvent(List dataFiles, List deleteFiles) { return new Event( GROUP_ID, - COMMIT_RESPONSE, - new CommitResponsePayload( + new DataWritten( Types.StructType.of(), PAYLOAD_COMMIT_ID, TABLE_NAME, dataFiles, deleteFiles)); } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index c7954900..5e21e287 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -27,15 +27,16 @@ import io.tabular.iceberg.connect.data.IcebergWriter; import io.tabular.iceberg.connect.data.IcebergWriterFactory; import io.tabular.iceberg.connect.data.WriterResult; -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.EventTestUtil; -import io.tabular.iceberg.connect.events.EventType; +import io.tabular.iceberg.connect.fixtures.EventTestUtil; import java.util.Map; import java.util.UUID; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.events.AvroUtil; +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.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.ImmutableSet; @@ -54,6 +55,7 @@ public class WorkerTest extends ChannelTestBase { @Test public void testStaticRoute() { when(config.tables()).thenReturn(ImmutableList.of(TABLE_NAME)); + when(config.catalogName()).thenReturn("catalog"); Map value = ImmutableMap.of(FIELD_NAME, "val"); workerTest(value); } @@ -62,6 +64,8 @@ public void testStaticRoute() { public void testDynamicRoute() { when(config.dynamicTablesEnabled()).thenReturn(true); when(config.tablesRouteField()).thenReturn(FIELD_NAME); + when(config.catalogName()).thenReturn("catalog"); + Map value = ImmutableMap.of(FIELD_NAME, TABLE_NAME); workerTest(value); } @@ -93,24 +97,22 @@ private void workerTest(Map value) { worker.save(ImmutableList.of(rec)); UUID commitId = UUID.randomUUID(); - Event commitRequest = - new Event( - config.controlGroupId(), EventType.COMMIT_REQUEST, new CommitRequestPayload(commitId)); - byte[] bytes = Event.encode(commitRequest); + Event commitRequest = new Event(config.controlGroupId(), new StartCommit(commitId)); + byte[] bytes = AvroUtil.encode(commitRequest); consumer.addRecord(new ConsumerRecord<>(CTL_TOPIC_NAME, 0, 1, "key", bytes)); worker.process(); assertThat(producer.history()).hasSize(2); - Event event = Event.decode(producer.history().get(0).value()); - assertThat(event.type()).isEqualTo(EventType.COMMIT_RESPONSE); - CommitResponsePayload responsePayload = (CommitResponsePayload) event.payload(); + Event event = AvroUtil.decode(producer.history().get(0).value()); + assertThat(event.type()).isEqualTo(PayloadType.DATA_WRITTEN); + DataWritten responsePayload = (DataWritten) event.payload(); assertThat(responsePayload.commitId()).isEqualTo(commitId); - event = Event.decode(producer.history().get(1).value()); - assertThat(event.type()).isEqualTo(EventType.COMMIT_READY); - CommitReadyPayload readyPayload = (CommitReadyPayload) event.payload(); + event = AvroUtil.decode(producer.history().get(1).value()); + assertThat(event.type()).isEqualTo(PayloadType.DATA_COMPLETE); + DataComplete readyPayload = (DataComplete) event.payload(); assertThat(readyPayload.commitId()).isEqualTo(commitId); assertThat(readyPayload.assignments()).hasSize(1); // offset should be one more than the record offset diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/fixtures/EventTestUtil.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/fixtures/EventTestUtil.java index 3a7e7143..77a4a129 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/fixtures/EventTestUtil.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/fixtures/EventTestUtil.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events; +package io.tabular.iceberg.connect.fixtures; import java.util.UUID; import org.apache.iceberg.DataFile; From bbf72f45a9b0421dc56183439a10312026d1abf8 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Thu, 11 Apr 2024 20:05:40 -0600 Subject: [PATCH 03/13] prefer config catalog name over catalog.name --- .../java/io/tabular/iceberg/connect/channel/Coordinator.java | 2 +- .../src/main/java/io/tabular/iceberg/connect/data/Offset.java | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java index 63469bd7..aae21974 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Coordinator.java @@ -257,7 +257,7 @@ private void commitToTable( config.controlGroupId(), new CommitToTable( commitState.currentCommitId(), - TableReference.of(catalog.name(), tableIdentifier), + TableReference.of(config.catalogName(), tableIdentifier), snapshotId, vtts)); send(event); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Offset.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Offset.java index c04a7bec..bfe05ad8 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Offset.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Offset.java @@ -39,10 +39,6 @@ public Long offset() { return offset; } - // public Long timestamp() { - // return timestamp; - // } - public OffsetDateTime timestamp() { if (timestamp == null) { return null; From 38c50e41334352c9e4379593d589df429dee51c9 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Sun, 21 Apr 2024 11:08:51 -0600 Subject: [PATCH 04/13] fix mocks --- .../java/io/tabular/iceberg/connect/channel/ChannelTestBase.java | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java index 7bfcfd65..6f1215b9 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/ChannelTestBase.java @@ -95,6 +95,7 @@ public void before() { when(config.commitThreads()).thenReturn(1); when(config.controlGroupId()).thenReturn(CONTROL_CONSUMER_GROUP_ID); when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + when(config.catalogName()).thenReturn("catalog"); TopicPartitionInfo partitionInfo = mock(TopicPartitionInfo.class); when(partitionInfo.partition()).thenReturn(0); From 3a1263620dd0c39f26afe7f7963a162c58cb1539 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 30 Apr 2024 15:45:51 -0700 Subject: [PATCH 05/13] fallback avro deserializer --- kafka-connect-events/build.gradle | 28 + .../deprecated/CommitCompletePayload.java | 97 ++++ .../events/deprecated/CommitReadyPayload.java | 99 ++++ .../deprecated/CommitRequestPayload.java | 79 +++ .../deprecated/CommitResponsePayload.java | 154 ++++++ .../events/deprecated/CommitTablePayload.java | 129 +++++ .../connect/events/deprecated/Element.java | 37 ++ .../connect/events/deprecated/Event.java | 174 +++++++ .../connect/events/deprecated/EventType.java | 37 ++ .../connect/events/deprecated/Payload.java | 21 + .../connect/events/deprecated/TableName.java | 108 ++++ .../deprecated/TopicPartitionOffset.java | 129 +++++ .../avro/DeprecatedAvroEncoderUtil.java | 89 ++++ .../avro/DeprecatedAvroSchemaUtil.java | 480 ++++++++++++++++++ .../avro/DeprecatedGenericAvroReader.java | 186 +++++++ .../iceberg/avro/DeprecatedTypeToSchema.java | 245 +++++++++ .../deprecated/EventSerializationTest.java | 132 +++++ .../events/deprecated/EventTestUtil.java | 49 ++ kafka-connect/build.gradle | 2 + .../iceberg/connect/channel/Channel.java | 2 +- .../iceberg/connect/channel/EventDecoder.java | 57 +++ 21 files changed, 2333 insertions(+), 1 deletion(-) create mode 100644 kafka-connect-events/build.gradle create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitCompletePayload.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitReadyPayload.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitRequestPayload.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitResponsePayload.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitTablePayload.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Element.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Event.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/EventType.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Payload.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TableName.java create mode 100644 kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TopicPartitionOffset.java create mode 100644 kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroEncoderUtil.java create mode 100644 kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroSchemaUtil.java create mode 100644 kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedGenericAvroReader.java create mode 100644 kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedTypeToSchema.java create mode 100644 kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/deprecated/EventSerializationTest.java create mode 100644 kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/deprecated/EventTestUtil.java create mode 100644 kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java diff --git a/kafka-connect-events/build.gradle b/kafka-connect-events/build.gradle new file mode 100644 index 00000000..9817c844 --- /dev/null +++ b/kafka-connect-events/build.gradle @@ -0,0 +1,28 @@ +plugins { + id "java-test-fixtures" +} + +dependencies { + implementation libs.iceberg.core + implementation libs.iceberg.common + implementation libs.iceberg.guava + implementation libs.avro + + testImplementation libs.junit.api + testRuntimeOnly libs.junit.engine + + testImplementation libs.mockito + testImplementation libs.assertj + + testFixturesImplementation libs.iceberg.common + testFixturesImplementation libs.iceberg.core + testFixturesImplementation libs.avro +} + +publishing { + publications { + mavenJava(MavenPublication) { + from components.java + } + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitCompletePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitCompletePayload.java new file mode 100644 index 00000000..b5f4d272 --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitCompletePayload.java @@ -0,0 +1,97 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import java.util.UUID; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; + +public class CommitCompletePayload implements Payload { + + private UUID commitId; + private Long vtts; + private final Schema avroSchema; + + private static final Schema AVRO_SCHEMA = + SchemaBuilder.builder() + .record(CommitCompletePayload.class.getName()) + .fields() + .name("commitId") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type(UUID_SCHEMA) + .noDefault() + .name("vtts") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .longType() + .noDefault() + .endRecord(); + + // Used by Avro reflection to instantiate this class when reading events + public CommitCompletePayload(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public CommitCompletePayload(UUID commitId, Long vtts) { + this.commitId = commitId; + this.vtts = vtts; + this.avroSchema = AVRO_SCHEMA; + } + + public UUID commitId() { + return commitId; + } + + public Long vtts() { + return vtts; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + @SuppressWarnings("unchecked") + public void put(int i, Object v) { + switch (i) { + case 0: + this.commitId = (UUID) v; + return; + case 1: + this.vtts = (Long) v; + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return commitId; + case 1: + return vtts; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitReadyPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitReadyPayload.java new file mode 100644 index 00000000..4d027885 --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitReadyPayload.java @@ -0,0 +1,99 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import java.util.List; +import java.util.UUID; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; + +public class CommitReadyPayload implements Payload { + + private UUID commitId; + private List assignments; + private final Schema avroSchema; + + private static final Schema AVRO_SCHEMA = + SchemaBuilder.builder() + .record(CommitReadyPayload.class.getName()) + .fields() + .name("commitId") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type(UUID_SCHEMA) + .noDefault() + .name("assignments") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .array() + .items(TopicPartitionOffset.AVRO_SCHEMA) + .noDefault() + .endRecord(); + + // Used by Avro reflection to instantiate this class when reading events + public CommitReadyPayload(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public CommitReadyPayload(UUID commitId, List assignments) { + this.commitId = commitId; + this.assignments = assignments; + this.avroSchema = AVRO_SCHEMA; + } + + public UUID commitId() { + return commitId; + } + + public List assignments() { + return assignments; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + @SuppressWarnings("unchecked") + public void put(int i, Object v) { + switch (i) { + case 0: + this.commitId = (UUID) v; + return; + case 1: + this.assignments = (List) v; + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return commitId; + case 1: + return assignments; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitRequestPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitRequestPayload.java new file mode 100644 index 00000000..8426ed5e --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitRequestPayload.java @@ -0,0 +1,79 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import java.util.UUID; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; + +public class CommitRequestPayload implements Payload { + + private UUID commitId; + private final Schema avroSchema; + + public static final Schema AVRO_SCHEMA = + SchemaBuilder.builder() + .record(CommitRequestPayload.class.getName()) + .fields() + .name("commitId") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type(UUID_SCHEMA) + .noDefault() + .endRecord(); + + // Used by Avro reflection to instantiate this class when reading events + public CommitRequestPayload(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public CommitRequestPayload(UUID commitId) { + this.commitId = commitId; + this.avroSchema = AVRO_SCHEMA; + } + + public UUID commitId() { + return commitId; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + public void put(int i, Object v) { + switch (i) { + case 0: + this.commitId = (UUID) v; + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return commitId; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitResponsePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitResponsePayload.java new file mode 100644 index 00000000..1a63142b --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitResponsePayload.java @@ -0,0 +1,154 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.avro.DeprecatedAvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types.StructType; + +public class CommitResponsePayload implements Payload { + + private UUID commitId; + private TableName tableName; + private List dataFiles; + private List deleteFiles; + private final Schema avroSchema; + + // Used by Avro reflection to instantiate this class when reading events + public CommitResponsePayload(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public CommitResponsePayload( + StructType partitionType, + UUID commitId, + TableName tableName, + List dataFiles, + List deleteFiles) { + this.commitId = commitId; + this.tableName = tableName; + this.dataFiles = dataFiles; + this.deleteFiles = deleteFiles; + + StructType dataFileStruct = DataFile.getType(partitionType); + + Map dataFileNames = Maps.newHashMap(); + dataFileNames.put(dataFileStruct, "org.apache.iceberg.GenericDataFile"); + dataFileNames.put(partitionType, "org.apache.iceberg.PartitionData"); + Schema dataFileSchema = DeprecatedAvroSchemaUtil.convert(dataFileStruct, dataFileNames); + + Map deleteFileNames = Maps.newHashMap(); + deleteFileNames.put(dataFileStruct, "org.apache.iceberg.GenericDeleteFile"); + deleteFileNames.put(partitionType, "org.apache.iceberg.PartitionData"); + Schema deleteFileSchema = DeprecatedAvroSchemaUtil.convert(dataFileStruct, deleteFileNames); + + this.avroSchema = + SchemaBuilder.builder() + .record(getClass().getName()) + .fields() + .name("commitId") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type(UUID_SCHEMA) + .noDefault() + .name("tableName") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type(TableName.AVRO_SCHEMA) + .noDefault() + .name("dataFiles") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .array() + .items(dataFileSchema) + .noDefault() + .name("deleteFiles") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .array() + .items(deleteFileSchema) + .noDefault() + .endRecord(); + } + + public UUID commitId() { + return commitId; + } + + public TableName tableName() { + return tableName; + } + + public List dataFiles() { + return dataFiles; + } + + public List deleteFiles() { + return deleteFiles; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + @SuppressWarnings("unchecked") + public void put(int i, Object v) { + switch (i) { + case 0: + this.commitId = (UUID) v; + return; + case 1: + this.tableName = (TableName) v; + return; + case 2: + this.dataFiles = (List) v; + return; + case 3: + this.deleteFiles = (List) v; + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return commitId; + case 1: + return tableName; + case 2: + return dataFiles; + case 3: + return deleteFiles; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitTablePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitTablePayload.java new file mode 100644 index 00000000..37c89805 --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitTablePayload.java @@ -0,0 +1,129 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import java.util.UUID; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; + +public class CommitTablePayload implements Payload { + + private UUID commitId; + private TableName tableName; + private Long snapshotId; + private Long vtts; + private final Schema avroSchema; + + private static final Schema AVRO_SCHEMA = + SchemaBuilder.builder() + .record(CommitTablePayload.class.getName()) + .fields() + .name("commitId") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type(UUID_SCHEMA) + .noDefault() + .name("tableName") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type(TableName.AVRO_SCHEMA) + .noDefault() + .name("snapshotId") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .longType() + .noDefault() + .name("vtts") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .longType() + .noDefault() + .endRecord(); + + // Used by Avro reflection to instantiate this class when reading events + public CommitTablePayload(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public CommitTablePayload(UUID commitId, TableName tableName, Long snapshotId, Long vtts) { + this.commitId = commitId; + this.tableName = tableName; + this.snapshotId = snapshotId; + this.vtts = vtts; + this.avroSchema = AVRO_SCHEMA; + } + + public UUID commitId() { + return commitId; + } + + public TableName tableName() { + return tableName; + } + + public Long snapshotId() { + return snapshotId; + } + + public Long vtts() { + return vtts; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + @SuppressWarnings("unchecked") + public void put(int i, Object v) { + switch (i) { + case 0: + this.commitId = (UUID) v; + return; + case 1: + this.tableName = (TableName) v; + return; + case 2: + this.snapshotId = (Long) v; + return; + case 3: + this.vtts = (Long) v; + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return commitId; + case 1: + return tableName; + case 2: + return snapshotId; + case 3: + return vtts; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Element.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Element.java new file mode 100644 index 00000000..0f39fbdd --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Element.java @@ -0,0 +1,37 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.specific.SpecificData.SchemaConstructable; +import org.apache.iceberg.avro.DeprecatedAvroSchemaUtil; + +public interface Element extends IndexedRecord, SchemaConstructable { + // this is required by Iceberg's Avro deserializer to check for special metadata + // fields, but we aren't using any + String DUMMY_FIELD_ID = "-1"; + + String FIELD_ID_PROP = DeprecatedAvroSchemaUtil.FIELD_ID_PROP; + + Schema UUID_SCHEMA = + LogicalTypes.uuid().addToSchema(SchemaBuilder.builder().fixed("uuid").size(16)); +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Event.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Event.java new file mode 100644 index 00000000..dcb4991e --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Event.java @@ -0,0 +1,174 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Map; +import java.util.UUID; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.iceberg.avro.DeprecatedAvroEncoderUtil; +import org.apache.iceberg.common.DynFields; +import org.apache.iceberg.data.avro.DecoderResolver; + +public class Event implements Element { + + private UUID id; + private EventType type; + private Long timestamp; + private String groupId; + private Payload payload; + private final Schema avroSchema; + + private static final ThreadLocal> DECODER_CACHES = decoderCaches(); + + public static byte[] encode(Event event) { + try { + return DeprecatedAvroEncoderUtil.encode(event, event.getSchema()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public static Event decode(byte[] bytes) { + try { + Event event = DeprecatedAvroEncoderUtil.decode(bytes); + // workaround for memory leak, until this is addressed upstream + DECODER_CACHES.get().clear(); + return event; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + // Used by Avro reflection to instantiate this class when reading events + public Event(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public Event(String groupId, EventType type, Payload payload) { + this.id = UUID.randomUUID(); + this.type = type; + this.timestamp = System.currentTimeMillis(); + this.groupId = groupId; + this.payload = payload; + + this.avroSchema = + SchemaBuilder.builder() + .record(getClass().getName()) + .fields() + .name("id") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type(UUID_SCHEMA) + .noDefault() + .name("type") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .intType() + .noDefault() + .name("timestamp") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .longType() + .noDefault() + .name("payload") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type(payload.getSchema()) + .noDefault() + .name("groupId") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .stringType() + .noDefault() + .endRecord(); + } + + public UUID id() { + return id; + } + + public EventType type() { + return type; + } + + public Long timestamp() { + return timestamp; + } + + public Payload payload() { + return payload; + } + + public String groupId() { + return groupId; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + public void put(int i, Object v) { + switch (i) { + case 0: + this.id = (UUID) v; + return; + case 1: + this.type = v == null ? null : EventType.values()[(Integer) v]; + return; + case 2: + this.timestamp = (Long) v; + return; + case 3: + this.payload = (Payload) v; + return; + case 4: + this.groupId = v == null ? null : v.toString(); + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return id; + case 1: + return type == null ? null : type.id(); + case 2: + return timestamp; + case 3: + return payload; + case 4: + return groupId; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } + + @SuppressWarnings("unchecked") + private static ThreadLocal> decoderCaches() { + return (ThreadLocal>) + DynFields.builder().hiddenImpl(DecoderResolver.class, "DECODER_CACHES").buildStatic().get(); + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/EventType.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/EventType.java new file mode 100644 index 00000000..edea00bc --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/EventType.java @@ -0,0 +1,37 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +public enum EventType { + COMMIT_REQUEST(0), + COMMIT_RESPONSE(1), + COMMIT_READY(2), + COMMIT_TABLE(3), + COMMIT_COMPLETE(4); + + private final int id; + + EventType(int id) { + this.id = id; + } + + public int id() { + return id; + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Payload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Payload.java new file mode 100644 index 00000000..d2adf374 --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Payload.java @@ -0,0 +1,21 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +public interface Payload extends Element {} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TableName.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TableName.java new file mode 100644 index 00000000..9710c1da --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TableName.java @@ -0,0 +1,108 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import static java.util.stream.Collectors.toList; + +import java.util.Arrays; +import java.util.List; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.util.Utf8; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; + +public class TableName implements Element { + + private List namespace; + private String name; + private final Schema avroSchema; + + public static final Schema AVRO_SCHEMA = + SchemaBuilder.builder() + .record(TableName.class.getName()) + .fields() + .name("namespace") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .array() + .items() + .stringType() + .noDefault() + .name("name") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .stringType() + .noDefault() + .endRecord(); + + public static TableName of(TableIdentifier tableIdentifier) { + return new TableName( + Arrays.asList(tableIdentifier.namespace().levels()), tableIdentifier.name()); + } + + // Used by Avro reflection to instantiate this class when reading events + public TableName(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public TableName(List namespace, String name) { + this.namespace = namespace; + this.name = name; + this.avroSchema = AVRO_SCHEMA; + } + + public TableIdentifier toIdentifier() { + Namespace icebergNamespace = Namespace.of(namespace.toArray(new String[0])); + return TableIdentifier.of(icebergNamespace, name); + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + @SuppressWarnings("unchecked") + public void put(int i, Object v) { + switch (i) { + case 0: + this.namespace = + v == null ? null : ((List) v).stream().map(Utf8::toString).collect(toList()); + return; + case 1: + this.name = v == null ? null : v.toString(); + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return namespace; + case 1: + return name; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } +} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TopicPartitionOffset.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TopicPartitionOffset.java new file mode 100644 index 00000000..c3cc2f64 --- /dev/null +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TopicPartitionOffset.java @@ -0,0 +1,129 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; + +public class TopicPartitionOffset implements Element { + + private String topic; + private Integer partition; + private Long offset; + private Long timestamp; + private final Schema avroSchema; + + public static final Schema AVRO_SCHEMA = + SchemaBuilder.builder() + .record(TopicPartitionOffset.class.getName()) + .fields() + .name("topic") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .stringType() + .noDefault() + .name("partition") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .intType() + .noDefault() + .name("offset") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .longType() + .noDefault() + .name("timestamp") + .prop(FIELD_ID_PROP, DUMMY_FIELD_ID) + .type() + .nullable() + .longType() + .noDefault() + .endRecord(); + + // Used by Avro reflection to instantiate this class when reading events + public TopicPartitionOffset(Schema avroSchema) { + this.avroSchema = avroSchema; + } + + public TopicPartitionOffset(String topic, int partition, Long offset, Long timestamp) { + this.topic = topic; + this.partition = partition; + this.offset = offset; + this.timestamp = timestamp; + this.avroSchema = AVRO_SCHEMA; + } + + public String topic() { + return topic; + } + + public Integer partition() { + return partition; + } + + public Long offset() { + return offset; + } + + public Long timestamp() { + return timestamp; + } + + @Override + public Schema getSchema() { + return avroSchema; + } + + @Override + public void put(int i, Object v) { + switch (i) { + case 0: + this.topic = v == null ? null : v.toString(); + return; + case 1: + this.partition = (Integer) v; + return; + case 2: + this.offset = (Long) v; + return; + case 3: + this.timestamp = (Long) v; + return; + default: + // ignore the object, it must be from a newer version of the format + } + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return topic; + case 1: + return partition; + case 2: + return offset; + case 3: + return timestamp; + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } +} diff --git a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroEncoderUtil.java b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroEncoderUtil.java new file mode 100644 index 00000000..25d9cc7e --- /dev/null +++ b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroEncoderUtil.java @@ -0,0 +1,89 @@ +/* + * 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.avro; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.EncoderFactory; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class DeprecatedAvroEncoderUtil { + + private DeprecatedAvroEncoderUtil() {} + + static { + LogicalTypes.register(LogicalMap.NAME, schema -> LogicalMap.get()); + } + + private static final byte[] MAGIC_BYTES = new byte[] {(byte) 0xC2, (byte) 0x01}; + + public static byte[] encode(T datum, Schema avroSchema) throws IOException { + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + DataOutputStream dataOut = new DataOutputStream(out); + + // Write the magic bytes + dataOut.write(MAGIC_BYTES); + + // Write avro schema + dataOut.writeUTF(avroSchema.toString()); + + // Encode the datum with avro schema. + BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null); + DatumWriter writer = new GenericAvroWriter<>(avroSchema); + writer.write(datum, encoder); + encoder.flush(); + + return out.toByteArray(); + } + } + + public static T decode(byte[] data) throws IOException { + try (ByteArrayInputStream in = new ByteArrayInputStream(data, 0, data.length)) { + DataInputStream dataInput = new DataInputStream(in); + + // Read the magic bytes + byte header0 = dataInput.readByte(); + byte header1 = dataInput.readByte(); + Preconditions.checkState( + header0 == MAGIC_BYTES[0] && header1 == MAGIC_BYTES[1], + "Unrecognized header bytes: 0x%02X 0x%02X", + header0, + header1); + + // Read avro schema + Schema avroSchema = new Schema.Parser().parse(dataInput.readUTF()); + + // Decode the datum with the parsed avro schema. + BinaryDecoder binaryDecoder = DecoderFactory.get().binaryDecoder(in, null); + DatumReader reader = new DeprecatedGenericAvroReader<>(avroSchema); + reader.setSchema(avroSchema); + return reader.read(null, binaryDecoder); + } + } +} diff --git a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroSchemaUtil.java b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroSchemaUtil.java new file mode 100644 index 00000000..1188e6d5 --- /dev/null +++ b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroSchemaUtil.java @@ -0,0 +1,480 @@ +/* + * 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.avro; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.iceberg.mapping.MappedField; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +public class DeprecatedAvroSchemaUtil { + + private DeprecatedAvroSchemaUtil() {} + + // Original Iceberg field name corresponding to a sanitized Avro name + public static final String ICEBERG_FIELD_NAME_PROP = "iceberg-field-name"; + public static final String FIELD_ID_PROP = "field-id"; + public static final String KEY_ID_PROP = "key-id"; + public static final String VALUE_ID_PROP = "value-id"; + public static final String ELEMENT_ID_PROP = "element-id"; + public static final String ADJUST_TO_UTC_PROP = "adjust-to-utc"; + + private static final Schema NULL = Schema.create(Schema.Type.NULL); + private static final Schema.Type MAP = Schema.Type.MAP; + private static final Schema.Type ARRAY = Schema.Type.ARRAY; + private static final Schema.Type UNION = Schema.Type.UNION; + private static final Schema.Type RECORD = Schema.Type.RECORD; + + public static Schema convert(org.apache.iceberg.Schema schema, String tableName) { + return convert(schema, ImmutableMap.of(schema.asStruct(), tableName)); + } + + public static Schema convert( + org.apache.iceberg.Schema schema, Map names) { + return TypeUtil.visit(schema, new DeprecatedTypeToSchema(names)); + } + + public static Schema convert(Type type) { + return convert(type, ImmutableMap.of()); + } + + public static Schema convert(Types.StructType type, String name) { + return convert(type, ImmutableMap.of(type, name)); + } + + public static Schema convert(Type type, Map names) { + return TypeUtil.visit(type, new DeprecatedTypeToSchema(names)); + } + + public static Type convert(Schema schema) { + return AvroSchemaVisitor.visit(schema, new SchemaToType(schema)); + } + + public static org.apache.iceberg.Schema toIceberg(Schema schema) { + final List fields = convert(schema).asNestedType().asStructType().fields(); + return new org.apache.iceberg.Schema(fields); + } + + static boolean hasIds(Schema schema) { + return AvroCustomOrderSchemaVisitor.visit(schema, new HasIds()); + } + + /** + * Check if any of the nodes in a given avro schema is missing an ID + * + *

To have an ID for a node: + * + *

    + *
  • a field node under struct (record) schema should have {@link #FIELD_ID_PROP} property + *
  • an element node under list (array) schema should have {@link #ELEMENT_ID_PROP} property + *
  • a pair of key and value node under map schema should have {@link #KEY_ID_PROP} and {@link + * #VALUE_ID_PROP} respectively + *
  • a primitive node is not assigned any ID properties + *
+ * + *

+ * + * @param schema an Avro Schema + * @return true if any of the nodes of the given Avro Schema is missing an ID property, false + * otherwise + */ + static boolean missingIds(Schema schema) { + return AvroCustomOrderSchemaVisitor.visit(schema, new MissingIds()); + } + + public static Map convertTypes(Types.StructType type, String name) { + DeprecatedTypeToSchema converter = new DeprecatedTypeToSchema(ImmutableMap.of(type, name)); + TypeUtil.visit(type, converter); + return ImmutableMap.copyOf(converter.getConversionMap()); + } + + public static Schema pruneColumns( + Schema schema, Set selectedIds, NameMapping nameMapping) { + return new PruneColumns(selectedIds, nameMapping).rootSchema(schema); + } + + public static Schema buildAvroProjection( + Schema schema, org.apache.iceberg.Schema expected, Map renames) { + return AvroCustomOrderSchemaVisitor.visit(schema, new BuildAvroProjection(expected, renames)); + } + + public static boolean isTimestamptz(Schema schema) { + LogicalType logicalType = schema.getLogicalType(); + if (logicalType instanceof LogicalTypes.TimestampMillis + || logicalType instanceof LogicalTypes.TimestampMicros) { + // timestamptz is adjusted to UTC + Object value = schema.getObjectProp(ADJUST_TO_UTC_PROP); + + if (value == null) { + // not all avro timestamp logical types will have the adjust_to_utc prop, default to + // timestamp without timezone + return false; + } else if (value instanceof Boolean) { + return (Boolean) value; + } else if (value instanceof String) { + return Boolean.parseBoolean((String) value); + } + } + + return false; + } + + public static boolean isOptionSchema(Schema schema) { + if (schema.getType() == UNION && schema.getTypes().size() == 2) { + if (schema.getTypes().get(0).getType() == Schema.Type.NULL) { + return true; + } else if (schema.getTypes().get(1).getType() == Schema.Type.NULL) { + return true; + } + } + return false; + } + + static Schema toOption(Schema schema) { + if (schema.getType() == UNION) { + Preconditions.checkArgument( + isOptionSchema(schema), "Union schemas are not supported: %s", schema); + return schema; + } else { + return Schema.createUnion(NULL, schema); + } + } + + static Schema fromOption(Schema schema) { + Preconditions.checkArgument( + schema.getType() == UNION, "Expected union schema but was passed: %s", schema); + Preconditions.checkArgument( + schema.getTypes().size() == 2, "Expected optional schema, but was passed: %s", schema); + if (schema.getTypes().get(0).getType() == Schema.Type.NULL) { + return schema.getTypes().get(1); + } else { + return schema.getTypes().get(0); + } + } + + static Schema fromOptions(List options) { + Preconditions.checkArgument( + options.size() == 2, "Expected two schemas, but was passed: %s options", options.size()); + if (options.get(0).getType() == Schema.Type.NULL) { + return options.get(1); + } else { + return options.get(0); + } + } + + public static boolean isKeyValueSchema(Schema schema) { + return schema.getType() == RECORD && schema.getFields().size() == 2; + } + + static Schema createMap(int keyId, Schema keySchema, int valueId, Schema valueSchema) { + String keyValueName = "k" + keyId + "_v" + valueId; + + Schema.Field keyField = new Schema.Field("key", keySchema, null, (Object) null); + keyField.addProp(FIELD_ID_PROP, keyId); + + Schema.Field valueField = + new Schema.Field( + "value", + valueSchema, + null, + isOptionSchema(valueSchema) ? JsonProperties.NULL_VALUE : null); + valueField.addProp(FIELD_ID_PROP, valueId); + + return LogicalMap.get() + .addToSchema( + Schema.createArray( + Schema.createRecord( + keyValueName, null, null, false, ImmutableList.of(keyField, valueField)))); + } + + static Schema createProjectionMap( + String recordName, + int keyId, + String keyName, + Schema keySchema, + int valueId, + String valueName, + Schema valueSchema) { + String keyValueName = "k" + keyId + "_v" + valueId; + + Schema.Field keyField = new Schema.Field("key", keySchema, null, (Object) null); + if (!"key".equals(keyName)) { + keyField.addAlias(keyName); + } + keyField.addProp(FIELD_ID_PROP, keyId); + + Schema.Field valueField = + new Schema.Field( + "value", + valueSchema, + null, + isOptionSchema(valueSchema) ? JsonProperties.NULL_VALUE : null); + valueField.addProp(FIELD_ID_PROP, valueId); + if (!"value".equals(valueName)) { + valueField.addAlias(valueName); + } + + Schema keyValueRecord = + Schema.createRecord( + keyValueName, null, null, false, ImmutableList.of(keyField, valueField)); + if (!keyValueName.equals(recordName)) { + keyValueRecord.addAlias(recordName); + } + + return LogicalMap.get().addToSchema(Schema.createArray(keyValueRecord)); + } + + private static Integer getId(Schema schema, String propertyName) { + Integer id = getId(schema, propertyName, null, null); + Preconditions.checkNotNull(id, "Missing expected '%s' property", propertyName); + return id; + } + + private static Integer getId( + Schema schema, String propertyName, NameMapping nameMapping, List names) { + if (schema.getType() == UNION) { + return getId(fromOption(schema), propertyName, nameMapping, names); + } + + Object id = schema.getObjectProp(propertyName); + if (id != null) { + return toInt(id); + } else if (nameMapping != null) { + MappedField mappedField = nameMapping.find(names); + if (mappedField != null) { + return mappedField.id(); + } + } + + return null; + } + + static boolean hasProperty(Schema schema, String propertyName) { + if (schema.getType() == UNION) { + return hasProperty(fromOption(schema), propertyName); + } + return schema.getObjectProp(propertyName) != null; + } + + public static int getKeyId(Schema schema) { + Preconditions.checkArgument( + schema.getType() == MAP, "Cannot get map key id for non-map schema: %s", schema); + return getId(schema, KEY_ID_PROP); + } + + static Integer getKeyId( + Schema schema, NameMapping nameMapping, Iterable parentFieldNames) { + Preconditions.checkArgument( + schema.getType() == MAP, "Cannot get map key id for non-map schema: %s", schema); + List names = Lists.newArrayList(parentFieldNames); + names.add("key"); + return getId(schema, KEY_ID_PROP, nameMapping, names); + } + + public static int getValueId(Schema schema) { + Preconditions.checkArgument( + schema.getType() == MAP, "Cannot get map value id for non-map schema: %s", schema); + return getId(schema, VALUE_ID_PROP); + } + + static Integer getValueId( + Schema schema, NameMapping nameMapping, Iterable parentFieldNames) { + Preconditions.checkArgument( + schema.getType() == MAP, "Cannot get map value id for non-map schema: %s", schema); + List names = Lists.newArrayList(parentFieldNames); + names.add("value"); + return getId(schema, VALUE_ID_PROP, nameMapping, names); + } + + public static int getElementId(Schema schema) { + Preconditions.checkArgument( + schema.getType() == ARRAY, "Cannot get array element id for non-array schema: %s", schema); + return getId(schema, ELEMENT_ID_PROP); + } + + static Integer getElementId( + Schema schema, NameMapping nameMapping, Iterable parentFieldNames) { + Preconditions.checkArgument( + schema.getType() == ARRAY, "Cannot get array element id for non-array schema: %s", schema); + List names = Lists.newArrayList(parentFieldNames); + names.add("element"); + return getId(schema, ELEMENT_ID_PROP, nameMapping, names); + } + + public static int getFieldId(Schema.Field field) { + Integer id = getFieldId(field, null, null); + Preconditions.checkNotNull(id, "Missing expected '%s' property", FIELD_ID_PROP); + return id; + } + + static Integer getFieldId( + Schema.Field field, NameMapping nameMapping, Iterable parentFieldNames) { + Object id = field.getObjectProp(FIELD_ID_PROP); + if (id != null) { + return toInt(id); + } else if (nameMapping != null) { + List names = Lists.newArrayList(parentFieldNames); + names.add(field.name()); + MappedField mappedField = nameMapping.find(names); + if (mappedField != null) { + return mappedField.id(); + } + } + + return null; + } + + public static boolean hasFieldId(Schema.Field field) { + return field.getObjectProp(FIELD_ID_PROP) != null; + } + + private static int toInt(Object value) { + if (value instanceof Number) { + return ((Number) value).intValue(); + } else if (value instanceof String) { + return Integer.parseInt((String) value); + } + + throw new UnsupportedOperationException("Cannot coerce value to int: " + value); + } + + static Schema copyRecord(Schema record, List newFields, String newName) { + Schema copy; + if (newName != null) { + copy = Schema.createRecord(newName, record.getDoc(), null, record.isError(), newFields); + // the namespace is defaulted to the record's namespace if it is null, which causes renames + // without the namespace to fail. using "" instead of null changes this behavior to match the + // original schema. + copy.addAlias(record.getName(), record.getNamespace() == null ? "" : record.getNamespace()); + } else { + copy = + Schema.createRecord( + record.getName(), + record.getDoc(), + record.getNamespace(), + record.isError(), + newFields); + } + + for (Map.Entry prop : record.getObjectProps().entrySet()) { + copy.addProp(prop.getKey(), prop.getValue()); + } + + return copy; + } + + static Schema.Field copyField(Schema.Field field, Schema newSchema, String newName) { + Schema.Field copy = + new Schema.Field(newName, newSchema, field.doc(), field.defaultVal(), field.order()); + + for (Map.Entry prop : field.getObjectProps().entrySet()) { + copy.addProp(prop.getKey(), prop.getValue()); + } + + if (!newName.equals(field.name())) { + copy.addAlias(field.name()); + } + + return copy; + } + + static Schema replaceElement(Schema array, Schema elementSchema) { + Preconditions.checkArgument( + array.getType() == ARRAY, "Cannot invoke replaceElement on non array schema: %s", array); + Schema copy = Schema.createArray(elementSchema); + for (Map.Entry prop : array.getObjectProps().entrySet()) { + copy.addProp(prop.getKey(), prop.getValue()); + } + return copy; + } + + static Schema replaceValue(Schema map, Schema valueSchema) { + Preconditions.checkArgument( + map.getType() == MAP, "Cannot invoke replaceValue on non map schema: %s", map); + Schema copy = Schema.createMap(valueSchema); + for (Map.Entry prop : map.getObjectProps().entrySet()) { + copy.addProp(prop.getKey(), prop.getValue()); + } + return copy; + } + + public static String makeCompatibleName(String name) { + if (!validAvroName(name)) { + return sanitize(name); + } + return name; + } + + static boolean validAvroName(String name) { + int length = name.length(); + Preconditions.checkArgument(length > 0, "Empty name"); + char first = name.charAt(0); + if (!(Character.isLetter(first) || first == '_')) { + return false; + } + + for (int i = 1; i < length; i++) { + char character = name.charAt(i); + if (!(Character.isLetterOrDigit(character) || character == '_')) { + return false; + } + } + return true; + } + + static String sanitize(String name) { + int length = name.length(); + StringBuilder sb = new StringBuilder(name.length()); + char first = name.charAt(0); + if (!(Character.isLetter(first) || first == '_')) { + sb.append(sanitize(first)); + } else { + sb.append(first); + } + + for (int i = 1; i < length; i++) { + char character = name.charAt(i); + if (!(Character.isLetterOrDigit(character) || character == '_')) { + sb.append(sanitize(character)); + } else { + sb.append(character); + } + } + return sb.toString(); + } + + private static String sanitize(char character) { + if (Character.isDigit(character)) { + return "_" + character; + } + return "_x" + Integer.toHexString(character).toUpperCase(); + } +} diff --git a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedGenericAvroReader.java b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedGenericAvroReader.java new file mode 100644 index 00000000..82446070 --- /dev/null +++ b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedGenericAvroReader.java @@ -0,0 +1,186 @@ +/* + * 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.avro; + +import java.io.IOException; +import java.util.List; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.iceberg.common.DynClasses; +import org.apache.iceberg.data.avro.DecoderResolver; + +public class DeprecatedGenericAvroReader implements DatumReader, SupportsRowPosition { + + private final Schema readSchema; + private ClassLoader loader = Thread.currentThread().getContextClassLoader(); + private Schema fileSchema = null; + private ValueReader reader = null; + + public static GenericAvroReader create(Schema schema) { + return new GenericAvroReader<>(schema); + } + + DeprecatedGenericAvroReader(Schema readSchema) { + this.readSchema = readSchema; + } + + @SuppressWarnings("unchecked") + private void initReader() { + this.reader = (ValueReader) AvroSchemaVisitor.visit(readSchema, new ReadBuilder(loader)); + } + + @Override + public void setSchema(Schema schema) { + this.fileSchema = Schema.applyAliases(schema, readSchema); + initReader(); + } + + public void setClassLoader(ClassLoader newClassLoader) { + this.loader = newClassLoader; + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + @Override + public T read(T reuse, Decoder decoder) throws IOException { + return DecoderResolver.resolveAndRead(decoder, readSchema, fileSchema, reader, reuse); + } + + private static class ReadBuilder extends AvroSchemaVisitor> { + private final ClassLoader loader; + + private ReadBuilder(ClassLoader loader) { + this.loader = loader; + } + + @Override + @SuppressWarnings("unchecked") + public ValueReader record(Schema record, List names, List> fields) { + try { + Class recordClass = + DynClasses.builder().loader(loader).impl(record.getFullName()).buildChecked(); + if (IndexedRecord.class.isAssignableFrom(recordClass)) { + return ValueReaders.record(fields, (Class) recordClass, record); + } + + return ValueReaders.record(fields, record); + + } catch (ClassNotFoundException e) { + return ValueReaders.record(fields, record); + } + } + + @Override + public ValueReader union(Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader array(Schema array, ValueReader elementReader) { + if (array.getLogicalType() instanceof LogicalMap) { + ValueReaders.StructReader keyValueReader = (ValueReaders.StructReader) elementReader; + ValueReader keyReader = keyValueReader.reader(0); + ValueReader valueReader = keyValueReader.reader(1); + + if (keyReader == ValueReaders.utf8s()) { + return ValueReaders.arrayMap(ValueReaders.strings(), valueReader); + } + + return ValueReaders.arrayMap(keyReader, valueReader); + } + + return ValueReaders.array(elementReader); + } + + @Override + public ValueReader map(Schema map, ValueReader valueReader) { + return ValueReaders.map(ValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + // Spark uses the same representation + return ValueReaders.ints(); + + case "time-micros": + return ValueReaders.longs(); + + case "timestamp-millis": + // adjust to microseconds + ValueReader longs = ValueReaders.longs(); + return (ValueReader) (decoder, ignored) -> longs.read(decoder, null) * 1000L; + + case "timestamp-micros": + // Spark uses the same representation + return ValueReaders.longs(); + + case "decimal": + return ValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + ((LogicalTypes.Decimal) logicalType).getScale()); + + case "uuid": + return ValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return ValueReaders.utf8s(); + case FIXED: + return ValueReaders.fixed(primitive); + case BYTES: + return ValueReaders.byteBuffers(); + case ENUM: + return ValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedTypeToSchema.java b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedTypeToSchema.java new file mode 100644 index 00000000..cebcaa17 --- /dev/null +++ b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedTypeToSchema.java @@ -0,0 +1,245 @@ +/* + * 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.avro; + +import java.util.Deque; +import java.util.List; +import java.util.Map; +import org.apache.avro.JsonProperties; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +class DeprecatedTypeToSchema extends TypeUtil.SchemaVisitor { + private static final Schema BOOLEAN_SCHEMA = Schema.create(Schema.Type.BOOLEAN); + private static final Schema INTEGER_SCHEMA = Schema.create(Schema.Type.INT); + private static final Schema LONG_SCHEMA = Schema.create(Schema.Type.LONG); + private static final Schema FLOAT_SCHEMA = Schema.create(Schema.Type.FLOAT); + private static final Schema DOUBLE_SCHEMA = Schema.create(Schema.Type.DOUBLE); + private static final Schema DATE_SCHEMA = + LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)); + private static final Schema TIME_SCHEMA = + LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG)); + private static final Schema TIMESTAMP_SCHEMA = + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + private static final Schema TIMESTAMPTZ_SCHEMA = + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + private static final Schema STRING_SCHEMA = Schema.create(Schema.Type.STRING); + private static final Schema UUID_SCHEMA = + LogicalTypes.uuid().addToSchema(Schema.createFixed("uuid_fixed", null, null, 16)); + private static final Schema BINARY_SCHEMA = Schema.create(Schema.Type.BYTES); + + static { + TIMESTAMP_SCHEMA.addProp(AvroSchemaUtil.ADJUST_TO_UTC_PROP, false); + TIMESTAMPTZ_SCHEMA.addProp(AvroSchemaUtil.ADJUST_TO_UTC_PROP, true); + } + + private final Deque fieldIds = Lists.newLinkedList(); + private final Map results = Maps.newHashMap(); + private final Map names; + + DeprecatedTypeToSchema(Map names) { + this.names = names; + } + + Map getConversionMap() { + return results; + } + + @Override + public Schema schema(org.apache.iceberg.Schema schema, Schema structSchema) { + return structSchema; + } + + @Override + public void beforeField(Types.NestedField field) { + fieldIds.push(field.fieldId()); + } + + @Override + public void afterField(Types.NestedField field) { + fieldIds.pop(); + } + + @Override + public Schema struct(Types.StructType struct, List fieldSchemas) { + Schema recordSchema = results.get(struct); + if (recordSchema != null) { + return recordSchema; + } + + String recordName = names.get(struct); + if (recordName == null) { + recordName = "r" + fieldIds.peek(); + } + + List structFields = struct.fields(); + List fields = Lists.newArrayListWithExpectedSize(fieldSchemas.size()); + for (int i = 0; i < structFields.size(); i += 1) { + Types.NestedField structField = structFields.get(i); + String origFieldName = structField.name(); + boolean isValidFieldName = AvroSchemaUtil.validAvroName(origFieldName); + String fieldName = isValidFieldName ? origFieldName : AvroSchemaUtil.sanitize(origFieldName); + Schema.Field field = + new Schema.Field( + fieldName, + fieldSchemas.get(i), + structField.doc(), + structField.isOptional() ? JsonProperties.NULL_VALUE : null); + if (!isValidFieldName) { + field.addProp(AvroSchemaUtil.ICEBERG_FIELD_NAME_PROP, origFieldName); + } + field.addProp(AvroSchemaUtil.FIELD_ID_PROP, structField.fieldId()); + fields.add(field); + } + + recordSchema = Schema.createRecord(recordName, null, null, false, fields); + + results.put(struct, recordSchema); + + return recordSchema; + } + + @Override + public Schema field(Types.NestedField field, Schema fieldSchema) { + if (field.isOptional()) { + return AvroSchemaUtil.toOption(fieldSchema); + } else { + return fieldSchema; + } + } + + @Override + public Schema list(Types.ListType list, Schema elementSchema) { + Schema listSchema = results.get(list); + if (listSchema != null) { + return listSchema; + } + + if (list.isElementOptional()) { + listSchema = Schema.createArray(AvroSchemaUtil.toOption(elementSchema)); + } else { + listSchema = Schema.createArray(elementSchema); + } + + listSchema.addProp(AvroSchemaUtil.ELEMENT_ID_PROP, list.elementId()); + + results.put(list, listSchema); + + return listSchema; + } + + @Override + public Schema map(Types.MapType map, Schema keySchema, Schema valueSchema) { + Schema mapSchema = results.get(map); + if (mapSchema != null) { + return mapSchema; + } + + if (keySchema.getType() == Schema.Type.STRING) { + // if the map has string keys, use Avro's map type + mapSchema = + Schema.createMap( + map.isValueOptional() ? AvroSchemaUtil.toOption(valueSchema) : valueSchema); + mapSchema.addProp(AvroSchemaUtil.KEY_ID_PROP, map.keyId()); + mapSchema.addProp(AvroSchemaUtil.VALUE_ID_PROP, map.valueId()); + + } else { + mapSchema = + AvroSchemaUtil.createMap( + map.keyId(), + keySchema, + map.valueId(), + map.isValueOptional() ? AvroSchemaUtil.toOption(valueSchema) : valueSchema); + } + + results.put(map, mapSchema); + + return mapSchema; + } + + @Override + public Schema primitive(Type.PrimitiveType primitive) { + Schema primitiveSchema; + switch (primitive.typeId()) { + case BOOLEAN: + primitiveSchema = BOOLEAN_SCHEMA; + break; + case INTEGER: + primitiveSchema = INTEGER_SCHEMA; + break; + case LONG: + primitiveSchema = LONG_SCHEMA; + break; + case FLOAT: + primitiveSchema = FLOAT_SCHEMA; + break; + case DOUBLE: + primitiveSchema = DOUBLE_SCHEMA; + break; + case DATE: + primitiveSchema = DATE_SCHEMA; + break; + case TIME: + primitiveSchema = TIME_SCHEMA; + break; + case TIMESTAMP: + if (((Types.TimestampType) primitive).shouldAdjustToUTC()) { + primitiveSchema = TIMESTAMPTZ_SCHEMA; + } else { + primitiveSchema = TIMESTAMP_SCHEMA; + } + break; + case STRING: + primitiveSchema = STRING_SCHEMA; + break; + case UUID: + primitiveSchema = UUID_SCHEMA; + break; + case FIXED: + Types.FixedType fixed = (Types.FixedType) primitive; + primitiveSchema = Schema.createFixed("fixed_" + fixed.length(), null, null, fixed.length()); + break; + case BINARY: + primitiveSchema = BINARY_SCHEMA; + break; + case DECIMAL: + Types.DecimalType decimal = (Types.DecimalType) primitive; + primitiveSchema = + LogicalTypes.decimal(decimal.precision(), decimal.scale()) + .addToSchema( + Schema.createFixed( + "decimal_" + decimal.precision() + "_" + decimal.scale(), + null, + null, + TypeUtil.decimalRequiredBytes(decimal.precision()))); + break; + default: + throw new UnsupportedOperationException("Unsupported type ID: " + primitive.typeId()); + } + + results.put(primitive, primitiveSchema); + + return primitiveSchema; + } +} diff --git a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/deprecated/EventSerializationTest.java b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/deprecated/EventSerializationTest.java new file mode 100644 index 00000000..5e8b6e23 --- /dev/null +++ b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/deprecated/EventSerializationTest.java @@ -0,0 +1,132 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.Collections; +import java.util.UUID; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types.StructType; +import org.junit.jupiter.api.Test; + +public class EventSerializationTest { + + @Test + public void testCommitRequestSerialization() { + UUID commitId = UUID.randomUUID(); + Event event = + new Event("cg-connector", EventType.COMMIT_REQUEST, new CommitRequestPayload(commitId)); + + byte[] data = Event.encode(event); + Event result = Event.decode(data); + + assertThat(result.type()).isEqualTo(event.type()); + CommitRequestPayload payload = (CommitRequestPayload) result.payload(); + assertThat(payload.commitId()).isEqualTo(commitId); + } + + @Test + public void testCommitResponseSerialization() { + UUID commitId = UUID.randomUUID(); + Event event = + new Event( + "cg-connector", + EventType.COMMIT_RESPONSE, + new CommitResponsePayload( + StructType.of(), + commitId, + new TableName(Collections.singletonList("db"), "tbl"), + Arrays.asList(EventTestUtil.createDataFile(), EventTestUtil.createDataFile()), + Arrays.asList(EventTestUtil.createDeleteFile(), EventTestUtil.createDeleteFile()))); + + byte[] data = Event.encode(event); + Event result = Event.decode(data); + + assertThat(result.type()).isEqualTo(event.type()); + CommitResponsePayload payload = (CommitResponsePayload) result.payload(); + assertThat(payload.commitId()).isEqualTo(commitId); + assertThat(payload.tableName().toIdentifier()).isEqualTo(TableIdentifier.parse("db.tbl")); + assertThat(payload.dataFiles()).hasSize(2); + assertThat(payload.dataFiles()).allMatch(f -> f.specId() == 0); + assertThat(payload.deleteFiles()).hasSize(2); + assertThat(payload.deleteFiles()).allMatch(f -> f.specId() == 0); + } + + @Test + public void testCommitReadySerialization() { + UUID commitId = UUID.randomUUID(); + Event event = + new Event( + "cg-connector", + EventType.COMMIT_READY, + new CommitReadyPayload( + commitId, + Arrays.asList( + new TopicPartitionOffset("topic", 1, 1L, 1L), + new TopicPartitionOffset("topic", 2, null, null)))); + + byte[] data = Event.encode(event); + Event result = Event.decode(data); + + assertThat(result.type()).isEqualTo(event.type()); + CommitReadyPayload payload = (CommitReadyPayload) result.payload(); + assertThat(payload.commitId()).isEqualTo(commitId); + assertThat(payload.assignments()).hasSize(2); + assertThat(payload.assignments()).allMatch(tp -> tp.topic().equals("topic")); + } + + @Test + public void testCommitTableSerialization() { + UUID commitId = UUID.randomUUID(); + Event event = + new Event( + "cg-connector", + EventType.COMMIT_TABLE, + new CommitTablePayload( + commitId, new TableName(Collections.singletonList("db"), "tbl"), 1L, 2L)); + + byte[] data = Event.encode(event); + Event result = Event.decode(data); + + assertThat(result.type()).isEqualTo(event.type()); + CommitTablePayload payload = (CommitTablePayload) result.payload(); + assertThat(payload.commitId()).isEqualTo(commitId); + assertThat(payload.tableName().toIdentifier()).isEqualTo(TableIdentifier.parse("db.tbl")); + assertThat(payload.snapshotId()).isEqualTo(1L); + assertThat(payload.vtts()).isEqualTo(2L); + } + + @Test + public void testCommitCompleteSerialization() { + UUID commitId = UUID.randomUUID(); + Event event = + new Event( + "cg-connector", EventType.COMMIT_COMPLETE, new CommitCompletePayload(commitId, 2L)); + + byte[] data = Event.encode(event); + Event result = Event.decode(data); + + assertThat(result.type()).isEqualTo(event.type()); + CommitCompletePayload payload = (CommitCompletePayload) result.payload(); + assertThat(payload.commitId()).isEqualTo(commitId); + assertThat(payload.vtts()).isEqualTo(2L); + } +} diff --git a/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/deprecated/EventTestUtil.java b/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/deprecated/EventTestUtil.java new file mode 100644 index 00000000..09678401 --- /dev/null +++ b/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/deprecated/EventTestUtil.java @@ -0,0 +1,49 @@ +/* + * 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 io.tabular.iceberg.connect.events.deprecated; + +import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; + +public class EventTestUtil { + public static DataFile createDataFile() { + return DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(UUID.randomUUID() + ".parquet") + .withFormat(FileFormat.PARQUET) + .withFileSizeInBytes(100L) + .withRecordCount(5) + .build(); + } + + public static DeleteFile createDeleteFile() { + return FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes(1) + .withPath(UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + } + + private EventTestUtil() {} +} diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 756dbc6e..b937767f 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -1,10 +1,12 @@ dependencies { + implementation project(":iceberg-kafka-connect-events") implementation libs.bundles.iceberg implementation libs.bundles.jackson implementation libs.avro implementation libs.slf4j compileOnly libs.bundles.kafka.connect + testImplementation(testFixtures(project(":iceberg-kafka-connect-events"))) testImplementation libs.hadoop.client testImplementation libs.junit.api diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java index 4efbfd6c..5c1e9e10 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java @@ -123,7 +123,7 @@ record -> { // so increment the record offset by one controlTopicOffsets.put(record.partition(), record.offset() + 1); - Event event = AvroUtil.decode(record.value()); + Event event = EventDecoder.decode(record.value()); if (event.groupId().equals(groupId)) { LOG.debug("Received event of type: {}", event.type().name()); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java new file mode 100644 index 00000000..c97d0d48 --- /dev/null +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java @@ -0,0 +1,57 @@ +/* + * 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 io.tabular.iceberg.connect.channel; + +import io.tabular.iceberg.connect.events.deprecated.CommitRequestPayload; +import java.util.UUID; +import org.apache.avro.SchemaParseException; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.Payload; +import org.apache.iceberg.connect.events.StartCommit; + +public class EventDecoder { + + private EventDecoder() {} + + public static Event decode(byte[] value) { + try { + return AvroUtil.decode(value); + } catch (SchemaParseException exception) { + io.tabular.iceberg.connect.events.deprecated.Event event = + io.tabular.iceberg.connect.events.deprecated.Event.decode(value); + return convertLegacy(event); + } + } + + private static Event convertLegacy(io.tabular.iceberg.connect.events.deprecated.Event event) { + Payload payload = convertPayload(event.payload()); + return new Event(event.groupId(), payload); + } + + private static Payload convertPayload( + io.tabular.iceberg.connect.events.deprecated.Payload payload) { + if (payload instanceof io.tabular.iceberg.connect.events.deprecated.CommitRequestPayload) { + CommitRequestPayload pay = (CommitRequestPayload) payload; + return new StartCommit((UUID) pay.get(0)); + } else { + throw new RuntimeException("borp"); + } + } +} From 2de6c3720fcc6a72b35f8946cb8c77ee9059ee4b Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 30 Apr 2024 16:30:56 -0700 Subject: [PATCH 06/13] starting to convert records --- .../connect/events/{deprecated => }/CommitCompletePayload.java | 0 .../connect/events/{deprecated => }/CommitReadyPayload.java | 0 .../connect/events/{deprecated => }/CommitRequestPayload.java | 0 .../connect/events/{deprecated => }/CommitResponsePayload.java | 0 .../connect/events/{deprecated => }/CommitTablePayload.java | 0 .../iceberg/connect/events/{deprecated => }/Element.java | 0 .../tabular/iceberg/connect/events/{deprecated => }/Event.java | 0 .../iceberg/connect/events/{deprecated => }/EventType.java | 0 .../iceberg/connect/events/{deprecated => }/Payload.java | 0 .../iceberg/connect/events/{deprecated => }/TableName.java | 0 .../connect/events/{deprecated => }/TopicPartitionOffset.java | 0 .../connect/events/{deprecated => }/EventSerializationTest.java | 2 +- .../iceberg/connect/events/{deprecated => }/EventTestUtil.java | 0 13 files changed, 1 insertion(+), 1 deletion(-) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/CommitCompletePayload.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/CommitReadyPayload.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/CommitRequestPayload.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/CommitResponsePayload.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/CommitTablePayload.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/Element.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/Event.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/EventType.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/Payload.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/TableName.java (100%) rename kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/{deprecated => }/TopicPartitionOffset.java (100%) rename kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/{deprecated => }/EventSerializationTest.java (98%) rename kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/{deprecated => }/EventTestUtil.java (100%) diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitCompletePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitCompletePayload.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitReadyPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitReadyPayload.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitRequestPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitRequestPayload.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitResponsePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitResponsePayload.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitTablePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/CommitTablePayload.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Element.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Element.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Event.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Event.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/EventType.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/EventType.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Payload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/Payload.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TableName.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TableName.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TopicPartitionOffset.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java similarity index 100% rename from kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/deprecated/TopicPartitionOffset.java rename to kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java diff --git a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/deprecated/EventSerializationTest.java b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java similarity index 98% rename from kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/deprecated/EventSerializationTest.java rename to kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java index 5e8b6e23..50029d00 100644 --- a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/deprecated/EventSerializationTest.java +++ b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import static org.assertj.core.api.Assertions.assertThat; diff --git a/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/deprecated/EventTestUtil.java b/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/EventTestUtil.java similarity index 100% rename from kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/deprecated/EventTestUtil.java rename to kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/EventTestUtil.java From 2d4b7cccf62d41da1d39050e9cf2094096d1be6b Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 30 Apr 2024 16:31:19 -0700 Subject: [PATCH 07/13] starting to convert events --- .../connect/events/CommitCompletePayload.java | 2 +- .../connect/events/CommitReadyPayload.java | 2 +- .../connect/events/CommitRequestPayload.java | 2 +- .../connect/events/CommitResponsePayload.java | 2 +- .../connect/events/CommitTablePayload.java | 2 +- .../iceberg/connect/events/Element.java | 2 +- .../tabular/iceberg/connect/events/Event.java | 2 +- .../iceberg/connect/events/EventType.java | 2 +- .../iceberg/connect/events/Payload.java | 2 +- .../iceberg/connect/events/TableName.java | 2 +- .../connect/events/TopicPartitionOffset.java | 2 +- .../iceberg/connect/events/EventTestUtil.java | 2 +- .../iceberg/connect/channel/EventDecoder.java | 29 +++++++++------ .../connect/channel/EventDecoderTest.java | 35 ++++++++++++++++++ .../src/test/resources/commit_request.event | Bin 0 -> 559 bytes 15 files changed, 65 insertions(+), 23 deletions(-) create mode 100644 kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java create mode 100644 kafka-connect/src/test/resources/commit_request.event diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java index b5f4d272..53015a93 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitCompletePayload.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import java.util.UUID; import org.apache.avro.Schema; diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java index 4d027885..fe766ac9 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitReadyPayload.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import java.util.List; import java.util.UUID; diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java index 8426ed5e..bc0ced46 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitRequestPayload.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import java.util.UUID; import org.apache.avro.Schema; diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java index 1a63142b..49bd7203 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitResponsePayload.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import java.util.List; import java.util.Map; diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java index 37c89805..9e6d3257 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/CommitTablePayload.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import java.util.UUID; import org.apache.avro.Schema; diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java index 0f39fbdd..a7cd9fb8 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Element.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java index dcb4991e..f495c24f 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import java.io.IOException; import java.io.UncheckedIOException; diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java index edea00bc..a76ea9fe 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/EventType.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; public enum EventType { COMMIT_REQUEST(0), diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java index d2adf374..2aedbb55 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Payload.java @@ -16,6 +16,6 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; public interface Payload extends Element {} diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java index 9710c1da..309ce49a 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TableName.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import static java.util.stream.Collectors.toList; diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java index c3cc2f64..2d989de4 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/TopicPartitionOffset.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; diff --git a/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/EventTestUtil.java b/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/EventTestUtil.java index 09678401..3a7e7143 100644 --- a/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/EventTestUtil.java +++ b/kafka-connect-events/src/testFixtures/java/io/tabular/iceberg/connect/events/EventTestUtil.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.tabular.iceberg.connect.events.deprecated; +package io.tabular.iceberg.connect.events; import java.util.UUID; import org.apache.iceberg.DataFile; diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java index c97d0d48..f95be006 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java @@ -18,13 +18,13 @@ */ package io.tabular.iceberg.connect.channel; -import io.tabular.iceberg.connect.events.deprecated.CommitRequestPayload; +import io.tabular.iceberg.connect.events.CommitRequestPayload; import java.util.UUID; + +import io.tabular.iceberg.connect.events.CommitResponsePayload; +import org.apache.avro.Schema; import org.apache.avro.SchemaParseException; -import org.apache.iceberg.connect.events.AvroUtil; -import org.apache.iceberg.connect.events.Event; -import org.apache.iceberg.connect.events.Payload; -import org.apache.iceberg.connect.events.StartCommit; +import org.apache.iceberg.connect.events.*; public class EventDecoder { @@ -34,23 +34,30 @@ public static Event decode(byte[] value) { try { return AvroUtil.decode(value); } catch (SchemaParseException exception) { - io.tabular.iceberg.connect.events.deprecated.Event event = - io.tabular.iceberg.connect.events.deprecated.Event.decode(value); + io.tabular.iceberg.connect.events.Event event = + io.tabular.iceberg.connect.events.Event.decode(value); return convertLegacy(event); } } - private static Event convertLegacy(io.tabular.iceberg.connect.events.deprecated.Event event) { + private static Event convertLegacy(io.tabular.iceberg.connect.events.Event event) { Payload payload = convertPayload(event.payload()); return new Event(event.groupId(), payload); } private static Payload convertPayload( - io.tabular.iceberg.connect.events.deprecated.Payload payload) { - if (payload instanceof io.tabular.iceberg.connect.events.deprecated.CommitRequestPayload) { + io.tabular.iceberg.connect.events.Payload payload) { + if (payload instanceof CommitRequestPayload) { CommitRequestPayload pay = (CommitRequestPayload) payload; return new StartCommit((UUID) pay.get(0)); - } else { + } else if (payload instanceof CommitResponsePayload) { + CommitResponsePayload pay = (CommitResponsePayload) payload; + Schema schema = pay.getSchema(); + // need to get a PartitionType somehow . + throw new RuntimeException("stuck here"); + } + + else { throw new RuntimeException("borp"); } } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java new file mode 100644 index 00000000..99446190 --- /dev/null +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java @@ -0,0 +1,35 @@ +package io.tabular.iceberg.connect.channel; + +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.PayloadType; +import org.apache.iceberg.connect.events.StartCommit; +import org.junit.jupiter.api.Test; + +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.UUID; + +import static org.assertj.core.api.Assertions.assertThat; + +public class EventDecoderTest { + + private byte[] event(String file) { + try { + URL jsonResource = this.getClass().getClassLoader().getResource(file); + return Files.readAllBytes(Paths.get(jsonResource.toURI())); + } catch (Exception e) { + throw new RuntimeException(String.format("Could not load %s for test", file)); + } + } + + @Test + public void testCommitRequestBecomesStartCommit() { + Event result = EventDecoder.decode(event("commit_request.event")); + assertThat(result.type()).isEqualTo(PayloadType.START_COMMIT); + assertThat(result.payload()).isInstanceOf(StartCommit.class); + StartCommit payload = (StartCommit) result.payload(); + // hardcoded in the commit_request.event file + assertThat(payload.commitId()).isEqualTo(UUID.fromString("389c1c97-c3a4-4f0a-be9c-60ba49d490ff")); + } +} diff --git a/kafka-connect/src/test/resources/commit_request.event b/kafka-connect/src/test/resources/commit_request.event new file mode 100644 index 0000000000000000000000000000000000000000..e66a2dae9428c4bb7725149756914d3b6d8494a8 GIT binary patch literal 559 zcmX@a$oRclsid+XRmn=JC^b31C`Cy}DK9Y>#CI)A%`1U&iwhEyL1LNtdL@ZTr8$X3 zdYQ?oNvTEYddc~Dd8x@IdZ{4g#X$9GnW;G`#Y$Gu)i9GXL8d}%f!mgrS&@olS!pR) z5y)zoT|hm>nN>hr4b60va`MwNlM{17AWj9Vu7z5q3)E?)q-&^Ds{^+YY#Gc>rA)9t z!0y7TI1^}TNn&mR+?7D<^3q{m!KS4ku`(w=5sSCb1JOA@H#f5+D7CN@Xhi^A7iJhG zgB5xrj8uXK37WHG@p(JFD8IA-*^J_nqD*W)iLH%bzqr6f%U5PY9mC3Jr`dZL)}325 h`B}2X9GU5dm-ut-o0G81^U8$(63OYh&=}4y0sx(GyD|U( literal 0 HcmV?d00001 From 178d7be033f735b03cfb1de282fc37d121713204 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Thu, 9 May 2024 15:09:53 -0700 Subject: [PATCH 08/13] better tests for non-breaking change eventDecoder version --- .../events/EventSerializationTest.java | 6 +- .../iceberg/connect/channel/Channel.java | 16 +- .../iceberg/connect/channel/EventDecoder.java | 123 +++++++- .../connect/channel/EventDecoderTest.java | 288 ++++++++++++++++-- .../src/test/resources/commit_request.event | Bin 559 -> 0 bytes 5 files changed, 383 insertions(+), 50 deletions(-) delete mode 100644 kafka-connect/src/test/resources/commit_request.event diff --git a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java index 50029d00..22cdf095 100644 --- a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java +++ b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java @@ -28,10 +28,10 @@ import org.junit.jupiter.api.Test; public class EventSerializationTest { + private final UUID commitId = UUID.randomUUID(); @Test public void testCommitRequestSerialization() { - UUID commitId = UUID.randomUUID(); Event event = new Event("cg-connector", EventType.COMMIT_REQUEST, new CommitRequestPayload(commitId)); @@ -45,7 +45,6 @@ public void testCommitRequestSerialization() { @Test public void testCommitResponseSerialization() { - UUID commitId = UUID.randomUUID(); Event event = new Event( "cg-connector", @@ -72,7 +71,6 @@ public void testCommitResponseSerialization() { @Test public void testCommitReadySerialization() { - UUID commitId = UUID.randomUUID(); Event event = new Event( "cg-connector", @@ -95,7 +93,6 @@ public void testCommitReadySerialization() { @Test public void testCommitTableSerialization() { - UUID commitId = UUID.randomUUID(); Event event = new Event( "cg-connector", @@ -116,7 +113,6 @@ public void testCommitTableSerialization() { @Test public void testCommitCompleteSerialization() { - UUID commitId = UUID.randomUUID(); Event event = new Event( "cg-connector", EventType.COMMIT_COMPLETE, new CommitCompletePayload(commitId, 2L)); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java index 5c1e9e10..41f92114 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java @@ -55,6 +55,8 @@ public abstract class Channel { private final Map controlTopicOffsets = Maps.newHashMap(); private final String producerId; + private final EventDecoder eventDecoder; + public Channel( String name, String consumerGroupId, @@ -70,6 +72,7 @@ public Channel( this.admin = clientFactory.createAdmin(); this.producerId = UUID.randomUUID().toString(); + this.eventDecoder = new EventDecoder(config.catalogName()); } protected void send(Event event) { @@ -123,12 +126,13 @@ record -> { // so increment the record offset by one controlTopicOffsets.put(record.partition(), record.offset() + 1); - Event event = EventDecoder.decode(record.value()); - - if (event.groupId().equals(groupId)) { - LOG.debug("Received event of type: {}", event.type().name()); - if (receive(new Envelope(event, record.partition(), record.offset()))) { - LOG.info("Handled event of type: {}", event.type().name()); + Event event = eventDecoder.decode(record.value()); + if (event != null) { + if (event.groupId().equals(groupId)) { + LOG.debug("Received event of type: {}", event.type().name()); + if (receive(new Envelope(event, record.partition(), record.offset()))) { + LOG.info("Handled event of type: {}", event.type().name()); + } } } }); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java index f95be006..4683d8a9 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java @@ -18,19 +18,44 @@ */ package io.tabular.iceberg.connect.channel; +import io.tabular.iceberg.connect.events.CommitCompletePayload; +import io.tabular.iceberg.connect.events.CommitReadyPayload; import io.tabular.iceberg.connect.events.CommitRequestPayload; -import java.util.UUID; - import io.tabular.iceberg.connect.events.CommitResponsePayload; +import io.tabular.iceberg.connect.events.CommitTablePayload; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.SchemaParseException; -import org.apache.iceberg.connect.events.*; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.CommitComplete; +import org.apache.iceberg.connect.events.CommitToTable; +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.Payload; +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.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; public class EventDecoder { - private EventDecoder() {} + private final String catalogName; + + public EventDecoder(String catalogName) { + this.catalogName = catalogName; + } - public static Event decode(byte[] value) { + public Event decode(byte[] value) { try { return AvroUtil.decode(value); } catch (SchemaParseException exception) { @@ -40,25 +65,93 @@ public static Event decode(byte[] value) { } } - private static Event convertLegacy(io.tabular.iceberg.connect.events.Event event) { + private Event convertLegacy(io.tabular.iceberg.connect.events.Event event) { Payload payload = convertPayload(event.payload()); + if (payload == null) { + return null; + } return new Event(event.groupId(), payload); } - private static Payload convertPayload( - io.tabular.iceberg.connect.events.Payload payload) { + private Payload convertPayload(io.tabular.iceberg.connect.events.Payload payload) { if (payload instanceof CommitRequestPayload) { CommitRequestPayload pay = (CommitRequestPayload) payload; - return new StartCommit((UUID) pay.get(0)); - } else if (payload instanceof CommitResponsePayload) { + return new StartCommit(pay.commitId()); + } else if (payload instanceof CommitResponsePayload) { CommitResponsePayload pay = (CommitResponsePayload) payload; - Schema schema = pay.getSchema(); - // need to get a PartitionType somehow . - throw new RuntimeException("stuck here"); + return convertCommitResponse(pay); + } else if (payload instanceof CommitReadyPayload) { + CommitReadyPayload pay = (CommitReadyPayload) payload; + List legacyTPO = pay.assignments(); + List converted = + legacyTPO.stream() + .map( + t -> + new TopicPartitionOffset( + t.topic(), + t.partition(), + t.offset() == null ? null : t.offset(), + t.timestamp() == null + ? null + : OffsetDateTime.ofInstant( + Instant.ofEpochMilli(t.timestamp()), ZoneOffset.UTC))) + .collect(Collectors.toList()); + return new DataComplete(pay.commitId(), converted); + } else if (payload instanceof CommitTablePayload) { + CommitTablePayload pay = (CommitTablePayload) payload; + return new CommitToTable( + pay.commitId(), + TableReference.of(catalogName, pay.tableName().toIdentifier()), + pay.snapshotId(), + OffsetDateTime.ofInstant(Instant.ofEpochMilli(pay.vtts()), ZoneOffset.UTC)); + } else if (payload instanceof CommitCompletePayload) { + CommitCompletePayload pay = (CommitCompletePayload) payload; + return new CommitComplete( + pay.commitId(), + OffsetDateTime.ofInstant(Instant.ofEpochMilli(pay.vtts()), ZoneOffset.UTC)); + } else { + throw new IllegalStateException( + String.format("Unknown event payload: %s", payload.getSchema())); + } + } + + private Payload convertCommitResponse(CommitResponsePayload payload) { + List dataFiles = payload.dataFiles(); + List deleteFiles = payload.deleteFiles(); + if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { + return null; } - else { - throw new RuntimeException("borp"); + String target = (dataFiles.isEmpty()) ? "deleteFiles" : "dataFiles"; + List fields = + payload.getSchema().getField(target).schema().getTypes().stream() + .filter(s -> s.getType() != Schema.Type.NULL) + .findFirst() + .get() + .getElementType() + .getField("partition") + .schema() + .getFields(); + + List convertedFields = Lists.newArrayListWithExpectedSize(fields.size()); + + for (Schema.Field f : fields) { + Schema fieldSchema = + f.schema().getTypes().stream() + .filter(s -> s.getType() != Schema.Type.NULL) + .findFirst() + .get(); + Type fieldType = AvroSchemaUtil.convert(fieldSchema); + int fieldId = (int) f.getObjectProp("field-id"); + convertedFields.add(Types.NestedField.optional(fieldId, f.name(), fieldType)); } + + Types.StructType convertedStructType = Types.StructType.of(convertedFields); + return new DataWritten( + convertedStructType, + payload.commitId(), + TableReference.of(catalogName, payload.tableName().toIdentifier()), + payload.dataFiles(), + payload.deleteFiles()); } } diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java index 99446190..6961f270 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java @@ -1,35 +1,275 @@ +/* + * 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 io.tabular.iceberg.connect.channel; +import static org.assertj.core.api.Assertions.assertThat; + +import io.tabular.iceberg.connect.data.SchemaUtils; +import io.tabular.iceberg.connect.events.CommitCompletePayload; +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.CommitTablePayload; +import io.tabular.iceberg.connect.events.EventTestUtil; +import io.tabular.iceberg.connect.events.EventType; +import io.tabular.iceberg.connect.events.TableName; +import io.tabular.iceberg.connect.events.TopicPartitionOffset; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.avro.Schema; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.events.CommitComplete; +import org.apache.iceberg.connect.events.CommitToTable; +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.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.UUID; +public class EventDecoderTest { + private final UUID commitId = UUID.fromString("e0bf874a-0f83-4242-97c9-f0a7b5cb3f45"); + private final String catalogName = "catalog"; -import static org.assertj.core.api.Assertions.assertThat; + private final EventDecoder eventDecoder = new EventDecoder(catalogName); -public class EventDecoderTest { + @Test + public void testCommitRequestBecomesStartCommit() { + io.tabular.iceberg.connect.events.Event event = + new io.tabular.iceberg.connect.events.Event( + "cg-connector", + io.tabular.iceberg.connect.events.EventType.COMMIT_REQUEST, + new CommitRequestPayload(commitId)); + + byte[] data = io.tabular.iceberg.connect.events.Event.encode(event); + + Event result = eventDecoder.decode(data); + + assertThat(result.groupId()).isEqualTo("cg-connector"); + assertThat(result.type()).isEqualTo(PayloadType.START_COMMIT); + assertThat(result.payload()).isInstanceOf(StartCommit.class); + StartCommit payload = (StartCommit) result.payload(); + assertThat(payload.commitId()).isEqualTo(commitId); + } + + @Test + public void testCommitResponseBecomesDataWrittenUnpartitioned() { + io.tabular.iceberg.connect.events.Event event = + new io.tabular.iceberg.connect.events.Event( + "cg-connector", + EventType.COMMIT_RESPONSE, + new CommitResponsePayload( + Types.StructType.of(), + commitId, + new TableName(Collections.singletonList("db"), "tbl"), + Arrays.asList(EventTestUtil.createDataFile(), EventTestUtil.createDataFile()), + Arrays.asList(EventTestUtil.createDeleteFile(), EventTestUtil.createDeleteFile()))); + + byte[] data = io.tabular.iceberg.connect.events.Event.encode(event); + + Event result = eventDecoder.decode(data); + + assertThat(result.groupId()).isEqualTo("cg-connector"); + assertThat(result.type()).isEqualTo(PayloadType.DATA_WRITTEN); + assertThat(result.payload()).isInstanceOf(DataWritten.class); + DataWritten payload = (DataWritten) result.payload(); + assertThat(payload.commitId()).isEqualTo(commitId); + assertThat(payload.dataFiles().size()).isEqualTo(2); + assertThat(payload.deleteFiles().size()).isEqualTo(2); + assertThat(payload.tableReference().catalog()).isEqualTo("catalog"); + assertThat(payload.tableReference().identifier()).isEqualTo(TableIdentifier.of("db", "tbl")); + // should have an empty partition spec on the schema + Schema.Field field = + payload.getSchema().getFields().get(2).schema().getTypes().stream() + .filter(s -> s.getType() != Schema.Type.NULL) + .findFirst() + .get() + .getElementType() + .getField("partition"); + assertThat(field.schema().getFields()).isEmpty(); + } + + @Test + public void testCommitResponseBecomesDataWrittenPartitioned() { + + org.apache.iceberg.Schema schemaSpec = + new org.apache.iceberg.Schema( + Types.NestedField.required(1, "i", Types.IntegerType.get()), + Types.NestedField.required(2, "s", Types.StringType.get()), + Types.NestedField.required(3, "ts1", Types.TimestampType.withZone()), + Types.NestedField.required(4, "ts2", Types.TimestampType.withZone()), + Types.NestedField.required(5, "ts3", Types.TimestampType.withZone()), + Types.NestedField.required(6, "ts4", Types.TimestampType.withZone())); + + List partitionFields = + ImmutableList.of( + "year(ts1)", + "month(ts2)", + "day(ts3)", + "hour(ts4)", + "bucket(i, 4)", + "truncate(s, 10)", + "s"); + PartitionSpec spec = SchemaUtils.createPartitionSpec(schemaSpec, partitionFields); + + Types.StructType structType = spec.partitionType(); + + io.tabular.iceberg.connect.events.Event event = + new io.tabular.iceberg.connect.events.Event( + "cg-connector", + EventType.COMMIT_RESPONSE, + new CommitResponsePayload( + structType, + commitId, + new TableName(Collections.singletonList("db"), "tbl"), + Arrays.asList(EventTestUtil.createDataFile(), EventTestUtil.createDataFile()), + Arrays.asList(EventTestUtil.createDeleteFile(), EventTestUtil.createDeleteFile()))); + + byte[] data = io.tabular.iceberg.connect.events.Event.encode(event); + + Event result = eventDecoder.decode(data); + + assertThat(event.groupId()).isEqualTo("cg-connector"); + assertThat(result.type()).isEqualTo(PayloadType.DATA_WRITTEN); + assertThat(result.payload()).isInstanceOf(DataWritten.class); + DataWritten payload = (DataWritten) result.payload(); + assertThat(payload.commitId()).isEqualTo(commitId); + assertThat(payload.dataFiles().size()).isEqualTo(2); + assertThat(payload.deleteFiles().size()).isEqualTo(2); + assertThat(payload.tableReference().catalog()).isEqualTo("catalog"); + assertThat(payload.tableReference().identifier()).isEqualTo(TableIdentifier.of("db", "tbl")); + + Schema.Field field = + payload.getSchema().getFields().get(2).schema().getTypes().stream() + .filter(s -> s.getType() != Schema.Type.NULL) + .findFirst() + .get() + .getElementType() + .getField("partition"); + List resultPartitionInfo = + field.schema().getFields().stream() + .map( + f -> { + Type fieldType = AvroSchemaUtil.convert(f.schema()); + int fieldId = (int) f.getObjectProp("field-id"); + return Types.NestedField.optional(fieldId, f.name(), fieldType); + }) + .collect(Collectors.toList()); + + List expectedPartitionInfo = + Lists.newArrayList( + Types.NestedField.optional(1000, "ts1_year", Types.IntegerType.get()), + Types.NestedField.optional(1001, "ts2_month", Types.IntegerType.get()), + Types.NestedField.optional(1002, "ts3_day", Types.DateType.get()), + Types.NestedField.optional(1003, "ts4_hour", Types.IntegerType.get()), + Types.NestedField.optional(1004, "i_bucket", Types.IntegerType.get()), + Types.NestedField.optional(1005, "s_trunc", Types.StringType.get()), + Types.NestedField.optional(1006, "s", Types.StringType.get())); + + assertThat(resultPartitionInfo).isEqualTo(expectedPartitionInfo); + } + + @Test + public void testCommitReadyBecomesDataComplete() { + io.tabular.iceberg.connect.events.Event event = + new io.tabular.iceberg.connect.events.Event( + "cg-connector", + EventType.COMMIT_READY, + new CommitReadyPayload( + commitId, + Arrays.asList( + new TopicPartitionOffset("topic", 1, 1L, 1L), + new TopicPartitionOffset("topic", 2, null, null)))); + + byte[] data = io.tabular.iceberg.connect.events.Event.encode(event); + + Event result = eventDecoder.decode(data); + assertThat(event.groupId()).isEqualTo("cg-connector"); + + assertThat(result.type()).isEqualTo(PayloadType.DATA_COMPLETE); + assertThat(result.payload()).isInstanceOf(DataComplete.class); + DataComplete payload = (DataComplete) result.payload(); + + assertThat(payload.commitId()).isEqualTo(commitId); + assertThat(payload.assignments().get(0).topic()).isEqualTo("topic"); + assertThat(payload.assignments().get(0).partition()).isEqualTo(1); + assertThat(payload.assignments().get(0).offset()).isEqualTo(1L); + assertThat(payload.assignments().get(0).timestamp()) + .isEqualTo(OffsetDateTime.ofInstant(Instant.ofEpochMilli(1), ZoneOffset.UTC)); + + assertThat(payload.assignments().get(1).topic()).isEqualTo("topic"); + assertThat(payload.assignments().get(1).partition()).isEqualTo(2); + assertThat(payload.assignments().get(1).offset()).isNull(); + assertThat(payload.assignments().get(1).timestamp()).isNull(); + } + + @Test + public void testCommitTableBecomesCommitToTable() { + io.tabular.iceberg.connect.events.Event event = + new io.tabular.iceberg.connect.events.Event( + "cg-connector", + EventType.COMMIT_TABLE, + new CommitTablePayload( + commitId, new TableName(Collections.singletonList("db"), "tbl"), 1L, 2L)); + + byte[] data = io.tabular.iceberg.connect.events.Event.encode(event); + + Event result = eventDecoder.decode(data); + assertThat(event.groupId()).isEqualTo("cg-connector"); + assertThat(result.type()).isEqualTo(PayloadType.COMMIT_TO_TABLE); + assertThat(result.payload()).isInstanceOf(CommitToTable.class); + CommitToTable payload = (CommitToTable) result.payload(); + + assertThat(payload.commitId()).isEqualTo(commitId); + assertThat(payload.snapshotId()).isEqualTo(1L); + assertThat(payload.validThroughTs()) + .isEqualTo(OffsetDateTime.ofInstant(Instant.ofEpochMilli(2L), ZoneOffset.UTC)); + assertThat(payload.tableReference().catalog()).isEqualTo(catalogName); + assertThat(payload.tableReference().identifier()).isEqualTo(TableIdentifier.of("db", "tbl")); + } + + @Test + public void testCommitCompleteBecomesCommitCompleteSerialization() { + io.tabular.iceberg.connect.events.Event event = + new io.tabular.iceberg.connect.events.Event( + "cg-connector", EventType.COMMIT_COMPLETE, new CommitCompletePayload(commitId, 2L)); + + byte[] data = io.tabular.iceberg.connect.events.Event.encode(event); - private byte[] event(String file) { - try { - URL jsonResource = this.getClass().getClassLoader().getResource(file); - return Files.readAllBytes(Paths.get(jsonResource.toURI())); - } catch (Exception e) { - throw new RuntimeException(String.format("Could not load %s for test", file)); - } - } - - @Test - public void testCommitRequestBecomesStartCommit() { - Event result = EventDecoder.decode(event("commit_request.event")); - assertThat(result.type()).isEqualTo(PayloadType.START_COMMIT); - assertThat(result.payload()).isInstanceOf(StartCommit.class); - StartCommit payload = (StartCommit) result.payload(); - // hardcoded in the commit_request.event file - assertThat(payload.commitId()).isEqualTo(UUID.fromString("389c1c97-c3a4-4f0a-be9c-60ba49d490ff")); - } + Event result = eventDecoder.decode(data); + assertThat(result.type()).isEqualTo(PayloadType.COMMIT_COMPLETE); + assertThat(result.payload()).isInstanceOf(CommitComplete.class); + CommitComplete payload = (CommitComplete) result.payload(); + assertThat(payload.commitId()).isEqualTo(commitId); + assertThat(payload.validThroughTs()) + .isEqualTo(OffsetDateTime.ofInstant(Instant.ofEpochMilli(2L), ZoneOffset.UTC)); + } } diff --git a/kafka-connect/src/test/resources/commit_request.event b/kafka-connect/src/test/resources/commit_request.event deleted file mode 100644 index e66a2dae9428c4bb7725149756914d3b6d8494a8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 559 zcmX@a$oRclsid+XRmn=JC^b31C`Cy}DK9Y>#CI)A%`1U&iwhEyL1LNtdL@ZTr8$X3 zdYQ?oNvTEYddc~Dd8x@IdZ{4g#X$9GnW;G`#Y$Gu)i9GXL8d}%f!mgrS&@olS!pR) z5y)zoT|hm>nN>hr4b60va`MwNlM{17AWj9Vu7z5q3)E?)q-&^Ds{^+YY#Gc>rA)9t z!0y7TI1^}TNn&mR+?7D<^3q{m!KS4ku`(w=5sSCb1JOA@H#f5+D7CN@Xhi^A7iJhG zgB5xrj8uXK37WHG@p(JFD8IA-*^J_nqD*W)iLH%bzqr6f%U5PY9mC3Jr`dZL)}325 h`B}2X9GU5dm-ut-o0G81^U8$(63OYh&=}4y0sx(GyD|U( From 4e761f18ef3676ee6ad9f2ccc2e5996b011155c3 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Thu, 9 May 2024 15:24:15 -0700 Subject: [PATCH 09/13] 1.5.1 --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 991537e7..a6c4950d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -5,7 +5,7 @@ awaitility-ver = "4.2.0" hadoop-ver = "3.3.6" hive-ver = "2.3.9" http-client-ver = "5.2.1" -iceberg-ver = "1.5.0" +iceberg-ver = "1.5.1" jackson-ver = "2.14.2" junit-ver = "5.10.0" kafka-ver = "3.5.1" From e2c623b8aa8b9c5b420bc989f1e1c2b0f1d07523 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Thu, 9 May 2024 15:28:16 -0700 Subject: [PATCH 10/13] iceberg 1.5.2, still need to do merge conflicts --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index a6c4950d..cedf9607 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -5,7 +5,7 @@ awaitility-ver = "4.2.0" hadoop-ver = "3.3.6" hive-ver = "2.3.9" http-client-ver = "5.2.1" -iceberg-ver = "1.5.1" +iceberg-ver = "1.5.2" jackson-ver = "2.14.2" junit-ver = "5.10.0" kafka-ver = "3.5.1" From 8c46cb8f0725072c3ade2f12f2cfaed05051d582 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 10 May 2024 09:55:12 -0700 Subject: [PATCH 11/13] merge conflicts, tests, docs --- .../tabular/iceberg/connect/events/Event.java | 7 ++ .../avro/DeprecatedAvroEncoderUtil.java | 7 ++ .../avro/DeprecatedAvroSchemaUtil.java | 7 ++ .../avro/DeprecatedGenericAvroReader.java | 7 ++ .../iceberg/avro/DeprecatedTypeToSchema.java | 7 ++ .../iceberg/connect/channel/Channel.java | 23 ++-- .../connect/channel/CommitterImpl.java | 26 ++-- .../iceberg/connect/channel/EventDecoder.java | 13 ++ .../connect/channel/CommitterImplTest.java | 115 +++++++++++------- .../iceberg/connect/channel/WorkerTest.java | 6 - 10 files changed, 144 insertions(+), 74 deletions(-) diff --git a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java index f495c24f..fdb1d17a 100644 --- a/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java +++ b/kafka-connect-events/src/main/java/io/tabular/iceberg/connect/events/Event.java @@ -63,6 +63,13 @@ public Event(Schema avroSchema) { this.avroSchema = avroSchema; } + /** + * @deprecated + *

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.

+ */ + @Deprecated public Event(String groupId, EventType type, Payload payload) { this.id = UUID.randomUUID(); this.type = type; diff --git a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroEncoderUtil.java b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroEncoderUtil.java index 25d9cc7e..679da6dc 100644 --- a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroEncoderUtil.java +++ b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroEncoderUtil.java @@ -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() {} diff --git a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroSchemaUtil.java b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroSchemaUtil.java index 1188e6d5..acaf5939 100644 --- a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroSchemaUtil.java +++ b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedAvroSchemaUtil.java @@ -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() {} diff --git a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedGenericAvroReader.java b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedGenericAvroReader.java index 82446070..70302ef3 100644 --- a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedGenericAvroReader.java +++ b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedGenericAvroReader.java @@ -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 implements DatumReader, SupportsRowPosition { private final Schema readSchema; diff --git a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedTypeToSchema.java b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedTypeToSchema.java index cebcaa17..626ecb7b 100644 --- a/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedTypeToSchema.java +++ b/kafka-connect-events/src/main/java/org/apache/iceberg/avro/DeprecatedTypeToSchema.java @@ -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 { private static final Schema BOOLEAN_SCHEMA = Schema.create(Schema.Type.BOOLEAN); private static final Schema INTEGER_SCHEMA = Schema.create(Schema.Type.INT); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java index 919a7316..d6fc5e6e 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/Channel.java @@ -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; @@ -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 producer; private final Consumer consumer; @@ -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> 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) { @@ -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); } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java index d8c299a0..10730d99 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitterImpl.java @@ -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; @@ -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; @@ -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; } @@ -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())); @@ -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 offsets = committable.offsetsByTopicPartition(); diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java index 4683d8a9..fd8f3b0c 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java @@ -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; @@ -55,6 +62,12 @@ public EventDecoder(String catalogName) { this.catalogName = catalogName; } + /** + * @deprecated + *

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

+ */ + @Deprecated public Event decode(byte[] value) { try { return AvroUtil.decode(value); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java index f87e5324..fb27221c 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java @@ -30,14 +30,10 @@ import io.tabular.iceberg.connect.IcebergSinkConfig; import io.tabular.iceberg.connect.data.Offset; import io.tabular.iceberg.connect.data.WriterResult; -import io.tabular.iceberg.connect.events.CommitCompletePayload; -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.EventTestUtil; -import io.tabular.iceberg.connect.events.EventType; import java.io.IOException; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; import java.util.List; import java.util.Map; import java.util.Optional; @@ -47,9 +43,20 @@ import org.apache.commons.lang3.NotImplementedException; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.CommitComplete; +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.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.ImmutableSet; @@ -78,6 +85,8 @@ import org.mockito.Mockito; class CommitterImplTest { + + private static final String CATALOG_NAME = "iceberg"; private static final String SOURCE_TOPIC = "source-topic-name"; private static final TopicPartition SOURCE_TP0 = new TopicPartition(SOURCE_TOPIC, 0); private static final TopicPartition SOURCE_TP1 = new TopicPartition(SOURCE_TOPIC, 1); @@ -229,7 +238,7 @@ private static > void assertSameContentFiles( expected.stream().map(CommitterImplTest::toPath).collect(Collectors.toList())); } - private void assertCommitResponse( + private void assertDataWritten( ProducerRecord producerRecord, UUID expectedProducerId, UUID expectedCommitId, @@ -238,27 +247,28 @@ private void assertCommitResponse( List expectedDeleteFiles) { assertThat(producerRecord.key()).isEqualTo(expectedProducerId.toString()); - Event event = Event.decode(producerRecord.value()); - assertThat(event.type()).isEqualTo(EventType.COMMIT_RESPONSE); - assertThat(event.payload()).isInstanceOf(CommitResponsePayload.class); - CommitResponsePayload commitResponsePayload = (CommitResponsePayload) event.payload(); - assertThat(commitResponsePayload.commitId()).isEqualTo(expectedCommitId); - assertThat(commitResponsePayload.tableName().toIdentifier()).isEqualTo(expectedTableIdentifier); - assertSameContentFiles(commitResponsePayload.dataFiles(), expectedDataFiles); - assertSameContentFiles(commitResponsePayload.deleteFiles(), expectedDeleteFiles); + Event event = AvroUtil.decode(producerRecord.value()); + assertThat(event.type()).isEqualTo(PayloadType.DATA_WRITTEN); + assertThat(event.payload()).isInstanceOf(DataWritten.class); + DataWritten payload = (DataWritten) event.payload(); + assertThat(payload.commitId()).isEqualTo(expectedCommitId); + assertThat(payload.tableReference().identifier()).isEqualTo(expectedTableIdentifier); + assertThat(payload.tableReference().catalog()).isEqualTo(CATALOG_NAME); + assertSameContentFiles(payload.dataFiles(), expectedDataFiles); + assertSameContentFiles(payload.deleteFiles(), expectedDeleteFiles); } - private void assertCommitReady( + private void assertDataComplete( ProducerRecord producerRecord, UUID expectedProducerId, UUID expectedCommitId, - Map> expectedAssignments) { + Map> expectedAssignments) { assertThat(producerRecord.key()).isEqualTo(expectedProducerId.toString()); - Event event = Event.decode(producerRecord.value()); - assertThat(event.type()).isEqualTo(EventType.COMMIT_READY); - assertThat(event.payload()).isInstanceOf(CommitReadyPayload.class); - CommitReadyPayload commitReadyPayload = (CommitReadyPayload) event.payload(); + Event event = AvroUtil.decode(producerRecord.value()); + assertThat(event.type()).isEqualTo(PayloadType.DATA_COMPLETE); + assertThat(event.payload()).isInstanceOf(DataComplete.class); + DataComplete commitReadyPayload = (DataComplete) event.payload(); assertThat(commitReadyPayload.commitId()).isEqualTo(expectedCommitId); assertThat( commitReadyPayload.assignments().stream() @@ -274,6 +284,28 @@ private void assertCommitReady( .collect(Collectors.toList())); } + private OffsetDateTime offsetDateTime(Long ms) { + return OffsetDateTime.ofInstant(Instant.ofEpochMilli(ms), ZoneOffset.UTC); + } + + public static DataFile createDataFile() { + return DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(UUID.randomUUID() + ".parquet") + .withFormat(FileFormat.PARQUET) + .withFileSizeInBytes(100L) + .withRecordCount(5) + .build(); + } + + public static DeleteFile createDeleteFile() { + return FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes(1) + .withPath(UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + } + @Test public void testShouldRewindOffsetsToStableControlGroupConsumerOffsetsForAssignedPartitionsOnConstruction() @@ -386,11 +418,10 @@ public void testCommitShouldDoNothingIfThereIsNoCommitRequestMessage() throws IO CONTROL_TOPIC_PARTITION.partition(), 0, UUID.randomUUID().toString(), - Event.encode( + AvroUtil.encode( new Event( CONFIG.controlGroupId(), - EventType.COMMIT_COMPLETE, - new CommitCompletePayload(UUID.randomUUID(), 100L))))); + new CommitComplete(UUID.randomUUID(), offsetDateTime(100L)))))); committer.commit(committableSupplier); @@ -410,7 +441,7 @@ public void testCommitShouldRespondToCommitRequest() throws IOException { ImmutableMap.of( CONFIG.controlGroupId(), ImmutableMap.of(SOURCE_TP0, 110L, SOURCE_TP1, 100L))); - List dataFiles = ImmutableList.of(EventTestUtil.createDataFile()); + List dataFiles = ImmutableList.of(createDataFile()); List deleteFiles = ImmutableList.of(); Types.StructType partitionStruct = Types.StructType.of(); Map sourceOffsets = ImmutableMap.of(SOURCE_TP0, new Offset(100L, 200L)); @@ -432,28 +463,27 @@ public void testCommitShouldRespondToCommitRequest() throws IOException { CONTROL_TOPIC_PARTITION.partition(), 0, UUID.randomUUID().toString(), - Event.encode( + AvroUtil.encode( new Event( CONFIG.controlGroupId(), - EventType.COMMIT_REQUEST, - new CommitRequestPayload(commitId))))); + new StartCommit(commitId))))); committer.commit(committableSupplier); assertThat(producer.transactionCommitted()).isTrue(); assertThat(producer.history()).hasSize(2); - assertCommitResponse( + assertDataWritten( producer.history().get(0), producerId, commitId, TABLE_1_IDENTIFIER, dataFiles, deleteFiles); - assertCommitReady( + assertDataComplete( producer.history().get(1), producerId, commitId, - ImmutableMap.of(SOURCE_TP0, Pair.of(100L, 200L))); + ImmutableMap.of(SOURCE_TP0, Pair.of(100L, offsetDateTime(200L)))); assertThat(producer.consumerGroupOffsetsHistory()).hasSize(2); Map expectedConsumerOffset = @@ -491,17 +521,17 @@ public void testCommitWhenCommittableIsEmpty() throws IOException { CONTROL_TOPIC_PARTITION.partition(), 0, UUID.randomUUID().toString(), - Event.encode( + AvroUtil.encode( new Event( CONFIG.controlGroupId(), - EventType.COMMIT_REQUEST, - new CommitRequestPayload(commitId))))); + new StartCommit(commitId))))); + committer.commit(committableSupplier); assertThat(producer.transactionCommitted()).isTrue(); assertThat(producer.history()).hasSize(1); - assertCommitReady( + assertDataComplete( producer.history().get(0), producerId, commitId, @@ -529,7 +559,7 @@ public void testCommitShouldCommitOffsetsOnlyForPartitionsWeMadeProgressOn() thr ImmutableMap.of( CONFIG.controlGroupId(), ImmutableMap.of(SOURCE_TP0, 110L, SOURCE_TP1, 100L))); - List dataFiles = ImmutableList.of(EventTestUtil.createDataFile()); + List dataFiles = ImmutableList.of(createDataFile()); List deleteFiles = ImmutableList.of(); Types.StructType partitionStruct = Types.StructType.of(); CommittableSupplier committableSupplier = @@ -550,30 +580,29 @@ public void testCommitShouldCommitOffsetsOnlyForPartitionsWeMadeProgressOn() thr CONTROL_TOPIC_PARTITION.partition(), 0, UUID.randomUUID().toString(), - Event.encode( + AvroUtil.encode( new Event( CONFIG.controlGroupId(), - EventType.COMMIT_REQUEST, - new CommitRequestPayload(commitId))))); + new StartCommit(commitId))))); committer.commit(committableSupplier); assertThat(producer.transactionCommitted()).isTrue(); assertThat(producer.history()).hasSize(2); - assertCommitResponse( + assertDataWritten( producer.history().get(0), producerId, commitId, TABLE_1_IDENTIFIER, dataFiles, deleteFiles); - assertCommitReady( + assertDataComplete( producer.history().get(1), producerId, commitId, ImmutableMap.of( sourceTp0, Pair.of(null, null), - sourceTp1, Pair.of(100L, 200L))); + sourceTp1, Pair.of(100L, offsetDateTime(200L)))); assertThat(producer.consumerGroupOffsetsHistory()).hasSize(2); Map expectedConsumerOffset = diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java index 2dea0109..8ce6c0c8 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/WorkerTest.java @@ -31,12 +31,6 @@ import io.tabular.iceberg.connect.events.EventTestUtil; import java.util.Map; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.connect.events.AvroUtil; -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.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types.StructType; From 3626568529a49febb2d31c359f31df47a97aba39 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Fri, 17 May 2024 09:51:32 -0700 Subject: [PATCH 12/13] comments --- .../events/EventSerializationTest.java | 56 ++++++++++--------- kafka-connect/build.gradle | 1 - .../iceberg/connect/channel/CommitState.java | 17 ++---- .../iceberg/connect/channel/EventDecoder.java | 4 +- .../connect/channel/CommitterImplTest.java | 23 +------- .../connect/channel/EventDecoderTest.java | 13 +---- 6 files changed, 38 insertions(+), 76 deletions(-) diff --git a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java index 22cdf095..e84e3c15 100644 --- a/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java +++ b/kafka-connect-events/src/test/java/io/tabular/iceberg/connect/events/EventSerializationTest.java @@ -28,12 +28,12 @@ import org.junit.jupiter.api.Test; public class EventSerializationTest { - private final UUID commitId = UUID.randomUUID(); @Test public void testCommitRequestSerialization() { + UUID commitId = UUID.randomUUID(); Event event = - new Event("cg-connector", EventType.COMMIT_REQUEST, new CommitRequestPayload(commitId)); + new Event("cg-connector", EventType.COMMIT_REQUEST, new CommitRequestPayload(commitId)); byte[] data = Event.encode(event); Event result = Event.decode(data); @@ -45,16 +45,17 @@ public void testCommitRequestSerialization() { @Test public void testCommitResponseSerialization() { + UUID commitId = UUID.randomUUID(); Event event = - new Event( - "cg-connector", - EventType.COMMIT_RESPONSE, - new CommitResponsePayload( - StructType.of(), - commitId, - new TableName(Collections.singletonList("db"), "tbl"), - Arrays.asList(EventTestUtil.createDataFile(), EventTestUtil.createDataFile()), - Arrays.asList(EventTestUtil.createDeleteFile(), EventTestUtil.createDeleteFile()))); + new Event( + "cg-connector", + EventType.COMMIT_RESPONSE, + new CommitResponsePayload( + StructType.of(), + commitId, + new TableName(Collections.singletonList("db"), "tbl"), + Arrays.asList(EventTestUtil.createDataFile(), EventTestUtil.createDataFile()), + Arrays.asList(EventTestUtil.createDeleteFile(), EventTestUtil.createDeleteFile()))); byte[] data = Event.encode(event); Event result = Event.decode(data); @@ -71,15 +72,16 @@ public void testCommitResponseSerialization() { @Test public void testCommitReadySerialization() { + UUID commitId = UUID.randomUUID(); Event event = - new Event( - "cg-connector", - EventType.COMMIT_READY, - new CommitReadyPayload( - commitId, - Arrays.asList( - new TopicPartitionOffset("topic", 1, 1L, 1L), - new TopicPartitionOffset("topic", 2, null, null)))); + new Event( + "cg-connector", + EventType.COMMIT_READY, + new CommitReadyPayload( + commitId, + Arrays.asList( + new TopicPartitionOffset("topic", 1, 1L, 1L), + new TopicPartitionOffset("topic", 2, null, null)))); byte[] data = Event.encode(event); Event result = Event.decode(data); @@ -93,12 +95,13 @@ public void testCommitReadySerialization() { @Test public void testCommitTableSerialization() { + UUID commitId = UUID.randomUUID(); Event event = - new Event( - "cg-connector", - EventType.COMMIT_TABLE, - new CommitTablePayload( - commitId, new TableName(Collections.singletonList("db"), "tbl"), 1L, 2L)); + new Event( + "cg-connector", + EventType.COMMIT_TABLE, + new CommitTablePayload( + commitId, new TableName(Collections.singletonList("db"), "tbl"), 1L, 2L)); byte[] data = Event.encode(event); Event result = Event.decode(data); @@ -113,9 +116,10 @@ public void testCommitTableSerialization() { @Test public void testCommitCompleteSerialization() { + UUID commitId = UUID.randomUUID(); Event event = - new Event( - "cg-connector", EventType.COMMIT_COMPLETE, new CommitCompletePayload(commitId, 2L)); + new Event( + "cg-connector", EventType.COMMIT_COMPLETE, new CommitCompletePayload(commitId, 2L)); byte[] data = Event.encode(event); Event result = Event.decode(data); diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index b937767f..4f6337c4 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -15,7 +15,6 @@ dependencies { testImplementation libs.mockito testImplementation libs.assertj - testImplementation libs.iceberg.kafka.connect.events testImplementation 'ch.qos.logback:logback-classic:1.5.3' } diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java index 99ea7039..502a01bd 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/CommitState.java @@ -45,6 +45,9 @@ public class CommitState { private UUID currentCommitId; private final IcebergSinkConfig config; + private final Comparator dateTimeComparator = + OffsetDateTime::compareTo; + public CommitState(IcebergSinkConfig config) { this.config = config; } @@ -149,18 +152,6 @@ public Map> tableCommitMap() { } public OffsetDateTime vtts(boolean partialCommit) { - - // move to static on the class - Comparator comparator = - new Comparator() { - - @Override - public int compare(OffsetDateTime o1, OffsetDateTime o2) { - return o1.compareTo(o2); - } - }; - - boolean validVtts = !partialCommit && readyBuffer.stream() @@ -173,7 +164,7 @@ public int compare(OffsetDateTime o1, OffsetDateTime o2) { readyBuffer.stream() .flatMap(event -> event.assignments().stream()) .map(TopicPartitionOffset::timestamp) - .min(comparator); + .min(dateTimeComparator); if (maybeResult.isPresent()) { result = maybeResult.get(); } else { diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java index fd8f3b0c..590cf394 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java @@ -103,7 +103,7 @@ private Payload convertPayload(io.tabular.iceberg.connect.events.Payload payload new TopicPartitionOffset( t.topic(), t.partition(), - t.offset() == null ? null : t.offset(), + t.offset(), t.timestamp() == null ? null : OffsetDateTime.ofInstant( @@ -156,7 +156,7 @@ private Payload convertCommitResponse(CommitResponsePayload payload) { .get(); Type fieldType = AvroSchemaUtil.convert(fieldSchema); int fieldId = (int) f.getObjectProp("field-id"); - convertedFields.add(Types.NestedField.optional(fieldId, f.name(), fieldType)); + convertedFields.add(Types.NestedField.of(fieldId, false, f.name(), fieldType)); } Types.StructType convertedStructType = Types.StructType.of(convertedFields); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java index fb27221c..b3365be3 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/CommitterImplTest.java @@ -18,6 +18,7 @@ */ package io.tabular.iceberg.connect.channel; +import static io.tabular.iceberg.connect.fixtures.EventTestUtil.createDataFile; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; @@ -43,11 +44,7 @@ import org.apache.commons.lang3.NotImplementedException; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileMetadata; -import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.connect.events.AvroUtil; @@ -288,24 +285,6 @@ private OffsetDateTime offsetDateTime(Long ms) { return OffsetDateTime.ofInstant(Instant.ofEpochMilli(ms), ZoneOffset.UTC); } - public static DataFile createDataFile() { - return DataFiles.builder(PartitionSpec.unpartitioned()) - .withPath(UUID.randomUUID() + ".parquet") - .withFormat(FileFormat.PARQUET) - .withFileSizeInBytes(100L) - .withRecordCount(5) - .build(); - } - - public static DeleteFile createDeleteFile() { - return FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) - .ofEqualityDeletes(1) - .withPath(UUID.randomUUID() + ".parquet") - .withFileSizeInBytes(10) - .withRecordCount(1) - .build(); - } - @Test public void testShouldRewindOffsetsToStableControlGroupConsumerOffsetsForAssignedPartitionsOnConstruction() diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java index 6961f270..c3285930 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java @@ -50,7 +50,6 @@ import org.apache.iceberg.connect.events.PayloadType; import org.apache.iceberg.connect.events.StartCommit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -184,17 +183,7 @@ public void testCommitResponseBecomesDataWrittenPartitioned() { }) .collect(Collectors.toList()); - List expectedPartitionInfo = - Lists.newArrayList( - Types.NestedField.optional(1000, "ts1_year", Types.IntegerType.get()), - Types.NestedField.optional(1001, "ts2_month", Types.IntegerType.get()), - Types.NestedField.optional(1002, "ts3_day", Types.DateType.get()), - Types.NestedField.optional(1003, "ts4_hour", Types.IntegerType.get()), - Types.NestedField.optional(1004, "i_bucket", Types.IntegerType.get()), - Types.NestedField.optional(1005, "s_trunc", Types.StringType.get()), - Types.NestedField.optional(1006, "s", Types.StringType.get())); - - assertThat(resultPartitionInfo).isEqualTo(expectedPartitionInfo); + assertThat(resultPartitionInfo).isEqualTo(spec.partitionType().fields()); } @Test From 94e9c01586018dc8aad95ed18354fb184ec7a536 Mon Sep 17 00:00:00 2001 From: Mathew Fournier Date: Tue, 21 May 2024 08:21:38 -0700 Subject: [PATCH 13/13] review comments --- .../iceberg/connect/channel/EventDecoder.java | 2 +- .../connect/channel/EventDecoderTest.java | 36 ++++++++----------- 2 files changed, 16 insertions(+), 22 deletions(-) diff --git a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java index 590cf394..7ba4f878 100644 --- a/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java +++ b/kafka-connect/src/main/java/io/tabular/iceberg/connect/channel/EventDecoder.java @@ -156,7 +156,7 @@ private Payload convertCommitResponse(CommitResponsePayload payload) { .get(); Type fieldType = AvroSchemaUtil.convert(fieldSchema); int fieldId = (int) f.getObjectProp("field-id"); - convertedFields.add(Types.NestedField.of(fieldId, false, f.name(), fieldType)); + convertedFields.add(Types.NestedField.of(fieldId, f.schema().isNullable(), f.name(), fieldType)); } Types.StructType convertedStructType = Types.StructType.of(convertedFields); diff --git a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java index c3285930..3d1ecab9 100644 --- a/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java +++ b/kafka-connect/src/test/java/io/tabular/iceberg/connect/channel/EventDecoderTest.java @@ -37,10 +37,9 @@ import java.util.Collections; import java.util.List; import java.util.UUID; -import java.util.stream.Collectors; import org.apache.avro.Schema; +import org.apache.iceberg.DataFile; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.events.CommitComplete; import org.apache.iceberg.connect.events.CommitToTable; @@ -49,8 +48,8 @@ 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.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; @@ -166,24 +165,19 @@ public void testCommitResponseBecomesDataWrittenPartitioned() { assertThat(payload.tableReference().catalog()).isEqualTo("catalog"); assertThat(payload.tableReference().identifier()).isEqualTo(TableIdentifier.of("db", "tbl")); - Schema.Field field = - payload.getSchema().getFields().get(2).schema().getTypes().stream() - .filter(s -> s.getType() != Schema.Type.NULL) - .findFirst() - .get() - .getElementType() - .getField("partition"); - List resultPartitionInfo = - field.schema().getFields().stream() - .map( - f -> { - Type fieldType = AvroSchemaUtil.convert(f.schema()); - int fieldId = (int) f.getObjectProp("field-id"); - return Types.NestedField.optional(fieldId, f.name(), fieldType); - }) - .collect(Collectors.toList()); - - assertThat(resultPartitionInfo).isEqualTo(spec.partitionType().fields()); + assertThat(payload.writeSchema()).isEqualTo( + Types.StructType.of( + Types.NestedField.required(10_300, "commit_id", Types.UUIDType.get()), + Types.NestedField.required( + 10_301, "table_reference", TableReference.ICEBERG_SCHEMA), + Types.NestedField.optional( + 10_302, + "data_files", + Types.ListType.ofRequired(10_303, DataFile.getType(spec.partitionType()))), + Types.NestedField.optional( + 10_304, + "delete_files", + Types.ListType.ofRequired(10_304, DataFile.getType(spec.partitionType()))))); } @Test