From e0a464f1cfd81823940501ecb9f0c88905981ee1 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 29 Jul 2024 15:34:32 -0700 Subject: [PATCH 01/14] Flink: improve snapshot compatibility check by comparing projected sort schema in SortKeySerializer. also add unit tests for serializer snapshot. (#10794) --- .../flink/sink/shuffle/SortKeySerializer.java | 23 +- .../TestSortKeySerializerSnapshot.java | 213 ++++++++++++++++++ .../flink/sink/shuffle/SortKeySerializer.java | 23 +- .../TestSortKeySerializerSnapshot.java | 213 ++++++++++++++++++ .../flink/sink/shuffle/SortKeySerializer.java | 31 +-- .../TestSortKeySerializerSnapshot.java | 213 ++++++++++++++++++ 6 files changed, 696 insertions(+), 20 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index 4ddc5a32d6bf..d1d75019fa2e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -24,7 +24,9 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; @@ -41,6 +43,7 @@ import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.types.CheckCompatibility; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; class SortKeySerializer extends TypeSerializer { @@ -319,12 +322,26 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } - SortKeySerializer sortKeySerializer = (SortKeySerializer) newSerializer; - if (!sortOrder.sameOrder(sortKeySerializer.sortOrder)) { + // Sort order should be identical + SortKeySerializerSnapshot newSnapshot = + (SortKeySerializerSnapshot) newSerializer.snapshotConfiguration(); + if (!sortOrder.sameOrder(newSnapshot.sortOrder)) { return TypeSerializerSchemaCompatibility.incompatible(); } - return resolveSchemaCompatibility(sortKeySerializer.schema, schema); + Set sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema newSortSchema = TypeUtil.project(newSnapshot.schema, sortFieldIds); + + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, newSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); } @Override diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java new file mode 100644 index 000000000000..012654603b04 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() throws Exception { + RowData rowData = GenericRowData.of(StringData.fromString("str"), 1); + RowDataWrapper rowDataWrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(struct); + + SortKeySerializer originalSerializer = new SortKeySerializer(SCHEMA, SORT_ORDER); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + + DataOutputSerializer output = new DataOutputSerializer(1024); + originalSerializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = restoredSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } + + @Test + public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + SortKeySerializer newSerializer = new SortKeySerializer(schema, sortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index 4ddc5a32d6bf..d1d75019fa2e 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -24,7 +24,9 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; @@ -41,6 +43,7 @@ import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.types.CheckCompatibility; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; class SortKeySerializer extends TypeSerializer { @@ -319,12 +322,26 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } - SortKeySerializer sortKeySerializer = (SortKeySerializer) newSerializer; - if (!sortOrder.sameOrder(sortKeySerializer.sortOrder)) { + // Sort order should be identical + SortKeySerializerSnapshot newSnapshot = + (SortKeySerializerSnapshot) newSerializer.snapshotConfiguration(); + if (!sortOrder.sameOrder(newSnapshot.sortOrder)) { return TypeSerializerSchemaCompatibility.incompatible(); } - return resolveSchemaCompatibility(sortKeySerializer.schema, schema); + Set sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema newSortSchema = TypeUtil.project(newSnapshot.schema, sortFieldIds); + + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, newSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); } @Override diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java new file mode 100644 index 000000000000..012654603b04 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() throws Exception { + RowData rowData = GenericRowData.of(StringData.fromString("str"), 1); + RowDataWrapper rowDataWrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(struct); + + SortKeySerializer originalSerializer = new SortKeySerializer(SCHEMA, SORT_ORDER); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + + DataOutputSerializer output = new DataOutputSerializer(1024); + originalSerializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = restoredSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } + + @Test + public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + SortKeySerializer newSerializer = new SortKeySerializer(schema, sortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index 9c0e65a1fc49..b82fc8250763 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -24,8 +24,9 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.UUID; -import org.apache.flink.annotation.VisibleForTesting; +import java.util.stream.Collectors; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; @@ -41,6 +42,7 @@ import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.types.CheckCompatibility; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; class SortKeySerializer extends TypeSerializer { @@ -319,12 +321,25 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } + // Sort order should be identical SortKeySerializerSnapshot oldSnapshot = (SortKeySerializerSnapshot) oldSerializerSnapshot; if (!sortOrder.sameOrder(oldSnapshot.sortOrder)) { return TypeSerializerSchemaCompatibility.incompatible(); } - return resolveSchemaCompatibility(oldSnapshot.schema, schema); + Set sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema oldSortSchema = TypeUtil.project(oldSnapshot.schema, sortFieldIds); + + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, oldSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); } @Override @@ -340,17 +355,5 @@ private void readV1(DataInputView in) throws IOException { this.schema = SchemaParser.fromJson(schemaJson); this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); } - - @VisibleForTesting - static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( - Schema readSchema, Schema writeSchema) { - List compatibilityErrors = - CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); - if (compatibilityErrors.isEmpty()) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } - - return TypeSerializerSchemaCompatibility.incompatible(); - } } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java new file mode 100644 index 000000000000..c0f688f2589e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() throws Exception { + RowData rowData = GenericRowData.of(StringData.fromString("str"), 1); + RowDataWrapper rowDataWrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(struct); + + SortKeySerializer originalSerializer = new SortKeySerializer(SCHEMA, SORT_ORDER); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + + DataOutputSerializer output = new DataOutputSerializer(1024); + originalSerializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = restoredSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } + + @Test + public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} From f7585932a6d89b04c0d45b5f9dfe6f45483efd0b Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 29 Jul 2024 15:35:18 -0700 Subject: [PATCH 02/14] Flink: support limit pushdown in FLIP-27 source (#10748) --- .../iceberg/flink/source/IcebergSource.java | 3 +- .../source/reader/LimitableDataIterator.java | 56 +++++++++++++ .../flink/source/reader/RecordLimiter.java | 45 ++++++++++ .../source/reader/RowDataReaderFunction.java | 40 ++++++++- .../flink/source/TestFlinkSourceConfig.java | 7 +- .../reader/TestLimitableDataIterator.java | 84 +++++++++++++++++++ 6 files changed, 228 insertions(+), 7 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 48201ea09359..ccbd0d9997ed 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -521,7 +521,8 @@ public IcebergSource build() { context.caseSensitive(), table.io(), table.encryption(), - context.filters()); + context.filters(), + context.limit()); this.readerFunction = (ReaderFunction) rowDataReaderFunction; } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator extends DataIterator { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java index 5d0a00954e7a..c9208a0e1834 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -39,6 +39,9 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction { private final FileIO io; private final EncryptionManager encryption; private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; public RowDataReaderFunction( ReadableConfig config, @@ -49,6 +52,28 @@ public RowDataReaderFunction( FileIO io, EncryptionManager encryption, List filters) { + this( + config, + tableSchema, + projectedSchema, + nameMapping, + caseSensitive, + io, + encryption, + filters, + -1L); + } + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { super( new ArrayPoolDataIteratorBatcher<>( config, @@ -61,19 +86,30 @@ public RowDataReaderFunction( this.io = io; this.encryption = encryption; this.filters = filters; + this.limit = limit; } @Override public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( + return new LimitableDataIterator<>( new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), split.task(), io, - encryption); + encryption, + lazyLimiter()); } private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); return projectedSchema == null ? tableSchema : projectedSchema; } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 8131bd7ab0d3..14131d9e96d5 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; import org.apache.flink.types.Row; @@ -49,11 +48,11 @@ public void testFlinkHintConfig() { @TestTemplate public void testReadOptionHierarchy() { - // TODO: FLIP-27 source doesn't implement limit pushdown yet - assumeThat(useFlip27Source).isFalse(); - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + totalRecords = 3 * 2; + } + + @ParameterizedTest + @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) + public void testUnlimited(long limit) { + LimitableDataIterator dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} From 96587abf7fbbeb9c728d60cbdc7bdd7e2096dad5 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 31 Jul 2024 00:18:39 +0900 Subject: [PATCH 03/14] Flink: Remove MiniClusterResource (#10817) --- .../iceberg/flink/MiniClusterResource.java | 53 --------------- .../org/apache/iceberg/flink/TestBase.java | 2 +- .../iceberg/flink/TestFlinkTableSink.java | 2 +- .../apache/iceberg/flink/TestFlinkUpsert.java | 2 +- .../iceberg/flink/TestIcebergConnector.java | 2 +- ...TestBucketPartitionerFlinkIcebergSink.java | 4 +- .../flink/sink/TestFlinkIcebergSink.java | 7 +- .../sink/TestFlinkIcebergSinkBranch.java | 4 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 5 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 4 +- .../flink/source/ChangeLogTableTestBase.java | 4 +- .../iceberg/flink/source/TestFlinkScan.java | 2 +- ...stIcebergSourceWithWatermarkExtractor.java | 2 +- .../iceberg/flink/source/TestSqlBase.java | 2 +- .../flink/source/TestStreamScanSql.java | 4 +- .../iceberg/flink/MiniClusterResource.java | 68 ------------------- .../org/apache/iceberg/flink/TestBase.java | 2 +- .../iceberg/flink/TestFlinkTableSink.java | 2 +- .../apache/iceberg/flink/TestFlinkUpsert.java | 2 +- .../iceberg/flink/TestIcebergConnector.java | 2 +- ...TestBucketPartitionerFlinkIcebergSink.java | 4 +- .../flink/sink/TestFlinkIcebergSink.java | 7 +- .../sink/TestFlinkIcebergSinkBranch.java | 4 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 5 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 4 +- .../flink/source/ChangeLogTableTestBase.java | 4 +- .../iceberg/flink/source/TestFlinkScan.java | 2 +- .../source/TestIcebergSourceContinuous.java | 2 +- ...stIcebergSourceWithWatermarkExtractor.java | 2 +- .../iceberg/flink/source/TestSqlBase.java | 2 +- .../flink/source/TestStreamScanSql.java | 4 +- .../iceberg/flink/MiniClusterResource.java | 68 ------------------- .../org/apache/iceberg/flink/TestBase.java | 2 +- .../iceberg/flink/TestFlinkTableSink.java | 2 +- .../apache/iceberg/flink/TestFlinkUpsert.java | 2 +- .../iceberg/flink/TestIcebergConnector.java | 2 +- .../operator/OperatorTestBase.java | 4 +- ...TestBucketPartitionerFlinkIcebergSink.java | 4 +- .../flink/sink/TestFlinkIcebergSink.java | 7 +- .../sink/TestFlinkIcebergSinkBranch.java | 4 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 5 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 4 +- .../flink/source/ChangeLogTableTestBase.java | 4 +- .../iceberg/flink/source/TestFlinkScan.java | 2 +- ...stIcebergSourceWithWatermarkExtractor.java | 2 +- ...estIcebergSpeculativeExecutionSupport.java | 2 +- .../iceberg/flink/source/TestSqlBase.java | 2 +- .../flink/source/TestStreamScanSql.java | 4 +- 48 files changed, 70 insertions(+), 265 deletions(-) delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java delete mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 45af9241b743..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,53 +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 org.apache.iceberg.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 6367a064f283..a74226092f38 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -43,7 +43,7 @@ public abstract class TestBase extends TestBaseUtils { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index b7fce104f490..b73300e3f170 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -91,7 +91,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index 5674c83e40b8..d52d54e159e6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -75,7 +75,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index 013b98e3b82b..b709c0058f7d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -176,7 +176,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index dc3eb93280df..ba0ea867ffb7 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; @@ -63,7 +63,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 8cad35c859c4..61ab087f2ca3 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -44,7 +44,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; @@ -62,7 +61,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -113,7 +112,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -271,7 +270,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 3edaafca0e42..441b5ed2a4ae 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -36,7 +36,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -88,7 +88,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 55909874ccce..577c54976b9a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -38,7 +38,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +56,7 @@ @Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -89,7 +88,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index ffeab673386d..0b0c55f51c32 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -31,7 +31,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -71,7 +71,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index d3748e008b8e..5dfbbe3abe73 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestBase; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(400) .setMaxParallelism(1) .setParallelism(1); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 049ddf9e3f1e..cf6b233dcec6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -64,7 +64,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkScan { @RegisterExtension - protected static MiniClusterExtension miniClusterResource = + protected static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 5e6a2b3caec6..70889f4f76aa 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; import java.nio.file.Path; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 8013bce3f415..f9b776397cfc 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -52,7 +52,7 @@ /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index d6cf6791270e..57ee7baf202c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -42,7 +42,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; @@ -70,7 +70,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); StreamTableEnvironment streamTableEnv = diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 399d7aaff64c..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,68 +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 org.apache.iceberg.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } - - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled( - InMemoryReporter inMemoryReporter) { - Configuration configuration = - new Configuration(MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - inMemoryReporter.addToConfiguration(configuration); - - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(MiniClusterResource.DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(MiniClusterResource.DEFAULT_PARALLELISM) - .setConfiguration(configuration) - .build()); - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 6367a064f283..a74226092f38 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -43,7 +43,7 @@ public abstract class TestBase extends TestBaseUtils { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index b7fce104f490..b73300e3f170 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -91,7 +91,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index 5674c83e40b8..d52d54e159e6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -75,7 +75,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index 013b98e3b82b..b709c0058f7d 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -176,7 +176,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index dc3eb93280df..ba0ea867ffb7 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; @@ -63,7 +63,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 8cad35c859c4..61ab087f2ca3 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -44,7 +44,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; @@ -62,7 +61,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -113,7 +112,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -271,7 +270,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 3edaafca0e42..441b5ed2a4ae 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -36,7 +36,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -88,7 +88,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 55909874ccce..577c54976b9a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -38,7 +38,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +56,7 @@ @Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -89,7 +88,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index ffeab673386d..0b0c55f51c32 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -31,7 +31,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -71,7 +71,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index d3748e008b8e..5dfbbe3abe73 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestBase; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(400) .setMaxParallelism(1) .setParallelism(1); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 049ddf9e3f1e..cf6b233dcec6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -64,7 +64,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkScan { @RegisterExtension - protected static MiniClusterExtension miniClusterResource = + protected static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 749cbf89338a..e0e2bf5e61e2 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -67,7 +67,7 @@ public class TestIcebergSourceContinuous { @TempDir protected Path temporaryFolder; @RegisterExtension - public static MiniClusterExtension miniClusterExtension = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); @RegisterExtension diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 5e6a2b3caec6..70889f4f76aa 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; import java.nio.file.Path; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 8013bce3f415..f9b776397cfc 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -52,7 +52,7 @@ /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index d6cf6791270e..57ee7baf202c 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -42,7 +42,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; @@ -70,7 +70,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); StreamTableEnvironment streamTableEnv = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 399d7aaff64c..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,68 +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 org.apache.iceberg.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } - - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled( - InMemoryReporter inMemoryReporter) { - Configuration configuration = - new Configuration(MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - inMemoryReporter.addToConfiguration(configuration); - - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(MiniClusterResource.DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(MiniClusterResource.DEFAULT_PARALLELISM) - .setConfiguration(configuration) - .build()); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 773d22e19e64..633690044692 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -44,7 +44,7 @@ public abstract class TestBase extends TestBaseUtils { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index 3f66174049a4..a0341e6834d4 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -91,7 +91,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index baf13017ff99..c5becb6caca1 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -75,7 +75,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index c978ec6f8bd4..fdb0e0cf19df 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -176,7 +176,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 269ae681b02e..272e0b693fd3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -34,7 +34,7 @@ class OperatorTestBase { static final String TABLE_NAME = "test_table"; @RegisterExtension - protected static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index dc3eb93280df..ba0ea867ffb7 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; @@ -63,7 +63,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 8cad35c859c4..61ab087f2ca3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -44,7 +44,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; @@ -62,7 +61,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -113,7 +112,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -271,7 +270,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 3edaafca0e42..441b5ed2a4ae 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -36,7 +36,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -88,7 +88,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 55909874ccce..577c54976b9a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -38,7 +38,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +56,7 @@ @Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -89,7 +88,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index ffeab673386d..0b0c55f51c32 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -31,7 +31,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -71,7 +71,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index d3748e008b8e..5dfbbe3abe73 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestBase; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(400) .setMaxParallelism(1) .setParallelism(1); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 049ddf9e3f1e..cf6b233dcec6 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -64,7 +64,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkScan { @RegisterExtension - protected static MiniClusterExtension miniClusterResource = + protected static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 5e6a2b3caec6..70889f4f76aa 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; import java.nio.file.Path; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 3285a16a1277..b21010a91bed 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -55,7 +55,7 @@ public class TestIcebergSpeculativeExecutionSupport extends TestBase { private static final int NUM_TASK_SLOTS = 3; @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUM_TASK_MANAGERS) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 8013bce3f415..f9b776397cfc 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -52,7 +52,7 @@ /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index fcf5c1479df5..97ed4ca1e93f 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -42,7 +42,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; @@ -70,7 +70,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); StreamTableEnvironment streamTableEnv = From 4d1ceac275859df0ab16a8f919a76fcd3b97d622 Mon Sep 17 00:00:00 2001 From: liu yang Date: Wed, 31 Jul 2024 05:25:56 +0800 Subject: [PATCH 04/14] Docs: Use link addresses instead of descriptions in releases.md (#10815) --- site/docs/releases.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 73d8bc32a071..cc29857ed802 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -83,7 +83,7 @@ The 1.6.0 release contains fixes, dependency updates, and new features (like Kaf - Add EnvironmentContext to commit summary ([\#9273](https://github.com/apache/iceberg/pull/9273)) - Add explicit JSON parser for ConfigResponse ([\#9952](https://github.com/apache/iceberg/pull/9952)) - Calling rewrite_position_delete_files fails on tables with more than 1k columns ([\#10020](https://github.com/apache/iceberg/pull/10020)) - - Expose table incremental scan for appends API in SerializableTable ([\#10682](Expose table incremental scan for appends API in SerializableTable)) + - Expose table incremental scan for appends API in SerializableTable ([\#10682](https://github.com/apache/iceberg/pull/10682)) - Fix NPE during conflict handling of NULL partitions ([\#10680](https://github.com/apache/iceberg/pull/10680)) - Fix ParallelIterable memory leak where queue continues to be populated even after iterator close ([\#9402](https://github.com/apache/iceberg/pull/9402)) - Fix logging table name in scanning metadata table ([\#10141](https://github.com/apache/iceberg/pull/10141)) @@ -100,7 +100,7 @@ The 1.6.0 release contains fixes, dependency updates, and new features (like Kaf - REST Catalog: Handles possible heap data corruption of OAuth2Util.AuthSession#headers ([\#10615](https://github.com/apache/iceberg/pull/10615)) - REST Catalog: Handles potential NPE in RESTSessionCatalog#newSessionCache ([\#10607](https://github.com/apache/iceberg/pull/10607)) - REST Catalog: Introduce AuthConfig ([\#10161](https://github.com/apache/iceberg/pull/10161)) - - REST Catalog: Mark 502 and 504 statuses as retryable to the REST exponential retry strategy ([\#9885](Mark 502 and 504 statuses as retryable to the REST exponential retry strategy)) + - REST Catalog: Mark 502 and 504 statuses as retryable to the REST exponential retry strategy ([\#9885](https://github.com/apache/iceberg/pull/9885)) - REST Catalog: disallow overriding "credential" in table sessions ([\#10345](https://github.com/apache/iceberg/pull/10345)) - REST Catalog: fix incorrect token refresh thread name ([\#10223](https://github.com/apache/iceberg/pull/10223)) - REST Catalog: fix spurious warning when shutting down refresh executor ([\#10087](https://github.com/apache/iceberg/pull/10087)) From 0ff90e7732574fa1a1e094bb66c7c3793e3d1ebb Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Tue, 30 Jul 2024 14:26:55 -0700 Subject: [PATCH 05/14] Build: Declare avro as an api dependency of iceberg-core (#10573) iceberg-core should declare an api dependency on avro. For example, the public class `org.apache.iceberg.PartitionData` extends avro-specific types. In addition, there are public methods that deal with avro types, see `org.apache.iceberg.avro.AvroSchemaUtil` --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 09a04e3ae977..a0b1a2e018bf 100644 --- a/build.gradle +++ b/build.gradle @@ -347,7 +347,7 @@ project(':iceberg-core') { annotationProcessor libs.immutables.value compileOnly libs.immutables.value - implementation(libs.avro.avro) { + api(libs.avro.avro) { exclude group: 'org.tukaani' // xz compression is not supported } From 72b39ab91dfa04d713552e70b009c24510a1cd07 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Tue, 30 Jul 2024 14:49:44 -0700 Subject: [PATCH 06/14] Flink: backport PR #10748 for limit pushdown (#10813) --- .../iceberg/flink/source/IcebergSource.java | 3 +- .../source/reader/LimitableDataIterator.java | 56 +++++++++++++ .../flink/source/reader/RecordLimiter.java | 45 ++++++++++ .../source/reader/RowDataReaderFunction.java | 40 ++++++++- .../flink/source/TestFlinkSourceConfig.java | 7 +- .../reader/TestLimitableDataIterator.java | 84 +++++++++++++++++++ .../iceberg/flink/source/IcebergSource.java | 3 +- .../source/reader/LimitableDataIterator.java | 56 +++++++++++++ .../flink/source/reader/RecordLimiter.java | 45 ++++++++++ .../source/reader/RowDataReaderFunction.java | 40 ++++++++- .../flink/source/TestFlinkSourceConfig.java | 7 +- .../reader/TestLimitableDataIterator.java | 84 +++++++++++++++++++ 12 files changed, 456 insertions(+), 14 deletions(-) create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 48201ea09359..ccbd0d9997ed 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -521,7 +521,8 @@ public IcebergSource build() { context.caseSensitive(), table.io(), table.encryption(), - context.filters()); + context.filters(), + context.limit()); this.readerFunction = (ReaderFunction) rowDataReaderFunction; } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator extends DataIterator { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java index 5d0a00954e7a..c9208a0e1834 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -39,6 +39,9 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction { private final FileIO io; private final EncryptionManager encryption; private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; public RowDataReaderFunction( ReadableConfig config, @@ -49,6 +52,28 @@ public RowDataReaderFunction( FileIO io, EncryptionManager encryption, List filters) { + this( + config, + tableSchema, + projectedSchema, + nameMapping, + caseSensitive, + io, + encryption, + filters, + -1L); + } + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { super( new ArrayPoolDataIteratorBatcher<>( config, @@ -61,19 +86,30 @@ public RowDataReaderFunction( this.io = io; this.encryption = encryption; this.filters = filters; + this.limit = limit; } @Override public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( + return new LimitableDataIterator<>( new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), split.task(), io, - encryption); + encryption, + lazyLimiter()); } private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); return projectedSchema == null ? tableSchema : projectedSchema; } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 8131bd7ab0d3..14131d9e96d5 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; import org.apache.flink.types.Row; @@ -49,11 +48,11 @@ public void testFlinkHintConfig() { @TestTemplate public void testReadOptionHierarchy() { - // TODO: FLIP-27 source doesn't implement limit pushdown yet - assumeThat(useFlip27Source).isFalse(); - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + totalRecords = 3 * 2; + } + + @ParameterizedTest + @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) + public void testUnlimited(long limit) { + LimitableDataIterator dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 48201ea09359..ccbd0d9997ed 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -521,7 +521,8 @@ public IcebergSource build() { context.caseSensitive(), table.io(), table.encryption(), - context.filters()); + context.filters(), + context.limit()); this.readerFunction = (ReaderFunction) rowDataReaderFunction; } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator extends DataIterator { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java index 5d0a00954e7a..c9208a0e1834 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -39,6 +39,9 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction { private final FileIO io; private final EncryptionManager encryption; private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; public RowDataReaderFunction( ReadableConfig config, @@ -49,6 +52,28 @@ public RowDataReaderFunction( FileIO io, EncryptionManager encryption, List filters) { + this( + config, + tableSchema, + projectedSchema, + nameMapping, + caseSensitive, + io, + encryption, + filters, + -1L); + } + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { super( new ArrayPoolDataIteratorBatcher<>( config, @@ -61,19 +86,30 @@ public RowDataReaderFunction( this.io = io; this.encryption = encryption; this.filters = filters; + this.limit = limit; } @Override public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( + return new LimitableDataIterator<>( new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), split.task(), io, - encryption); + encryption, + lazyLimiter()); } private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); return projectedSchema == null ? tableSchema : projectedSchema; } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 8131bd7ab0d3..14131d9e96d5 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; import org.apache.flink.types.Row; @@ -49,11 +48,11 @@ public void testFlinkHintConfig() { @TestTemplate public void testReadOptionHierarchy() { - // TODO: FLIP-27 source doesn't implement limit pushdown yet - assumeThat(useFlip27Source).isFalse(); - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + totalRecords = 3 * 2; + } + + @ParameterizedTest + @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) + public void testUnlimited(long limit) { + LimitableDataIterator dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} From 76dba8fe83c4496318cc34436e610cf50f43054d Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Wed, 31 Jul 2024 12:59:13 +0200 Subject: [PATCH 07/14] Docs: Fix header for entries metadata table (#10826) --- docs/docs/spark-queries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index b606d849a692..494ca698533a 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -288,6 +288,7 @@ order by made_current_at; | 2019-02-09 16:24:30.13 | delete | 29641004024753 | false | application_1520379288616_151109 | | 2019-02-09 16:32:47.336 | append | 57897183625154 | true | application_1520379288616_155055 | | 2019-02-08 03:47:55.948 | overwrite | 51792995261850 | true | application_1520379288616_152431 | + ### Entries To show all the table's current manifest entries for both data and delete files. From 506fee492620bc8e13d7da4f104462fb97ceef82 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 31 Jul 2024 08:41:20 -0700 Subject: [PATCH 08/14] Spark 3.5: Support Reporting Column Stats (#10659) Co-authored-by: Karuppayya Rajendran --- .../apache/iceberg/spark/SparkReadConf.java | 8 + .../iceberg/spark/SparkSQLProperties.java | 4 + .../spark/source/SparkChangelogScan.java | 2 +- .../spark/source/SparkColumnStatistics.java | 88 +++++++++ .../iceberg/spark/source/SparkScan.java | 54 +++++- .../apache/iceberg/spark/source/Stats.java | 12 +- .../iceberg/spark/source/TestSparkScan.java | 183 ++++++++++++++++++ 7 files changed, 346 insertions(+), 5 deletions(-) create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index bd29fb0d6d42..67e9d78ada4d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -347,4 +347,12 @@ private boolean executorCacheLocalityEnabledInternal() { .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT) .parse(); } + + public boolean reportColumnStats() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.REPORT_COLUMN_STATS) + .defaultValue(SparkSQLProperties.REPORT_COLUMN_STATS_DEFAULT) + .parse(); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index ea8f6fe0718b..77ae796ffb76 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -90,4 +90,8 @@ private SparkSQLProperties() {} public static final String EXECUTOR_CACHE_LOCALITY_ENABLED = "spark.sql.iceberg.executor-cache.locality.enabled"; public static final boolean EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT = false; + + // Controls whether to report available column statistics to Spark for query optimization. + public static final String REPORT_COLUMN_STATS = "spark.sql.iceberg.report-column-stats"; + public static final boolean REPORT_COLUMN_STATS_DEFAULT = true; } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java index 7cde3e1fbe11..71b53d70262f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java @@ -88,7 +88,7 @@ class SparkChangelogScan implements Scan, SupportsReportStatistics { public Statistics estimateStatistics() { long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum(); long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount); - return new Stats(sizeInBytes, rowsCount); + return new Stats(sizeInBytes, rowsCount, Collections.emptyMap()); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java new file mode 100644 index 000000000000..faaff3631d7c --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java @@ -0,0 +1,88 @@ +/* + * 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.spark.source; + +import java.util.Optional; +import java.util.OptionalLong; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; +import org.apache.spark.sql.connector.read.colstats.Histogram; + +class SparkColumnStatistics implements ColumnStatistics { + + private final OptionalLong distinctCount; + private final Optional min; + private final Optional max; + private final OptionalLong nullCount; + private final OptionalLong avgLen; + private final OptionalLong maxLen; + private final Optional histogram; + + SparkColumnStatistics( + Long distinctCount, + Object min, + Object max, + Long nullCount, + Long avgLen, + Long maxLen, + Histogram histogram) { + this.distinctCount = + (distinctCount == null) ? OptionalLong.empty() : OptionalLong.of(distinctCount); + this.min = Optional.ofNullable(min); + this.max = Optional.ofNullable(max); + this.nullCount = (nullCount == null) ? OptionalLong.empty() : OptionalLong.of(nullCount); + this.avgLen = (avgLen == null) ? OptionalLong.empty() : OptionalLong.of(avgLen); + this.maxLen = (maxLen == null) ? OptionalLong.empty() : OptionalLong.of(maxLen); + this.histogram = Optional.ofNullable(histogram); + } + + @Override + public OptionalLong distinctCount() { + return distinctCount; + } + + @Override + public Optional min() { + return min; + } + + @Override + public Optional max() { + return max; + } + + @Override + public OptionalLong nullCount() { + return nullCount; + } + + @Override + public OptionalLong avgLen() { + return avgLen; + } + + @Override + public OptionalLong maxLen() { + return maxLen; + } + + @Override + public Optional histogram() { + return histogram; + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 6efe8a080bde..8b88cf49c692 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -23,15 +23,19 @@ import java.util.Map; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.iceberg.BlobMetadata; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -75,22 +79,28 @@ import org.apache.iceberg.util.TableScanUtil; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.expressions.FieldReference; +import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.connector.metric.CustomMetric; import org.apache.spark.sql.connector.metric.CustomTaskMetric; import org.apache.spark.sql.connector.read.Batch; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsReportStatistics; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; abstract class SparkScan implements Scan, SupportsReportStatistics { private static final Logger LOG = LoggerFactory.getLogger(SparkScan.class); + private static final String NDV_KEY = "ndv"; private final JavaSparkContext sparkContext; private final Table table; + private final SparkSession spark; private final SparkReadConf readConf; private final boolean caseSensitive; private final Schema expectedSchema; @@ -111,6 +121,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch()); SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema); + this.spark = spark; this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); this.table = table; this.readConf = readConf; @@ -175,7 +186,43 @@ public Statistics estimateStatistics() { protected Statistics estimateStatistics(Snapshot snapshot) { // its a fresh table, no data if (snapshot == null) { - return new Stats(0L, 0L); + return new Stats(0L, 0L, Collections.emptyMap()); + } + + boolean cboEnabled = + Boolean.parseBoolean(spark.conf().get(SQLConf.CBO_ENABLED().key(), "false")); + Map colStatsMap = Collections.emptyMap(); + if (readConf.reportColumnStats() && cboEnabled) { + colStatsMap = Maps.newHashMap(); + List files = table.statisticsFiles(); + if (!files.isEmpty()) { + List metadataList = (files.get(0)).blobMetadata(); + + for (BlobMetadata blobMetadata : metadataList) { + int id = blobMetadata.fields().get(0); + String colName = table.schema().findColumnName(id); + NamedReference ref = FieldReference.column(colName); + + Long ndv = null; + if (blobMetadata + .type() + .equals(org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1)) { + String ndvStr = blobMetadata.properties().get(NDV_KEY); + if (!Strings.isNullOrEmpty(ndvStr)) { + ndv = Long.parseLong(ndvStr); + } else { + LOG.debug("ndv is not set in BlobMetadata for column {}", colName); + } + } else { + LOG.debug("DataSketch blob is not available for column {}", colName); + } + + ColumnStatistics colStats = + new SparkColumnStatistics(ndv, null, null, null, null, null, null); + + colStatsMap.put(ref, colStats); + } + } } // estimate stats using snapshot summary only for partitioned tables @@ -186,12 +233,13 @@ protected Statistics estimateStatistics(Snapshot snapshot) { snapshot.snapshotId(), table.name()); long totalRecords = totalRecords(snapshot); - return new Stats(SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords); + return new Stats( + SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords, colStatsMap); } long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum(); long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount); - return new Stats(sizeInBytes, rowsCount); + return new Stats(sizeInBytes, rowsCount, colStatsMap); } private long totalRecords(Snapshot snapshot) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java index ddf6ca834d9b..ccf523cb4b05 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java @@ -18,16 +18,21 @@ */ package org.apache.iceberg.spark.source; +import java.util.Map; import java.util.OptionalLong; +import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; class Stats implements Statistics { private final OptionalLong sizeInBytes; private final OptionalLong numRows; + private final Map colstats; - Stats(long sizeInBytes, long numRows) { + Stats(long sizeInBytes, long numRows, Map colstats) { this.sizeInBytes = OptionalLong.of(sizeInBytes); this.numRows = OptionalLong.of(numRows); + this.colstats = colstats; } @Override @@ -39,4 +44,9 @@ public OptionalLong sizeInBytes() { public OptionalLong numRows() { return numRows; } + + @Override + public Map columnStats() { + return colstats; + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index d539b0123951..7d5475ff919e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.source; +import static org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createPartitionedTable; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createUnpartitionedTable; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToDayOrdinal; @@ -28,14 +29,22 @@ import static org.apache.spark.sql.functions.expr; import static org.assertj.core.api.Assertions.assertThat; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +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.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.spark.functions.BucketFunction; import org.apache.iceberg.spark.functions.DaysFunction; @@ -44,6 +53,7 @@ import org.apache.iceberg.spark.functions.TruncateFunction; import org.apache.iceberg.spark.functions.YearsFunction; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; @@ -60,6 +70,8 @@ import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.junit.jupiter.api.AfterEach; @@ -130,6 +142,157 @@ public void testEstimatedRowCount() throws NoSuchTableException { assertThat(stats.numRows().getAsLong()).isEqualTo(10000L); } + @TestTemplate + public void testTableWithoutColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + // The expected col NDVs are nulls + withSQLConf( + reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); + } + + @TestTemplate + public void testTableWithOneColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedOneNDV = Maps.newHashMap(); + expectedOneNDV.put("id", 4L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedOneNDV)); + } + + @TestTemplate + public void testTableWithTwoColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")), + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(2), + ImmutableMap.of("ndv", "2")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedTwoNDVs = Maps.newHashMap(); + expectedTwoNDVs.put("id", 4L); + expectedTwoNDVs.put("data", 2L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedTwoNDVs)); + } + @TestTemplate public void testUnpartitionedYears() throws Exception { createUnpartitionedTable(spark, tableName); @@ -734,6 +897,26 @@ private Expression[] expressions(Expression... expressions) { return expressions; } + private void checkColStatisticsNotReported(SparkScan scan, long expectedRowCount) { + Statistics stats = scan.estimateStatistics(); + assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); + + Map columnStats = stats.columnStats(); + assertThat(columnStats.isEmpty()); + } + + private void checkColStatisticsReported( + SparkScan scan, long expectedRowCount, Map expectedNDVs) { + Statistics stats = scan.estimateStatistics(); + assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); + + Map columnStats = stats.columnStats(); + for (Map.Entry entry : expectedNDVs.entrySet()) { + assertThat(columnStats.get(FieldReference.column(entry.getKey())).distinctCount().getAsLong()) + .isEqualTo(entry.getValue()); + } + } + private static LiteralValue intLit(int value) { return LiteralValue.apply(value, DataTypes.IntegerType); } From 84c91251738cb86f741952bd1b23daa45c80d2aa Mon Sep 17 00:00:00 2001 From: Venkata krishnan Sowrirajan Date: Thu, 1 Aug 2024 00:22:51 -0700 Subject: [PATCH 09/14] Flink: Backport #10548 to v1.18 and v1.17 (#10776) --- .../enumerator/AbstractIcebergEnumerator.java | 11 +- ...estIcebergSpeculativeExecutionSupport.java | 184 ++++++++++++++++++ .../enumerator/AbstractIcebergEnumerator.java | 11 +- ...estIcebergSpeculativeExecutionSupport.java | 184 ++++++++++++++++++ ...estIcebergSpeculativeExecutionSupport.java | 2 +- 5 files changed, 389 insertions(+), 3 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 3aca390755ed..801baf77a612 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; import org.apache.iceberg.flink.source.assigner.GetSplitResult; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -41,7 +42,8 @@ * resolved */ abstract class AbstractIcebergEnumerator - implements SplitEnumerator { + implements SplitEnumerator, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext enumeratorContext; @@ -93,6 +95,13 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } } + // Flink's SourceCoordinator already keeps track of subTask to splits mapping. + // It already takes care of re-assigning splits to speculated attempts as well. + @Override + public void handleSourceEvent(int subTaskId, int attemptNumber, SourceEvent sourceEvent) { + handleSourceEvent(subTaskId, sourceEvent); + } + @Override public void addSplitsBack(List splits, int subtaskId) { LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..95d0b90b6ca9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SlowTaskDetectorOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .setConfiguration(configure()) + .build()); + + private StreamTableEnvironment tEnv; + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String INPUT_TABLE_NAME = "test_table"; + private static final String OUTPUT_TABLE_NAME = "sink_table"; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configure()); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + } + } + + return tEnv; + } + + @BeforeEach + public void before() throws IOException { + String warehouse = + String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + + sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); + sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); + sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); + sql("DROP DATABASE %s", DATABASE_NAME); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testSpeculativeExecution() throws Exception { + Table table = + tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME)); + DataStream slowStream = + tEnv.toDataStream(table, Row.class) + .map(new TestingMap()) + .name("test_map") + .returns( + Types.ROW_NAMED( + new String[] {"i", "j", "subTask", "attempt"}, + Types.INT, + Types.INT, + Types.INT, + Types.INT)) + .setParallelism(NUM_TASK_SLOTS); + + tEnv.fromDataStream(slowStream) + .executeInsert(String.format("%s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)) + .await(); + + List output = sql(String.format("SELECT * FROM %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)); + + // Ensure that all subTasks has attemptNum > 0 + assertThat(output.stream().map(x -> x.getField(3)).collect(Collectors.toSet())).contains(1); + + // Ensure the test_table rows are returned exactly the same after the slow map task from the + // sink_table + assertSameElements( + output.stream().map(x -> Row.of(x.getField(0), x.getField(1))).collect(Collectors.toList()), + Arrays.asList(Row.of(1, -1), Row.of(2, -1), Row.of(3, -1))); + } + + /** A testing map function that simulates the slow task. */ + private static class TestingMap extends RichMapFunction { + @Override + public Row map(Row row) throws Exception { + // Put the subtasks with the first attempt to sleep to trigger speculative + // execution + if (getRuntimeContext().getAttemptNumber() <= 0) { + Thread.sleep(Integer.MAX_VALUE); + } + + Row output = + Row.of( + row.getField(0), + row.getField(1), + getRuntimeContext().getIndexOfThisSubtask(), + getRuntimeContext().getAttemptNumber()); + + return output; + } + } + + private static Configuration configure() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + configuration.set(RestOptions.BIND_PORT, "0"); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + + // Use FLIP-27 source + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + + // for speculative execution + configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); + + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); + configuration.set( + SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); + configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); + + return configuration; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 6c9a855bc149..280a126a46ce 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; import org.apache.iceberg.flink.source.assigner.GetSplitResult; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -37,7 +38,8 @@ import org.slf4j.LoggerFactory; abstract class AbstractIcebergEnumerator - implements SplitEnumerator { + implements SplitEnumerator, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext enumeratorContext; @@ -95,6 +97,13 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } } + // Flink's SourceCoordinator already keeps track of subTask to splits mapping. + // It already takes care of re-assigning splits to speculated attempts as well. + @Override + public void handleSourceEvent(int subTaskId, int attemptNumber, SourceEvent sourceEvent) { + handleSourceEvent(subTaskId, sourceEvent); + } + @Override public void addSplitsBack(List splits, int subtaskId) { LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..95d0b90b6ca9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SlowTaskDetectorOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .setConfiguration(configure()) + .build()); + + private StreamTableEnvironment tEnv; + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String INPUT_TABLE_NAME = "test_table"; + private static final String OUTPUT_TABLE_NAME = "sink_table"; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configure()); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + } + } + + return tEnv; + } + + @BeforeEach + public void before() throws IOException { + String warehouse = + String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + + sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); + sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); + sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); + sql("DROP DATABASE %s", DATABASE_NAME); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testSpeculativeExecution() throws Exception { + Table table = + tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME)); + DataStream slowStream = + tEnv.toDataStream(table, Row.class) + .map(new TestingMap()) + .name("test_map") + .returns( + Types.ROW_NAMED( + new String[] {"i", "j", "subTask", "attempt"}, + Types.INT, + Types.INT, + Types.INT, + Types.INT)) + .setParallelism(NUM_TASK_SLOTS); + + tEnv.fromDataStream(slowStream) + .executeInsert(String.format("%s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)) + .await(); + + List output = sql(String.format("SELECT * FROM %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)); + + // Ensure that all subTasks has attemptNum > 0 + assertThat(output.stream().map(x -> x.getField(3)).collect(Collectors.toSet())).contains(1); + + // Ensure the test_table rows are returned exactly the same after the slow map task from the + // sink_table + assertSameElements( + output.stream().map(x -> Row.of(x.getField(0), x.getField(1))).collect(Collectors.toList()), + Arrays.asList(Row.of(1, -1), Row.of(2, -1), Row.of(3, -1))); + } + + /** A testing map function that simulates the slow task. */ + private static class TestingMap extends RichMapFunction { + @Override + public Row map(Row row) throws Exception { + // Put the subtasks with the first attempt to sleep to trigger speculative + // execution + if (getRuntimeContext().getAttemptNumber() <= 0) { + Thread.sleep(Integer.MAX_VALUE); + } + + Row output = + Row.of( + row.getField(0), + row.getField(1), + getRuntimeContext().getIndexOfThisSubtask(), + getRuntimeContext().getAttemptNumber()); + + return output; + } + } + + private static Configuration configure() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + configuration.set(RestOptions.BIND_PORT, "0"); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + + // Use FLIP-27 source + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + + // for speculative execution + configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); + + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); + configuration.set( + SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); + configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); + + return configuration; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index b21010a91bed..41b023b93617 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -144,7 +144,7 @@ public void testSpeculativeExecution() throws Exception { private static class TestingMap extends RichMapFunction { @Override public Row map(Row row) throws Exception { - // Put the even subtask indices with the first attempt to sleep to trigger speculative + // Put the subtasks with the first attempt to sleep to trigger speculative // execution if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { Thread.sleep(Integer.MAX_VALUE); From 806da5cfc7dba7b8fd872cf7fc6a6b36ac8a3876 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 1 Aug 2024 09:30:04 +0200 Subject: [PATCH 10/14] Infra: Improve feature request template (#10825) --- .github/ISSUE_TEMPLATE/iceberg_improvement.yml | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml index f3d6d6de923a..80048f99465a 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml @@ -50,4 +50,12 @@ body: - Hive - Other validations: - required: false \ No newline at end of file + required: false + - type: checkboxes + attributes: + label: Willingness to contribute + description: The Apache Iceberg community encourages contributions. Would you or another member of your organization be willing to contribute this improvement/feature to the Apache Iceberg codebase? + options: + - label: I can contribute this improvement/feature independently + - label: I would be willing to contribute this improvement/feature with guidance from the Iceberg community + - label: I cannot contribute this improvement/feature at this time From 99b8e88a88486f541b0ad2703fdc97ab615c5398 Mon Sep 17 00:00:00 2001 From: hsiang-c <137842490+hsiang-c@users.noreply.github.com> Date: Thu, 1 Aug 2024 23:23:05 +0800 Subject: [PATCH 11/14] Core: Replace the duplicated ALL_DATA_FILES with ALL_DELETE_FILES (#10836) --- .../java/org/apache/iceberg/TestMetadataTableFilters.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index 8125e064f0d0..9e535dd77747 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -40,7 +40,7 @@ public class TestMetadataTableFilters extends TestBase { private static final Set AGG_FILE_TABLES = Sets.newHashSet( MetadataTableType.ALL_DATA_FILES, - MetadataTableType.ALL_DATA_FILES, + MetadataTableType.ALL_DELETE_FILES, MetadataTableType.ALL_FILES, MetadataTableType.ALL_ENTRIES); @@ -132,9 +132,9 @@ private int expectedScanTaskCount(int partitions) { } case DATA_FILES: case DELETE_FILES: - case ALL_DELETE_FILES: return partitions; case ALL_DATA_FILES: + case ALL_DELETE_FILES: return partitions * 2; // ScanTask for Data Manifest in DELETED and ADDED states case ALL_FILES: case ALL_ENTRIES: From eb9d3951eeefc51824b87d36ca3824f7a968e81e Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 1 Aug 2024 10:23:49 -0500 Subject: [PATCH 12/14] Core: Adds Basic Classes for Iceberg Table Version 3 (#10760) --- .../iceberg/BaseUpdatePartitionSpec.java | 2 +- .../org/apache/iceberg/ManifestFiles.java | 4 + .../apache/iceberg/ManifestListWriter.java | 35 ++ .../org/apache/iceberg/ManifestLists.java | 3 + .../org/apache/iceberg/ManifestWriter.java | 73 +++ .../org/apache/iceberg/TableMetadata.java | 2 +- .../java/org/apache/iceberg/V3Metadata.java | 575 ++++++++++++++++++ .../iceberg/DeleteFileIndexTestBase.java | 2 +- .../iceberg/MetadataTableScanTestBase.java | 2 +- .../ScanPlanningAndReportingTestBase.java | 2 +- .../java/org/apache/iceberg/TestBase.java | 2 +- .../org/apache/iceberg/TestBatchScans.java | 2 +- .../apache/iceberg/TestCommitReporting.java | 2 +- .../apache/iceberg/TestCreateTransaction.java | 2 +- .../iceberg/TestEntriesMetadataTable.java | 2 +- .../org/apache/iceberg/TestFastAppend.java | 2 +- .../org/apache/iceberg/TestFindFiles.java | 2 +- .../apache/iceberg/TestFormatVersions.java | 2 +- .../iceberg/TestIncrementalDataTableScan.java | 2 +- .../apache/iceberg/TestLocalFilterFiles.java | 2 +- .../apache/iceberg/TestLocationProvider.java | 2 +- .../apache/iceberg/TestManifestCleanup.java | 2 +- .../iceberg/TestManifestReaderStats.java | 2 +- .../apache/iceberg/TestManifestWriter.java | 2 +- .../iceberg/TestMetadataTableScans.java | 46 +- ...adataTableScansWithPartitionEvolution.java | 3 +- .../java/org/apache/iceberg/TestMetrics.java | 2 +- .../org/apache/iceberg/TestMetricsModes.java | 2 +- .../apache/iceberg/TestMicroBatchBuilder.java | 2 +- .../apache/iceberg/TestPartitionSpecInfo.java | 2 +- .../iceberg/TestReplaceTransaction.java | 2 +- .../apache/iceberg/TestRewriteManifests.java | 2 +- .../org/apache/iceberg/TestScanSummary.java | 2 +- .../iceberg/TestScansAndSchemaEvolution.java | 2 +- .../iceberg/TestSchemaAndMappingUpdate.java | 2 +- .../java/org/apache/iceberg/TestSchemaID.java | 2 +- .../iceberg/TestSetPartitionStatistics.java | 2 +- .../org/apache/iceberg/TestSetStatistics.java | 2 +- .../java/org/apache/iceberg/TestSnapshot.java | 2 +- .../apache/iceberg/TestSnapshotLoading.java | 2 +- .../apache/iceberg/TestSnapshotManager.java | 2 +- .../apache/iceberg/TestSnapshotSelection.java | 2 +- .../apache/iceberg/TestSnapshotSummary.java | 2 +- .../org/apache/iceberg/TestSortOrder.java | 2 +- .../org/apache/iceberg/TestSplitPlanning.java | 2 +- .../TestTableMetadataSerialization.java | 2 +- .../iceberg/TestTableUpdatePartitionSpec.java | 2 +- .../iceberg/TestTimestampPartitions.java | 2 +- .../org/apache/iceberg/TestTransaction.java | 2 +- .../iceberg/TestUpdatePartitionSpec.java | 2 +- .../org/apache/iceberg/TestWapWorkflow.java | 2 +- .../actions/TestSizeBasedRewriter.java | 2 +- .../iceberg/io/TestOutputFileFactory.java | 2 +- .../iceberg/mapping/TestMappingUpdates.java | 2 +- .../TableMetadataUnsupportedVersion.json | 4 +- 55 files changed, 762 insertions(+), 75 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/V3Metadata.java diff --git a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java index b59292c397a1..2e1c9199174c 100644 --- a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java +++ b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java @@ -118,7 +118,7 @@ private int assignFieldId() { */ private PartitionField recycleOrCreatePartitionField( Pair> sourceTransform, String name) { - if (formatVersion == 2 && base != null) { + if (formatVersion >= 2 && base != null) { int sourceId = sourceTransform.first(); Transform transform = sourceTransform.second(); diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index 840c90bebdde..f630bb3eb743 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -182,6 +182,8 @@ public static ManifestWriter write( return new ManifestWriter.V1Writer(spec, encryptedOutputFile, snapshotId); case 2: return new ManifestWriter.V2Writer(spec, encryptedOutputFile, snapshotId); + case 3: + return new ManifestWriter.V3Writer(spec, encryptedOutputFile, snapshotId); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); @@ -238,6 +240,8 @@ public static ManifestWriter writeDeleteManifest( throw new IllegalArgumentException("Cannot write delete files in a v1 table"); case 2: return new ManifestWriter.V2DeleteWriter(spec, outputFile, snapshotId); + case 3: + return new ManifestWriter.V3DeleteWriter(spec, outputFile, snapshotId); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); diff --git a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java index 3f7f20d4df6c..b17eedad18af 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java @@ -70,6 +70,41 @@ public long length() { return writer.length(); } + static class V3Writer extends ManifestListWriter { + private final V3Metadata.IndexedManifestFile wrapper; + + V3Writer(OutputFile snapshotFile, long snapshotId, Long parentSnapshotId, long sequenceNumber) { + super( + snapshotFile, + ImmutableMap.of( + "snapshot-id", String.valueOf(snapshotId), + "parent-snapshot-id", String.valueOf(parentSnapshotId), + "sequence-number", String.valueOf(sequenceNumber), + "format-version", "3")); + this.wrapper = new V3Metadata.IndexedManifestFile(snapshotId, sequenceNumber); + } + + @Override + protected ManifestFile prepare(ManifestFile manifest) { + return wrapper.wrap(manifest); + } + + @Override + protected FileAppender newAppender(OutputFile file, Map meta) { + try { + return Avro.write(file) + .schema(V3Metadata.MANIFEST_LIST_SCHEMA) + .named("manifest_file") + .meta(meta) + .overwrite() + .build(); + + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to create snapshot list writer for path: %s", file); + } + } + } + static class V2Writer extends ManifestListWriter { private final V2Metadata.IndexedManifestFile wrapper; diff --git a/core/src/main/java/org/apache/iceberg/ManifestLists.java b/core/src/main/java/org/apache/iceberg/ManifestLists.java index c7b3e5fee5a9..f20a481cf25a 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestLists.java +++ b/core/src/main/java/org/apache/iceberg/ManifestLists.java @@ -66,6 +66,9 @@ static ManifestListWriter write( case 2: return new ManifestListWriter.V2Writer( manifestListFile, snapshotId, parentSnapshotId, sequenceNumber); + case 3: + return new ManifestListWriter.V3Writer( + manifestListFile, snapshotId, parentSnapshotId, sequenceNumber); } throw new UnsupportedOperationException( "Cannot write manifest list for table version: " + formatVersion); diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index cea907ddaca1..88587a1ebc89 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -217,6 +217,79 @@ public void close() throws IOException { writer.close(); } + static class V3Writer extends ManifestWriter { + private final V3Metadata.IndexedManifestEntry entryWrapper; + + V3Writer(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { + super(spec, file, snapshotId); + this.entryWrapper = new V3Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); + } + + @Override + protected ManifestEntry prepare(ManifestEntry entry) { + return entryWrapper.wrap(entry); + } + + @Override + protected FileAppender> newAppender( + PartitionSpec spec, OutputFile file) { + Schema manifestSchema = V3Metadata.entrySchema(spec.partitionType()); + try { + return Avro.write(file) + .schema(manifestSchema) + .named("manifest_entry") + .meta("schema", SchemaParser.toJson(spec.schema())) + .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) + .meta("partition-spec-id", String.valueOf(spec.specId())) + .meta("format-version", "3") + .meta("content", "data") + .overwrite() + .build(); + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to create manifest writer for path: %s", file); + } + } + } + + static class V3DeleteWriter extends ManifestWriter { + private final V3Metadata.IndexedManifestEntry entryWrapper; + + V3DeleteWriter(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { + super(spec, file, snapshotId); + this.entryWrapper = new V3Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); + } + + @Override + protected ManifestEntry prepare(ManifestEntry entry) { + return entryWrapper.wrap(entry); + } + + @Override + protected FileAppender> newAppender( + PartitionSpec spec, OutputFile file) { + Schema manifestSchema = V3Metadata.entrySchema(spec.partitionType()); + try { + return Avro.write(file) + .schema(manifestSchema) + .named("manifest_entry") + .meta("schema", SchemaParser.toJson(spec.schema())) + .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) + .meta("partition-spec-id", String.valueOf(spec.specId())) + .meta("format-version", "3") + .meta("content", "deletes") + .overwrite() + .build(); + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to create manifest writer for path: %s", file); + } + } + + @Override + protected ManifestContent content() { + return ManifestContent.DELETES; + } + } + static class V2Writer extends ManifestWriter { private final V2Metadata.IndexedManifestEntry entryWrapper; diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 74b8ad0bbddc..bd1c8a1a0371 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -51,7 +51,7 @@ public class TableMetadata implements Serializable { static final long INITIAL_SEQUENCE_NUMBER = 0; static final long INVALID_SEQUENCE_NUMBER = -1; static final int DEFAULT_TABLE_FORMAT_VERSION = 2; - static final int SUPPORTED_TABLE_FORMAT_VERSION = 2; + static final int SUPPORTED_TABLE_FORMAT_VERSION = 3; static final int INITIAL_SPEC_ID = 0; static final int INITIAL_SORT_ORDER_ID = 1; static final int INITIAL_SCHEMA_ID = 0; diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java new file mode 100644 index 000000000000..94e20ea99858 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -0,0 +1,575 @@ +/* + * 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; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.avro.generic.IndexedRecord; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; + +class V3Metadata { + private V3Metadata() {} + + static final Schema MANIFEST_LIST_SCHEMA = + new Schema( + ManifestFile.PATH, + ManifestFile.LENGTH, + ManifestFile.SPEC_ID, + ManifestFile.MANIFEST_CONTENT.asRequired(), + ManifestFile.SEQUENCE_NUMBER.asRequired(), + ManifestFile.MIN_SEQUENCE_NUMBER.asRequired(), + ManifestFile.SNAPSHOT_ID.asRequired(), + ManifestFile.ADDED_FILES_COUNT.asRequired(), + ManifestFile.EXISTING_FILES_COUNT.asRequired(), + ManifestFile.DELETED_FILES_COUNT.asRequired(), + ManifestFile.ADDED_ROWS_COUNT.asRequired(), + ManifestFile.EXISTING_ROWS_COUNT.asRequired(), + ManifestFile.DELETED_ROWS_COUNT.asRequired(), + ManifestFile.PARTITION_SUMMARIES, + ManifestFile.KEY_METADATA); + + /** + * A wrapper class to write any ManifestFile implementation to Avro using the v3 write schema. + * + *

This is used to maintain compatibility with v3 by writing manifest list files with the old + * schema, instead of writing a sequence number into metadata files in v3 tables. + */ + static class IndexedManifestFile implements ManifestFile, IndexedRecord { + private static final org.apache.avro.Schema AVRO_SCHEMA = + AvroSchemaUtil.convert(MANIFEST_LIST_SCHEMA, "manifest_file"); + + private final long commitSnapshotId; + private final long sequenceNumber; + private ManifestFile wrapped = null; + + IndexedManifestFile(long commitSnapshotId, long sequenceNumber) { + this.commitSnapshotId = commitSnapshotId; + this.sequenceNumber = sequenceNumber; + } + + public ManifestFile wrap(ManifestFile file) { + this.wrapped = file; + return this; + } + + @Override + public org.apache.avro.Schema getSchema() { + return AVRO_SCHEMA; + } + + @Override + public void put(int i, Object v) { + throw new UnsupportedOperationException("Cannot modify IndexedManifestFile wrapper via put"); + } + + @Override + public Object get(int pos) { + switch (pos) { + case 0: + return wrapped.path(); + case 1: + return wrapped.length(); + case 2: + return wrapped.partitionSpecId(); + case 3: + return wrapped.content().id(); + case 4: + if (wrapped.sequenceNumber() == ManifestWriter.UNASSIGNED_SEQ) { + // if the sequence number is being assigned here, then the manifest must be created by + // the current + // operation. to validate this, check that the snapshot id matches the current commit + Preconditions.checkState( + commitSnapshotId == wrapped.snapshotId(), + "Found unassigned sequence number for a manifest from snapshot: %s", + wrapped.snapshotId()); + return sequenceNumber; + } else { + return wrapped.sequenceNumber(); + } + case 5: + if (wrapped.minSequenceNumber() == ManifestWriter.UNASSIGNED_SEQ) { + // same sanity check as above + Preconditions.checkState( + commitSnapshotId == wrapped.snapshotId(), + "Found unassigned sequence number for a manifest from snapshot: %s", + wrapped.snapshotId()); + // if the min sequence number is not determined, then there was no assigned sequence + // number for any file + // written to the wrapped manifest. replace the unassigned sequence number with the one + // for this commit + return sequenceNumber; + } else { + return wrapped.minSequenceNumber(); + } + case 6: + return wrapped.snapshotId(); + case 7: + return wrapped.addedFilesCount(); + case 8: + return wrapped.existingFilesCount(); + case 9: + return wrapped.deletedFilesCount(); + case 10: + return wrapped.addedRowsCount(); + case 11: + return wrapped.existingRowsCount(); + case 12: + return wrapped.deletedRowsCount(); + case 13: + return wrapped.partitions(); + case 14: + return wrapped.keyMetadata(); + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + pos); + } + } + + @Override + public String path() { + return wrapped.path(); + } + + @Override + public long length() { + return wrapped.length(); + } + + @Override + public int partitionSpecId() { + return wrapped.partitionSpecId(); + } + + @Override + public ManifestContent content() { + return wrapped.content(); + } + + @Override + public long sequenceNumber() { + return wrapped.sequenceNumber(); + } + + @Override + public long minSequenceNumber() { + return wrapped.minSequenceNumber(); + } + + @Override + public Long snapshotId() { + return wrapped.snapshotId(); + } + + @Override + public boolean hasAddedFiles() { + return wrapped.hasAddedFiles(); + } + + @Override + public Integer addedFilesCount() { + return wrapped.addedFilesCount(); + } + + @Override + public Long addedRowsCount() { + return wrapped.addedRowsCount(); + } + + @Override + public boolean hasExistingFiles() { + return wrapped.hasExistingFiles(); + } + + @Override + public Integer existingFilesCount() { + return wrapped.existingFilesCount(); + } + + @Override + public Long existingRowsCount() { + return wrapped.existingRowsCount(); + } + + @Override + public boolean hasDeletedFiles() { + return wrapped.hasDeletedFiles(); + } + + @Override + public Integer deletedFilesCount() { + return wrapped.deletedFilesCount(); + } + + @Override + public Long deletedRowsCount() { + return wrapped.deletedRowsCount(); + } + + @Override + public List partitions() { + return wrapped.partitions(); + } + + @Override + public ByteBuffer keyMetadata() { + return wrapped.keyMetadata(); + } + + @Override + public ManifestFile copy() { + return wrapped.copy(); + } + } + + static Schema entrySchema(Types.StructType partitionType) { + return wrapFileSchema(fileType(partitionType)); + } + + static Schema wrapFileSchema(Types.StructType fileSchema) { + // this is used to build projection schemas + return new Schema( + ManifestEntry.STATUS, + ManifestEntry.SNAPSHOT_ID, + ManifestEntry.SEQUENCE_NUMBER, + ManifestEntry.FILE_SEQUENCE_NUMBER, + required(ManifestEntry.DATA_FILE_ID, "data_file", fileSchema)); + } + + static Types.StructType fileType(Types.StructType partitionType) { + return Types.StructType.of( + DataFile.CONTENT.asRequired(), + DataFile.FILE_PATH, + DataFile.FILE_FORMAT, + required( + DataFile.PARTITION_ID, DataFile.PARTITION_NAME, partitionType, DataFile.PARTITION_DOC), + DataFile.RECORD_COUNT, + DataFile.FILE_SIZE, + DataFile.COLUMN_SIZES, + DataFile.VALUE_COUNTS, + DataFile.NULL_VALUE_COUNTS, + DataFile.NAN_VALUE_COUNTS, + DataFile.LOWER_BOUNDS, + DataFile.UPPER_BOUNDS, + DataFile.KEY_METADATA, + DataFile.SPLIT_OFFSETS, + DataFile.EQUALITY_IDS, + DataFile.SORT_ORDER_ID); + } + + static class IndexedManifestEntry> + implements ManifestEntry, IndexedRecord { + private final org.apache.avro.Schema avroSchema; + private final Long commitSnapshotId; + private final IndexedDataFile fileWrapper; + private ManifestEntry wrapped = null; + + IndexedManifestEntry(Long commitSnapshotId, Types.StructType partitionType) { + this.avroSchema = AvroSchemaUtil.convert(entrySchema(partitionType), "manifest_entry"); + this.commitSnapshotId = commitSnapshotId; + this.fileWrapper = new IndexedDataFile<>(partitionType); + } + + public IndexedManifestEntry wrap(ManifestEntry entry) { + this.wrapped = entry; + return this; + } + + @Override + public org.apache.avro.Schema getSchema() { + return avroSchema; + } + + @Override + public void put(int i, Object v) { + throw new UnsupportedOperationException("Cannot modify IndexedManifestEntry wrapper via put"); + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return wrapped.status().id(); + case 1: + return wrapped.snapshotId(); + case 2: + if (wrapped.dataSequenceNumber() == null) { + // if the entry's data sequence number is null, + // then it will inherit the sequence number of the current commit. + // to validate that this is correct, check that the snapshot id is either null (will + // also be inherited) or that it matches the id of the current commit. + Preconditions.checkState( + wrapped.snapshotId() == null || wrapped.snapshotId().equals(commitSnapshotId), + "Found unassigned sequence number for an entry from snapshot: %s", + wrapped.snapshotId()); + + // inheritance should work only for ADDED entries + Preconditions.checkState( + wrapped.status() == Status.ADDED, + "Only entries with status ADDED can have null sequence number"); + + return null; + } + return wrapped.dataSequenceNumber(); + case 3: + return wrapped.fileSequenceNumber(); + case 4: + return fileWrapper.wrap(wrapped.file()); + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } + + @Override + public Status status() { + return wrapped.status(); + } + + @Override + public Long snapshotId() { + return wrapped.snapshotId(); + } + + @Override + public void setSnapshotId(long snapshotId) { + wrapped.setSnapshotId(snapshotId); + } + + @Override + public Long dataSequenceNumber() { + return wrapped.dataSequenceNumber(); + } + + @Override + public void setDataSequenceNumber(long dataSequenceNumber) { + wrapped.setDataSequenceNumber(dataSequenceNumber); + } + + @Override + public Long fileSequenceNumber() { + return wrapped.fileSequenceNumber(); + } + + @Override + public void setFileSequenceNumber(long fileSequenceNumber) { + wrapped.setFileSequenceNumber(fileSequenceNumber); + } + + @Override + public F file() { + return wrapped.file(); + } + + @Override + public ManifestEntry copy() { + return wrapped.copy(); + } + + @Override + public ManifestEntry copyWithoutStats() { + return wrapped.copyWithoutStats(); + } + } + + /** Wrapper used to write DataFile or DeleteFile to v3 metadata. */ + static class IndexedDataFile implements ContentFile, IndexedRecord { + private final org.apache.avro.Schema avroSchema; + private final IndexedStructLike partitionWrapper; + private ContentFile wrapped = null; + + IndexedDataFile(Types.StructType partitionType) { + this.avroSchema = AvroSchemaUtil.convert(fileType(partitionType), "data_file"); + this.partitionWrapper = new IndexedStructLike(avroSchema.getField("partition").schema()); + } + + @SuppressWarnings("unchecked") + IndexedDataFile wrap(ContentFile file) { + this.wrapped = (ContentFile) file; + return this; + } + + @Override + public org.apache.avro.Schema getSchema() { + return avroSchema; + } + + @Override + public Object get(int pos) { + switch (pos) { + case 0: + return wrapped.content().id(); + case 1: + return wrapped.path().toString(); + case 2: + return wrapped.format() != null ? wrapped.format().toString() : null; + case 3: + return partitionWrapper.wrap(wrapped.partition()); + case 4: + return wrapped.recordCount(); + case 5: + return wrapped.fileSizeInBytes(); + case 6: + return wrapped.columnSizes(); + case 7: + return wrapped.valueCounts(); + case 8: + return wrapped.nullValueCounts(); + case 9: + return wrapped.nanValueCounts(); + case 10: + return wrapped.lowerBounds(); + case 11: + return wrapped.upperBounds(); + case 12: + return wrapped.keyMetadata(); + case 13: + return wrapped.splitOffsets(); + case 14: + return wrapped.equalityFieldIds(); + case 15: + return wrapped.sortOrderId(); + } + throw new IllegalArgumentException("Unknown field ordinal: " + pos); + } + + @Override + public void put(int i, Object v) { + throw new UnsupportedOperationException("Cannot modify IndexedDataFile wrapper via put"); + } + + @Override + public Long pos() { + return null; + } + + @Override + public int specId() { + return wrapped.specId(); + } + + @Override + public FileContent content() { + return wrapped.content(); + } + + @Override + public CharSequence path() { + return wrapped.path(); + } + + @Override + public FileFormat format() { + return wrapped.format(); + } + + @Override + public StructLike partition() { + return wrapped.partition(); + } + + @Override + public long recordCount() { + return wrapped.recordCount(); + } + + @Override + public long fileSizeInBytes() { + return wrapped.fileSizeInBytes(); + } + + @Override + public Map columnSizes() { + return wrapped.columnSizes(); + } + + @Override + public Map valueCounts() { + return wrapped.valueCounts(); + } + + @Override + public Map nullValueCounts() { + return wrapped.nullValueCounts(); + } + + @Override + public Map nanValueCounts() { + return wrapped.nanValueCounts(); + } + + @Override + public Map lowerBounds() { + return wrapped.lowerBounds(); + } + + @Override + public Map upperBounds() { + return wrapped.upperBounds(); + } + + @Override + public ByteBuffer keyMetadata() { + return wrapped.keyMetadata(); + } + + @Override + public List splitOffsets() { + return wrapped.splitOffsets(); + } + + @Override + public List equalityFieldIds() { + return wrapped.equalityFieldIds(); + } + + @Override + public Integer sortOrderId() { + return wrapped.sortOrderId(); + } + + @Override + public Long dataSequenceNumber() { + return wrapped.dataSequenceNumber(); + } + + @Override + public Long fileSequenceNumber() { + return wrapped.fileSequenceNumber(); + } + + @Override + public F copy() { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + + @Override + public F copyWithStats(Set requestedColumnIds) { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + + @Override + public F copyWithoutStats() { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java index 229650566ca8..836a1ddd80f5 100644 --- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java @@ -46,7 +46,7 @@ public abstract class DeleteFileIndexTestBase< @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } static final DeleteFile FILE_A_POS_1 = diff --git a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java index a4e964b017ba..98d2d8f38af6 100644 --- a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java @@ -40,7 +40,7 @@ public abstract class MetadataTableScanTestBase extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } protected Set scannedPaths(TableScan scan) { diff --git a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java index 3ba74c3b4b71..13e96869b454 100644 --- a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java @@ -47,7 +47,7 @@ public abstract class ScanPlanningAndReportingTestBase< @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } protected abstract ScanT newScan(Table table); diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 2322062dad85..e03a1efd5156 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -174,7 +174,7 @@ public class TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter protected int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestBatchScans.java b/core/src/test/java/org/apache/iceberg/TestBatchScans.java index c7210486e05e..1597f44f6338 100644 --- a/core/src/test/java/org/apache/iceberg/TestBatchScans.java +++ b/core/src/test/java/org/apache/iceberg/TestBatchScans.java @@ -34,7 +34,7 @@ public class TestBatchScans extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java index bbba192fab7f..41b301668722 100644 --- a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java +++ b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java @@ -37,7 +37,7 @@ public class TestCommitReporting extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java index 0c6b50b37792..766dd85a0655 100644 --- a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java @@ -40,7 +40,7 @@ public class TestCreateTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java index 60e79aad91ce..9bce4e60a4f3 100644 --- a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java +++ b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java @@ -34,7 +34,7 @@ public class TestEntriesMetadataTable extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 9dd479ecf0fc..b281536ab0fa 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -39,7 +39,7 @@ public class TestFastAppend extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestFindFiles.java b/core/src/test/java/org/apache/iceberg/TestFindFiles.java index 68d757de5cdb..191dbd384197 100644 --- a/core/src/test/java/org/apache/iceberg/TestFindFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestFindFiles.java @@ -36,7 +36,7 @@ public class TestFindFiles extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java index 2f6a01c6e603..4a9f18581d09 100644 --- a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java @@ -71,7 +71,7 @@ public void testFormatVersionUpgradeNotSupported() { base, base.upgradeToFormatVersion(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot upgrade table to unsupported format version: v3 (supported: v2)"); + .hasMessage("Cannot upgrade table to unsupported format version: v4 (supported: v3)"); assertThat(ops.current().formatVersion()).isEqualTo(1); } diff --git a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java index fe6b9b0c763c..ecd6a14ffefb 100644 --- a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java +++ b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java @@ -43,7 +43,7 @@ public class TestIncrementalDataTableScan extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java b/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java index 275b89df3695..8a4f5db256f0 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java @@ -28,7 +28,7 @@ public class TestLocalFilterFiles @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java index 93dd1a8555ce..7afb69483490 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java +++ b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java @@ -34,7 +34,7 @@ public class TestLocationProvider extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } // publicly visible for testing to be dynamically loaded diff --git a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java index b5f6d05cc6a0..37ccbb8bb845 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java @@ -30,7 +30,7 @@ public class TestManifestCleanup extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java index 4c60a943f76c..175178e48167 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java @@ -39,7 +39,7 @@ public class TestManifestReaderStats extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final Map VALUE_COUNT = ImmutableMap.of(3, 3L); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java index eb7910a79fc5..7dd3ea2d1ce7 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java @@ -38,7 +38,7 @@ public class TestManifestWriter extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final int FILE_SIZE_CHECK_ROWS_DIVISOR = 250; diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 0a14a89c01b5..755eb57de8e8 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -55,7 +55,7 @@ public class TestMetadataTableScans extends MetadataTableScanTestBase { private void preparePartitionedTable(boolean transactional) { preparePartitionedTableData(transactional); - if (formatVersion == 2) { + if (formatVersion >= 2) { if (transactional) { table .newRowDelta() @@ -485,7 +485,7 @@ public void testPartitionsTableScanNoFilter() { CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanNoFilter); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -510,7 +510,7 @@ public void testPartitionsTableScanWithProjection() { assertThat(scanWithProjection.schema().asStruct()).isEqualTo(expected); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanWithProjection); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -552,7 +552,7 @@ public void testPartitionsTableScanAndFilter() { TableScan scanAndEq = partitionsTable.newScan().filter(andEquals); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanAndEq); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(2); } else { assertThat(entries).hasSize(1); @@ -574,7 +574,7 @@ public void testPartitionsTableScanLtFilter() { TableScan scanLtAnd = partitionsTable.newScan().filter(ltAnd); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanLtAnd); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(4); } else { assertThat(entries).hasSize(2); @@ -598,7 +598,7 @@ public void testPartitionsTableScanOrFilter() { CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanOr); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -619,7 +619,7 @@ public void testPartitionsScanNotFilter() { TableScan scanNot = partitionsTable.newScan().filter(not); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanNot); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(4); } else { assertThat(entries).hasSize(2); @@ -639,7 +639,7 @@ public void testPartitionsTableScanInFilter() { TableScan scanSet = partitionsTable.newScan().filter(set); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanSet); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(4); } else { assertThat(entries).hasSize(2); @@ -659,7 +659,7 @@ public void testPartitionsTableScanNotNullFilter() { TableScan scanUnary = partitionsTable.newScan().filter(unary); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanUnary); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -719,8 +719,7 @@ public void testFilesTableScanWithDroppedPartition() throws IOException { @TestTemplate public void testDeleteFilesTableSelection() throws IOException { - assumeThat(formatVersion).as("Only V2 Tables Support Deletes").isGreaterThanOrEqualTo(2); - + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); table.newFastAppend().appendFile(FILE_A).commit(); table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); @@ -960,7 +959,7 @@ public void testPartitionSpecEvolutionAdditive() { TableScan scan = metadataTable.newScan().filter(filter); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { // Four data files and delete files of old spec, one new data file of new spec assertThat(entries).hasSize(9); } else { @@ -975,7 +974,7 @@ public void testPartitionSpecEvolutionAdditive() { scan = metadataTable.newScan().filter(filter); entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { // 1 original data file and delete file written by old spec, plus 1 new data file written by // new spec assertThat(entries).hasSize(3); @@ -1026,7 +1025,7 @@ public void testPartitionSpecEvolutionRemoval() { CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { // Four data and delete files of original spec, one data file written by new spec assertThat(entries).hasSize(9); } else { @@ -1187,7 +1186,7 @@ public void testPartitionsTableScanWithPlanExecutor() { })); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -1366,7 +1365,7 @@ public void testAllManifestsTableSnapshotNot() { @TestTemplate public void testPositionDeletesWithFilter() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); preparePartitionedTable(); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1429,7 +1428,7 @@ public void testPositionDeletesBaseTableFilterEntriesLevel() { } private void testPositionDeletesBaseTableFilter(boolean transactional) { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); preparePartitionedTable(transactional); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1490,9 +1489,9 @@ private void testPositionDeletesBaseTableFilter(boolean transactional) { @TestTemplate public void testPositionDeletesWithBaseTableFilterNot() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); - - // use identity rather than bucket partition spec, + assumeThat(formatVersion) + .as("Position deletes are not supported by V1 Tables") + .isNotEqualTo(1); // use identity rather than bucket partition spec, // as bucket.project does not support projecting notEq table.updateSpec().removeField("data_bucket").addField("id").commit(); PartitionSpec spec = table.spec(); @@ -1574,7 +1573,7 @@ public void testPositionDeletesWithBaseTableFilterNot() { @TestTemplate public void testPositionDeletesResiduals() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); preparePartitionedTable(); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1603,7 +1602,7 @@ public void testPositionDeletesResiduals() { @TestTemplate public void testPositionDeletesUnpartitioned() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); table.updateSpec().removeField(Expressions.bucket("data", BUCKETS_NUMBER)).commit(); assertThat(table.spec().fields()).as("Table should now be unpartitioned").hasSize(0); @@ -1694,8 +1693,7 @@ public void testPositionDeletesUnpartitioned() { @TestTemplate public void testPositionDeletesManyColumns() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); - + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); UpdateSchema updateSchema = table.updateSchema(); for (int i = 0; i <= 2000; i++) { updateSchema.addColumn(String.valueOf(i), Types.IntegerType.get()); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java index a2e5386d29df..ac96642319a3 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java @@ -163,8 +163,7 @@ public void testPartitionsTableScanWithAddPartitionOnNestedField() { @TestTemplate public void testPositionDeletesPartitionSpecRemoval() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); - + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); table.updateSpec().removeField("id").commit(); DeleteFile deleteFile = newDeleteFile(table.ops().current().spec().specId(), "nested.id=1"); diff --git a/core/src/test/java/org/apache/iceberg/TestMetrics.java b/core/src/test/java/org/apache/iceberg/TestMetrics.java index b95b92979f91..2c4849135f64 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetrics.java +++ b/core/src/test/java/org/apache/iceberg/TestMetrics.java @@ -69,7 +69,7 @@ public abstract class TestMetrics { @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TempDir public Path temp; diff --git a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java index 31ae459df506..00e961097c34 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java +++ b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java @@ -47,7 +47,7 @@ public class TestMetricsModes { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TempDir private Path temp; diff --git a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java index 733bb0bb38fd..8c6f18619ac0 100644 --- a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java +++ b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java @@ -34,7 +34,7 @@ public class TestMicroBatchBuilder extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java index ff54929504ee..a324b8af2e70 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java @@ -47,7 +47,7 @@ public class TestPartitionSpecInfo { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java index 3a6d2017eb82..b1b481dd5305 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java @@ -50,7 +50,7 @@ public class TestReplaceTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index 176f61079fc3..f1d23de32a42 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -48,7 +48,7 @@ public class TestRewriteManifests extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestScanSummary.java b/core/src/test/java/org/apache/iceberg/TestScanSummary.java index b326274842a3..10c22b759988 100644 --- a/core/src/test/java/org/apache/iceberg/TestScanSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestScanSummary.java @@ -39,7 +39,7 @@ public class TestScanSummary extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java index 0d5b1bc7066b..3fc691ce942a 100644 --- a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java @@ -55,7 +55,7 @@ public class TestScansAndSchemaEvolution { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java index 1bd1761ffc60..111693792f78 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java @@ -43,7 +43,7 @@ public class TestSchemaAndMappingUpdate extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaID.java b/core/src/test/java/org/apache/iceberg/TestSchemaID.java index 33ac84d20801..e3c58af1cbe3 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaID.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaID.java @@ -36,7 +36,7 @@ public class TestSchemaID extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java index 3ebe047e90b7..9504f78f5345 100644 --- a/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java +++ b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java @@ -29,7 +29,7 @@ public class TestSetPartitionStatistics extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSetStatistics.java b/core/src/test/java/org/apache/iceberg/TestSetStatistics.java index 41941e3c6630..03c644117a95 100644 --- a/core/src/test/java/org/apache/iceberg/TestSetStatistics.java +++ b/core/src/test/java/org/apache/iceberg/TestSetStatistics.java @@ -31,7 +31,7 @@ public class TestSetStatistics extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java index 2ec6abd4e428..8a30036f3242 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java @@ -33,7 +33,7 @@ public class TestSnapshot extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java index 89312201265d..96dde0f48eb7 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java @@ -43,7 +43,7 @@ public class TestSnapshotLoading extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private Snapshot currentSnapshot; diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java index 88233dd99097..61dd082701c2 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java @@ -51,7 +51,7 @@ public class TestSnapshotManager extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java index 7ce59e9df1c9..6d339c0cbeaf 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java @@ -33,7 +33,7 @@ public class TestSnapshotSelection extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java index 053a9c374178..529e0cc614f6 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java @@ -32,7 +32,7 @@ public class TestSnapshotSummary extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSortOrder.java b/core/src/test/java/org/apache/iceberg/TestSortOrder.java index a3ba69a808b3..ad773192b417 100644 --- a/core/src/test/java/org/apache/iceberg/TestSortOrder.java +++ b/core/src/test/java/org/apache/iceberg/TestSortOrder.java @@ -76,7 +76,7 @@ public class TestSortOrder { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java index 04bb2ae215d8..ea2142982382 100644 --- a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java +++ b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java @@ -58,7 +58,7 @@ public class TestSplitPlanning extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java b/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java index f47968434bfe..94a3d35b35a6 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java @@ -35,7 +35,7 @@ public class TestTableMetadataSerialization extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java index a4e587068e74..482514c40093 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java @@ -36,7 +36,7 @@ public class TestTableUpdatePartitionSpec extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java index 08714dec01f1..656dd782cfe4 100644 --- a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java @@ -35,7 +35,7 @@ public class TestTimestampPartitions extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 393494da1283..8fed7134fae1 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -42,7 +42,7 @@ public class TestTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java index 33b003cfd8c5..97f695315e16 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java @@ -54,7 +54,7 @@ public class TestUpdatePartitionSpec extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java index 58b83f43f399..b1e8e57850e8 100644 --- a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java +++ b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java @@ -35,7 +35,7 @@ public class TestWapWorkflow extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java index a9a23d587ac9..77d16d3bc821 100644 --- a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java +++ b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java @@ -41,7 +41,7 @@ public class TestSizeBasedRewriter extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java index ceffeb3749da..bb98925f504c 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java +++ b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java @@ -40,7 +40,7 @@ public class TestOutputFileFactory extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final int PARTITION_ID = 1; diff --git a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java index b9ae9dc273f2..2161b40a60b5 100644 --- a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java +++ b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java @@ -36,7 +36,7 @@ public class TestMappingUpdates extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/resources/TableMetadataUnsupportedVersion.json b/core/src/test/resources/TableMetadataUnsupportedVersion.json index 0633a71d24d5..c40a0c9cd5ae 100644 --- a/core/src/test/resources/TableMetadataUnsupportedVersion.json +++ b/core/src/test/resources/TableMetadataUnsupportedVersion.json @@ -1,5 +1,5 @@ { - "format-version": 3, + "format-version": 4, "table-uuid": "d20125c8-7284-442c-9aea-15fee620737c", "location": "s3://bucket/test/location", "last-updated-ms": 1602638573874, @@ -33,4 +33,4 @@ "properties": {}, "current-snapshot-id": -1, "snapshots": [] -} \ No newline at end of file +} From 39373d09c276586ddcec971fe35951975bdac66f Mon Sep 17 00:00:00 2001 From: Grant Nicholas <43971820+grantatspothero@users.noreply.github.com> Date: Thu, 1 Aug 2024 14:31:18 -0500 Subject: [PATCH 13/14] Core: Allow SnapshotProducer to skip uncommitted manifest cleanup after commit (#10523) --- .../java/org/apache/iceberg/FastAppend.java | 10 ++++ .../org/apache/iceberg/SnapshotProducer.java | 43 ++++++++-------- .../org/apache/iceberg/TestFastAppend.java | 50 +++++++++++++++++++ 3 files changed, 82 insertions(+), 21 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 1439289130d7..4976a8081c44 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -198,6 +198,16 @@ protected void cleanUncommitted(Set committed) { } } + /** + * Cleanup after committing is disabled for FastAppend unless there are rewrittenAppendManifests + * because: 1.) Appended manifests are never rewritten 2.) Manifests which are written out as part + * of appendFile are already cleaned up between commit attempts in writeNewManifests + */ + @Override + protected boolean cleanupAfterCommit() { + return !rewrittenAppendManifests.isEmpty(); + } + private List writeNewManifests() throws IOException { if (hasNewFiles && newManifests != null) { newManifests.forEach(file -> deleteFile(file.path())); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 9f4bcbc6bba9..0a040fe34471 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -41,7 +41,7 @@ import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptingFileIO; @@ -368,8 +368,8 @@ protected TableMetadata refresh() { @Override @SuppressWarnings("checkstyle:CyclomaticComplexity") public void commit() { - // this is always set to the latest commit attempt's snapshot id. - AtomicLong newSnapshotId = new AtomicLong(-1L); + // this is always set to the latest commit attempt's snapshot + AtomicReference stagedSnapshot = new AtomicReference<>(); try (Timed ignore = commitMetrics().totalDuration().start()) { try { Tasks.foreach(ops) @@ -384,7 +384,7 @@ public void commit() { .run( taskOps -> { Snapshot newSnapshot = apply(); - newSnapshotId.set(newSnapshot.snapshotId()); + stagedSnapshot.set(newSnapshot); TableMetadata.Builder update = TableMetadata.buildFrom(base); if (base.snapshot(newSnapshot.snapshotId()) != null) { // this is a rollback operation @@ -422,26 +422,23 @@ public void commit() { throw e; } + // at this point, the commit must have succeeded so the stagedSnapshot is committed + Snapshot committedSnapshot = stagedSnapshot.get(); try { - LOG.info("Committed snapshot {} ({})", newSnapshotId.get(), getClass().getSimpleName()); - - // at this point, the commit must have succeeded. after a refresh, the snapshot is loaded by - // id in case another commit was added between this commit and the refresh. - Snapshot saved = ops.refresh().snapshot(newSnapshotId.get()); - if (saved != null) { - cleanUncommitted(Sets.newHashSet(saved.allManifests(ops.io()))); - // also clean up unused manifest lists created by multiple attempts - for (String manifestList : manifestLists) { - if (!saved.manifestListLocation().equals(manifestList)) { - deleteFile(manifestList); - } + LOG.info( + "Committed snapshot {} ({})", + committedSnapshot.snapshotId(), + getClass().getSimpleName()); + + if (cleanupAfterCommit()) { + cleanUncommitted(Sets.newHashSet(committedSnapshot.allManifests(ops.io()))); + } + // also clean up unused manifest lists created by multiple attempts + for (String manifestList : manifestLists) { + if (!committedSnapshot.manifestListLocation().equals(manifestList)) { + deleteFile(manifestList); } - } else { - // saved may not be present if the latest metadata couldn't be loaded due to eventual - // consistency problems in refresh. in that case, don't clean up. - LOG.warn("Failed to load committed snapshot, skipping manifest clean-up"); } - } catch (Throwable e) { LOG.warn( "Failed to load committed table metadata or during cleanup, skipping further cleanup", @@ -565,6 +562,10 @@ protected boolean canInheritSnapshotId() { return canInheritSnapshotId; } + protected boolean cleanupAfterCommit() { + return true; + } + private static ManifestFile addMetadata(TableOperations ops, ManifestFile manifest) { try (ManifestReader reader = ManifestFiles.read(manifest, ops.io(), ops.current().specsById())) { diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index b281536ab0fa..8125c528d9c3 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -324,6 +324,56 @@ public void testRecoveryWithoutManifestList() { assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).contains(newManifest); } + @TestTemplate + public void testWriteNewManifestsIdempotency() { + // inject 3 failures, the last try will succeed + TestTables.TestTableOperations ops = table.ops(); + ops.failCommits(3); + + AppendFiles append = table.newFastAppend().appendFile(FILE_B); + Snapshot pending = append.apply(); + ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); + assertThat(new File(newManifest.path())).exists(); + + append.commit(); + + TableMetadata metadata = readMetadata(); + + // contains only a single manifest, does not duplicate manifests on retries + validateSnapshot(null, metadata.currentSnapshot(), FILE_B); + assertThat(new File(newManifest.path())).exists(); + assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).contains(newManifest); + assertThat(listManifestFiles(tableDir)).containsExactly(new File(newManifest.path())); + } + + @TestTemplate + public void testWriteNewManifestsCleanup() { + // append file, stage changes with apply() but do not commit + AppendFiles append = table.newFastAppend().appendFile(FILE_A); + Snapshot pending = append.apply(); + ManifestFile oldManifest = pending.allManifests(FILE_IO).get(0); + assertThat(new File(oldManifest.path())).exists(); + + // append file, stage changes with apply() but do not commit + // validate writeNewManifests deleted the old staged manifest + append.appendFile(FILE_B); + Snapshot newPending = append.apply(); + List manifestFiles = newPending.allManifests(FILE_IO); + assertThat(manifestFiles).hasSize(1); + ManifestFile newManifest = manifestFiles.get(0); + assertThat(newManifest.path()).isNotEqualTo(oldManifest.path()); + + append.commit(); + TableMetadata metadata = readMetadata(); + + // contains only a single manifest, old staged manifest is deleted + validateSnapshot(null, metadata.currentSnapshot(), FILE_A, FILE_B); + assertThat(new File(oldManifest.path())).doesNotExist(); + assertThat(new File(newManifest.path())).exists(); + assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).containsExactly(newManifest); + assertThat(listManifestFiles(tableDir)).containsExactly(new File(newManifest.path())); + } + @TestTemplate public void testAppendManifestWithSnapshotIdInheritance() throws IOException { table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); From 6e7113a5291dffad38ffacc7d264456a2366a707 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 1 Aug 2024 14:10:37 -0700 Subject: [PATCH 14/14] Flink: a few small fixes or tuning for range partitioner (#10823) --- .../shuffle/AggregatedStatisticsTracker.java | 23 +++-- .../sink/shuffle/CompletedStatistics.java | 8 ++ .../shuffle/DataStatisticsCoordinator.java | 45 ++++++---- .../flink/sink/shuffle/RangePartitioner.java | 8 +- .../sink/shuffle/SketchRangePartitioner.java | 19 +--- .../flink/sink/shuffle/SketchUtil.java | 17 ++++ .../flink/sink/shuffle/SortKeyUtil.java | 59 +++++++++++++ .../sink/shuffle/TestRangePartitioner.java | 65 ++++++++++++++ .../shuffle/TestSketchRangePartitioner.java | 88 +++++++++++++++++++ .../flink/sink/shuffle/TestSketchUtil.java | 64 +++++++++++++- .../flink/sink/shuffle/TestSortKeyUtil.java | 73 +++++++++++++++ 11 files changed, 420 insertions(+), 49 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 338523b7b074..5525f02c873e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -223,7 +223,9 @@ private void merge(DataStatistics taskStatistics) { convertCoordinatorToSketch(); } - sketchStatistics.update(taskSketch); + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } } } @@ -242,13 +244,18 @@ private CompletedStatistics completedStatistics(long checkpointId) { return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); } else { ReservoirItemsSketch sketch = sketchStatistics.getResult(); - LOG.info( - "Completed sketch statistics aggregation: " - + "reservoir size = {}, number of items seen = {}, number of samples = {}", - sketch.getK(), - sketch.getN(), - sketch.getNumSamples()); - return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index c0e228965ddd..e4cba174f0f2 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -100,4 +100,12 @@ Map keyFrequency() { SortKey[] keySamples() { return keySamples; } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 3b21fbae315a..4bfde7204acf 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -38,11 +38,11 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; @@ -91,7 +91,7 @@ class DataStatisticsCoordinator implements OperatorCoordinator { this.context = context; this.schema = schema; this.sortOrder = sortOrder; - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.downstreamParallelism = downstreamParallelism; this.statisticsType = statisticsType; this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; @@ -202,17 +202,23 @@ private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); if (maybeCompletedStatistics != null) { - // completedStatistics contains the complete samples, which is needed to compute - // the range bounds in globalStatistics if downstreamParallelism changed. - this.completedStatistics = maybeCompletedStatistics; - // globalStatistics only contains assignment calculated based on Map or Sketch statistics - this.globalStatistics = - globalStatistics( - maybeCompletedStatistics, - downstreamParallelism, - comparator, - closeFileCostWeightPercentage); - sendGlobalStatisticsToSubtasks(globalStatistics); + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } } } @@ -324,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - StatisticsUtil.serializeCompletedStatistics( - completedStatistics, completedStatisticsSerializer)); + if (completedStatistics == null) { + // null checkpoint result is not allowed, hence supply an empty byte array + resultFuture.complete(new byte[0]); + } else { + resultFuture.complete( + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); + } }, String.format("taking checkpoint %d", checkpointId)); } @@ -338,7 +349,7 @@ public void notifyCheckpointComplete(long checkpointId) {} public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { + if (checkpointData == null || checkpointData.length == 0) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", operatorName, diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java index 83a9461233d2..6608b938f5a8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -23,12 +23,13 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The wrapper class */ +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ @Internal public class RangePartitioner implements Partitioner { private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); @@ -94,9 +95,8 @@ static int adjustPartitionWithRescale( if (numPartitionsStatsCalculation <= numPartitions) { // no rescale or scale-up case. // new subtasks are ignored and not assigned any keys, which is sub-optimal and only - // transient. - // when rescale is detected, operator requests new statistics from coordinator upon - // initialization. + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. return partition; } else { // scale-down case. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java index af78271ea5dc..dddb0d8722c0 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -18,17 +18,16 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; import java.util.Comparator; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; class SketchRangePartitioner implements Partitioner { private final SortKey sortKey; @@ -38,7 +37,7 @@ class SketchRangePartitioner implements Partitioner { SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.rangeBounds = rangeBounds; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); } @@ -47,18 +46,6 @@ class SketchRangePartitioner implements Partitioner { public int partition(RowData row, int numPartitions) { // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); - - // binarySearch either returns the match location or -[insertion point]-1 - if (partition < 0) { - partition = -partition - 1; - } - - if (partition > rangeBounds.length) { - partition = rangeBounds.length; - } - - return RangePartitioner.adjustPartitionWithRescale( - partition, rangeBounds.length + 1, numPartitions); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java index a58310611e8d..871ef9ef1149 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -139,4 +139,21 @@ static void convertMapToSketch( } }); } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator comparator) { + int partition = Arrays.binarySearch(rangeBounds, key, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java new file mode 100644 index 000000000000..1e5bdbbac3e4 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeyUtil { + private SortKeyUtil() {} + + /** Compute the result schema of {@code SortKey} transformation */ + static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { + List sortFields = sortOrder.fields(); + int size = sortFields.size(); + List transformedFields = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; ++i) { + int sourceFieldId = sortFields.get(i).sourceId(); + Types.NestedField sourceField = schema.findField(sourceFieldId); + Preconditions.checkArgument( + sourceField != null, "Cannot find source field: %s", sourceFieldId); + Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); + // There could be multiple transformations on the same source column, like in the PartitionKey + // case. To resolve the collision, field id is set to transform index and field name is set to + // sourceFieldName_transformIndex + Types.NestedField transformedField = + Types.NestedField.of( + i, + sourceField.isOptional(), + sourceField.name() + '_' + i, + transformedType, + sourceField.doc()); + transformedFields.add(transformedField); + } + + return new Schema(transformedFields); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java new file mode 100644 index 000000000000..0485fdb7fa04 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Set; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.api.Test; + +public class TestRangePartitioner { + private final int numPartitions = 4; + + @Test + public void testRoundRobinRecordsBeforeStatisticsAvailable() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + results.add( + partitioner.partition( + StatisticsOrRecord.fromRecord(GenericRowData.of(StringData.fromString("a"), 1)), + numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } + + @Test + public void testRoundRobinStatisticsWrapper() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + GlobalStatistics statistics = + GlobalStatistics.fromRangeBounds(1L, new SortKey[] {CHAR_KEYS.get("a")}); + results.add( + partitioner.partition(StatisticsOrRecord.fromStatistics(statistics), numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java new file mode 100644 index 000000000000..378c6afff077 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.junit.jupiter.api.Test; + +public class TestSketchRangePartitioner { + // sort on the long id field + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final int NUM_PARTITIONS = 16; + private static final long RANGE_STEP = 1_000; + private static final long MAX_ID = RANGE_STEP * NUM_PARTITIONS; + private static final SortKey[] RANGE_BOUNDS = createRangeBounds(); + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
    + *
  • age <= 15 + *
  • age > 15 && age <= 32 + *
  • age >32 && age <= 60 + *
  • age > 60 + *
+ */ + private static SortKey[] createRangeBounds() { + SortKey[] rangeBounds = new SortKey[NUM_PARTITIONS - 1]; + for (int i = 0; i < NUM_PARTITIONS - 1; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString("data"), + RANGE_STEP * (i + 1), + StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + sortKey.wrap(keyWrapper); + rangeBounds[i] = sortKey; + } + + return rangeBounds; + } + + @Test + public void testRangePartitioningWithRangeBounds() { + SketchRangePartitioner partitioner = + new SketchRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, RANGE_BOUNDS); + GenericRowData row = + GenericRowData.of(StringData.fromString("data"), 0L, StringData.fromString("2023-06-20")); + for (long id = 0; id < MAX_ID; ++id) { + row.setField(1, id); + int partition = partitioner.partition(row, NUM_PARTITIONS); + assertThat(partition).isGreaterThanOrEqualTo(0).isLessThan(NUM_PARTITIONS); + int expectedPartition = id == 0L ? 0 : (int) ((id - 1) / RANGE_STEP); + assertThat(partition).isEqualTo(expectedPartition); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java index 31dae5c76aeb..16202c075ea0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -19,10 +19,13 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.SortKey; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestSketchUtil { @Test @@ -55,7 +58,7 @@ public void testRangeBoundsOneChannel() { assertThat( SketchUtil.rangeBounds( 1, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -72,7 +75,7 @@ public void testRangeBoundsDivisible() { assertThat( SketchUtil.rangeBounds( 3, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -90,7 +93,7 @@ public void testRangeBoundsNonDivisible() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -113,7 +116,7 @@ public void testRangeBoundsSkipDuplicates() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -130,4 +133,57 @@ public void testRangeBoundsSkipDuplicates() { // skipped duplicate c's .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java new file mode 100644 index 000000000000..1be7e27f2c01 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeyUtil { + @Test + public void testResultSchema() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "ratio", Types.DoubleType.get()), + Types.NestedField.optional( + 3, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), + Types.NestedField.optional( + 14, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get()), + Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc("ratio") + .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("user.location.blob", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + + assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) + .isEqualTo( + Types.StructType.of( + Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), + Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), + Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), + Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); + } +}