From 60f6611e5ea07404fb000c94565b9186057ec302 Mon Sep 17 00:00:00 2001 From: tsreaper Date: Tue, 12 Nov 2024 22:21:54 +0800 Subject: [PATCH 001/157] [flink] Support updating row type to another row type in Flink (#4499) --- .../paimon/catalog/AbstractCatalog.java | 3 +- .../apache/paimon/schema/SchemaChange.java | 56 ++++---- .../apache/paimon/schema/SchemaManager.java | 22 ++-- .../paimon/schema/SchemaManagerTest.java | 20 +-- .../UpdatedDataFieldsProcessFunctionBase.java | 8 +- .../org/apache/paimon/flink/FlinkCatalog.java | 120 ++++++++++++++++-- .../paimon/flink/SchemaChangeITCase.java | 51 +++++++- .../org/apache/paimon/spark/SparkCatalog.java | 8 +- 8 files changed, 213 insertions(+), 75 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index c2e4afe5d533e..a1cf941cda621 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -48,6 +48,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -560,7 +561,7 @@ private void validateFieldNameCaseInsensitiveInSchemaChange(List c for (SchemaChange change : changes) { if (change instanceof SchemaChange.AddColumn) { SchemaChange.AddColumn addColumn = (SchemaChange.AddColumn) change; - fieldNames.addAll(addColumn.fieldNames()); + fieldNames.addAll(Arrays.asList(addColumn.fieldNames())); } else if (change instanceof SchemaChange.RenameColumn) { SchemaChange.RenameColumn rename = (SchemaChange.RenameColumn) change; fieldNames.add(rename.newName()); diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java index 1c1d601bced8a..cefa3c6eb9e74 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java @@ -25,8 +25,6 @@ import java.io.Serializable; import java.util.Arrays; -import java.util.Collections; -import java.util.List; import java.util.Objects; /** @@ -54,46 +52,45 @@ static SchemaChange addColumn(String fieldName, DataType dataType) { } static SchemaChange addColumn(String fieldName, DataType dataType, String comment) { - return new AddColumn(Collections.singletonList(fieldName), dataType, comment, null); + return new AddColumn(new String[] {fieldName}, dataType, comment, null); } static SchemaChange addColumn(String fieldName, DataType dataType, String comment, Move move) { - return new AddColumn(Collections.singletonList(fieldName), dataType, comment, move); + return new AddColumn(new String[] {fieldName}, dataType, comment, move); } static SchemaChange addColumn( - List fieldNames, DataType dataType, String comment, Move move) { + String[] fieldNames, DataType dataType, String comment, Move move) { return new AddColumn(fieldNames, dataType, comment, move); } static SchemaChange renameColumn(String fieldName, String newName) { - return new RenameColumn(Collections.singletonList(fieldName), newName); + return new RenameColumn(new String[] {fieldName}, newName); } - static SchemaChange renameColumn(List fieldNames, String newName) { + static SchemaChange renameColumn(String[] fieldNames, String newName) { return new RenameColumn(fieldNames, newName); } static SchemaChange dropColumn(String fieldName) { - return new DropColumn(Collections.singletonList(fieldName)); + return new DropColumn(new String[] {fieldName}); } - static SchemaChange dropColumn(List fieldNames) { + static SchemaChange dropColumn(String[] fieldNames) { return new DropColumn(fieldNames); } static SchemaChange updateColumnType(String fieldName, DataType newDataType) { - return new UpdateColumnType(Collections.singletonList(fieldName), newDataType, false); + return new UpdateColumnType(new String[] {fieldName}, newDataType, false); } static SchemaChange updateColumnType( String fieldName, DataType newDataType, boolean keepNullability) { - return new UpdateColumnType( - Collections.singletonList(fieldName), newDataType, keepNullability); + return new UpdateColumnType(new String[] {fieldName}, newDataType, keepNullability); } static SchemaChange updateColumnType( - List fieldNames, DataType newDataType, boolean keepNullability) { + String[] fieldNames, DataType newDataType, boolean keepNullability) { return new UpdateColumnType(fieldNames, newDataType, keepNullability); } @@ -228,20 +225,19 @@ final class AddColumn implements SchemaChange { private static final long serialVersionUID = 1L; - private final List fieldNames; + private final String[] fieldNames; private final DataType dataType; private final String description; private final Move move; - private AddColumn( - List fieldNames, DataType dataType, String description, Move move) { + private AddColumn(String[] fieldNames, DataType dataType, String description, Move move) { this.fieldNames = fieldNames; this.dataType = dataType; this.description = description; this.move = move; } - public List fieldNames() { + public String[] fieldNames() { return fieldNames; } @@ -268,7 +264,7 @@ public boolean equals(Object o) { return false; } AddColumn addColumn = (AddColumn) o; - return Objects.equals(fieldNames, addColumn.fieldNames) + return Arrays.equals(fieldNames, addColumn.fieldNames) && dataType.equals(addColumn.dataType) && Objects.equals(description, addColumn.description) && move.equals(addColumn.move); @@ -288,15 +284,15 @@ final class RenameColumn implements SchemaChange { private static final long serialVersionUID = 1L; - private final List fieldNames; + private final String[] fieldNames; private final String newName; - private RenameColumn(List fieldNames, String newName) { + private RenameColumn(String[] fieldNames, String newName) { this.fieldNames = fieldNames; this.newName = newName; } - public List fieldNames() { + public String[] fieldNames() { return fieldNames; } @@ -313,7 +309,7 @@ public boolean equals(Object o) { return false; } RenameColumn that = (RenameColumn) o; - return Objects.equals(fieldNames, that.fieldNames) + return Arrays.equals(fieldNames, that.fieldNames) && Objects.equals(newName, that.newName); } @@ -330,13 +326,13 @@ final class DropColumn implements SchemaChange { private static final long serialVersionUID = 1L; - private final List fieldNames; + private final String[] fieldNames; - private DropColumn(List fieldNames) { + private DropColumn(String[] fieldNames) { this.fieldNames = fieldNames; } - public List fieldNames() { + public String[] fieldNames() { return fieldNames; } @@ -349,7 +345,7 @@ public boolean equals(Object o) { return false; } DropColumn that = (DropColumn) o; - return Objects.equals(fieldNames, that.fieldNames); + return Arrays.equals(fieldNames, that.fieldNames); } @Override @@ -363,19 +359,19 @@ final class UpdateColumnType implements SchemaChange { private static final long serialVersionUID = 1L; - private final List fieldNames; + private final String[] fieldNames; private final DataType newDataType; // If true, do not change the target field nullability private final boolean keepNullability; private UpdateColumnType( - List fieldNames, DataType newDataType, boolean keepNullability) { + String[] fieldNames, DataType newDataType, boolean keepNullability) { this.fieldNames = fieldNames; this.newDataType = newDataType; this.keepNullability = keepNullability; } - public List fieldNames() { + public String[] fieldNames() { return fieldNames; } @@ -396,7 +392,7 @@ public boolean equals(Object o) { return false; } UpdateColumnType that = (UpdateColumnType) o; - return Objects.equals(fieldNames, that.fieldNames) + return Arrays.equals(fieldNames, that.fieldNames) && newDataType.equals(that.newDataType); } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index 86ed96d5b01ba..28cc69cf99d7b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -290,7 +290,7 @@ public TableSchema commitChanges(List changes) DataType dataType = ReassignFieldId.reassign(addColumn.dataType(), highestFieldId); - new NestedColumnModifier(addColumn.fieldNames().toArray(new String[0])) { + new NestedColumnModifier(addColumn.fieldNames()) { @Override protected void updateLastColumn(List newFields, String fieldName) throws Catalog.ColumnAlreadyExistException { @@ -320,7 +320,7 @@ protected void updateLastColumn(List newFields, String fieldName) } else if (change instanceof RenameColumn) { RenameColumn rename = (RenameColumn) change; assertNotUpdatingPrimaryKeys(oldTableSchema, rename.fieldNames(), "rename"); - new NestedColumnModifier(rename.fieldNames().toArray(new String[0])) { + new NestedColumnModifier(rename.fieldNames()) { @Override protected void updateLastColumn(List newFields, String fieldName) throws Catalog.ColumnNotExistException, @@ -347,7 +347,7 @@ protected void updateLastColumn(List newFields, String fieldName) } else if (change instanceof DropColumn) { DropColumn drop = (DropColumn) change; dropColumnValidation(oldTableSchema, drop); - new NestedColumnModifier(drop.fieldNames().toArray(new String[0])) { + new NestedColumnModifier(drop.fieldNames()) { @Override protected void updateLastColumn(List newFields, String fieldName) throws Catalog.ColumnNotExistException { @@ -364,7 +364,7 @@ protected void updateLastColumn(List newFields, String fieldName) assertNotUpdatingPrimaryKeys(oldTableSchema, update.fieldNames(), "update"); updateNestedColumn( newFields, - update.fieldNames().toArray(new String[0]), + update.fieldNames(), (field) -> { DataType targetType = update.newDataType(); if (update.keepNullability()) { @@ -558,8 +558,8 @@ private static List applyNotNestedColumnRename( Map columnNames = Maps.newHashMap(); for (RenameColumn renameColumn : renames) { - if (renameColumn.fieldNames().size() == 1) { - columnNames.put(renameColumn.fieldNames().get(0), renameColumn.newName()); + if (renameColumn.fieldNames().length == 1) { + columnNames.put(renameColumn.fieldNames()[0], renameColumn.newName()); } } @@ -571,10 +571,10 @@ private static List applyNotNestedColumnRename( private static void dropColumnValidation(TableSchema schema, DropColumn change) { // primary keys and partition keys can't be nested columns - if (change.fieldNames().size() > 1) { + if (change.fieldNames().length > 1) { return; } - String columnToDrop = change.fieldNames().get(0); + String columnToDrop = change.fieldNames()[0]; if (schema.partitionKeys().contains(columnToDrop) || schema.primaryKeys().contains(columnToDrop)) { throw new UnsupportedOperationException( @@ -583,12 +583,12 @@ private static void dropColumnValidation(TableSchema schema, DropColumn change) } private static void assertNotUpdatingPrimaryKeys( - TableSchema schema, List fieldNames, String operation) { + TableSchema schema, String[] fieldNames, String operation) { // partition keys can't be nested columns - if (fieldNames.size() > 1) { + if (fieldNames.length > 1) { return; } - String columnToRename = fieldNames.get(0); + String columnToRename = fieldNames[0]; if (schema.partitionKeys().contains(columnToRename)) { throw new UnsupportedOperationException( String.format( diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java index ac8d4cd91e1d8..088cb72f92e6c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java @@ -555,7 +555,7 @@ public void testAddAndDropNestedColumns() throws Exception { SchemaChange addColumn = SchemaChange.addColumn( - Arrays.asList("v", "f2", "f3"), + new String[] {"v", "f2", "f3"}, DataTypes.STRING(), "", SchemaChange.Move.after("f3", "f1")); @@ -579,11 +579,11 @@ public void testAddAndDropNestedColumns() throws Exception { .hasMessageContaining("Column v.f2.f3 already exists"); SchemaChange middleColumnNotExistAddColumn = SchemaChange.addColumn( - Arrays.asList("v", "invalid", "f4"), DataTypes.STRING(), "", null); + new String[] {"v", "invalid", "f4"}, DataTypes.STRING(), "", null); assertThatCode(() -> manager.commitChanges(middleColumnNotExistAddColumn)) .hasMessageContaining("Column v.invalid does not exist"); - SchemaChange dropColumn = SchemaChange.dropColumn(Arrays.asList("v", "f2", "f1")); + SchemaChange dropColumn = SchemaChange.dropColumn(new String[] {"v", "f2", "f1"}); manager.commitChanges(dropColumn); innerType = @@ -602,7 +602,7 @@ public void testAddAndDropNestedColumns() throws Exception { assertThatCode(() -> manager.commitChanges(dropColumn)) .hasMessageContaining("Column v.f2.f1 does not exist"); SchemaChange middleColumnNotExistDropColumn = - SchemaChange.dropColumn(Arrays.asList("v", "invalid", "f2")); + SchemaChange.dropColumn(new String[] {"v", "invalid", "f2"}); assertThatCode(() -> manager.commitChanges(middleColumnNotExistDropColumn)) .hasMessageContaining("Column v.invalid does not exist"); } @@ -632,7 +632,7 @@ public void testRenameNestedColumns() throws Exception { manager.createTable(schema); SchemaChange renameColumn = - SchemaChange.renameColumn(Arrays.asList("v", "f2", "f1"), "f100"); + SchemaChange.renameColumn(new String[] {"v", "f2", "f1"}, "f100"); manager.commitChanges(renameColumn); innerType = @@ -649,17 +649,17 @@ public void testRenameNestedColumns() throws Exception { assertThat(manager.latest().get().logicalRowType()).isEqualTo(outerType); SchemaChange middleColumnNotExistRenameColumn = - SchemaChange.renameColumn(Arrays.asList("v", "invalid", "f2"), "f200"); + SchemaChange.renameColumn(new String[] {"v", "invalid", "f2"}, "f200"); assertThatCode(() -> manager.commitChanges(middleColumnNotExistRenameColumn)) .hasMessageContaining("Column v.invalid does not exist"); SchemaChange lastColumnNotExistRenameColumn = - SchemaChange.renameColumn(Arrays.asList("v", "f2", "invalid"), "new_invalid"); + SchemaChange.renameColumn(new String[] {"v", "f2", "invalid"}, "new_invalid"); assertThatCode(() -> manager.commitChanges(lastColumnNotExistRenameColumn)) .hasMessageContaining("Column v.f2.invalid does not exist"); SchemaChange newNameAlreadyExistRenameColumn = - SchemaChange.renameColumn(Arrays.asList("v", "f2", "f2"), "f100"); + SchemaChange.renameColumn(new String[] {"v", "f2", "f2"}, "f100"); assertThatCode(() -> manager.commitChanges(newNameAlreadyExistRenameColumn)) .hasMessageContaining("Column v.f2.f100 already exists"); } @@ -690,7 +690,7 @@ public void testUpdateNestedColumnType() throws Exception { SchemaChange updateColumnType = SchemaChange.updateColumnType( - Arrays.asList("v", "f2", "f1"), DataTypes.BIGINT(), true); + new String[] {"v", "f2", "f1"}, DataTypes.BIGINT(), true); manager.commitChanges(updateColumnType); innerType = @@ -708,7 +708,7 @@ public void testUpdateNestedColumnType() throws Exception { SchemaChange middleColumnNotExistUpdateColumnType = SchemaChange.updateColumnType( - Arrays.asList("v", "invalid", "f1"), DataTypes.BIGINT(), true); + new String[] {"v", "invalid", "f1"}, DataTypes.BIGINT(), true); assertThatCode(() -> manager.commitChanges(middleColumnNotExistUpdateColumnType)) .hasMessageContaining("Column v.invalid does not exist"); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java index 0e93fdb073244..c2e928bd4a0aa 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java @@ -101,7 +101,7 @@ protected void applySchemaChange( SchemaChange.UpdateColumnType updateColumnType = (SchemaChange.UpdateColumnType) schemaChange; Preconditions.checkState( - updateColumnType.fieldNames().size() == 1, + updateColumnType.fieldNames().length == 1, "Paimon CDC currently does not support nested type schema evolution."); TableSchema schema = schemaManager @@ -110,11 +110,11 @@ protected void applySchemaChange( () -> new RuntimeException( "Table does not exist. This is unexpected.")); - int idx = schema.fieldNames().indexOf(updateColumnType.fieldNames().get(0)); + int idx = schema.fieldNames().indexOf(updateColumnType.fieldNames()[0]); Preconditions.checkState( idx >= 0, "Field name " - + updateColumnType.fieldNames().get(0) + + updateColumnType.fieldNames()[0] + " does not exist in table. This is unexpected."); DataType oldType = schema.fields().get(idx).type(); DataType newType = updateColumnType.newDataType(); @@ -126,7 +126,7 @@ protected void applySchemaChange( throw new UnsupportedOperationException( String.format( "Cannot convert field %s from type %s to %s of Paimon table %s.", - updateColumnType.fieldNames().get(0), + updateColumnType.fieldNames()[0], oldType, newType, identifier.getFullName())); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index cae6e6f0e3672..ae30fa569d59e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -37,6 +37,8 @@ import org.apache.paimon.table.Table; import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypeRoot; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.paimon.utils.Preconditions; @@ -98,7 +100,6 @@ import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.factories.Factory; import org.apache.flink.table.procedures.Procedure; -import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -110,11 +111,13 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -595,17 +598,12 @@ private List toSchemaChange( if (!oldTableNonPhysicalColumnIndex.containsKey( ((ModifyPhysicalColumnType) change).getOldColumn().getName())) { ModifyPhysicalColumnType modify = (ModifyPhysicalColumnType) change; - LogicalType newColumnType = modify.getNewType().getLogicalType(); - LogicalType oldColumnType = modify.getOldColumn().getDataType().getLogicalType(); - if (newColumnType.isNullable() != oldColumnType.isNullable()) { - schemaChanges.add( - SchemaChange.updateColumnNullability( - modify.getNewColumn().getName(), newColumnType.isNullable())); - } - schemaChanges.add( - SchemaChange.updateColumnType( - modify.getOldColumn().getName(), - LogicalTypeConversion.toDataType(newColumnType))); + generateNestedColumnUpdates( + Collections.singletonList(modify.getOldColumn().getName()), + LogicalTypeConversion.toDataType( + modify.getOldColumn().getDataType().getLogicalType()), + LogicalTypeConversion.toDataType(modify.getNewType().getLogicalType()), + schemaChanges); } return schemaChanges; } else if (change instanceof ModifyColumnPosition) { @@ -670,6 +668,104 @@ && handleMaterializedTableChange(change, schemaChanges)) { throw new UnsupportedOperationException("Change is not supported: " + change.getClass()); } + private void generateNestedColumnUpdates( + List fieldNames, + org.apache.paimon.types.DataType oldType, + org.apache.paimon.types.DataType newType, + List schemaChanges) { + if (oldType.getTypeRoot() == DataTypeRoot.ROW) { + Preconditions.checkArgument( + newType.getTypeRoot() == DataTypeRoot.ROW, + "Column " + + String.join(".", fieldNames) + + " can only be updated to row type, and cannot be updated to " + + newType + + " type"); + org.apache.paimon.types.RowType oldRowType = (org.apache.paimon.types.RowType) oldType; + org.apache.paimon.types.RowType newRowType = (org.apache.paimon.types.RowType) newType; + + // check that existing fields have same order + Map oldFieldOrders = new HashMap<>(); + for (int i = 0; i < oldRowType.getFieldCount(); i++) { + oldFieldOrders.put(oldRowType.getFields().get(i).name(), i); + } + int lastIdx = -1; + String lastFieldName = ""; + for (DataField newField : newRowType.getFields()) { + String name = newField.name(); + if (oldFieldOrders.containsKey(name)) { + int idx = oldFieldOrders.get(name); + Preconditions.checkState( + lastIdx < idx, + "Order of existing fields in column %s must be kept the same. " + + "However, field %s and %s have changed their orders.", + String.join(".", fieldNames), + lastFieldName, + name); + lastIdx = idx; + lastFieldName = name; + } + } + + // drop fields + Set newFieldNames = new HashSet<>(newRowType.getFieldNames()); + for (String name : oldRowType.getFieldNames()) { + if (!newFieldNames.contains(name)) { + List dropColumnNames = new ArrayList<>(fieldNames); + dropColumnNames.add(name); + schemaChanges.add( + SchemaChange.dropColumn(dropColumnNames.toArray(new String[0]))); + } + } + + for (int i = 0; i < newRowType.getFieldCount(); i++) { + DataField field = newRowType.getFields().get(i); + String name = field.name(); + List fullFieldNames = new ArrayList<>(fieldNames); + fullFieldNames.add(name); + if (!oldFieldOrders.containsKey(name)) { + // add fields + SchemaChange.Move move; + if (i == 0) { + move = SchemaChange.Move.first(name); + } else { + String lastName = newRowType.getFields().get(i - 1).name(); + move = SchemaChange.Move.after(name, lastName); + } + schemaChanges.add( + SchemaChange.addColumn( + fullFieldNames.toArray(new String[0]), + field.type(), + field.description(), + move)); + } else { + // update existing fields + DataField oldField = oldRowType.getFields().get(oldFieldOrders.get(name)); + if (!Objects.equals(oldField.description(), field.description())) { + schemaChanges.add( + SchemaChange.updateColumnComment( + fullFieldNames.toArray(new String[0]), + field.description())); + } + generateNestedColumnUpdates( + fullFieldNames, oldField.type(), field.type(), schemaChanges); + } + } + } else { + if (!oldType.equalsIgnoreNullable(newType)) { + schemaChanges.add( + SchemaChange.updateColumnType( + fieldNames.toArray(new String[0]), newType, false)); + } + } + + if (oldType.isNullable() != newType.isNullable()) { + schemaChanges.add( + SchemaChange.updateColumnNullability( + fieldNames.toArray(new String[0]), newType.isNullable())); + } + } + /** * Try handle change related to materialized table. * diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java index 08f79efccb763..ba161fe84008f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java @@ -25,6 +25,8 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.types.Row; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.time.format.DateTimeFormatter; import java.util.List; @@ -35,6 +37,7 @@ import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** ITCase for schema changes. */ @@ -1015,7 +1018,6 @@ public void testSequenceFieldSortOrder() { sql("INSERT INTO T1 VALUES ('a', 'b', 'l')"); sql("INSERT INTO T1 VALUES ('a', 'd', 'n')"); sql("INSERT INTO T1 VALUES ('a', 'e', 'm')"); - List sql = sql("select * from T1"); assertThat(sql("select * from T1").toString()).isEqualTo("[+I[a, d, n]]"); // test for get small record @@ -1024,7 +1026,6 @@ public void testSequenceFieldSortOrder() { sql("INSERT INTO T2 VALUES ('a', 'b', 1)"); sql("INSERT INTO T2 VALUES ('a', 'd', 3)"); sql("INSERT INTO T2 VALUES ('a', 'e', 2)"); - sql = sql("select * from T2"); assertThat(sql("select * from T2").toString()).isEqualTo("[+I[a, b, 1]]"); // test for get largest record @@ -1033,7 +1034,6 @@ public void testSequenceFieldSortOrder() { sql("INSERT INTO T3 VALUES ('a', 'b', 1.0)"); sql("INSERT INTO T3 VALUES ('a', 'd', 3.0)"); sql("INSERT INTO T3 VALUES ('a', 'e', 2.0)"); - sql = sql("select * from T3"); assertThat(sql("select * from T3").toString()).isEqualTo("[+I[a, d, 3.0]]"); } @@ -1089,4 +1089,49 @@ public void testAlterBucket() { UnsupportedOperationException.class, "Cannot change bucket to -1.")); } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testUpdateNestedColumn(String formatType) { + sql( + "CREATE TABLE T " + + "( k INT, v ROW(f1 INT, f2 ROW(f1 STRING, f2 INT NOT NULL)), PRIMARY KEY (k) NOT ENFORCED ) " + + "WITH ( 'bucket' = '1', 'file.format' = '" + + formatType + + "' )"); + sql( + "INSERT INTO T VALUES (1, ROW(10, ROW('apple', 100))), (2, ROW(20, ROW('banana', 200)))"); + assertThat(sql("SELECT * FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, Row.of(10, Row.of("apple", 100))), + Row.of(2, Row.of(20, Row.of("banana", 200)))); + + sql("ALTER TABLE T MODIFY (v ROW(f1 BIGINT, f2 ROW(f3 DOUBLE, f2 INT), f3 STRING))"); + sql( + "INSERT INTO T VALUES " + + "(1, ROW(1000000000001, ROW(101.0, 101), 'cat')), " + + "(3, ROW(3000000000001, ROW(301.0, CAST(NULL AS INT)), 'dog'))"); + assertThat(sql("SELECT * FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, Row.of(1000000000001L, Row.of(101.0, 101), "cat")), + Row.of(2, Row.of(20L, Row.of(null, 200), null)), + Row.of(3, Row.of(3000000000001L, Row.of(301.0, null), "dog"))); + + sql( + "ALTER TABLE T MODIFY (v ROW(f1 BIGINT, f2 ROW(f3 DOUBLE, f1 STRING, f2 INT), f3 STRING))"); + sql( + "INSERT INTO T VALUES " + + "(1, ROW(1000000000002, ROW(102.0, 'APPLE', 102), 'cat')), " + + "(4, ROW(4000000000002, ROW(402.0, 'LEMON', 402), 'tiger'))"); + assertThat(sql("SELECT k, v.f2.f1, v.f3 FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, "APPLE", "cat"), + Row.of(2, null, null), + Row.of(3, null, "dog"), + Row.of(4, "LEMON", "tiger")); + + assertThatCode(() -> sql("ALTER TABLE T MODIFY (v ROW(f1 BIGINT, f2 INT, f3 STRING))")) + .hasRootCauseMessage( + "Column v.f2 can only be updated to row type, and cannot be updated to INT type"); + } } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index 5fde2c56596f6..89448c1f4310a 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -373,20 +373,20 @@ private SchemaChange toSchemaChange(TableChange change) { TableChange.AddColumn add = (TableChange.AddColumn) change; SchemaChange.Move move = getMove(add.position(), add.fieldNames()); return SchemaChange.addColumn( - Arrays.asList(add.fieldNames()), + add.fieldNames(), toPaimonType(add.dataType()).copy(add.isNullable()), add.comment(), move); } else if (change instanceof TableChange.RenameColumn) { TableChange.RenameColumn rename = (TableChange.RenameColumn) change; - return SchemaChange.renameColumn(Arrays.asList(rename.fieldNames()), rename.newName()); + return SchemaChange.renameColumn(rename.fieldNames(), rename.newName()); } else if (change instanceof TableChange.DeleteColumn) { TableChange.DeleteColumn delete = (TableChange.DeleteColumn) change; - return SchemaChange.dropColumn(Arrays.asList(delete.fieldNames())); + return SchemaChange.dropColumn(delete.fieldNames()); } else if (change instanceof TableChange.UpdateColumnType) { TableChange.UpdateColumnType update = (TableChange.UpdateColumnType) change; return SchemaChange.updateColumnType( - Arrays.asList(update.fieldNames()), toPaimonType(update.newDataType()), true); + update.fieldNames(), toPaimonType(update.newDataType()), true); } else if (change instanceof TableChange.UpdateColumnNullability) { TableChange.UpdateColumnNullability update = (TableChange.UpdateColumnNullability) change; From 4d32ed8b558d13ead30df74efe1941704c822bd8 Mon Sep 17 00:00:00 2001 From: wangwj Date: Tue, 12 Nov 2024 22:23:44 +0800 Subject: [PATCH 002/157] [core] FilesTable file_path col should be the path instead of name (#4503) --- .../org/apache/paimon/table/system/FilesTable.java | 4 +++- .../apache/paimon/table/system/FilesTableTest.java | 11 ++++++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java index 53d2078126737..0232fc2d2ddee 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java @@ -385,7 +385,9 @@ private LazyGenericRow toRow( partitionConverter.convert( dataSplit.partition()))), dataSplit::bucket, - () -> BinaryString.fromString(dataFileMeta.fileName()), + () -> + BinaryString.fromString( + dataSplit.bucketPath() + "/" + dataFileMeta.fileName()), () -> BinaryString.fromString( DataFilePathFactory.formatIdentifier( diff --git a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java index 1a692270ac5be..f0280560c2675 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java @@ -191,7 +191,16 @@ private List getExpectedResult(long snapshotId) { BinaryString.fromString( Arrays.toString(new String[] {partition1, partition2})), fileEntry.bucket(), - BinaryString.fromString(file.fileName()), + BinaryString.fromString( + table.location() + + "/pt1=" + + partition1 + + "/pt2=" + + partition2 + + "/bucket-" + + fileEntry.bucket() + + "/" + + file.fileName()), BinaryString.fromString(file.fileFormat()), file.schemaId(), file.level(), From c3a83195f40ea33f07a447d4d39f98890e4a4334 Mon Sep 17 00:00:00 2001 From: Kerwin <37063904+zhuangchong@users.noreply.github.com> Date: Tue, 12 Nov 2024 22:24:19 +0800 Subject: [PATCH 003/157] [doc] fix flink procedure document sql syntax error. (#4504) --- docs/content/flink/procedures.md | 2 +- docs/content/maintenance/manage-snapshots.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/content/flink/procedures.md b/docs/content/flink/procedures.md index d55a8daddef66..7e669a89d43bf 100644 --- a/docs/content/flink/procedures.md +++ b/docs/content/flink/procedures.md @@ -409,7 +409,7 @@ All available procedures are listed below. expire_snapshots -- Use named argument
- CALL [catalog.]sys.reset_consumer(
+ CALL [catalog.]sys.expire_snapshots(
`table` => 'identifier',
retain_max => 'retain_max',
retain_min => 'retain_min',
diff --git a/docs/content/maintenance/manage-snapshots.md b/docs/content/maintenance/manage-snapshots.md index eed0f72c08c2c..721d5d0bafb28 100644 --- a/docs/content/maintenance/manage-snapshots.md +++ b/docs/content/maintenance/manage-snapshots.md @@ -308,9 +308,9 @@ submit a `remove_orphan_files` job to clean them: {{< tab "Spark SQL/Flink SQL" >}} ```sql -CALL sys.remove_orphan_files(`table` => "my_db.my_table", [older_than => "2023-10-31 12:00:00"]) +CALL sys.remove_orphan_files(`table` => 'my_db.my_table', [older_than => '2023-10-31 12:00:00']) -CALL sys.remove_orphan_files(`table` => "my_db.*", [older_than => "2023-10-31 12:00:00"]) +CALL sys.remove_orphan_files(`table` => 'my_db.*', [older_than => '2023-10-31 12:00:00']) ``` {{< /tab >}} From 3ffeab94956882862802c2abfa73915e1789fe86 Mon Sep 17 00:00:00 2001 From: askwang <135721692+askwang@users.noreply.github.com> Date: Wed, 13 Nov 2024 10:59:08 +0800 Subject: [PATCH 004/157] [doc] Remove unused config in record-level expire (#4509) --- docs/content/primary-key-table/compaction.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/content/primary-key-table/compaction.md b/docs/content/primary-key-table/compaction.md index ada7e0289b359..bee8c16e46e9d 100644 --- a/docs/content/primary-key-table/compaction.md +++ b/docs/content/primary-key-table/compaction.md @@ -76,7 +76,6 @@ In compaction, you can configure record-Level expire time to expire records, you 1. `'record-level.expire-time'`: time retain for records. 2. `'record-level.time-field'`: time field for record level expire. -3. `'record-level.time-field-type'`: time field type for record level expire, it can be seconds-int,seconds-long or millis-long. Expiration happens in compaction, and there is no strong guarantee to expire records in time. From 5e3c24b89876eaa7d5a4d7783ac01b64a939fae7 Mon Sep 17 00:00:00 2001 From: askwang <135721692+askwang@users.noreply.github.com> Date: Wed, 13 Nov 2024 11:19:56 +0800 Subject: [PATCH 005/157] [core] Adjust default value of 'snapshot.expire.limit' (#4508) --- docs/layouts/shortcodes/generated/core_configuration.html | 2 +- paimon-common/src/main/java/org/apache/paimon/CoreOptions.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 1305dfe9263b6..7287cacc2c53e 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -751,7 +751,7 @@
snapshot.expire.limit
- 10 + 50 Integer The maximum number of snapshots allowed to expire at a time. diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index bb1661d6fd829..db603c5b61f06 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -315,7 +315,7 @@ public class CoreOptions implements Serializable { public static final ConfigOption SNAPSHOT_EXPIRE_LIMIT = key("snapshot.expire.limit") .intType() - .defaultValue(10) + .defaultValue(50) .withDescription( "The maximum number of snapshots allowed to expire at a time."); From 72c25d58c575d4966fd2cb12fcc883071e393890 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 13 Nov 2024 11:27:07 +0800 Subject: [PATCH 006/157] [core] Add Table.uuid method (#4213) --- .../paimon/catalog/AbstractCatalog.java | 32 +++++++++++++++++-- .../apache/paimon/schema/SchemaManager.java | 19 +++++++++++ .../paimon/table/AbstractFileStoreTable.java | 9 ++++++ .../paimon/table/CatalogEnvironment.java | 10 +++++- .../paimon/table/DelegatedFileStoreTable.java | 5 +++ .../java/org/apache/paimon/table/Table.java | 9 ++++++ .../paimon/catalog/CatalogTestBase.java | 12 +++++++ .../org/apache/paimon/hive/HiveCatalog.java | 19 +++++++++-- 8 files changed, 108 insertions(+), 7 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index a1cf941cda621..93018f12c9f1b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -424,16 +424,17 @@ public Table getTable(Identifier identifier) throws TableNotExistException { protected Table getDataOrFormatTable(Identifier identifier) throws TableNotExistException { Preconditions.checkArgument(identifier.getSystemTableName() == null); - TableSchema tableSchema = getDataTableSchema(identifier); + TableMeta tableMeta = getDataTableMeta(identifier); return FileStoreTableFactory.create( fileIO, getTableLocation(identifier), - tableSchema, + tableMeta.schema, new CatalogEnvironment( identifier, + tableMeta.uuid, Lock.factory( lockFactory().orElse(null), lockContext().orElse(null), identifier), - metastoreClientFactory(identifier, tableSchema).orElse(null), + metastoreClientFactory(identifier, tableMeta.schema).orElse(null), lineageMetaFactory)); } @@ -475,6 +476,10 @@ public Map> allTablePaths() { } } + protected TableMeta getDataTableMeta(Identifier identifier) throws TableNotExistException { + return new TableMeta(getDataTableSchema(identifier), null); + } + protected abstract TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException; @@ -627,4 +632,25 @@ public Optional tableSchemaInFileSystem(Path tablePath, String bran } }); } + + /** Table metadata. */ + protected static class TableMeta { + + private final TableSchema schema; + @Nullable private final String uuid; + + public TableMeta(TableSchema schema, @Nullable String uuid) { + this.schema = schema; + this.uuid = uuid; + } + + public TableSchema schema() { + return schema; + } + + @Nullable + public String uuid() { + return uuid; + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index 28cc69cf99d7b..86e365a88f831 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -78,6 +78,7 @@ import static org.apache.paimon.catalog.Identifier.UNKNOWN_DATABASE; import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; import static org.apache.paimon.utils.FileUtils.listVersionedFiles; +import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.Preconditions.checkState; /** Schema Manager to manage schema versions. */ @@ -123,6 +124,24 @@ public Optional latest() { } } + public long earliestCreationTime() { + try { + long earliest = 0; + if (!schemaExists(0)) { + Optional min = + listVersionedFiles(fileIO, schemaDirectory(), SCHEMA_PREFIX) + .reduce(Math::min); + checkArgument(min.isPresent()); + earliest = min.get(); + } + + Path schemaPath = toSchemaPath(earliest); + return fileIO.getFileStatus(schemaPath).getModificationTime(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + public List listAll() { return listAllIds().stream().map(this::schema).collect(Collectors.toList()); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index af0c3d71ec1ca..07c0e88645ac3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -167,6 +167,15 @@ public Identifier identifier() { : identifier; } + @Override + public String uuid() { + if (catalogEnvironment.uuid() != null) { + return catalogEnvironment.uuid(); + } + long earliestCreationTime = schemaManager().earliestCreationTime(); + return fullName() + "." + earliestCreationTime; + } + @Override public Optional statistics() { Snapshot snapshot = TimeTravelUtil.resolveSnapshot(this); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java b/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java index ebaff12661556..9ff5f9b4f6a83 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java @@ -36,23 +36,26 @@ public class CatalogEnvironment implements Serializable { private static final long serialVersionUID = 1L; @Nullable private final Identifier identifier; + @Nullable private final String uuid; private final Lock.Factory lockFactory; @Nullable private final MetastoreClient.Factory metastoreClientFactory; @Nullable private final LineageMetaFactory lineageMetaFactory; public CatalogEnvironment( @Nullable Identifier identifier, + @Nullable String uuid, Lock.Factory lockFactory, @Nullable MetastoreClient.Factory metastoreClientFactory, @Nullable LineageMetaFactory lineageMetaFactory) { this.identifier = identifier; + this.uuid = uuid; this.lockFactory = lockFactory; this.metastoreClientFactory = metastoreClientFactory; this.lineageMetaFactory = lineageMetaFactory; } public static CatalogEnvironment empty() { - return new CatalogEnvironment(null, Lock.emptyFactory(), null, null); + return new CatalogEnvironment(null, null, Lock.emptyFactory(), null, null); } @Nullable @@ -60,6 +63,11 @@ public Identifier identifier() { return identifier; } + @Nullable + public String uuid() { + return uuid; + } + public Lock.Factory lockFactory() { return lockFactory; } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java index f6f3930baade8..2b369e5005cc4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java @@ -72,6 +72,11 @@ public String fullName() { return wrapped.fullName(); } + @Override + public String uuid() { + return wrapped.uuid(); + } + @Override public SnapshotReader newSnapshotReader() { return wrapped.newSnapshotReader(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/Table.java b/paimon-core/src/main/java/org/apache/paimon/table/Table.java index db6848f5f1a84..7ed7ba48a8ebd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/Table.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/Table.java @@ -52,10 +52,19 @@ public interface Table extends Serializable { /** A name to identify this table. */ String name(); + /** Full name of the table, default is database.tableName. */ default String fullName() { return name(); } + /** + * UUID of the table, metastore can provide the true UUID of this table, default is the full + * name. + */ + default String uuid() { + return fullName(); + } + /** Returns the row type of this table. */ RowType rowType(); diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java index f130920a7c0e4..98a9b92c5c38c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java @@ -948,4 +948,16 @@ public void testFormatTable() throws Exception { .isInstanceOf(Catalog.TableNotExistException.class); assertThat(catalog.getTable(newIdentifier)).isInstanceOf(FormatTable.class); } + + @Test + public void testTableUUID() throws Exception { + catalog.createDatabase("test_db", false); + Identifier identifier = Identifier.create("test_db", "test_table"); + catalog.createTable(identifier, DEFAULT_TABLE_SCHEMA, false); + Table table = catalog.getTable(identifier); + String uuid = table.uuid(); + assertThat(uuid).startsWith(identifier.getFullName() + "."); + assertThat(Long.parseLong(uuid.substring((identifier.getFullName() + ".").length()))) + .isGreaterThan(0); + } } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 2bf16c0f44b1d..8b8b62934dbdb 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -411,6 +411,18 @@ protected List listTablesImpl(String databaseName) { } } + @Override + protected TableMeta getDataTableMeta(Identifier identifier) throws TableNotExistException { + return getDataTableMeta(identifier, getHmsTable(identifier)); + } + + private TableMeta getDataTableMeta(Identifier identifier, Table table) + throws TableNotExistException { + return new TableMeta( + getDataTableSchema(identifier, table), + identifier.getFullName() + "." + table.getCreateTime()); + } + @Override public TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { Table table = getHmsTable(identifier); @@ -567,18 +579,19 @@ public org.apache.paimon.table.Table getDataOrFormatTable(Identifier identifier) Preconditions.checkArgument(identifier.getSystemTableName() == null); Table table = getHmsTable(identifier); try { - TableSchema tableSchema = getDataTableSchema(identifier, table); + TableMeta tableMeta = getDataTableMeta(identifier, table); return FileStoreTableFactory.create( fileIO, getTableLocation(identifier, table), - tableSchema, + tableMeta.schema(), new CatalogEnvironment( identifier, + tableMeta.uuid(), Lock.factory( lockFactory().orElse(null), lockContext().orElse(null), identifier), - metastoreClientFactory(identifier, tableSchema).orElse(null), + metastoreClientFactory(identifier, tableMeta.schema()).orElse(null), lineageMetaFactory)); } catch (TableNotExistException ignore) { } From 787a981cc2c009075d8c2aa5f3e2c66d5988aeeb Mon Sep 17 00:00:00 2001 From: Jingsong Date: Wed, 13 Nov 2024 11:32:13 +0800 Subject: [PATCH 007/157] [flink] Improve Exception message for consumer without expire time --- .../org/apache/paimon/flink/source/FlinkSourceBuilder.java | 4 +++- .../java/org/apache/paimon/flink/CatalogTableITCase.java | 6 ++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java index ed94043c035d8..a648bfba607d4 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java @@ -259,7 +259,9 @@ public DataStream build() { if (conf.contains(CoreOptions.CONSUMER_ID) && !conf.contains(CoreOptions.CONSUMER_EXPIRATION_TIME)) { throw new IllegalArgumentException( - "consumer.expiration-time should be specified when using consumer-id."); + "You need to configure 'consumer.expiration-time' (ALTER TABLE) and restart your write job for it" + + " to take effect, when you need consumer-id feature. This is to prevent consumers from leaving" + + " too many snapshots that could pose a risk to the file system."); } if (sourceBounded) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java index ba063248ee460..8a3e068a72a02 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java @@ -972,7 +972,8 @@ public void testConsumerIdExpInBatchMode() { "SELECT * FROM T /*+ OPTIONS('consumer-id' = 'test-id') */ WHERE a = 1")) .rootCause() .isInstanceOf(IllegalArgumentException.class) - .hasMessage("consumer.expiration-time should be specified when using consumer-id."); + .hasMessageContaining( + "You need to configure 'consumer.expiration-time' (ALTER TABLE) and restart your write job for it"); } @Test @@ -985,7 +986,8 @@ public void testConsumerIdExpInStreamingMode() { streamSqlIter( "SELECT * FROM T /*+ OPTIONS('consumer-id'='test-id') */")) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("consumer.expiration-time should be specified when using consumer-id."); + .hasMessageContaining( + "You need to configure 'consumer.expiration-time' (ALTER TABLE) and restart your write job for it"); } @Test From 2b94a33bc1dc841476b8970d9a0bb69a7e85502e Mon Sep 17 00:00:00 2001 From: Tan-JiaLiang Date: Wed, 13 Nov 2024 13:22:33 +0800 Subject: [PATCH 008/157] [core] Introduce bitmap index record reader (#4502) --- .../ApplyBitmapIndexFileRecordIterator.java | 78 +++++++++++++++++ .../bitmap/ApplyBitmapIndexRecordReader.java | 64 ++++++++++++++ .../apache/paimon/utils/RoaringBitmap32.java | 4 + .../ApplyDeletionVectorReader.java | 2 +- .../apache/paimon/io/FileIndexEvaluator.java | 9 ++ .../paimon/operation/RawFileSplitRead.java | 10 ++- .../table/AppendOnlyFileStoreTableTest.java | 11 +++ .../table/PrimaryKeyFileStoreTableTest.java | 87 +++++++++++++++++++ 8 files changed, 263 insertions(+), 2 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexFileRecordIterator.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexRecordReader.java diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexFileRecordIterator.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexFileRecordIterator.java new file mode 100644 index 0000000000000..eec931d3e98f8 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexFileRecordIterator.java @@ -0,0 +1,78 @@ +/* + * 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.paimon.fileindex.bitmap; + +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fs.Path; +import org.apache.paimon.reader.FileRecordIterator; +import org.apache.paimon.utils.RoaringBitmap32; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** + * A {@link FileRecordIterator} wraps a {@link FileRecordIterator} and {@link BitmapIndexResult}. + */ +public class ApplyBitmapIndexFileRecordIterator implements FileRecordIterator { + + private final FileRecordIterator iterator; + private final RoaringBitmap32 bitmap; + private final int last; + + public ApplyBitmapIndexFileRecordIterator( + FileRecordIterator iterator, BitmapIndexResult fileIndexResult) { + this.iterator = iterator; + this.bitmap = fileIndexResult.get(); + this.last = bitmap.last(); + } + + @Override + public long returnedPosition() { + return iterator.returnedPosition(); + } + + @Override + public Path filePath() { + return iterator.filePath(); + } + + @Nullable + @Override + public InternalRow next() throws IOException { + while (true) { + InternalRow next = iterator.next(); + if (next == null) { + return null; + } + int position = (int) returnedPosition(); + if (position > last) { + return null; + } + if (bitmap.contains(position)) { + return next; + } + } + } + + @Override + public void releaseBatch() { + iterator.releaseBatch(); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexRecordReader.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexRecordReader.java new file mode 100644 index 0000000000000..d5d15095f2eda --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexRecordReader.java @@ -0,0 +1,64 @@ +/* + * 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.paimon.fileindex.bitmap; + +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.reader.FileRecordIterator; +import org.apache.paimon.reader.RecordReader; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.paimon.utils.Preconditions.checkArgument; + +/** A {@link RecordReader} which apply {@link BitmapIndexResult} to filter record. */ +public class ApplyBitmapIndexRecordReader implements RecordReader { + + private final RecordReader reader; + + private final BitmapIndexResult fileIndexResult; + + public ApplyBitmapIndexRecordReader( + RecordReader reader, BitmapIndexResult fileIndexResult) { + this.reader = reader; + this.fileIndexResult = fileIndexResult; + } + + @Nullable + @Override + public RecordIterator readBatch() throws IOException { + RecordIterator batch = reader.readBatch(); + if (batch == null) { + return null; + } + + checkArgument( + batch instanceof FileRecordIterator, + "There is a bug, RecordIterator in ApplyBitmapIndexRecordReader must be FileRecordIterator"); + + return new ApplyBitmapIndexFileRecordIterator( + (FileRecordIterator) batch, fileIndexResult); + } + + @Override + public void close() throws IOException { + reader.close(); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/RoaringBitmap32.java b/paimon-common/src/main/java/org/apache/paimon/utils/RoaringBitmap32.java index 1d3468a9fcdb3..5f352f61cd3c6 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/RoaringBitmap32.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/RoaringBitmap32.java @@ -72,6 +72,10 @@ public long rangeCardinality(long start, long end) { return roaringBitmap.rangeCardinality(start, end); } + public int last() { + return roaringBitmap.last(); + } + public RoaringBitmap32 clone() { return new RoaringBitmap32(roaringBitmap.clone()); } diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java index 18ab033fb2762..c1dc16a78d950 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java @@ -60,7 +60,7 @@ public RecordIterator readBatch() throws IOException { checkArgument( batch instanceof FileRecordIterator, - "There is a bug, RecordIterator in ApplyDeletionVectorReader must be RecordWithPositionIterator"); + "There is a bug, RecordIterator in ApplyDeletionVectorReader must be FileRecordIterator"); return new ApplyDeletionFileRecordIterator( (FileRecordIterator) batch, deletionVector); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java b/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java index c34d1b0d3ba75..530b87165322f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java @@ -40,6 +40,15 @@ public static FileIndexResult evaluate( DataFileMeta file) throws IOException { if (dataFilter != null && !dataFilter.isEmpty()) { + byte[] embeddedIndex = file.embeddedIndex(); + if (embeddedIndex != null) { + try (FileIndexPredicate predicate = + new FileIndexPredicate(embeddedIndex, dataSchema.logicalRowType())) { + return predicate.evaluate( + PredicateBuilder.and(dataFilter.toArray(new Predicate[0]))); + } + } + List indexFiles = file.extraFiles().stream() .filter(name -> name.endsWith(DataFilePathFactory.INDEX_PATH_SUFFIX)) diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java index 9c612a9f8cf06..4a6fa5b3db7f2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java @@ -24,6 +24,8 @@ import org.apache.paimon.deletionvectors.DeletionVector; import org.apache.paimon.disk.IOManager; import org.apache.paimon.fileindex.FileIndexResult; +import org.apache.paimon.fileindex.bitmap.ApplyBitmapIndexRecordReader; +import org.apache.paimon.fileindex.bitmap.BitmapIndexResult; import org.apache.paimon.format.FileFormatDiscover; import org.apache.paimon.format.FormatKey; import org.apache.paimon.format.FormatReaderContext; @@ -212,7 +214,7 @@ private RecordReader createFileReader( dataFilePathFactory.toPath(file.fileName()), file.fileSize(), fileIndexResult); - FileRecordReader fileRecordReader = + RecordReader fileRecordReader = new FileRecordReader( bulkFormatMapping.getReaderFactory(), formatReaderContext, @@ -220,6 +222,12 @@ private RecordReader createFileReader( bulkFormatMapping.getCastMapping(), PartitionUtils.create(bulkFormatMapping.getPartitionPair(), partition)); + if (fileIndexResult instanceof BitmapIndexResult) { + fileRecordReader = + new ApplyBitmapIndexRecordReader( + fileRecordReader, (BitmapIndexResult) fileIndexResult); + } + DeletionVector deletionVector = dvFactory == null ? null : dvFactory.get(); if (deletionVector != null && !deletionVector.isEmpty()) { return new ApplyDeletionVectorReader(fileRecordReader, deletionVector); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java index 81dd30262058a..0328cc6bada34 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java @@ -79,6 +79,7 @@ import static org.apache.paimon.CoreOptions.BUCKET; import static org.apache.paimon.CoreOptions.BUCKET_KEY; import static org.apache.paimon.CoreOptions.FILE_INDEX_IN_MANIFEST_THRESHOLD; +import static org.apache.paimon.CoreOptions.METADATA_STATS_MODE; import static org.apache.paimon.io.DataFileTestUtils.row; import static org.apache.paimon.table.sink.KeyAndBucketExtractor.bucket; import static org.apache.paimon.table.sink.KeyAndBucketExtractor.bucketKeyHashCode; @@ -574,6 +575,7 @@ public void testBSIAndBitmapIndexInMemory() throws Exception { createUnawareBucketFileStoreTable( rowType, options -> { + options.set(METADATA_STATS_MODE, "NONE"); options.set( FileIndexOptions.FILE_INDEX + "." @@ -600,7 +602,11 @@ public void testBSIAndBitmapIndexInMemory() throws Exception { write.write(GenericRow.of(1, BinaryString.fromString("B"), 3L)); write.write(GenericRow.of(1, BinaryString.fromString("C"), 3L)); result.addAll(write.prepareCommit(true, 0)); + write.write(GenericRow.of(1, BinaryString.fromString("A"), 4L)); + write.write(GenericRow.of(1, BinaryString.fromString("B"), 3L)); write.write(GenericRow.of(1, BinaryString.fromString("C"), 4L)); + write.write(GenericRow.of(1, BinaryString.fromString("D"), 2L)); + write.write(GenericRow.of(1, BinaryString.fromString("D"), 4L)); result.addAll(write.prepareCommit(true, 0)); commit.commit(0, result); result.clear(); @@ -639,6 +645,7 @@ public void testBSIAndBitmapIndexInDisk() throws Exception { createUnawareBucketFileStoreTable( rowType, options -> { + options.set(METADATA_STATS_MODE, "NONE"); options.set( FileIndexOptions.FILE_INDEX + "." @@ -665,7 +672,11 @@ public void testBSIAndBitmapIndexInDisk() throws Exception { write.write(GenericRow.of(1, BinaryString.fromString("B"), 3L)); write.write(GenericRow.of(1, BinaryString.fromString("C"), 3L)); result.addAll(write.prepareCommit(true, 0)); + write.write(GenericRow.of(1, BinaryString.fromString("A"), 4L)); + write.write(GenericRow.of(1, BinaryString.fromString("B"), 3L)); write.write(GenericRow.of(1, BinaryString.fromString("C"), 4L)); + write.write(GenericRow.of(1, BinaryString.fromString("D"), 2L)); + write.write(GenericRow.of(1, BinaryString.fromString("D"), 4L)); result.addAll(write.prepareCommit(true, 0)); commit.commit(0, result); result.clear(); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java index dca86aa61ec28..4917d076014a9 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java @@ -99,6 +99,7 @@ import static org.apache.paimon.CoreOptions.ChangelogProducer.LOOKUP; import static org.apache.paimon.CoreOptions.DELETION_VECTORS_ENABLED; import static org.apache.paimon.CoreOptions.FILE_FORMAT; +import static org.apache.paimon.CoreOptions.FILE_INDEX_IN_MANIFEST_THRESHOLD; import static org.apache.paimon.CoreOptions.LOOKUP_LOCAL_FILE_TYPE; import static org.apache.paimon.CoreOptions.MERGE_ENGINE; import static org.apache.paimon.CoreOptions.MergeEngine; @@ -862,6 +863,92 @@ public void testDeletionVectorsWithFileIndexInMeta() throws Exception { assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(1); } + @Test + public void testDeletionVectorsWithBitmapFileIndexInFile() throws Exception { + FileStoreTable table = + createFileStoreTable( + conf -> { + conf.set(BUCKET, 1); + conf.set(DELETION_VECTORS_ENABLED, true); + conf.set(TARGET_FILE_SIZE, MemorySize.ofBytes(1)); + conf.set(FILE_INDEX_IN_MANIFEST_THRESHOLD, MemorySize.ofBytes(1)); + conf.set("file-index.bitmap.columns", "b"); + }); + + StreamTableWrite write = + table.newWrite(commitUser).withIOManager(new IOManagerImpl(tempDir.toString())); + StreamTableCommit commit = table.newCommit(commitUser); + + write.write(rowData(1, 1, 300L)); + write.write(rowData(1, 2, 400L)); + write.write(rowData(1, 3, 100L)); + write.write(rowData(1, 4, 100L)); + commit.commit(0, write.prepareCommit(true, 0)); + + write.write(rowData(1, 1, 100L)); + write.write(rowData(1, 2, 100L)); + write.write(rowData(1, 3, 300L)); + write.write(rowData(1, 5, 100L)); + commit.commit(1, write.prepareCommit(true, 1)); + + write.write(rowData(1, 4, 200L)); + commit.commit(2, write.prepareCommit(true, 2)); + + PredicateBuilder builder = new PredicateBuilder(ROW_TYPE); + List splits = toSplits(table.newSnapshotReader().read().dataSplits()); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(2); + TableRead read = table.newRead().withFilter(builder.equal(2, 100L)); + assertThat(getResult(read, splits, BATCH_ROW_TO_STRING)) + .hasSameElementsAs( + Arrays.asList( + "1|1|100|binary|varbinary|mapKey:mapVal|multiset", + "1|2|100|binary|varbinary|mapKey:mapVal|multiset", + "1|5|100|binary|varbinary|mapKey:mapVal|multiset")); + } + + @Test + public void testDeletionVectorsWithBitmapFileIndexInMeta() throws Exception { + FileStoreTable table = + createFileStoreTable( + conf -> { + conf.set(BUCKET, 1); + conf.set(DELETION_VECTORS_ENABLED, true); + conf.set(TARGET_FILE_SIZE, MemorySize.ofBytes(1)); + conf.set(FILE_INDEX_IN_MANIFEST_THRESHOLD, MemorySize.ofMebiBytes(1)); + conf.set("file-index.bitmap.columns", "b"); + }); + + StreamTableWrite write = + table.newWrite(commitUser).withIOManager(new IOManagerImpl(tempDir.toString())); + StreamTableCommit commit = table.newCommit(commitUser); + + write.write(rowData(1, 1, 300L)); + write.write(rowData(1, 2, 400L)); + write.write(rowData(1, 3, 100L)); + write.write(rowData(1, 4, 100L)); + commit.commit(0, write.prepareCommit(true, 0)); + + write.write(rowData(1, 1, 100L)); + write.write(rowData(1, 2, 100L)); + write.write(rowData(1, 3, 300L)); + write.write(rowData(1, 5, 100L)); + commit.commit(1, write.prepareCommit(true, 1)); + + write.write(rowData(1, 4, 200L)); + commit.commit(2, write.prepareCommit(true, 2)); + + PredicateBuilder builder = new PredicateBuilder(ROW_TYPE); + List splits = toSplits(table.newSnapshotReader().read().dataSplits()); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(2); + TableRead read = table.newRead().withFilter(builder.equal(2, 100L)); + assertThat(getResult(read, splits, BATCH_ROW_TO_STRING)) + .hasSameElementsAs( + Arrays.asList( + "1|1|100|binary|varbinary|mapKey:mapVal|multiset", + "1|2|100|binary|varbinary|mapKey:mapVal|multiset", + "1|5|100|binary|varbinary|mapKey:mapVal|multiset")); + } + @Test public void testWithShardFirstRow() throws Exception { FileStoreTable table = From d16287eb16f45f0de0eba243cf9a5517d54d88a9 Mon Sep 17 00:00:00 2001 From: wangkang Date: Wed, 13 Nov 2024 15:42:20 +0800 Subject: [PATCH 009/157] [doc] Update hll_sketch and theta_sketch in aggregation.md (#4518) --- .../primary-key-table/merge-engine/aggregation.md | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/docs/content/primary-key-table/merge-engine/aggregation.md b/docs/content/primary-key-table/merge-engine/aggregation.md index fa667ed356871..0cc6507f2b4ce 100644 --- a/docs/content/primary-key-table/merge-engine/aggregation.md +++ b/docs/content/primary-key-table/merge-engine/aggregation.md @@ -247,16 +247,17 @@ An example: uv VARBINARY ) WITH ( 'merge-engine' = 'aggregation', - 'fields.f0.aggregate-function' = 'hll_sketch' + 'fields.uv.aggregate-function' = 'hll_sketch' ); -- Register the following class as a Flink function with the name "HLL_SKETCH" + -- for example: create TEMPORARY function HLL_SKETCH as 'HllSketchFunction'; -- which is used to transform input to sketch bytes array: -- -- public static class HllSketchFunction extends ScalarFunction { -- public byte[] eval(String user_id) { -- HllSketch hllSketch = new HllSketch(); - -- hllSketch.update(id); + -- hllSketch.update(user_id); -- return hllSketch.toCompactByteArray(); -- } -- } @@ -264,6 +265,7 @@ An example: INSERT INTO UV_AGG SELECT id, HLL_SKETCH(user_id) FROM VISITS; -- Register the following class as a Flink function with the name "HLL_SKETCH_COUNT" + -- for example: create TEMPORARY function HLL_SKETCH_COUNT as 'HllSketchCountFunction'; -- which is used to get cardinality from sketch bytes array: -- -- public static class HllSketchCountFunction extends ScalarFunction { @@ -307,10 +309,11 @@ An example: uv VARBINARY ) WITH ( 'merge-engine' = 'aggregation', - 'fields.f0.aggregate-function' = 'theta_sketch' + 'fields.uv.aggregate-function' = 'theta_sketch' ); -- Register the following class as a Flink function with the name "THETA_SKETCH" + -- for example: create TEMPORARY function THETA_SKETCH as 'ThetaSketchFunction'; -- which is used to transform input to sketch bytes array: -- -- public static class ThetaSketchFunction extends ScalarFunction { @@ -324,6 +327,7 @@ An example: INSERT INTO UV_AGG SELECT id, THETA_SKETCH(user_id) FROM VISITS; -- Register the following class as a Flink function with the name "THETA_SKETCH_COUNT" + -- for example: create TEMPORARY function THETA_SKETCH_COUNT as 'ThetaSketchCountFunction'; -- which is used to get cardinality from sketch bytes array: -- -- public static class ThetaSketchCountFunction extends ScalarFunction { From bbba0171423983714c106e4e62a050f757ecdc12 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 13 Nov 2024 15:43:16 +0800 Subject: [PATCH 010/157] [core] Make FormatReaderFactory return FileRecordReader to reduce cast (#4512) --- .../bitmap/ApplyBitmapIndexRecordReader.java | 20 +++----- .../paimon/format/FormatReaderFactory.java | 3 +- .../paimon/reader/EmptyFileRecordReader.java | 36 ++++++++++++++ .../paimon/reader/FileRecordIterator.java | 4 +- .../paimon/reader/FileRecordReader.java | 31 ++++++++++++ .../ApplyDeletionVectorReader.java | 20 +++----- ...dReader.java => DataFileRecordReader.java} | 47 ++++++++++--------- .../io/KeyValueDataFileRecordReader.java | 12 +++-- .../paimon/io/KeyValueFileReaderFactory.java | 7 +-- .../paimon/operation/RawFileSplitRead.java | 13 ++--- ...CompactedChangelogFormatReaderFactory.java | 4 +- .../paimon/format/avro/AvroBulkFormat.java | 8 ++-- .../paimon/format/orc/OrcReaderFactory.java | 8 ++-- .../format/parquet/ParquetReaderFactory.java | 9 ++-- 14 files changed, 140 insertions(+), 82 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/reader/EmptyFileRecordReader.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/reader/FileRecordReader.java rename paimon-core/src/main/java/org/apache/paimon/io/{FileRecordReader.java => DataFileRecordReader.java} (88%) diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexRecordReader.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexRecordReader.java index d5d15095f2eda..3b1207c8bd6e5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexRecordReader.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/bitmap/ApplyBitmapIndexRecordReader.java @@ -20,41 +20,35 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.reader.FileRecordIterator; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader; import javax.annotation.Nullable; import java.io.IOException; -import static org.apache.paimon.utils.Preconditions.checkArgument; - /** A {@link RecordReader} which apply {@link BitmapIndexResult} to filter record. */ -public class ApplyBitmapIndexRecordReader implements RecordReader { +public class ApplyBitmapIndexRecordReader implements FileRecordReader { - private final RecordReader reader; + private final FileRecordReader reader; private final BitmapIndexResult fileIndexResult; public ApplyBitmapIndexRecordReader( - RecordReader reader, BitmapIndexResult fileIndexResult) { + FileRecordReader reader, BitmapIndexResult fileIndexResult) { this.reader = reader; this.fileIndexResult = fileIndexResult; } @Nullable @Override - public RecordIterator readBatch() throws IOException { - RecordIterator batch = reader.readBatch(); + public FileRecordIterator readBatch() throws IOException { + FileRecordIterator batch = reader.readBatch(); if (batch == null) { return null; } - checkArgument( - batch instanceof FileRecordIterator, - "There is a bug, RecordIterator in ApplyBitmapIndexRecordReader must be FileRecordIterator"); - - return new ApplyBitmapIndexFileRecordIterator( - (FileRecordIterator) batch, fileIndexResult); + return new ApplyBitmapIndexFileRecordIterator(batch, fileIndexResult); } @Override diff --git a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java index 420d44e0f61d9..d8af3e2fe37ca 100644 --- a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java +++ b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java @@ -22,6 +22,7 @@ import org.apache.paimon.fileindex.FileIndexResult; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader; import java.io.IOException; @@ -29,7 +30,7 @@ /** A factory to create {@link RecordReader} for file. */ public interface FormatReaderFactory { - RecordReader createReader(Context context) throws IOException; + FileRecordReader createReader(Context context) throws IOException; /** Context for creating reader. */ interface Context { diff --git a/paimon-common/src/main/java/org/apache/paimon/reader/EmptyFileRecordReader.java b/paimon-common/src/main/java/org/apache/paimon/reader/EmptyFileRecordReader.java new file mode 100644 index 0000000000000..3fa25dce5c490 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/reader/EmptyFileRecordReader.java @@ -0,0 +1,36 @@ +/* + * 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.paimon.reader; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** An empty {@link FileRecordReader}. */ +public class EmptyFileRecordReader implements FileRecordReader { + + @Nullable + @Override + public FileRecordIterator readBatch() throws IOException { + return null; + } + + @Override + public void close() throws IOException {} +} diff --git a/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java b/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java index d22b27053f986..2d3c85f193dcf 100644 --- a/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java +++ b/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordIterator.java @@ -27,10 +27,8 @@ import java.util.function.Function; /** - * Wrap {@link RecordReader.RecordIterator} to support returning the record's row position and file + * A {@link RecordReader.RecordIterator} to support returning the record's row position and file * Path. - * - * @param The type of the record. */ public interface FileRecordIterator extends RecordReader.RecordIterator { diff --git a/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordReader.java b/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordReader.java new file mode 100644 index 0000000000000..4d5356edf2757 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/reader/FileRecordReader.java @@ -0,0 +1,31 @@ +/* + * 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.paimon.reader; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** A {@link RecordReader} to support returning {@link FileRecordIterator}. */ +public interface FileRecordReader extends RecordReader { + + @Override + @Nullable + FileRecordIterator readBatch() throws IOException; +} diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java index c1dc16a78d950..2fc292e54d340 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java @@ -20,23 +20,22 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.reader.FileRecordIterator; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader; import javax.annotation.Nullable; import java.io.IOException; -import static org.apache.paimon.utils.Preconditions.checkArgument; - /** A {@link RecordReader} which apply {@link DeletionVector} to filter record. */ -public class ApplyDeletionVectorReader implements RecordReader { +public class ApplyDeletionVectorReader implements FileRecordReader { - private final RecordReader reader; + private final FileRecordReader reader; private final DeletionVector deletionVector; public ApplyDeletionVectorReader( - RecordReader reader, DeletionVector deletionVector) { + FileRecordReader reader, DeletionVector deletionVector) { this.reader = reader; this.deletionVector = deletionVector; } @@ -51,19 +50,14 @@ public DeletionVector deletionVector() { @Nullable @Override - public RecordIterator readBatch() throws IOException { - RecordIterator batch = reader.readBatch(); + public FileRecordIterator readBatch() throws IOException { + FileRecordIterator batch = reader.readBatch(); if (batch == null) { return null; } - checkArgument( - batch instanceof FileRecordIterator, - "There is a bug, RecordIterator in ApplyDeletionVectorReader must be FileRecordIterator"); - - return new ApplyDeletionFileRecordIterator( - (FileRecordIterator) batch, deletionVector); + return new ApplyDeletionFileRecordIterator(batch, deletionVector); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/io/FileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java similarity index 88% rename from paimon-core/src/main/java/org/apache/paimon/io/FileRecordReader.java rename to paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java index 1e12025ba5330..d2559fe6240b2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/FileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java @@ -25,7 +25,8 @@ import org.apache.paimon.data.PartitionInfo; import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.format.FormatReaderFactory; -import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.reader.FileRecordIterator; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.utils.FileUtils; import org.apache.paimon.utils.ProjectedRow; @@ -34,17 +35,35 @@ import java.io.IOException; /** Reads {@link InternalRow} from data files. */ -public class FileRecordReader implements RecordReader { +public class DataFileRecordReader implements FileRecordReader { - private final RecordReader reader; + private final FileRecordReader reader; @Nullable private final int[] indexMapping; @Nullable private final PartitionInfo partitionInfo; @Nullable private final CastFieldGetter[] castMapping; + public DataFileRecordReader( + FormatReaderFactory readerFactory, + FormatReaderFactory.Context context, + @Nullable int[] indexMapping, + @Nullable CastFieldGetter[] castMapping, + @Nullable PartitionInfo partitionInfo) + throws IOException { + try { + this.reader = readerFactory.createReader(context); + } catch (Exception e) { + FileUtils.checkExists(context.fileIO(), context.filePath()); + throw e; + } + this.indexMapping = indexMapping; + this.partitionInfo = partitionInfo; + this.castMapping = castMapping; + } + @Nullable @Override - public RecordReader.RecordIterator readBatch() throws IOException { - RecordIterator iterator = reader.readBatch(); + public FileRecordIterator readBatch() throws IOException { + FileRecordIterator iterator = reader.readBatch(); if (iterator == null) { return null; } @@ -71,24 +90,6 @@ public RecordReader.RecordIterator readBatch() throws IOException { return iterator; } - public FileRecordReader( - FormatReaderFactory readerFactory, - FormatReaderFactory.Context context, - @Nullable int[] indexMapping, - @Nullable CastFieldGetter[] castMapping, - @Nullable PartitionInfo partitionInfo) - throws IOException { - try { - this.reader = readerFactory.createReader(context); - } catch (Exception e) { - FileUtils.checkExists(context.fileIO(), context.filePath()); - throw e; - } - this.indexMapping = indexMapping; - this.partitionInfo = partitionInfo; - this.castMapping = castMapping; - } - @Override public void close() throws IOException { reader.close(); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java index e44ad79ff53ee..6cf08769703fc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java @@ -21,6 +21,8 @@ import org.apache.paimon.KeyValue; import org.apache.paimon.KeyValueSerializer; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.reader.FileRecordIterator; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.types.RowType; @@ -29,14 +31,14 @@ import java.io.IOException; /** {@link RecordReader} for reading {@link KeyValue} data files. */ -public class KeyValueDataFileRecordReader implements RecordReader { +public class KeyValueDataFileRecordReader implements FileRecordReader { - private final RecordReader reader; + private final FileRecordReader reader; private final KeyValueSerializer serializer; private final int level; public KeyValueDataFileRecordReader( - RecordReader reader, RowType keyType, RowType valueType, int level) { + FileRecordReader reader, RowType keyType, RowType valueType, int level) { this.reader = reader; this.serializer = new KeyValueSerializer(keyType, valueType); this.level = level; @@ -44,8 +46,8 @@ public KeyValueDataFileRecordReader( @Nullable @Override - public RecordIterator readBatch() throws IOException { - RecordReader.RecordIterator iterator = reader.readBatch(); + public FileRecordIterator readBatch() throws IOException { + FileRecordIterator iterator = reader.readBatch(); if (iterator == null) { return null; } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java index fdbb727e56747..7d3acd729c55a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java @@ -32,6 +32,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.partition.PartitionUtils; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.KeyValueFieldsExtractor; import org.apache.paimon.schema.SchemaManager; @@ -109,7 +110,7 @@ public RecordReader createRecordReader( return createRecordReader(schemaId, fileName, level, true, null, fileSize); } - private RecordReader createRecordReader( + private FileRecordReader createRecordReader( long schemaId, String fileName, int level, @@ -134,8 +135,8 @@ private RecordReader createRecordReader( : formatSupplier.get(); Path filePath = pathFactory.toPath(fileName); - RecordReader fileRecordReader = - new FileRecordReader( + FileRecordReader fileRecordReader = + new DataFileRecordReader( bulkFormatMapping.getReaderFactory(), orcPoolSize == null ? new FormatReaderContext(fileIO, filePath, fileSize) diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java index 4a6fa5b3db7f2..46977457c4be5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java @@ -32,12 +32,13 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFilePathFactory; +import org.apache.paimon.io.DataFileRecordReader; import org.apache.paimon.io.FileIndexEvaluator; -import org.apache.paimon.io.FileRecordReader; import org.apache.paimon.mergetree.compact.ConcatRecordReader; import org.apache.paimon.partition.PartitionUtils; import org.apache.paimon.predicate.Predicate; -import org.apache.paimon.reader.EmptyRecordReader; +import org.apache.paimon.reader.EmptyFileRecordReader; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.ReaderSupplier; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.SchemaManager; @@ -187,7 +188,7 @@ public RecordReader createReader( return ConcatRecordReader.create(suppliers); } - private RecordReader createFileReader( + private FileRecordReader createFileReader( BinaryRow partition, DataFileMeta file, DataFilePathFactory dataFilePathFactory, @@ -204,7 +205,7 @@ private RecordReader createFileReader( dataFilePathFactory, file); if (!fileIndexResult.remain()) { - return new EmptyRecordReader<>(); + return new EmptyFileRecordReader<>(); } } @@ -214,8 +215,8 @@ private RecordReader createFileReader( dataFilePathFactory.toPath(file.fileName()), file.fileSize(), fileIndexResult); - RecordReader fileRecordReader = - new FileRecordReader( + FileRecordReader fileRecordReader = + new DataFileRecordReader( bulkFormatMapping.getReaderFactory(), formatReaderContext, bulkFormatMapping.getIndexMapping(), diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogFormatReaderFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogFormatReaderFactory.java index e17566f302cdc..e0aed448db93c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogFormatReaderFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogFormatReaderFactory.java @@ -27,7 +27,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.fs.PositionOutputStream; import org.apache.paimon.fs.SeekableInputStream; -import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.reader.FileRecordReader; import java.io.EOFException; import java.io.IOException; @@ -60,7 +60,7 @@ public CompactedChangelogFormatReaderFactory(FormatReaderFactory wrapped) { } @Override - public RecordReader createReader(Context context) throws IOException { + public FileRecordReader createReader(Context context) throws IOException { OffsetReadOnlyFileIO fileIO = new OffsetReadOnlyFileIO(context.fileIO()); long length = decodePath(context.filePath()).length; diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java index 7f3e275183cf3..a06ca9948c444 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java @@ -22,7 +22,7 @@ import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; -import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.IOUtils; import org.apache.paimon.utils.IteratorResultIterator; @@ -49,12 +49,12 @@ public AvroBulkFormat(RowType projectedRowType) { } @Override - public RecordReader createReader(FormatReaderFactory.Context context) + public FileRecordReader createReader(FormatReaderFactory.Context context) throws IOException { return new AvroReader(context.fileIO(), context.filePath(), context.fileSize()); } - private class AvroReader implements RecordReader { + private class AvroReader implements FileRecordReader { private final FileIO fileIO; private final DataFileReader reader; @@ -90,7 +90,7 @@ private DataFileReader createReaderFromPath(Path path, long fileSiz @Nullable @Override - public RecordIterator readBatch() throws IOException { + public IteratorResultIterator readBatch() throws IOException { Object ticket; try { ticket = pool.pollEntry(); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java index dbc5de265c492..05f3dd7851e84 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java @@ -30,6 +30,7 @@ import org.apache.paimon.format.orc.filter.OrcFilters; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader.RecordIterator; import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowType; @@ -184,7 +185,7 @@ public VectorizedRowBatch orcVectorizedRowBatch() { return orcVectorizedRowBatch; } - private RecordIterator convertAndGetIterator( + private ColumnarRowIterator convertAndGetIterator( VectorizedRowBatch orcBatch, long rowNumber) { // no copying from the ORC column vectors to the Paimon columns vectors necessary, // because they point to the same data arrays internally design @@ -209,8 +210,7 @@ private RecordIterator convertAndGetIterator( * batch is addressed by the starting row number of the batch, plus the number of records to be * skipped before. */ - private static final class OrcVectorizedReader - implements org.apache.paimon.reader.RecordReader { + private static final class OrcVectorizedReader implements FileRecordReader { private final RecordReader orcReader; private final Pool pool; @@ -222,7 +222,7 @@ private OrcVectorizedReader(final RecordReader orcReader, final Pool readBatch() throws IOException { + public ColumnarRowIterator readBatch() throws IOException { final OrcReaderBatch batch = getCachedEntry(); final VectorizedRowBatch orcVectorBatch = batch.orcVectorizedRowBatch(); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java index 2a62c0bc89471..2e792d153c1b1 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java @@ -31,8 +31,7 @@ import org.apache.paimon.format.parquet.type.ParquetField; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; -import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.reader.RecordReader.RecordIterator; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; @@ -307,7 +306,7 @@ private VectorizedColumnBatch createVectorizedColumnBatch( return new VectorizedColumnBatch(vectors); } - private class ParquetReader implements RecordReader { + private class ParquetReader implements FileRecordReader { private ParquetFileReader reader; @@ -360,7 +359,7 @@ private ParquetReader( @Nullable @Override - public RecordIterator readBatch() throws IOException { + public ColumnarRowIterator readBatch() throws IOException { final ParquetReaderBatch batch = getCachedEntry(); if (!nextBatch(batch)) { @@ -488,7 +487,7 @@ public void recycle() { recycler.recycle(this); } - public RecordIterator convertAndGetIterator(long rowNumber) { + public ColumnarRowIterator convertAndGetIterator(long rowNumber) { result.reset(rowNumber); return result; } From e0ae6c8f5634056bdee8dbc7309d269ac175d381 Mon Sep 17 00:00:00 2001 From: tsreaper Date: Wed, 13 Nov 2024 15:49:52 +0800 Subject: [PATCH 011/157] [parquet] Fix nested array/map has no id in parquet files (#4513) --- .../apache/paimon/table/SpecialFields.java | 34 +++- .../format/parquet/ParquetReaderFactory.java | 33 ++-- .../parquet/ParquetSchemaConverter.java | 165 ++++++++---------- .../format/parquet/ParquetReadWriteTest.java | 29 ++- 4 files changed, 142 insertions(+), 119 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java b/paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java index 36f22a3107556..d438bfb0ffe9e 100644 --- a/paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java +++ b/paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java @@ -48,9 +48,18 @@ * directly by id. These ids are not stored in {@link org.apache.paimon.types.DataField}. * *
    - *
  • Array element field: ID = 536870911 + (array-field-id). - *
  • Map key field: ID = 536870911 + (array-field-id). - *
  • Map value field: ID = 536870911 - (array-field-id). + *
  • Array element field: ID = 536870911 + 1024 * (array-field-id) + depth. + *
  • Map key field: ID = 536870911 - 1024 * (array-field-id) - depth. + *
  • Map value field: ID = 536870911 + 1024 * (array-field-id) + depth. + *
+ * + *

Examples: + * + *

    + *
  • ARRAY(MAP(INT, ARRAY(INT))) type, outer array has field id 10, then map (element of outer + * array) has field id 536870911 + 1024 * 10 + 1, map key (int) has field id 536870911 - 1024 + * * 10 - 2, map value (inner array) has field id 536870911 + 1024 * 10 + 2, inner array + * element (int) has field id 536870911 + 1024 * 10 + 3 *
*/ public class SpecialFields { @@ -95,16 +104,23 @@ public static boolean isSystemField(String field) { // ---------------------------------------------------------------------------------------- public static final int STRUCTURED_TYPE_FIELD_ID_BASE = Integer.MAX_VALUE / 4; + public static final int STRUCTURED_TYPE_FIELD_DEPTH_LIMIT = 1 << 10; - public static int getArrayElementFieldId(int arrayFieldId) { - return STRUCTURED_TYPE_FIELD_ID_BASE + arrayFieldId; + public static int getArrayElementFieldId(int arrayFieldId, int depth) { + return STRUCTURED_TYPE_FIELD_ID_BASE + + arrayFieldId * STRUCTURED_TYPE_FIELD_DEPTH_LIMIT + + depth; } - public static int getMapKeyFieldId(int mapFieldId) { - return STRUCTURED_TYPE_FIELD_ID_BASE + mapFieldId; + public static int getMapKeyFieldId(int mapFieldId, int depth) { + return STRUCTURED_TYPE_FIELD_ID_BASE + - mapFieldId * STRUCTURED_TYPE_FIELD_DEPTH_LIMIT + - depth; } - public static int getMapValueFieldId(int mapFieldId) { - return STRUCTURED_TYPE_FIELD_ID_BASE - mapFieldId; + public static int getMapValueFieldId(int mapFieldId, int depth) { + return STRUCTURED_TYPE_FIELD_ID_BASE + + mapFieldId * STRUCTURED_TYPE_FIELD_DEPTH_LIMIT + + depth; } } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java index 2e792d153c1b1..cede205f4d513 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java @@ -87,8 +87,8 @@ public class ParquetReaderFactory implements FormatReaderFactory { private final Options conf; private final RowType projectedType; - private final String[] projectedFields; - private final DataType[] projectedTypes; + private final String[] projectedColumnNames; + private final DataField[] projectedFields; private final int batchSize; private final FilterCompat.Filter filter; private final Set unknownFieldsIndices = new HashSet<>(); @@ -97,14 +97,15 @@ public ParquetReaderFactory( Options conf, RowType projectedType, int batchSize, FilterCompat.Filter filter) { this.conf = conf; this.projectedType = projectedType; - this.projectedFields = projectedType.getFieldNames().toArray(new String[0]); - this.projectedTypes = projectedType.getFieldTypes().toArray(new DataType[0]); + this.projectedColumnNames = projectedType.getFieldNames().toArray(new String[0]); + this.projectedFields = projectedType.getFields().toArray(new DataField[0]); this.batchSize = batchSize; this.filter = filter; } @Override - public ParquetReader createReader(FormatReaderFactory.Context context) throws IOException { + public RecordReader createReader(FormatReaderFactory.Context context) + throws IOException { ParquetReadOptions.Builder builder = ParquetReadOptions.builder().withRange(0, context.fileSize()); setReadOptions(builder); @@ -153,20 +154,20 @@ private void setReadOptions(ParquetReadOptions.Builder builder) { /** Clips `parquetSchema` according to `fieldNames`. */ private MessageType clipParquetSchema(GroupType parquetSchema) { - Type[] types = new Type[projectedFields.length]; - for (int i = 0; i < projectedFields.length; ++i) { - String fieldName = projectedFields[i]; + Type[] types = new Type[projectedColumnNames.length]; + for (int i = 0; i < projectedColumnNames.length; ++i) { + String fieldName = projectedColumnNames[i]; if (!parquetSchema.containsField(fieldName)) { LOG.warn( "{} does not exist in {}, will fill the field with null.", fieldName, parquetSchema); types[i] = - ParquetSchemaConverter.convertToParquetType(fieldName, projectedTypes[i]); + ParquetSchemaConverter.convertToParquetType(fieldName, projectedFields[i]); unknownFieldsIndices.add(i); } else { Type parquetType = parquetSchema.getType(fieldName); - types[i] = clipParquetType(projectedTypes[i], parquetType); + types[i] = clipParquetType(projectedFields[i].type(), parquetType); } } @@ -220,7 +221,7 @@ private Type clipParquetType(DataType readType, Type parquetType) { private void checkSchema(MessageType fileSchema, MessageType requestedSchema) throws IOException, UnsupportedOperationException { - if (projectedFields.length != requestedSchema.getFieldCount()) { + if (projectedColumnNames.length != requestedSchema.getFieldCount()) { throw new RuntimeException( "The quality of field type is incompatible with the request schema!"); } @@ -268,13 +269,13 @@ private ParquetReaderBatch createReaderBatch( } private WritableColumnVector[] createWritableVectors(MessageType requestedSchema) { - WritableColumnVector[] columns = new WritableColumnVector[projectedTypes.length]; + WritableColumnVector[] columns = new WritableColumnVector[projectedFields.length]; List types = requestedSchema.getFields(); - for (int i = 0; i < projectedTypes.length; i++) { + for (int i = 0; i < projectedFields.length; i++) { columns[i] = createWritableColumnVector( batchSize, - projectedTypes[i], + projectedFields[i].type(), types.get(i), requestedSchema.getColumns(), 0); @@ -290,7 +291,7 @@ private VectorizedColumnBatch createVectorizedColumnBatch( WritableColumnVector[] writableVectors) { ColumnVector[] vectors = new ColumnVector[writableVectors.length]; for (int i = 0; i < writableVectors.length; i++) { - switch (projectedTypes[i].getTypeRoot()) { + switch (projectedFields[i].type().getTypeRoot()) { case DECIMAL: vectors[i] = new ParquetDecimalVector(writableVectors[i]); break; @@ -416,7 +417,7 @@ private void readNextRowGroup() throws IOException { if (!unknownFieldsIndices.contains(i)) { columnReaders[i] = createColumnReader( - projectedTypes[i], + projectedFields[i].type(), types.get(i), requestedSchema.getColumns(), rowGroup, diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java index 5cf81c375a37d..708e5eb7ea3d4 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java @@ -22,7 +22,6 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypeRoot; import org.apache.paimon.types.DecimalType; import org.apache.paimon.types.IntType; import org.apache.paimon.types.LocalZonedTimestampType; @@ -39,9 +38,6 @@ import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types; -import java.util.ArrayList; -import java.util.List; - import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; @@ -59,86 +55,110 @@ public static MessageType convertToParquetMessageType(String name, RowType rowTy return new MessageType(name, convertToParquetTypes(rowType)); } - public static Type convertToParquetType(String name, DataType type) { - Type.Repetition repetition = - type.isNullable() ? Type.Repetition.OPTIONAL : Type.Repetition.REQUIRED; - return convertToParquetType(name, type, repetition); + public static Type convertToParquetType(String name, DataField field) { + return convertToParquetType(name, field.type(), field.id(), 0); + } + + private static Type[] convertToParquetTypes(RowType rowType) { + return rowType.getFields().stream() + .map(f -> convertToParquetType(f.name(), f.type(), f.id(), 0)) + .toArray(Type[]::new); } - private static Type convertToParquetType( - String name, DataType type, Type.Repetition repetition) { + private static Type convertToParquetType(String name, DataType type, int fieldId, int depth) { + Type.Repetition repetition = + type.isNullable() ? Type.Repetition.OPTIONAL : Type.Repetition.REQUIRED; switch (type.getTypeRoot()) { case CHAR: case VARCHAR: return Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition) .as(LogicalTypeAnnotation.stringType()) - .named(name); + .named(name) + .withId(fieldId); case BOOLEAN: return Types.primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, repetition) - .named(name); + .named(name) + .withId(fieldId); case BINARY: case VARBINARY: return Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition) - .named(name); + .named(name) + .withId(fieldId); case DECIMAL: int precision = ((DecimalType) type).getPrecision(); int scale = ((DecimalType) type).getScale(); if (is32BitDecimal(precision)) { return Types.primitive(INT32, repetition) .as(LogicalTypeAnnotation.decimalType(scale, precision)) - .named(name); + .named(name) + .withId(fieldId); } else if (is64BitDecimal(precision)) { return Types.primitive(INT64, repetition) .as(LogicalTypeAnnotation.decimalType(scale, precision)) - .named(name); + .named(name) + .withId(fieldId); } else { return Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition) .as(LogicalTypeAnnotation.decimalType(scale, precision)) .length(computeMinBytesForDecimalPrecision(precision)) - .named(name); + .named(name) + .withId(fieldId); } case TINYINT: return Types.primitive(INT32, repetition) .as(LogicalTypeAnnotation.intType(8, true)) - .named(name); + .named(name) + .withId(fieldId); case SMALLINT: return Types.primitive(INT32, repetition) .as(LogicalTypeAnnotation.intType(16, true)) - .named(name); + .named(name) + .withId(fieldId); case INTEGER: - return Types.primitive(INT32, repetition).named(name); + return Types.primitive(INT32, repetition).named(name).withId(fieldId); case BIGINT: - return Types.primitive(INT64, repetition).named(name); + return Types.primitive(INT64, repetition).named(name).withId(fieldId); case FLOAT: return Types.primitive(PrimitiveType.PrimitiveTypeName.FLOAT, repetition) - .named(name); + .named(name) + .withId(fieldId); case DOUBLE: return Types.primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, repetition) - .named(name); + .named(name) + .withId(fieldId); case DATE: return Types.primitive(INT32, repetition) .as(LogicalTypeAnnotation.dateType()) - .named(name); + .named(name) + .withId(fieldId); case TIME_WITHOUT_TIME_ZONE: return Types.primitive(INT32, repetition) .as( LogicalTypeAnnotation.timeType( true, LogicalTypeAnnotation.TimeUnit.MILLIS)) - .named(name); + .named(name) + .withId(fieldId); case TIMESTAMP_WITHOUT_TIME_ZONE: TimestampType timestampType = (TimestampType) type; return createTimestampWithLogicalType( - name, timestampType.getPrecision(), repetition, false); + name, timestampType.getPrecision(), repetition, false) + .withId(fieldId); case TIMESTAMP_WITH_LOCAL_TIME_ZONE: LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) type; return createTimestampWithLogicalType( - name, localZonedTimestampType.getPrecision(), repetition, true); + name, localZonedTimestampType.getPrecision(), repetition, true) + .withId(fieldId); case ARRAY: ArrayType arrayType = (ArrayType) type; - return ConversionPatterns.listOfElements( - repetition, - name, - convertToParquetType(LIST_ELEMENT_NAME, arrayType.getElementType())); + Type elementParquetType = + convertToParquetType( + LIST_ELEMENT_NAME, + arrayType.getElementType(), + fieldId, + depth + 1) + .withId(SpecialFields.getArrayElementFieldId(fieldId, depth + 1)); + return ConversionPatterns.listOfElements(repetition, name, elementParquetType) + .withId(fieldId); case MAP: MapType mapType = (MapType) type; DataType keyType = mapType.getKeyType(); @@ -147,12 +167,20 @@ private static Type convertToParquetType( // it as not nullable keyType = keyType.copy(false); } + Type mapKeyParquetType = + convertToParquetType(MAP_KEY_NAME, keyType, fieldId, depth + 1) + .withId(SpecialFields.getMapKeyFieldId(fieldId, depth + 1)); + Type mapValueParquetType = + convertToParquetType( + MAP_VALUE_NAME, mapType.getValueType(), fieldId, depth + 1) + .withId(SpecialFields.getMapValueFieldId(fieldId, depth + 1)); return ConversionPatterns.mapType( - repetition, - name, - MAP_REPEATED_NAME, - convertToParquetType(MAP_KEY_NAME, keyType), - convertToParquetType(MAP_VALUE_NAME, mapType.getValueType())); + repetition, + name, + MAP_REPEATED_NAME, + mapKeyParquetType, + mapValueParquetType) + .withId(fieldId); case MULTISET: MultisetType multisetType = (MultisetType) type; DataType elementType = multisetType.getElementType(); @@ -161,15 +189,23 @@ private static Type convertToParquetType( // so we configure it as not nullable elementType = elementType.copy(false); } + Type multisetKeyParquetType = + convertToParquetType(MAP_KEY_NAME, elementType, fieldId, depth + 1) + .withId(SpecialFields.getMapKeyFieldId(fieldId, depth + 1)); + Type multisetValueParquetType = + convertToParquetType(MAP_VALUE_NAME, new IntType(false), fieldId, depth + 1) + .withId(SpecialFields.getMapValueFieldId(fieldId, depth + 1)); return ConversionPatterns.mapType( - repetition, - name, - MAP_REPEATED_NAME, - convertToParquetType(MAP_KEY_NAME, elementType), - convertToParquetType(MAP_VALUE_NAME, new IntType(false))); + repetition, + name, + MAP_REPEATED_NAME, + multisetKeyParquetType, + multisetValueParquetType) + .withId(fieldId); case ROW: RowType rowType = (RowType) type; - return new GroupType(repetition, name, convertToParquetTypes(rowType)); + return new GroupType(repetition, name, convertToParquetTypes(rowType)) + .withId(fieldId); default: throw new UnsupportedOperationException("Unsupported type: " + type); } @@ -194,53 +230,6 @@ private static Type createTimestampWithLogicalType( } } - private static List convertToParquetTypes(RowType rowType) { - List types = new ArrayList<>(rowType.getFieldCount()); - for (DataField field : rowType.getFields()) { - Type parquetType = convertToParquetType(field.name(), field.type()); - Type typeWithId = parquetType.withId(field.id()); - if (field.type().getTypeRoot() == DataTypeRoot.ARRAY) { - GroupType groupType = (GroupType) parquetType; - GroupType wrapperType = (GroupType) groupType.getFields().get(0); - Type elementTypeWithId = - wrapperType - .getFields() - .get(0) - .withId(SpecialFields.getArrayElementFieldId(field.id())); - typeWithId = - ConversionPatterns.listOfElements( - groupType.getRepetition(), - groupType.getName(), - elementTypeWithId) - .withId(field.id()); - } else if (field.type().getTypeRoot() == DataTypeRoot.MAP - || field.type().getTypeRoot() == DataTypeRoot.MULTISET) { - GroupType groupType = (GroupType) parquetType; - GroupType wrapperType = (GroupType) groupType.getFields().get(0); - Type keyTypeWithId = - wrapperType - .getFields() - .get(0) - .withId(SpecialFields.getMapKeyFieldId(field.id())); - Type valueTypeWithId = - wrapperType - .getFields() - .get(1) - .withId(SpecialFields.getMapValueFieldId(field.id())); - typeWithId = - ConversionPatterns.mapType( - groupType.getRepetition(), - groupType.getName(), - MAP_REPEATED_NAME, - keyTypeWithId, - valueTypeWithId) - .withId(field.id()); - } - types.add(typeWithId); - } - return types; - } - public static int computeMinBytesForDecimalPrecision(int precision) { int numBytes = 1; while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) { diff --git a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java index 099811d98b528..ffe4d6008296a 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java @@ -97,6 +97,7 @@ import java.util.stream.Collectors; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -489,11 +490,17 @@ public void testConvertToParquetTypeWithId() { new DataField(0, "a", DataTypes.INT()), new DataField(1, "b", DataTypes.ARRAY(DataTypes.STRING())), new DataField( - 2, "c", DataTypes.MAP(DataTypes.INT(), new RowType(nestedFields)))); + 2, + "c", + DataTypes.MAP( + DataTypes.INT(), + DataTypes.MAP( + DataTypes.BIGINT(), new RowType(nestedFields))))); RowType rowType = new RowType(fields); int baseId = 536870911; - Type mapValueType = + int depthLimit = 1 << 10; + Type innerMapValueType = new GroupType( Type.Repetition.OPTIONAL, "value", @@ -506,7 +513,17 @@ public void testConvertToParquetTypeWithId() { .as(LogicalTypeAnnotation.stringType()) .named("v2") .withId(4)) - .withId(baseId - 2); + .withId(baseId + depthLimit * 2 + 2); + Type outerMapValueType = + ConversionPatterns.mapType( + Type.Repetition.OPTIONAL, + "value", + "key_value", + Types.primitive(INT64, Type.Repetition.REQUIRED) + .named("key") + .withId(baseId - depthLimit * 2 - 2), + innerMapValueType) + .withId(baseId + depthLimit * 2 + 1); Type expected = new MessageType( "table", @@ -519,7 +536,7 @@ public void testConvertToParquetTypeWithId() { Type.Repetition.OPTIONAL) .as(LogicalTypeAnnotation.stringType()) .named("element") - .withId(baseId + 1)) + .withId(baseId + depthLimit + 1)) .withId(1), ConversionPatterns.mapType( Type.Repetition.OPTIONAL, @@ -527,8 +544,8 @@ public void testConvertToParquetTypeWithId() { "key_value", Types.primitive(INT32, Type.Repetition.REQUIRED) .named("key") - .withId(baseId + 2), - mapValueType) + .withId(baseId - depthLimit * 2 - 1), + outerMapValueType) .withId(2)); Type actual = ParquetSchemaConverter.convertToParquetMessageType("table", rowType); assertThat(actual).isEqualTo(expected); From 19119e3c073619bc5db9ce52c151527af919b8cc Mon Sep 17 00:00:00 2001 From: tsreaper Date: Wed, 13 Nov 2024 16:02:39 +0800 Subject: [PATCH 012/157] [hotfix] Fix compile error in ParquetReaderFactory --- .../org/apache/paimon/format/parquet/ParquetReaderFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java index cede205f4d513..53b4b1634b5fc 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java @@ -104,7 +104,7 @@ public ParquetReaderFactory( } @Override - public RecordReader createReader(FormatReaderFactory.Context context) + public FileRecordReader createReader(FormatReaderFactory.Context context) throws IOException { ParquetReadOptions.Builder builder = ParquetReadOptions.builder().withRange(0, context.fileSize()); From 7eaf30f776b9d1573d86a274ac53633d267262af Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Wed, 13 Nov 2024 17:02:55 +0800 Subject: [PATCH 013/157] [common] Fix overflow problem of NumericToBooleanCastRule (#4519) --- .../org/apache/paimon/casting/NumericToBooleanCastRule.java | 2 +- .../test/java/org/apache/paimon/flink/SchemaChangeITCase.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/NumericToBooleanCastRule.java b/paimon-common/src/main/java/org/apache/paimon/casting/NumericToBooleanCastRule.java index 06fa89fe35994..5b47741e6d351 100644 --- a/paimon-common/src/main/java/org/apache/paimon/casting/NumericToBooleanCastRule.java +++ b/paimon-common/src/main/java/org/apache/paimon/casting/NumericToBooleanCastRule.java @@ -37,6 +37,6 @@ private NumericToBooleanCastRule() { @Override public CastExecutor create(DataType inputType, DataType targetType) { - return value -> value.intValue() != 0; + return value -> value.longValue() != 0; } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java index ba161fe84008f..a2ef1d5c8acec 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java @@ -251,7 +251,8 @@ public void testModifyColumnTypeFromNumericToDecimal() { public void testModifyColumnTypeBooleanAndNumeric() { // boolean To numeric and numeric To boolean sql("CREATE TABLE T (a BOOLEAN, b BOOLEAN, c TINYINT, d INT, e BIGINT, f DOUBLE)"); - sql("INSERT INTO T VALUES(true, false, cast(0 as TINYINT), 1 , 123, 3.14)"); + sql( + "INSERT INTO T VALUES(true, false, cast(0 as TINYINT), 1 , -9223372036854775808, 3.14)"); sql("ALTER TABLE T MODIFY (a TINYINT, b INT, c BOOLEAN, d BOOLEAN, e BOOLEAN)"); List result = sql("SHOW CREATE TABLE T"); From 30bf5036d6117aedce7297dacec23155a7d5778c Mon Sep 17 00:00:00 2001 From: "liming.1018" Date: Wed, 13 Nov 2024 17:04:39 +0800 Subject: [PATCH 014/157] [flink] support multiple writers writing to the same partition when using kafka as logSystem in unaware bucket mode. (#4516) --- .../apache/paimon/manifest/ManifestCommittable.java | 7 ++++--- .../src/test/java/org/apache/paimon/TestFileStore.java | 3 ++- .../manifest/ManifestCommittableSerializerTest.java | 2 +- .../org/apache/paimon/flink/sink/StoreCommitter.java | 10 +++++++++- .../apache/paimon/flink/sink/StoreMultiCommitter.java | 10 ++++++---- .../sink/WrappedManifestCommittableSerializerTest.java | 2 +- 6 files changed, 23 insertions(+), 11 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java index 61c4619bd6d6a..b4abd0e9ec0ed 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java @@ -62,13 +62,14 @@ public void addFileCommittable(CommitMessage commitMessage) { commitMessages.add(commitMessage); } - public void addLogOffset(int bucket, long offset) { - if (logOffsets.containsKey(bucket)) { + public void addLogOffset(int bucket, long offset, boolean allowDuplicate) { + if (!allowDuplicate && logOffsets.containsKey(bucket)) { throw new RuntimeException( String.format( "bucket-%d appears multiple times, which is not possible.", bucket)); } - logOffsets.put(bucket, offset); + long newOffset = Math.max(logOffsets.getOrDefault(bucket, offset), offset); + logOffsets.put(bucket, newOffset); } public long identifier() { diff --git a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java index 303879337780f..5218a515a3374 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java @@ -222,7 +222,8 @@ public List commitData( null, Collections.emptyList(), (commit, committable) -> { - logOffsets.forEach(committable::addLogOffset); + logOffsets.forEach( + (bucket, offset) -> committable.addLogOffset(bucket, offset, false)); commit.commit(committable, Collections.emptyMap()); }); } diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java index c179a2c0a789f..8de8309bc8fbb 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java @@ -83,7 +83,7 @@ private static void addFileCommittables( if (!committable.logOffsets().containsKey(bucket)) { int offset = ID.incrementAndGet(); - committable.addLogOffset(bucket, offset); + committable.addLogOffset(bucket, offset, false); assertThat(committable.logOffsets().get(bucket)).isEqualTo(offset); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java index d237f4da56cf7..4908b99317bae 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java @@ -23,6 +23,7 @@ import org.apache.paimon.flink.sink.partition.PartitionListeners; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.manifest.ManifestCommittable; +import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; @@ -44,6 +45,7 @@ public class StoreCommitter implements Committer committables) { if (committerMetrics == null) { return; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java index aeb3e1857b9b7..537a98f97fb03 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java @@ -92,11 +92,11 @@ public WrappedManifestCommittable combine( WrappedManifestCommittable wrappedManifestCommittable, List committables) { for (MultiTableCommittable committable : committables) { + Identifier identifier = + Identifier.create(committable.getDatabase(), committable.getTable()); ManifestCommittable manifestCommittable = wrappedManifestCommittable.computeCommittableIfAbsent( - Identifier.create(committable.getDatabase(), committable.getTable()), - checkpointId, - watermark); + identifier, checkpointId, watermark); switch (committable.kind()) { case FILE: @@ -106,7 +106,9 @@ public WrappedManifestCommittable combine( case LOG_OFFSET: LogOffsetCommittable offset = (LogOffsetCommittable) committable.wrappedCommittable(); - manifestCommittable.addLogOffset(offset.bucket(), offset.offset()); + StoreCommitter committer = tableCommitters.get(identifier); + manifestCommittable.addLogOffset( + offset.bucket(), offset.offset(), committer.allowLogOffsetDuplicate()); break; } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java index 298f3155ba34e..b0aa76f157ac8 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java @@ -98,7 +98,7 @@ public static void addFileCommittables( if (!committable.logOffsets().containsKey(bucket)) { int offset = ID.incrementAndGet(); - committable.addLogOffset(bucket, offset); + committable.addLogOffset(bucket, offset, false); assertThat(committable.logOffsets().get(bucket)).isEqualTo(offset); } } From c7dfcfa18411af1511035b1481229499008eda05 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 13 Nov 2024 17:05:24 +0800 Subject: [PATCH 015/157] [core] Partitions system table support time travel (#4511) --- .../paimon/table/system/PartitionsTable.java | 2 +- .../table/system/PartitionsTableTest.java | 24 ++++++++++++++++--- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java index 736347b9aff3e..7e0b1f1d75682 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java @@ -178,7 +178,7 @@ public RecordReader createReader(Split split) throws IOException { throw new IllegalArgumentException("Unsupported split: " + split.getClass()); } - List partitions = fileStoreTable.newSnapshotReader().partitionEntries(); + List partitions = fileStoreTable.newScan().listPartitionEntries(); RowDataToObjectArrayConverter converter = new RowDataToObjectArrayConverter( diff --git a/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java index a17dc75466a68..8d12dc707bf5d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java @@ -40,6 +40,7 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import static org.assertj.core.api.Assertions.assertThat; @@ -76,7 +77,7 @@ public void before() throws Exception { partitionsTable = (PartitionsTable) catalog.getTable(filesTableId); // snapshot 1: append - write(table, GenericRow.of(1, 1, 1), GenericRow.of(1, 2, 5)); + write(table, GenericRow.of(1, 1, 1), GenericRow.of(1, 3, 5)); write(table, GenericRow.of(1, 1, 3), GenericRow.of(1, 2, 4)); } @@ -85,19 +86,36 @@ public void before() throws Exception { public void testPartitionRecordCount() throws Exception { List expectedRow = new ArrayList<>(); expectedRow.add(GenericRow.of(BinaryString.fromString("[1]"), 2L)); - expectedRow.add(GenericRow.of(BinaryString.fromString("[2]"), 2L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[2]"), 1L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[3]"), 1L)); // Only read partition and record count, record size may not stable. List result = read(partitionsTable, new int[][] {{0}, {1}}); assertThat(result).containsExactlyInAnyOrderElementsOf(expectedRow); } + @Test + public void testPartitionTimeTravel() throws Exception { + List expectedRow = new ArrayList<>(); + expectedRow.add(GenericRow.of(BinaryString.fromString("[1]"), 1L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[3]"), 1L)); + + // Only read partition and record count, record size may not stable. + List result = + read( + partitionsTable.copy( + Collections.singletonMap(CoreOptions.SCAN_VERSION.key(), "1")), + new int[][] {{0}, {1}}); + assertThat(result).containsExactlyInAnyOrderElementsOf(expectedRow); + } + @Test public void testPartitionValue() throws Exception { write(table, GenericRow.of(2, 1, 3), GenericRow.of(3, 1, 4)); List expectedRow = new ArrayList<>(); expectedRow.add(GenericRow.of(BinaryString.fromString("[1]"), 4L, 3L)); - expectedRow.add(GenericRow.of(BinaryString.fromString("[2]"), 2L, 2L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[2]"), 1L, 1L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[3]"), 1L, 1L)); List result = read(partitionsTable, new int[][] {{0}, {1}, {3}}); assertThat(result).containsExactlyInAnyOrderElementsOf(expectedRow); From c95c3e6f9132dc89fd39cf0427d9b5fba544476c Mon Sep 17 00:00:00 2001 From: "liming.1018" Date: Thu, 14 Nov 2024 10:57:02 +0800 Subject: [PATCH 016/157] [core] supports using dynamic parameters to query the system table of a specified branch. (#4527) --- .../table/system/AggregationFieldsTable.java | 18 +++---- .../paimon/table/system/BranchesTable.java | 13 +++-- .../paimon/table/system/ConsumersTable.java | 18 +++---- .../paimon/table/system/OptionsTable.java | 18 +++---- .../paimon/table/system/SchemasTable.java | 18 +++---- .../paimon/table/system/SnapshotsTable.java | 17 ++---- .../apache/paimon/table/system/TagsTable.java | 18 +++---- .../apache/paimon/flink/BranchSqlITCase.java | 52 +++++++++++++++++++ 8 files changed, 97 insertions(+), 75 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java index a88bde9e5d720..10a046ca70b52 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java @@ -80,17 +80,13 @@ public class AggregationFieldsTable implements ReadonlyTable { private final Path location; private final String branch; - public AggregationFieldsTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public AggregationFieldsTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public AggregationFieldsTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.branch = CoreOptions.branch(dataTable.schema().options()); + this.dataTable = dataTable; } @Override @@ -120,7 +116,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new AggregationFieldsTable(fileIO, location, branch); + return new AggregationFieldsTable(dataTable.copy(dynamicOptions)); } private class SchemasScan extends ReadOnceTableScan { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java index f523f20e9d200..384a2eee92c82 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java @@ -81,13 +81,12 @@ public class BranchesTable implements ReadonlyTable { private final FileIO fileIO; private final Path location; - public BranchesTable(FileStoreTable dataTable) { - this(dataTable.fileIO(), dataTable.location()); - } + private final FileStoreTable dataTable; - public BranchesTable(FileIO fileIO, Path location) { - this.fileIO = fileIO; - this.location = location; + public BranchesTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.dataTable = dataTable; } @Override @@ -117,7 +116,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new BranchesTable(fileIO, location); + return new BranchesTable(dataTable.copy(dynamicOptions)); } private class BranchesScan extends ReadOnceTableScan { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java index 9f7d12961e2f1..7e4816b13510b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java @@ -74,17 +74,13 @@ public class ConsumersTable implements ReadonlyTable { private final Path location; private final String branch; - public ConsumersTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public ConsumersTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public ConsumersTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.branch = CoreOptions.branch(dataTable.schema().options()); + this.dataTable = dataTable; } @Override @@ -114,7 +110,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new ConsumersTable(fileIO, location, branch); + return new ConsumersTable(dataTable.copy(dynamicOptions)); } private class ConsumersScan extends ReadOnceTableScan { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java index b4a3b82a2f5f7..c7dec03343d09 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java @@ -72,17 +72,13 @@ public class OptionsTable implements ReadonlyTable { private final Path location; private final String branch; - public OptionsTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public OptionsTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public OptionsTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.branch = CoreOptions.branch(dataTable.schema().options()); + this.dataTable = dataTable; } @Override @@ -112,7 +108,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new OptionsTable(fileIO, location, branch); + return new OptionsTable(dataTable.copy(dynamicOptions)); } private class OptionsScan extends ReadOnceTableScan { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java index 86e2598c609cf..d0df75b34f512 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java @@ -102,17 +102,13 @@ public class SchemasTable implements ReadonlyTable { private final Path location; private final String branch; - public SchemasTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public SchemasTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public SchemasTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.branch = CoreOptions.branch(dataTable.schema().options()); + this.dataTable = dataTable; } @Override @@ -142,7 +138,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new SchemasTable(fileIO, location, branch); + return new SchemasTable(dataTable.copy(dynamicOptions)); } private class SchemasScan extends ReadOnceTableScan { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java index a958432194400..10e5b691acc32 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java @@ -116,19 +116,10 @@ public class SnapshotsTable implements ReadonlyTable { private final FileStoreTable dataTable; public SnapshotsTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - dataTable, - CoreOptions.branch(dataTable.schema().options())); - } - - public SnapshotsTable( - FileIO fileIO, Path location, FileStoreTable dataTable, String branchName) { - this.fileIO = fileIO; - this.location = location; + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.branch = CoreOptions.branch(dataTable.schema().options()); this.dataTable = dataTable; - this.branch = branchName; } @Override @@ -158,7 +149,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new SnapshotsTable(fileIO, location, dataTable.copy(dynamicOptions), branch); + return new SnapshotsTable(dataTable.copy(dynamicOptions)); } private class SnapshotsScan extends ReadOnceTableScan { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java index 4d1b4e22ab189..9aafdb5983fd5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java @@ -95,17 +95,13 @@ public class TagsTable implements ReadonlyTable { private final Path location; private final String branch; - public TagsTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public TagsTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public TagsTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.branch = CoreOptions.branch(dataTable.schema().options()); + this.dataTable = dataTable; } @Override @@ -135,7 +131,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new TagsTable(fileIO, location, branch); + return new TagsTable(dataTable.copy(dynamicOptions)); } private class TagsScan extends ReadOnceTableScan { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java index 1d33a9e8a6f25..c25d99cb44591 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java @@ -347,6 +347,11 @@ public void testBranchOptionsTable() throws Exception { "+I[bucket, 2]", "+I[snapshot.time-retained, 1 h]", "+I[scan.infer-parallelism, false]"); + assertThat(collectResult("SELECT * FROM t$options /*+ OPTIONS('branch'='test') */")) + .containsExactlyInAnyOrder( + "+I[bucket, 2]", + "+I[snapshot.time-retained, 1 h]", + "+I[scan.infer-parallelism, false]"); } @Test @@ -360,6 +365,10 @@ public void testBranchSchemasTable() throws Exception { sql("ALTER TABLE t$branch_b1 SET ('snapshot.time-retained' = '5 h')"); assertThat(collectResult("SELECT schema_id FROM t$branch_b1$schemas order by schema_id")) .containsExactlyInAnyOrder("+I[0]", "+I[1]"); + assertThat( + collectResult( + "SELECT schema_id FROM t$schemas /*+ OPTIONS('branch'='b1') */ order by schema_id")) + .containsExactlyInAnyOrder("+I[0]", "+I[1]"); } @Test @@ -373,6 +382,8 @@ public void testBranchAuditLogTable() throws Exception { sql("INSERT INTO t$branch_b1 VALUES (3, 4)"); assertThat(collectResult("SELECT * FROM t$branch_b1$audit_log")) .containsExactlyInAnyOrder("+I[+I, 3, 4]"); + assertThat(collectResult("SELECT * FROM t$audit_log /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[+I, 3, 4]"); } @Test @@ -385,6 +396,8 @@ public void testBranchReadOptimizedTable() throws Exception { sql("INSERT INTO t$branch_b1 VALUES (3, 4)"); assertThat(collectResult("SELECT * FROM t$branch_b1$ro")) .containsExactlyInAnyOrder("+I[3, 4]"); + assertThat(collectResult("SELECT * FROM t$ro /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[3, 4]"); } @Test @@ -400,6 +413,10 @@ public void testBranchFilesTable() throws Exception { .containsExactlyInAnyOrder("+I[{a=1, b=2}]"); assertThat(collectResult("SELECT min_value_stats FROM t$branch_b1$files")) .containsExactlyInAnyOrder("+I[{a=3, b=4}]", "+I[{a=5, b=6}]"); + assertThat( + collectResult( + "SELECT min_value_stats FROM t$files /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[{a=3, b=4}]", "+I[{a=5, b=6}]"); } @Test @@ -416,6 +433,10 @@ public void testBranchTagsTable() throws Exception { .containsExactlyInAnyOrder("+I[tag1, 1, 1]"); assertThat(collectResult("SELECT tag_name,snapshot_id,record_count FROM t$branch_b1$tags")) .containsExactlyInAnyOrder("+I[tag1, 1, 1]", "+I[tag2, 2, 2]"); + assertThat( + collectResult( + "SELECT tag_name,snapshot_id,record_count FROM t$tags /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[tag1, 1, 1]", "+I[tag2, 2, 2]"); } @Test @@ -435,6 +456,8 @@ public void testBranchConsumersTable() throws Exception { assertThat(collectResult("SELECT * FROM t$consumers")).isEmpty(); assertThat(collectResult("SELECT * FROM t$branch_b1$consumers")) .containsExactlyInAnyOrder("+I[id1, 2]"); + assertThat(collectResult("SELECT * FROM t$consumers /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[id1, 2]"); } @Test @@ -458,6 +481,31 @@ public void testBranchManifestsTable() { .isTrue(); assertThat((long) row.getField(2)).isGreaterThan(0L); }); + List dynamicOptionRes = + sql( + "SELECT schema_id, file_name, file_size FROM t$manifests /*+ OPTIONS('branch'='b1') */"); + assertThat(dynamicOptionRes).containsExactlyInAnyOrderElementsOf(res); + } + + @Test + public void testBranchSnapshotsTable() throws Exception { + sql("CREATE TABLE t (a INT, b INT)"); + sql("INSERT INTO t VALUES (1, 2)"); + + sql("CALL sys.create_branch('default.t', 'b1')"); + sql("INSERT INTO t$branch_b1 VALUES (3, 4)"); + sql("INSERT INTO t$branch_b1 VALUES (5, 6)"); + + assertThat(collectResult("SELECT snapshot_id, schema_id, commit_kind FROM t$snapshots")) + .containsExactlyInAnyOrder("+I[1, 0, APPEND]"); + assertThat( + collectResult( + "SELECT snapshot_id, schema_id, commit_kind FROM t$branch_b1$snapshots")) + .containsExactlyInAnyOrder("+I[1, 0, APPEND]", "+I[2, 0, APPEND]"); + assertThat( + collectResult( + "SELECT snapshot_id, schema_id, commit_kind FROM t$snapshots /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[1, 0, APPEND]", "+I[2, 0, APPEND]"); } @Test @@ -479,6 +527,10 @@ public void testBranchPartitionsTable() throws Exception { collectResult( "SELECT `partition`, record_count, file_count FROM t$branch_b1$partitions")) .containsExactlyInAnyOrder("+I[[1], 2, 2]", "+I[[2], 3, 2]"); + assertThat( + collectResult( + "SELECT `partition`, record_count, file_count FROM t$partitions /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[[1], 2, 2]", "+I[[2], 3, 2]"); } @Test From 84a97ee952365c0cdbcf816c2caefc5003ec1ee7 Mon Sep 17 00:00:00 2001 From: Yubin Li Date: Thu, 14 Nov 2024 11:05:12 +0800 Subject: [PATCH 017/157] [core] Support deleting rows in partial update of specific sequence group (#4525) --- .../merge-engine/partial-update.md | 3 +- .../generated/core_configuration.html | 6 ++ .../java/org/apache/paimon/CoreOptions.java | 8 ++ .../compact/PartialUpdateMergeFunction.java | 47 +++++++++-- .../PartialUpdateMergeFunctionTest.java | 36 ++++++++ .../table/PrimaryKeyFileStoreTableTest.java | 84 +++++++++++++++++++ 6 files changed, 178 insertions(+), 6 deletions(-) diff --git a/docs/content/primary-key-table/merge-engine/partial-update.md b/docs/content/primary-key-table/merge-engine/partial-update.md index 83b664c1bc7f5..d664694e96652 100644 --- a/docs/content/primary-key-table/merge-engine/partial-update.md +++ b/docs/content/primary-key-table/merge-engine/partial-update.md @@ -51,7 +51,8 @@ By default, Partial update can not accept delete records, you can choose one of - Configure 'ignore-delete' to ignore delete records. - Configure 'partial-update.remove-record-on-delete' to remove the whole row when receiving delete records. - Configure 'sequence-group's to retract partial columns. - {{< /hint >}} + * Configure 'partial-update.remove-record-on-sequence-group' to remove the whole row when receiving delete records of specified sequence group. +{{< /hint >}} ## Sequence Group diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 7287cacc2c53e..3efeaeb3e8a9a 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -557,6 +557,12 @@ Boolean Whether to remove the whole row in partial-update engine when -D records are received. + +
partial-update.remove-record-on-sequence-group
+ (none) + String + Whether to remove the whole row in partial-update engine when -D records of specified sequence group are received. +
partition
(none) diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index db603c5b61f06..9875e2fac45f2 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -631,6 +631,14 @@ public class CoreOptions implements Serializable { .withDescription( "Whether to remove the whole row in partial-update engine when -D records are received."); + @Immutable + public static final ConfigOption PARTIAL_UPDATE_REMOVE_RECORD_ON_SEQUENCE_GROUP = + key("partial-update.remove-record-on-sequence-group") + .stringType() + .noDefaultValue() + .withDescription( + "Whether to remove the whole row in partial-update engine when -D records of specified sequence group are received."); + @Immutable public static final ConfigOption ROWKIND_FIELD = key("rowkind.field") diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunction.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunction.java index 4d720cb3f0750..ab25794129baf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunction.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunction.java @@ -52,6 +52,7 @@ import static org.apache.paimon.CoreOptions.FIELDS_PREFIX; import static org.apache.paimon.CoreOptions.FIELDS_SEPARATOR; import static org.apache.paimon.CoreOptions.PARTIAL_UPDATE_REMOVE_RECORD_ON_DELETE; +import static org.apache.paimon.CoreOptions.PARTIAL_UPDATE_REMOVE_RECORD_ON_SEQUENCE_GROUP; import static org.apache.paimon.utils.InternalRowUtils.createFieldGetters; /** @@ -68,6 +69,7 @@ public class PartialUpdateMergeFunction implements MergeFunction { private final boolean fieldSequenceEnabled; private final Map fieldAggregators; private final boolean removeRecordOnDelete; + private final Set sequenceGroupPartialDelete; private InternalRow currentKey; private long latestSequenceNumber; @@ -81,13 +83,15 @@ protected PartialUpdateMergeFunction( Map fieldSeqComparators, Map fieldAggregators, boolean fieldSequenceEnabled, - boolean removeRecordOnDelete) { + boolean removeRecordOnDelete, + Set sequenceGroupPartialDelete) { this.getters = getters; this.ignoreDelete = ignoreDelete; this.fieldSeqComparators = fieldSeqComparators; this.fieldAggregators = fieldAggregators; this.fieldSequenceEnabled = fieldSequenceEnabled; this.removeRecordOnDelete = removeRecordOnDelete; + this.sequenceGroupPartialDelete = sequenceGroupPartialDelete; } @Override @@ -220,8 +224,15 @@ private void retractWithSequenceGroup(KeyValue kv) { .anyMatch(field -> field == index)) { for (int field : seqComparator.compareFields()) { if (!updatedSequenceFields.contains(field)) { - row.setField(field, getters[field].getFieldOrNull(kv.value())); - updatedSequenceFields.add(field); + if (kv.valueKind() == RowKind.DELETE + && sequenceGroupPartialDelete.contains(field)) { + currentDeleteRow = true; + row = new GenericRow(getters.length); + return; + } else { + row.setField(field, getters[field].getFieldOrNull(kv.value())); + updatedSequenceFields.add(field); + } } } } else { @@ -278,13 +289,21 @@ private static class Factory implements MergeFunctionFactory { private final boolean removeRecordOnDelete; + private final String removeRecordOnSequenceGroup; + + private Set sequenceGroupPartialDelete; + private Factory(Options options, RowType rowType, List primaryKeys) { this.ignoreDelete = options.get(CoreOptions.IGNORE_DELETE); this.rowType = rowType; this.tableTypes = rowType.getFieldTypes(); + this.removeRecordOnSequenceGroup = + options.get(PARTIAL_UPDATE_REMOVE_RECORD_ON_SEQUENCE_GROUP); + this.sequenceGroupPartialDelete = new HashSet<>(); List fieldNames = rowType.getFieldNames(); this.fieldSeqComparators = new HashMap<>(); + Map sequenceGroupMap = new HashMap<>(); for (Map.Entry entry : options.toMap().entrySet()) { String k = entry.getKey(); String v = entry.getValue(); @@ -323,6 +342,7 @@ private Factory(Options options, RowType rowType, List primaryKeys) { fieldName -> { int index = fieldNames.indexOf(fieldName); fieldSeqComparators.put(index, userDefinedSeqComparator); + sequenceGroupMap.put(fieldName, index); }); } } @@ -345,6 +365,21 @@ private Factory(Options options, RowType rowType, List primaryKeys) { String.format( "sequence group and %s have conflicting behavior so should not be enabled at the same time.", PARTIAL_UPDATE_REMOVE_RECORD_ON_DELETE)); + + if (removeRecordOnSequenceGroup != null) { + String[] sequenceGroupArr = removeRecordOnSequenceGroup.split(FIELDS_SEPARATOR); + Preconditions.checkState( + sequenceGroupMap.keySet().containsAll(Arrays.asList(sequenceGroupArr)), + String.format( + "field '%s' defined in '%s' option must be part of sequence groups", + removeRecordOnSequenceGroup, + PARTIAL_UPDATE_REMOVE_RECORD_ON_SEQUENCE_GROUP.key())); + sequenceGroupPartialDelete = + Arrays.stream(sequenceGroupArr) + .filter(sequenceGroupMap::containsKey) + .map(sequenceGroupMap::get) + .collect(Collectors.toSet()); + } } @Override @@ -405,7 +440,8 @@ public MergeFunction create(@Nullable int[][] projection) { projectedSeqComparators, projectedAggregators, !fieldSeqComparators.isEmpty(), - removeRecordOnDelete); + removeRecordOnDelete, + sequenceGroupPartialDelete); } else { Map fieldSeqComparators = new HashMap<>(); this.fieldSeqComparators.forEach( @@ -419,7 +455,8 @@ public MergeFunction create(@Nullable int[][] projection) { fieldSeqComparators, fieldAggregators, !fieldSeqComparators.isEmpty(), - removeRecordOnDelete); + removeRecordOnDelete, + sequenceGroupPartialDelete); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunctionTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunctionTest.java index a6e1b5f90fe23..93f634944e6e8 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunctionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunctionTest.java @@ -95,6 +95,42 @@ public void testSequenceGroup() { validate(func, 1, null, null, 6, null, null, 6); } + @Test + public void testSequenceGroupPartialDelete() { + Options options = new Options(); + options.set("fields.f3.sequence-group", "f1,f2"); + options.set("fields.f6.sequence-group", "f4,f5"); + options.set("partial-update.remove-record-on-sequence-group", "f6"); + RowType rowType = + RowType.of( + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT()); + MergeFunction func = + PartialUpdateMergeFunction.factory(options, rowType, ImmutableList.of("f0")) + .create(); + func.reset(); + add(func, 1, 1, 1, 1, 1, 1, 1); + add(func, 1, 2, 2, 2, 2, 2, null); + validate(func, 1, 2, 2, 2, 1, 1, 1); + add(func, 1, 3, 3, 1, 3, 3, 3); + validate(func, 1, 2, 2, 2, 3, 3, 3); + + // delete + add(func, RowKind.DELETE, 1, 1, 1, 3, 1, 1, null); + validate(func, 1, null, null, 3, 3, 3, 3); + add(func, RowKind.DELETE, 1, 1, 1, 3, 1, 1, 4); + validate(func, null, null, null, null, null, null, null); + add(func, 1, 4, 4, 4, 5, 5, 5); + validate(func, 1, 4, 4, 4, 5, 5, 5); + add(func, RowKind.DELETE, 1, 1, 1, 6, 1, 1, 6); + validate(func, null, null, null, null, null, null, null); + } + @Test public void testMultiSequenceFields() { Options options = new Options(); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java index 4917d076014a9..133913c487cdf 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java @@ -1213,6 +1213,90 @@ public void testPartialUpdateRemoveRecordOnDelete() throws Exception { commit.close(); } + @Test + public void testPartialUpdateRemoveRecordOnSequenceGroup() throws Exception { + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT() + }, + new String[] {"pt", "a", "b", "seq1", "c", "d", "seq2"}); + FileStoreTable table = + createFileStoreTable( + options -> { + options.set("merge-engine", "partial-update"); + options.set("fields.seq1.sequence-group", "b"); + options.set("fields.seq2.sequence-group", "c,d"); + options.set("partial-update.remove-record-on-sequence-group", "seq2"); + }, + rowType); + FileStoreTable wrongTable = + createFileStoreTable( + options -> { + options.set("merge-engine", "partial-update"); + options.set("fields.seq1.sequence-group", "b"); + options.set("fields.seq2.sequence-group", "c,d"); + options.set("partial-update.remove-record-on-sequence-group", "b"); + }, + rowType); + Function rowToString = row -> internalRowToString(row, rowType); + + assertThatThrownBy(() -> wrongTable.newWrite("")) + .hasMessageContaining( + "field 'b' defined in 'partial-update.remove-record-on-sequence-group' option must be part of sequence groups"); + + SnapshotReader snapshotReader = table.newSnapshotReader(); + TableRead read = table.newRead(); + StreamTableWrite write = table.newWrite(""); + StreamTableCommit commit = table.newCommit(""); + // 1. Inserts + write.write(GenericRow.of(1, 1, 10, 1, 20, 20, 1)); + write.write(GenericRow.of(1, 1, 11, 2, 25, 25, 0)); + write.write(GenericRow.of(1, 1, 12, 1, 29, 29, 2)); + commit.commit(0, write.prepareCommit(true, 0)); + List result = + getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, 11, 2, 29, 29, 2]"); + + // 2. Update Before + write.write(GenericRow.ofKind(RowKind.UPDATE_BEFORE, 1, 1, 11, 2, 29, 29, 2)); + commit.commit(1, write.prepareCommit(true, 1)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, NULL, 2, NULL, NULL, 2]"); + + // 3. Update After + write.write(GenericRow.ofKind(RowKind.UPDATE_AFTER, 1, 1, 11, 2, 30, 30, 3)); + commit.commit(2, write.prepareCommit(true, 2)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, 11, 2, 30, 30, 3]"); + + // 4. Retracts + write.write(GenericRow.ofKind(RowKind.DELETE, 1, 1, 12, 3, 30, 30, 2)); + commit.commit(3, write.prepareCommit(true, 3)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, NULL, 3, 30, 30, 3]"); + + write.write(GenericRow.ofKind(RowKind.DELETE, 1, 1, 12, 2, 30, 31, 5)); + commit.commit(4, write.prepareCommit(true, 4)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).isEmpty(); + + // 5. Inserts + write.write(GenericRow.of(1, 1, 11, 2, 30, 31, 6)); + commit.commit(5, write.prepareCommit(true, 5)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, 11, 2, 30, 31, 6]"); + + write.close(); + commit.close(); + } + @Test public void testPartialUpdateWithAgg() throws Exception { RowType rowType = From 7e1fe395df0831a9d65bf4b6126562252a7a7bf2 Mon Sep 17 00:00:00 2001 From: tsreaper Date: Thu, 14 Nov 2024 18:36:02 +0800 Subject: [PATCH 018/157] [flink] Support updating row type nested in array/map in Flink (#4528) --- .../paimon/casting/CastElementGetter.java | 41 ++++ .../paimon/casting/CastFieldGetter.java | 3 + .../apache/paimon/casting/CastedArray.java | 201 ++++++++++++++++++ .../org/apache/paimon/casting/CastedMap.java | 70 ++++++ .../org/apache/paimon/casting/CastedRow.java | 2 - .../paimon/schema/SchemaEvolutionUtil.java | 99 +++++---- .../apache/paimon/schema/SchemaManager.java | 44 +++- .../paimon/schema/SchemaManagerTest.java | 50 ++++- .../org/apache/paimon/flink/FlinkCatalog.java | 41 +++- .../paimon/flink/SchemaChangeITCase.java | 66 +++++- 10 files changed, 562 insertions(+), 55 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/casting/CastElementGetter.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/casting/CastedArray.java create mode 100644 paimon-common/src/main/java/org/apache/paimon/casting/CastedMap.java diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/CastElementGetter.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastElementGetter.java new file mode 100644 index 0000000000000..b8a91f572a355 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastElementGetter.java @@ -0,0 +1,41 @@ +/* + * 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.paimon.casting; + +import org.apache.paimon.data.InternalArray; + +/** Get element from array and cast it according to specific {@link CastExecutor}. */ +public class CastElementGetter { + + private final InternalArray.ElementGetter elementGetter; + private final CastExecutor castExecutor; + + @SuppressWarnings("unchecked") + public CastElementGetter( + InternalArray.ElementGetter elementGetter, CastExecutor castExecutor) { + this.elementGetter = elementGetter; + this.castExecutor = (CastExecutor) castExecutor; + } + + @SuppressWarnings("unchecked") + public V getElementOrNull(InternalArray array, int pos) { + Object value = elementGetter.getElementOrNull(array, pos); + return value == null ? null : (V) castExecutor.cast(value); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/CastFieldGetter.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastFieldGetter.java index 02168300a8420..208ef5f30f5bd 100644 --- a/paimon-common/src/main/java/org/apache/paimon/casting/CastFieldGetter.java +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastFieldGetter.java @@ -24,14 +24,17 @@ * Get field value from row with given pos and cast it according to specific {@link CastExecutor}. */ public class CastFieldGetter { + private final InternalRow.FieldGetter fieldGetter; private final CastExecutor castExecutor; + @SuppressWarnings("unchecked") public CastFieldGetter(InternalRow.FieldGetter fieldGetter, CastExecutor castExecutor) { this.fieldGetter = fieldGetter; this.castExecutor = (CastExecutor) castExecutor; } + @SuppressWarnings("unchecked") public V getFieldOrNull(InternalRow row) { Object value = fieldGetter.getFieldOrNull(row); return value == null ? null : (V) castExecutor.cast(value); diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/CastedArray.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastedArray.java new file mode 100644 index 0000000000000..778b11d1f8873 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastedArray.java @@ -0,0 +1,201 @@ +/* + * 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.paimon.casting; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.Timestamp; + +/** + * An implementation of {@link InternalArray} which provides a casted view of the underlying {@link + * InternalArray}. + * + *

It reads data from underlying {@link InternalArray} according to source logical type and casts + * it with specific {@link CastExecutor}. + */ +public class CastedArray implements InternalArray { + + private final CastElementGetter castElementGetter; + private InternalArray array; + + protected CastedArray(CastElementGetter castElementGetter) { + this.castElementGetter = castElementGetter; + } + + /** + * Replaces the underlying {@link InternalArray} backing this {@link CastedArray}. + * + *

This method replaces the array in place and does not return a new object. This is done for + * performance reasons. + */ + public static CastedArray from(CastElementGetter castElementGetter) { + return new CastedArray(castElementGetter); + } + + public CastedArray replaceArray(InternalArray array) { + this.array = array; + return this; + } + + @Override + public int size() { + return array.size(); + } + + @Override + public boolean[] toBooleanArray() { + boolean[] result = new boolean[size()]; + for (int i = 0; i < result.length; i++) { + result[i] = castElementGetter.getElementOrNull(array, i); + } + return result; + } + + @Override + public byte[] toByteArray() { + byte[] result = new byte[size()]; + for (int i = 0; i < result.length; i++) { + result[i] = castElementGetter.getElementOrNull(array, i); + } + return result; + } + + @Override + public short[] toShortArray() { + short[] result = new short[size()]; + for (int i = 0; i < result.length; i++) { + result[i] = castElementGetter.getElementOrNull(array, i); + } + return result; + } + + @Override + public int[] toIntArray() { + int[] result = new int[size()]; + for (int i = 0; i < result.length; i++) { + result[i] = castElementGetter.getElementOrNull(array, i); + } + return result; + } + + @Override + public long[] toLongArray() { + long[] result = new long[size()]; + for (int i = 0; i < result.length; i++) { + result[i] = castElementGetter.getElementOrNull(array, i); + } + return result; + } + + @Override + public float[] toFloatArray() { + float[] result = new float[size()]; + for (int i = 0; i < result.length; i++) { + result[i] = castElementGetter.getElementOrNull(array, i); + } + return result; + } + + @Override + public double[] toDoubleArray() { + double[] result = new double[size()]; + for (int i = 0; i < result.length; i++) { + result[i] = castElementGetter.getElementOrNull(array, i); + } + return result; + } + + @Override + public boolean isNullAt(int pos) { + return castElementGetter.getElementOrNull(array, pos) == null; + } + + @Override + public boolean getBoolean(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public byte getByte(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public short getShort(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public int getInt(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public long getLong(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public float getFloat(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public double getDouble(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public BinaryString getString(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public Timestamp getTimestamp(int pos, int precision) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public byte[] getBinary(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public InternalArray getArray(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public InternalMap getMap(int pos) { + return castElementGetter.getElementOrNull(array, pos); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + return castElementGetter.getElementOrNull(array, pos); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/CastedMap.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastedMap.java new file mode 100644 index 0000000000000..4068407ca71c4 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastedMap.java @@ -0,0 +1,70 @@ +/* + * 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.paimon.casting; + +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; + +/** + * An implementation of {@link InternalMap} which provides a casted view of the underlying {@link + * InternalMap}. + * + *

It reads data from underlying {@link InternalMap} according to source logical type and casts + * it with specific {@link CastExecutor}. + */ +public class CastedMap implements InternalMap { + + private final CastedArray castedValueArray; + private InternalMap map; + + protected CastedMap(CastElementGetter castValueGetter) { + this.castedValueArray = CastedArray.from(castValueGetter); + } + + /** + * Replaces the underlying {@link InternalMap} backing this {@link CastedMap}. + * + *

This method replaces the map in place and does not return a new object. This is done for + * performance reasons. + */ + public static CastedMap from(CastElementGetter castValueGetter) { + return new CastedMap(castValueGetter); + } + + public CastedMap replaceMap(InternalMap map) { + this.castedValueArray.replaceArray(map.valueArray()); + this.map = map; + return this; + } + + @Override + public int size() { + return map.size(); + } + + @Override + public InternalArray keyArray() { + return map.keyArray(); + } + + @Override + public InternalArray valueArray() { + return castedValueArray; + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/CastedRow.java b/paimon-common/src/main/java/org/apache/paimon/casting/CastedRow.java index 25c5744255ef4..f9216d10b3a85 100644 --- a/paimon-common/src/main/java/org/apache/paimon/casting/CastedRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/casting/CastedRow.java @@ -34,8 +34,6 @@ * *

It reads data from underlying {@link InternalRow} according to source logical type and casts * it with specific {@link CastExecutor}. - * - *

Note: This class supports only top-level castings, not nested castings. */ public class CastedRow implements InternalRow { diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java index b5d7307073595..0ae2798c29e00 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java @@ -19,10 +19,15 @@ package org.apache.paimon.schema; import org.apache.paimon.KeyValue; +import org.apache.paimon.casting.CastElementGetter; import org.apache.paimon.casting.CastExecutor; import org.apache.paimon.casting.CastExecutors; import org.apache.paimon.casting.CastFieldGetter; +import org.apache.paimon.casting.CastedArray; +import org.apache.paimon.casting.CastedMap; import org.apache.paimon.casting.CastedRow; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; import org.apache.paimon.predicate.LeafPredicate; import org.apache.paimon.predicate.Predicate; @@ -31,7 +36,6 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.MapType; -import org.apache.paimon.types.MultisetType; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.InternalRowUtils; import org.apache.paimon.utils.ProjectedRow; @@ -372,6 +376,7 @@ private static CastFieldGetter[] createCastFieldGetterMapping( List tableFields, List dataFields, int[] indexMapping) { CastFieldGetter[] converterMapping = new CastFieldGetter[tableFields.size()]; boolean castExist = false; + for (int i = 0; i < tableFields.size(); i++) { int dataIndex = indexMapping == null ? i : indexMapping[i]; if (dataIndex < 0) { @@ -380,53 +385,39 @@ private static CastFieldGetter[] createCastFieldGetterMapping( } else { DataField tableField = tableFields.get(i); DataField dataField = dataFields.get(dataIndex); - if (dataField.type().equalsIgnoreNullable(tableField.type())) { - // Create getter with index i and projected row data will convert to underlying - // data - converterMapping[i] = - new CastFieldGetter( - InternalRowUtils.createNullCheckingFieldGetter( - dataField.type(), i), - CastExecutors.identityCastExecutor()); - } else { - // TODO support column type evolution in nested type - checkState( - !(tableField.type() instanceof MapType - || dataField.type() instanceof ArrayType - || dataField.type() instanceof MultisetType), - "Only support column type evolution in atomic and row data type."); - - CastExecutor castExecutor; - if (tableField.type() instanceof RowType - && dataField.type() instanceof RowType) { - castExecutor = - createRowCastExecutor( - (RowType) dataField.type(), (RowType) tableField.type()); - } else { - castExecutor = CastExecutors.resolve(dataField.type(), tableField.type()); - } - checkNotNull( - castExecutor, - "Cannot cast from type " - + dataField.type() - + " to type " - + tableField.type()); - - // Create getter with index i and projected row data will convert to underlying - // data - converterMapping[i] = - new CastFieldGetter( - InternalRowUtils.createNullCheckingFieldGetter( - dataField.type(), i), - castExecutor); + if (!dataField.type().equalsIgnoreNullable(tableField.type())) { castExist = true; } + + // Create getter with index i and projected row data will convert to underlying data + converterMapping[i] = + new CastFieldGetter( + InternalRowUtils.createNullCheckingFieldGetter(dataField.type(), i), + createCastExecutor(dataField.type(), tableField.type())); } } return castExist ? converterMapping : null; } + private static CastExecutor createCastExecutor(DataType inputType, DataType targetType) { + if (targetType.equalsIgnoreNullable(inputType)) { + return CastExecutors.identityCastExecutor(); + } else if (inputType instanceof RowType && targetType instanceof RowType) { + return createRowCastExecutor((RowType) inputType, (RowType) targetType); + } else if (inputType instanceof ArrayType && targetType instanceof ArrayType) { + return createArrayCastExecutor((ArrayType) inputType, (ArrayType) targetType); + } else if (inputType instanceof MapType && targetType instanceof MapType) { + return createMapCastExecutor((MapType) inputType, (MapType) targetType); + } else { + return checkNotNull( + CastExecutors.resolve(inputType, targetType), + "Cannot cast from type %s to type %s", + inputType, + targetType); + } + } + private static CastExecutor createRowCastExecutor( RowType inputType, RowType targetType) { int[] indexMapping = createIndexMapping(targetType.getFields(), inputType.getFields()); @@ -446,4 +437,32 @@ private static CastExecutor createRowCastExecutor( return value; }; } + + private static CastExecutor createArrayCastExecutor( + ArrayType inputType, ArrayType targetType) { + CastElementGetter castElementGetter = + new CastElementGetter( + InternalArray.createElementGetter(inputType.getElementType()), + createCastExecutor( + inputType.getElementType(), targetType.getElementType())); + + CastedArray castedArray = CastedArray.from(castElementGetter); + return castedArray::replaceArray; + } + + private static CastExecutor createMapCastExecutor( + MapType inputType, MapType targetType) { + checkState( + inputType.getKeyType().equals(targetType.getKeyType()), + "Cannot cast map type %s to map type %s, because they have different key types.", + inputType.getKeyType(), + targetType.getKeyType()); + CastElementGetter castElementGetter = + new CastElementGetter( + InternalArray.createElementGetter(inputType.getValueType()), + createCastExecutor(inputType.getValueType(), targetType.getValueType())); + + CastedMap castedMap = CastedMap.from(castElementGetter); + return castedMap::replaceMap; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index 86e365a88f831..a84348810b990 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -37,9 +37,11 @@ import org.apache.paimon.schema.SchemaChange.UpdateColumnType; import org.apache.paimon.schema.SchemaChange.UpdateComment; import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeCasts; +import org.apache.paimon.types.MapType; import org.apache.paimon.types.ReassignFieldId; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.BranchManager; @@ -636,17 +638,17 @@ public void updateIntermediateColumn(List newFields, int depth) continue; } + String fullFieldName = + String.join(".", Arrays.asList(updateFieldNames).subList(0, depth + 1)); List nestedFields = - new ArrayList<>( - ((org.apache.paimon.types.RowType) field.type()).getFields()); + new ArrayList<>(extractRowType(field.type(), fullFieldName).getFields()); updateIntermediateColumn(nestedFields, depth + 1); newFields.set( i, new DataField( field.id(), field.name(), - new org.apache.paimon.types.RowType( - field.type().isNullable(), nestedFields), + wrapNewRowType(field.type(), nestedFields), field.description())); return; } @@ -656,6 +658,40 @@ public void updateIntermediateColumn(List newFields, int depth) String.join(".", Arrays.asList(updateFieldNames).subList(0, depth + 1))); } + private RowType extractRowType(DataType type, String fullFieldName) { + switch (type.getTypeRoot()) { + case ROW: + return (RowType) type; + case ARRAY: + return extractRowType(((ArrayType) type).getElementType(), fullFieldName); + case MAP: + return extractRowType(((MapType) type).getValueType(), fullFieldName); + default: + throw new IllegalArgumentException( + fullFieldName + " is not a structured type."); + } + } + + private DataType wrapNewRowType(DataType type, List nestedFields) { + switch (type.getTypeRoot()) { + case ROW: + return new RowType(type.isNullable(), nestedFields); + case ARRAY: + return new ArrayType( + type.isNullable(), + wrapNewRowType(((ArrayType) type).getElementType(), nestedFields)); + case MAP: + MapType mapType = (MapType) type; + return new MapType( + type.isNullable(), + mapType.getKeyType(), + wrapNewRowType(mapType.getValueType(), nestedFields)); + default: + throw new IllegalStateException( + "Trying to wrap a row type in " + type + ". This is unexpected."); + } + } + protected abstract void updateLastColumn(List newFields, String fieldName) throws Catalog.ColumnNotExistException, Catalog.ColumnAlreadyExistException; diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java index 088cb72f92e6c..f0d6543699659 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java @@ -31,6 +31,7 @@ import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.table.sink.TableCommitImpl; import org.apache.paimon.table.sink.TableWriteImpl; +import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; @@ -690,7 +691,7 @@ public void testUpdateNestedColumnType() throws Exception { SchemaChange updateColumnType = SchemaChange.updateColumnType( - new String[] {"v", "f2", "f1"}, DataTypes.BIGINT(), true); + new String[] {"v", "f2", "f1"}, DataTypes.BIGINT(), false); manager.commitChanges(updateColumnType); innerType = @@ -708,8 +709,53 @@ public void testUpdateNestedColumnType() throws Exception { SchemaChange middleColumnNotExistUpdateColumnType = SchemaChange.updateColumnType( - new String[] {"v", "invalid", "f1"}, DataTypes.BIGINT(), true); + new String[] {"v", "invalid", "f1"}, DataTypes.BIGINT(), false); assertThatCode(() -> manager.commitChanges(middleColumnNotExistUpdateColumnType)) .hasMessageContaining("Column v.invalid does not exist"); } + + @Test + public void testUpdateRowTypeInArrayAndMap() throws Exception { + RowType innerType = + RowType.of( + new DataField(2, "f1", DataTypes.INT()), + new DataField(3, "f2", DataTypes.BIGINT())); + RowType outerType = + RowType.of( + new DataField(0, "k", DataTypes.INT()), + new DataField( + 1, "v", new ArrayType(new MapType(DataTypes.INT(), innerType)))); + + Schema schema = + new Schema( + outerType.getFields(), + Collections.singletonList("k"), + Collections.emptyList(), + new HashMap<>(), + ""); + SchemaManager manager = new SchemaManager(LocalFileIO.create(), path); + manager.createTable(schema); + + SchemaChange addColumn = + SchemaChange.addColumn( + new String[] {"v", "f3"}, + DataTypes.STRING(), + null, + SchemaChange.Move.first("f3")); + SchemaChange dropColumn = SchemaChange.dropColumn(new String[] {"v", "f2"}); + SchemaChange updateColumnType = + SchemaChange.updateColumnType(new String[] {"v", "f1"}, DataTypes.BIGINT(), false); + manager.commitChanges(addColumn, dropColumn, updateColumnType); + + innerType = + RowType.of( + new DataField(4, "f3", DataTypes.STRING()), + new DataField(2, "f1", DataTypes.BIGINT())); + outerType = + RowType.of( + new DataField(0, "k", DataTypes.INT()), + new DataField( + 1, "v", new ArrayType(new MapType(DataTypes.INT(), innerType)))); + assertThat(manager.latest().get().logicalRowType()).isEqualTo(outerType); + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index ae30fa569d59e..09fc0328ef655 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -673,14 +673,13 @@ private void generateNestedColumnUpdates( org.apache.paimon.types.DataType oldType, org.apache.paimon.types.DataType newType, List schemaChanges) { + String joinedNames = String.join(".", fieldNames); if (oldType.getTypeRoot() == DataTypeRoot.ROW) { Preconditions.checkArgument( newType.getTypeRoot() == DataTypeRoot.ROW, - "Column " - + String.join(".", fieldNames) - + " can only be updated to row type, and cannot be updated to " - + newType - + " type"); + "Column %s can only be updated to row type, and cannot be updated to %s type", + joinedNames, + newType.getTypeRoot()); org.apache.paimon.types.RowType oldRowType = (org.apache.paimon.types.RowType) oldType; org.apache.paimon.types.RowType newRowType = (org.apache.paimon.types.RowType) newType; @@ -699,7 +698,7 @@ private void generateNestedColumnUpdates( lastIdx < idx, "Order of existing fields in column %s must be kept the same. " + "However, field %s and %s have changed their orders.", - String.join(".", fieldNames), + joinedNames, lastFieldName, name); lastIdx = idx; @@ -751,6 +750,36 @@ private void generateNestedColumnUpdates( fullFieldNames, oldField.type(), field.type(), schemaChanges); } } + } else if (oldType.getTypeRoot() == DataTypeRoot.ARRAY) { + Preconditions.checkArgument( + newType.getTypeRoot() == DataTypeRoot.ARRAY, + "Column %s can only be updated to array type, and cannot be updated to %s type", + joinedNames, + newType); + generateNestedColumnUpdates( + fieldNames, + ((org.apache.paimon.types.ArrayType) oldType).getElementType(), + ((org.apache.paimon.types.ArrayType) newType).getElementType(), + schemaChanges); + } else if (oldType.getTypeRoot() == DataTypeRoot.MAP) { + Preconditions.checkArgument( + newType.getTypeRoot() == DataTypeRoot.MAP, + "Column %s can only be updated to map type, and cannot be updated to %s type", + joinedNames, + newType); + org.apache.paimon.types.MapType oldMapType = (org.apache.paimon.types.MapType) oldType; + org.apache.paimon.types.MapType newMapType = (org.apache.paimon.types.MapType) newType; + Preconditions.checkArgument( + oldMapType.getKeyType().equals(newMapType.getKeyType()), + "Cannot update key type of column %s from %s type to %s type", + joinedNames, + oldMapType.getKeyType(), + newMapType.getKeyType()); + generateNestedColumnUpdates( + fieldNames, + oldMapType.getValueType(), + newMapType.getValueType(), + schemaChanges); } else { if (!oldType.equalsIgnoreNullable(newType)) { schemaChanges.add( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java index a2ef1d5c8acec..a8e8332156b37 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java @@ -29,6 +29,7 @@ import org.junit.jupiter.params.provider.ValueSource; import java.time.format.DateTimeFormatter; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -1133,6 +1134,69 @@ public void testUpdateNestedColumn(String formatType) { assertThatCode(() -> sql("ALTER TABLE T MODIFY (v ROW(f1 BIGINT, f2 INT, f3 STRING))")) .hasRootCauseMessage( - "Column v.f2 can only be updated to row type, and cannot be updated to INT type"); + "Column v.f2 can only be updated to row type, and cannot be updated to INTEGER type"); + } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testUpdateRowInArrayAndMap(String formatType) { + sql( + "CREATE TABLE T " + + "( k INT, v1 ARRAY, v2 MAP, PRIMARY KEY (k) NOT ENFORCED ) " + + "WITH ( 'bucket' = '1', 'file.format' = '" + + formatType + + "' )"); + sql( + "INSERT INTO T VALUES " + + "(1, ARRAY[ROW(100, 'apple'), ROW(101, 'banana')], MAP[100, ROW('cat', 1000), 101, ROW('dog', 1001)]), " + + "(2, ARRAY[ROW(200, 'pear'), ROW(201, 'grape')], MAP[200, ROW('tiger', 2000), 201, ROW('wolf', 2001)])"); + + Map map1 = new HashMap<>(); + map1.put(100, Row.of("cat", 1000)); + map1.put(101, Row.of("dog", 1001)); + Map map2 = new HashMap<>(); + map2.put(200, Row.of("tiger", 2000)); + map2.put(201, Row.of("wolf", 2001)); + assertThat(sql("SELECT * FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, new Row[] {Row.of(100, "apple"), Row.of(101, "banana")}, map1), + Row.of(2, new Row[] {Row.of(200, "pear"), Row.of(201, "grape")}, map2)); + + sql( + "ALTER TABLE T MODIFY (v1 ARRAY, v2 MAP)"); + sql( + "INSERT INTO T VALUES " + + "(1, ARRAY[ROW(1000000000000, 'apple', 'A'), ROW(1000000000001, 'banana', 'B')], MAP[100, ROW(1000.0, 1000), 101, ROW(1001.0, 1001)]), " + + "(3, ARRAY[ROW(3000000000000, 'mango', 'M'), ROW(3000000000001, 'cherry', 'C')], MAP[300, ROW(3000.0, 3000), 301, ROW(3001.0, 3001)])"); + + map1.clear(); + map1.put(100, Row.of(1000.0, 1000)); + map1.put(101, Row.of(1001.0, 1001)); + map2.clear(); + map2.put(200, Row.of(null, 2000)); + map2.put(201, Row.of(null, 2001)); + Map map3 = new HashMap<>(); + map3.put(300, Row.of(3000.0, 3000)); + map3.put(301, Row.of(3001.0, 3001)); + assertThat(sql("SELECT v2, v1, k FROM T")) + .containsExactlyInAnyOrder( + Row.of( + map1, + new Row[] { + Row.of(1000000000000L, "apple", "A"), + Row.of(1000000000001L, "banana", "B") + }, + 1), + Row.of( + map2, + new Row[] {Row.of(200L, "pear", null), Row.of(201L, "grape", null)}, + 2), + Row.of( + map3, + new Row[] { + Row.of(3000000000000L, "mango", "M"), + Row.of(3000000000001L, "cherry", "C") + }, + 3)); } } From 9e4b28ae27a786fa9833571556560ce0b9c07f01 Mon Sep 17 00:00:00 2001 From: WenjunMin Date: Thu, 14 Nov 2024 18:39:54 +0800 Subject: [PATCH 019/157] [core] Introduce binlog system table to pack the UB and UA (#4520) --- docs/content/maintenance/system-tables.md | 19 +++ .../paimon/reader/PackChangelogReader.java | 131 +++++++++++++++ .../paimon/table/system/AuditLogTable.java | 12 +- .../paimon/table/system/BinlogTable.java | 151 ++++++++++++++++++ .../table/system/SystemTableLoader.java | 2 + paimon-flink/paimon-flink-common/pom.xml | 6 + .../paimon/flink/SystemTableITCase.java | 66 ++++++++ 7 files changed, 381 insertions(+), 6 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/reader/PackChangelogReader.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java diff --git a/docs/content/maintenance/system-tables.md b/docs/content/maintenance/system-tables.md index 462f8c27f887d..0246d6faf8dc5 100644 --- a/docs/content/maintenance/system-tables.md +++ b/docs/content/maintenance/system-tables.md @@ -406,4 +406,23 @@ SELECT * FROM T$statistics; 1 rows in set */ ``` +### Binlog Table + +You can streaming or batch query the binlog through binlog table. In this system table, +the update before and update after will be packed in one row. + +``` +/* ++------------------+----------------------+-----------------------+ +| rowkind | column_0 | column_1 | ++------------------+----------------------+-----------------------+ +| +I | [col_0] | [col_1] | ++------------------+----------------------+-----------------------+ +| +U | [col_0_ub, col_0_ua] | [col_1_ub, col_1_ua] | ++------------------+----------------------+-----------------------+ +| -D | [col_0] | [col_1] | ++------------------+----------------------+-----------------------+ +*/ +``` + diff --git a/paimon-common/src/main/java/org/apache/paimon/reader/PackChangelogReader.java b/paimon-common/src/main/java/org/apache/paimon/reader/PackChangelogReader.java new file mode 100644 index 0000000000000..a60780ff5e065 --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/reader/PackChangelogReader.java @@ -0,0 +1,131 @@ +/* + * 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.paimon.reader; + +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.serializer.InternalRowSerializer; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.types.RowType; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.function.BiFunction; + +/** The reader which will pack the update before and update after message together. */ +public class PackChangelogReader implements RecordReader { + + private final RecordReader reader; + private final BiFunction function; + private final InternalRowSerializer serializer; + private boolean initialized = false; + + public PackChangelogReader( + RecordReader reader, + BiFunction function, + RowType rowType) { + this.reader = reader; + this.function = function; + this.serializer = new InternalRowSerializer(rowType); + } + + @Nullable + @Override + public RecordIterator readBatch() throws IOException { + if (!initialized) { + initialized = true; + return new InternRecordIterator(reader, function, serializer); + } + return null; + } + + @Override + public void close() throws IOException { + reader.close(); + } + + private static class InternRecordIterator implements RecordIterator { + + private RecordIterator currentBatch; + + private final BiFunction function; + private final RecordReader reader; + private final InternalRowSerializer serializer; + private boolean endOfData; + + public InternRecordIterator( + RecordReader reader, + BiFunction function, + InternalRowSerializer serializer) { + this.reader = reader; + this.function = function; + this.serializer = serializer; + this.endOfData = false; + } + + @Nullable + @Override + public InternalRow next() throws IOException { + InternalRow row1 = nextRow(); + if (row1 == null) { + return null; + } + InternalRow row2 = null; + if (row1.getRowKind() == RowKind.UPDATE_BEFORE) { + row1 = serializer.copy(row1); + row2 = nextRow(); + } + return function.apply(row1, row2); + } + + @Nullable + private InternalRow nextRow() throws IOException { + InternalRow row = null; + while (!endOfData && row == null) { + RecordIterator batch = nextBatch(); + if (batch == null) { + endOfData = true; + return null; + } + + row = batch.next(); + if (row == null) { + releaseBatch(); + } + } + return row; + } + + @Nullable + private RecordIterator nextBatch() throws IOException { + if (currentBatch == null) { + currentBatch = reader.readBatch(); + } + return currentBatch; + } + + @Override + public void releaseBatch() { + if (currentBatch != null) { + currentBatch.releaseBatch(); + currentBatch = null; + } + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java index e0acd9fb38ea6..7438f9393d604 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java @@ -526,13 +526,13 @@ public DataTableScan withShard(int indexOfThisSubtask, int numberOfParallelSubta } } - private class AuditLogRead implements InnerTableRead { + class AuditLogRead implements InnerTableRead { - private final InnerTableRead dataRead; + protected final InnerTableRead dataRead; - private int[] readProjection; + protected int[] readProjection; - private AuditLogRead(InnerTableRead dataRead) { + protected AuditLogRead(InnerTableRead dataRead) { this.dataRead = dataRead.forceKeepDelete(); this.readProjection = defaultProjection(); } @@ -600,9 +600,9 @@ private InternalRow convertRow(InternalRow data) { } /** A {@link ProjectedRow} which returns row kind when mapping index is negative. */ - private static class AuditLogRow extends ProjectedRow { + static class AuditLogRow extends ProjectedRow { - private AuditLogRow(int[] indexMapping, InternalRow row) { + AuditLogRow(int[] indexMapping, InternalRow row) { super(indexMapping); replaceRow(row); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java new file mode 100644 index 0000000000000..96f9f6ed6185f --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java @@ -0,0 +1,151 @@ +/* + * 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.paimon.table.system; + +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.reader.PackChangelogReader; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.InnerTableRead; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.RowType; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER; + +/** + * A {@link Table} for reading binlog of table. The binlog format is as below. + * + *

INSERT: [+I, [co1, null], [col2, null]] + * + *

UPDATE: [+U, [co1_ub, col1_ua], [col2_ub, col2_ua]] + * + *

DELETE: [-D, [co1, null], [col2, null]] + */ +public class BinlogTable extends AuditLogTable { + + public static final String BINLOG = "binlog"; + + private final FileStoreTable wrapped; + + public BinlogTable(FileStoreTable wrapped) { + super(wrapped); + this.wrapped = wrapped; + } + + @Override + public String name() { + return wrapped.name() + SYSTEM_TABLE_SPLITTER + BINLOG; + } + + @Override + public RowType rowType() { + List fields = new ArrayList<>(); + fields.add(SpecialFields.ROW_KIND); + for (DataField field : wrapped.rowType().getFields()) { + DataField newField = + new DataField( + field.id(), + field.name(), + new ArrayType(field.type().nullable()), // convert to nullable + field.description()); + fields.add(newField); + } + return new RowType(fields); + } + + @Override + public InnerTableRead newRead() { + return new BinlogRead(wrapped.newRead()); + } + + @Override + public Table copy(Map dynamicOptions) { + return new BinlogTable(wrapped.copy(dynamicOptions)); + } + + private class BinlogRead extends AuditLogRead { + + private BinlogRead(InnerTableRead dataRead) { + super(dataRead); + } + + @Override + public RecordReader createReader(Split split) throws IOException { + DataSplit dataSplit = (DataSplit) split; + if (dataSplit.isStreaming()) { + return new PackChangelogReader( + dataRead.createReader(split), + (row1, row2) -> + new AuditLogRow( + readProjection, + convertToArray( + row1, row2, wrapped.rowType().fieldGetters())), + wrapped.rowType()); + } else { + return dataRead.createReader(split) + .transform( + (row) -> + new AuditLogRow( + readProjection, + convertToArray( + row, + null, + wrapped.rowType().fieldGetters()))); + } + } + + private InternalRow convertToArray( + InternalRow row1, + @Nullable InternalRow row2, + InternalRow.FieldGetter[] fieldGetters) { + GenericRow row = new GenericRow(row1.getFieldCount()); + for (int i = 0; i < row1.getFieldCount(); i++) { + Object o1 = fieldGetters[i].getFieldOrNull(row1); + Object o2; + if (row2 != null) { + o2 = fieldGetters[i].getFieldOrNull(row2); + row.setField(i, new GenericArray(new Object[] {o1, o2})); + } else { + row.setField(i, new GenericArray(new Object[] {o1})); + } + } + // If no row2 provided, then follow the row1 kind. + if (row2 == null) { + row.setRowKind(row1.getRowKind()); + } else { + row.setRowKind(row2.getRowKind()); + } + return row; + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java index a84f41ec1a514..3d5b211316ec1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java @@ -41,6 +41,7 @@ import static org.apache.paimon.table.system.AggregationFieldsTable.AGGREGATION_FIELDS; import static org.apache.paimon.table.system.AllTableOptionsTable.ALL_TABLE_OPTIONS; import static org.apache.paimon.table.system.AuditLogTable.AUDIT_LOG; +import static org.apache.paimon.table.system.BinlogTable.BINLOG; import static org.apache.paimon.table.system.BranchesTable.BRANCHES; import static org.apache.paimon.table.system.BucketsTable.BUCKETS; import static org.apache.paimon.table.system.CatalogOptionsTable.CATALOG_OPTIONS; @@ -77,6 +78,7 @@ public class SystemTableLoader { .put(READ_OPTIMIZED, ReadOptimizedTable::new) .put(AGGREGATION_FIELDS, AggregationFieldsTable::new) .put(STATISTICS, StatisticTable::new) + .put(BINLOG, BinlogTable::new) .build(); public static final List SYSTEM_TABLES = new ArrayList<>(SYSTEM_TABLE_LOADERS.keySet()); diff --git a/paimon-flink/paimon-flink-common/pom.xml b/paimon-flink/paimon-flink-common/pom.xml index 4452af266e5e6..91222983bf6ba 100644 --- a/paimon-flink/paimon-flink-common/pom.xml +++ b/paimon-flink/paimon-flink-common/pom.xml @@ -162,6 +162,12 @@ under the License. iceberg-data ${iceberg.version} test + + + parquet-avro + org.apache.parquet + + diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java new file mode 100644 index 0000000000000..771f4acc5e585 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java @@ -0,0 +1,66 @@ +/* + * 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.paimon.flink; + +import org.apache.paimon.utils.BlockingIterator; + +import org.apache.flink.types.Row; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for system table. */ +public class SystemTableITCase extends CatalogTableITCase { + + @Test + public void testBinlogTableStreamRead() throws Exception { + sql( + "CREATE TABLE T (a INT, b INT, primary key (a) NOT ENFORCED) with ('changelog-producer' = 'lookup', " + + "'bucket' = '2')"); + BlockingIterator iterator = + streamSqlBlockIter("SELECT * FROM T$binlog /*+ OPTIONS('scan.mode' = 'latest') */"); + sql("INSERT INTO T VALUES (1, 2)"); + sql("INSERT INTO T VALUES (1, 3)"); + sql("INSERT INTO T VALUES (2, 2)"); + List rows = iterator.collect(3); + assertThat(rows) + .containsExactly( + Row.of("+I", new Integer[] {1}, new Integer[] {2}), + Row.of("+U", new Integer[] {1, 1}, new Integer[] {2, 3}), + Row.of("+I", new Integer[] {2}, new Integer[] {2})); + iterator.close(); + } + + @Test + public void testBinlogTableBatchRead() throws Exception { + sql( + "CREATE TABLE T (a INT, b INT, primary key (a) NOT ENFORCED) with ('changelog-producer' = 'lookup', " + + "'bucket' = '2')"); + sql("INSERT INTO T VALUES (1, 2)"); + sql("INSERT INTO T VALUES (1, 3)"); + sql("INSERT INTO T VALUES (2, 2)"); + List rows = sql("SELECT * FROM T$binlog /*+ OPTIONS('scan.mode' = 'latest') */"); + assertThat(rows) + .containsExactly( + Row.of("+I", new Integer[] {1}, new Integer[] {3}), + Row.of("+I", new Integer[] {2}, new Integer[] {2})); + } +} From 9b9ed72efaef15f7f3e2333b519bf476c122352b Mon Sep 17 00:00:00 2001 From: WenjunMin Date: Thu, 14 Nov 2024 19:05:20 +0800 Subject: [PATCH 020/157] [doc] Add the query binlog example in doc (#4532) --- docs/content/maintenance/system-tables.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/content/maintenance/system-tables.md b/docs/content/maintenance/system-tables.md index 0246d6faf8dc5..5ef5121c25a92 100644 --- a/docs/content/maintenance/system-tables.md +++ b/docs/content/maintenance/system-tables.md @@ -408,10 +408,11 @@ SELECT * FROM T$statistics; ``` ### Binlog Table -You can streaming or batch query the binlog through binlog table. In this system table, -the update before and update after will be packed in one row. +You can query the binlog through binlog table. In the binlog system table, the update before and update after will be packed in one row. + +```sql +SELECT * FROM T$binlog; -``` /* +------------------+----------------------+-----------------------+ | rowkind | column_0 | column_1 | From 6f515203e3edf8c8ecc852054d111c8fc5e54cc4 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Thu, 14 Nov 2024 19:19:13 +0800 Subject: [PATCH 021/157] [hotfix] Fix comments in BinlogTable --- .../main/java/org/apache/paimon/table/system/BinlogTable.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java index 96f9f6ed6185f..b17d61d44e776 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java @@ -45,11 +45,11 @@ /** * A {@link Table} for reading binlog of table. The binlog format is as below. * - *

INSERT: [+I, [co1, null], [col2, null]] + *

INSERT: [+I, [co1], [col2]] * *

UPDATE: [+U, [co1_ub, col1_ua], [col2_ub, col2_ua]] * - *

DELETE: [-D, [co1, null], [col2, null]] + *

DELETE: [-D, [co1], [col2]] */ public class BinlogTable extends AuditLogTable { From bf8f5598d9a48f907f2346df39507c27877a5952 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Thu, 14 Nov 2024 20:28:30 +0800 Subject: [PATCH 022/157] [parquet] Fix that cannot read parquet ROW data (#4533) --- .../columnar/heap/AbstractHeapVector.java | 4 +- .../data/columnar/heap/ElementCountable.java | 23 ++------ .../paimon/flink/BatchFileStoreITCase.java | 19 ++++++ .../format/parquet/ParquetReaderFactory.java | 6 +- .../parquet/reader/NestedColumnReader.java | 3 +- .../reader/NestedPrimitiveColumnReader.java | 6 +- .../parquet/reader/ParquetDecimalVector.java | 16 ++++- .../parquet/reader/RowColumnReader.java | 59 ------------------- 8 files changed, 50 insertions(+), 86 deletions(-) rename paimon-format/src/main/java/org/apache/paimon/format/parquet/position/RowPosition.java => paimon-common/src/main/java/org/apache/paimon/data/columnar/heap/ElementCountable.java (60%) delete mode 100644 paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RowColumnReader.java diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/heap/AbstractHeapVector.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/heap/AbstractHeapVector.java index 702877642327c..f0e82eac4fb10 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/heap/AbstractHeapVector.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/heap/AbstractHeapVector.java @@ -25,7 +25,8 @@ import java.util.Arrays; /** Heap vector that nullable shared structure. */ -public abstract class AbstractHeapVector extends AbstractWritableVector { +public abstract class AbstractHeapVector extends AbstractWritableVector + implements ElementCountable { public static final boolean LITTLE_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN; @@ -116,6 +117,7 @@ public HeapIntVector getDictionaryIds() { return dictionaryIds; } + @Override public int getLen() { return this.len; } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/RowPosition.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/heap/ElementCountable.java similarity index 60% rename from paimon-format/src/main/java/org/apache/paimon/format/parquet/position/RowPosition.java rename to paimon-common/src/main/java/org/apache/paimon/data/columnar/heap/ElementCountable.java index fb6378349007d..a32762d659fd0 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/RowPosition.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/heap/ElementCountable.java @@ -16,25 +16,10 @@ * limitations under the License. */ -package org.apache.paimon.format.parquet.position; +package org.apache.paimon.data.columnar.heap; -import javax.annotation.Nullable; +/** Container with a known number of elements. */ +public interface ElementCountable { -/** To represent struct's position in repeated type. */ -public class RowPosition { - @Nullable private final boolean[] isNull; - private final int positionsCount; - - public RowPosition(boolean[] isNull, int positionsCount) { - this.isNull = isNull; - this.positionsCount = positionsCount; - } - - public boolean[] getIsNull() { - return isNull; - } - - public int getPositionsCount() { - return positionsCount; - } + int getLen(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java index c30e6cd5612d3..cdc114b048a11 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java @@ -34,6 +34,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import java.math.BigDecimal; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -573,6 +574,24 @@ public void testCountStarPK() { validateCount1NotPushDown(sql); } + @Test + public void testParquetRowDecimalAndTimestamp() { + sql( + "CREATE TABLE parquet_row_decimal(`row` ROW) WITH ('file.format' = 'parquet')"); + sql("INSERT INTO parquet_row_decimal VALUES ( (ROW(1.2)) )"); + + assertThat(sql("SELECT * FROM parquet_row_decimal")) + .containsExactly(Row.of(Row.of(new BigDecimal("1.2")))); + + sql( + "CREATE TABLE parquet_row_timestamp(`row` ROW) WITH ('file.format' = 'parquet')"); + sql("INSERT INTO parquet_row_timestamp VALUES ( (ROW(TIMESTAMP'2024-11-13 18:00:00')) )"); + + assertThat(sql("SELECT * FROM parquet_row_timestamp")) + .containsExactly( + Row.of(Row.of(DateTimeUtils.toLocalDateTime("2024-11-13 18:00:00", 0)))); + } + private void validateCount1PushDown(String sql) { Transformation transformation = AbstractTestBase.translate(tEnv, sql); while (!transformation.getInputs().isEmpty()) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java index 53b4b1634b5fc..f0151d6f3d8fb 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.columnar.ColumnarRow; import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.data.columnar.VectorizedColumnBatch; +import org.apache.paimon.data.columnar.heap.ElementCountable; import org.apache.paimon.data.columnar.writable.WritableColumnVector; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.parquet.reader.ColumnReader; @@ -293,7 +294,10 @@ private VectorizedColumnBatch createVectorizedColumnBatch( for (int i = 0; i < writableVectors.length; i++) { switch (projectedFields[i].type().getTypeRoot()) { case DECIMAL: - vectors[i] = new ParquetDecimalVector(writableVectors[i]); + vectors[i] = + new ParquetDecimalVector( + writableVectors[i], + ((ElementCountable) writableVectors[i]).getLen()); break; case TIMESTAMP_WITHOUT_TIME_ZONE: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java index c89c77603dac2..68225fbd13207 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java @@ -20,6 +20,7 @@ import org.apache.paimon.data.columnar.ColumnVector; import org.apache.paimon.data.columnar.heap.AbstractHeapVector; +import org.apache.paimon.data.columnar.heap.ElementCountable; import org.apache.paimon.data.columnar.heap.HeapArrayVector; import org.apache.paimon.data.columnar.heap.HeapMapVector; import org.apache.paimon.data.columnar.heap.HeapRowVector; @@ -134,7 +135,7 @@ private Pair readRow( String.format("Row field does not have any children: %s.", field)); } - int len = ((AbstractHeapVector) finalChildrenVectors[0]).getLen(); + int len = ((ElementCountable) finalChildrenVectors[0]).getLen(); boolean[] isNull = new boolean[len]; Arrays.fill(isNull, true); boolean hasNull = false; diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java index 7ee33a0bb5cc8..7d00ff79234ae 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java @@ -495,7 +495,7 @@ private WritableColumnVector fillColumnVector(int total, List valueList) { phiv.vector[i] = ((List) valueList).get(i); } } - return new ParquetDecimalVector(phiv); + return new ParquetDecimalVector(phiv, total); case INT64: HeapLongVector phlv = new HeapLongVector(total); for (int i = 0; i < valueList.size(); i++) { @@ -505,10 +505,10 @@ private WritableColumnVector fillColumnVector(int total, List valueList) { phlv.vector[i] = ((List) valueList).get(i); } } - return new ParquetDecimalVector(phlv); + return new ParquetDecimalVector(phlv, total); default: HeapBytesVector phbv = getHeapBytesVector(total, valueList); - return new ParquetDecimalVector(phbv); + return new ParquetDecimalVector(phbv, total); } default: throw new RuntimeException("Unsupported type in the list: " + type); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetDecimalVector.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetDecimalVector.java index 28d308bac61f5..42714ab066dae 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetDecimalVector.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetDecimalVector.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.columnar.Dictionary; import org.apache.paimon.data.columnar.IntColumnVector; import org.apache.paimon.data.columnar.LongColumnVector; +import org.apache.paimon.data.columnar.heap.ElementCountable; import org.apache.paimon.data.columnar.writable.WritableBytesVector; import org.apache.paimon.data.columnar.writable.WritableColumnVector; import org.apache.paimon.data.columnar.writable.WritableIntVector; @@ -38,12 +39,18 @@ * {@link DecimalColumnVector} interface. */ public class ParquetDecimalVector - implements DecimalColumnVector, WritableLongVector, WritableIntVector, WritableBytesVector { + implements DecimalColumnVector, + WritableLongVector, + WritableIntVector, + WritableBytesVector, + ElementCountable { private final ColumnVector vector; + private final int len; - public ParquetDecimalVector(ColumnVector vector) { + public ParquetDecimalVector(ColumnVector vector, int len) { this.vector = vector; + this.len = len; } @Override @@ -225,4 +232,9 @@ public void fill(long value) { ((WritableLongVector) vector).fill(value); } } + + @Override + public int getLen() { + return len; + } } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RowColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RowColumnReader.java deleted file mode 100644 index fa2da03ef312f..0000000000000 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RowColumnReader.java +++ /dev/null @@ -1,59 +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.paimon.format.parquet.reader; - -import org.apache.paimon.data.columnar.heap.HeapRowVector; -import org.apache.paimon.data.columnar.writable.WritableColumnVector; - -import java.io.IOException; -import java.util.List; - -/** Row {@link ColumnReader}. */ -public class RowColumnReader implements ColumnReader { - - private final List fieldReaders; - - public RowColumnReader(List fieldReaders) { - this.fieldReaders = fieldReaders; - } - - @Override - public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { - HeapRowVector rowVector = (HeapRowVector) vector; - WritableColumnVector[] vectors = rowVector.getFields(); - // row vector null array - boolean[] isNulls = new boolean[readNumber]; - for (int i = 0; i < vectors.length; i++) { - fieldReaders.get(i).readToVector(readNumber, vectors[i]); - - for (int j = 0; j < readNumber; j++) { - if (i == 0) { - isNulls[j] = vectors[i].isNullAt(j); - } else { - isNulls[j] = isNulls[j] && vectors[i].isNullAt(j); - } - if (i == vectors.length - 1 && isNulls[j]) { - // rowColumnVector[j] is null only when all fields[j] of rowColumnVector[j] is - // null - rowVector.setNullAt(j); - } - } - } - } -} From 8b772071df5516a60cf9dc2f0b1029ecea36adee Mon Sep 17 00:00:00 2001 From: Jingsong Date: Thu, 14 Nov 2024 22:55:52 +0800 Subject: [PATCH 023/157] [doc] Update roadmap --- docs/content/project/roadmap.md | 49 ++++----------------------------- 1 file changed, 6 insertions(+), 43 deletions(-) diff --git a/docs/content/project/roadmap.md b/docs/content/project/roadmap.md index 2f6b63af00a10..34628e28c80f4 100644 --- a/docs/content/project/roadmap.md +++ b/docs/content/project/roadmap.md @@ -26,16 +26,6 @@ under the License. # Roadmap -## Native Format IO - -Integrate native Parquet & ORC reader & writer. - -## Deletion Vectors (Merge On Write) - -1. Primary Key Table Deletion Vectors Mode supports async compaction. -2. Append Table supports DELETE & UPDATE with Deletion Vectors Mode. (Now only Spark SQL) -3. Optimize lookup performance for HDD disk. - ## Flink Lookup Join Support Flink Custom Data Distribution Lookup Join to reach large-scale data lookup join. @@ -44,51 +34,24 @@ Support Flink Custom Data Distribution Lookup Join to reach large-scale data loo Introduce a mode to produce Iceberg snapshots. -## Branch - -Branch production ready. - -## Changelog life cycle decouple - -Changelog life cycle decouple supports none changelog-producer. - -## Partition Mark Done - -Support partition mark done. - -## Default File Format - -- Default compression is ZSTD with level 1. -- Parquet supports filter push down. -- Parquet supports arrow with row type element. -- Parquet becomes default file format. - ## Variant Type Support Variant Type with Spark 4.0 and Flink 2.0. Unlocking support for semi-structured data. -## Bucketed Join - -Support Bucketed Join with Spark SQL to reduce shuffler in Join. - ## File Index Add more index: -1. Bitmap -2. Inverse -## Column Family +1. Inverse + +## Vector Compaction -Support Column Family for super Wide Table. +Support Vector Compaction for super Wide Table. -## View & Function support +## Function support -Paimon Catalog supports views and functions. +Paimon Catalog supports functions. ## Files Schema Evolution Ingestion Introduce a files Ingestion with Schema Evolution. - -## Foreign Key Join - -Explore Foreign Key Join solution. From d3384ac16d0ee955918f4ba8d2ddc756570877e4 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Thu, 14 Nov 2024 23:01:32 +0800 Subject: [PATCH 024/157] [doc] Add doc to 'partition.mark-done-action' --- docs/content/flink/sql-write.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/content/flink/sql-write.md b/docs/content/flink/sql-write.md index 008fe498363cd..33adac5fb7f49 100644 --- a/docs/content/flink/sql-write.md +++ b/docs/content/flink/sql-write.md @@ -257,7 +257,8 @@ CREATE TABLE my_partitioned_table ( 'partition.timestamp-formatter'='yyyyMMdd', 'partition.timestamp-pattern'='$dt', 'partition.time-interval'='1 d', - 'partition.idle-time-to-done'='15 m' + 'partition.idle-time-to-done'='15 m', + 'partition.mark-done-action'='done-partition' ); ``` @@ -267,4 +268,5 @@ CREATE TABLE my_partitioned_table ( and then it will be marked as done. 3. Thirdly, by default, partition mark done will create _SUCCESS file, the content of _SUCCESS file is a json, contains `creationTime` and `modificationTime`, they can help you understand if there is any delayed data. You can also - configure other actions. + configure other actions, like `'done-partition'`, for example, partition `'dt=20240501'` with produce + `'dt=20240501.done'` done partition. From 203db415d519106d9f3d20a0fe83af58c71e78ea Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Mon, 18 Nov 2024 13:36:32 +0800 Subject: [PATCH 025/157] [core] Fix that batch unaware bucket compact cannot stop when there is no snapshot (#4539) --- .../UnawareAppendTableCompactionCoordinator.java | 3 +++ .../UnawareAppendTableCompactionCoordinatorTest.java | 11 +++++++++++ 2 files changed, 14 insertions(+) diff --git a/paimon-core/src/main/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinator.java b/paimon-core/src/main/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinator.java index 9a54ea72e7cc9..842b223167b27 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinator.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinator.java @@ -387,6 +387,9 @@ private void assignNewIterator() { if (nextSnapshot == null) { nextSnapshot = snapshotManager.latestSnapshotId(); if (nextSnapshot == null) { + if (!streamingMode) { + throw new EndOfScanException(); + } return; } snapshotReader.withMode(ScanMode.ALL); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinatorTest.java b/paimon-core/src/test/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinatorTest.java index 95826c195ec72..9bb461ffe151c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinatorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinatorTest.java @@ -29,6 +29,7 @@ import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.source.EndOfScanException; import org.apache.paimon.types.DataTypes; import org.junit.jupiter.api.BeforeEach; @@ -43,7 +44,9 @@ import static org.apache.paimon.mergetree.compact.MergeTreeCompactManagerTest.row; import static org.apache.paimon.stats.StatsTestUtils.newSimpleStats; +import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link UnawareAppendTableCompactionCoordinator}. */ public class UnawareAppendTableCompactionCoordinatorTest { @@ -135,6 +138,14 @@ public void testAgeGrowUp() { .isEqualTo(0); } + @Test + public void testBatchScanEmptyTable() { + compactionCoordinator = + new UnawareAppendTableCompactionCoordinator(appendOnlyFileStoreTable, false); + assertThatThrownBy(() -> compactionCoordinator.scan()) + .satisfies(anyCauseMatches(EndOfScanException.class)); + } + private void assertTasks(List files, int taskNum) { compactionCoordinator.notifyNewFiles(partition, files); List tasks = compactionCoordinator.compactPlan(); From 220789d5ab4c566f72584d2b85980c777fd7807d Mon Sep 17 00:00:00 2001 From: wangwj Date: Mon, 18 Nov 2024 15:20:42 +0800 Subject: [PATCH 026/157] [core] support drop stats in result of scan plan (#4506) --- .../org/apache/paimon/io/DataFileMeta.java | 21 ++++++++++++++++++ .../apache/paimon/manifest/ManifestEntry.java | 4 ++++ .../operation/AbstractFileStoreScan.java | 13 +++++++++++ .../paimon/operation/FileStoreScan.java | 2 ++ .../table/source/AbstractDataTableScan.java | 6 +++++ .../paimon/table/source/InnerTableScan.java | 5 +++++ .../paimon/table/source/ReadBuilder.java | 3 +++ .../paimon/table/source/ReadBuilderImpl.java | 11 ++++++++++ .../table/source/snapshot/SnapshotReader.java | 2 ++ .../source/snapshot/SnapshotReaderImpl.java | 6 +++++ .../paimon/table/system/AuditLogTable.java | 6 +++++ .../operation/KeyValueFileStoreScanTest.java | 22 +++++++++++++++++++ .../source/ContinuousFileStoreSource.java | 2 +- .../paimon/flink/source/FlinkTableSource.java | 8 ++++++- .../flink/source/StaticFileStoreSource.java | 2 +- .../source/operator/MonitorFunction.java | 2 +- 16 files changed, 111 insertions(+), 4 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java index b6cac5ae51304..bb9e45ff002da 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java @@ -409,6 +409,27 @@ public DataFileMeta rename(String newFileName) { valueStatsCols); } + public DataFileMeta copyWithoutStats() { + return new DataFileMeta( + fileName, + fileSize, + rowCount, + minKey, + maxKey, + keyStats, + EMPTY_STATS, + minSequenceNumber, + maxSequenceNumber, + schemaId, + level, + extraFiles, + creationTime, + deleteRowCount, + embeddedIndex, + fileSource, + Collections.emptyList()); + } + public List collectFiles(DataFilePathFactory pathFactory) { List paths = new ArrayList<>(); paths.add(pathFactory.toPath(fileName)); diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java index f7c5c4639a6f2..ee5dc2c344214 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java @@ -121,6 +121,10 @@ public Identifier identifier() { file.embeddedIndex()); } + public ManifestEntry copyWithoutStats() { + return new ManifestEntry(kind, partition, bucket, totalBuckets, file.copyWithoutStats()); + } + @Override public boolean equals(Object o) { if (!(o instanceof ManifestEntry)) { diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java index 683e6ffda481e..0e1f9357e312c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java @@ -90,6 +90,7 @@ public abstract class AbstractFileStoreScan implements FileStoreScan { private ManifestCacheFilter manifestCacheFilter = null; private ScanMetrics scanMetrics = null; + private boolean dropStats; public AbstractFileStoreScan( ManifestsReader manifestsReader, @@ -105,6 +106,7 @@ public AbstractFileStoreScan( this.manifestFileFactory = manifestFileFactory; this.tableSchemas = new ConcurrentHashMap<>(); this.parallelism = parallelism; + this.dropStats = false; } @Override @@ -215,6 +217,12 @@ public FileStoreScan withMetrics(ScanMetrics metrics) { return this; } + @Override + public FileStoreScan dropStats() { + this.dropStats = true; + return this; + } + @Nullable @Override public Integer parallelism() { @@ -291,6 +299,11 @@ public Snapshot snapshot() { @Override public List files() { + if (dropStats) { + return files.stream() + .map(ManifestEntry::copyWithoutStats) + .collect(Collectors.toList()); + } return files; } }; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java index bc0d7ff27301a..e643bf1617b4d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java @@ -81,6 +81,8 @@ public interface FileStoreScan { FileStoreScan withMetrics(ScanMetrics metrics); + FileStoreScan dropStats(); + @Nullable Integer parallelism(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java index 6a8aa9265e5c6..24c6943f546fe 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java @@ -102,6 +102,12 @@ public AbstractDataTableScan withMetricsRegistry(MetricRegistry metricsRegistry) return this; } + @Override + public AbstractDataTableScan dropStats() { + snapshotReader.dropStats(); + return this; + } + public CoreOptions options() { return options; } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java index 00a4fc0cde18b..c2425ff16f97f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java @@ -55,4 +55,9 @@ default InnerTableScan withMetricsRegistry(MetricRegistry metricRegistry) { // do nothing, should implement this if need return this; } + + default InnerTableScan dropStats() { + // do nothing, should implement this if need + return this; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java index 91d5f1004e914..0c1386ce441d8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java @@ -150,6 +150,9 @@ default ReadBuilder withProjection(int[][] projection) { */ ReadBuilder withShard(int indexOfThisSubtask, int numberOfParallelSubtasks); + /** Delete stats in scan plan result. */ + ReadBuilder dropStats(); + /** Create a {@link TableScan} to perform batch planning. */ TableScan newScan(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java index 577b0a20a99b3..95bfe6f24bc77 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java @@ -51,6 +51,8 @@ public class ReadBuilderImpl implements ReadBuilder { private @Nullable RowType readType; + private boolean dropStats = false; + public ReadBuilderImpl(InnerTable table) { this.table = table; } @@ -124,6 +126,12 @@ public ReadBuilder withBucketFilter(Filter bucketFilter) { return this; } + @Override + public ReadBuilder dropStats() { + this.dropStats = true; + return this; + } + @Override public TableScan newScan() { InnerTableScan tableScan = configureScan(table.newScan()); @@ -156,6 +164,9 @@ private InnerTableScan configureScan(InnerTableScan scan) { if (bucketFilter != null) { scan.withBucketFilter(bucketFilter); } + if (dropStats) { + scan.dropStats(); + } return scan; } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java index 2dd02be04f7c5..b59cf98bbb4c2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java @@ -85,6 +85,8 @@ public interface SnapshotReader { SnapshotReader withDataFileNameFilter(Filter fileNameFilter); + SnapshotReader dropStats(); + SnapshotReader withShard(int indexOfThisSubtask, int numberOfParallelSubtasks); SnapshotReader withMetricRegistry(MetricRegistry registry); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java index f4591734b68e3..7ce537ee52ec1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java @@ -264,6 +264,12 @@ public SnapshotReader withDataFileNameFilter(Filter fileNameFilter) { return this; } + @Override + public SnapshotReader dropStats() { + scan.dropStats(); + return this; + } + @Override public SnapshotReader withShard(int indexOfThisSubtask, int numberOfParallelSubtasks) { if (splitGenerator.alwaysRawConvertible()) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java index 7438f9393d604..e56ee90412ceb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java @@ -342,6 +342,12 @@ public SnapshotReader withDataFileNameFilter(Filter fileNameFilter) { return this; } + @Override + public SnapshotReader dropStats() { + wrapped.dropStats(); + return this; + } + @Override public SnapshotReader withShard(int indexOfThisSubtask, int numberOfParallelSubtasks) { wrapped.withShard(indexOfThisSubtask, numberOfParallelSubtasks); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java index ce17450538b1b..2fd8c10cd9445 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java @@ -50,6 +50,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; +import static org.apache.paimon.stats.SimpleStats.EMPTY_STATS; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link KeyValueFileStoreScan}. */ @@ -274,6 +275,27 @@ public void testWithManifestList() throws Exception { runTestExactMatch(scan, null, expected); } + @Test + public void testDropStatsInPlan() throws Exception { + ThreadLocalRandom random = ThreadLocalRandom.current(); + List data = generateData(100, 0, (long) Math.abs(random.nextInt(1000))); + writeData(data, 0); + data = generateData(100, 1, (long) Math.abs(random.nextInt(1000)) + 1000); + writeData(data, 0); + data = generateData(100, 2, (long) Math.abs(random.nextInt(1000)) + 2000); + writeData(data, 0); + data = generateData(100, 3, (long) Math.abs(random.nextInt(1000)) + 3000); + Snapshot snapshot = writeData(data, 0); + + KeyValueFileStoreScan scan = store.newScan(); + scan.withSnapshot(snapshot.id()).dropStats(); + List files = scan.plan().files(); + + for (ManifestEntry manifestEntry : files) { + assertThat(manifestEntry.file().valueStats()).isEqualTo(EMPTY_STATS); + } + } + private void runTestExactMatch( FileStoreScan scan, Long expectedSnapshotId, Map expected) throws Exception { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileStoreSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileStoreSource.java index 559976921e2e0..b7eb1d625ce3b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileStoreSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/ContinuousFileStoreSource.java @@ -77,7 +77,7 @@ public SplitEnumerator restoreEnu nextSnapshotId = checkpoint.currentSnapshotId(); splits = checkpoint.splits(); } - StreamTableScan scan = readBuilder.newStreamScan(); + StreamTableScan scan = readBuilder.dropStats().newStreamScan(); if (metricGroup(context) != null) { ((StreamDataTableScan) scan) .withMetricsRegistry(new FlinkMetricRegistry(context.metricGroup())); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java index 2be0248f3ce8c..9bfd36fdfaa81 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java @@ -175,6 +175,7 @@ protected void scanSplitsForInference() { List partitionEntries = table.newReadBuilder() .withFilter(predicate) + .dropStats() .newScan() .listPartitionEntries(); long totalSize = 0; @@ -188,7 +189,12 @@ protected void scanSplitsForInference() { new SplitStatistics((int) (totalSize / splitTargetSize + 1), rowCount); } else { List splits = - table.newReadBuilder().withFilter(predicate).newScan().plan().splits(); + table.newReadBuilder() + .withFilter(predicate) + .dropStats() + .newScan() + .plan() + .splits(); splitStatistics = new SplitStatistics( splits.size(), splits.stream().mapToLong(Split::rowCount).sum()); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/StaticFileStoreSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/StaticFileStoreSource.java index af425aab5e469..c388a6dccbbce 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/StaticFileStoreSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/StaticFileStoreSource.java @@ -87,7 +87,7 @@ public SplitEnumerator restoreEnu private List getSplits(SplitEnumeratorContext context) { FileStoreSourceSplitGenerator splitGenerator = new FileStoreSourceSplitGenerator(); - TableScan scan = readBuilder.newScan(); + TableScan scan = readBuilder.dropStats().newScan(); // register scan metrics if (context.metricGroup() != null) { ((InnerTableScan) scan) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java index 3805f6f8c536e..f21922670471b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java @@ -106,7 +106,7 @@ public MonitorFunction( @Override public void initializeState(FunctionInitializationContext context) throws Exception { - this.scan = readBuilder.newStreamScan(); + this.scan = readBuilder.dropStats().newStreamScan(); this.checkpointState = context.getOperatorStateStore() From 9a334c80d6fc1e4c0bfc13561a25eda674990d29 Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Tue, 19 Nov 2024 08:53:19 +0800 Subject: [PATCH 027/157] [hive] Sync db owner prop to hms when creating the db (#4544) --- .../org/apache/paimon/catalog/Catalog.java | 6 +++-- .../org/apache/paimon/hive/HiveCatalog.java | 11 +++++--- .../sql/DDLWithHiveCatalogTestBase.scala | 27 ++++++++----------- 3 files changed, 23 insertions(+), 21 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index 6a6a047bd38cd..d919c59782975 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -52,10 +52,12 @@ public interface Catalog extends AutoCloseable { String SYSTEM_TABLE_SPLITTER = "$"; String SYSTEM_DATABASE_NAME = "sys"; String SYSTEM_BRANCH_PREFIX = "branch_"; - String COMMENT_PROP = "comment"; String TABLE_DEFAULT_OPTION_PREFIX = "table-default."; - String DB_LOCATION_PROP = "location"; String DB_SUFFIX = ".db"; + + String COMMENT_PROP = "comment"; + String OWNER_PROP = "owner"; + String DB_LOCATION_PROP = "location"; String NUM_ROWS_PROP = "numRows"; String NUM_FILES_PROP = "numFiles"; String TOTAL_SIZE_PROP = "totalSize"; diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 8b8b62934dbdb..eed0fdb9bff14 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -283,6 +283,8 @@ private Database convertToHiveDatabase(String name, Map properti (key, value) -> { if (key.equals(COMMENT_PROP)) { database.setDescription(value); + } else if (key.equals(OWNER_PROP)) { + database.setOwnerName(value); } else if (key.equals(DB_LOCATION_PROP)) { database.setLocationUri(value); } else if (value != null) { @@ -299,12 +301,15 @@ public org.apache.paimon.catalog.Database getDatabaseImpl(String name) try { Database database = clients.run(client -> client.getDatabase(name)); Map options = new HashMap<>(database.getParameters()); - if (database.getLocationUri() != null) { - options.put(DB_LOCATION_PROP, database.getLocationUri()); - } if (database.getDescription() != null) { options.put(COMMENT_PROP, database.getDescription()); } + if (database.getOwnerName() != null) { + options.put(OWNER_PROP, database.getOwnerName()); + } + if (database.getLocationUri() != null) { + options.put(DB_LOCATION_PROP, database.getLocationUri()); + } return org.apache.paimon.catalog.Database.of(name, options, database.getDescription()); } catch (NoSuchObjectException e) { throw new DatabaseNotExistException(name); diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 7478f9628cfbf..33b993160dfd7 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -189,6 +189,7 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { val props = getDatabaseProps("paimon_db") Assertions.assertEquals(props("k1"), "v1") Assertions.assertEquals(props("k2"), "v2") + Assertions.assertTrue(getDatabaseOwner("paimon_db").nonEmpty) } } } @@ -296,29 +297,23 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } } - def getDatabaseLocation(dbName: String): String = { + def getDatabaseProp(dbName: String, propertyName: String): String = { spark - .sql(s"DESC DATABASE $dbName") - .filter("info_name == 'Location'") + .sql(s"DESC DATABASE EXTENDED $dbName") + .filter(s"info_name == '$propertyName'") .head() .getAs[String]("info_value") - .split(":")(1) } - def getDatabaseComment(dbName: String): String = { - spark - .sql(s"DESC DATABASE $dbName") - .filter("info_name == 'Comment'") - .head() - .getAs[String]("info_value") - } + def getDatabaseLocation(dbName: String): String = + getDatabaseProp(dbName, "Location").split(":")(1) + + def getDatabaseComment(dbName: String): String = getDatabaseProp(dbName, "Comment") + + def getDatabaseOwner(dbName: String): String = getDatabaseProp(dbName, "Owner") def getDatabaseProps(dbName: String): Map[String, String] = { - val dbPropsStr = spark - .sql(s"DESC DATABASE EXTENDED $dbName") - .filter("info_name == 'Properties'") - .head() - .getAs[String]("info_value") + val dbPropsStr = getDatabaseProp(dbName, "Properties") val pattern = "\\(([^,]+),([^)]+)\\)".r pattern .findAllIn(dbPropsStr.drop(1).dropRight(1)) From 19d7f66a006d02ee3ad233937fc6116a07acefbb Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Tue, 19 Nov 2024 11:02:16 +0800 Subject: [PATCH 028/157] [spark] Introduce View Support to SparkCatalog (#4538) --- .../RewritePaimonViewCommands.scala | 80 ++++++++++++ .../paimon/spark/sql/PaimonViewTest.scala | 21 ++++ .../RewritePaimonViewCommands.scala | 79 ++++++++++++ .../paimon/spark/sql/PaimonViewTest.scala | 21 ++++ .../paimon/spark/sql/PaimonViewTest.scala | 21 ++++ .../paimon/spark/sql/PaimonViewTest.scala | 21 ++++ .../paimon/spark/sql/PaimonViewTest.scala | 21 ++++ .../org/apache/paimon/spark/SparkCatalog.java | 49 ++------ .../apache/paimon/spark/SparkTypeUtils.java | 4 + .../paimon/spark/catalog/SupportView.java | 86 +++++++++++++ .../paimon/spark/utils/CatalogUtils.java | 41 ++++++ .../analysis/PaimonViewResolver.scala | 85 +++++++++++++ .../plans/logical/PaimonViewCommand.scala | 74 +++++++++++ .../spark/execution/PaimonStrategy.scala | 37 +++++- .../spark/execution/PaimonViewExec.scala | 117 ++++++++++++++++++ .../PaimonSparkSessionExtensions.scala | 3 +- .../paimon/spark/leafnode/package.scala | 7 +- .../PaimonSparkSqlExtensionsParser.scala | 4 +- .../PaimonSqlExtensionsAstBuilder.scala | 13 +- .../RewritePaimonViewCommands.scala | 77 ++++++++++++ .../sql/DDLWithHiveCatalogTestBase.scala | 4 +- .../paimon/spark/sql/PaimonViewTestBase.scala | 96 ++++++++++++++ 22 files changed, 914 insertions(+), 47 deletions(-) create mode 100644 paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala create mode 100644 paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala create mode 100644 paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala create mode 100644 paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala create mode 100644 paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala create mode 100644 paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala create mode 100644 paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala create mode 100644 paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportView.java create mode 100644 paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/CatalogUtils.java create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonViewCommand.scala create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonViewExec.scala create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala create mode 100644 paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala diff --git a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala new file mode 100644 index 0000000000000..e759edd0c2c69 --- /dev/null +++ b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala @@ -0,0 +1,80 @@ +/* + * 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.spark.sql.catalyst.parser.extensions + +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.spark.catalyst.plans.logical.{CreatePaimonView, DropPaimonView, ResolvedIdentifier, ShowPaimonViews} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{CTESubstitution, ResolvedNamespace, UnresolvedView} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{CatalogManager, LookupCatalog} + +case class RewritePaimonViewCommands(spark: SparkSession) + extends Rule[LogicalPlan] + with LookupCatalog { + + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + + case CreateViewStatement( + ResolvedIdent(resolved), + userSpecifiedColumns, + comment, + properties, + Some(originalText), + child, + allowExisting, + replace, + _) => + CreatePaimonView( + child = resolved, + queryText = originalText, + query = CTESubstitution.apply(child), + columnAliases = userSpecifiedColumns.map(_._1), + columnComments = userSpecifiedColumns.map(_._2.orElse(Option.empty)), + comment = comment, + properties = properties, + allowExisting = allowExisting, + replace = replace + ) + + case DropView(ResolvedIdent(resolved), ifExists: Boolean) => + DropPaimonView(resolved, ifExists) + + case ShowViews(_, pattern, output) if catalogManager.currentCatalog.isInstanceOf[SupportView] => + ShowPaimonViews( + ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, + output) + } + + private object ResolvedIdent { + def unapply(unresolved: Any): Option[ResolvedIdentifier] = unresolved match { + case CatalogAndIdentifier(viewCatalog: SupportView, ident) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case UnresolvedView(CatalogAndIdentifier(viewCatalog: SupportView, ident), _, _, _) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case _ => + None + } + } +} diff --git a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.sql + +class PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala new file mode 100644 index 0000000000000..5d57cda2f34b2 --- /dev/null +++ b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.parser.extensions + +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.spark.catalyst.plans.logical.{CreatePaimonView, DropPaimonView, ResolvedIdentifier, ShowPaimonViews} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{CTESubstitution, ResolvedNamespace, UnresolvedDBObjectName, UnresolvedView} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{CatalogManager, LookupCatalog} + +case class RewritePaimonViewCommands(spark: SparkSession) + extends Rule[LogicalPlan] + with LookupCatalog { + + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + + case CreateView( + ResolvedIdent(resolved), + userSpecifiedColumns, + comment, + properties, + Some(queryText), + query, + allowExisting, + replace) => + CreatePaimonView( + child = resolved, + queryText = queryText, + query = CTESubstitution.apply(query), + columnAliases = userSpecifiedColumns.map(_._1), + columnComments = userSpecifiedColumns.map(_._2.orElse(Option.empty)), + comment = comment, + properties = properties, + allowExisting = allowExisting, + replace = replace + ) + + case DropView(ResolvedIdent(resolved), ifExists: Boolean) => + DropPaimonView(resolved, ifExists) + + case ShowViews(_, pattern, output) if catalogManager.currentCatalog.isInstanceOf[SupportView] => + ShowPaimonViews( + ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, + output) + } + + private object ResolvedIdent { + def unapply(unresolved: Any): Option[ResolvedIdentifier] = unresolved match { + case UnresolvedDBObjectName(CatalogAndIdentifier(viewCatalog: SupportView, ident), _) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case UnresolvedView(CatalogAndIdentifier(viewCatalog: SupportView, ident), _, _, _) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case _ => + None + } + } +} diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.sql + +class PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.sql + +class PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.sql + +class PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.sql + +class PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index 89448c1f4310a..3b9af1694eef7 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -27,6 +27,7 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.spark.catalog.SparkBaseCatalog; import org.apache.paimon.spark.catalog.SupportFunction; +import org.apache.paimon.spark.catalog.SupportView; import org.apache.paimon.table.FormatTable; import org.apache.paimon.table.FormatTableOptions; @@ -72,10 +73,12 @@ import static org.apache.paimon.spark.SparkCatalogOptions.DEFAULT_DATABASE; import static org.apache.paimon.spark.SparkTypeUtils.toPaimonType; import static org.apache.paimon.spark.util.OptionUtils.copyWithSQLConf; +import static org.apache.paimon.spark.utils.CatalogUtils.checkNamespace; +import static org.apache.paimon.spark.utils.CatalogUtils.toIdentifier; import static org.apache.paimon.utils.Preconditions.checkArgument; /** Spark {@link TableCatalog} for paimon. */ -public class SparkCatalog extends SparkBaseCatalog implements SupportFunction { +public class SparkCatalog extends SparkBaseCatalog implements SupportFunction, SupportView { private static final Logger LOG = LoggerFactory.getLogger(SparkCatalog.class); @@ -126,10 +129,7 @@ public String[] defaultNamespace() { @Override public void createNamespace(String[] namespace, Map metadata) throws NamespaceAlreadyExistsException { - checkArgument( - isValidateNamespace(namespace), - "Namespace %s is not valid", - Arrays.toString(namespace)); + checkNamespace(namespace); try { catalog.createDatabase(namespace[0], false, metadata); } catch (Catalog.DatabaseAlreadyExistException e) { @@ -152,9 +152,7 @@ public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceExcep if (namespace.length == 0) { return listNamespaces(); } - if (!isValidateNamespace(namespace)) { - throw new NoSuchNamespaceException(namespace); - } + checkNamespace(namespace); try { catalog.getDatabase(namespace[0]); return new String[0][]; @@ -166,10 +164,7 @@ public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceExcep @Override public Map loadNamespaceMetadata(String[] namespace) throws NoSuchNamespaceException { - checkArgument( - isValidateNamespace(namespace), - "Namespace %s is not valid", - Arrays.toString(namespace)); + checkNamespace(namespace); String dataBaseName = namespace[0]; try { return catalog.getDatabase(dataBaseName).options(); @@ -207,10 +202,7 @@ public boolean dropNamespace(String[] namespace) throws NoSuchNamespaceException */ public boolean dropNamespace(String[] namespace, boolean cascade) throws NoSuchNamespaceException { - checkArgument( - isValidateNamespace(namespace), - "Namespace %s is not valid", - Arrays.toString(namespace)); + checkNamespace(namespace); try { catalog.dropDatabase(namespace[0], false, cascade); return true; @@ -224,10 +216,7 @@ public boolean dropNamespace(String[] namespace, boolean cascade) @Override public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException { - checkArgument( - isValidateNamespace(namespace), - "Missing database in namespace: %s", - Arrays.toString(namespace)); + checkNamespace(namespace); try { return catalog.listTables(namespace[0]).stream() .map(table -> Identifier.of(namespace, table)) @@ -239,10 +228,7 @@ public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceExcepti @Override public void invalidateTable(Identifier ident) { - try { - catalog.invalidateTable(toIdentifier(ident)); - } catch (NoSuchTableException ignored) { - } + catalog.invalidateTable(toIdentifier(ident)); } @Override @@ -347,7 +333,7 @@ public boolean dropTable(Identifier ident) { try { catalog.dropTable(toIdentifier(ident), false); return true; - } catch (Catalog.TableNotExistException | NoSuchTableException e) { + } catch (Catalog.TableNotExistException e) { return false; } } @@ -454,10 +440,6 @@ private void validateAlterProperty(String alterKey) { } } - private boolean isValidateNamespace(String[] namespace) { - return namespace.length == 1; - } - @Override public void renameTable(Identifier oldIdent, Identifier newIdent) throws NoSuchTableException, TableAlreadyExistsException { @@ -472,15 +454,6 @@ public void renameTable(Identifier oldIdent, Identifier newIdent) // --------------------- tools ------------------------------------------ - protected org.apache.paimon.catalog.Identifier toIdentifier(Identifier ident) - throws NoSuchTableException { - if (!isValidateNamespace(ident.namespace())) { - throw new NoSuchTableException(ident); - } - - return new org.apache.paimon.catalog.Identifier(ident.namespace()[0], ident.name()); - } - protected org.apache.spark.sql.connector.catalog.Table loadSparkTable( Identifier ident, Map extraOptions) throws NoSuchTableException { try { diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java index 8bba676200ce7..f6643f7584065 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java @@ -81,6 +81,10 @@ public static DataType fromPaimonType(org.apache.paimon.types.DataType type) { return type.accept(PaimonToSparkTypeVisitor.INSTANCE); } + public static org.apache.paimon.types.RowType toPaimonRowType(StructType type) { + return (RowType) toPaimonType(type); + } + public static org.apache.paimon.types.DataType toPaimonType(DataType dataType) { return SparkToPaimonTypeVisitor.visit(dataType); } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportView.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportView.java new file mode 100644 index 0000000000000..b8ce86e892866 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportView.java @@ -0,0 +1,86 @@ +/* + * 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.paimon.spark.catalog; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.spark.SparkTypeUtils; +import org.apache.paimon.view.View; +import org.apache.paimon.view.ViewImpl; + +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.types.StructType; + +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.spark.utils.CatalogUtils.checkNamespace; +import static org.apache.paimon.spark.utils.CatalogUtils.toIdentifier; + +/** Catalog methods for working with Views. */ +public interface SupportView extends WithPaimonCatalog { + + default List listViews(String[] namespace) throws NoSuchNamespaceException { + try { + checkNamespace(namespace); + return paimonCatalog().listViews(namespace[0]); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchNamespaceException(namespace); + } + } + + default View loadView(Identifier ident) throws Catalog.ViewNotExistException { + return paimonCatalog().getView(toIdentifier(ident)); + } + + default void createView( + Identifier ident, + StructType schema, + String queryText, + String comment, + Map properties, + Boolean ignoreIfExists) + throws NoSuchNamespaceException { + org.apache.paimon.catalog.Identifier paimonIdent = toIdentifier(ident); + try { + paimonCatalog() + .createView( + paimonIdent, + new ViewImpl( + paimonIdent, + SparkTypeUtils.toPaimonRowType(schema), + queryText, + comment, + properties), + ignoreIfExists); + } catch (Catalog.ViewAlreadyExistException e) { + throw new RuntimeException("view already exists: " + ident, e); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchNamespaceException(ident.namespace()); + } + } + + default void dropView(Identifier ident, Boolean ignoreIfExists) { + try { + paimonCatalog().dropView(toIdentifier(ident), ignoreIfExists); + } catch (Catalog.ViewNotExistException e) { + throw new RuntimeException("view not exists: " + ident, e); + } + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/CatalogUtils.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/CatalogUtils.java new file mode 100644 index 0000000000000..fca9df210e708 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/CatalogUtils.java @@ -0,0 +1,41 @@ +/* + * 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.paimon.spark.utils; + +import org.apache.spark.sql.connector.catalog.Identifier; + +import java.util.Arrays; + +import static org.apache.paimon.utils.Preconditions.checkArgument; + +/** Utils of catalog. */ +public class CatalogUtils { + + public static void checkNamespace(String[] namespace) { + checkArgument( + namespace.length == 1, + "Paimon only support single namespace, but got %s", + Arrays.toString(namespace)); + } + + public static org.apache.paimon.catalog.Identifier toIdentifier(Identifier ident) { + checkNamespace(ident.namespace()); + return new org.apache.paimon.catalog.Identifier(ident.namespace()[0], ident.name()); + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala new file mode 100644 index 0000000000000..a375a296583ec --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala @@ -0,0 +1,85 @@ +/* + * 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.paimon.spark.catalyst.analysis + +import org.apache.paimon.catalog.Catalog.ViewNotExistException +import org.apache.paimon.spark.SparkTypeUtils +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.view.View + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedRelation} +import org.apache.spark.sql.catalyst.expressions.{Alias, UpCast} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.parser.extensions.{CurrentOrigin, Origin} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.PaimonLookupCatalog + +case class PaimonViewResolver(spark: SparkSession) + extends Rule[LogicalPlan] + with PaimonLookupCatalog { + + protected lazy val catalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { + case u @ UnresolvedRelation(parts @ CatalogAndIdentifier(catalog: SupportView, ident), _, _) => + try { + val view = catalog.loadView(ident) + createViewRelation(parts, view) + } catch { + case _: ViewNotExistException => + u + } + } + + private def createViewRelation(nameParts: Seq[String], view: View): LogicalPlan = { + val parsedPlan = parseViewText(nameParts.toArray.mkString("."), view.query) + + val aliases = SparkTypeUtils.fromPaimonRowType(view.rowType()).fields.zipWithIndex.map { + case (expected, pos) => + val attr = GetColumnByOrdinal(pos, expected.dataType) + Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = + Some(expected.metadata)) + } + + SubqueryAlias(nameParts, Project(aliases, parsedPlan)) + } + + private def parseViewText(name: String, viewText: String): LogicalPlan = { + val origin = Origin( + objectType = Some("VIEW"), + objectName = Some(name) + ) + try { + CurrentOrigin.withOrigin(origin) { + try { + spark.sessionState.sqlParser.parseQuery(viewText) + } catch { + // For compatibility with Spark 3.2 and below + case _: NoSuchMethodError => + spark.sessionState.sqlParser.parsePlan(viewText) + } + } + } catch { + case _: ParseException => + throw new RuntimeException("Failed to parse view text: " + viewText) + } + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonViewCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonViewCommand.scala new file mode 100644 index 0000000000000..24b27bb0e6cca --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonViewCommand.scala @@ -0,0 +1,74 @@ +/* + * 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.paimon.spark.catalyst.plans.logical + +import org.apache.paimon.spark.leafnode.{PaimonBinaryCommand, PaimonUnaryCommand} + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, ShowViews, Statistics} +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier} + +case class CreatePaimonView( + child: LogicalPlan, + queryText: String, + query: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String] = Seq.empty, + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean) + extends PaimonBinaryCommand { + + override def left: LogicalPlan = child + + override def right: LogicalPlan = query + + override protected def withNewChildrenInternal( + newLeft: LogicalPlan, + newRight: LogicalPlan): LogicalPlan = + copy(child = newLeft, query = newRight) +} + +case class DropPaimonView(child: LogicalPlan, ifExists: Boolean) extends PaimonUnaryCommand { + + override protected def withNewChildInternal(newChild: LogicalPlan): DropPaimonView = + copy(child = newChild) +} + +case class ShowPaimonViews( + namespace: LogicalPlan, + pattern: Option[String], + override val output: Seq[Attribute] = ShowViews.getOutputAttrs) + extends PaimonUnaryCommand { + + override def child: LogicalPlan = namespace + + override protected def withNewChildInternal(newChild: LogicalPlan): ShowPaimonViews = + copy(namespace = newChild) +} + +/** Copy from spark 3.4+ */ +case class ResolvedIdentifier(catalog: CatalogPlugin, identifier: Identifier) extends LeafNode { + + override def output: Seq[Attribute] = Nil + + override def stats: Statistics = Statistics.DUMMY +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala index 0c3d3e6b6dc64..0c3865f7d979a 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala @@ -19,10 +19,12 @@ package org.apache.paimon.spark.execution import org.apache.paimon.spark.{SparkCatalog, SparkUtils} -import org.apache.paimon.spark.catalyst.plans.logical.{CreateOrReplaceTagCommand, DeleteTagCommand, PaimonCallCommand, RenameTagCommand, ShowTagsCommand} +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.spark.catalyst.plans.logical.{CreateOrReplaceTagCommand, CreatePaimonView, DeleteTagCommand, DropPaimonView, PaimonCallCommand, RenameTagCommand, ResolvedIdentifier, ShowPaimonViews, ShowTagsCommand} import org.apache.spark.sql.{SparkSession, Strategy} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow, PredicateHelper} import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LogicalPlan} import org.apache.spark.sql.connector.catalog.{Identifier, PaimonLookupCatalog, TableCatalog} @@ -65,6 +67,39 @@ case class PaimonStrategy(spark: SparkSession) case RenameTagCommand(PaimonCatalogAndIdentifier(catalog, ident), sourceTag, targetTag) => RenameTagExec(catalog, ident, sourceTag, targetTag) :: Nil + case CreatePaimonView( + ResolvedIdentifier(viewCatalog: SupportView, ident), + queryText, + query, + columnAliases, + columnComments, + queryColumnNames, + comment, + properties, + allowExisting, + replace) => + CreatePaimonViewExec( + viewCatalog, + ident, + queryText, + query.schema, + columnAliases, + columnComments, + queryColumnNames, + comment, + properties, + allowExisting, + replace) :: Nil + + case DropPaimonView(ResolvedIdentifier(viewCatalog: SupportView, ident), ifExists) => + DropPaimonViewExec(viewCatalog, ident, ifExists) :: Nil + + // A new member was added to ResolvedNamespace since spark4.0, + // unapply pattern matching is not used here to ensure compatibility across multiple spark versions. + case ShowPaimonViews(r: ResolvedNamespace, pattern, output) + if r.catalog.isInstanceOf[SupportView] => + ShowPaimonViewsExec(output, r.catalog.asInstanceOf[SupportView], r.namespace, pattern) :: Nil + case _ => Nil } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonViewExec.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonViewExec.scala new file mode 100644 index 0000000000000..7a4b907c72f12 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonViewExec.scala @@ -0,0 +1,117 @@ +/* + * 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.paimon.spark.execution + +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.spark.leafnode.PaimonLeafV2CommandExec + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericInternalRow} +import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.types.UTF8String + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +case class CreatePaimonViewExec( + catalog: SupportView, + ident: Identifier, + queryText: String, + viewSchema: StructType, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String], + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean +) extends PaimonLeafV2CommandExec { + + override def output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + if (columnAliases.nonEmpty || columnComments.nonEmpty || queryColumnNames.nonEmpty) { + throw new UnsupportedOperationException( + "columnAliases, columnComments and queryColumnNames are not supported now") + } + + // Note: for replace just drop then create ,this operation is non-atomic. + if (replace) { + catalog.dropView(ident, true) + } + + catalog.createView( + ident, + viewSchema, + queryText, + comment.orNull, + properties.asJava, + allowExisting) + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"CreatePaimonViewExec: $ident" + } +} + +case class DropPaimonViewExec(catalog: SupportView, ident: Identifier, ifExists: Boolean) + extends PaimonLeafV2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.dropView(ident, ifExists) + Nil + } + + override def simpleString(maxFields: Int): String = { + s"DropPaimonViewExec: $ident" + } +} + +case class ShowPaimonViewsExec( + output: Seq[Attribute], + catalog: SupportView, + namespace: Seq[String], + pattern: Option[String]) + extends PaimonLeafV2CommandExec { + + override protected def run(): Seq[InternalRow] = { + val rows = new ArrayBuffer[InternalRow]() + catalog.listViews(namespace.toArray).asScala.map { + viewName => + if (pattern.forall(StringUtils.filterPattern(Seq(viewName), _).nonEmpty)) { + rows += new GenericInternalRow( + Array( + UTF8String.fromString(namespace.mkString(".")), + UTF8String.fromString(viewName), + false)) + } + } + rows.toSeq + } + + override def simpleString(maxFields: Int): String = { + s"ShowPaimonViewsExec: $namespace" + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala index 4fe217ee09bd8..6f47a77ef3080 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala @@ -18,7 +18,7 @@ package org.apache.paimon.spark.extensions -import org.apache.paimon.spark.catalyst.analysis.{PaimonAnalysis, PaimonDeleteTable, PaimonIncompatiblePHRRules, PaimonIncompatibleResolutionRules, PaimonMergeInto, PaimonPostHocResolutionRules, PaimonProcedureResolver, PaimonUpdateTable} +import org.apache.paimon.spark.catalyst.analysis.{PaimonAnalysis, PaimonDeleteTable, PaimonIncompatiblePHRRules, PaimonIncompatibleResolutionRules, PaimonMergeInto, PaimonPostHocResolutionRules, PaimonProcedureResolver, PaimonUpdateTable, PaimonViewResolver} import org.apache.paimon.spark.catalyst.optimizer.{EvalSubqueriesForDeleteTable, MergePaimonScalarSubqueries} import org.apache.paimon.spark.catalyst.plans.logical.PaimonTableValuedFunctions import org.apache.paimon.spark.execution.PaimonStrategy @@ -37,6 +37,7 @@ class PaimonSparkSessionExtensions extends (SparkSessionExtensions => Unit) { // analyzer extensions extensions.injectResolutionRule(spark => new PaimonAnalysis(spark)) extensions.injectResolutionRule(spark => PaimonProcedureResolver(spark)) + extensions.injectResolutionRule(spark => PaimonViewResolver(spark)) extensions.injectResolutionRule(spark => PaimonIncompatibleResolutionRules(spark)) extensions.injectPostHocResolutionRule(spark => PaimonPostHocResolutionRules(spark)) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/leafnode/package.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/leafnode/package.scala index 5befb88dae43f..6ebab038480a4 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/leafnode/package.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/leafnode/package.scala @@ -18,7 +18,7 @@ package org.apache.paimon.spark -import org.apache.spark.sql.catalyst.plans.logical.{LeafCommand, LeafParsedStatement} +import org.apache.spark.sql.catalyst.plans.logical.{BinaryCommand, LeafCommand, LeafParsedStatement, UnaryCommand} import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec @@ -30,6 +30,9 @@ package object leafnode { trait PaimonLeafCommand extends LeafCommand - trait PaimonLeafV2CommandExec extends LeafV2CommandExec + trait PaimonUnaryCommand extends UnaryCommand + + trait PaimonBinaryCommand extends BinaryCommand + trait PaimonLeafV2CommandExec extends LeafV2CommandExec } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala index dd0a48159b2ae..9ece186930d71 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala @@ -23,7 +23,7 @@ import org.antlr.v4.runtime.atn.PredictionMode import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} import org.antlr.v4.runtime.tree.TerminalNodeImpl import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface} @@ -61,7 +61,7 @@ class PaimonSparkSqlExtensionsParser(val delegate: ParserInterface) parse(sqlTextAfterSubstitution)(parser => astBuilder.visit(parser.singleStatement())) .asInstanceOf[LogicalPlan] } else { - delegate.parsePlan(sqlText) + RewritePaimonViewCommands(SparkSession.active).apply(delegate.parsePlan(sqlText)) } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSqlExtensionsAstBuilder.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSqlExtensionsAstBuilder.scala index b864894e7498c..a1289a5f0b50f 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSqlExtensionsAstBuilder.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSqlExtensionsAstBuilder.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.parser.extensions import org.apache.paimon.spark.catalyst.plans.logical -import org.apache.paimon.spark.catalyst.plans.logical.{CreateOrReplaceTagCommand, DeleteTagCommand, PaimonCallArgument, PaimonCallStatement, PaimonNamedArgument, PaimonPositionalArgument, RenameTagCommand, ShowTagsCommand, TagOptions} +import org.apache.paimon.spark.catalyst.plans.logical._ import org.apache.paimon.utils.TimeUtils import org.antlr.v4.runtime._ @@ -212,5 +212,16 @@ object CurrentOrigin { def get: Origin = value.get() def set(o: Origin): Unit = value.set(o) def reset(): Unit = value.set(Origin()) + + def withOrigin[A](o: Origin)(f: => A): A = { + // remember the previous one so it can be reset to this + // way withOrigin can be recursive + val previous = get + set(o) + val ret = + try f + finally { set(previous) } + ret + } } /* Apache Spark copy end */ diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala new file mode 100644 index 0000000000000..f69e5d92038ed --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala @@ -0,0 +1,77 @@ +/* + * 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.spark.sql.catalyst.parser.extensions + +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.spark.catalyst.plans.logical.{CreatePaimonView, DropPaimonView, ResolvedIdentifier, ShowPaimonViews} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{CTESubstitution, ResolvedNamespace, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{CatalogManager, LookupCatalog} + +case class RewritePaimonViewCommands(spark: SparkSession) + extends Rule[LogicalPlan] + with LookupCatalog { + + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + + // A new member was added to CreatePaimonView since spark4.0, + // unapply pattern matching is not used here to ensure compatibility across multiple spark versions. + case c: CreateView => + ResolvedIdent + .unapply(c.child) + .map { + resolved => + CreatePaimonView( + child = resolved, + queryText = c.originalText.get, + query = CTESubstitution.apply(c.query), + columnAliases = c.userSpecifiedColumns.map(_._1), + columnComments = c.userSpecifiedColumns.map(_._2.orElse(None)), + comment = c.comment, + properties = c.properties, + allowExisting = c.allowExisting, + replace = c.replace + ) + } + .getOrElse(c) + + case DropView(ResolvedIdent(resolved), ifExists: Boolean) => + DropPaimonView(resolved, ifExists) + + case ShowViews(_, pattern, output) if catalogManager.currentCatalog.isInstanceOf[SupportView] => + ShowPaimonViews( + ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, + output) + } + + private object ResolvedIdent { + def unapply(unresolved: Any): Option[ResolvedIdentifier] = unresolved match { + case UnresolvedIdentifier(CatalogAndIdentifier(viewCatalog: SupportView, ident), _) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case _ => + None + } + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 33b993160dfd7..56922ae2aeff8 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -241,12 +241,12 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { spark.sql(s"CREATE DATABASE paimon_db") spark.sql(s"USE paimon_db") spark.sql(s"CREATE TABLE paimon_tbl (id int, name string, dt string) using paimon") - // Currently, only spark_catalog supports create other table or view + // Only spark_catalog supports create other table if (catalogName.equals(sparkCatalogName)) { spark.sql(s"CREATE TABLE parquet_tbl (id int, name string, dt string) using parquet") spark.sql(s"CREATE VIEW parquet_tbl_view AS SELECT * FROM parquet_tbl") - spark.sql(s"CREATE VIEW paimon_tbl_view AS SELECT * FROM paimon_tbl") } + spark.sql(s"CREATE VIEW paimon_tbl_view AS SELECT * FROM paimon_tbl") spark.sql(s"USE default") spark.sql(s"DROP DATABASE paimon_db CASCADE") } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala new file mode 100644 index 0000000000000..39ed8e8a769d3 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala @@ -0,0 +1,96 @@ +/* + * 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.paimon.spark.sql + +import org.apache.paimon.spark.PaimonHiveTestBase + +import org.apache.spark.sql.Row + +abstract class PaimonViewTestBase extends PaimonHiveTestBase { + + test("Paimon View: create and drop view") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + { + sql(s"USE $catalogName") + withDatabase("test_db") { + sql("CREATE DATABASE test_db") + sql("USE test_db") + withTable("t") { + withView("v1") { + sql("CREATE TABLE t (id INT) USING paimon") + sql("INSERT INTO t VALUES (1), (2)") + + sql("CREATE VIEW v1 AS SELECT * FROM t") + checkAnswer(sql("SHOW VIEWS"), Seq(Row("test_db", "v1", false))) + checkAnswer(sql("SELECT * FROM v1"), Seq(Row(1), Row(2))) + checkAnswer( + sql("SELECT * FROM v1 WHERE id >= (SELECT max(id) FROM v1)"), + Seq(Row(2))) + + // test drop view + sql("DROP VIEW IF EXISTS v1") + checkAnswer(sql("SHOW VIEWS"), Seq()) + sql("CREATE VIEW v1 AS SELECT * FROM t WHERE id > 1") + checkAnswer(sql("SHOW VIEWS"), Seq(Row("test_db", "v1", false))) + checkAnswer(sql("SELECT * FROM v1"), Seq(Row(2))) + + // test create or replace view + intercept[Exception] { + sql("CREATE VIEW v1 AS SELECT * FROM t WHERE id < 2") + } + sql("CREATE OR REPLACE VIEW v1 AS SELECT * FROM t WHERE id < 2") + checkAnswer(sql("SELECT * FROM v1"), Seq(Row(1))) + } + } + } + } + } + } + + test("Paimon View: show views") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + { + sql(s"USE $catalogName") + withDatabase("test_db") { + sql("CREATE DATABASE test_db") + sql("USE test_db") + withTable("t") { + withView("va", "vab", "vc") { + sql("CREATE TABLE t (id INT) USING paimon") + sql("CREATE VIEW va AS SELECT * FROM t") + sql("CREATE VIEW vab AS SELECT * FROM t") + sql("CREATE VIEW vc AS SELECT * FROM t") + checkAnswer( + sql("SHOW VIEWS"), + Seq( + Row("test_db", "va", false), + Row("test_db", "vab", false), + Row("test_db", "vc", false))) + checkAnswer( + sql("SHOW VIEWS LIKE 'va*'"), + Seq(Row("test_db", "va", false), Row("test_db", "vab", false))) + } + } + } + } + } + } +} From f09d9c200a6f2d1913e7f1f318b164205d226ac0 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Tue, 19 Nov 2024 11:20:56 +0800 Subject: [PATCH 029/157] [doc] Move system tables to concepts --- docs/content/concepts/spec/_index.md | 2 +- .../system-tables.md | 87 ++++++++++--------- docs/content/concepts/table-types.md | 33 ++++++- docs/content/flink/savepoint.md | 2 +- .../primary-key-table/query-performance.md | 2 +- docs/content/primary-key-table/table-mode.md | 2 +- 6 files changed, 81 insertions(+), 47 deletions(-) rename docs/content/{maintenance => concepts}/system-tables.md (97%) diff --git a/docs/content/concepts/spec/_index.md b/docs/content/concepts/spec/_index.md index 166ce4eeaa549..ef5f03098e20d 100644 --- a/docs/content/concepts/spec/_index.md +++ b/docs/content/concepts/spec/_index.md @@ -1,7 +1,7 @@ --- title: Specification bookCollapseSection: true -weight: 6 +weight: 7 --- + + commons-codec + commons-codec + 1.9 + test + + org.apache.paimon paimon-common diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java deleted file mode 100644 index 723c71dc565d7..0000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java +++ /dev/null @@ -1,22 +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.flink.streaming.runtime.streamrecord; - -/** Placeholder class for new feature introduced since flink 1.19. Should never be used. */ -public class RecordAttributes extends StreamElement {} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java deleted file mode 100644 index 6eabd1db7f382..0000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java +++ /dev/null @@ -1,34 +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.flink.table.catalog; - -/** - * Dummy placeholder to resolve compatibility issue of CatalogMaterializedTable(introduced in flink - * 1.20). - */ -public interface CatalogMaterializedTable extends CatalogBaseTable { - /** Dummy LogicalRefreshMode placeholder. */ - enum LogicalRefreshMode {} - - /** Dummy RefreshMode placeholder. */ - enum RefreshMode {} - - /** Dummy RefreshStatus placeholder. */ - enum RefreshStatus {} -} From 253378850431ca97687847199e900d908e04e6df Mon Sep 17 00:00:00 2001 From: Kerwin <37063904+zhuangchong@users.noreply.github.com> Date: Mon, 25 Nov 2024 13:31:25 +0800 Subject: [PATCH 055/157] [hive] Improve paimon format table conversion hive table in hive catalog. (#4522) --- .../org/apache/paimon/table/FormatTable.java | 14 ++ .../org/apache/paimon/hive/HiveCatalog.java | 124 +++++++++--------- .../apache/paimon/hive/HiveCatalogTest.java | 4 +- 3 files changed, 76 insertions(+), 66 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FormatTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FormatTable.java index a53ba545c25e4..a4c7788c38afe 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FormatTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FormatTable.java @@ -34,6 +34,7 @@ import javax.annotation.Nullable; import java.time.Duration; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -70,6 +71,19 @@ enum Format { CSV } + /** Parses a file format string to a corresponding {@link Format} enum constant. */ + static Format parseFormat(String fileFormat) { + try { + return Format.valueOf(fileFormat.toUpperCase()); + } catch (IllegalArgumentException e) { + throw new UnsupportedOperationException( + "Format table unsupported file format: " + + fileFormat + + ". Supported formats: " + + Arrays.toString(Format.values())); + } + } + /** Create a new builder for {@link FormatTable}. */ static FormatTable.Builder builder() { return new FormatTable.Builder(); diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index ebd5a1edf89b2..0ecc78469e154 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -112,7 +112,6 @@ import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; import static org.apache.paimon.utils.HadoopUtils.addHadoopConfIfFound; import static org.apache.paimon.utils.Preconditions.checkArgument; -import static org.apache.paimon.utils.Preconditions.checkNotNull; import static org.apache.paimon.utils.StringUtils.isNullOrWhitespaceOnly; /** A catalog implementation for Hive. */ @@ -122,7 +121,7 @@ public class HiveCatalog extends AbstractCatalog { // Reserved properties public static final String TABLE_TYPE_PROP = "table_type"; - public static final String PAIMON_TABLE_TYPE_VALUE = "paimon"; + public static final String PAIMON_TABLE_IDENTIFIER = "PAIMON"; // we don't include paimon-hive-connector as dependencies because it depends on // hive-exec @@ -766,33 +765,24 @@ private Table createHiveTable( } } - Table table = - newHmsTable(identifier, tblProperties, PAIMON_TABLE_TYPE_VALUE, externalTable); - updateHmsTable(table, identifier, tableSchema, PAIMON_TABLE_TYPE_VALUE, location); + Table table = newHmsTable(identifier, tblProperties, null, externalTable); + updateHmsTable(table, identifier, tableSchema, null, location); return table; } private Table createHiveFormatTable( Identifier identifier, TableSchema tableSchema, Path location, boolean externalTable) { - Options options = Options.fromMap(tableSchema.options()); - checkArgument(options.get(TYPE) == FORMAT_TABLE); + CoreOptions coreOptions = new CoreOptions(tableSchema.options()); + checkArgument(coreOptions.type() == FORMAT_TABLE); - String provider = tableSchema.options().get(FILE_FORMAT.key()); - checkNotNull(provider, FILE_FORMAT.key() + " should be configured."); - // valid supported format - FormatTable.Format.valueOf(provider.toUpperCase()); + // file.format option has a default value and cannot be empty. + FormatTable.Format provider = FormatTable.parseFormat(coreOptions.formatType()); Map tblProperties = new HashMap<>(); Table table = newHmsTable(identifier, tblProperties, provider, externalTable); updateHmsTable(table, identifier, tableSchema, provider, location); - if (FormatTable.Format.CSV.toString().equalsIgnoreCase(provider)) { - table.getSd() - .getSerdeInfo() - .getParameters() - .put(FIELD_DELIM, options.get(FIELD_DELIMITER)); - } return table; } @@ -879,7 +869,8 @@ private void alterTableToHms(Table table, Identifier identifier, TableSchema new throws TException, InterruptedException { updateHmsTablePars(table, newSchema); Path location = getTableLocation(identifier, table); - updateHmsTable(table, identifier, newSchema, newSchema.options().get("provider"), location); + // file format is null, because only data table support alter table. + updateHmsTable(table, identifier, newSchema, null, location); clients.execute(client -> HiveAlterTableUtils.alterTable(client, identifier, table)); } @@ -1059,12 +1050,9 @@ private boolean isExternalTable(Table table) { private Table newHmsTable( Identifier identifier, Map tableParameters, - String provider, + @Nullable FormatTable.Format provider, boolean externalTable) { long currentTimeMillis = System.currentTimeMillis(); - if (provider == null) { - provider = PAIMON_TABLE_TYPE_VALUE; - } Table table = new Table( identifier.getTableName(), @@ -1082,67 +1070,83 @@ private Table newHmsTable( externalTable ? TableType.EXTERNAL_TABLE.name() : TableType.MANAGED_TABLE.name()); - table.getParameters().put(TABLE_TYPE_PROP, provider.toUpperCase()); - if (PAIMON_TABLE_TYPE_VALUE.equalsIgnoreCase(provider)) { + + if (provider == null) { + // normal paimon table + table.getParameters().put(TABLE_TYPE_PROP, PAIMON_TABLE_IDENTIFIER); table.getParameters() .put(hive_metastoreConstants.META_TABLE_STORAGE, STORAGE_HANDLER_CLASS_NAME); } else { - table.getParameters().put(FILE_FORMAT.key(), provider.toLowerCase()); + // format table + table.getParameters().put(TABLE_TYPE_PROP, provider.name()); + table.getParameters().put(FILE_FORMAT.key(), provider.name().toLowerCase()); table.getParameters().put(TYPE.key(), FORMAT_TABLE.toString()); } + if (externalTable) { table.getParameters().put(HIVE_EXTERNAL_TABLE_PROP, "TRUE"); } return table; } - private String getSerdeClassName(String provider) { - if (provider == null || provider.equalsIgnoreCase(PAIMON_TABLE_TYPE_VALUE)) { - return SERDE_CLASS_NAME; - } else if (provider.equalsIgnoreCase("csv")) { - return "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; - } else if (provider.equalsIgnoreCase("parquet")) { - return "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"; - } else if (provider.equalsIgnoreCase("orc")) { - return "org.apache.hadoop.hive.ql.io.orc.OrcSerde"; - } else { + private String getSerdeClassName(@Nullable FormatTable.Format provider) { + if (provider == null) { return SERDE_CLASS_NAME; } + switch (provider) { + case CSV: + return "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; + case PARQUET: + return "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"; + case ORC: + return "org.apache.hadoop.hive.ql.io.orc.OrcSerde"; + } + return SERDE_CLASS_NAME; } - private String getInputFormatName(String provider) { - if (provider == null || provider.equalsIgnoreCase(PAIMON_TABLE_TYPE_VALUE)) { - return INPUT_FORMAT_CLASS_NAME; - } else if (provider.equalsIgnoreCase("csv")) { - return "org.apache.hadoop.mapred.TextInputFormat"; - } else if (provider.equalsIgnoreCase("parquet")) { - return "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; - } else if (provider.equalsIgnoreCase("orc")) { - return "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; - } else { + private String getInputFormatName(@Nullable FormatTable.Format provider) { + if (provider == null) { return INPUT_FORMAT_CLASS_NAME; } + switch (provider) { + case CSV: + return "org.apache.hadoop.mapred.TextInputFormat"; + case PARQUET: + return "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; + case ORC: + return "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; + } + return INPUT_FORMAT_CLASS_NAME; } - private String getOutputFormatClassName(String provider) { - if (provider == null || provider.equalsIgnoreCase(PAIMON_TABLE_TYPE_VALUE)) { - return OUTPUT_FORMAT_CLASS_NAME; - } else if (provider.equalsIgnoreCase("csv")) { - return "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"; - } else if (provider.equalsIgnoreCase("parquet")) { - return "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"; - } else if (provider.equalsIgnoreCase("orc")) { - return "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"; - } else { + private String getOutputFormatClassName(@Nullable FormatTable.Format provider) { + if (provider == null) { return OUTPUT_FORMAT_CLASS_NAME; } + switch (provider) { + case CSV: + return "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"; + case PARQUET: + return "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"; + case ORC: + return "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"; + } + return OUTPUT_FORMAT_CLASS_NAME; + } + + private Map setSerDeInfoParam(@Nullable FormatTable.Format provider) { + Map param = new HashMap<>(); + if (provider == FormatTable.Format.CSV) { + param.put(FIELD_DELIM, options.get(FIELD_DELIMITER)); + } + return param; } private void updateHmsTable( Table table, Identifier identifier, TableSchema schema, - String provider, + @Nullable FormatTable.Format provider, Path location) { StorageDescriptor sd = table.getSd() != null ? table.getSd() : new StorageDescriptor(); @@ -1206,14 +1210,6 @@ private void updateHmsTable( locationHelper.specifyTableLocation(table, location.toString()); } - private Map setSerDeInfoParam(String provider) { - Map param = new HashMap<>(); - if (provider != null && provider.equalsIgnoreCase("csv")) { - param.put(FIELD_DELIM, options.get(FIELD_DELIMITER)); - } - return param; - } - private void updateHmsTablePars(Table table, TableSchema schema) { if (syncAllProperties()) { table.getParameters().putAll(schema.options()); diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java index 3ba3f89e412fc..267bdf0c71002 100644 --- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java +++ b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java @@ -55,7 +55,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTORECONNECTURLKEY; -import static org.apache.paimon.hive.HiveCatalog.PAIMON_TABLE_TYPE_VALUE; +import static org.apache.paimon.hive.HiveCatalog.PAIMON_TABLE_IDENTIFIER; import static org.apache.paimon.hive.HiveCatalog.TABLE_TYPE_PROP; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -218,7 +218,7 @@ public void testAddHiveTableParameters() { assertThat(tableProperties).containsEntry("comment", "this is a hive table"); assertThat(tableProperties) .containsEntry( - TABLE_TYPE_PROP, PAIMON_TABLE_TYPE_VALUE.toUpperCase(Locale.ROOT)); + TABLE_TYPE_PROP, PAIMON_TABLE_IDENTIFIER.toUpperCase(Locale.ROOT)); } catch (Exception e) { fail("Test failed due to exception: " + e.getMessage()); } From dc9849f87f098cddffa52889a95e29620ad3ecc3 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Mon, 25 Nov 2024 14:25:41 +0800 Subject: [PATCH 056/157] [doc] Move Catalog API together with Java API --- docs/content/program-api/catalog-api.md | 2 +- docs/content/program-api/python-api.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/program-api/catalog-api.md b/docs/content/program-api/catalog-api.md index d016cfa7b2049..570577437d865 100644 --- a/docs/content/program-api/catalog-api.md +++ b/docs/content/program-api/catalog-api.md @@ -1,6 +1,6 @@ --- title: "Catalog API" -weight: 4 +weight: 3 type: docs aliases: - /api/catalog-api.html diff --git a/docs/content/program-api/python-api.md b/docs/content/program-api/python-api.md index 940e8cb52b4d3..ff457cc55f446 100644 --- a/docs/content/program-api/python-api.md +++ b/docs/content/program-api/python-api.md @@ -1,6 +1,6 @@ --- title: "Python API" -weight: 3 +weight: 4 type: docs aliases: - /api/python-api.html From d9a1b80a41169c41eb2628790d8bc4e7fc68467c Mon Sep 17 00:00:00 2001 From: Jingsong Date: Mon, 25 Nov 2024 15:23:44 +0800 Subject: [PATCH 057/157] [doc] Document changelog producer to use lookup --- .../primary-key-table/changelog-producer.md | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/docs/content/primary-key-table/changelog-producer.md b/docs/content/primary-key-table/changelog-producer.md index bf7a23fae2a54..011f7b6f27a7f 100644 --- a/docs/content/primary-key-table/changelog-producer.md +++ b/docs/content/primary-key-table/changelog-producer.md @@ -58,9 +58,11 @@ By specifying `'changelog-producer' = 'input'`, Paimon writers rely on their inp ## Lookup -If your input can’t produce a complete changelog but you still want to get rid of the costly normalized operator, you may consider using the `'lookup'` changelog producer. +If your input can’t produce a complete changelog but you still want to get rid of the costly normalized operator, you +may consider using the `'lookup'` changelog producer. -By specifying `'changelog-producer' = 'lookup'`, Paimon will generate changelog through `'lookup'` before committing the data writing. +By specifying `'changelog-producer' = 'lookup'`, Paimon will generate changelog through `'lookup'` before committing +the data writing (You can also enable [Async Compaction]({{< ref "primary-key-table/compaction#asynchronous-compaction" >}})). {{< img src="/img/changelog-producer-lookup.png">}} @@ -105,23 +107,26 @@ important for performance). ## Full Compaction -If you think the resource consumption of 'lookup' is too large, you can consider using 'full-compaction' changelog producer, -which can decouple data writing and changelog generation, and is more suitable for scenarios with high latency (For example, 10 minutes). +You can also consider using 'full-compaction' changelog producer to generate changelog, and is more suitable for scenarios +with large latency (For example, 30 minutes). -By specifying `'changelog-producer' = 'full-compaction'`, Paimon will compare the results between full compactions and produce the differences as changelog. The latency of changelog is affected by the frequency of full compactions. +1. By specifying `'changelog-producer' = 'full-compaction'`, Paimon will compare the results between full compactions and +produce the differences as changelog. The latency of changelog is affected by the frequency of full compactions. +2. By specifying `full-compaction.delta-commits` table property, full compaction will be constantly triggered after delta +commits (checkpoints). This is set to 1 by default, so each checkpoint will have a full compression and generate a +changelog. -By specifying `full-compaction.delta-commits` table property, full compaction will be constantly triggered after delta commits (checkpoints). This is set to 1 by default, so each checkpoint will have a full compression and generate a change log. +Generally speaking, the cost and consumption of full compaction are high, so we recommend using `'lookup'` changelog +producer. {{< img src="/img/changelog-producer-full-compaction.png">}} {{< hint info >}} -Full compaction changelog producer can produce complete changelog for any type of source. However it is not as efficient as the input changelog producer and the latency to produce changelog might be high. +Full compaction changelog producer can produce complete changelog for any type of source. However it is not as +efficient as the input changelog producer and the latency to produce changelog might be high. {{< /hint >}} Full-compaction changelog-producer supports `changelog-producer.row-deduplicate` to avoid generating -U, +U changelog for the same record. - -(Note: Please increase `'execution.checkpointing.max-concurrent-checkpoints'` Flink configuration, this is very -important for performance). From 76a4134838058559fbba2f6a022d7aa3efcd45b0 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 25 Nov 2024 20:10:38 +0800 Subject: [PATCH 058/157] [flink] Adopt open(OpenContext) in RichFunction (#4581) --- .../api/common/functions/OpenContext.java | 29 +++++++++++++++++++ .../api/common/functions/OpenContext.java | 29 +++++++++++++++++++ .../api/common/functions/OpenContext.java | 29 +++++++++++++++++++ .../api/common/functions/OpenContext.java | 29 +++++++++++++++++++ .../paimon/flink/kafka/KafkaSinkFunction.java | 12 +++++++- ...CdcDynamicTableParsingProcessFunction.java | 12 +++++++- .../CdcMultiTableParsingProcessFunction.java | 12 +++++++- .../sink/cdc/CdcParsingProcessFunction.java | 12 +++++++- .../UpdatedDataFieldsProcessFunctionBase.java | 12 +++++++- .../flink/service/QueryAddressRegister.java | 12 +++++++- .../flink/service/QueryFileMonitor.java | 12 +++++++- .../paimon/flink/shuffle/RangeShuffle.java | 15 ++++++++-- .../flink/sink/RowDataStoreWriteOperator.java | 28 ++++++++++++++---- .../apache/paimon/flink/sorter/SortUtils.java | 28 ++++++++++++++++-- .../source/BucketUnawareCompactSource.java | 12 +++++++- .../CombinedAwareBatchSourceFunction.java | 12 +++++++- .../CombinedAwareStreamingSourceFunction.java | 12 +++++++- .../CombinedCompactorSourceFunction.java | 12 +++++++- .../CombinedUnawareBatchSourceFunction.java | 12 +++++++- ...ombinedUnawareStreamingSourceFunction.java | 12 +++++++- 20 files changed, 319 insertions(+), 24 deletions(-) create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java new file mode 100644 index 0000000000000..4ff5484b3b087 --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java @@ -0,0 +1,29 @@ +/* + * 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.flink.api.common.functions; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * The {@link OpenContext} interface provides necessary information required by the {@link + * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be + * used to add more methods without affecting the signature of {@code RichFunction#open}. + */ +@PublicEvolving +public interface OpenContext {} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java new file mode 100644 index 0000000000000..4ff5484b3b087 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java @@ -0,0 +1,29 @@ +/* + * 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.flink.api.common.functions; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * The {@link OpenContext} interface provides necessary information required by the {@link + * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be + * used to add more methods without affecting the signature of {@code RichFunction#open}. + */ +@PublicEvolving +public interface OpenContext {} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java new file mode 100644 index 0000000000000..4ff5484b3b087 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java @@ -0,0 +1,29 @@ +/* + * 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.flink.api.common.functions; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * The {@link OpenContext} interface provides necessary information required by the {@link + * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be + * used to add more methods without affecting the signature of {@code RichFunction#open}. + */ +@PublicEvolving +public interface OpenContext {} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java new file mode 100644 index 0000000000000..4ff5484b3b087 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java @@ -0,0 +1,29 @@ +/* + * 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.flink.api.common.functions; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * The {@link OpenContext} interface provides necessary information required by the {@link + * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be + * used to add more methods without affecting the signature of {@code RichFunction#open}. + */ +@PublicEvolving +public interface OpenContext {} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java index 72a177adceaf3..41e7141cf48a8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java @@ -21,6 +21,7 @@ import org.apache.paimon.flink.sink.LogSinkFunction; import org.apache.paimon.table.sink.SinkRecord; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaException; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; @@ -65,7 +66,16 @@ public void setWriteCallback(WriteCallback writeCallback) { this.writeCallback = writeCallback; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration configuration) throws Exception { super.open(configuration); Callback baseCallback = requireNonNull(callback); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java index 0961ff160048d..886e33e2046aa 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.types.DataField; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -74,7 +75,16 @@ public CdcDynamicTableParsingProcessFunction( this.parserFactory = parserFactory; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { parser = parserFactory.create(); catalog = catalogLoader.load(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java index b18a05c280cbf..4c5e0600bb47e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java @@ -20,6 +20,7 @@ import org.apache.paimon.types.DataField; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.configuration.Configuration; @@ -51,7 +52,16 @@ public CdcMultiTableParsingProcessFunction(EventParser.Factory parserFactory) this.parserFactory = parserFactory; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { parser = parserFactory.create(); updatedDataFieldsOutputTags = new HashMap<>(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java index 3456634942c8d..eec228f3c09bb 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java @@ -20,6 +20,7 @@ import org.apache.paimon.types.DataField; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -50,7 +51,16 @@ public CdcParsingProcessFunction(EventParser.Factory parserFactory) { this.parserFactory = parserFactory; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { parser = parserFactory.create(); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java index c2e928bd4a0aa..4f02b784c2ba6 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java @@ -31,6 +31,7 @@ import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.StringUtils; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.slf4j.Logger; @@ -73,7 +74,16 @@ protected UpdatedDataFieldsProcessFunctionBase(Catalog.Loader catalogLoader) { this.catalogLoader = catalogLoader; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) { this.catalog = catalogLoader.load(); this.allowUpperCase = this.catalog.allowUpperCase(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java index df3cf7abf2a5f..524f2e5f01c17 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java @@ -23,6 +23,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.sink.SinkFunction; @@ -44,7 +45,16 @@ public QueryAddressRegister(Table table) { this.serviceManager = ((FileStoreTable) table).store().newServiceManager(); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { this.executors = new TreeMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java index 43cf654e91fe6..02f8a654112ea 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java @@ -31,6 +31,7 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.table.system.FileMonitorTable; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -70,7 +71,16 @@ public QueryFileMonitor(Table table) { .toMillis(); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); ReadBuilder readBuilder = monitorTable.newReadBuilder(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java index 54104130438bf..8760f1dc5f804 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java @@ -27,6 +27,7 @@ import org.apache.paimon.utils.SerializableSupplier; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; @@ -182,9 +183,19 @@ public KeyAndSizeExtractor(RowType rowType, boolean isSortBySize) { this.isSortBySize = isSortBySize; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink + * 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink + * 2.0+. + */ public void open(Configuration parameters) throws Exception { - super.open(parameters); InternalRowToSizeVisitor internalRowToSizeVisitor = new InternalRowToSizeVisitor(); fieldSizeCalculator = rowType.getFieldTypes().stream() diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java index 07fe275543a18..2b25f074667c3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java @@ -23,6 +23,8 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.SinkRecord; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.state.CheckpointListener; @@ -42,6 +44,8 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.List; import java.util.Objects; @@ -97,17 +101,29 @@ public void open() throws Exception { this.sinkContext = new SimpleContext(getProcessingTimeService()); if (logSinkFunction != null) { - // to stay compatible with Flink 1.18- - if (logSinkFunction instanceof RichFunction) { - RichFunction richFunction = (RichFunction) logSinkFunction; - richFunction.open(new Configuration()); - } - + openFunction(logSinkFunction); logCallback = new LogWriteCallback(); logSinkFunction.setWriteCallback(logCallback); } } + private static void openFunction(Function function) throws Exception { + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + + try { + Method method = RichFunction.class.getDeclaredMethod("open", OpenContext.class); + method.invoke(richFunction, new OpenContext() {}); + return; + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + // to stay compatible with Flink 1.18- + } + + Method method = RichFunction.class.getDeclaredMethod("open", Configuration.class); + method.invoke(richFunction, new Configuration()); + } + } + @Override public void processWatermark(Watermark mark) throws Exception { super.processWatermark(mark); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java index f590c2fb7fff0..b30e145512966 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java @@ -31,6 +31,7 @@ import org.apache.paimon.utils.KeyProjectedRow; import org.apache.paimon.utils.SerializableSupplier; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -119,9 +120,19 @@ public static DataStream sortStreamByKey( .map( new RichMapFunction>() { - @Override + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { - super.open(parameters); shuffleKeyAbstract.open(); } @@ -172,7 +183,18 @@ public Tuple2 map(RowData value) { private transient KeyProjectedRow keyProjectedRow; - @Override + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 2.0+. + */ public void open(Configuration parameters) { keyProjectedRow = new KeyProjectedRow(valueProjectionMap); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java index d306c7d8e1e5f..e768c717ddaa1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java @@ -26,6 +26,7 @@ import org.apache.paimon.table.source.EndOfScanException; import org.apache.paimon.utils.Preconditions; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStreamSource; @@ -72,7 +73,16 @@ public BucketUnawareCompactSource( this.filter = filter; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { compactionCoordinator = new UnawareAppendTableCompactionCoordinator(table, streaming, filter); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java index cee6081aa29f1..2157be51aee4d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java @@ -25,6 +25,7 @@ import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -62,7 +63,16 @@ public CombinedAwareBatchSourceFunction( super(catalogLoader, includingPattern, excludingPattern, databasePattern, false); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { super.open(parameters); tableScan = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java index bff690ea30c23..01e0127e9fda7 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java @@ -25,6 +25,7 @@ import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -59,7 +60,16 @@ public CombinedAwareStreamingSourceFunction( this.monitorInterval = monitorInterval; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { super.open(parameters); tableScan = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java index 1964927b5cddc..02bb8786505db 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.table.source.Split; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; @@ -70,7 +71,16 @@ public CombinedCompactorSourceFunction( this.isStreaming = isStreaming; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { isRunning = new AtomicBoolean(true); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java index 8ec8d5f2c1a2e..6a40f10ada615 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java @@ -29,6 +29,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; @@ -69,7 +70,16 @@ public CombinedUnawareBatchSourceFunction( super(catalogLoader, includingPattern, excludingPattern, databasePattern, false); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { super.open(parameters); tableScan = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java index e398e09a84518..b64518a7ef60a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java @@ -24,6 +24,7 @@ import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; @@ -55,7 +56,16 @@ public CombinedUnawareStreamingSourceFunction( this.monitorInterval = monitorInterval; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { super.open(parameters); tableScan = From ca4af649cf27f4d280c005cc77bee9c860d50bb1 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 25 Nov 2024 20:11:46 +0800 Subject: [PATCH 059/157] [flink] Adopt getTaskInfo() when acquiring parallelism info (#4583) --- .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../flink/service/QueryExecutorOperator.java | 10 +++--- .../paimon/flink/sink/CommitterOperator.java | 5 ++- .../sink/HashBucketAssignerOperator.java | 5 +-- .../sink/MultiTablesStoreCompactOperator.java | 7 ++-- .../flink/sink/StoreCompactOperator.java | 7 ++-- .../paimon/flink/sink/TableWriteOperator.java | 5 +-- .../index/GlobalIndexAssignerOperator.java | 5 +-- .../sink/index/IndexBootstrapOperator.java | 5 +-- .../paimon/flink/sorter/SortOperator.java | 4 ++- .../AppendBypassCoordinateOperator.java | 3 +- .../source/BucketUnawareCompactSource.java | 3 +- .../flink/utils/RuntimeContextUtils.java | 32 +++++++++++++++++++ .../UnawareBucketAppendOnlyTableITCase.java | 3 +- 17 files changed, 201 insertions(+), 21 deletions(-) create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..460fea55ad7a3 --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java @@ -0,0 +1,32 @@ +/* + * 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.paimon.flink.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..460fea55ad7a3 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java @@ -0,0 +1,32 @@ +/* + * 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.paimon.flink.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..460fea55ad7a3 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java @@ -0,0 +1,32 @@ +/* + * 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.paimon.flink.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..460fea55ad7a3 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java @@ -0,0 +1,32 @@ +/* + * 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.paimon.flink.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java index 556c308396884..bf0521d55049c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.service.network.NetworkUtils; @@ -77,8 +78,8 @@ public void initializeState(StateInitializationContext context) throws Exception this.query = ((FileStoreTable) table).newLocalTableQuery().withIOManager(ioManager); KvQueryServer server = new KvQueryServer( - getRuntimeContext().getIndexOfThisSubtask(), - getRuntimeContext().getNumberOfParallelSubtasks(), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()), NetworkUtils.findHostAddress(), Collections.singletonList(0).iterator(), 1, @@ -96,8 +97,9 @@ public void initializeState(StateInitializationContext context) throws Exception this.output.collect( new StreamRecord<>( GenericRow.of( - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask(), + RuntimeContextUtils.getNumberOfParallelSubtasks( + getRuntimeContext()), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), BinaryString.fromString(address.getHostName()), address.getPort()))); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java index 2ec90b8c6c40c..021a5db413d54 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.utils.Preconditions; import org.apache.flink.runtime.state.StateInitializationContext; @@ -129,7 +130,9 @@ public void initializeState(StateInitializationContext context) throws Exception super.initializeState(context); Preconditions.checkArgument( - !forceSingleParallelism || getRuntimeContext().getNumberOfParallelSubtasks() == 1, + !forceSingleParallelism + || RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) + == 1, "Committer Operator parallelism in paimon MUST be one."); this.currentWatermark = Long.MIN_VALUE; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java index 70fac7a83e93c..0c101c6d1e01f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.flink.ProcessRecordAttributesUtil; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.index.BucketAssigner; import org.apache.paimon.index.HashBucketAssigner; import org.apache.paimon.index.SimpleHashBucketAssigner; @@ -76,8 +77,8 @@ public void initializeState(StateInitializationContext context) throws Exception StateUtils.getSingleValueFromState( context, "commit_user_state", String.class, initialCommitUser); - int numberTasks = getRuntimeContext().getNumberOfParallelSubtasks(); - int taskId = getRuntimeContext().getIndexOfThisSubtask(); + int numberTasks = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); + int taskId = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); long targetRowNum = table.coreOptions().dynamicBucketTargetRowNum(); this.assigner = overwrite diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java index 7cb5d30c2f8ed..8a1d3a02df811 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.options.Options; @@ -109,8 +110,10 @@ public void initializeState(StateInitializationContext context) throws Exception ChannelComputer.select( partition, bucket, - getRuntimeContext().getNumberOfParallelSubtasks()) - == getRuntimeContext().getIndexOfThisSubtask()); + RuntimeContextUtils.getNumberOfParallelSubtasks( + getRuntimeContext())) + == RuntimeContextUtils.getIndexOfThisSubtask( + getRuntimeContext())); tables = new HashMap<>(); writes = new HashMap<>(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java index 9b152a81ca225..ac10345bc4257 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.options.Options; @@ -92,8 +93,10 @@ public void initializeState(StateInitializationContext context) throws Exception ChannelComputer.select( partition, bucket, - getRuntimeContext().getNumberOfParallelSubtasks()) - == getRuntimeContext().getIndexOfThisSubtask()); + RuntimeContextUtils.getNumberOfParallelSubtasks( + getRuntimeContext())) + == RuntimeContextUtils.getIndexOfThisSubtask( + getRuntimeContext())); write = storeSinkWriteProvider.provide( table, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java index 67b4720e29645..32fcdd03bdfd5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java @@ -21,6 +21,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.flink.ProcessRecordAttributesUtil; import org.apache.paimon.flink.sink.StoreSinkWriteState.StateValueFilter; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.ChannelComputer; @@ -58,14 +59,14 @@ public void initializeState(StateInitializationContext context) throws Exception super.initializeState(context); boolean containLogSystem = containLogSystem(); - int numTasks = getRuntimeContext().getNumberOfParallelSubtasks(); + int numTasks = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); StateValueFilter stateFilter = (tableName, partition, bucket) -> { int task = containLogSystem ? ChannelComputer.select(bucket, numTasks) : ChannelComputer.select(partition, bucket, numTasks); - return task == getRuntimeContext().getIndexOfThisSubtask(); + return task == RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); }; state = createState(context, stateFilter); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java index 7fee3f45f3db7..99cce07fdc574 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java @@ -22,6 +22,7 @@ import org.apache.paimon.crosspartition.KeyPartOrRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.table.Table; import org.apache.flink.api.java.tuple.Tuple2; @@ -59,8 +60,8 @@ public void initializeState(StateInitializationContext context) throws Exception assigner.open( computeManagedMemory(this), ioManager, - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask(), + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), this::collect); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java index 501e35dff46cb..5c8ba8f9441f3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java @@ -21,6 +21,7 @@ import org.apache.paimon.crosspartition.IndexBootstrap; import org.apache.paimon.crosspartition.KeyPartOrRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.utils.SerializableFunction; import org.apache.flink.api.java.tuple.Tuple2; @@ -50,8 +51,8 @@ public IndexBootstrapOperator( public void initializeState(StateInitializationContext context) throws Exception { super.initializeState(context); bootstrap.bootstrap( - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask(), + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), this::collect); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java index d4d5dd7416816..b6847125fbc66 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.options.MemorySize; import org.apache.paimon.sort.BinaryExternalSortBuffer; import org.apache.paimon.types.RowType; @@ -79,7 +80,8 @@ public SortOperator( public void open() throws Exception { super.open(); initBuffer(); - if (sinkParallelism != getRuntimeContext().getNumberOfParallelSubtasks()) { + if (sinkParallelism + != RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext())) { throw new IllegalArgumentException( "Please ensure that the runtime parallelism of the sink matches the initial configuration " + "to avoid potential issues with skewed range partitioning."); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java index 668aa24c145d7..45090f7b68b40 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java @@ -20,6 +20,7 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.ExecutorUtils; @@ -67,7 +68,7 @@ public AppendBypassCoordinateOperator( public void open() throws Exception { super.open(); checkArgument( - getRuntimeContext().getNumberOfParallelSubtasks() == 1, + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) == 1, "Compaction Coordinator parallelism in paimon MUST be one."); long intervalMs = table.coreOptions().continuousDiscoveryInterval().toMillis(); this.compactTasks = new LinkedBlockingQueue<>(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java index e768c717ddaa1..79ee827fe6e41 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java @@ -21,6 +21,7 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; import org.apache.paimon.flink.sink.CompactionTaskTypeInfo; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.EndOfScanException; @@ -87,7 +88,7 @@ public void open(Configuration parameters) throws Exception { compactionCoordinator = new UnawareAppendTableCompactionCoordinator(table, streaming, filter); Preconditions.checkArgument( - this.getRuntimeContext().getNumberOfParallelSubtasks() == 1, + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) == 1, "Compaction Operator parallelism in paimon MUST be one."); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..34e0d041b6a0c --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java @@ -0,0 +1,32 @@ +/* + * 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.paimon.flink.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getTaskInfo().getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getTaskInfo().getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java index cb323542d4c11..f6dfb1b23046b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.reader.RecordReader; @@ -395,7 +396,7 @@ private TestStatelessWriterSource(FileStoreTable table) { @Override public void run(SourceContext sourceContext) throws Exception { - int taskId = getRuntimeContext().getIndexOfThisSubtask(); + int taskId = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); // wait some time in parallelism #2, // so that it does not commit in the same checkpoint with parallelism #1 int waitCount = (taskId == 0 ? 0 : 10); From 8d57d3d90a945c8705d3acda3e7bf0ef2cb157ef Mon Sep 17 00:00:00 2001 From: wangwj Date: Tue, 26 Nov 2024 13:02:01 +0800 Subject: [PATCH 060/157] [core] add deletedFileTotalSizeInBytes in result of OrphanFilesClean (#4545) --- .../operation/CleanOrphanFilesResult.java | 54 ++++++++++ .../operation/LocalOrphanFilesClean.java | 64 +++++++----- .../paimon/operation/OrphanFilesClean.java | 38 +++++-- .../apache/paimon/utils/SnapshotManager.java | 10 +- .../operation/LocalOrphanFilesCleanTest.java | 18 ++-- .../procedure/RemoveOrphanFilesProcedure.java | 13 ++- .../flink/RemoveOrphanFilesActionITCase.java | 2 +- .../flink/orphan/FlinkOrphanFilesClean.java | 98 +++++++++++++------ .../procedure/RemoveOrphanFilesProcedure.java | 12 ++- .../RemoveOrphanFilesActionITCaseBase.java | 2 +- .../procedure/RemoveOrphanFilesProcedure.java | 18 +++- .../spark/orphan/SparkOrphanFilesClean.scala | 64 +++++++----- .../RemoveOrphanFilesProcedureTest.scala | 30 +++--- 13 files changed, 291 insertions(+), 132 deletions(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java b/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java new file mode 100644 index 0000000000000..5a3bc67f9c95e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java @@ -0,0 +1,54 @@ +/* + * 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.paimon.operation; + +import org.apache.paimon.fs.Path; + +import java.util.List; + +/** The result of OrphanFilesClean. */ +public class CleanOrphanFilesResult { + + private List deletedFilesPath; + private final long deletedFileCount; + private final long deletedFileTotalLenInBytes; + + public CleanOrphanFilesResult(long deletedFileCount, long deletedFileTotalLenInBytes) { + this.deletedFileCount = deletedFileCount; + this.deletedFileTotalLenInBytes = deletedFileTotalLenInBytes; + } + + public CleanOrphanFilesResult( + List deletedFilesPath, long deletedFileCount, long deletedFileTotalLenInBytes) { + this(deletedFileCount, deletedFileTotalLenInBytes); + this.deletedFilesPath = deletedFilesPath; + } + + public long getDeletedFileCount() { + return deletedFileCount; + } + + public long getDeletedFileTotalLenInBytes() { + return deletedFileTotalLenInBytes; + } + + public List getDeletedFilesPath() { + return deletedFilesPath; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java b/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java index a5eea6d650cf9..511c5fc7fb79b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java @@ -21,12 +21,12 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.SerializableConsumer; import javax.annotation.Nullable; @@ -47,6 +47,7 @@ import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; @@ -68,6 +69,8 @@ public class LocalOrphanFilesClean extends OrphanFilesClean { private final List deleteFiles; + private final AtomicLong deletedFilesLenInBytes = new AtomicLong(0); + private Set candidateDeletes; public LocalOrphanFilesClean(FileStoreTable table) { @@ -87,16 +90,18 @@ public LocalOrphanFilesClean( table.coreOptions().deleteFileThreadNum(), "ORPHAN_FILES_CLEAN"); } - public List clean() throws IOException, ExecutionException, InterruptedException { + public CleanOrphanFilesResult clean() + throws IOException, ExecutionException, InterruptedException { List branches = validBranches(); // specially handle to clear snapshot dir - cleanSnapshotDir(branches, deleteFiles::add); + cleanSnapshotDir(branches, deleteFiles::add, deletedFilesLenInBytes::addAndGet); // delete candidate files - Map candidates = getCandidateDeletingFiles(); + Map> candidates = getCandidateDeletingFiles(); if (candidates.isEmpty()) { - return deleteFiles; + return new CleanOrphanFilesResult( + deleteFiles, deleteFiles.size(), deletedFilesLenInBytes.get()); } candidateDeletes = new HashSet<>(candidates.keySet()); @@ -108,12 +113,22 @@ public List clean() throws IOException, ExecutionException, InterruptedExc // delete unused files candidateDeletes.removeAll(usedFiles); - candidateDeletes.stream().map(candidates::get).forEach(fileCleaner); + candidateDeletes.stream() + .map(candidates::get) + .forEach( + deleteFileInfo -> { + deletedFilesLenInBytes.addAndGet(deleteFileInfo.getRight()); + fileCleaner.accept(deleteFileInfo.getLeft()); + }); deleteFiles.addAll( - candidateDeletes.stream().map(candidates::get).collect(Collectors.toList())); + candidateDeletes.stream() + .map(candidates::get) + .map(Pair::getLeft) + .collect(Collectors.toList())); candidateDeletes.clear(); - return deleteFiles; + return new CleanOrphanFilesResult( + deleteFiles, deleteFiles.size(), deletedFilesLenInBytes.get()); } private void collectWithoutDataFile( @@ -172,19 +187,20 @@ private Set getUsedFiles(String branch) { * Get all the candidate deleting files in the specified directories and filter them by * olderThanMillis. */ - private Map getCandidateDeletingFiles() { + private Map> getCandidateDeletingFiles() { List fileDirs = listPaimonFileDirs(); - Function> processor = + Function>> processor = path -> tryBestListingDirs(path).stream() .filter(this::oldEnough) - .map(FileStatus::getPath) + .map(status -> Pair.of(status.getPath(), status.getLen())) .collect(Collectors.toList()); - Iterator allPaths = randomlyExecuteSequentialReturn(executor, processor, fileDirs); - Map result = new HashMap<>(); - while (allPaths.hasNext()) { - Path next = allPaths.next(); - result.put(next.getName(), next); + Iterator> allFilesInfo = + randomlyExecuteSequentialReturn(executor, processor, fileDirs); + Map> result = new HashMap<>(); + while (allFilesInfo.hasNext()) { + Pair fileInfo = allFilesInfo.next(); + result.put(fileInfo.getLeft().getName(), fileInfo); } return result; } @@ -197,7 +213,6 @@ public static List createOrphanFilesCleans( SerializableConsumer fileCleaner, @Nullable Integer parallelism) throws Catalog.DatabaseNotExistException, Catalog.TableNotExistException { - List orphanFilesCleans = new ArrayList<>(); List tableNames = Collections.singletonList(tableName); if (tableName == null || "*".equals(tableName)) { tableNames = catalog.listTables(databaseName); @@ -214,6 +229,7 @@ public static List createOrphanFilesCleans( } }; + List orphanFilesCleans = new ArrayList<>(tableNames.size()); for (String t : tableNames) { Identifier identifier = new Identifier(databaseName, t); Table table = catalog.getTable(identifier).copy(dynamicOptions); @@ -230,7 +246,7 @@ public static List createOrphanFilesCleans( return orphanFilesCleans; } - public static long executeDatabaseOrphanFiles( + public static CleanOrphanFilesResult executeDatabaseOrphanFiles( Catalog catalog, String databaseName, @Nullable String tableName, @@ -249,15 +265,17 @@ public static long executeDatabaseOrphanFiles( ExecutorService executorService = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); - List>> tasks = new ArrayList<>(); + List> tasks = new ArrayList<>(tableCleans.size()); for (LocalOrphanFilesClean clean : tableCleans) { tasks.add(executorService.submit(clean::clean)); } - List cleanOrphanFiles = new ArrayList<>(); - for (Future> task : tasks) { + long deletedFileCount = 0; + long deletedFileTotalLenInBytes = 0; + for (Future task : tasks) { try { - cleanOrphanFiles.addAll(task.get()); + deletedFileCount += task.get().getDeletedFileCount(); + deletedFileTotalLenInBytes += task.get().getDeletedFileTotalLenInBytes(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); @@ -267,6 +285,6 @@ public static long executeDatabaseOrphanFiles( } executorService.shutdownNow(); - return cleanOrphanFiles.size(); + return new CleanOrphanFilesResult(deletedFileCount, deletedFileTotalLenInBytes); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java index 869100d9cfb8d..274cdd52fe140 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java @@ -120,23 +120,47 @@ protected List validBranches() { return branches; } - protected void cleanSnapshotDir(List branches, Consumer deletedFileConsumer) { + protected void cleanSnapshotDir( + List branches, + Consumer deletedFilesConsumer, + Consumer deletedFilesLenInBytesConsumer) { for (String branch : branches) { FileStoreTable branchTable = table.switchToBranch(branch); SnapshotManager snapshotManager = branchTable.snapshotManager(); // specially handle the snapshot directory - List nonSnapshotFiles = snapshotManager.tryGetNonSnapshotFiles(this::oldEnough); - nonSnapshotFiles.forEach(fileCleaner); - nonSnapshotFiles.forEach(deletedFileConsumer); + List> nonSnapshotFiles = + snapshotManager.tryGetNonSnapshotFiles(this::oldEnough); + nonSnapshotFiles.forEach( + nonSnapshotFile -> + cleanFile( + nonSnapshotFile, + deletedFilesConsumer, + deletedFilesLenInBytesConsumer)); // specially handle the changelog directory - List nonChangelogFiles = snapshotManager.tryGetNonChangelogFiles(this::oldEnough); - nonChangelogFiles.forEach(fileCleaner); - nonChangelogFiles.forEach(deletedFileConsumer); + List> nonChangelogFiles = + snapshotManager.tryGetNonChangelogFiles(this::oldEnough); + nonChangelogFiles.forEach( + nonChangelogFile -> + cleanFile( + nonChangelogFile, + deletedFilesConsumer, + deletedFilesLenInBytesConsumer)); } } + private void cleanFile( + Pair deleteFileInfo, + Consumer deletedFilesConsumer, + Consumer deletedFilesLenInBytesConsumer) { + Path filePath = deleteFileInfo.getLeft(); + Long fileSize = deleteFileInfo.getRight(); + deletedFilesConsumer.accept(filePath); + deletedFilesLenInBytesConsumer.accept(fileSize); + fileCleaner.accept(filePath); + } + protected Set safelyGetAllSnapshots(String branch) throws IOException { FileStoreTable branchTable = table.switchToBranch(branch); SnapshotManager snapshotManager = branchTable.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java index 9a120042eaaad..cbe33ffaf4569 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java @@ -563,15 +563,15 @@ private void collectSnapshots(Consumer pathConsumer, List paths) * Try to get non snapshot files. If any error occurred, just ignore it and return an empty * result. */ - public List tryGetNonSnapshotFiles(Predicate fileStatusFilter) { + public List> tryGetNonSnapshotFiles(Predicate fileStatusFilter) { return listPathWithFilter(snapshotDirectory(), fileStatusFilter, nonSnapshotFileFilter()); } - public List tryGetNonChangelogFiles(Predicate fileStatusFilter) { + public List> tryGetNonChangelogFiles(Predicate fileStatusFilter) { return listPathWithFilter(changelogDirectory(), fileStatusFilter, nonChangelogFileFilter()); } - private List listPathWithFilter( + private List> listPathWithFilter( Path directory, Predicate fileStatusFilter, Predicate fileFilter) { try { FileStatus[] statuses = fileIO.listStatus(directory); @@ -581,8 +581,8 @@ private List listPathWithFilter( return Arrays.stream(statuses) .filter(fileStatusFilter) - .map(FileStatus::getPath) - .filter(fileFilter) + .filter(status -> fileFilter.test(status.getPath())) + .map(status -> Pair.of(status.getPath(), status.getLen())) .collect(Collectors.toList()); } catch (IOException ignored) { return Collections.emptyList(); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java index fdc68b34abb44..5139dd44957d7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java @@ -165,22 +165,20 @@ public void testNormallyRemoving() throws Throwable { // randomly delete tags List deleteTags = Collections.emptyList(); - if (!allTags.isEmpty()) { - deleteTags = randomlyPick(allTags); - for (String tagName : deleteTags) { - table.deleteTag(tagName); - } + deleteTags = randomlyPick(allTags); + for (String tagName : deleteTags) { + table.deleteTag(tagName); } // first check, nothing will be deleted because the default olderThan interval is 1 day LocalOrphanFilesClean orphanFilesClean = new LocalOrphanFilesClean(table); - assertThat(orphanFilesClean.clean().size()).isEqualTo(0); + assertThat(orphanFilesClean.clean().getDeletedFilesPath().size()).isEqualTo(0); // second check orphanFilesClean = new LocalOrphanFilesClean( table, System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(2)); - List deleted = orphanFilesClean.clean(); + List deleted = orphanFilesClean.clean().getDeletedFilesPath(); try { validate(deleted, snapshotData, new HashMap<>()); } catch (Throwable t) { @@ -363,13 +361,13 @@ public void testCleanOrphanFilesWithChangelogDecoupled(String changelogProducer) // first check, nothing will be deleted because the default olderThan interval is 1 day LocalOrphanFilesClean orphanFilesClean = new LocalOrphanFilesClean(table); - assertThat(orphanFilesClean.clean().size()).isEqualTo(0); + assertThat(orphanFilesClean.clean().getDeletedFilesPath().size()).isEqualTo(0); // second check orphanFilesClean = new LocalOrphanFilesClean( table, System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(2)); - List deleted = orphanFilesClean.clean(); + List deleted = orphanFilesClean.clean().getDeletedFilesPath(); validate(deleted, snapshotData, changelogData); } @@ -399,7 +397,7 @@ public void testAbnormallyRemoving() throws Exception { LocalOrphanFilesClean orphanFilesClean = new LocalOrphanFilesClean( table, System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(2)); - assertThat(orphanFilesClean.clean().size()).isGreaterThan(0); + assertThat(orphanFilesClean.clean().getDeletedFilesPath().size()).isGreaterThan(0); } private void writeData( diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java index 7695c510b1dc7..b4a3a6b359d91 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java @@ -20,6 +20,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.orphan.FlinkOrphanFilesClean; +import org.apache.paimon.operation.CleanOrphanFilesResult; import org.apache.paimon.operation.LocalOrphanFilesClean; import org.apache.flink.table.procedure.ProcedureContext; @@ -86,11 +87,12 @@ public String[] call( if (mode == null) { mode = "DISTRIBUTED"; } - long deletedFiles; + + CleanOrphanFilesResult cleanOrphanFilesResult; try { switch (mode.toUpperCase(Locale.ROOT)) { case "DISTRIBUTED": - deletedFiles = + cleanOrphanFilesResult = FlinkOrphanFilesClean.executeDatabaseOrphanFiles( procedureContext.getExecutionEnvironment(), catalog, @@ -101,7 +103,7 @@ public String[] call( tableName); break; case "LOCAL": - deletedFiles = + cleanOrphanFilesResult = LocalOrphanFilesClean.executeDatabaseOrphanFiles( catalog, databaseName, @@ -116,7 +118,10 @@ public String[] call( + mode + ". Only 'DISTRIBUTED' and 'LOCAL' are supported."); } - return new String[] {String.valueOf(deletedFiles)}; + return new String[] { + String.valueOf(cleanOrphanFilesResult.getDeletedFileCount()), + String.valueOf(cleanOrphanFilesResult.getDeletedFileTotalLenInBytes()) + }; } catch (Exception e) { throw new RuntimeException(e); } diff --git a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java index 46b62b6bf3072..a168c3785c7c2 100644 --- a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java +++ b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java @@ -137,7 +137,7 @@ public void testRunWithoutException() throws Exception { database, tableName); ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(withOlderThan)); - assertThat(actualDeleteFile).containsExactlyInAnyOrder(Row.of("2")); + assertThat(actualDeleteFile).containsExactlyInAnyOrder(Row.of("2"), Row.of("2")); } @Test diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/orphan/FlinkOrphanFilesClean.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/orphan/FlinkOrphanFilesClean.java index 61bebca24af45..23bbbc9b609ca 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/orphan/FlinkOrphanFilesClean.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/orphan/FlinkOrphanFilesClean.java @@ -27,12 +27,15 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; +import org.apache.paimon.operation.CleanOrphanFilesResult; import org.apache.paimon.operation.OrphanFilesClean; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.SerializableConsumer; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; @@ -61,7 +64,6 @@ import java.util.function.Consumer; import java.util.stream.Collectors; -import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.LONG_TYPE_INFO; import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO; import static org.apache.flink.util.Preconditions.checkState; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -81,7 +83,7 @@ public FlinkOrphanFilesClean( } @Nullable - public DataStream doOrphanClean(StreamExecutionEnvironment env) { + public DataStream doOrphanClean(StreamExecutionEnvironment env) { Configuration flinkConf = new Configuration(); flinkConf.set(ExecutionOptions.RUNTIME_MODE, RuntimeExecutionMode.BATCH); flinkConf.set(ExecutionOptions.SORT_INPUTS, false); @@ -97,8 +99,12 @@ public DataStream doOrphanClean(StreamExecutionEnvironment env) { // snapshot and changelog files are the root of everything, so they are handled specially // here, and subsequently, we will not count their orphan files. - AtomicLong deletedInLocal = new AtomicLong(0); - cleanSnapshotDir(branches, p -> deletedInLocal.incrementAndGet()); + AtomicLong deletedFilesCountInLocal = new AtomicLong(0); + AtomicLong deletedFilesLenInBytesInLocal = new AtomicLong(0); + cleanSnapshotDir( + branches, + path -> deletedFilesCountInLocal.incrementAndGet(), + deletedFilesLenInBytesInLocal::addAndGet); // branch and manifest file final OutputTag> manifestOutputTag = @@ -203,36 +209,45 @@ public void endInput() throws IOException { .map(Path::toUri) .map(Object::toString) .collect(Collectors.toList()); - DataStream candidates = + DataStream> candidates = env.fromCollection(fileDirs) .process( - new ProcessFunction() { + new ProcessFunction>() { @Override public void processElement( String dir, - ProcessFunction.Context ctx, - Collector out) { + ProcessFunction>.Context ctx, + Collector> out) { for (FileStatus fileStatus : tryBestListingDirs(new Path(dir))) { if (oldEnough(fileStatus)) { out.collect( - fileStatus.getPath().toUri().toString()); + Pair.of( + fileStatus + .getPath() + .toUri() + .toString(), + fileStatus.getLen())); } } } }); - DataStream deleted = + DataStream deleted = usedFiles .keyBy(f -> f) - .connect(candidates.keyBy(path -> new Path(path).getName())) + .connect( + candidates.keyBy( + pathAndSize -> new Path(pathAndSize.getKey()).getName())) .transform( "files_join", - LONG_TYPE_INFO, - new BoundedTwoInputOperator() { + TypeInformation.of(CleanOrphanFilesResult.class), + new BoundedTwoInputOperator< + String, Pair, CleanOrphanFilesResult>() { private boolean buildEnd; - private long emitted; + private long emittedFilesCount; + private long emittedFilesLen; private final Set used = new HashSet<>(); @@ -254,8 +269,15 @@ public void endInput(int inputId) { case 2: checkState(buildEnd, "Should build ended."); LOG.info("Finish probe phase."); - LOG.info("Clean files: {}", emitted); - output.collect(new StreamRecord<>(emitted)); + LOG.info( + "Clean files count : {}", + emittedFilesCount); + LOG.info("Clean files size : {}", emittedFilesLen); + output.collect( + new StreamRecord<>( + new CleanOrphanFilesResult( + emittedFilesCount, + emittedFilesLen))); break; } } @@ -266,25 +288,34 @@ public void processElement1(StreamRecord element) { } @Override - public void processElement2(StreamRecord element) { + public void processElement2( + StreamRecord> element) { checkState(buildEnd, "Should build ended."); - String value = element.getValue(); + Pair fileInfo = element.getValue(); + String value = fileInfo.getLeft(); Path path = new Path(value); if (!used.contains(path.getName())) { + emittedFilesCount++; + emittedFilesLen += fileInfo.getRight(); fileCleaner.accept(path); LOG.info("Dry clean: {}", path); - emitted++; } } }); - if (deletedInLocal.get() != 0) { - deleted = deleted.union(env.fromElements(deletedInLocal.get())); + if (deletedFilesCountInLocal.get() != 0 || deletedFilesLenInBytesInLocal.get() != 0) { + deleted = + deleted.union( + env.fromElements( + new CleanOrphanFilesResult( + deletedFilesCountInLocal.get(), + deletedFilesLenInBytesInLocal.get()))); } + return deleted; } - public static long executeDatabaseOrphanFiles( + public static CleanOrphanFilesResult executeDatabaseOrphanFiles( StreamExecutionEnvironment env, Catalog catalog, long olderThanMillis, @@ -293,12 +324,13 @@ public static long executeDatabaseOrphanFiles( String databaseName, @Nullable String tableName) throws Catalog.DatabaseNotExistException, Catalog.TableNotExistException { - List> orphanFilesCleans = new ArrayList<>(); List tableNames = Collections.singletonList(tableName); if (tableName == null || "*".equals(tableName)) { tableNames = catalog.listTables(databaseName); } + List> orphanFilesCleans = + new ArrayList<>(tableNames.size()); for (String t : tableNames) { Identifier identifier = new Identifier(databaseName, t); Table table = catalog.getTable(identifier); @@ -307,7 +339,7 @@ public static long executeDatabaseOrphanFiles( "Only FileStoreTable supports remove-orphan-files action. The table type is '%s'.", table.getClass().getName()); - DataStream clean = + DataStream clean = new FlinkOrphanFilesClean( (FileStoreTable) table, olderThanMillis, @@ -319,8 +351,8 @@ public static long executeDatabaseOrphanFiles( } } - DataStream result = null; - for (DataStream clean : orphanFilesCleans) { + DataStream result = null; + for (DataStream clean : orphanFilesCleans) { if (result == null) { result = clean; } else { @@ -331,20 +363,24 @@ public static long executeDatabaseOrphanFiles( return sum(result); } - private static long sum(DataStream deleted) { - long deleteCount = 0; + private static CleanOrphanFilesResult sum(DataStream deleted) { + long deletedFilesCount = 0; + long deletedFilesLenInBytes = 0; if (deleted != null) { try { - CloseableIterator iterator = + CloseableIterator iterator = deleted.global().executeAndCollect("OrphanFilesClean"); while (iterator.hasNext()) { - deleteCount += iterator.next(); + CleanOrphanFilesResult cleanOrphanFilesResult = iterator.next(); + deletedFilesCount += cleanOrphanFilesResult.getDeletedFileCount(); + deletedFilesLenInBytes += + cleanOrphanFilesResult.getDeletedFileTotalLenInBytes(); } iterator.close(); } catch (Exception e) { throw new RuntimeException(e); } } - return deleteCount; + return new CleanOrphanFilesResult(deletedFilesCount, deletedFilesLenInBytes); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java index 10ad878e0ccb5..4cd1b3e003038 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java @@ -20,6 +20,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.orphan.FlinkOrphanFilesClean; +import org.apache.paimon.operation.CleanOrphanFilesResult; import org.apache.paimon.operation.LocalOrphanFilesClean; import org.apache.flink.table.annotation.ArgumentHint; @@ -75,11 +76,11 @@ public String[] call( if (mode == null) { mode = "DISTRIBUTED"; } - long deletedFiles; + CleanOrphanFilesResult cleanOrphanFilesResult; try { switch (mode.toUpperCase(Locale.ROOT)) { case "DISTRIBUTED": - deletedFiles = + cleanOrphanFilesResult = FlinkOrphanFilesClean.executeDatabaseOrphanFiles( procedureContext.getExecutionEnvironment(), catalog, @@ -90,7 +91,7 @@ public String[] call( tableName); break; case "LOCAL": - deletedFiles = + cleanOrphanFilesResult = LocalOrphanFilesClean.executeDatabaseOrphanFiles( catalog, databaseName, @@ -105,7 +106,10 @@ public String[] call( + mode + ". Only 'DISTRIBUTED' and 'LOCAL' are supported."); } - return new String[] {String.valueOf(deletedFiles)}; + return new String[] { + String.valueOf(cleanOrphanFilesResult.getDeletedFileCount()), + String.valueOf(cleanOrphanFilesResult.getDeletedFileTotalLenInBytes()) + }; } catch (Exception e) { throw new RuntimeException(e); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java index 5f874a5a7f9b2..77f3be2f0c765 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java @@ -148,7 +148,7 @@ public void testRunWithoutException(boolean isNamedArgument) throws Exception { tableName); ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(withOlderThan)); - assertThat(actualDeleteFile).containsExactlyInAnyOrder(Row.of("2")); + assertThat(actualDeleteFile).containsExactlyInAnyOrder(Row.of("2"), Row.of("2")); } @ParameterizedTest diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedure.java index 293e84ca14bd6..a929641106c69 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedure.java @@ -19,6 +19,7 @@ package org.apache.paimon.spark.procedure; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.operation.CleanOrphanFilesResult; import org.apache.paimon.operation.LocalOrphanFilesClean; import org.apache.paimon.operation.OrphanFilesClean; import org.apache.paimon.spark.catalog.WithPaimonCatalog; @@ -66,7 +67,9 @@ public class RemoveOrphanFilesProcedure extends BaseProcedure { private static final StructType OUTPUT_TYPE = new StructType( new StructField[] { - new StructField("result", LongType, true, Metadata.empty()) + new StructField("deletedFileCount", LongType, true, Metadata.empty()), + new StructField( + "deletedFileTotalLenInBytes", LongType, true, Metadata.empty()) }); private RemoveOrphanFilesProcedure(TableCatalog tableCatalog) { @@ -104,11 +107,11 @@ public InternalRow[] call(InternalRow args) { Catalog catalog = ((WithPaimonCatalog) tableCatalog()).paimonCatalog(); String mode = args.isNullAt(4) ? "DISTRIBUTED" : args.getString(4); - long deletedFiles; + CleanOrphanFilesResult cleanOrphanFilesResult; try { switch (mode.toUpperCase(Locale.ROOT)) { case "LOCAL": - deletedFiles = + cleanOrphanFilesResult = LocalOrphanFilesClean.executeDatabaseOrphanFiles( catalog, identifier.getDatabaseName(), @@ -120,7 +123,7 @@ public InternalRow[] call(InternalRow args) { args.isNullAt(3) ? null : args.getInt(3)); break; case "DISTRIBUTED": - deletedFiles = + cleanOrphanFilesResult = SparkOrphanFilesClean.executeDatabaseOrphanFiles( catalog, identifier.getDatabaseName(), @@ -137,7 +140,12 @@ public InternalRow[] call(InternalRow args) { + mode + ". Only 'DISTRIBUTED' and 'LOCAL' are supported."); } - return new InternalRow[] {newInternalRow(deletedFiles)}; + + return new InternalRow[] { + newInternalRow( + cleanOrphanFilesResult.getDeletedFileCount(), + cleanOrphanFilesResult.getDeletedFileTotalLenInBytes()) + }; } catch (Exception e) { throw new RuntimeException(e); } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/orphan/SparkOrphanFilesClean.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/orphan/SparkOrphanFilesClean.scala index 488d70e349356..fca0493ede287 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/orphan/SparkOrphanFilesClean.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/orphan/SparkOrphanFilesClean.scala @@ -22,15 +22,14 @@ import org.apache.paimon.{utils, Snapshot} import org.apache.paimon.catalog.{Catalog, Identifier} import org.apache.paimon.fs.Path import org.apache.paimon.manifest.{ManifestEntry, ManifestFile} -import org.apache.paimon.operation.OrphanFilesClean +import org.apache.paimon.operation.{CleanOrphanFilesResult, OrphanFilesClean} import org.apache.paimon.operation.OrphanFilesClean.retryReadingFiles import org.apache.paimon.table.FileStoreTable import org.apache.paimon.utils.SerializableConsumer import org.apache.spark.internal.Logging -import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.{functions, Dataset, SparkSession} import org.apache.spark.sql.catalyst.SQLConfHelper -import org.apache.spark.sql.functions.sum import java.util import java.util.Collections @@ -50,14 +49,18 @@ case class SparkOrphanFilesClean( with SQLConfHelper with Logging { - def doOrphanClean(): (Dataset[Long], Dataset[BranchAndManifestFile]) = { + def doOrphanClean(): (Dataset[(Long, Long)], Dataset[BranchAndManifestFile]) = { import spark.implicits._ val branches = validBranches() - val deletedInLocal = new AtomicLong(0) + val deletedFilesCountInLocal = new AtomicLong(0) + val deletedFilesLenInBytesInLocal = new AtomicLong(0) // snapshot and changelog files are the root of everything, so they are handled specially // here, and subsequently, we will not count their orphan files. - cleanSnapshotDir(branches, (_: Path) => deletedInLocal.incrementAndGet) + cleanSnapshotDir( + branches, + (_: Path) => deletedFilesCountInLocal.incrementAndGet, + size => deletedFilesLenInBytesInLocal.addAndGet(size)) val maxBranchParallelism = Math.min(branches.size(), parallelism) // find snapshots using branch and find manifests(manifest, index, statistics) using snapshot @@ -121,10 +124,10 @@ case class SparkOrphanFilesClean( .flatMap { dir => tryBestListingDirs(new Path(dir)).asScala.filter(oldEnough).map { - file => (file.getPath.getName, file.getPath.toUri.toString) + file => (file.getPath.getName, file.getPath.toUri.toString, file.getLen) } } - .toDF("name", "path") + .toDF("name", "path", "len") .repartition(parallelism) // use left anti to filter files which is not used @@ -132,21 +135,30 @@ case class SparkOrphanFilesClean( .join(usedFiles, $"name" === $"used_name", "left_anti") .mapPartitions { it => - var deleted = 0L + var deletedFilesCount = 0L + var deletedFilesLenInBytes = 0L + while (it.hasNext) { - val pathToClean = it.next().getString(1) - specifiedFileCleaner.accept(new Path(pathToClean)) + val fileInfo = it.next(); + val pathToClean = fileInfo.getString(1) + val deletedPath = new Path(pathToClean) + deletedFilesLenInBytes += fileInfo.getLong(2) + specifiedFileCleaner.accept(deletedPath) logInfo(s"Cleaned file: $pathToClean") - deleted += 1 + deletedFilesCount += 1 } - logInfo(s"Total cleaned files: $deleted"); - Iterator.single(deleted) + logInfo( + s"Total cleaned files: $deletedFilesCount, Total cleaned files len : $deletedFilesLenInBytes") + Iterator.single((deletedFilesCount, deletedFilesLenInBytes)) + } + val finalDeletedDataset = + if (deletedFilesCountInLocal.get() != 0 || deletedFilesLenInBytesInLocal.get() != 0) { + deleted.union( + spark.createDataset( + Seq((deletedFilesCountInLocal.get(), deletedFilesLenInBytesInLocal.get())))) + } else { + deleted } - val finalDeletedDataset = if (deletedInLocal.get() != 0) { - deleted.union(spark.createDataset(Seq(deletedInLocal.get()))) - } else { - deleted - } (finalDeletedDataset, usedManifestFiles) } @@ -169,7 +181,7 @@ object SparkOrphanFilesClean extends SQLConfHelper { tableName: String, olderThanMillis: Long, fileCleaner: SerializableConsumer[Path], - parallelismOpt: Integer): Long = { + parallelismOpt: Integer): CleanOrphanFilesResult = { val spark = SparkSession.active val parallelism = if (parallelismOpt == null) { Math.max(spark.sparkContext.defaultParallelism, conf.numShufflePartitions) @@ -192,7 +204,7 @@ object SparkOrphanFilesClean extends SQLConfHelper { table.asInstanceOf[FileStoreTable] } if (tables.isEmpty) { - return 0 + return new CleanOrphanFilesResult(0, 0) } val (deleted, waitToRelease) = tables.map { table => @@ -207,15 +219,15 @@ object SparkOrphanFilesClean extends SQLConfHelper { try { val result = deleted .reduce((l, r) => l.union(r)) - .toDF("deleted") - .agg(sum("deleted")) + .toDF("deletedFilesCount", "deletedFilesLenInBytes") + .agg(functions.sum("deletedFilesCount"), functions.sum("deletedFilesLenInBytes")) .head() - assert(result.schema.size == 1, result.schema) + assert(result.schema.size == 2, result.schema) if (result.isNullAt(0)) { // no files can be deleted - 0 + new CleanOrphanFilesResult(0, 0) } else { - result.getLong(0) + new CleanOrphanFilesResult(result.getLong(0), result.getLong(1)) } } finally { waitToRelease.foreach(_.unpersist()) diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala index d9d73811266dd..3ffe7fba264f0 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala @@ -52,7 +52,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { fileIO.tryToWriteAtomic(orphanFile2, "b") // by default, no file deleted - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) val orphanFile2ModTime = fileIO.getFileStatus(orphanFile2).getModificationTime val older_than1 = DateTimeUtils.formatLocalDateTime( @@ -63,7 +63,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql(s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than1')"), - Row(1) :: Nil) + Row(1, 1) :: Nil) val older_than2 = DateTimeUtils.formatLocalDateTime( DateTimeUtils.toLocalDateTime(System.currentTimeMillis()), @@ -71,9 +71,9 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql(s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than2')"), - Row(1) :: Nil) + Row(1, 1) :: Nil) - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) } test("Paimon procedure: dry run remove orphan files") { @@ -97,7 +97,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { fileIO.writeFile(orphanFile2, "b", true) // by default, no file deleted - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) val older_than = DateTimeUtils.formatLocalDateTime( DateTimeUtils.toLocalDateTime(System.currentTimeMillis()), @@ -106,10 +106,10 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql( s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than', dry_run => true)"), - Row(2) :: Nil + Row(2, 2) :: Nil ) - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) } test("Paimon procedure: remove database orphan files") { @@ -146,7 +146,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { fileIO2.tryToWriteAtomic(orphanFile22, "b") // by default, no file deleted - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*')"), Row(0, 0) :: Nil) val orphanFile12ModTime = fileIO1.getFileStatus(orphanFile12).getModificationTime val older_than1 = DateTimeUtils.formatLocalDateTime( @@ -157,7 +157,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*', older_than => '$older_than1')"), - Row(2) :: Nil + Row(2, 2) :: Nil ) val older_than2 = DateTimeUtils.formatLocalDateTime( @@ -166,10 +166,10 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*', older_than => '$older_than2')"), - Row(2) :: Nil + Row(2, 2) :: Nil ) - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*')"), Row(0, 0) :: Nil) } test("Paimon procedure: remove orphan files with mode") { @@ -193,7 +193,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { fileIO.tryToWriteAtomic(orphanFile2, "b") // by default, no file deleted - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) val orphanFile2ModTime = fileIO.getFileStatus(orphanFile2).getModificationTime val older_than1 = DateTimeUtils.formatLocalDateTime( @@ -205,7 +205,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql( s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than1', mode => 'diSTributed')"), - Row(1) :: Nil) + Row(1, 1) :: Nil) val older_than2 = DateTimeUtils.formatLocalDateTime( DateTimeUtils.toLocalDateTime(System.currentTimeMillis()), @@ -214,9 +214,9 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql( s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than2', mode => 'local')"), - Row(1) :: Nil) + Row(1, 1) :: Nil) - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) } } From ee466bcac14bd7f1229beeaf4e405da0956792ca Mon Sep 17 00:00:00 2001 From: Jingsong Date: Tue, 26 Nov 2024 13:07:43 +0800 Subject: [PATCH 061/157] [core] CleanOrphanFilesResult.deletedFilesPath should be nullable --- .../paimon/operation/CleanOrphanFilesResult.java | 16 +++++++++++----- .../paimon/operation/LocalOrphanFilesClean.java | 4 ++-- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java b/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java index 5a3bc67f9c95e..d29eede720ac9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java @@ -20,24 +20,29 @@ import org.apache.paimon.fs.Path; +import javax.annotation.Nullable; + import java.util.List; /** The result of OrphanFilesClean. */ public class CleanOrphanFilesResult { - private List deletedFilesPath; private final long deletedFileCount; private final long deletedFileTotalLenInBytes; + @Nullable private final List deletedFilesPath; + public CleanOrphanFilesResult(long deletedFileCount, long deletedFileTotalLenInBytes) { - this.deletedFileCount = deletedFileCount; - this.deletedFileTotalLenInBytes = deletedFileTotalLenInBytes; + this(deletedFileCount, deletedFileTotalLenInBytes, null); } public CleanOrphanFilesResult( - List deletedFilesPath, long deletedFileCount, long deletedFileTotalLenInBytes) { - this(deletedFileCount, deletedFileTotalLenInBytes); + long deletedFileCount, + long deletedFileTotalLenInBytes, + @Nullable List deletedFilesPath) { this.deletedFilesPath = deletedFilesPath; + this.deletedFileCount = deletedFileCount; + this.deletedFileTotalLenInBytes = deletedFileTotalLenInBytes; } public long getDeletedFileCount() { @@ -48,6 +53,7 @@ public long getDeletedFileTotalLenInBytes() { return deletedFileTotalLenInBytes; } + @Nullable public List getDeletedFilesPath() { return deletedFilesPath; } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java b/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java index 511c5fc7fb79b..6a4276662468b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java @@ -101,7 +101,7 @@ public CleanOrphanFilesResult clean() Map> candidates = getCandidateDeletingFiles(); if (candidates.isEmpty()) { return new CleanOrphanFilesResult( - deleteFiles, deleteFiles.size(), deletedFilesLenInBytes.get()); + deleteFiles.size(), deletedFilesLenInBytes.get(), deleteFiles); } candidateDeletes = new HashSet<>(candidates.keySet()); @@ -128,7 +128,7 @@ public CleanOrphanFilesResult clean() candidateDeletes.clear(); return new CleanOrphanFilesResult( - deleteFiles, deleteFiles.size(), deletedFilesLenInBytes.get()); + deleteFiles.size(), deletedFilesLenInBytes.get(), deleteFiles); } private void collectWithoutDataFile( From 61ee6f8f4526603deae5be44fffb8a0168823565 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Tue, 26 Nov 2024 14:27:22 +0800 Subject: [PATCH 062/157] [flink] Avoid deprecated usages about Configuration (#4584) --- .../apache/paimon/benchmark/QueryRunner.java | 2 +- .../serialization/SerializerConfig.java | 22 ++++++ .../serialization/SerializerConfigImpl.java | 22 ++++++ .../serialization/SerializerConfig.java | 22 ++++++ .../serialization/SerializerConfigImpl.java | 22 ++++++ .../serialization/SerializerConfig.java | 22 ++++++ .../serialization/SerializerConfigImpl.java | 22 ++++++ .../serialization/SerializerConfig.java | 22 ++++++ .../serialization/SerializerConfigImpl.java | 22 ++++++ ...afkaDebeziumAvroDeserializationSchema.java | 2 +- .../strategy/MongoVersionStrategy.java | 8 +-- .../action/cdc/mysql/MySqlRecordParser.java | 15 +++-- ...lsarDebeziumAvroDeserializationSchema.java | 2 +- .../cdc/mongodb/MongodbSchemaITCase.java | 67 +++++++++---------- .../cdc/mysql/MySqlSyncTableActionITCase.java | 10 ++- .../sink/cdc/CdcRecordSerializeITCase.java | 28 ++++++-- .../changelog/ChangelogTaskTypeInfo.java | 13 +++- .../flink/sink/CommittableTypeInfo.java | 12 +++- .../flink/sink/CompactionTaskTypeInfo.java | 12 +++- .../apache/paimon/flink/sink/FlinkSink.java | 7 +- .../sink/MultiTableCommittableTypeInfo.java | 12 +++- .../MultiTableCompactionTaskTypeInfo.java | 13 +++- .../flink/source/FlinkSourceBuilder.java | 16 ++--- .../AlignedContinuousFileStoreSource.java | 2 +- .../paimon/flink/utils/InternalTypeInfo.java | 14 +++- .../paimon/flink/utils/JavaTypeInfo.java | 16 ++++- .../paimon/flink/FileSystemCatalogITCase.java | 3 +- .../paimon/flink/FlinkJobRecoveryITCase.java | 9 ++- .../paimon/flink/RescaleBucketITCase.java | 4 +- .../UnawareBucketAppendOnlyTableITCase.java | 14 +++- .../flink/sink/SinkSavepointITCase.java | 2 +- .../paimon/flink/util/AbstractTestBase.java | 14 ++-- .../flink/util/ReadWriteTableTestUtil.java | 26 ++++--- .../paimon/hive/HiveCatalogITCaseBase.java | 6 +- 34 files changed, 396 insertions(+), 109 deletions(-) create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java diff --git a/paimon-benchmark/paimon-cluster-benchmark/src/main/java/org/apache/paimon/benchmark/QueryRunner.java b/paimon-benchmark/paimon-cluster-benchmark/src/main/java/org/apache/paimon/benchmark/QueryRunner.java index b07cdef8465ee..8bfe4b6c9c03a 100644 --- a/paimon-benchmark/paimon-cluster-benchmark/src/main/java/org/apache/paimon/benchmark/QueryRunner.java +++ b/paimon-benchmark/paimon-cluster-benchmark/src/main/java/org/apache/paimon/benchmark/QueryRunner.java @@ -77,7 +77,7 @@ public Result run() { String sinkPathConfig = BenchmarkGlobalConfiguration.loadConfiguration() - .getString(BenchmarkOptions.SINK_PATH); + .get(BenchmarkOptions.SINK_PATH); if (sinkPathConfig == null) { throw new IllegalArgumentException( BenchmarkOptions.SINK_PATH.key() + " must be set"); diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java new file mode 100644 index 0000000000000..16987469a948f --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public interface SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java new file mode 100644 index 0000000000000..374d33f6500d5 --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public class SerializerConfigImpl implements SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java new file mode 100644 index 0000000000000..16987469a948f --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public interface SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java new file mode 100644 index 0000000000000..374d33f6500d5 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public class SerializerConfigImpl implements SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java new file mode 100644 index 0000000000000..16987469a948f --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public interface SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java new file mode 100644 index 0000000000000..374d33f6500d5 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public class SerializerConfigImpl implements SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java new file mode 100644 index 0000000000000..16987469a948f --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public interface SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java new file mode 100644 index 0000000000000..374d33f6500d5 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public class SerializerConfigImpl implements SerializerConfig {} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java index fc672b9dc0ab6..eea364d460dee 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java @@ -48,7 +48,7 @@ public class KafkaDebeziumAvroDeserializationSchema public KafkaDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { this.topic = KafkaActionUtils.findOneTopic(cdcSourceConfig); - this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + this.schemaRegistryUrl = cdcSourceConfig.get(SCHEMA_REGISTRY_URL); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java index 64f127571134b..df288a4150e63 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java @@ -83,7 +83,7 @@ default Map getExtractRow( Configuration mongodbConfig) throws JsonProcessingException { SchemaAcquisitionMode mode = - SchemaAcquisitionMode.valueOf(mongodbConfig.getString(START_MODE).toUpperCase()); + SchemaAcquisitionMode.valueOf(mongodbConfig.get(START_MODE).toUpperCase()); ObjectNode objectNode = JsonSerdeUtil.asSpecificNodeType(jsonNode.asText(), ObjectNode.class); JsonNode idNode = objectNode.get(ID_FIELD); @@ -92,7 +92,7 @@ default Map getExtractRow( "The provided MongoDB JSON document does not contain an _id field."); } JsonNode document = - mongodbConfig.getBoolean(DEFAULT_ID_GENERATION) + mongodbConfig.get(DEFAULT_ID_GENERATION) ? objectNode.set( ID_FIELD, idNode.get(OID_FIELD) == null ? idNode : idNode.get(OID_FIELD)) @@ -101,8 +101,8 @@ default Map getExtractRow( case SPECIFIED: return parseFieldsFromJsonRecord( document.toString(), - mongodbConfig.getString(PARSER_PATH), - mongodbConfig.getString(FIELD_NAME), + mongodbConfig.get(PARSER_PATH), + mongodbConfig.get(FIELD_NAME), computedColumns, rowTypeBuilder); case DYNAMIC: diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java index 502e6237a477e..26579e718f564 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java @@ -45,6 +45,8 @@ import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import org.slf4j.Logger; @@ -99,11 +101,14 @@ public MySqlRecordParser( .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); String stringifyServerTimeZone = mySqlConfig.get(MySqlSourceOptions.SERVER_TIME_ZONE); - this.isDebeziumSchemaCommentsEnabled = - mySqlConfig.getBoolean( - DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX - + RelationalDatabaseConnectorConfig.INCLUDE_SCHEMA_COMMENTS.name(), - false); + ConfigOption includeSchemaCommentsConfig = + ConfigOptions.key( + DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX + + RelationalDatabaseConnectorConfig.INCLUDE_SCHEMA_COMMENTS + .name()) + .booleanType() + .defaultValue(false); + this.isDebeziumSchemaCommentsEnabled = mySqlConfig.get(includeSchemaCommentsConfig); this.serverTimeZone = stringifyServerTimeZone == null ? ZoneId.systemDefault() diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java index b0d1d1bf620f5..f45ee034bec84 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java @@ -46,7 +46,7 @@ public class PulsarDebeziumAvroDeserializationSchema public PulsarDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { this.topic = PulsarActionUtils.findOneTopic(cdcSourceConfig); - this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + this.schemaRegistryUrl = cdcSourceConfig.get(SCHEMA_REGISTRY_URL); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java index 394cdd1f149bc..f0328b5663246 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java @@ -76,13 +76,12 @@ public static void initMongoDB() { @Test public void testCreateSchemaFromValidConfig() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); Schema schema = MongodbSchemaUtils.getMongodbSchema(mongodbConfig); assertNotNull(schema); } @@ -90,13 +89,12 @@ public void testCreateSchemaFromValidConfig() { @Test public void testCreateSchemaFromInvalidHost() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, "127.0.0.1:12345"); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, "127.0.0.1:12345"); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); assertThrows( RuntimeException.class, () -> MongodbSchemaUtils.getMongodbSchema(mongodbConfig)); @@ -106,7 +104,7 @@ public void testCreateSchemaFromInvalidHost() { public void testCreateSchemaFromIncompleteConfig() { // Create a Configuration object with missing necessary settings Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); // Expect an exception to be thrown due to missing necessary settings assertThrows( NullPointerException.class, @@ -117,13 +115,12 @@ public void testCreateSchemaFromIncompleteConfig() { public void testCreateSchemaFromDynamicConfig() { // Create a Configuration object with the necessary settings Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); // Call the method and check the results Schema schema = MongodbSchemaUtils.getMongodbSchema(mongodbConfig); @@ -142,13 +139,12 @@ public void testCreateSchemaFromDynamicConfig() { @Test public void testCreateSchemaFromInvalidDatabase() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "invalidDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "invalidDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); assertThrows( RuntimeException.class, () -> MongodbSchemaUtils.getMongodbSchema(mongodbConfig)); @@ -157,13 +153,12 @@ public void testCreateSchemaFromInvalidDatabase() { @Test public void testCreateSchemaFromInvalidCollection() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "invalidCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "invalidCollection"); assertThrows( RuntimeException.class, () -> MongodbSchemaUtils.getMongodbSchema(mongodbConfig)); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index bdeab07a746cc..febbe4e1deaad 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -31,7 +31,8 @@ import org.apache.paimon.utils.CommonTestUtils; import org.apache.paimon.utils.JsonSerdeUtil; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.BeforeAll; @@ -1285,8 +1286,11 @@ public void testDefaultCheckpointInterval() throws Exception { mySqlConfig.put("database-name", "default_checkpoint"); mySqlConfig.put("table-name", "t"); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setRestartStrategy(RestartStrategies.noRestart()); + // Using `none` to avoid compatibility issues with Flink 1.18-. + Configuration configuration = new Configuration(); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY, "none"); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build(); action.withStreamExecutionEnvironment(env); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java index 698900436e8d5..b202ca53c9cc9 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java @@ -25,6 +25,8 @@ import org.apache.paimon.types.VarCharType; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; +import org.apache.flink.api.common.serialization.SerializerConfigImpl; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -35,6 +37,8 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -49,7 +53,7 @@ public class CdcRecordSerializeITCase { @Test - public void testCdcRecordKryoSerialize() throws IOException { + public void testCdcRecordKryoSerialize() throws Exception { KryoSerializer kr = createFlinkKryoSerializer(RichCdcMultiplexRecord.class); RowType.Builder rowType = RowType.builder(); @@ -78,7 +82,7 @@ public void testCdcRecordKryoSerialize() throws IOException { } @Test - public void testUnmodifiableListKryoSerialize() throws IOException { + public void testUnmodifiableListKryoSerialize() throws Exception { KryoSerializer kryoSerializer = createFlinkKryoSerializer(List.class); RowType.Builder rowType = RowType.builder(); rowType.field("id", new BigIntType()); @@ -101,8 +105,24 @@ public void testUnmodifiableListKryoSerialize() throws IOException { assertThat(deserializeRecord).isEqualTo(fields); } - public static KryoSerializer createFlinkKryoSerializer(Class type) { - return new KryoSerializer<>(type, new ExecutionConfig()); + @SuppressWarnings({"unchecked", "rawtypes"}) + public static KryoSerializer createFlinkKryoSerializer(Class type) + throws NoSuchMethodException, InvocationTargetException, InstantiationException, + IllegalAccessException { + try { + Constructor constructor = + KryoSerializer.class.getConstructor(Class.class, SerializerConfig.class); + return (KryoSerializer) constructor.newInstance(type, new SerializerConfigImpl()); + } catch (NoSuchMethodException + | InvocationTargetException + | IllegalAccessException + | InstantiationException e) { + // to stay compatible with Flink 1.18- + } + + Constructor constructor = + KryoSerializer.class.getConstructor(Class.class, ExecutionConfig.class); + return (KryoSerializer) constructor.newInstance(type, new ExecutionConfig()); } private static final class TestOutputView extends DataOutputStream implements DataOutputView { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java index 5cae899a07040..a529e6764faea 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java @@ -21,6 +21,7 @@ import org.apache.paimon.flink.sink.NoneCopyVersionedSerializerTypeSerializerProxy; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -56,7 +57,17 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer( + SerializerConfig serializerConfig) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // we don't need copy for task return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java index dcb87238b8337..92e826a913792 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java @@ -21,6 +21,7 @@ import org.apache.paimon.table.sink.CommitMessageSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -57,7 +58,16 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // no copy, so that data from writer is directly going into committer while chaining return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java index 47defa61a9713..6510a85b800af 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java @@ -22,6 +22,7 @@ import org.apache.paimon.table.sink.CompactionTaskSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -58,7 +59,16 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // we don't need copy for task return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java index 59f2f4b1035f0..dd364c196d8bf 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java @@ -42,7 +42,6 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.CheckpointConfig; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; @@ -338,13 +337,11 @@ public static void assertStreamingConfiguration(StreamExecutionEnvironment env) checkArgument( !env.getCheckpointConfig().isUnalignedCheckpointsEnabled(), "Paimon sink currently does not support unaligned checkpoints. Please set " - + ExecutionCheckpointingOptions.ENABLE_UNALIGNED.key() - + " to false."); + + "execution.checkpointing.unaligned.enabled to false."); checkArgument( env.getCheckpointConfig().getCheckpointingMode() == CheckpointingMode.EXACTLY_ONCE, "Paimon sink currently only supports EXACTLY_ONCE checkpoint mode. Please set " - + ExecutionCheckpointingOptions.CHECKPOINTING_MODE.key() - + " to exactly-once"); + + "execution.checkpointing.mode to exactly-once"); } public static void assertBatchAdaptiveParallelism( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java index f82f082098671..7da0ae0e20788 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java @@ -21,6 +21,7 @@ import org.apache.paimon.table.sink.CommitMessageSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -57,7 +58,16 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // no copy, so that data from writer is directly going into committer while chaining return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java index f27f29f87fe74..0116ff198811e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java @@ -23,6 +23,7 @@ import org.apache.paimon.table.sink.MultiTableCompactionTaskSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.io.SimpleVersionedSerializerTypeSerializerProxy; @@ -60,7 +61,17 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer( + SerializerConfig serializerConfig) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer( ExecutionConfig executionConfig) { return new SimpleVersionedSerializerTypeSerializerProxy< diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java index b3dcd4840cc1b..e864ec0500459 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java @@ -46,7 +46,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; @@ -331,30 +330,25 @@ private void assertStreamingConfigurationForAlignMode(StreamExecutionEnvironment checkArgument( checkpointConfig.isCheckpointingEnabled(), "The align mode of paimon source is only supported when checkpoint enabled. Please set " - + ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL.key() - + "larger than 0"); + + "execution.checkpointing.interval larger than 0"); checkArgument( checkpointConfig.getMaxConcurrentCheckpoints() == 1, "The align mode of paimon source supports at most one ongoing checkpoint at the same time. Please set " - + ExecutionCheckpointingOptions.MAX_CONCURRENT_CHECKPOINTS.key() - + " to 1"); + + "execution.checkpointing.max-concurrent-checkpoints to 1"); checkArgument( checkpointConfig.getCheckpointTimeout() > conf.get(FlinkConnectorOptions.SOURCE_CHECKPOINT_ALIGN_TIMEOUT) .toMillis(), "The align mode of paimon source requires that the timeout of checkpoint is greater than the timeout of the source's snapshot alignment. Please increase " - + ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT.key() - + " or decrease " + + "execution.checkpointing.timeout or decrease " + FlinkConnectorOptions.SOURCE_CHECKPOINT_ALIGN_TIMEOUT.key()); checkArgument( !env.getCheckpointConfig().isUnalignedCheckpointsEnabled(), "The align mode of paimon source currently does not support unaligned checkpoints. Please set " - + ExecutionCheckpointingOptions.ENABLE_UNALIGNED.key() - + " to false."); + + "execution.checkpointing.unaligned.enabled to false."); checkArgument( env.getCheckpointConfig().getCheckpointingMode() == CheckpointingMode.EXACTLY_ONCE, "The align mode of paimon source currently only supports EXACTLY_ONCE checkpoint mode. Please set " - + ExecutionCheckpointingOptions.CHECKPOINTING_MODE.key() - + " to exactly-once"); + + "execution.checkpointing.mode to exactly-once"); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java index d6b7060763acf..705e1d9a7a4c5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java @@ -73,7 +73,7 @@ public SourceReader createReader(SourceReaderCont limit, new FutureCompletingBlockingQueue<>( context.getConfiguration() - .getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY))); + .get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY))); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java index 4ea5db9f34d48..60898421ddea2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java @@ -22,6 +22,7 @@ import org.apache.paimon.types.RowType; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -73,8 +74,17 @@ public boolean isKeyType() { return false; } - @Override - public TypeSerializer createSerializer(ExecutionConfig config) { + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { return serializer.duplicate(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java index a36243c5bdacb..4aea809b51bc8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -78,7 +79,16 @@ public boolean isKeyType() { return Comparable.class.isAssignableFrom(typeClass); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { return new JavaSerializer<>(this.typeClass); } @@ -91,7 +101,9 @@ public TypeComparator createComparator( @SuppressWarnings("rawtypes") GenericTypeComparator comparator = new GenericTypeComparator( - sortOrderAscending, createSerializer(executionConfig), this.typeClass); + sortOrderAscending, + new JavaSerializer<>(this.typeClass), + this.typeClass); return (TypeComparator) comparator; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java index 239043ff79e1d..915c93680a0dc 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java @@ -27,7 +27,6 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.utils.BlockingIterator; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; @@ -60,7 +59,7 @@ public void setup() { tableEnvironmentBuilder() .streamingMode() .parallelism(1) - .setConf(ExecutionCheckpointingOptions.ENABLE_UNALIGNED, false) + .setString("execution.checkpointing.unaligned.enabled", "false") .build(); path = getTempDirPath(); tEnv.executeSql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java index c46c4c3589222..8df379a71b786 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java @@ -65,7 +65,7 @@ public void before() throws IOException { .set( CheckpointingOptions.EXTERNALIZED_CHECKPOINT_RETENTION, ExternalizedCheckpointRetention.RETAIN_ON_CANCELLATION) - .removeConfig(CheckpointingOptions.CHECKPOINTING_INTERVAL); + .removeKey("execution.checkpointing.interval"); // insert source data batchSql("INSERT INTO source_table1 VALUES (1, 'test-1', '20241030')"); @@ -219,10 +219,9 @@ private void testRecoverFromSavepoint( batchSql(sql); } - Configuration config = - sEnv.getConfig() - .getConfiguration() - .set(StateRecoveryOptions.SAVEPOINT_PATH, checkpointPath); + Configuration config = sEnv.getConfig().getConfiguration(); + // use config string to stay compatible with flink 1.19- + config.setString("execution.state-recovery.path", checkpointPath); for (Map.Entry entry : recoverOptions.entrySet()) { config.setString(entry.getKey(), entry.getValue()); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java index 08969bddfdb3a..d5747d2e28d4b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java @@ -26,7 +26,6 @@ import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.types.Row; import org.junit.jupiter.api.Test; @@ -106,9 +105,10 @@ public void testSuspendAndRecoverAfterRescaleOverwrite() throws Exception { assertThat(batchSql("SELECT * FROM T3")).containsExactlyInAnyOrderElementsOf(committedData); // step5: resume streaming job + // use config string to stay compatible with flink 1.19- sEnv.getConfig() .getConfiguration() - .set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + .setString("execution.state-recovery.path", savepointPath); JobClient resumedJobClient = startJobAndCommitSnapshot(streamSql, snapshotAfterRescale.id()); // stop job diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java index f6dfb1b23046b..6ca78b088fb79 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java @@ -28,6 +28,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.utils.FailingFileIO; +import org.apache.paimon.utils.TimeUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -50,7 +51,6 @@ import java.util.List; import java.util.Random; -import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -203,7 +203,11 @@ public void testCompactionInStreamingMode() throws Exception { batchSql("ALTER TABLE append_table SET ('compaction.early-max.file-num' = '4')"); batchSql("ALTER TABLE append_table SET ('continuous.discovery-interval' = '1 s')"); - sEnv.getConfig().getConfiguration().set(CHECKPOINTING_INTERVAL, Duration.ofMillis(500)); + sEnv.getConfig() + .getConfiguration() + .setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit(Duration.ofMillis(500))); sEnv.executeSql( "CREATE TEMPORARY TABLE Orders_in (\n" + " f0 INT,\n" @@ -224,7 +228,11 @@ public void testCompactionInStreamingModeWithMaxWatermark() throws Exception { batchSql("ALTER TABLE append_table SET ('compaction.early-max.file-num' = '4')"); batchSql("ALTER TABLE append_table SET ('continuous.discovery-interval' = '1 s')"); - sEnv.getConfig().getConfiguration().set(CHECKPOINTING_INTERVAL, Duration.ofMillis(500)); + sEnv.getConfig() + .getConfiguration() + .setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit(Duration.ofMillis(500))); sEnv.executeSql( "CREATE TEMPORARY TABLE Orders_in (\n" + " f0 INT,\n" diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java index 6b912d2e57fe9..b1486deacb0c6 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java @@ -137,7 +137,7 @@ private JobClient runRecoverFromSavepointJob(String failingPath, String savepoin .parallelism(1) .allowRestart() .setConf(conf) - .setConf(StateBackendOptions.STATE_BACKEND, "filesystem") + .setConf(StateBackendOptions.STATE_BACKEND, "hashmap") .setConf( CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + path + "/checkpoint") diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java index ce0017eb18747..ee838ed68255e 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.util; import org.apache.paimon.utils.FileIOUtils; +import org.apache.paimon.utils.TimeUtils; import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.dag.Transformation; @@ -29,7 +30,6 @@ import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -164,6 +164,11 @@ public TableEnvironmentBuilder setConf(ConfigOption option, T value) { return this; } + public TableEnvironmentBuilder setString(String key, String value) { + conf.setString(key, value); + return this; + } + public TableEnvironmentBuilder setConf(Configuration conf) { this.conf.addAll(conf); return this; @@ -182,9 +187,10 @@ public TableEnvironment build() { if (checkpointIntervalMs != null) { tEnv.getConfig() .getConfiguration() - .set( - ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, - Duration.ofMillis(checkpointIntervalMs)); + .setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit( + Duration.ofMillis(checkpointIntervalMs))); } } else { tEnv = diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java index 86b0014eb39cf..9c3170f9a96b1 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java @@ -23,8 +23,9 @@ import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -75,12 +76,11 @@ public static void init(String warehouse) { } public static void init(String warehouse, int parallelism) { - StreamExecutionEnvironment sExeEnv = buildStreamEnv(parallelism); - sExeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + // Using `none` to avoid compatibility issues with Flink 1.18-. + StreamExecutionEnvironment sExeEnv = buildStreamEnv(parallelism, "none"); sEnv = StreamTableEnvironment.create(sExeEnv); - bExeEnv = buildBatchEnv(parallelism); - bExeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + bExeEnv = buildBatchEnv(parallelism, "none"); bEnv = StreamTableEnvironment.create(bExeEnv, EnvironmentSettings.inBatchMode()); ReadWriteTableTestUtil.warehouse = warehouse; @@ -95,16 +95,24 @@ public static void init(String warehouse, int parallelism) { bEnv.useCatalog(catalog); } - public static StreamExecutionEnvironment buildStreamEnv(int parallelism) { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + public static StreamExecutionEnvironment buildStreamEnv( + int parallelism, String restartStrategy) { + Configuration configuration = new Configuration(); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY, restartStrategy); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); env.setRuntimeMode(RuntimeExecutionMode.STREAMING); env.enableCheckpointing(100); env.setParallelism(parallelism); return env; } - public static StreamExecutionEnvironment buildBatchEnv(int parallelism) { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + public static StreamExecutionEnvironment buildBatchEnv( + int parallelism, String restartStrategy) { + Configuration configuration = new Configuration(); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY, restartStrategy); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); env.setRuntimeMode(RuntimeExecutionMode.BATCH); env.setParallelism(parallelism); return env; diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java index 74d2d7e1c3437..2266a8484d9d2 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java @@ -31,12 +31,12 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.utils.IOUtils; +import org.apache.paimon.utils.TimeUtils; import com.klarna.hiverunner.HiveShell; import com.klarna.hiverunner.annotations.HiveSQL; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.Path; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; @@ -139,7 +139,9 @@ private void registerHiveCatalog(String catalogName, Map catalog EnvironmentSettings.newInstance().inStreamingMode().build()); sEnv.getConfig() .getConfiguration() - .set(ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, Duration.ofSeconds(1)); + .setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit(Duration.ofSeconds(1))); sEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); tEnv.executeSql( From a8a3751ab9ad62aba6d6ba93d2e7d2dbc4b532e7 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Tue, 26 Nov 2024 14:57:29 +0800 Subject: [PATCH 063/157] [doc] Fix doc with trino engine --- docs/content/engines/presto.md | 321 ------------------------------- docs/content/engines/trino.md | 61 +++--- docs/content/project/download.md | 10 +- 3 files changed, 27 insertions(+), 365 deletions(-) delete mode 100644 docs/content/engines/presto.md diff --git a/docs/content/engines/presto.md b/docs/content/engines/presto.md deleted file mode 100644 index c336226bcf0a9..0000000000000 --- a/docs/content/engines/presto.md +++ /dev/null @@ -1,321 +0,0 @@ ---- -title: "Presto" -weight: 6 -type: docs -aliases: -- /engines/presto.html ---- - - -# Presto - -This documentation is a guide for using Paimon in Presto. - -## Version - -Paimon currently supports Presto 0.236 and above. - -## Preparing Paimon Jar File - -{{< stable >}} - -Download from master: -https://paimon.apache.org/docs/master/project/download/ - -{{< /stable >}} - -{{< unstable >}} - -| Version | Jar | -|-----------------|-----------------------------------------------------------------------------------------------------------------------------------------------------| -| [0.236, 0.268) | [paimon-presto-0.236-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.236/{{< version >}}/) | -| [0.268, 0.273) | [paimon-presto-0.268-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.268/{{< version >}}/) | -| [0.273, latest] | [paimon-presto-0.273-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.273/{{< version >}}/) | - -{{< /unstable >}} - -You can also manually build a bundled jar from the source code. - -To build from the source code, [clone the git repository]({{< presto_github_repo >}}). - -Build presto connector plugin with the following command. - -``` -mvn clean install -DskipTests -``` - -After the packaging is complete, you can choose the corresponding connector based on your own Presto version: - -| Version | Package | -|-----------------|----------------------------------------------------------------------------------| -| [0.236, 0.268) | `./paimon-presto-0.236/target/paimon-presto-0.236-{{< version >}}-plugin.tar.gz` | -| [0.268, 0.273) | `./paimon-presto-0.268/target/paimon-presto-0.268-{{< version >}}-plugin.tar.gz` | -| [0.273, latest] | `./paimon-presto-0.273/target/paimon-presto-0.273-{{< version >}}-plugin.tar.gz` | - -Of course, we also support different versions of Hive and Hadoop. But note that we utilize -Presto-shaded versions of Hive and Hadoop packages to address dependency conflicts. -You can check the following two links to select the appropriate versions of Hive and Hadoop: - -[hadoop-apache2](https://mvnrepository.com/artifact/com.facebook.presto.hadoop/hadoop-apache2) - -[hive-apache](https://mvnrepository.com/artifact/com.facebook.presto.hive/hive-apache) - -Both Hive 2 and 3, as well as Hadoop 2 and 3, are supported. - -For example, if your presto version is 0.274, hive and hadoop version is 2.x, you could run: - -```bash -mvn clean install -DskipTests -am -pl paimon-presto-0.273 -Dpresto.version=0.274 -Dhadoop.apache2.version=2.7.4-9 -Dhive.apache.version=1.2.2-2 -``` - -## Tmp Dir - -Paimon will unzip some jars to the tmp directory for codegen. By default, Presto will use `'/tmp'` as the temporary -directory, but `'/tmp'` may be periodically deleted. - -You can configure this environment variable when Presto starts: -```shell --Djava.io.tmpdir=/path/to/other/tmpdir -``` - -Let Paimon use a secure temporary directory. - -## Configure Paimon Catalog - -### Install Paimon Connector - -```bash -tar -zxf paimon-presto-${PRESTO_VERSION}/target/paimon-presto-${PRESTO_VERSION}-${PAIMON_VERSION}-plugin.tar.gz -C ${PRESTO_HOME}/plugin -``` - -Note that, the variable `PRESTO_VERSION` is module name, must be one of 0.236, 0.268, 0.273. - -### Configuration - -```bash -cd ${PRESTO_HOME} -mkdir -p etc/catalog -``` - -```properties -connector.name=paimon -# set your filesystem path, such as hdfs://namenode01:8020/path and s3://${YOUR_S3_BUCKET}/path -warehouse=${YOUR_FS_PATH} -``` - -If you are using HDFS FileSystem, you will also need to do one more thing: choose one of the following ways to configure your HDFS: - -- set environment variable HADOOP_HOME. -- set environment variable HADOOP_CONF_DIR. -- configure `hadoop-conf-dir` in the properties. - -If you are using S3 FileSystem, you need to add `paimon-s3-${PAIMON_VERSION}.jar` in `${PRESTO_HOME}/plugin/paimon` and additionally configure the following properties in `paimon.properties`: - -```properties -s3.endpoint=${YOUR_ENDPOINTS} -s3.access-key=${YOUR_AK} -s3.secret-key=${YOUR_SK} -``` - -**Query HiveCatalog table:** - -```bash -vim etc/catalog/paimon.properties -``` - -and set the following config: - -```properties -connector.name=paimon -# set your filesystem path, such as hdfs://namenode01:8020/path and s3://${YOUR_S3_BUCKET}/path -warehouse=${YOUR_FS_PATH} -metastore=hive -uri=thrift://${YOUR_HIVE_METASTORE}:9083 -``` - -## Kerberos - -You can configure kerberos keytab file when using KERBEROS authentication in the properties. - -``` -security.kerberos.login.principal=hadoop-user -security.kerberos.login.keytab=/etc/presto/hdfs.keytab -``` - -Keytab files must be distributed to every node in the cluster that runs Presto. - -## Create Schema - -``` -CREATE SCHEMA paimon.test_db; -``` - -## Create Table - -``` -CREATE TABLE paimon.test_db.orders ( - order_key bigint, - order_status varchar, - total_price decimal(18,4), - order_date date -) -WITH ( - file_format = 'ORC', - primary_key = ARRAY['order_key','order_date'], - partitioned_by = ARRAY['order_date'], - bucket = '2', - bucket_key = 'order_key', - changelog_producer = 'input' -) -``` - -## Add Column - -``` -CREATE TABLE paimon.test_db.orders ( - order_key bigint, - orders_tatus varchar, - total_price decimal(18,4), - order_date date -) -WITH ( - file_format = 'ORC', - primary_key = ARRAY['order_key','order_date'], - partitioned_by = ARRAY['order_date'], - bucket = '2', - bucket_key = 'order_key', - changelog_producer = 'input' -) - -ALTER TABLE paimon.test_db.orders ADD COLUMN "shipping_address varchar; -``` - -## Query - -``` -SELECT * FROM paimon.default.MyTable -``` - -## Presto to Paimon type mapping - -This section lists all supported type conversion between Presto and Paimon. -All Presto's data types are available in package ` com.facebook.presto.common.type`. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
Presto Data TypePaimon Data TypeAtomic Type
RowTypeRowTypefalse
MapTypeMapTypefalse
ArrayTypeArrayTypefalse
BooleanTypeBooleanTypetrue
TinyintTypeTinyIntTypetrue
SmallintTypeSmallIntTypetrue
IntegerTypeIntTypetrue
BigintTypeBigIntTypetrue
RealTypeFloatTypetrue
DoubleTypeDoubleTypetrue
CharType(length)CharType(length)true
VarCharType(VarCharType.MAX_LENGTH)VarCharType(VarCharType.MAX_LENGTH)true
VarCharType(length)VarCharType(length), length is less than VarCharType.MAX_LENGTHtrue
DateTypeDateTypetrue
TimestampTypeTimestampTypetrue
DecimalType(precision, scale)DecimalType(precision, scale)true
VarBinaryType(length)VarBinaryType(length)true
TimestampWithTimeZoneTypeLocalZonedTimestampTypetrue
diff --git a/docs/content/engines/trino.md b/docs/content/engines/trino.md index 05fc47729d151..bef10f9d2870d 100644 --- a/docs/content/engines/trino.md +++ b/docs/content/engines/trino.md @@ -30,7 +30,7 @@ This documentation is a guide for using Paimon in Trino. ## Version -Paimon currently supports Trino 420 and above. +Paimon currently supports Trino 440. ## Filesystem @@ -40,26 +40,12 @@ filesystems for Trino on Trino official website. ## Preparing Paimon Jar File -{{< stable >}} - -Download from master: -https://paimon.apache.org/docs/master/project/download/ - -{{< /stable >}} - -{{< unstable >}} - -| Version | Package | -|---------------|-----------------------------------------------------------------------------------------------------------------------------------------------| -| [420, 426] | [paimon-trino-420-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-420/{{< version >}}/) | -| [427, latest] | [paimon-trino-427-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-427/{{< version >}}/) | - -{{< /unstable >}} +[Download]({{< ref "project/download" >}}) You can also manually build a bundled jar from the source code. However, there are a few preliminary steps that need to be taken before compiling: - To build from the source code, [clone the git repository]({{< trino_github_repo >}}). -- Install JDK17 locally, and configure JDK17 as a global environment variable; +- Install JDK21 locally, and configure JDK21 as a global environment variable; Then,you can build bundled jar with the following command: @@ -78,28 +64,17 @@ For example, if you want to use Hadoop 3.3.5-1, you can use the following comman mvn clean install -DskipTests -Dhadoop.apache.version=3.3.5-1 ``` -## Tmp Dir - -Paimon will unzip some jars to the tmp directory for codegen. By default, Trino will use `'/tmp'` as the temporary -directory, but `'/tmp'` may be periodically deleted. - -You can configure this environment variable when Trino starts: -```shell --Djava.io.tmpdir=/path/to/other/tmpdir -``` - -Let Paimon use a secure temporary directory. - ## Configure Paimon Catalog ### Install Paimon Connector ```bash tar -zxf paimon-trino--{{< version >}}-plugin.tar.gz -C ${TRINO_HOME}/plugin ``` -the variable `trino-version` is module name, must be one of 420, 427. -> NOTE: For JDK 17, when Deploying Trino, should add jvm options: `--add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED` + +> NOTE: For JDK 21, when Deploying Trino, should add jvm options: `--add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED` ### Configure + Catalogs are registered by creating a catalog properties file in the etc/catalog directory. For example, create etc/catalog/paimon.properties with the following contents to mount the paimon connector as the paimon catalog: ``` @@ -186,9 +161,6 @@ SELECT * FROM paimon.test_db.orders ``` ## Query with Time Traveling -{{< tabs "time-travel-example" >}} - -{{< tab "version >=420" >}} ```sql -- read the snapshot from specified timestamp @@ -208,10 +180,15 @@ you have a tag named '1' based on snapshot 2, the statement `SELECT * FROM paimo instead of snapshot 1. {{< /hint >}} -{{< /tab >}} +## Insert +``` +INSERT INTO paimon.test_db.orders VALUES (.....); +``` -{{< /tabs >}} +Supports: +- primary key table with fixed bucket. +- non-primary-key table with bucket -1. ## Trino to Paimon type mapping @@ -319,3 +296,15 @@ All Trino's data types are available in package `io.trino.spi.type`. + +## Tmp Dir + +Paimon will unzip some jars to the tmp directory for codegen. By default, Trino will use `'/tmp'` as the temporary +directory, but `'/tmp'` may be periodically deleted. + +You can configure this environment variable when Trino starts: +```shell +-Djava.io.tmpdir=/path/to/other/tmpdir +``` + +Let Paimon use a secure temporary directory. diff --git a/docs/content/project/download.md b/docs/content/project/download.md index 5e49811076a6b..23d0112b09a2b 100644 --- a/docs/content/project/download.md +++ b/docs/content/project/download.md @@ -49,13 +49,8 @@ This documentation is a guide for downloading Paimon Jars. | Hive 2.3 | [paimon-hive-connector-2.3-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-hive-connector-2.3/{{< version >}}/) | | Hive 2.2 | [paimon-hive-connector-2.2-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-hive-connector-2.2/{{< version >}}/) | | Hive 2.1 | [paimon-hive-connector-2.1-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-hive-connector-2.1/{{< version >}}/) | -| Hive 2.1-cdh-6.3 | [paimon-hive-connector-2.1-cdh-6.3-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-hive-connector-2.1-cdh-6.3/{{< version >}}/) | -| Presto 0.236 | [paimon-presto-0.236-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.236/{{< version >}}/) | -| Presto 0.268 | [paimon-presto-0.268-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.268/{{< version >}}/) | -| Presto 0.273 | [paimon-presto-0.273-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.273/{{< version >}}/) | -| Presto SQL 332 | [paimon-prestosql-332-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-prestosql-332/{{< version >}}/) | -| Trino 420 | [paimon-trino-420-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-420/{{< version >}}/) | -| Trino 427 | [paimon-trino-427-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-427/{{< version >}}/) | +| Hive 2.1-cdh-6.3 | [paimon-hive-connector-2.1-cdh-6.3-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-hive-connector-2.1-cdh-6.3/{{< version >}}/) | | +| Trino 440 | [paimon-trino-440-{{< version >}}-plugin.tar.gz](https://repository.apache.org/content/repositories/snapshots/org/apache/paimon/paimon-trino-440/{{< version >}}/) | {{< /unstable >}} @@ -79,7 +74,6 @@ This documentation is a guide for downloading Paimon Jars. | Hive 2.2 | [paimon-hive-connector-2.2-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-hive-connector-2.2/{{< version >}}/paimon-hive-connector-2.2-{{< version >}}.jar) | | Hive 2.1 | [paimon-hive-connector-2.1-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-hive-connector-2.1/{{< version >}}/paimon-hive-connector-2.1-{{< version >}}.jar) | | Hive 2.1-cdh-6.3 | [paimon-hive-connector-2.1-cdh-6.3-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-hive-connector-2.1-cdh-6.3/{{< version >}}/paimon-hive-connector-2.1-cdh-6.3-{{< version >}}.jar) | -| Presto | [Download from master](https://paimon.apache.org/docs/master/project/download/) | | Trino | [Download from master](https://paimon.apache.org/docs/master/project/download/) | {{< /stable >}} From f822bec3b18f371a58e5d5a0d09421748e5ea3ce Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Tue, 26 Nov 2024 15:47:28 +0800 Subject: [PATCH 064/157] [arrow][parquet] Add parquet field id for arrow schema. (#4582) --- .../org/apache/paimon/arrow/ArrowUtils.java | 121 ++++++++++++++---- .../apache/paimon/arrow/ArrowUtilsTest.java | 96 ++++++++++++++ 2 files changed, 192 insertions(+), 25 deletions(-) create mode 100644 paimon-arrow/src/test/java/org/apache/paimon/arrow/ArrowUtilsTest.java diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java index 0cf40ad9faae8..b3925a0a769e1 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java @@ -22,6 +22,7 @@ import org.apache.paimon.arrow.writer.ArrowFieldWriter; import org.apache.paimon.arrow.writer.ArrowFieldWriterFactoryVisitor; import org.apache.paimon.data.Timestamp; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; @@ -48,6 +49,7 @@ import java.io.OutputStream; import java.time.Instant; import java.time.ZoneId; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -56,6 +58,8 @@ /** Utilities for creating Arrow objects. */ public class ArrowUtils { + static final String PARQUET_FIELD_ID = "PARQUET:field_id"; + public static VectorSchemaRoot createVectorSchemaRoot( RowType rowType, BufferAllocator allocator) { return createVectorSchemaRoot(rowType, allocator, true); @@ -69,7 +73,9 @@ public static VectorSchemaRoot createVectorSchemaRoot( f -> toArrowField( allowUpperCase ? f.name() : f.name().toLowerCase(), - f.type())) + f.id(), + f.type(), + 0)) .collect(Collectors.toList()); return VectorSchemaRoot.create(new Schema(fields), allocator); } @@ -78,40 +84,105 @@ public static FieldVector createVector( DataField dataField, BufferAllocator allocator, boolean allowUpperCase) { return toArrowField( allowUpperCase ? dataField.name() : dataField.name().toLowerCase(), - dataField.type()) + dataField.id(), + dataField.type(), + 0) .createVector(allocator); } - public static Field toArrowField(String fieldName, DataType dataType) { + public static Field toArrowField(String fieldName, int fieldId, DataType dataType, int depth) { FieldType fieldType = dataType.accept(ArrowFieldTypeConversion.ARROW_FIELD_TYPE_VISITOR); + fieldType = + new FieldType( + fieldType.isNullable(), + fieldType.getType(), + fieldType.getDictionary(), + Collections.singletonMap(PARQUET_FIELD_ID, String.valueOf(fieldId))); List children = null; if (dataType instanceof ArrayType) { - children = - Collections.singletonList( - toArrowField( - ListVector.DATA_VECTOR_NAME, - ((ArrayType) dataType).getElementType())); + Field field = + toArrowField( + ListVector.DATA_VECTOR_NAME, + fieldId, + ((ArrayType) dataType).getElementType(), + depth + 1); + FieldType typeInner = field.getFieldType(); + field = + new Field( + field.getName(), + new FieldType( + typeInner.isNullable(), + typeInner.getType(), + typeInner.getDictionary(), + Collections.singletonMap( + PARQUET_FIELD_ID, + String.valueOf( + SpecialFields.getArrayElementFieldId( + fieldId, depth + 1)))), + field.getChildren()); + children = Collections.singletonList(field); } else if (dataType instanceof MapType) { MapType mapType = (MapType) dataType; - children = - Collections.singletonList( - new Field( - MapVector.DATA_VECTOR_NAME, - // data vector, key vector and value vector CANNOT be null - new FieldType(false, Types.MinorType.STRUCT.getType(), null), - Arrays.asList( - toArrowField( - MapVector.KEY_NAME, - mapType.getKeyType().notNull()), - toArrowField( - MapVector.VALUE_NAME, - mapType.getValueType().notNull())))); + + Field keyField = + toArrowField( + MapVector.KEY_NAME, fieldId, mapType.getKeyType().notNull(), depth + 1); + FieldType keyType = keyField.getFieldType(); + keyField = + new Field( + keyField.getName(), + new FieldType( + keyType.isNullable(), + keyType.getType(), + keyType.getDictionary(), + Collections.singletonMap( + PARQUET_FIELD_ID, + String.valueOf( + SpecialFields.getMapKeyFieldId( + fieldId, depth + 1)))), + keyField.getChildren()); + + Field valueField = + toArrowField( + MapVector.VALUE_NAME, + fieldId, + mapType.getValueType().notNull(), + depth + 1); + FieldType valueType = valueField.getFieldType(); + valueField = + new Field( + valueField.getName(), + new FieldType( + valueType.isNullable(), + valueType.getType(), + valueType.getDictionary(), + Collections.singletonMap( + PARQUET_FIELD_ID, + String.valueOf( + SpecialFields.getMapValueFieldId( + fieldId, depth + 1)))), + valueField.getChildren()); + + FieldType structType = + new FieldType( + false, + Types.MinorType.STRUCT.getType(), + null, + Collections.singletonMap(PARQUET_FIELD_ID, String.valueOf(fieldId))); + Field mapField = + new Field( + MapVector.DATA_VECTOR_NAME, + // data vector, key vector and value vector CANNOT be null + structType, + Arrays.asList(keyField, valueField)); + + children = Collections.singletonList(mapField); } else if (dataType instanceof RowType) { RowType rowType = (RowType) dataType; - children = - rowType.getFields().stream() - .map(f -> toArrowField(f.name(), f.type())) - .collect(Collectors.toList()); + children = new ArrayList<>(); + for (DataField field : rowType.getFields()) { + children.add(toArrowField(field.name(), field.id(), field.type(), 0)); + } } return new Field(fieldName, fieldType, children); } diff --git a/paimon-arrow/src/test/java/org/apache/paimon/arrow/ArrowUtilsTest.java b/paimon-arrow/src/test/java/org/apache/paimon/arrow/ArrowUtilsTest.java new file mode 100644 index 0000000000000..319df13ba10b3 --- /dev/null +++ b/paimon-arrow/src/test/java/org/apache/paimon/arrow/ArrowUtilsTest.java @@ -0,0 +1,96 @@ +/* + * 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.paimon.arrow; + +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.types.pojo.Field; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Random; + +/** Test for {@link ArrowUtils}. */ +public class ArrowUtilsTest { + + private static final Random RANDOM = new Random(); + + @Test + public void testParquetFieldId() { + Schema.Builder schemaBuilder = Schema.newBuilder(); + schemaBuilder.column("f0", DataTypes.INT()); + schemaBuilder.column("f1", DataTypes.INT()); + schemaBuilder.column("f2", DataTypes.SMALLINT()); + schemaBuilder.column("f3", DataTypes.STRING()); + schemaBuilder.column("f4", DataTypes.DOUBLE()); + schemaBuilder.column("f5", DataTypes.STRING()); + schemaBuilder.column("F6", DataTypes.STRING()); + schemaBuilder.column("f7", DataTypes.BOOLEAN()); + schemaBuilder.column("f8", DataTypes.DATE()); + schemaBuilder.column("f10", DataTypes.TIMESTAMP(6)); + schemaBuilder.column("f11", DataTypes.DECIMAL(7, 2)); + schemaBuilder.column("f12", DataTypes.BYTES()); + schemaBuilder.column("f13", DataTypes.FLOAT()); + schemaBuilder.column("f14", DataTypes.BINARY(10)); + schemaBuilder.column("f15", DataTypes.VARBINARY(10)); + schemaBuilder.column( + "f16", + DataTypes.ARRAY( + DataTypes.ROW( + DataTypes.FIELD(0, "f0", DataTypes.INT()), + DataTypes.FIELD(1, "f1", DataTypes.SMALLINT()), + DataTypes.FIELD(2, "f2", DataTypes.STRING()), + DataTypes.FIELD(3, "f3", DataTypes.DOUBLE()), + DataTypes.FIELD(4, "f4", DataTypes.BOOLEAN()), + DataTypes.FIELD(5, "f5", DataTypes.DATE()), + DataTypes.FIELD(6, "f6", DataTypes.TIMESTAMP(6)), + DataTypes.FIELD(7, "f7", DataTypes.DECIMAL(7, 2)), + DataTypes.FIELD(8, "f8", DataTypes.BYTES()), + DataTypes.FIELD(9, "f9", DataTypes.FLOAT()), + DataTypes.FIELD(10, "f10", DataTypes.BINARY(10))))); + + RowType rowType = schemaBuilder.build().rowType(); + + List fields = + ArrowUtils.createVectorSchemaRoot(rowType, new RootAllocator()) + .getSchema() + .getFields(); + + for (int i = 0; i < 16; i++) { + Assertions.assertThat( + Integer.parseInt( + fields.get(i).getMetadata().get(ArrowUtils.PARQUET_FIELD_ID))) + .isEqualTo(i); + } + + fields = fields.get(15).getChildren().get(0).getChildren(); + for (int i = 16; i < 26; i++) { + Assertions.assertThat( + Integer.parseInt( + fields.get(i - 16) + .getMetadata() + .get(ArrowUtils.PARQUET_FIELD_ID))) + .isEqualTo(i); + } + } +} From 408b78d0dfdd95263436e25a3312b482de714ecd Mon Sep 17 00:00:00 2001 From: Yann Byron Date: Tue, 26 Nov 2024 15:53:06 +0800 Subject: [PATCH 065/157] [spark] adjust paimon spark structure (#4573) --- .github/workflows/utitcase-spark-3.x.yml | 2 +- .github/workflows/utitcase-spark-4.x.yml | 2 +- .github/workflows/utitcase.yml | 2 +- paimon-spark/paimon-spark-3.2/pom.xml | 10 +- paimon-spark/paimon-spark-3.3/pom.xml | 10 +- .../spark/sql/InsertOverwriteTest.scala | 4 +- paimon-spark/paimon-spark-3.4/pom.xml | 10 +- paimon-spark/paimon-spark-3.5/pom.xml | 10 +- paimon-spark/paimon-spark-4.0/pom.xml | 10 +- paimon-spark/paimon-spark-common/pom.xml | 68 ------- ...Row.java => AbstractSparkInternalRow.java} | 103 ++-------- .../apache/paimon/spark/DataConverter.java | 117 ++++++++++++ .../apache/paimon/spark/SparkArrayData.java | 172 ----------------- .../paimon/spark/SparkGenericCatalog.java | 5 +- .../spark/PaimonPartitionManagement.scala | 3 +- .../paimon/spark/PaimonPartitionReader.scala | 1 + .../spark/PaimonPartitionReaderFactory.scala | 7 +- .../paimon/spark/PaimonStatistics.scala | 7 +- .../spark/aggregate/LocalAggregator.scala | 5 +- .../expressions/ExpressionHelper.scala | 11 +- .../MergePaimonScalarSubqueriesBase.scala | 4 +- .../spark/commands/BucketProcessor.scala | 4 +- .../spark/commands/MergeIntoPaimonTable.scala | 13 +- .../commands/UpdatePaimonTableCommand.scala | 7 +- .../paimon/spark/data/SparkArrayData.scala | 118 ++++++++++++ .../paimon/spark/data/SparkInternalRow.scala | 36 ++++ .../PaimonSparkSessionExtensions.scala | 6 +- ...tractPaimonSparkSqlExtensionsParser.scala} | 4 +- .../catalog/PaimonCatalogUtils.scala | 25 +-- .../spark/sql/paimon/ReflectUtils.scala | 43 +++++ .../spark/sql/paimon/shims/SparkShim.scala | 61 ++++++ .../sql/paimon/shims/SparkShimLoader.scala | 43 +++++ paimon-spark/paimon-spark-ut/pom.xml | 180 ++++++++++++++++++ .../spark/SparkCatalogWithHiveTest.java | 0 .../paimon/spark/SparkFileIndexITCase.java | 0 .../spark/SparkFilterConverterTest.java | 0 .../paimon/spark/SparkGenericCatalogTest.java | 0 .../SparkGenericCatalogWithHiveTest.java | 0 .../paimon/spark/SparkInternalRowTest.java | 6 +- .../apache/paimon/spark/SparkReadITCase.java | 0 .../paimon/spark/SparkReadTestBase.java | 0 .../apache/paimon/spark/SparkS3ITCase.java | 0 .../spark/SparkSchemaEvolutionITCase.java | 0 .../paimon/spark/SparkTimeTravelITCase.java | 0 .../SparkTimeTravelWithDataFrameITCase.java | 0 .../apache/paimon/spark/SparkTypeTest.java | 0 .../apache/paimon/spark/SparkWriteITCase.java | 0 .../spark/SparkWriteWithKyroITCase.java | 0 .../extensions/CallStatementParserTest.java | 0 .../org.junit.jupiter.api.extension.Extension | 0 .../src/test/resources/hive-site.xml | 0 .../src/test/resources/log4j2-test.properties | 0 .../paimon/spark/PaimonCDCSourceTest.scala | 0 .../paimon/spark/PaimonCommitTest.scala | 0 .../paimon/spark/PaimonHiveTestBase.scala | 2 +- .../apache/paimon/spark/PaimonSinkTest.scala | 0 .../paimon/spark/PaimonSourceTest.scala | 0 .../paimon/spark/PaimonSparkTestBase.scala | 39 +++- .../apache/paimon/spark/PaimonTableTest.scala | 0 .../apache/paimon/spark/ScanHelperTest.scala | 3 +- .../procedure/AlterBranchProcedureTest.scala | 0 .../spark/procedure/BranchProcedureTest.scala | 0 .../CompactManifestProcedureTest.scala | 0 .../procedure/CompactProcedureTestBase.scala | 0 .../CreateAndDeleteTagProcedureTest.scala | 0 .../CreateTagFromTimestampProcedureTest.scala | 0 .../ExpirePartitionsProcedureTest.scala | 0 .../ExpireSnapshotsProcedureTest.scala | 0 .../procedure/ExpireTagsProcedureTest.scala | 0 .../procedure/FastForwardProcedureTest.scala | 0 .../MarkPartitionDoneProcedureTest.scala | 0 .../MigrateDatabaseProcedureTest.scala | 0 .../procedure/MigrateFileProcedureTest.scala | 0 .../procedure/MigrateTableProcedureTest.scala | 0 .../spark/procedure/ProcedureTestBase.scala | 0 .../RemoveOrphanFilesProcedureTest.scala | 0 .../procedure/ReplaceTagProcedureTest.scala | 0 .../procedure/RollbackProcedureTest.scala | 0 .../spark/sql/AnalyzeTableTestBase.scala | 14 +- .../spark/sql/BucketedTableQueryTest.scala | 8 +- .../apache/paimon/spark/sql/DDLTestBase.scala | 14 +- .../sql/DDLWithHiveCatalogTestBase.scala | 2 +- .../paimon/spark/sql/DataFrameWriteTest.scala | 0 .../spark/sql/DeleteFromTableTestBase.scala | 0 .../paimon/spark/sql/DeletionVectorTest.scala | 2 +- .../paimon/spark/sql/DescribeTableTest.scala | 0 ...leUnnecessaryPaimonBucketedScanSuite.scala | 6 +- .../spark/sql/DynamicBucketTableTest.scala | 0 .../sql/InsertOverwriteTableTestBase.scala | 4 +- .../spark/sql/LookupCompactionTest.scala | 0 .../sql/MergeIntoNotMatchedBySourceTest.scala | 0 .../spark/sql/MergeIntoTableTestBase.scala | 0 .../paimon/spark/sql/ObjectTableTest.scala | 0 .../PaimonCompositePartitionKeyTestBase.scala | 0 .../paimon/spark/sql/PaimonFunctionTest.scala | 0 .../paimon/spark/sql/PaimonMetricTest.scala | 0 .../sql/PaimonOptimizationTestBase.scala | 35 ++-- .../paimon/spark/sql/PaimonOptionTest.scala | 26 +-- .../sql/PaimonPartitionManagementTest.scala | 0 .../paimon/spark/sql/PaimonPushDownTest.scala | 0 .../paimon/spark/sql/PaimonQueryTest.scala | 2 +- .../spark/sql/PaimonShowColumnsTestBase.scala | 0 .../spark/sql/PaimonSystemTableTest.scala | 0 .../spark/sql/PaimonTagDdlTestBase.scala | 0 .../paimon/spark/sql/PaimonViewTestBase.scala | 0 .../spark/sql/PushDownAggregatesTest.scala | 0 .../spark/sql/SparkVersionSupport.scala | 0 .../spark/sql/TableValuedFunctionsTest.scala | 0 .../spark/sql/UpdateTableTestBase.scala | 0 .../paimon/spark/sql/WithTableOptions.scala | 0 .../org/apache/spark/sql}/paimon/Utils.scala | 12 +- paimon-spark/paimon-spark3-common/pom.xml | 32 +++- ...rg.apache.spark.sql.paimon.shims.SparkShim | 16 ++ .../PaimonSpark3SqlExtensionsParser.scala | 25 +++ .../paimon/spark/data/Spark3ArrayData.scala} | 15 +- .../spark/data/Spark3InternalRow.scala} | 16 +- .../{shims.scala => shims/Spark3Shim.scala} | 54 +++--- paimon-spark/paimon-spark4-common/pom.xml | 51 ++++- ...rg.apache.spark.sql.paimon.shims.SparkShim | 16 ++ .../PaimonSpark4SqlExtensionsParser.scala | 28 +++ .../paimon/spark/data/Spark4ArrayData.scala} | 13 +- .../paimon/spark/data/Spark4InternalRow.scala | 28 +++ .../org/apache/spark/sql/paimon/shims.scala | 86 --------- .../spark/sql/paimon/shims/Spark4Shim.scala | 69 +++++++ paimon-spark/pom.xml | 38 ++-- 125 files changed, 1138 insertions(+), 627 deletions(-) rename paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/{SparkInternalRow.java => AbstractSparkInternalRow.java} (67%) create mode 100644 paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/DataConverter.java delete mode 100644 paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkArrayData.java create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala rename paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/{PaimonSparkSqlExtensionsParser.scala => AbstractPaimonSparkSqlExtensionsParser.scala} (98%) create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/ReflectUtils.scala create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala create mode 100644 paimon-spark/paimon-spark-ut/pom.xml rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java (95%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkReadITCase.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkTimeTravelITCase.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkTimeTravelWithDataFrameITCase.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkTypeTest.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/SparkWriteWithKyroITCase.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/resources/hive-site.xml (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/resources/log4j2-test.properties (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala (98%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala (79%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/PaimonTableTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala (97%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/FastForwardProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/MigrateFileProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/MigrateTableProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala (97%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala (97%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala (97%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala (99%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala (99%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala (97%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala (99%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/ObjectTableTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTestBase.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonMetricTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala (90%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala (91%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonPartitionManagementTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala (99%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonShowColumnsTestBase.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala (100%) rename paimon-spark/{paimon-spark-common => paimon-spark-ut}/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala (100%) rename paimon-spark/{paimon-spark-common/src/test/scala/org/apache/spark => paimon-spark-ut/src/test/scala/org/apache/spark/sql}/paimon/Utils.scala (74%) create mode 100644 paimon-spark/paimon-spark3-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim create mode 100644 paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark3SqlExtensionsParser.scala rename paimon-spark/{paimon-spark-3.2/src/test/scala/org/apache/spark/paimon/Utils.scala => paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3ArrayData.scala} (73%) rename paimon-spark/{paimon-spark-3.3/src/test/scala/org/apache/spark/paimon/Utils.scala => paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRow.scala} (73%) rename paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/{shims.scala => shims/Spark3Shim.scala} (51%) create mode 100644 paimon-spark/paimon-spark4-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim create mode 100644 paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala rename paimon-spark/{paimon-spark-3.4/src/test/scala/org/apache/spark/paimon/Utils.scala => paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4ArrayData.scala} (72%) create mode 100644 paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRow.scala delete mode 100644 paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala create mode 100644 paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala diff --git a/.github/workflows/utitcase-spark-3.x.yml b/.github/workflows/utitcase-spark-3.x.yml index 5edcfe49007ac..2d3df5f4d0053 100644 --- a/.github/workflows/utitcase-spark-3.x.yml +++ b/.github/workflows/utitcase-spark-3.x.yml @@ -54,7 +54,7 @@ jobs: jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" test_modules="" - for suffix in common_2.12 3.5 3.4 3.3 3.2; do + for suffix in ut 3.5 3.4 3.3 3.2; do test_modules+="org.apache.paimon:paimon-spark-${suffix}," done test_modules="${test_modules%,}" diff --git a/.github/workflows/utitcase-spark-4.x.yml b/.github/workflows/utitcase-spark-4.x.yml index 7fbac23dda4fc..c58fd7c03be27 100644 --- a/.github/workflows/utitcase-spark-4.x.yml +++ b/.github/workflows/utitcase-spark-4.x.yml @@ -54,7 +54,7 @@ jobs: jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" test_modules="" - for suffix in common_2.13 4.0; do + for suffix in ut 4.0; do test_modules+="org.apache.paimon:paimon-spark-${suffix}," done test_modules="${test_modules%,}" diff --git a/.github/workflows/utitcase.yml b/.github/workflows/utitcase.yml index bde67cb4c2039..8aa33f5b8218e 100644 --- a/.github/workflows/utitcase.yml +++ b/.github/workflows/utitcase.yml @@ -54,7 +54,7 @@ jobs: jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" test_modules="!paimon-e2e-tests," - for suffix in 3.5 3.4 3.3 3.2 common_2.12; do + for suffix in 3.5 3.4 3.3 3.2 ut; do test_modules+="!org.apache.paimon:paimon-spark-${suffix}," done test_modules="${test_modules%,}" diff --git a/paimon-spark/paimon-spark-3.2/pom.xml b/paimon-spark/paimon-spark-3.2/pom.xml index 626bb5bae8330..957319b47dab8 100644 --- a/paimon-spark/paimon-spark-3.2/pom.xml +++ b/paimon-spark/paimon-spark-3.2/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark3-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark3-common diff --git a/paimon-spark/paimon-spark-3.3/pom.xml b/paimon-spark/paimon-spark-3.3/pom.xml index 689e4131ccd9e..0a390d9267894 100644 --- a/paimon-spark/paimon-spark-3.3/pom.xml +++ b/paimon-spark/paimon-spark-3.3/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark3-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark3-common diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTest.scala b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTest.scala index 304b814b33d38..219d57c865c8c 100644 --- a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTest.scala +++ b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTest.scala @@ -248,7 +248,7 @@ class InsertOverwriteTest extends PaimonSparkTestBase { spark.sql("SELECT * FROM T ORDER BY a, b"), Row(1, 3, "3") :: Row(2, 4, "4") :: Nil) - withSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { + withSparkSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { // dynamic overwrite the a=1 partition spark.sql("INSERT OVERWRITE T VALUES (1, 5, '5'), (1, 7, '7')") checkAnswer( @@ -289,7 +289,7 @@ class InsertOverwriteTest extends PaimonSparkTestBase { "ptv2", 22) :: Nil) - withSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { + withSparkSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { // dynamic overwrite the pt2=22 partition spark.sql( "INSERT OVERWRITE T PARTITION (pt2 = 22) VALUES (3, 'c2', 'ptv1'), (4, 'd2', 'ptv3')") diff --git a/paimon-spark/paimon-spark-3.4/pom.xml b/paimon-spark/paimon-spark-3.4/pom.xml index d1ded508a927c..0f4cb30e4f7f6 100644 --- a/paimon-spark/paimon-spark-3.4/pom.xml +++ b/paimon-spark/paimon-spark-3.4/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark3-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark3-common diff --git a/paimon-spark/paimon-spark-3.5/pom.xml b/paimon-spark/paimon-spark-3.5/pom.xml index 92803cda540e4..1b9c968889083 100644 --- a/paimon-spark/paimon-spark-3.5/pom.xml +++ b/paimon-spark/paimon-spark-3.5/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark3-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark3-common diff --git a/paimon-spark/paimon-spark-4.0/pom.xml b/paimon-spark/paimon-spark-4.0/pom.xml index 9f819f820ce2c..8e7d166dc55b5 100644 --- a/paimon-spark/paimon-spark-4.0/pom.xml +++ b/paimon-spark/paimon-spark-4.0/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark4-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark4-common diff --git a/paimon-spark/paimon-spark-common/pom.xml b/paimon-spark/paimon-spark-common/pom.xml index 1cfc53f42d489..052c4c4265fc4 100644 --- a/paimon-spark/paimon-spark-common/pom.xml +++ b/paimon-spark/paimon-spark-common/pom.xml @@ -38,18 +38,6 @@ under the License. - - org.apache.paimon - ${paimon-sparkx-common} - ${project.version} - - - * - * - - - - org.apache.spark spark-sql_${scala.binary.version} @@ -72,46 +60,6 @@ under the License. org.apache.paimon paimon-bundle - - - - - org.apache.spark - spark-sql_${scala.binary.version} - ${spark.version} - tests - test - - - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${spark.version} - tests - test - - - - org.apache.spark - spark-core_${scala.binary.version} - ${spark.version} - tests - test - - - - org.apache.spark - spark-hive_${scala.binary.version} - ${spark.version} - test - - - - org.apache.spark - spark-avro_${scala.binary.version} - ${spark.version} - test - @@ -130,7 +78,6 @@ under the License. org.apache.paimon:paimon-bundle - org.apache.paimon:${paimon-sparkx-common} @@ -155,21 +102,6 @@ under the License. src/main/antlr4 - - - - org.apache.maven.plugins - maven-jar-plugin - - - prepare-test-jar - test-compile - - test-jar - - - - diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRow.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/AbstractSparkInternalRow.java similarity index 67% rename from paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRow.java rename to paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/AbstractSparkInternalRow.java index 147c6c2d77c84..28604a6d62933 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRow.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/AbstractSparkInternalRow.java @@ -18,24 +18,15 @@ package org.apache.paimon.spark; -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.data.Timestamp; -import org.apache.paimon.spark.util.shim.TypeUtils; +import org.apache.paimon.spark.data.SparkInternalRow; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeChecks; -import org.apache.paimon.types.IntType; -import org.apache.paimon.types.MapType; -import org.apache.paimon.types.MultisetType; import org.apache.paimon.types.RowType; -import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.BooleanType; @@ -61,19 +52,23 @@ import java.util.Objects; +import static org.apache.paimon.spark.DataConverter.fromPaimon; import static org.apache.paimon.utils.InternalRowUtils.copyInternalRow; -/** Spark {@link org.apache.spark.sql.catalyst.InternalRow} to wrap {@link InternalRow}. */ -public class SparkInternalRow extends org.apache.spark.sql.paimon.shims.InternalRow { +/** + * An abstract {@link SparkInternalRow} that overwrite all the common methods in spark3 and spark4. + */ +public abstract class AbstractSparkInternalRow extends SparkInternalRow { - private final RowType rowType; + protected RowType rowType; - private InternalRow row; + protected InternalRow row; - public SparkInternalRow(RowType rowType) { + public AbstractSparkInternalRow(RowType rowType) { this.rowType = rowType; } + @Override public SparkInternalRow replace(InternalRow row) { this.row = row; return this; @@ -96,7 +91,7 @@ public void update(int i, Object value) { @Override public org.apache.spark.sql.catalyst.InternalRow copy() { - return new SparkInternalRow(rowType).replace(copyInternalRow(row, rowType)); + return SparkInternalRow.create(rowType).replace(copyInternalRow(row, rowType)); } @Override @@ -255,7 +250,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - SparkInternalRow that = (SparkInternalRow) o; + AbstractSparkInternalRow that = (AbstractSparkInternalRow) o; return Objects.equals(rowType, that.rowType) && Objects.equals(row, that.row); } @@ -263,78 +258,4 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(rowType, row); } - - // ================== static methods ========================================= - - public static Object fromPaimon(Object o, DataType type) { - if (o == null) { - return null; - } - switch (type.getTypeRoot()) { - case TIMESTAMP_WITHOUT_TIME_ZONE: - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return fromPaimon((Timestamp) o); - case CHAR: - case VARCHAR: - return fromPaimon((BinaryString) o); - case DECIMAL: - return fromPaimon((org.apache.paimon.data.Decimal) o); - case ARRAY: - return fromPaimon((InternalArray) o, (ArrayType) type); - case MAP: - case MULTISET: - return fromPaimon((InternalMap) o, type); - case ROW: - return fromPaimon((InternalRow) o, (RowType) type); - default: - return o; - } - } - - public static UTF8String fromPaimon(BinaryString string) { - return UTF8String.fromBytes(string.toBytes()); - } - - public static Decimal fromPaimon(org.apache.paimon.data.Decimal decimal) { - return Decimal.apply(decimal.toBigDecimal()); - } - - public static org.apache.spark.sql.catalyst.InternalRow fromPaimon( - InternalRow row, RowType rowType) { - return new SparkInternalRow(rowType).replace(row); - } - - public static long fromPaimon(Timestamp timestamp) { - if (TypeUtils.treatPaimonTimestampTypeAsSparkTimestampType()) { - return DateTimeUtils.fromJavaTimestamp(timestamp.toSQLTimestamp()); - } else { - return timestamp.toMicros(); - } - } - - public static ArrayData fromPaimon(InternalArray array, ArrayType arrayType) { - return fromPaimonArrayElementType(array, arrayType.getElementType()); - } - - private static ArrayData fromPaimonArrayElementType(InternalArray array, DataType elementType) { - return new SparkArrayData(elementType).replace(array); - } - - public static MapData fromPaimon(InternalMap map, DataType mapType) { - DataType keyType; - DataType valueType; - if (mapType instanceof MapType) { - keyType = ((MapType) mapType).getKeyType(); - valueType = ((MapType) mapType).getValueType(); - } else if (mapType instanceof MultisetType) { - keyType = ((MultisetType) mapType).getElementType(); - valueType = new IntType(); - } else { - throw new UnsupportedOperationException("Unsupported type: " + mapType); - } - - return new ArrayBasedMapData( - fromPaimonArrayElementType(map.keyArray(), keyType), - fromPaimonArrayElementType(map.valueArray(), valueType)); - } } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/DataConverter.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/DataConverter.java new file mode 100644 index 0000000000000..0b5ea899476eb --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/DataConverter.java @@ -0,0 +1,117 @@ +/* + * 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.paimon.spark; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.spark.data.SparkArrayData; +import org.apache.paimon.spark.data.SparkInternalRow; +import org.apache.paimon.spark.util.shim.TypeUtils; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.MultisetType; +import org.apache.paimon.types.RowType; + +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; + +/** A data converter that convert Paimon data to Spark Data. */ +public class DataConverter { + + public static Object fromPaimon(Object o, DataType type) { + if (o == null) { + return null; + } + switch (type.getTypeRoot()) { + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return fromPaimon((Timestamp) o); + case CHAR: + case VARCHAR: + return fromPaimon((BinaryString) o); + case DECIMAL: + return fromPaimon((org.apache.paimon.data.Decimal) o); + case ARRAY: + return fromPaimon((InternalArray) o, (ArrayType) type); + case MAP: + case MULTISET: + return fromPaimon((InternalMap) o, type); + case ROW: + return fromPaimon((InternalRow) o, (RowType) type); + default: + return o; + } + } + + public static UTF8String fromPaimon(BinaryString string) { + return UTF8String.fromBytes(string.toBytes()); + } + + public static Decimal fromPaimon(org.apache.paimon.data.Decimal decimal) { + return Decimal.apply(decimal.toBigDecimal()); + } + + public static org.apache.spark.sql.catalyst.InternalRow fromPaimon( + InternalRow row, RowType rowType) { + return SparkInternalRow.create(rowType).replace(row); + } + + public static long fromPaimon(Timestamp timestamp) { + if (TypeUtils.treatPaimonTimestampTypeAsSparkTimestampType()) { + return DateTimeUtils.fromJavaTimestamp(timestamp.toSQLTimestamp()); + } else { + return timestamp.toMicros(); + } + } + + public static ArrayData fromPaimon(InternalArray array, ArrayType arrayType) { + return fromPaimonArrayElementType(array, arrayType.getElementType()); + } + + private static ArrayData fromPaimonArrayElementType(InternalArray array, DataType elementType) { + return SparkArrayData.create(elementType).replace(array); + } + + public static MapData fromPaimon(InternalMap map, DataType mapType) { + DataType keyType; + DataType valueType; + if (mapType instanceof MapType) { + keyType = ((MapType) mapType).getKeyType(); + valueType = ((MapType) mapType).getValueType(); + } else if (mapType instanceof MultisetType) { + keyType = ((MultisetType) mapType).getElementType(); + valueType = new IntType(); + } else { + throw new UnsupportedOperationException("Unsupported type: " + mapType); + } + + return new ArrayBasedMapData( + fromPaimonArrayElementType(map.keyArray(), keyType), + fromPaimonArrayElementType(map.valueArray(), valueType)); + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkArrayData.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkArrayData.java deleted file mode 100644 index 9934047a18251..0000000000000 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkArrayData.java +++ /dev/null @@ -1,172 +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.paimon.spark; - -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.types.ArrayType; -import org.apache.paimon.types.BigIntType; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypeChecks; -import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.InternalRowUtils; - -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.SpecializedGettersReader; -import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.MapData; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.unsafe.types.CalendarInterval; -import org.apache.spark.unsafe.types.UTF8String; - -import static org.apache.paimon.spark.SparkInternalRow.fromPaimon; -import static org.apache.paimon.utils.InternalRowUtils.copyArray; - -/** Spark {@link ArrayData} to wrap Paimon {@link InternalArray}. */ -public class SparkArrayData extends org.apache.spark.sql.paimon.shims.ArrayData { - - private final DataType elementType; - - private InternalArray array; - - public SparkArrayData(DataType elementType) { - this.elementType = elementType; - } - - public SparkArrayData replace(InternalArray array) { - this.array = array; - return this; - } - - @Override - public int numElements() { - return array.size(); - } - - @Override - public ArrayData copy() { - return new SparkArrayData(elementType).replace(copyArray(array, elementType)); - } - - @Override - public Object[] array() { - Object[] objects = new Object[numElements()]; - for (int i = 0; i < objects.length; i++) { - objects[i] = fromPaimon(InternalRowUtils.get(array, i, elementType), elementType); - } - return objects; - } - - @Override - public void setNullAt(int i) { - throw new UnsupportedOperationException(); - } - - @Override - public void update(int i, Object value) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isNullAt(int ordinal) { - return array.isNullAt(ordinal); - } - - @Override - public boolean getBoolean(int ordinal) { - return array.getBoolean(ordinal); - } - - @Override - public byte getByte(int ordinal) { - return array.getByte(ordinal); - } - - @Override - public short getShort(int ordinal) { - return array.getShort(ordinal); - } - - @Override - public int getInt(int ordinal) { - return array.getInt(ordinal); - } - - @Override - public long getLong(int ordinal) { - if (elementType instanceof BigIntType) { - return array.getLong(ordinal); - } - - return getTimestampMicros(ordinal); - } - - private long getTimestampMicros(int ordinal) { - return fromPaimon(array.getTimestamp(ordinal, DataTypeChecks.getPrecision(elementType))); - } - - @Override - public float getFloat(int ordinal) { - return array.getFloat(ordinal); - } - - @Override - public double getDouble(int ordinal) { - return array.getDouble(ordinal); - } - - @Override - public Decimal getDecimal(int ordinal, int precision, int scale) { - return fromPaimon(array.getDecimal(ordinal, precision, scale)); - } - - @Override - public UTF8String getUTF8String(int ordinal) { - return fromPaimon(array.getString(ordinal)); - } - - @Override - public byte[] getBinary(int ordinal) { - return array.getBinary(ordinal); - } - - @Override - public CalendarInterval getInterval(int ordinal) { - throw new UnsupportedOperationException(); - } - - @Override - public InternalRow getStruct(int ordinal, int numFields) { - return fromPaimon(array.getRow(ordinal, numFields), (RowType) elementType); - } - - @Override - public ArrayData getArray(int ordinal) { - return fromPaimon(array.getArray(ordinal), (ArrayType) elementType); - } - - @Override - public MapData getMap(int ordinal) { - return fromPaimon(array.getMap(ordinal), elementType); - } - - @Override - public Object get(int ordinal, org.apache.spark.sql.types.DataType dataType) { - return SpecializedGettersReader.read(this, ordinal, dataType, true, true); - } -} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java index d4b712fcb8ee0..9957f0cdf91ff 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java @@ -52,7 +52,7 @@ import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.internal.SessionState; import org.apache.spark.sql.internal.StaticSQLConf; -import org.apache.spark.sql.paimon.shims; +import org.apache.spark.sql.paimon.shims.SparkShimLoader; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.slf4j.Logger; @@ -203,7 +203,8 @@ public Table createTable( return sparkCatalog.createTable(ident, schema, partitions, properties); } else { // delegate to the session catalog - return shims.createTable(asTableCatalog(), ident, schema, partitions, properties); + return SparkShimLoader.getSparkShim() + .createTable(asTableCatalog(), ident, schema, partitions, properties); } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala index 54970bfe3cb21..9a305ca59a0f1 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala @@ -21,6 +21,7 @@ package org.apache.paimon.spark import org.apache.paimon.CoreOptions import org.apache.paimon.metastore.MetastoreClient import org.apache.paimon.operation.FileStoreCommit +import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.table.FileStoreTable import org.apache.paimon.table.sink.BatchWriteBuilder import org.apache.paimon.types.RowType @@ -116,7 +117,7 @@ trait PaimonPartitionManagement extends SupportsAtomicPartitionManagement { s"the partition schema '${partitionSchema.sql}'." ) table.newReadBuilder.newScan.listPartitions.asScala - .map(binaryRow => SparkInternalRow.fromPaimon(binaryRow, partitionRowType)) + .map(binaryRow => DataConverter.fromPaimon(binaryRow, partitionRowType)) .filter( sparkInternalRow => { partitionCols.zipWithIndex diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala index fa9072df31499..526178e28ec33 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala @@ -20,6 +20,7 @@ package org.apache.paimon.spark import org.apache.paimon.data.{InternalRow => PaimonInternalRow} import org.apache.paimon.reader.RecordReader +import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.table.source.{DataSplit, Split} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReaderFactory.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReaderFactory.scala index 94de0bec3b506..59b07a7944811 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReaderFactory.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReaderFactory.scala @@ -18,10 +18,11 @@ package org.apache.paimon.spark -import org.apache.paimon.data +import org.apache.paimon.data.{InternalRow => PaimonInternalRow} import org.apache.paimon.disk.IOManager import org.apache.paimon.reader.RecordReader import org.apache.paimon.spark.SparkUtils.createIOManager +import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.table.source.{ReadBuilder, Split} import org.apache.paimon.types.RowType @@ -45,13 +46,13 @@ case class PaimonPartitionReaderFactory( val dataFields = new JList(readBuilder.readType().getFields) dataFields.addAll(metadataColumns.map(_.toPaimonDataField).asJava) val rowType = new RowType(dataFields) - new SparkInternalRow(rowType) + SparkInternalRow.create(rowType) } override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { partition match { case paimonInputPartition: PaimonInputPartition => - val readFunc: Split => RecordReader[data.InternalRow] = + val readFunc: Split => RecordReader[PaimonInternalRow] = (split: Split) => readBuilder.newRead().withIOManager(ioManager).createReader(split) PaimonPartitionReader(readFunc, paimonInputPartition, row, metadataColumns) case _ => diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonStatistics.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonStatistics.scala index 28af4ac0a4fdb..8dd4649330327 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonStatistics.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonStatistics.scala @@ -18,6 +18,7 @@ package org.apache.paimon.spark +import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.stats.ColStats import org.apache.paimon.types.{DataField, DataType, RowType} @@ -118,8 +119,10 @@ object PaimonColumnStats { def apply(dateType: DataType, paimonColStats: ColStats[_]): PaimonColumnStats = { PaimonColumnStats( paimonColStats.nullCount, - Optional.ofNullable(SparkInternalRow.fromPaimon(paimonColStats.min().orElse(null), dateType)), - Optional.ofNullable(SparkInternalRow.fromPaimon(paimonColStats.max().orElse(null), dateType)), + Optional.ofNullable( + DataConverter + .fromPaimon(paimonColStats.min().orElse(null), dateType)), + Optional.ofNullable(DataConverter.fromPaimon(paimonColStats.max().orElse(null), dateType)), paimonColStats.distinctCount, paimonColStats.avgLen, paimonColStats.maxLen diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala index cd9718cf44eb2..41e7fd3c3ce90 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala @@ -20,7 +20,8 @@ package org.apache.paimon.spark.aggregate import org.apache.paimon.data.BinaryRow import org.apache.paimon.manifest.PartitionEntry -import org.apache.paimon.spark.{SparkInternalRow, SparkTypeUtils} +import org.apache.paimon.spark.SparkTypeUtils +import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.table.{DataTable, Table} import org.apache.paimon.utils.{InternalRowUtils, ProjectedRow} @@ -104,7 +105,7 @@ class LocalAggregator(table: Table) { ProjectedRow.from(requiredGroupByIndexMapping.toArray).replaceRow(partitionRow) // `ProjectedRow` does not support `hashCode`, so do a deep copy val genericRow = InternalRowUtils.copyInternalRow(projectedRow, partitionType) - new SparkInternalRow(partitionType).replace(genericRow) + SparkInternalRow.create(partitionType).replace(genericRow) } def update(partitionEntry: PartitionEntry): Unit = { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala index c008819fb0cc8..d4010ea338116 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala @@ -23,12 +23,13 @@ import org.apache.paimon.spark.SparkFilterConverter import org.apache.paimon.spark.catalyst.Compatibility import org.apache.paimon.types.RowType +import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.PaimonUtils.{normalizeExprs, translateFilter} -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, Cast, Expression, GetStructField, Literal, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.paimon.shims.SparkShimLoader import org.apache.spark.sql.types.{DataType, NullType} /** An expression helper. */ @@ -36,6 +37,14 @@ trait ExpressionHelper extends PredicateHelper { import ExpressionHelper._ + def toColumn(expr: Expression): Column = { + SparkShimLoader.getSparkShim.column(expr) + } + + def toExpression(spark: SparkSession, col: Column): Expression = { + SparkShimLoader.getSparkShim.convertToExpression(spark, col) + } + protected def resolveExpression( spark: SparkSession)(expr: Expression, plan: LogicalPlan): Expression = { if (expr.resolved) { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala index b0b1a76e7a1f1..3428ed89f0047 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.paimon.shims +import org.apache.spark.sql.paimon.shims.SparkShimLoader import org.apache.spark.sql.types.{DataType, StructType} import scala.collection.mutable.ArrayBuffer @@ -344,7 +344,7 @@ trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHe val Seq(newPlanSupportsHashAggregate, cachedPlanSupportsHashAggregate) = aggregateExpressionsSeq.zip(groupByExpressionSeq).map { case (aggregateExpressions, groupByExpressions) => - shims.Aggregate.supportsHashAggregate( + SparkShimLoader.getSparkShim.supportsHashAggregate( aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes), groupByExpressions) } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala index f252b3bb130b4..57a8a8e4abfd0 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala @@ -22,7 +22,7 @@ import org.apache.paimon.crosspartition.{GlobalIndexAssigner, KeyPartOrRow} import org.apache.paimon.data.{BinaryRow, GenericRow, InternalRow => PaimonInternalRow, JoinedRow} import org.apache.paimon.disk.IOManager import org.apache.paimon.index.HashBucketAssigner -import org.apache.paimon.spark.{SparkInternalRow, SparkRow} +import org.apache.paimon.spark.{DataConverter, SparkRow} import org.apache.paimon.spark.SparkUtils.createIOManager import org.apache.paimon.spark.util.EncoderUtils import org.apache.paimon.table.FileStoreTable @@ -179,7 +179,7 @@ class GlobalIndexAssignerIterator( extraRow.setField(1, bucket) queue.enqueue( encoderGroup.internalToRow( - SparkInternalRow.fromPaimon(new JoinedRow(row, extraRow), rowType))) + DataConverter.fromPaimon(new JoinedRow(row, extraRow), rowType))) } ) rowIterator.foreach { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala index f557a0cf38ee0..52e704172fc82 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.functions.{col, lit, monotonically_increasing_id, sum} -import org.apache.spark.sql.paimon.shims.ExpressionUtils.{column, convertToExpression} import org.apache.spark.sql.types.{ByteType, StructField, StructType} import scala.collection.mutable @@ -153,12 +152,12 @@ case class MergeIntoPaimonTable( } if (hasUpdate(matchedActions)) { touchedFilePathsSet ++= findTouchedFiles( - targetDS.join(sourceDS, column(mergeCondition), "inner"), + targetDS.join(sourceDS, toColumn(mergeCondition), "inner"), sparkSession) } if (hasUpdate(notMatchedBySourceActions)) { touchedFilePathsSet ++= findTouchedFiles( - targetDS.join(sourceDS, column(mergeCondition), "left_anti"), + targetDS.join(sourceDS, toColumn(mergeCondition), "left_anti"), sparkSession) } @@ -200,7 +199,7 @@ case class MergeIntoPaimonTable( val sourceDS = createDataset(sparkSession, sourceTable) .withColumn(SOURCE_ROW_COL, lit(true)) - val joinedDS = sourceDS.join(targetDS, column(mergeCondition), "fullOuter") + val joinedDS = sourceDS.join(targetDS, toColumn(mergeCondition), "fullOuter") val joinedPlan = joinedDS.queryExecution.analyzed def resolveOnJoinedPlan(exprs: Seq[Expression]): Seq[Expression] = { @@ -209,9 +208,9 @@ case class MergeIntoPaimonTable( val targetOutput = filteredTargetPlan.output val targetRowNotMatched = resolveOnJoinedPlan( - Seq(convertToExpression(sparkSession, col(SOURCE_ROW_COL).isNull))).head + Seq(toExpression(sparkSession, col(SOURCE_ROW_COL).isNull))).head val sourceRowNotMatched = resolveOnJoinedPlan( - Seq(convertToExpression(sparkSession, col(TARGET_ROW_COL).isNull))).head + Seq(toExpression(sparkSession, col(TARGET_ROW_COL).isNull))).head val matchedExprs = matchedActions.map(_.condition.getOrElse(TrueLiteral)) val notMatchedExprs = notMatchedActions.map(_.condition.getOrElse(TrueLiteral)) val notMatchedBySourceExprs = notMatchedBySourceActions.map(_.condition.getOrElse(TrueLiteral)) @@ -275,7 +274,7 @@ case class MergeIntoPaimonTable( .withColumn(ROW_ID_COL, monotonically_increasing_id()) val sourceDS = createDataset(sparkSession, sourceTable) val count = sourceDS - .join(targetDS, column(mergeCondition), "inner") + .join(targetDS, toColumn(mergeCondition), "inner") .select(col(ROW_ID_COL), lit(1).as("one")) .groupBy(ROW_ID_COL) .agg(sum("one").as("count")) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala index f2ea965d14071..47e3f77d0e2cd 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.plans.logical.{Assignment, Filter, Project, SupportsSubquery} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.functions.lit -import org.apache.spark.sql.paimon.shims.ExpressionUtils.column +import org.apache.spark.sql.paimon.shims.SparkShimLoader case class UpdatePaimonTableCommand( relation: DataSourceV2Relation, @@ -133,7 +133,8 @@ case class UpdatePaimonTableCommand( sparkSession: SparkSession, touchedDataSplits: Array[DataSplit]): Seq[CommitMessage] = { val updateColumns = updateExpressions.zip(relation.output).map { - case (update, origin) => column(update).as(origin.name, origin.metadata) + case (update, origin) => + SparkShimLoader.getSparkShim.column(update).as(origin.name, origin.metadata) } val toUpdateScanRelation = createNewRelation(touchedDataSplits, relation) @@ -156,7 +157,7 @@ case class UpdatePaimonTableCommand( } else { If(condition, update, origin) } - column(updated).as(origin.name, origin.metadata) + SparkShimLoader.getSparkShim.column(updated).as(origin.name, origin.metadata) } val data = createDataset(sparkSession, toUpdateScanRelation).select(updateColumns: _*) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala new file mode 100644 index 0000000000000..c6539a493cee9 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala @@ -0,0 +1,118 @@ +/* + * 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.paimon.spark.data + +import org.apache.paimon.data.InternalArray +import org.apache.paimon.spark.DataConverter +import org.apache.paimon.types.{ArrayType => PaimonArrayType, BigIntType, DataType => PaimonDataType, DataTypeChecks, RowType} +import org.apache.paimon.utils.InternalRowUtils + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.SpecializedGettersReader +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} +import org.apache.spark.sql.paimon.shims.SparkShimLoader +import org.apache.spark.sql.types.{DataType, Decimal} +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} + +abstract class SparkArrayData extends org.apache.spark.sql.catalyst.util.ArrayData { + + def replace(array: InternalArray): SparkArrayData +} + +abstract class AbstractSparkArrayData extends SparkArrayData { + + val elementType: PaimonDataType + + var paimonArray: InternalArray = _ + + override def replace(array: InternalArray): SparkArrayData = { + this.paimonArray = array + this + } + + override def numElements(): Int = paimonArray.size() + + override def copy(): ArrayData = { + SparkArrayData.create(elementType).replace(InternalRowUtils.copyArray(paimonArray, elementType)) + } + + override def array: Array[Any] = { + Array.range(0, numElements()).map { + i => + DataConverter + .fromPaimon(InternalRowUtils.get(paimonArray, i, elementType), elementType) + } + } + + override def setNullAt(i: Int): Unit = throw new UnsupportedOperationException() + + override def update(i: Int, value: Any): Unit = throw new UnsupportedOperationException() + + override def isNullAt(ordinal: Int): Boolean = paimonArray.isNullAt(ordinal) + + override def getBoolean(ordinal: Int): Boolean = paimonArray.getBoolean(ordinal) + + override def getByte(ordinal: Int): Byte = paimonArray.getByte(ordinal) + + override def getShort(ordinal: Int): Short = paimonArray.getShort(ordinal) + + override def getInt(ordinal: Int): Int = paimonArray.getInt(ordinal) + + override def getLong(ordinal: Int): Long = elementType match { + case _: BigIntType => paimonArray.getLong(ordinal) + case _ => + DataConverter.fromPaimon( + paimonArray.getTimestamp(ordinal, DataTypeChecks.getPrecision(elementType))) + } + + override def getFloat(ordinal: Int): Float = paimonArray.getFloat(ordinal) + + override def getDouble(ordinal: Int): Double = paimonArray.getDouble(ordinal) + + override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = + DataConverter.fromPaimon(paimonArray.getDecimal(ordinal, precision, scale)) + + override def getUTF8String(ordinal: Int): UTF8String = + DataConverter.fromPaimon(paimonArray.getString(ordinal)) + + override def getBinary(ordinal: Int): Array[Byte] = paimonArray.getBinary(ordinal) + + override def getInterval(ordinal: Int): CalendarInterval = + throw new UnsupportedOperationException() + + override def getStruct(ordinal: Int, numFields: Int): InternalRow = DataConverter + .fromPaimon(paimonArray.getRow(ordinal, numFields), elementType.asInstanceOf[RowType]) + + override def getArray(ordinal: Int): ArrayData = DataConverter.fromPaimon( + paimonArray.getArray(ordinal), + elementType.asInstanceOf[PaimonArrayType]) + + override def getMap(ordinal: Int): MapData = + DataConverter.fromPaimon(paimonArray.getMap(ordinal), elementType) + + override def get(ordinal: Int, dataType: DataType): AnyRef = + SpecializedGettersReader.read(this, ordinal, dataType, true, true) + +} + +object SparkArrayData { + def create(elementType: PaimonDataType): SparkArrayData = { + SparkShimLoader.getSparkShim.createSparkArrayData(elementType) + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala new file mode 100644 index 0000000000000..f3e607e9d7d20 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala @@ -0,0 +1,36 @@ +/* + * 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.paimon.spark.data + +import org.apache.paimon.types.RowType + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.paimon.shims.SparkShimLoader + +abstract class SparkInternalRow extends InternalRow { + def replace(row: org.apache.paimon.data.InternalRow): SparkInternalRow +} + +object SparkInternalRow { + + def create(rowType: RowType): SparkInternalRow = { + SparkShimLoader.getSparkShim.createSparkInternalRow(rowType) + } + +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala index 6f47a77ef3080..e8f75d394a818 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala @@ -25,14 +25,16 @@ import org.apache.paimon.spark.execution.PaimonStrategy import org.apache.paimon.spark.execution.adaptive.DisableUnnecessaryPaimonBucketedScan import org.apache.spark.sql.SparkSessionExtensions -import org.apache.spark.sql.catalyst.parser.extensions.PaimonSparkSqlExtensionsParser +import org.apache.spark.sql.paimon.shims.SparkShimLoader /** Spark session extension to extends the syntax and adds the rules. */ class PaimonSparkSessionExtensions extends (SparkSessionExtensions => Unit) { override def apply(extensions: SparkSessionExtensions): Unit = { // parser extensions - extensions.injectParser { case (_, parser) => new PaimonSparkSqlExtensionsParser(parser) } + extensions.injectParser { + case (_, parser) => SparkShimLoader.getSparkShim.createSparkParser(parser) + } // analyzer extensions extensions.injectResolutionRule(spark => new PaimonAnalysis(spark)) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/AbstractPaimonSparkSqlExtensionsParser.scala similarity index 98% rename from paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala rename to paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/AbstractPaimonSparkSqlExtensionsParser.scala index 9ece186930d71..c1d61e9738344 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/AbstractPaimonSparkSqlExtensionsParser.scala @@ -47,8 +47,8 @@ import java.util.Locale * @param delegate * The extension parser. */ -class PaimonSparkSqlExtensionsParser(val delegate: ParserInterface) - extends org.apache.spark.sql.paimon.shims.ParserInterface +abstract class AbstractPaimonSparkSqlExtensionsParser(val delegate: ParserInterface) + extends org.apache.spark.sql.catalyst.parser.ParserInterface with Logging { private lazy val substitutor = new VariableSubstitution() diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala index 265c828661952..2ab3dc4945245 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala @@ -23,10 +23,7 @@ import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.ExternalCatalog import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION -import org.apache.spark.util.Utils - -import scala.reflect.ClassTag -import scala.util.control.NonFatal +import org.apache.spark.sql.paimon.ReflectUtils object PaimonCatalogUtils { @@ -37,22 +34,10 @@ object PaimonCatalogUtils { } else { "org.apache.spark.sql.catalyst.catalog.InMemoryCatalog" } - reflect[ExternalCatalog, SparkConf, Configuration](externalCatalogClassName, conf, hadoopConf) - } - - private def reflect[T, Arg1 <: AnyRef, Arg2 <: AnyRef]( - className: String, - ctorArg1: Arg1, - ctorArg2: Arg2)(implicit ctorArgTag1: ClassTag[Arg1], ctorArgTag2: ClassTag[Arg2]): T = { - try { - val clazz = Utils.classForName(className) - val ctor = clazz.getDeclaredConstructor(ctorArgTag1.runtimeClass, ctorArgTag2.runtimeClass) - val args = Array[AnyRef](ctorArg1, ctorArg2) - ctor.newInstance(args: _*).asInstanceOf[T] - } catch { - case NonFatal(e) => - throw new IllegalArgumentException(s"Error while instantiating '$className':", e) - } + ReflectUtils.reflect[ExternalCatalog, SparkConf, Configuration]( + externalCatalogClassName, + conf, + hadoopConf) } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/ReflectUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/ReflectUtils.scala new file mode 100644 index 0000000000000..bedac542ab8bc --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/ReflectUtils.scala @@ -0,0 +1,43 @@ +/* + * 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.spark.sql.paimon + +import org.apache.spark.util.Utils + +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +object ReflectUtils { + + def reflect[T, Arg1 <: AnyRef, Arg2 <: AnyRef](className: String, ctorArg1: Arg1, ctorArg2: Arg2)( + implicit + ctorArgTag1: ClassTag[Arg1], + ctorArgTag2: ClassTag[Arg2]): T = { + try { + val clazz = Utils.classForName(className) + val ctor = clazz.getDeclaredConstructor(ctorArgTag1.runtimeClass, ctorArgTag2.runtimeClass) + val args = Array[AnyRef](ctorArg1, ctorArg2) + ctor.newInstance(args: _*).asInstanceOf[T] + } catch { + case NonFatal(e) => + throw new IllegalArgumentException(s"Error while instantiating '$className':", e) + } + } + +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala new file mode 100644 index 0000000000000..bd85282737e99 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala @@ -0,0 +1,61 @@ +/* + * 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.spark.sql.paimon.shims + +import org.apache.paimon.spark.data.{SparkArrayData, SparkInternalRow} +import org.apache.paimon.types.{DataType, RowType} + +import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.StructType + +import java.util.{Map => JMap} + +/** + * A spark shim trait. It declare methods which have incompatible implementations between Spark 3 + * and Spark 4. The specific SparkShim implementation will be loaded through Service Provider + * Interface. + */ +trait SparkShim { + + def createSparkParser(delegate: ParserInterface): ParserInterface + + def createSparkInternalRow(rowType: RowType): SparkInternalRow + + def createSparkArrayData(elementType: DataType): SparkArrayData + + def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean + + def createTable( + tableCatalog: TableCatalog, + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: JMap[String, String]): Table + + def column(expr: Expression): Column + + def convertToExpression(spark: SparkSession, column: Column): Expression + +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala new file mode 100644 index 0000000000000..920896547a1e2 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala @@ -0,0 +1,43 @@ +/* + * 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.spark.sql.paimon.shims + +import java.util.ServiceLoader + +import scala.collection.JavaConverters._ + +/** Load a [[SparkShim]]'s implementation. */ +object SparkShimLoader { + + private lazy val sparkShim: SparkShim = loadSparkShim() + + def getSparkShim: SparkShim = { + sparkShim + } + + private def loadSparkShim(): SparkShim = { + val shims = ServiceLoader.load(classOf[SparkShim]).asScala + if (shims.isEmpty) { + throw new IllegalStateException("No available spark shim here.") + } else if (shims.size > 1) { + throw new IllegalStateException("Found more than one spark shim here.") + } + shims.head + } +} diff --git a/paimon-spark/paimon-spark-ut/pom.xml b/paimon-spark/paimon-spark-ut/pom.xml new file mode 100644 index 0000000000000..0a18405964875 --- /dev/null +++ b/paimon-spark/paimon-spark-ut/pom.xml @@ -0,0 +1,180 @@ + + + + 4.0.0 + + + org.apache.paimon + paimon-spark + 1.0-SNAPSHOT + + + paimon-spark-ut + Paimon : Spark : UT + + + ${paimon-spark-common.spark.version} + ${paimon.shade.jackson.version} + + + + + org.apache.paimon + ${paimon-sparkx-common} + ${project.version} + test + + + + org.apache.paimon + paimon-spark-common_${scala.binary.version} + ${project.version} + test + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + test + + + com.fasterxml.jackson.core + * + + + com.fasterxml.jackson.module + * + + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + test + + + com.fasterxml.jackson.core + * + + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + tests + test + + + com.fasterxml.jackson.core + * + + + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${spark.version} + tests + test + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + tests + test + + + com.fasterxml.jackson.core + * + + + com.fasterxml.jackson.module + * + + + + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} + test + + + com.fasterxml.jackson.core + * + + + com.google.protobuf + protobuf-java + + + + + + org.apache.spark + spark-avro_${scala.binary.version} + ${spark.version} + test + + + + com.fasterxml.jackson.module + jackson-module-scala_${scala.binary.version} + ${jackson.version} + test + + + + com.google.protobuf + protobuf-java + ${protobuf-java.version} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + test-compile + + test-jar + + + + + + + + diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java similarity index 95% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java index b98213c0e662b..1117ad58c7376 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.Timestamp; +import org.apache.paimon.spark.data.SparkInternalRow; import org.apache.paimon.utils.DateTimeUtils; import org.apache.spark.sql.catalyst.CatalystTypeConverters; @@ -95,7 +96,7 @@ public void test() { SparkTypeUtils.fromPaimonType(ALL_TYPES))); org.apache.spark.sql.Row sparkRow = (org.apache.spark.sql.Row) - sparkConverter.apply(new SparkInternalRow(ALL_TYPES).replace(rowData)); + sparkConverter.apply(SparkInternalRow.create(ALL_TYPES).replace(rowData)); String expected = "1," @@ -122,7 +123,8 @@ public void test() { SparkRow sparkRowData = new SparkRow(ALL_TYPES, sparkRow); sparkRow = (org.apache.spark.sql.Row) - sparkConverter.apply(new SparkInternalRow(ALL_TYPES).replace(sparkRowData)); + sparkConverter.apply( + SparkInternalRow.create(ALL_TYPES).replace(sparkRowData)); assertThat(sparkRowToString(sparkRow)).isEqualTo(expected); TimeZone.setDefault(tz); } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTimeTravelITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTimeTravelITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTimeTravelITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTimeTravelITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTimeTravelWithDataFrameITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTimeTravelWithDataFrameITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTimeTravelWithDataFrameITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTimeTravelWithDataFrameITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTypeTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTypeTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTypeTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTypeTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteWithKyroITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteWithKyroITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteWithKyroITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteWithKyroITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/paimon-spark/paimon-spark-ut/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension rename to paimon-spark/paimon-spark-ut/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension diff --git a/paimon-spark/paimon-spark-common/src/test/resources/hive-site.xml b/paimon-spark/paimon-spark-ut/src/test/resources/hive-site.xml similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/resources/hive-site.xml rename to paimon-spark/paimon-spark-ut/src/test/resources/hive-site.xml diff --git a/paimon-spark/paimon-spark-common/src/test/resources/log4j2-test.properties b/paimon-spark/paimon-spark-ut/src/test/resources/log4j2-test.properties similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/resources/log4j2-test.properties rename to paimon-spark/paimon-spark-ut/src/test/resources/log4j2-test.properties diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala similarity index 98% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala index 842147615d1a7..6d2ffea04df54 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala @@ -22,7 +22,7 @@ import org.apache.paimon.hive.TestHiveMetastore import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf -import org.apache.spark.paimon.Utils +import org.apache.spark.sql.paimon.Utils import java.io.File diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala similarity index 79% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala index 9b4a344259a93..605b2e6ca5f26 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala @@ -25,11 +25,12 @@ import org.apache.paimon.spark.sql.{SparkVersionSupport, WithTableOptions} import org.apache.paimon.table.FileStoreTable import org.apache.spark.SparkConf -import org.apache.spark.paimon.Utils import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{Identifier => SparkIdentifier} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.paimon.Utils import org.apache.spark.sql.test.SharedSparkSession import org.scalactic.source.Position import org.scalatest.Tag @@ -105,7 +106,7 @@ class PaimonSparkTestBase } protected def withTimeZone(timeZone: String)(f: => Unit): Unit = { - withSQLConf("spark.sql.session.timeZone" -> timeZone) { + withSparkSQLConf("spark.sql.session.timeZone" -> timeZone) { val originTimeZone = TimeZone.getDefault try { TimeZone.setDefault(TimeZone.getTimeZone(timeZone)) @@ -116,6 +117,40 @@ class PaimonSparkTestBase } } + // Since SPARK-46227 has changed the definition of withSQLConf that resulted in + // incompatibility between the Spark3.x and Spark4.x, So Paimon declare a separate method + // to provide the same function. + protected def withSparkSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + withSparkSQLConf0(pairs: _*)(f) + } + + private def withSparkSQLConf0(pairs: (String, String)*)(f: => Unit): Unit = { + val conf = SQLConf.get + val (keys, values) = pairs.unzip + val currentValues = keys.map { + key => + if (conf.contains(key)) { + Some(conf.getConfString(key)) + } else { + None + } + } + (keys, values).zipped.foreach { + (k, v) => + if (SQLConf.isStaticConfigKey(k)) { + throw new RuntimeException(s"Cannot modify the value of a static config: $k") + } + conf.setConfString(k, v) + } + try f + finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } + override def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { println(testName) diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonTableTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonTableTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala index fc787246f9f12..a3223446f644b 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala @@ -26,7 +26,6 @@ import org.apache.paimon.table.source.{DataSplit, Split} import org.junit.jupiter.api.Assertions -import java.util import java.util.{HashMap => JHashMap} import scala.collection.JavaConverters._ @@ -35,7 +34,7 @@ import scala.collection.mutable class ScanHelperTest extends PaimonSparkTestBase { test("Paimon: reshuffle splits") { - withSQLConf(("spark.sql.leafNodeDefaultParallelism", "20")) { + withSparkSQLConf(("spark.sql.leafNodeDefaultParallelism", "20")) { val splitNum = 5 val fileNum = 100 diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/FastForwardProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/FastForwardProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/FastForwardProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/FastForwardProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateFileProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateFileProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateFileProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateFileProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateTableProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateTableProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateTableProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateTableProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala index 238dd039969a8..4f8ccae22dd55 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala @@ -86,7 +86,7 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS") - withSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { + withSparkSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(2, 0, 2, "{ }")) @@ -97,7 +97,7 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS") - withSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { + withSparkSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(5, 0, 4, "{ }")) @@ -111,31 +111,31 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'test_tag6', snapshot => 6)"), Row(true) :: Nil) - withSQLConf("spark.paimon.scan.tag-name" -> "test_tag5") { + withSparkSQLConf("spark.paimon.scan.tag-name" -> "test_tag5") { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(2, 0, 2, "{ }")) } - withSQLConf("spark.paimon.scan.tag-name" -> "test_tag6") { + withSparkSQLConf("spark.paimon.scan.tag-name" -> "test_tag6") { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(5, 0, 4, "{ }")) } - withSQLConf("spark.paimon.scan.snapshot-id" -> "3") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "3") { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(2, 0, 2, "{ }")) } - withSQLConf("spark.paimon.scan.snapshot-id" -> "4") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "4") { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(2, 0, 2, "{ }")) } - withSQLConf("spark.paimon.scan.snapshot-id" -> "6") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "6") { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(5, 0, 4, "{ }")) diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala index afc70bf9105df..35931924c4875 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala @@ -29,12 +29,12 @@ class BucketedTableQueryTest extends PaimonSparkTestBase with AdaptiveSparkPlanH private def checkAnswerAndShuffleSorts(query: String, numShuffles: Int, numSorts: Int): Unit = { var expectedResult: Array[Row] = null // avoid config default value change in future, so specify it manually - withSQLConf( + withSparkSQLConf( "spark.sql.sources.v2.bucketing.enabled" -> "false", "spark.sql.autoBroadcastJoinThreshold" -> "-1") { expectedResult = spark.sql(query).collect() } - withSQLConf( + withSparkSQLConf( "spark.sql.sources.v2.bucketing.enabled" -> "true", "spark.sql.autoBroadcastJoinThreshold" -> "-1") { val df = spark.sql(query) @@ -162,10 +162,10 @@ class BucketedTableQueryTest extends PaimonSparkTestBase with AdaptiveSparkPlanH checkAnswerAndShuffleSorts("select max(c) OVER (PARTITION BY id ORDER BY id) from t1", 0, 1) checkAnswerAndShuffleSorts("select sum(id) OVER (PARTITION BY c ORDER BY id) from t1", 1, 1) - withSQLConf("spark.sql.requireAllClusterKeysForDistribution" -> "false") { + withSparkSQLConf("spark.sql.requireAllClusterKeysForDistribution" -> "false") { checkAnswerAndShuffleSorts("SELECT id, c, count(*) FROM t1 GROUP BY id, c", 0, 0) } - withSQLConf("spark.sql.requireAllClusterKeysForDistribution" -> "true") { + withSparkSQLConf("spark.sql.requireAllClusterKeysForDistribution" -> "true") { checkAnswerAndShuffleSorts("SELECT id, c, count(*) FROM t1 GROUP BY id, c", 1, 0) } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala index b09a2be98dc8d..6ad5274496a90 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala @@ -238,21 +238,21 @@ abstract class DDLTestBase extends PaimonSparkTestBase { |USING PAIMON |""".stripMargin) - withSQLConf("spark.sql.legacy.charVarcharAsString" -> "true") { + withSparkSQLConf("spark.sql.legacy.charVarcharAsString" -> "true") { sql("INSERT INTO paimon_tbl VALUES (1, 'ab')") } - withSQLConf("spark.sql.legacy.charVarcharAsString" -> "false") { + withSparkSQLConf("spark.sql.legacy.charVarcharAsString" -> "false") { sql("INSERT INTO paimon_tbl VALUES (2, 'ab')") } if (gteqSpark3_4) { - withSQLConf("spark.sql.readSideCharPadding" -> "true") { + withSparkSQLConf("spark.sql.readSideCharPadding" -> "true") { checkAnswer( spark.sql("SELECT c FROM paimon_tbl ORDER BY id"), Row("ab ") :: Row("ab ") :: Nil) } - withSQLConf("spark.sql.readSideCharPadding" -> "false") { + withSparkSQLConf("spark.sql.readSideCharPadding" -> "false") { checkAnswer( spark.sql("SELECT c FROM paimon_tbl ORDER BY id"), Row("ab") :: Row("ab ") :: Nil) @@ -270,7 +270,8 @@ abstract class DDLTestBase extends PaimonSparkTestBase { format => Seq(true, false).foreach { datetimeJava8APIEnabled => - withSQLConf("spark.sql.datetime.java8API.enabled" -> datetimeJava8APIEnabled.toString) { + withSparkSQLConf( + "spark.sql.datetime.java8API.enabled" -> datetimeJava8APIEnabled.toString) { withTimeZone("Asia/Shanghai") { withTable("paimon_tbl") { // Spark support create table with timestamp_ntz since 3.4 @@ -430,7 +431,8 @@ abstract class DDLTestBase extends PaimonSparkTestBase { test("Paimon DDL: select table with timestamp and timestamp_ntz with filter") { Seq(true, false).foreach { datetimeJava8APIEnabled => - withSQLConf("spark.sql.datetime.java8API.enabled" -> datetimeJava8APIEnabled.toString) { + withSparkSQLConf( + "spark.sql.datetime.java8API.enabled" -> datetimeJava8APIEnabled.toString) { withTable("paimon_tbl") { // Spark support create table with timestamp_ntz since 3.4 if (gteqSpark3_4) { diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala similarity index 99% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 9be8e21a8df2d..e99e4434ef7f1 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -304,7 +304,7 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { withTempDir { tbLocation => withDatabase("paimon_db") { - spark.sql(s"CREATE DATABASE paimon_db") + spark.sql(s"CREATE DATABASE IF NOT EXISTS paimon_db") spark.sql(s"USE paimon_db") withTable("external_tbl", "managed_tbl") { val expertTbLocation = tbLocation.getCanonicalPath diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala similarity index 99% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala index e944429e42184..ea8309e14ffef 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala @@ -25,13 +25,13 @@ import org.apache.paimon.spark.{PaimonSparkTestBase, PaimonSplitScan} import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.table.FileStoreTable -import org.apache.spark.paimon.Utils import org.apache.spark.sql.Row import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation} import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.paimon.Utils import org.apache.spark.sql.util.QueryExecutionListener import org.junit.jupiter.api.Assertions diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala index 70339bd7cac3e..f47d40285aa9a 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala @@ -48,13 +48,13 @@ class DisableUnnecessaryPaimonBucketedScanSuite assert(bucketedScan.length == expectedNumBucketedScan, query) } - withSQLConf("spark.sql.sources.v2.bucketing.enabled" -> "true") { - withSQLConf("spark.sql.sources.bucketing.autoBucketedScan.enabled" -> "true") { + withSparkSQLConf("spark.sql.sources.v2.bucketing.enabled" -> "true") { + withSparkSQLConf("spark.sql.sources.bucketing.autoBucketedScan.enabled" -> "true") { val df = sql(query) val result = df.collect() checkNumBucketedScan(df, expectedNumScanWithAutoScanEnabled) - withSQLConf("spark.sql.sources.bucketing.autoBucketedScan.enabled" -> "false") { + withSparkSQLConf("spark.sql.sources.bucketing.autoBucketedScan.enabled" -> "false") { val expected = sql(query) checkAnswer(expected, result) checkNumBucketedScan(expected, expectedNumScanWithAutoScanDisabled) diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala similarity index 99% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala index 674b45fda68b2..03026e8574295 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala @@ -346,7 +346,7 @@ abstract class InsertOverwriteTableTestBase extends PaimonSparkTestBase { spark.sql("SELECT * FROM T ORDER BY a, b"), Row(1, 3, "3") :: Row(2, 4, "4") :: Nil) - withSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { + withSparkSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { // dynamic overwrite the a=1 partition spark.sql("INSERT OVERWRITE T VALUES (1, 5, '5'), (1, 7, '7')") checkAnswer( @@ -387,7 +387,7 @@ abstract class InsertOverwriteTableTestBase extends PaimonSparkTestBase { "ptv2", 22) :: Nil) - withSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { + withSparkSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { // dynamic overwrite the pt2=22 partition spark.sql( "INSERT OVERWRITE T PARTITION (pt2 = 22) VALUES (3, 'c2', 'ptv1'), (4, 'd2', 'ptv3')") diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/ObjectTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/ObjectTableTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/ObjectTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/ObjectTableTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonMetricTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonMetricTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonMetricTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonMetricTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala similarity index 90% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala index 78e8905fa969a..87f4c94486192 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala @@ -20,6 +20,7 @@ package org.apache.paimon.spark.sql import org.apache.paimon.Snapshot.CommitKind import org.apache.paimon.spark.PaimonSparkTestBase +import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper import org.apache.paimon.spark.catalyst.optimizer.MergePaimonScalarSubqueries import org.apache.spark.sql.Row @@ -27,11 +28,12 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, LogicalPlan, OneRowRelation, WithCTE} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.functions._ +import org.apache.spark.sql.paimon.Utils import org.junit.jupiter.api.Assertions import scala.collection.immutable -abstract class PaimonOptimizationTestBase extends PaimonSparkTestBase { +abstract class PaimonOptimizationTestBase extends PaimonSparkTestBase with ExpressionHelper { import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -59,24 +61,25 @@ abstract class PaimonOptimizationTestBase extends PaimonSparkTestBase { |""".stripMargin) val optimizedPlan = Optimize.execute(query.queryExecution.analyzed) - val relation = createRelationV2("T") - val mergedSubquery = relation + val df = Utils.createDataFrame(spark, createRelationV2("T")) + val mergedSubquery = df .select( - count(Literal(1)).as("cnt"), - sum(col("a").expr).as("sum_a"), - avg(col("b").expr).as("avg_b") + toColumn(count(Literal(1))).as("cnt"), + toColumn(sum(toExpression(spark, col("a")))).as("sum_a"), + toColumn(avg(toExpression(spark, col("b"))).as("avg_b")) ) .select( - CreateNamedStruct( - Seq( - Literal("cnt"), - 'cnt, - Literal("sum_a"), - 'sum_a, - Literal("avg_b"), - 'avg_b - )).as("mergedValue")) - val analyzedMergedSubquery = mergedSubquery.analyze + toColumn( + CreateNamedStruct( + Seq( + Literal("cnt"), + 'cnt, + Literal("sum_a"), + 'sum_a, + Literal("avg_b"), + 'avg_b + )).as("mergedValue"))) + val analyzedMergedSubquery = mergedSubquery.queryExecution.analyzed val correctAnswer = WithCTE( OneRowRelation() .select( diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala similarity index 91% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala index f74d6959b9f16..44df3e54ca724 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala @@ -29,7 +29,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { import testImplicits._ test("Paimon Option: create table with sql conf") { - withSQLConf("spark.paimon.scan.snapshot-id" -> "2") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "2") { sql("CREATE TABLE T (id INT)") val table = loadTable("T") // check options in schema file directly @@ -39,7 +39,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { } test("Paimon Option: create table by dataframe with sql conf") { - withSQLConf("spark.paimon.scan.snapshot-id" -> "2") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "2") { Seq((1L, "x1"), (2L, "x2")) .toDF("a", "b") .write @@ -61,13 +61,13 @@ class PaimonOptionTest extends PaimonSparkTestBase { val table = loadTable("T") // query with mutable option - withSQLConf("spark.paimon.scan.snapshot-id" -> "1") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "1") { checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1)) checkAnswer(spark.read.format("paimon").load(table.location().toString), Row(1)) } // query with immutable option - withSQLConf("spark.paimon.bucket" -> "1") { + withSparkSQLConf("spark.paimon.bucket" -> "1") { assertThrows[UnsupportedOperationException] { sql("SELECT * FROM T ORDER BY id") } @@ -85,19 +85,19 @@ class PaimonOptionTest extends PaimonSparkTestBase { val table = loadTable("T") // query with global options - withSQLConf("spark.paimon.scan.snapshot-id" -> "1") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "1") { checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1)) checkAnswer(spark.read.format("paimon").load(table.location().toString), Row(1)) } // query with table options - withSQLConf("spark.paimon.*.*.T.scan.snapshot-id" -> "1") { + withSparkSQLConf("spark.paimon.*.*.T.scan.snapshot-id" -> "1") { checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1)) checkAnswer(spark.read.format("paimon").load(table.location().toString), Row(1)) } // query with both global and table options - withSQLConf( + withSparkSQLConf( "spark.paimon.scan.snapshot-id" -> "1", "spark.paimon.*.*.T.scan.snapshot-id" -> "2") { checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1) :: Row(2) :: Nil) @@ -122,7 +122,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { val table2 = loadTable("T1") // query with global options - withSQLConf("spark.paimon.scan.snapshot-id" -> "1") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "1") { checkAnswer(sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), Row(1, 1)) checkAnswer( spark.read @@ -134,7 +134,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { } // query with table options - withSQLConf("spark.paimon.*.*.*.scan.snapshot-id" -> "1") { + withSparkSQLConf("spark.paimon.*.*.*.scan.snapshot-id" -> "1") { checkAnswer(sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), Row(1, 1)) checkAnswer( spark.read @@ -146,7 +146,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { } // query with both global and table options - withSQLConf( + withSparkSQLConf( "spark.paimon.scan.snapshot-id" -> "1", "spark.paimon.*.*.*.scan.snapshot-id" -> "2") { checkAnswer( @@ -161,7 +161,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { ) } - withSQLConf( + withSparkSQLConf( "spark.paimon.*.*.T1.scan.snapshot-id" -> "1", "spark.paimon.*.*.T2.scan.snapshot-id" -> "1") { checkAnswer(sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), Row(1, 1)) @@ -174,7 +174,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { ) } - withSQLConf( + withSparkSQLConf( "spark.paimon.*.*.T1.scan.snapshot-id" -> "1", "spark.paimon.*.*.T2.scan.snapshot-id" -> "2") { checkAnswer(sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), Row(1, 1)) @@ -187,7 +187,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { ) } - withSQLConf( + withSparkSQLConf( "spark.paimon.*.*.T1.scan.snapshot-id" -> "2", "spark.paimon.*.*.T2.scan.snapshot-id" -> "2") { checkAnswer( diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPartitionManagementTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPartitionManagementTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPartitionManagementTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPartitionManagementTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala similarity index 99% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala index beea19c35e925..08f5275f01b50 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala @@ -281,7 +281,7 @@ class PaimonQueryTest extends PaimonSparkTestBase { // Since Spark 4.0, when `spark.sql.ansi.enabled` is `true` and `array[i]` does not exist, an exception // will be thrown instead of returning null. Here, just disabled it and return null for test. - withSQLConf("spark.sql.ansi.enabled" -> "false") { + withSparkSQLConf("spark.sql.ansi.enabled" -> "false") { checkAnswer( sql(s""" |SELECT diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonShowColumnsTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonShowColumnsTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonShowColumnsTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonShowColumnsTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala similarity index 74% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala index 5ea2dd861e197..03f1c7706efbd 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala @@ -16,9 +16,10 @@ * limitations under the License. */ -package org.apache.spark.paimon +package org.apache.spark.sql.paimon -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.util.{Utils => SparkUtils} import java.io.File @@ -28,9 +29,14 @@ import java.io.File */ object Utils { - def createTempDir: File = SparkUtils.createTempDir() + def createTempDir: File = SparkUtils.createTempDir(System.getProperty("java.io.tmpdir"), "spark") def waitUntilEventEmpty(spark: SparkSession): Unit = { spark.sparkContext.listenerBus.waitUntilEmpty() } + + def createDataFrame(sparkSession: SparkSession, plan: LogicalPlan): DataFrame = { + Dataset.ofRows(sparkSession, plan) + } + } diff --git a/paimon-spark/paimon-spark3-common/pom.xml b/paimon-spark/paimon-spark3-common/pom.xml index 03d29ea05b3ad..5fd869f1b3938 100644 --- a/paimon-spark/paimon-spark3-common/pom.xml +++ b/paimon-spark/paimon-spark3-common/pom.xml @@ -39,9 +39,35 @@ under the License. - org.apache.spark - spark-sql_${scala.binary.version} - ${spark.version} + org.apache.paimon + paimon-spark-common_${scala.binary.version} + ${project.version} + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + org.apache.paimon:paimon-bundle + org.apache.paimon:paimon-spark-common_${scala.binary.version} + + + + + + + + \ No newline at end of file diff --git a/paimon-spark/paimon-spark3-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim b/paimon-spark/paimon-spark3-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim new file mode 100644 index 0000000000000..b79ef54f6e309 --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim @@ -0,0 +1,16 @@ +# 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. + +org.apache.spark.sql.paimon.shims.Spark3Shim \ No newline at end of file diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark3SqlExtensionsParser.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark3SqlExtensionsParser.scala new file mode 100644 index 0000000000000..07481b6f639f8 --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark3SqlExtensionsParser.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.catalyst.parser.extensions + +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.parser.extensions.AbstractPaimonSparkSqlExtensionsParser + +class PaimonSpark3SqlExtensionsParser(override val delegate: ParserInterface) + extends AbstractPaimonSparkSqlExtensionsParser(delegate) {} diff --git a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/spark/paimon/Utils.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3ArrayData.scala similarity index 73% rename from paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/spark/paimon/Utils.scala rename to paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3ArrayData.scala index 1a899f5001536..cb393d928dcbc 100644 --- a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/spark/paimon/Utils.scala +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3ArrayData.scala @@ -16,17 +16,8 @@ * limitations under the License. */ -package org.apache.spark.paimon +package org.apache.paimon.spark.data -import org.apache.spark.util.{Utils => SparkUtils} +import org.apache.paimon.types.DataType -import java.io.File - -/** - * A wrapper that some Objects or Classes is limited to access beyond [[org.apache.spark]] package. - */ -object Utils { - - def createTempDir: File = SparkUtils.createTempDir() - -} +class Spark3ArrayData(override val elementType: DataType) extends AbstractSparkArrayData {} diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/spark/paimon/Utils.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRow.scala similarity index 73% rename from paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/spark/paimon/Utils.scala rename to paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRow.scala index 1a899f5001536..9c9a1c6bac959 100644 --- a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/spark/paimon/Utils.scala +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRow.scala @@ -16,17 +16,9 @@ * limitations under the License. */ -package org.apache.spark.paimon +package org.apache.paimon.spark.data -import org.apache.spark.util.{Utils => SparkUtils} +import org.apache.paimon.spark.AbstractSparkInternalRow +import org.apache.paimon.types.RowType -import java.io.File - -/** - * A wrapper that some Objects or Classes is limited to access beyond [[org.apache.spark]] package. - */ -object Utils { - - def createTempDir: File = SparkUtils.createTempDir() - -} +class Spark3InternalRow(rowType: RowType) extends AbstractSparkInternalRow(rowType) {} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala similarity index 51% rename from paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala rename to paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala index 13ade3f3c5ac6..57d79d6474e9a 100644 --- a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala @@ -16,45 +16,44 @@ * limitations under the License. */ -package org.apache.spark.sql.paimon +package org.apache.spark.sql.paimon.shims + +import org.apache.paimon.spark.catalyst.parser.extensions.PaimonSpark3SqlExtensionsParser +import org.apache.paimon.spark.data.{Spark3ArrayData, Spark3InternalRow, SparkArrayData, SparkInternalRow} +import org.apache.paimon.types.{DataType, RowType} import org.apache.spark.sql.{Column, SparkSession} -import org.apache.spark.sql.catalyst.{InternalRow => SparkInternalRow} import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.parser.{ParserInterface => SparkParserInterface} -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate => SparkAggregate} -import org.apache.spark.sql.catalyst.util.{ArrayData => SparkArrayData} -import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog => SparkTableCatalog} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.Aggregate +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType import java.util.{Map => JMap} -/** Shims for Spark 3.x in [[org.apache.spark.sql]]. */ -object shims { - - /** In [[org.apache.spark.sql.catalyst]]. */ +class Spark3Shim extends SparkShim { - abstract class ParserInterface extends SparkParserInterface { - val delegate: SparkParserInterface + override def createSparkParser(delegate: ParserInterface): ParserInterface = { + new PaimonSpark3SqlExtensionsParser(delegate) } - abstract class ArrayData extends SparkArrayData {} - - abstract class InternalRow extends SparkInternalRow {} + override def createSparkInternalRow(rowType: RowType): SparkInternalRow = { + new Spark3InternalRow(rowType) + } - object Aggregate { - def supportsHashAggregate( - aggregateBufferAttributes: Seq[Attribute], - groupingExpression: Seq[Expression]): Boolean = { - SparkAggregate.supportsHashAggregate(aggregateBufferAttributes) - } + override def createSparkArrayData(elementType: DataType): SparkArrayData = { + new Spark3ArrayData(elementType) } - /** In [[org.apache.spark.sql.connector]]. */ + override def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean = { + Aggregate.supportsHashAggregate(aggregateBufferAttributes) + } - def createTable( - tableCatalog: SparkTableCatalog, + override def createTable( + tableCatalog: TableCatalog, ident: Identifier, schema: StructType, partitions: Array[Transform], @@ -62,11 +61,8 @@ object shims { tableCatalog.createTable(ident, schema, partitions, properties) } - /** In [[org.apache.spark.sql.internal]]. */ + override def column(expr: Expression): Column = new Column(expr) - object ExpressionUtils { - def column(expr: Expression): Column = new Column(expr) + override def convertToExpression(spark: SparkSession, column: Column): Expression = column.expr - def convertToExpression(spark: SparkSession, column: Column): Expression = column.expr - } } diff --git a/paimon-spark/paimon-spark4-common/pom.xml b/paimon-spark/paimon-spark4-common/pom.xml index dcc5b370d59aa..d160b984fa056 100644 --- a/paimon-spark/paimon-spark4-common/pom.xml +++ b/paimon-spark/paimon-spark4-common/pom.xml @@ -38,10 +38,57 @@ under the License. + + org.apache.paimon + paimon-spark-common_${scala.binary.version} + ${project.version} + + org.apache.spark - spark-sql_${scala.binary.version} + spark-sql-api_2.13 ${spark.version} + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + - \ No newline at end of file + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + org.apache.paimon:paimon-bundle + org.apache.paimon:paimon-spark-common_${scala.binary.version} + + + + + + + + + diff --git a/paimon-spark/paimon-spark4-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim b/paimon-spark/paimon-spark4-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim new file mode 100644 index 0000000000000..b0df8c67cf9a4 --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim @@ -0,0 +1,16 @@ +# 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. + +org.apache.spark.sql.paimon.shims.Spark4Shim \ No newline at end of file diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala new file mode 100644 index 0000000000000..ef1f5763d27b1 --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala @@ -0,0 +1,28 @@ +/* + * 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.paimon.spark.catalyst.parser.extensions + +import org.apache.spark.sql.catalyst.parser.{CompoundBody, ParserInterface} +import org.apache.spark.sql.catalyst.parser.extensions.AbstractPaimonSparkSqlExtensionsParser + +class PaimonSpark4SqlExtensionsParser(override val delegate: ParserInterface) + extends AbstractPaimonSparkSqlExtensionsParser(delegate) { + + def parseScript(sqlScriptText: String): CompoundBody = delegate.parseScript(sqlScriptText) +} diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/spark/paimon/Utils.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4ArrayData.scala similarity index 72% rename from paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/spark/paimon/Utils.scala rename to paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4ArrayData.scala index 1a899f5001536..be319c0a9c232 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/spark/paimon/Utils.scala +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4ArrayData.scala @@ -16,17 +16,14 @@ * limitations under the License. */ -package org.apache.spark.paimon +package org.apache.paimon.spark.data -import org.apache.spark.util.{Utils => SparkUtils} +import org.apache.paimon.types.DataType -import java.io.File +import org.apache.spark.unsafe.types.VariantVal -/** - * A wrapper that some Objects or Classes is limited to access beyond [[org.apache.spark]] package. - */ -object Utils { +class Spark4ArrayData(override val elementType: DataType) extends AbstractSparkArrayData { - def createTempDir: File = SparkUtils.createTempDir() + override def getVariant(ordinal: Int): VariantVal = throw new UnsupportedOperationException } diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRow.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRow.scala new file mode 100644 index 0000000000000..54b0f420ea931 --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRow.scala @@ -0,0 +1,28 @@ +/* + * 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.paimon.spark.data + +import org.apache.paimon.spark.AbstractSparkInternalRow +import org.apache.paimon.types.RowType + +import org.apache.spark.unsafe.types.VariantVal + +class Spark4InternalRow(rowType: RowType) extends AbstractSparkInternalRow(rowType) { + override def getVariant(i: Int): VariantVal = throw new UnsupportedOperationException +} diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala deleted file mode 100644 index ee6c9ad35857f..0000000000000 --- a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala +++ /dev/null @@ -1,86 +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.spark.sql.paimon - -import org.apache.spark.sql.{Column, SparkSession} -import org.apache.spark.sql.catalyst.{InternalRow => SparkInternalRow} -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.parser.{CompoundBody, ParserInterface => SparkParserInterface} -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate => SparkAggregate} -import org.apache.spark.sql.catalyst.util.{ArrayData => SparkArrayData} -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, Table, TableCatalog => SparkTableCatalog} -import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.internal.{ExpressionUtils => SparkExpressionUtils} -import org.apache.spark.sql.types.StructType -import org.apache.spark.unsafe.types.VariantVal - -import java.util.{Map => JMap} - -/** Shims for Spark 4.x in [[org.apache.spark.sql]]. */ -object shims { - - /** In [[org.apache.spark.sql.catalyst]]. */ - - abstract class ParserInterface extends SparkParserInterface { - val delegate: SparkParserInterface - - def parseScript(sqlScriptText: String): CompoundBody = delegate.parseScript(sqlScriptText) - } - - abstract class ArrayData extends SparkArrayData { - def getVariant(ordinal: Int): VariantVal = throw new UnsupportedOperationException - } - - abstract class InternalRow extends SparkInternalRow { - override def getVariant(i: Int): VariantVal = throw new UnsupportedOperationException - } - - object Aggregate { - def supportsHashAggregate( - aggregateBufferAttributes: Seq[Attribute], - groupingExpression: Seq[Expression]): Boolean = { - SparkAggregate.supportsHashAggregate(aggregateBufferAttributes, groupingExpression) - } - } - - /** In [[org.apache.spark.sql.connector]]. */ - - def createTable( - tableCatalog: SparkTableCatalog, - ident: Identifier, - schema: StructType, - partitions: Array[Transform], - properties: JMap[String, String]): Table = { - tableCatalog.createTable( - ident, - CatalogV2Util.structTypeToV2Columns(schema), - partitions, - properties) - } - - /** In [[org.apache.spark.sql.internal]]. */ - - object ExpressionUtils { - def column(expr: Expression): Column = SparkExpressionUtils.column(expr) - - def convertToExpression(spark: SparkSession, column: Column): Expression = { - spark.expression(column) - } - } -} diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala new file mode 100644 index 0000000000000..dfec4eb71f4f1 --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala @@ -0,0 +1,69 @@ +/* + * 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.spark.sql.paimon.shims + +import org.apache.paimon.spark.catalyst.parser.extensions.PaimonSpark4SqlExtensionsParser +import org.apache.paimon.spark.data.{Spark4ArrayData, Spark4InternalRow, SparkArrayData, SparkInternalRow} +import org.apache.paimon.types.{DataType, RowType} + +import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.Aggregate +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, Table, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.internal.ExpressionUtils +import org.apache.spark.sql.types.StructType + +import java.util.{Map => JMap} + +class Spark4Shim extends SparkShim { + + override def createSparkParser(delegate: ParserInterface): ParserInterface = { + new PaimonSpark4SqlExtensionsParser(delegate) + } + override def createSparkInternalRow(rowType: RowType): SparkInternalRow = { + new Spark4InternalRow(rowType) + } + + override def createSparkArrayData(elementType: DataType): SparkArrayData = { + new Spark4ArrayData(elementType) + } + + def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean = { + Aggregate.supportsHashAggregate(aggregateBufferAttributes, groupingExpression) + } + + def createTable( + tableCatalog: TableCatalog, + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: JMap[String, String]): Table = { + val columns = CatalogV2Util.structTypeToV2Columns(schema) + tableCatalog.createTable(ident, columns, partitions, properties) + } + + def column(expr: Expression): Column = ExpressionUtils.column(expr) + + def convertToExpression(spark: SparkSession, column: Column): Expression = + spark.expression(column) +} diff --git a/paimon-spark/pom.xml b/paimon-spark/pom.xml index aac73baa5fec4..61ecd20a0500a 100644 --- a/paimon-spark/pom.xml +++ b/paimon-spark/pom.xml @@ -39,6 +39,7 @@ under the License. paimon-spark-common + paimon-spark-ut @@ -114,6 +115,21 @@ under the License. + + org.apache.spark + spark-hive_${scala.binary.version} + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + org.apache.paimon paimon-bundle @@ -180,22 +196,6 @@ under the License. - - - org.apache.spark - spark-hive_${scala.binary.version} - test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - @@ -278,6 +278,12 @@ under the License. aws-java-sdk-core ${aws.version} test + + + com.fasterxml.jackson.core + * + + From 139b5a75a5f0b89bc9d9c91f8c06dfb68691c9e0 Mon Sep 17 00:00:00 2001 From: xleoken Date: Tue, 26 Nov 2024 17:00:45 +0800 Subject: [PATCH 066/157] [core] Improve the performance of show tables (#4592) --- .../org/apache/paimon/catalog/AbstractCatalog.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index d3a8d628a2bc6..16b76513d7efd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -653,7 +653,16 @@ protected List listTablesInFileSystem(Path databasePath) throws IOExcept } protected boolean tableExistsInFileSystem(Path tablePath, String branchName) { - return !new SchemaManager(fileIO, tablePath, branchName).listAllIds().isEmpty(); + SchemaManager schemaManager = new SchemaManager(fileIO, tablePath, branchName); + + // in order to improve the performance, check the schema-0 firstly. + boolean schemaZeroExists = schemaManager.schemaExists(0); + if (schemaZeroExists) { + return true; + } else { + // if schema-0 not exists, fallback to check other schemas + return !schemaManager.listAllIds().isEmpty(); + } } public Optional tableSchemaInFileSystem(Path tablePath, String branchName) { From 01d2bb21b22c8380acc94e30e2f2267809d575e1 Mon Sep 17 00:00:00 2001 From: Yann Byron Date: Tue, 26 Nov 2024 17:07:58 +0800 Subject: [PATCH 067/157] [spark] define spark properties globally (#4593) --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 24f2923319508..85a880f35158e 100644 --- a/pom.xml +++ b/pom.xml @@ -115,6 +115,12 @@ under the License. 3.0.1-1.18 8.0.27 + + paimon-spark3-common + 3.5.3 + 3.3 + 3.3.0 + 1.5.5-11 3.0.11 3.4.6 From a67bab179214be54b90b39119c2045cb075fe55e Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Tue, 26 Nov 2024 20:15:18 +0800 Subject: [PATCH 068/157] [spark] Explicitly set spark3 profile in publish snapshot (#4595) --- .github/workflows/publish_snapshot.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/publish_snapshot.yml b/.github/workflows/publish_snapshot.yml index 7d370814da947..c7f97cab991f9 100644 --- a/.github/workflows/publish_snapshot.yml +++ b/.github/workflows/publish_snapshot.yml @@ -64,6 +64,6 @@ jobs: echo "$ASF_PASSWORD" >> $tmp_settings echo "" >> $tmp_settings - mvn --settings $tmp_settings clean deploy -Dgpg.skip -Drat.skip -DskipTests -Papache-release + mvn --settings $tmp_settings clean deploy -Dgpg.skip -Drat.skip -DskipTests -Papache-release,spark3 rm $tmp_settings From 7a390135617a86906f172a391a182a9b3b3dae04 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Wed, 27 Nov 2024 10:24:40 +0800 Subject: [PATCH 069/157] [doc] Add changelog merging into changelog-producer --- docs/content/maintenance/write-performance.md | 9 --------- docs/content/primary-key-table/changelog-producer.md | 11 +++++++++++ 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/docs/content/maintenance/write-performance.md b/docs/content/maintenance/write-performance.md index 02362b9096e7e..ade2c3353e3cc 100644 --- a/docs/content/maintenance/write-performance.md +++ b/docs/content/maintenance/write-performance.md @@ -160,12 +160,3 @@ You can use fine-grained-resource-management of Flink to increase committer heap 1. Configure Flink Configuration `cluster.fine-grained-resource-management.enabled: true`. (This is default after Flink 1.18) 2. Configure Paimon Table Options: `sink.committer-memory`, for example 300 MB, depends on your `TaskManager`. (`sink.committer-cpu` is also supported) - -## Changelog Compaction - -If Flink's checkpoint interval is short (for example, 30 seconds) and the number of buckets is large, -each snapshot may produce lots of small changelog files. -Too many files may put a burden on the distributed storage cluster. - -In order to compact small changelog files into large ones, you can set the table option `changelog.precommit-compact = true`. -Default value of this option is false, if true, it will add a compact coordinator and worker operator after the writer operator, which copies changelog files into large ones. diff --git a/docs/content/primary-key-table/changelog-producer.md b/docs/content/primary-key-table/changelog-producer.md index 011f7b6f27a7f..a9364ee9f07c5 100644 --- a/docs/content/primary-key-table/changelog-producer.md +++ b/docs/content/primary-key-table/changelog-producer.md @@ -130,3 +130,14 @@ efficient as the input changelog producer and the latency to produce changelog m Full-compaction changelog-producer supports `changelog-producer.row-deduplicate` to avoid generating -U, +U changelog for the same record. + +## Changelog Merging + +For `input`, `lookup`, `full-compaction` 'changelog-producer'. + +If Flink's checkpoint interval is short (for example, 30 seconds) and the number of buckets is large, each snapshot may +produce lots of small changelog files. Too many files may put a burden on the distributed storage cluster. + +In order to compact small changelog files into large ones, you can set the table option `changelog.precommit-compact = true`. +Default value of this option is false, if true, it will add a compact coordinator and worker operator after the writer +operator, which copies changelog files into large ones. From 0185f7c19bd798e2f1e03934d48e82f09893576b Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 27 Nov 2024 14:25:18 +0800 Subject: [PATCH 070/157] [core] Drop stats for overwrite commit (#4598) --- .../java/org/apache/paimon/operation/FileStoreCommitImpl.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java index 69efce9189514..bbd9b27ee6572 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java @@ -176,7 +176,8 @@ public FileStoreCommitImpl( this.manifestFile = manifestFileFactory.create(); this.manifestList = manifestListFactory.create(); this.indexManifestFile = indexManifestFileFactory.create(); - this.scan = scan; + // Stats in DELETE Manifest Entries is useless + this.scan = scan.dropStats(); this.numBucket = numBucket; this.manifestTargetSize = manifestTargetSize; this.manifestFullCompactionSize = manifestFullCompactionSize; From a79b08bc668c16e143e3f0126d59dea2a7253b11 Mon Sep 17 00:00:00 2001 From: LsomeYeah <94825748+LsomeYeah@users.noreply.github.com> Date: Wed, 27 Nov 2024 15:09:19 +0800 Subject: [PATCH 071/157] [core] Add tests for drop stats for overwrite commit (#4600) --- .../paimon/operation/FileStoreCommitTest.java | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java index 67945df60ce83..de4ee684b830f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java @@ -31,8 +31,11 @@ import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.IndexManifestEntry; import org.apache.paimon.manifest.ManifestCommittable; +import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.manifest.ManifestFileMeta; import org.apache.paimon.mergetree.compact.DeduplicateMergeFunction; import org.apache.paimon.predicate.PredicateBuilder; @@ -80,6 +83,7 @@ import static org.apache.paimon.index.HashIndexFile.HASH_INDEX; import static org.apache.paimon.partition.PartitionPredicate.createPartitionPredicate; +import static org.apache.paimon.stats.SimpleStats.EMPTY_STATS; import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -943,6 +947,31 @@ public void testManifestCompact() throws Exception { .isEqualTo(0); } + @Test + public void testDropStatsForOverwrite() throws Exception { + TestFileStore store = createStore(false); + + List keyValues = generateDataList(1); + BinaryRow partition = gen.getPartition(keyValues.get(0)); + // commit 1 + Snapshot snapshot1 = + store.commitData(keyValues, s -> partition, kv -> 0, Collections.emptyMap()).get(0); + // overwrite commit 2 + Snapshot snapshot2 = + store.overwriteData(keyValues, s -> partition, kv -> 0, Collections.emptyMap()) + .get(0); + ManifestFile manifestFile = store.manifestFileFactory().create(); + List entries = + store.manifestListFactory().create().readDataManifests(snapshot2).stream() + .flatMap(meta -> manifestFile.read(meta.fileName()).stream()) + .collect(Collectors.toList()); + for (ManifestEntry manifestEntry : entries) { + if (manifestEntry.kind() == FileKind.DELETE) { + assertThat(manifestEntry.file().valueStats()).isEqualTo(EMPTY_STATS); + } + } + } + @Test public void testManifestCompactFull() throws Exception { // Disable full compaction by options. From 530efddbe2f201cbfbe9a5e40211464f3df4895a Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Wed, 27 Nov 2024 16:58:07 +0800 Subject: [PATCH 072/157] [test] Fix build Spark utit workflow on jdk11 (#4596) --- .github/workflows/utitcase-jdk11.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/utitcase-jdk11.yml b/.github/workflows/utitcase-jdk11.yml index f1d7c25cbe44a..878ce5f968981 100644 --- a/.github/workflows/utitcase-jdk11.yml +++ b/.github/workflows/utitcase-jdk11.yml @@ -53,7 +53,7 @@ jobs: jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" test_modules="!paimon-e2e-tests,!org.apache.paimon:paimon-hive-connector-3.1," - for suffix in 3.5 3.4 3.3 3.2 common; do + for suffix in 3.5 3.4 3.3 3.2 ut; do test_modules+="!org.apache.paimon:paimon-spark-${suffix}," done test_modules="${test_modules%,}" From 7e4148ab890f1e62b448447216a9b9558bd025bc Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Wed, 27 Nov 2024 17:46:13 +0800 Subject: [PATCH 073/157] [flink] Avoids miscellaneous deprecated APIs for 2.0 Preview (#4590) --- .../org/apache/paimon/flink/FlinkFileIO.java | 24 ++++++++++++++++-- .../flink/source/FileStoreSourceReader.java | 23 ----------------- .../source/align/AlignedSourceReader.java | 2 +- .../paimon/flink/CatalogTableITCase.java | 25 +++++++++++-------- .../flink/source/IteratorSourcesITCase.java | 4 +-- .../util/MiniClusterWithClientExtension.java | 6 ----- .../flink/util/ReadWriteTableTestUtil.java | 6 ++--- 7 files changed, 42 insertions(+), 48 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java index 74512409bfc80..617d25125f371 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java @@ -29,10 +29,10 @@ import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystem.WriteMode; -import org.apache.flink.core.fs.FileSystemKind; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Locale; /** Flink {@link FileIO} to use {@link FileSystem}. */ public class FlinkFileIO implements FileIO { @@ -48,7 +48,27 @@ public FlinkFileIO(Path path) { @Override public boolean isObjectStore() { try { - return path.getFileSystem().getKind() != FileSystemKind.FILE_SYSTEM; + FileSystem fs = path.getFileSystem(); + String scheme = fs.getUri().getScheme().toLowerCase(Locale.US); + + if (scheme.startsWith("s3") + || scheme.startsWith("emr") + || scheme.startsWith("oss") + || scheme.startsWith("wasb") + || scheme.startsWith("gs")) { + // the Amazon S3 storage or Aliyun OSS storage or Azure Blob Storage + // or Google Cloud Storage + return true; + } else if (scheme.startsWith("http") || scheme.startsWith("ftp")) { + // file servers instead of file systems + // they might actually be consistent, but we have no hard guarantees + // currently to rely on that + return true; + } else { + // the remainder should include hdfs, kosmos, ceph, ... + // this also includes federated HDFS (viewfs). + return false; + } } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java index 92adf5e04998e..8fc78c868ba57 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java @@ -25,9 +25,7 @@ import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.file.src.reader.BulkFormat.RecordIterator; import org.apache.flink.table.data.RowData; @@ -64,27 +62,6 @@ public FileStoreSourceReader( this.ioManager = ioManager; } - public FileStoreSourceReader( - SourceReaderContext readerContext, - TableRead tableRead, - FileStoreSourceReaderMetrics metrics, - IOManager ioManager, - @Nullable Long limit, - FutureCompletingBlockingQueue>> - elementsQueue) { - super( - elementsQueue, - () -> - new FileStoreSourceSplitReader( - tableRead, RecordLimiter.create(limit), metrics), - (element, output, state) -> - FlinkRecordsWithSplitIds.emitRecord( - readerContext, element, output, state, metrics), - readerContext.getConfiguration(), - readerContext); - this.ioManager = ioManager; - } - @Override public void start() { // we request a split only if we did not get splits during the checkpoint restore diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java index 1f0bbca314b63..a8ffe3de561f9 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java @@ -58,7 +58,7 @@ public AlignedSourceReader( @Nullable Long limit, FutureCompletingBlockingQueue>> elementsQueue) { - super(readerContext, tableRead, metrics, ioManager, limit, elementsQueue); + super(readerContext, tableRead, metrics, ioManager, limit); this.elementsQueue = elementsQueue; this.nextCheckpointId = null; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java index 2a855796d8d45..96334de3f87b5 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java @@ -251,17 +251,20 @@ public void testSchemasTable() { sql("ALTER TABLE T SET ('snapshot.num-retained.min' = '18')"); sql("ALTER TABLE T SET ('manifest.format' = 'avro')"); - assertThat(sql("SHOW CREATE TABLE T$schemas").toString()) - .isEqualTo( - "[+I[CREATE TABLE `PAIMON`.`default`.`T$schemas` (\n" - + " `schema_id` BIGINT NOT NULL,\n" - + " `fields` VARCHAR(2147483647) NOT NULL,\n" - + " `partition_keys` VARCHAR(2147483647) NOT NULL,\n" - + " `primary_keys` VARCHAR(2147483647) NOT NULL,\n" - + " `options` VARCHAR(2147483647) NOT NULL,\n" - + " `comment` VARCHAR(2147483647),\n" - + " `update_time` TIMESTAMP(3) NOT NULL\n" - + ") ]]"); + String actualResult = sql("SHOW CREATE TABLE T$schemas").toString(); + String expectedResult = + "[+I[CREATE TABLE `PAIMON`.`default`.`T$schemas` (\n" + + " `schema_id` BIGINT NOT NULL,\n" + + " `fields` VARCHAR(2147483647) NOT NULL,\n" + + " `partition_keys` VARCHAR(2147483647) NOT NULL,\n" + + " `primary_keys` VARCHAR(2147483647) NOT NULL,\n" + + " `options` VARCHAR(2147483647) NOT NULL,\n" + + " `comment` VARCHAR(2147483647),\n" + + " `update_time` TIMESTAMP(3) NOT NULL\n" + + ") ]]"; + actualResult = actualResult.replace(" ", "").replace("\n", ""); + expectedResult = expectedResult.replace(" ", "").replace("\n", ""); + assertThat(actualResult).isEqualTo(expectedResult); List result = sql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java index 8404d994fa9f2..0c5d485af7bc9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java @@ -18,10 +18,10 @@ package org.apache.paimon.flink.source; +import org.apache.commons.collections.IteratorUtils; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamUtils; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; import org.apache.flink.test.util.MiniClusterWithClientResource; @@ -67,7 +67,7 @@ public void testParallelSourceExecution() throws Exception { "iterator source"); final List result = - DataStreamUtils.collectBoundedStream(stream, "Iterator Source Test"); + IteratorUtils.toList(stream.executeAndCollect("Iterator Source Test")); verifySequence(result, 1L, 1_000L); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java index cfc23a0a44d8d..39939f78670b9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.test.junit5.InjectClusterClient; -import org.apache.flink.test.util.TestEnvironment; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.AfterEachCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; @@ -167,17 +166,12 @@ private void registerEnv(InternalMiniClusterExtension internalMiniClusterExtensi .getOptional(CoreOptions.DEFAULT_PARALLELISM) .orElse(internalMiniClusterExtension.getNumberSlots()); - TestEnvironment executionEnvironment = - new TestEnvironment( - internalMiniClusterExtension.getMiniCluster(), defaultParallelism, false); - executionEnvironment.setAsContext(); TestStreamEnvironment.setAsContext( internalMiniClusterExtension.getMiniCluster(), defaultParallelism); } private void unregisterEnv(InternalMiniClusterExtension internalMiniClusterExtension) { TestStreamEnvironment.unsetAsContext(); - TestEnvironment.unsetAsContext(); } private MiniClusterClient createMiniClusterClient( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java index 9c3170f9a96b1..0eac2ed2936e0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java @@ -23,7 +23,6 @@ import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -37,6 +36,7 @@ import javax.annotation.Nullable; import java.nio.file.Paths; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -53,7 +53,7 @@ /** Test util for {@link ReadWriteTableITCase}. */ public class ReadWriteTableTestUtil { - private static final Time TIME_OUT = Time.seconds(10); + private static final Duration TIME_OUT = Duration.ofSeconds(10); public static final int DEFAULT_PARALLELISM = 2; @@ -278,7 +278,7 @@ public static void testBatchRead(String query, List expected) throws Except try (BlockingIterator iterator = BlockingIterator.of(resultItr)) { if (!expected.isEmpty()) { List result = - iterator.collect(expected.size(), TIME_OUT.getSize(), TIME_OUT.getUnit()); + iterator.collect(expected.size(), TIME_OUT.getSeconds(), TimeUnit.SECONDS); assertThat(toInsertOnlyRows(result)) .containsExactlyInAnyOrderElementsOf(toInsertOnlyRows(expected)); } From fe693135015c535f13cbc97b2b1a94bb17cbd6c0 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 27 Nov 2024 19:01:52 +0800 Subject: [PATCH 074/157] [core] Cache statistics in AbstractFileStoreTable (#4601) --- .../apache/paimon/catalog/CachingCatalog.java | 7 +++ .../apache/paimon/stats/StatsFileHandler.java | 15 +++--- .../paimon/table/AbstractFileStoreTable.java | 47 +++++++++++++++++-- .../paimon/table/DelegatedFileStoreTable.java | 5 ++ .../apache/paimon/table/FileStoreTable.java | 3 ++ .../paimon/catalog/CachingCatalogTest.java | 7 ++- .../paimon/flink/FlinkAnalyzeTableITCase.java | 15 +++++- 7 files changed, 86 insertions(+), 13 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java index 1912ad60623cb..f67f19700d6e2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java @@ -265,6 +265,13 @@ private void putTableCache(Identifier identifier, Table table) { .maximumSize(snapshotMaxNumPerTable) .executor(Runnable::run) .build()); + storeTable.setStatsCache( + Caffeine.newBuilder() + .softValues() + .expireAfterAccess(expirationInterval) + .maximumSize(5) + .executor(Runnable::run) + .build()); if (manifestCache != null) { storeTable.setManifestCache(manifestCache); } diff --git a/paimon-core/src/main/java/org/apache/paimon/stats/StatsFileHandler.java b/paimon-core/src/main/java/org/apache/paimon/stats/StatsFileHandler.java index f9e057c7cbb32..5cb88f7257a7e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/stats/StatsFileHandler.java +++ b/paimon-core/src/main/java/org/apache/paimon/stats/StatsFileHandler.java @@ -71,13 +71,14 @@ public Optional readStats(long snapshotId) { } public Optional readStats(Snapshot snapshot) { - if (snapshot.statistics() == null) { - return Optional.empty(); - } else { - Statistics stats = statsFile.read(snapshot.statistics()); - stats.deserializeFieldsFromString(schemaManager.schema(stats.schemaId())); - return Optional.of(stats); - } + String file = snapshot.statistics(); + return file == null ? Optional.empty() : Optional.of(readStats(file)); + } + + public Statistics readStats(String file) { + Statistics stats = statsFile.read(file); + stats.deserializeFieldsFromString(schemaManager.schema(stats.schemaId())); + return stats; } /** Delete stats of the specified snapshot. */ diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index 4180ff11c1677..57966d24ce474 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -92,6 +92,7 @@ abstract class AbstractFileStoreTable implements FileStoreTable { private static final long serialVersionUID = 1L; + private static final String WATERMARK_PREFIX = "watermark-"; protected final FileIO fileIO; @@ -99,6 +100,10 @@ abstract class AbstractFileStoreTable implements FileStoreTable { protected final TableSchema tableSchema; protected final CatalogEnvironment catalogEnvironment; + @Nullable protected transient SegmentsCache manifestCache; + @Nullable protected transient Cache snapshotCache; + @Nullable protected transient Cache statsCache; + protected AbstractFileStoreTable( FileIO fileIO, Path path, @@ -122,14 +127,21 @@ public String currentBranch() { @Override public void setManifestCache(SegmentsCache manifestCache) { + this.manifestCache = manifestCache; store().setManifestCache(manifestCache); } @Override public void setSnapshotCache(Cache cache) { + this.snapshotCache = cache; store().setSnapshotCache(cache); } + @Override + public void setStatsCache(Cache cache) { + this.statsCache = cache; + } + @Override public OptionalLong latestSnapshotId() { Long snapshot = store().snapshotManager().latestSnapshotId(); @@ -187,7 +199,21 @@ public String uuid() { public Optional statistics() { Snapshot snapshot = TimeTravelUtil.resolveSnapshot(this); if (snapshot != null) { - return store().newStatsFileHandler().readStats(snapshot); + String file = snapshot.statistics(); + if (file == null) { + return Optional.empty(); + } + if (statsCache != null) { + Statistics stats = statsCache.getIfPresent(file); + if (stats != null) { + return Optional.of(stats); + } + } + Statistics stats = store().newStatsFileHandler().readStats(file); + if (statsCache != null) { + statsCache.put(file, stats); + } + return Optional.of(stats); } return Optional.empty(); } @@ -342,9 +368,22 @@ public FileStoreTable copyWithLatestSchema() { @Override public FileStoreTable copy(TableSchema newTableSchema) { - return newTableSchema.primaryKeys().isEmpty() - ? new AppendOnlyFileStoreTable(fileIO, path, newTableSchema, catalogEnvironment) - : new PrimaryKeyFileStoreTable(fileIO, path, newTableSchema, catalogEnvironment); + AbstractFileStoreTable copied = + newTableSchema.primaryKeys().isEmpty() + ? new AppendOnlyFileStoreTable( + fileIO, path, newTableSchema, catalogEnvironment) + : new PrimaryKeyFileStoreTable( + fileIO, path, newTableSchema, catalogEnvironment); + if (snapshotCache != null) { + copied.setSnapshotCache(snapshotCache); + } + if (manifestCache != null) { + copied.setManifestCache(manifestCache); + } + if (statsCache != null) { + copied.setStatsCache(statsCache); + } + return copied; } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java index 624476b5b43e0..0a548941bedcd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java @@ -130,6 +130,11 @@ public void setSnapshotCache(Cache cache) { wrapped.setSnapshotCache(cache); } + @Override + public void setStatsCache(Cache cache) { + wrapped.setStatsCache(cache); + } + @Override public TableSchema schema() { return wrapped.schema(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java index d37e57e4e57ec..61aa77d5f36a5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java @@ -24,6 +24,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.stats.Statistics; import org.apache.paimon.table.query.LocalTableQuery; import org.apache.paimon.table.sink.RowKeyExtractor; import org.apache.paimon.table.sink.TableCommitImpl; @@ -47,6 +48,8 @@ public interface FileStoreTable extends DataTable { void setSnapshotCache(Cache cache); + void setStatsCache(Cache cache); + @Override default RowType rowType() { return schema().logicalRowType(); diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java index e4f0a1510b8de..7567f682ae604 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java @@ -352,6 +352,10 @@ public void testSnapshotCache() throws Exception { Snapshot snapshot = table.snapshot(1); assertThat(snapshot).isSameAs(table.snapshot(1)); + + // copy + Snapshot copied = table.copy(Collections.singletonMap("a", "b")).snapshot(1); + assertThat(copied).isSameAs(snapshot); } @Test @@ -386,7 +390,8 @@ private void innerTestManifestCache(long manifestCacheThreshold) throws Exceptio // repeat read for (int i = 0; i < 5; i++) { - table = catalog.getTable(tableIdent); + // test copy too + table = catalog.getTable(tableIdent).copy(Collections.singletonMap("a", "b")); ReadBuilder readBuilder = table.newReadBuilder(); TableScan scan = readBuilder.newScan(); TableRead read = readBuilder.newRead(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkAnalyzeTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkAnalyzeTableITCase.java index ad8a2d45a0368..e186080d9f45d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkAnalyzeTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkAnalyzeTableITCase.java @@ -22,11 +22,13 @@ import org.apache.paimon.data.Decimal; import org.apache.paimon.stats.ColStats; import org.apache.paimon.stats.Statistics; +import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.DateTimeUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.Collections; import java.util.Map; import java.util.Optional; @@ -51,7 +53,8 @@ public void testAnalyzeTable() throws Catalog.TableNotExistException { sql("INSERT INTO T VALUES ('2', 'aaa', 1, 2)"); sql("ANALYZE TABLE T COMPUTE STATISTICS"); - Optional statisticsOpt = paimonTable("T").statistics(); + FileStoreTable table = paimonTable("T"); + Optional statisticsOpt = table.statistics(); assertThat(statisticsOpt.isPresent()).isTrue(); Statistics stats = statisticsOpt.get(); @@ -60,6 +63,16 @@ public void testAnalyzeTable() throws Catalog.TableNotExistException { Assertions.assertTrue(stats.mergedRecordSize().isPresent()); Assertions.assertTrue(stats.colStats().isEmpty()); + + // by default, caching catalog should cache it + Optional newStats = table.statistics(); + assertThat(newStats.isPresent()).isTrue(); + assertThat(newStats.get()).isSameAs(stats); + + // copy the table + newStats = table.copy(Collections.singletonMap("a", "b")).statistics(); + assertThat(newStats.isPresent()).isTrue(); + assertThat(newStats.get()).isSameAs(stats); } @Test From 4bcf8576b3a5af66adb47dba7b8427581d080aab Mon Sep 17 00:00:00 2001 From: HunterXHunter <1356469429@qq.com> Date: Wed, 27 Nov 2024 19:11:39 +0800 Subject: [PATCH 075/157] [flink] Support minor compact strategy for dedicated compaction action. (#4589) --- docs/content/flink/procedures.md | 15 +- .../maintenance/dedicated-compaction.md | 8 + docs/content/spark/procedures.md | 4 +- .../procedure/CompactDatabaseProcedure.java | 13 +- .../flink/procedure/CompactProcedure.java | 16 +- .../ProcedurePositionalArgumentsITCase.java | 8 +- .../paimon/flink/action/ActionFactory.java | 4 + .../paimon/flink/action/CompactAction.java | 17 +- .../flink/action/CompactActionFactory.java | 24 +- .../flink/action/CompactDatabaseAction.java | 47 ++-- .../action/CompactDatabaseActionFactory.java | 15 +- .../procedure/CompactDatabaseProcedure.java | 13 +- .../flink/procedure/CompactProcedure.java | 13 +- .../sink/CombinedTableCompactorSink.java | 17 +- .../flink/sink/CompactorSinkBuilder.java | 10 +- .../sink/MultiTablesStoreCompactOperator.java | 6 +- .../flink/action/CompactActionITCase.java | 35 --- .../flink/action/CompactActionITCaseBase.java | 36 +++ .../action/MinorCompactActionITCase.java | 205 ++++++++++++++++++ .../procedure/CompactProcedureITCase.java | 112 ++++++++++ .../flink/sink/CompactorSinkITCase.java | 14 +- .../spark/procedure/CompactProcedure.java | 35 ++- .../procedure/CompactProcedureTestBase.scala | 50 +++++ 23 files changed, 622 insertions(+), 95 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MinorCompactActionITCase.java diff --git a/docs/content/flink/procedures.md b/docs/content/flink/procedures.md index 7e669a89d43bf..59b02f82bf8c0 100644 --- a/docs/content/flink/procedures.md +++ b/docs/content/flink/procedures.md @@ -67,7 +67,8 @@ All available procedures are listed below. order_by => 'order_by', options => 'options', `where` => 'where', - partition_idle_time => 'partition_idle_time')

+ partition_idle_time => 'partition_idle_time', + compact_strategy => 'compact_strategy')

-- Use indexed argument
CALL [catalog.]sys.compact('table')

CALL [catalog.]sys.compact('table', 'partitions')

@@ -76,6 +77,7 @@ All available procedures are listed below. CALL [catalog.]sys.compact('table', 'partitions', 'order_strategy', 'order_by', 'options')

CALL [catalog.]sys.compact('table', 'partitions', 'order_strategy', 'order_by', 'options', 'where')

CALL [catalog.]sys.compact('table', 'partitions', 'order_strategy', 'order_by', 'options', 'where', 'partition_idle_time')

+ CALL [catalog.]sys.compact('table', 'partitions', 'order_strategy', 'order_by', 'options', 'where', 'partition_idle_time', 'compact_strategy')

To compact a table. Arguments: @@ -86,6 +88,7 @@ All available procedures are listed below.

  • options(optional): additional dynamic options of the table.
  • where(optional): partition predicate(Can't be used together with "partitions"). Note: as where is a keyword,a pair of backticks need to add around like `where`.
  • partition_idle_time(optional): this is used to do a full compaction for partition which had not received any new data for 'partition_idle_time'. And only these partitions will be compacted. This argument can not be used with order compact.
  • +
  • compact_strategy(optional): this determines how to pick files to be merged, the default is determined by the runtime execution mode. 'full' strategy only supports batch mode. All files will be selected for merging. 'minor' strategy: Pick the set of files that need to be merged based on specified conditions.
  • -- use partition filter
    @@ -104,7 +107,8 @@ All available procedures are listed below. including_tables => 'includingTables', excluding_tables => 'excludingTables', table_options => 'tableOptions', - partition_idle_time => 'partitionIdleTime')

    + partition_idle_time => 'partitionIdleTime', + compact_strategy => 'compact_strategy')

    -- Use indexed argument
    CALL [catalog.]sys.compact_database()

    CALL [catalog.]sys.compact_database('includingDatabases')

    @@ -112,7 +116,8 @@ All available procedures are listed below. CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables')

    CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables')

    CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions')

    - CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions', 'partitionIdleTime') + CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions', 'partitionIdleTime')

    + CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions', 'partitionIdleTime', 'compact_strategy')

    To compact databases. Arguments: @@ -124,6 +129,7 @@ All available procedures are listed below.
  • excludingTables: to specify tables that are not compacted. You can use regular expression.
  • tableOptions: additional dynamic options of the table.
  • partition_idle_time: this is used to do a full compaction for partition which had not received any new data for 'partition_idle_time'. And only these partitions will be compacted.
  • +
  • compact_strategy(optional): this determines how to pick files to be merged, the default is determined by the runtime execution mode. 'full' strategy only supports batch mode. All files will be selected for merging. 'minor' strategy: Pick the set of files that need to be merged based on specified conditions.
  • CALL sys.compact_database( @@ -131,7 +137,8 @@ All available procedures are listed below. mode => 'combined', including_tables => 'table_.*', excluding_tables => 'ignore', - table_options => 'sink.parallelism=4') + table_options => 'sink.parallelism=4', + compat_strategy => 'full') diff --git a/docs/content/maintenance/dedicated-compaction.md b/docs/content/maintenance/dedicated-compaction.md index c0010bf9cc854..63e0aa5e66e4b 100644 --- a/docs/content/maintenance/dedicated-compaction.md +++ b/docs/content/maintenance/dedicated-compaction.md @@ -107,6 +107,7 @@ Run the following command to submit a compaction job for the table. --database \ --table \ [--partition ] \ + [--compact_strategy ] \ [--table_conf ] \ [--catalog_conf [--catalog_conf ...]] ``` @@ -123,10 +124,14 @@ Example: compact table --partition dt=20221126,hh=08 \ --partition dt=20221127,hh=09 \ --table_conf sink.parallelism=10 \ + --compact_strategy minor \ --catalog_conf s3.endpoint=https://****.com \ --catalog_conf s3.access-key=***** \ --catalog_conf s3.secret-key=***** ``` +* `--compact_strategy` Determines how to pick files to be merged, the default is determined by the runtime execution mode, streaming-mode use `minor` strategy and batch-mode use `full` strategy. + * `full` : Only supports batch mode. All files will be selected for merging. + * `minor` : Pick the set of files that need to be merged based on specified conditions. You can use `-D execution.runtime-mode=batch` or `-yD execution.runtime-mode=batch` (for the ON-YARN scenario) to control batch or streaming mode. If you submit a batch job, all current table files will be compacted. If you submit a streaming job, the job will continuously monitor new changes @@ -190,6 +195,7 @@ CALL sys.compact_database( [--including_tables ] \ [--excluding_tables ] \ [--mode ] \ + [--compact_strategy ] \ [--catalog_conf [--catalog_conf ...]] \ [--table_conf [--table_conf ...]] ``` @@ -346,6 +352,7 @@ CALL sys.compact(`table` => 'default.T', 'partition_idle_time' => '1 d') --table \ --partition_idle_time \ [--partition ] \ + [--compact_strategy ] \ [--catalog_conf [--catalog_conf ...]] \ [--table_conf [--table_conf ] ...] ``` @@ -406,6 +413,7 @@ CALL sys.compact_database( [--including_tables ] \ [--excluding_tables ] \ [--mode ] \ + [--compact_strategy ] \ [--catalog_conf [--catalog_conf ...]] \ [--table_conf [--table_conf ...]] ``` diff --git a/docs/content/spark/procedures.md b/docs/content/spark/procedures.md index 1f3f554106ec5..88d46fabbb2bc 100644 --- a/docs/content/spark/procedures.md +++ b/docs/content/spark/procedures.md @@ -47,12 +47,14 @@ This section introduce all available spark procedures about paimon.
  • order_strategy: 'order' or 'zorder' or 'hilbert' or 'none'. Left empty for 'none'.
  • order_columns: the columns need to be sort. Left empty if 'order_strategy' is 'none'.
  • partition_idle_time: this is used to do a full compaction for partition which had not received any new data for 'partition_idle_time'. And only these partitions will be compacted. This argument can not be used with order compact.
  • +
  • compact_strategy: this determines how to pick files to be merged, the default is determined by the runtime execution mode. 'full' strategy only supports batch mode. All files will be selected for merging. 'minor' strategy: Pick the set of files that need to be merged based on specified conditions.
  • SET spark.sql.shuffle.partitions=10; --set the compact parallelism

    CALL sys.compact(table => 'T', partitions => 'p=0;p=1', order_strategy => 'zorder', order_by => 'a,b')

    CALL sys.compact(table => 'T', where => 'p>0 and p<3', order_strategy => 'zorder', order_by => 'a,b')

    - CALL sys.compact(table => 'T', partition_idle_time => '60s') + CALL sys.compact(table => 'T', partition_idle_time => '60s')

    + CALL sys.compact(table => 'T', compact_strategy => 'minor')

    diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java index 99f205bacb58e..ac4340c113365 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java @@ -26,6 +26,8 @@ import java.util.Map; +import static org.apache.paimon.flink.action.ActionFactory.FULL; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; import static org.apache.paimon.utils.ParameterUtils.parseCommaSeparatedKeyValues; /** @@ -51,6 +53,7 @@ * * -- set table options ('k=v,...') * CALL sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions') + * * */ public class CompactDatabaseProcedure extends ProcedureBase { @@ -106,7 +109,8 @@ public String[] call( includingTables, excludingTables, tableOptions, - ""); + "", + null); } public String[] call( @@ -116,7 +120,8 @@ public String[] call( String includingTables, String excludingTables, String tableOptions, - String partitionIdleTime) + String partitionIdleTime, + String compactStrategy) throws Exception { String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); @@ -133,6 +138,10 @@ public String[] call( action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } + return execute(procedureContext, action, "Compact database job"); } diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java index 63aa6c906b947..560e532a6dbb3 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java @@ -31,6 +31,9 @@ import java.util.Collections; import java.util.Map; +import static org.apache.paimon.flink.action.ActionFactory.FULL; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; + /** * Stay compatible with 1.18 procedure which doesn't support named argument. Usage: * @@ -49,6 +52,9 @@ * -- compact specific partitions with sorting * CALL sys.compact('tableId', 'partitions', 'ORDER/ZORDER', 'col1,col2', 'sink.parallelism=6') * + * -- compact with specific compact strategy + * CALL sys.compact('tableId', 'partitions', 'ORDER/ZORDER', 'col1,col2', 'sink.parallelism=6', 'minor') + * * */ public class CompactProcedure extends ProcedureBase { @@ -118,7 +124,8 @@ public String[] call( orderByColumns, tableOptions, whereSql, - ""); + "", + null); } public String[] call( @@ -129,7 +136,8 @@ public String[] call( String orderByColumns, String tableOptions, String whereSql, - String partitionIdleTime) + String partitionIdleTime, + String compactStrategy) throws Exception { String warehouse = catalog.warehouse(); @@ -152,6 +160,10 @@ public String[] call( if (!(StringUtils.isNullOrWhitespaceOnly(partitionIdleTime))) { action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } jobName = "Compact Job"; } else if (!orderStrategy.isEmpty() && !orderByColumns.isEmpty()) { Preconditions.checkArgument( diff --git a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java index f2385e66d2a1d..f79d6fb716b4c 100644 --- a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java +++ b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java @@ -68,10 +68,16 @@ public void testCompactDatabaseAndTable() { sql( "CALL sys.compact('default.T', '', '', '', 'sink.parallelism=1','pt=1')")) .doesNotThrowAnyException(); - assertThatCode(() -> sql("CALL sys.compact('default.T', '', 'zorder', 'k', '','','5s')")) + assertThatCode( + () -> + sql( + "CALL sys.compact('default.T', '' ,'zorder', 'k', '','','5s', '')")) .message() .contains("sort compact do not support 'partition_idle_time'."); + assertThatCode(() -> sql("CALL sys.compact('default.T', '', '' ,'', '', '', '', 'full')")) + .doesNotThrowAnyException(); + assertThatCode(() -> sql("CALL sys.compact_database('default')")) .doesNotThrowAnyException(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java index 43719f715d9de..fbf8f12f49eb6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java @@ -58,6 +58,10 @@ public interface ActionFactory extends Factory { String TIMESTAMPFORMATTER = "timestamp_formatter"; String EXPIRE_STRATEGY = "expire_strategy"; String TIMESTAMP_PATTERN = "timestamp_pattern"; + // Supports `full` and `minor`. + String COMPACT_STRATEGY = "compact_strategy"; + String MINOR = "minor"; + String FULL = "full"; Optional create(MultipleParameterToolAdapter params); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java index ce88857f1b141..84e37a5b10f9c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java @@ -59,6 +59,8 @@ public class CompactAction extends TableActionBase { @Nullable private Duration partitionIdleTime = null; + private Boolean fullCompaction; + public CompactAction(String warehouse, String database, String tableName) { this(warehouse, database, tableName, Collections.emptyMap(), Collections.emptyMap()); } @@ -100,6 +102,11 @@ public CompactAction withPartitionIdleTime(@Nullable Duration partitionIdleTime) return this; } + public CompactAction withFullCompaction(Boolean fullCompaction) { + this.fullCompaction = fullCompaction; + return this; + } + @Override public void build() throws Exception { ReadableConfig conf = env.getConfiguration(); @@ -124,6 +131,13 @@ public void build() throws Exception { private void buildForTraditionalCompaction( StreamExecutionEnvironment env, FileStoreTable table, boolean isStreaming) throws Exception { + if (fullCompaction == null) { + fullCompaction = !isStreaming; + } else { + Preconditions.checkArgument( + !(fullCompaction && isStreaming), + "The full compact strategy is only supported in batch mode. Please add -Dexecution.runtime-mode=BATCH."); + } if (isStreaming) { // for completely asynchronous compaction HashMap dynamicOptions = @@ -138,8 +152,7 @@ private void buildForTraditionalCompaction( } CompactorSourceBuilder sourceBuilder = new CompactorSourceBuilder(identifier.getFullName(), table); - CompactorSinkBuilder sinkBuilder = - new CompactorSinkBuilder(table).withFullCompaction(!isStreaming); + CompactorSinkBuilder sinkBuilder = new CompactorSinkBuilder(table, fullCompaction); sourceBuilder.withPartitionPredicate(getPredicate()); DataStreamSource source = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java index f43c7a747c990..fc60a870eabe1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java @@ -76,6 +76,10 @@ public Optional create(MultipleParameterToolAdapter params) { action.withPartitionIdleTime( TimeUtils.parseDuration(params.get(PARTITION_IDLE_TIME))); } + String compactStrategy = params.get(COMPACT_STRATEGY); + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } } if (params.has(PARTITION)) { @@ -88,6 +92,19 @@ public Optional create(MultipleParameterToolAdapter params) { return Optional.of(action); } + public static boolean checkCompactStrategy(String compactStrategy) { + if (compactStrategy != null) { + Preconditions.checkArgument( + compactStrategy.equalsIgnoreCase(MINOR) + || compactStrategy.equalsIgnoreCase(FULL), + String.format( + "The compact strategy only supports 'full' or 'minor', but '%s' is configured.", + compactStrategy)); + return true; + } + return false; + } + @Override public void printHelp() { System.out.println( @@ -101,7 +118,8 @@ public void printHelp() { + "[--order_strategy ]" + "[--table_conf =]" + "[--order_by ]" - + "[--partition_idle_time ]"); + + "[--partition_idle_time ]" + + "[--compact_strategy ]"); System.out.println( " compact --warehouse s3://path/to/warehouse --database " + "--table [--catalog_conf [--catalog_conf ...]]"); @@ -132,6 +150,10 @@ public void printHelp() { System.out.println( " compact --warehouse hdfs:///path/to/warehouse --database test_db --table test_table " + "--partition_idle_time 10s"); + System.out.println( + "--compact_strategy determines how to pick files to be merged, the default is determined by the runtime execution mode. " + + "`full` : Only supports batch mode. All files will be selected for merging." + + "`minor`: Pick the set of files that need to be merged based on specified conditions."); System.out.println( " compact --warehouse s3:///path/to/warehouse " + "--database test_db " diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java index 471c6fdd4da69..124d3ca687761 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java @@ -72,6 +72,10 @@ public class CompactDatabaseAction extends ActionBase { @Nullable private Duration partitionIdleTime = null; + private Boolean fullCompaction; + + private boolean isStreaming; + public CompactDatabaseAction(String warehouse, Map catalogConfig) { super(warehouse, catalogConfig); } @@ -110,6 +114,11 @@ public CompactDatabaseAction withPartitionIdleTime(@Nullable Duration partitionI return this; } + public CompactDatabaseAction withFullCompaction(boolean fullCompaction) { + this.fullCompaction = fullCompaction; + return this; + } + private boolean shouldCompactionTable(String paimonFullTableName) { boolean shouldCompaction = includingPattern.matcher(paimonFullTableName).matches(); if (excludingPattern != null) { @@ -124,6 +133,12 @@ private boolean shouldCompactionTable(String paimonFullTableName) { @Override public void build() { + ReadableConfig conf = env.getConfiguration(); + isStreaming = conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; + + if (fullCompaction == null) { + fullCompaction = !isStreaming; + } if (databaseCompactMode == MultiTablesSinkMode.DIVIDED) { buildForDividedMode(); } else { @@ -170,24 +185,19 @@ private void buildForDividedMode() { !tableMap.isEmpty(), "no tables to be compacted. possible cause is that there are no tables detected after pattern matching"); - ReadableConfig conf = env.getConfiguration(); - boolean isStreaming = - conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; for (Map.Entry entry : tableMap.entrySet()) { FileStoreTable fileStoreTable = entry.getValue(); switch (fileStoreTable.bucketMode()) { case BUCKET_UNAWARE: { - buildForUnawareBucketCompaction( - env, entry.getKey(), fileStoreTable, isStreaming); + buildForUnawareBucketCompaction(env, entry.getKey(), fileStoreTable); break; } case HASH_FIXED: case HASH_DYNAMIC: default: { - buildForTraditionalCompaction( - env, entry.getKey(), fileStoreTable, isStreaming); + buildForTraditionalCompaction(env, entry.getKey(), fileStoreTable); } } } @@ -195,9 +205,6 @@ private void buildForDividedMode() { private void buildForCombinedMode() { - ReadableConfig conf = env.getConfiguration(); - boolean isStreaming = - conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; CombinedTableCompactorSourceBuilder sourceBuilder = new CombinedTableCompactorSourceBuilder( catalogLoader(), @@ -234,15 +241,17 @@ private void buildForCombinedMode() { .buildForUnawareBucketsTableSource(), parallelism); - new CombinedTableCompactorSink(catalogLoader(), tableOptions) + new CombinedTableCompactorSink(catalogLoader(), tableOptions, fullCompaction) .sinkFrom(awareBucketTableSource, unawareBucketTableSource); } private void buildForTraditionalCompaction( - StreamExecutionEnvironment env, - String fullName, - FileStoreTable table, - boolean isStreaming) { + StreamExecutionEnvironment env, String fullName, FileStoreTable table) { + + Preconditions.checkArgument( + !(fullCompaction && isStreaming), + "The full compact strategy is only supported in batch mode. Please add -Dexecution.runtime-mode=BATCH."); + if (isStreaming) { // for completely asynchronous compaction HashMap dynamicOptions = @@ -259,8 +268,7 @@ private void buildForTraditionalCompaction( CompactorSourceBuilder sourceBuilder = new CompactorSourceBuilder(fullName, table) .withPartitionIdleTime(partitionIdleTime); - CompactorSinkBuilder sinkBuilder = - new CompactorSinkBuilder(table).withFullCompaction(!isStreaming); + CompactorSinkBuilder sinkBuilder = new CompactorSinkBuilder(table, fullCompaction); DataStreamSource source = sourceBuilder.withEnv(env).withContinuousMode(isStreaming).build(); @@ -268,10 +276,7 @@ private void buildForTraditionalCompaction( } private void buildForUnawareBucketCompaction( - StreamExecutionEnvironment env, - String fullName, - FileStoreTable table, - boolean isStreaming) { + StreamExecutionEnvironment env, String fullName, FileStoreTable table) { UnawareBucketCompactionTopoBuilder unawareBucketCompactionTopoBuilder = new UnawareBucketCompactionTopoBuilder(env, fullName, table); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java index b268709078093..5672f99dc30f3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java @@ -22,6 +22,8 @@ import java.util.Optional; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; + /** Factory to create {@link CompactDatabaseAction}. */ public class CompactDatabaseActionFactory implements ActionFactory { @@ -55,6 +57,11 @@ public Optional create(MultipleParameterToolAdapter params) { action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + String compactStrategy = params.get(COMPACT_STRATEGY); + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } + return Optional.of(action); } @@ -70,7 +77,8 @@ public void printHelp() { + "[--including_tables ] " + "[--excluding_tables ] " + "[--mode ]" - + "[--partition_idle_time ]"); + + "[--partition_idle_time ]" + + "[--compact_strategy ]"); System.out.println( " compact_database --warehouse s3://path/to/warehouse --including_databases " + "[--catalog_conf [--catalog_conf ...]]"); @@ -93,6 +101,11 @@ public void printHelp() { System.out.println( "--partition_idle_time is used to do a full compaction for partition which had not receive any new data for 'partition_idle_time' time. And only these partitions will be compacted."); System.out.println("--partition_idle_time is only supported in batch mode. "); + System.out.println( + "--compact_strategy determines how to pick files to be merged, the default is determined by the runtime execution mode. " + + "`full` : Only supports batch mode. All files will be selected for merging." + + "`minor`: Pick the set of files that need to be merged based on specified conditions."); + System.out.println(); System.out.println("Examples:"); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java index dd71e974c7b16..80602b755aa59 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java @@ -29,6 +29,8 @@ import java.util.Map; +import static org.apache.paimon.flink.action.ActionFactory.FULL; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; import static org.apache.paimon.utils.ParameterUtils.parseCommaSeparatedKeyValues; /** @@ -82,6 +84,10 @@ public class CompactDatabaseProcedure extends ProcedureBase { @ArgumentHint( name = "partition_idle_time", type = @DataTypeHint("STRING"), + isOptional = true), + @ArgumentHint( + name = "compact_strategy", + type = @DataTypeHint("STRING"), isOptional = true) }) public String[] call( @@ -91,7 +97,8 @@ public String[] call( String includingTables, String excludingTables, String tableOptions, - String partitionIdleTime) + String partitionIdleTime, + String compactStrategy) throws Exception { partitionIdleTime = notnull(partitionIdleTime); String warehouse = catalog.warehouse(); @@ -109,6 +116,10 @@ public String[] call( action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } + return execute(procedureContext, action, "Compact database job"); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java index 8589069126988..282f5af340432 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java @@ -32,6 +32,8 @@ import java.util.Collections; import java.util.Map; +import static org.apache.paimon.flink.action.ActionFactory.FULL; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; import static org.apache.paimon.utils.ParameterUtils.getPartitions; import static org.apache.paimon.utils.ParameterUtils.parseCommaSeparatedKeyValues; import static org.apache.paimon.utils.StringUtils.isNullOrWhitespaceOnly; @@ -58,6 +60,10 @@ public class CompactProcedure extends ProcedureBase { @ArgumentHint( name = "partition_idle_time", type = @DataTypeHint("STRING"), + isOptional = true), + @ArgumentHint( + name = "compact_strategy", + type = @DataTypeHint("STRING"), isOptional = true) }) public String[] call( @@ -68,7 +74,8 @@ public String[] call( String orderByColumns, String tableOptions, String where, - String partitionIdleTime) + String partitionIdleTime, + String compactStrategy) throws Exception { String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); @@ -90,6 +97,10 @@ public String[] call( if (!isNullOrWhitespaceOnly(partitionIdleTime)) { action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } jobName = "Compact Job"; } else if (!isNullOrWhitespaceOnly(orderStrategy) && !isNullOrWhitespaceOnly(orderByColumns)) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java index 87a28091fa302..ce4e37305909a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java @@ -57,11 +57,15 @@ public class CombinedTableCompactorSink implements Serializable { private final Catalog.Loader catalogLoader; private final boolean ignorePreviousFiles; + private final boolean fullCompaction; + private final Options options; - public CombinedTableCompactorSink(Catalog.Loader catalogLoader, Options options) { + public CombinedTableCompactorSink( + Catalog.Loader catalogLoader, Options options, boolean fullCompaction) { this.catalogLoader = catalogLoader; this.ignorePreviousFiles = false; + this.fullCompaction = fullCompaction; this.options = options; } @@ -104,7 +108,10 @@ public DataStream doWrite( String.format("%s-%s", "Multi-Bucket-Table", WRITER_NAME), new MultiTableCommittableTypeInfo(), combinedMultiComacptionWriteOperator( - env.getCheckpointConfig(), isStreaming, commitUser)) + env.getCheckpointConfig(), + isStreaming, + fullCompaction, + commitUser)) .setParallelism(awareBucketTableSource.getParallelism()); SingleOutputStreamOperator unawareBucketTableRewriter = @@ -168,13 +175,17 @@ protected DataStreamSink doCommit( // TODO:refactor FlinkSink to adopt this sink protected OneInputStreamOperator combinedMultiComacptionWriteOperator( - CheckpointConfig checkpointConfig, boolean isStreaming, String commitUser) { + CheckpointConfig checkpointConfig, + boolean isStreaming, + boolean fullCompaction, + String commitUser) { return new MultiTablesStoreCompactOperator( catalogLoader, commitUser, checkpointConfig, isStreaming, ignorePreviousFiles, + fullCompaction, options); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSinkBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSinkBuilder.java index 2173b1d34a72e..2d84ae6726fd2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSinkBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSinkBuilder.java @@ -37,10 +37,11 @@ public class CompactorSinkBuilder { private DataStream input; - private boolean fullCompaction; + private final boolean fullCompaction; - public CompactorSinkBuilder(FileStoreTable table) { + public CompactorSinkBuilder(FileStoreTable table, boolean fullCompaction) { this.table = table; + this.fullCompaction = fullCompaction; } public CompactorSinkBuilder withInput(DataStream input) { @@ -48,11 +49,6 @@ public CompactorSinkBuilder withInput(DataStream input) { return this; } - public CompactorSinkBuilder withFullCompaction(boolean fullCompaction) { - this.fullCompaction = fullCompaction; - return this; - } - public DataStreamSink build() { BucketMode bucketMode = table.bucketMode(); switch (bucketMode) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java index 8a1d3a02df811..57d2e8413cb5b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java @@ -63,6 +63,7 @@ public class MultiTablesStoreCompactOperator private final CheckpointConfig checkpointConfig; private final boolean isStreaming; private final boolean ignorePreviousFiles; + private final boolean fullCompaction; private final String initialCommitUser; private transient StoreSinkWriteState state; @@ -81,6 +82,7 @@ public MultiTablesStoreCompactOperator( CheckpointConfig checkpointConfig, boolean isStreaming, boolean ignorePreviousFiles, + boolean fullCompaction, Options options) { super(options); this.catalogLoader = catalogLoader; @@ -88,6 +90,7 @@ public MultiTablesStoreCompactOperator( this.checkpointConfig = checkpointConfig; this.isStreaming = isStreaming; this.ignorePreviousFiles = ignorePreviousFiles; + this.fullCompaction = fullCompaction; } @Override @@ -162,13 +165,14 @@ public void processElement(StreamRecord element) throws Exception { if (write.streamingMode()) { write.notifyNewFiles(snapshotId, partition, bucket, files); + // The full compact is not supported in streaming mode. write.compact(partition, bucket, false); } else { Preconditions.checkArgument( files.isEmpty(), "Batch compact job does not concern what files are compacted. " + "They only need to know what buckets are compacted."); - write.compact(partition, bucket, true); + write.compact(partition, bucket, fullCompaction); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java index bc849f0a135ff..2c4fb64f331c3 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java @@ -23,13 +23,9 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.table.sink.StreamWriteBuilder; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.StreamTableScan; import org.apache.paimon.table.source.TableScan; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; import org.apache.paimon.utils.CommonTestUtils; import org.apache.paimon.utils.SnapshotManager; @@ -56,12 +52,6 @@ /** IT cases for {@link CompactAction}. */ public class CompactActionITCase extends CompactActionITCaseBase { - private static final DataType[] FIELD_TYPES = - new DataType[] {DataTypes.INT(), DataTypes.INT(), DataTypes.INT(), DataTypes.STRING()}; - - private static final RowType ROW_TYPE = - RowType.of(FIELD_TYPES, new String[] {"k", "v", "hh", "dt"}); - @Test @Timeout(60) public void testBatchCompact() throws Exception { @@ -402,31 +392,6 @@ public void testWrongUsage() throws Exception { .hasMessage("sort compact do not support 'partition_idle_time'."); } - private FileStoreTable prepareTable( - List partitionKeys, - List primaryKeys, - List bucketKey, - Map tableOptions) - throws Exception { - FileStoreTable table = - createFileStoreTable(ROW_TYPE, partitionKeys, primaryKeys, bucketKey, tableOptions); - - StreamWriteBuilder streamWriteBuilder = - table.newStreamWriteBuilder().withCommitUser(commitUser); - write = streamWriteBuilder.newWrite(); - commit = streamWriteBuilder.newCommit(); - - return table; - } - - private void checkLatestSnapshot( - FileStoreTable table, long snapshotId, Snapshot.CommitKind commitKind) { - SnapshotManager snapshotManager = table.snapshotManager(); - Snapshot snapshot = snapshotManager.snapshot(snapshotManager.latestSnapshotId()); - assertThat(snapshot.id()).isEqualTo(snapshotId); - assertThat(snapshot.commitKind()).isEqualTo(commitKind); - } - private void runAction(boolean isStreaming) throws Exception { runAction(isStreaming, false); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCaseBase.java index 4c646444cb72c..41d01bdf7f35e 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCaseBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCaseBase.java @@ -18,17 +18,22 @@ package org.apache.paimon.flink.action; +import org.apache.paimon.Snapshot; import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.operation.FileStoreScan; import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.StreamWriteBuilder; import org.apache.paimon.table.source.StreamTableScan; import org.apache.paimon.table.source.TableScan; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.SnapshotManager; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.TimeoutException; @@ -37,6 +42,12 @@ /** Base IT cases for {@link CompactAction} and {@link CompactDatabaseAction} . */ public class CompactActionITCaseBase extends ActionITCaseBase { + protected static final DataType[] FIELD_TYPES = + new DataType[] {DataTypes.INT(), DataTypes.INT(), DataTypes.INT(), DataTypes.STRING()}; + + protected static final RowType ROW_TYPE = + RowType.of(FIELD_TYPES, new String[] {"k", "v", "hh", "dt"}); + protected void validateResult( FileStoreTable table, RowType rowType, @@ -87,4 +98,29 @@ protected void checkFileAndRowSize( assertThat(files.size()).isEqualTo(fileNum); assertThat(count).isEqualTo(rowCount); } + + protected void checkLatestSnapshot( + FileStoreTable table, long snapshotId, Snapshot.CommitKind commitKind) { + SnapshotManager snapshotManager = table.snapshotManager(); + Snapshot snapshot = snapshotManager.snapshot(snapshotManager.latestSnapshotId()); + assertThat(snapshot.id()).isEqualTo(snapshotId); + assertThat(snapshot.commitKind()).isEqualTo(commitKind); + } + + protected FileStoreTable prepareTable( + List partitionKeys, + List primaryKeys, + List bucketKey, + Map tableOptions) + throws Exception { + FileStoreTable table = + createFileStoreTable(ROW_TYPE, partitionKeys, primaryKeys, bucketKey, tableOptions); + + StreamWriteBuilder streamWriteBuilder = + table.newStreamWriteBuilder().withCommitUser(commitUser); + write = streamWriteBuilder.newWrite(); + commit = streamWriteBuilder.newCommit(); + + return table; + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MinorCompactActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MinorCompactActionITCase.java new file mode 100644 index 0000000000000..0373eb01a2d92 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MinorCompactActionITCase.java @@ -0,0 +1,205 @@ +/* + * 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.paimon.flink.action; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.Snapshot; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.source.DataSplit; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT cases for compact strategy {@link CompactAction}. */ +public class MinorCompactActionITCase extends CompactActionITCaseBase { + + @Test + @Timeout(60) + public void testBatchMinorCompactStrategy() throws Exception { + FileStoreTable table = + prepareTable( + Arrays.asList("dt", "hh"), + Arrays.asList("dt", "hh", "k"), + Collections.emptyList(), + Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "true")); + + writeData( + rowData(1, 100, 15, BinaryString.fromString("20221208")), + rowData(1, 100, 16, BinaryString.fromString("20221208"))); + + writeData( + rowData(2, 100, 15, BinaryString.fromString("20221208")), + rowData(2, 100, 16, BinaryString.fromString("20221208"))); + + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + CompactAction action = + createAction( + CompactAction.class, + "compact", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--compact_strategy", + "minor", + "--table_conf", + CoreOptions.NUM_SORTED_RUNS_COMPACTION_TRIGGER.key() + "=3"); + StreamExecutionEnvironment env = streamExecutionEnvironmentBuilder().batchMode().build(); + action.withStreamExecutionEnvironment(env).build(); + env.execute(); + + // Due to the limitation of parameter 'num-sorted-run.compaction-trigger', so compact is not + // performed. + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + // Make par-15 has 3 datafile and par-16 has 2 datafile, so par-16 will not be picked out to + // compact. + writeData(rowData(2, 100, 15, BinaryString.fromString("20221208"))); + + env = streamExecutionEnvironmentBuilder().batchMode().build(); + action.withStreamExecutionEnvironment(env).build(); + env.execute(); + + checkLatestSnapshot(table, 4, Snapshot.CommitKind.COMPACT); + + List splits = table.newSnapshotReader().read().dataSplits(); + assertThat(splits.size()).isEqualTo(2); + for (DataSplit split : splits) { + // Par-16 is not compacted. + assertThat(split.dataFiles().size()) + .isEqualTo(split.partition().getInt(1) == 16 ? 2 : 1); + } + } + + @Test + @Timeout(60) + public void testBatchFullCompactStrategy() throws Exception { + FileStoreTable table = + prepareTable( + Arrays.asList("dt", "hh"), + Arrays.asList("dt", "hh", "k"), + Collections.emptyList(), + Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "true")); + + writeData( + rowData(1, 100, 15, BinaryString.fromString("20221208")), + rowData(1, 100, 16, BinaryString.fromString("20221208"))); + + writeData( + rowData(2, 100, 15, BinaryString.fromString("20221208")), + rowData(2, 100, 16, BinaryString.fromString("20221208"))); + + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + CompactAction action = + createAction( + CompactAction.class, + "compact", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--compact_strategy", + "full", + "--table_conf", + CoreOptions.NUM_SORTED_RUNS_COMPACTION_TRIGGER.key() + "=3"); + StreamExecutionEnvironment env = streamExecutionEnvironmentBuilder().batchMode().build(); + action.withStreamExecutionEnvironment(env).build(); + env.execute(); + + checkLatestSnapshot(table, 3, Snapshot.CommitKind.COMPACT); + + List splits = table.newSnapshotReader().read().dataSplits(); + assertThat(splits.size()).isEqualTo(2); + for (DataSplit split : splits) { + assertThat(split.dataFiles().size()).isEqualTo(1); + } + } + + @Test + @Timeout(60) + public void testStreamingFullCompactStrategy() throws Exception { + prepareTable( + Arrays.asList("dt", "hh"), + Arrays.asList("dt", "hh", "k"), + Collections.emptyList(), + Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "true")); + CompactAction action = + createAction( + CompactAction.class, + "compact", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--compact_strategy", + "full", + "--table_conf", + CoreOptions.NUM_SORTED_RUNS_COMPACTION_TRIGGER.key() + "=3"); + StreamExecutionEnvironment env = + streamExecutionEnvironmentBuilder().streamingMode().build(); + Assertions.assertThatThrownBy(() -> action.withStreamExecutionEnvironment(env).build()) + .hasMessage( + "The full compact strategy is only supported in batch mode. Please add -Dexecution.runtime-mode=BATCH."); + } + + @Test + @Timeout(60) + public void testCompactStrategyWithWrongUsage() throws Exception { + prepareTable( + Arrays.asList("dt", "hh"), + Arrays.asList("dt", "hh", "k"), + Collections.emptyList(), + Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "true")); + Assertions.assertThatThrownBy( + () -> + createAction( + CompactAction.class, + "compact", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--compact_strategy", + "wrong_usage", + "--table_conf", + CoreOptions.NUM_SORTED_RUNS_COMPACTION_TRIGGER.key() + + "=3")) + .hasMessage( + "The compact strategy only supports 'full' or 'minor', but 'wrong_usage' is configured."); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java index bec669acd30d3..d79d13f0260cd 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java @@ -31,6 +31,7 @@ import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import java.util.List; @@ -240,6 +241,117 @@ public void testDynamicBucketSortCompact() throws Exception { checkLatestSnapshot(table, 21, Snapshot.CommitKind.OVERWRITE); } + // ----------------------- Minor Compact ----------------------- + + @Test + public void testBatchMinorCompactStrategy() throws Exception { + sql( + "CREATE TABLE T (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING," + + " PRIMARY KEY (k, dt, hh) NOT ENFORCED" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'" + + ")"); + FileStoreTable table = paimonTable("T"); + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + + sql("INSERT INTO T VALUES (1, 100, 15, '20221208'), (1, 100, 16, '20221208')"); + sql("INSERT INTO T VALUES (2, 100, 15, '20221208'), (2, 100, 16, '20221208')"); + + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + sql( + "CALL sys.compact(`table` => 'default.T', compact_strategy => 'minor', " + + "options => 'num-sorted-run.compaction-trigger=3')"); + + // Due to the limitation of parameter 'num-sorted-run.compaction-trigger' = 3, so compact is + // not + // performed. + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + // Make par-15 has 3 datafile and par-16 has 2 datafile, so par-16 will not be picked out to + // compact. + sql("INSERT INTO T VALUES (1, 100, 15, '20221208')"); + + sql( + "CALL sys.compact(`table` => 'default.T', compact_strategy => 'minor', " + + "options => 'num-sorted-run.compaction-trigger=3')"); + + checkLatestSnapshot(table, 4, Snapshot.CommitKind.COMPACT); + + List splits = table.newSnapshotReader().read().dataSplits(); + assertThat(splits.size()).isEqualTo(2); + for (DataSplit split : splits) { + // Par-16 is not compacted. + assertThat(split.dataFiles().size()) + .isEqualTo(split.partition().getInt(1) == 16 ? 2 : 1); + } + } + + @Test + public void testBatchFullCompactStrategy() throws Exception { + sql( + "CREATE TABLE T (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING," + + " PRIMARY KEY (k, dt, hh) NOT ENFORCED" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'" + + ")"); + FileStoreTable table = paimonTable("T"); + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + + sql("INSERT INTO T VALUES (1, 100, 15, '20221208'), (1, 100, 16, '20221208')"); + sql("INSERT INTO T VALUES (2, 100, 15, '20221208'), (2, 100, 16, '20221208')"); + + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + sql( + "CALL sys.compact(`table` => 'default.T', compact_strategy => 'full', " + + "options => 'num-sorted-run.compaction-trigger=3')"); + + checkLatestSnapshot(table, 3, Snapshot.CommitKind.COMPACT); + + List splits = table.newSnapshotReader().read().dataSplits(); + assertThat(splits.size()).isEqualTo(2); + for (DataSplit split : splits) { + // Par-16 is not compacted. + assertThat(split.dataFiles().size()).isEqualTo(1); + } + } + + @Test + public void testStreamFullCompactStrategy() throws Exception { + sql( + "CREATE TABLE T (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING," + + " PRIMARY KEY (k, dt, hh) NOT ENFORCED" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'" + + ")"); + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + + Assertions.assertThatThrownBy( + () -> + streamSqlIter( + "CALL sys.compact(`table` => 'default.T', compact_strategy => 'full', " + + "options => 'num-sorted-run.compaction-trigger=3')") + .close()) + .hasMessageContaining( + "The full compact strategy is only supported in batch mode. Please add -Dexecution.runtime-mode=BATCH."); + } + private void checkLatestSnapshot( FileStoreTable table, long snapshotId, Snapshot.CommitKind commitKind) { SnapshotManager snapshotManager = table.snapshotManager(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java index c38ac4b3d685c..42293ca2842e1 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java @@ -132,7 +132,7 @@ public void testCompact() throws Exception { .withContinuousMode(false) .withPartitionPredicate(predicate) .build(); - new CompactorSinkBuilder(table).withFullCompaction(true).withInput(source).build(); + new CompactorSinkBuilder(table, true).withInput(source).build(); env.execute(); snapshot = snapshotManager.snapshot(snapshotManager.latestSnapshotId()); @@ -181,8 +181,8 @@ public void testCompactParallelism() throws Exception { FlinkConnectorOptions.SINK_PARALLELISM.key(), String.valueOf(sinkParalellism)); } - })) - .withFullCompaction(false) + }), + false) .withInput(source) .build(); @@ -275,7 +275,13 @@ protected StoreCompactOperator createCompactOperator(FileStoreTable table) { protected MultiTablesStoreCompactOperator createMultiTablesCompactOperator( Catalog.Loader catalogLoader) throws Exception { return new MultiTablesStoreCompactOperator( - catalogLoader, commitUser, new CheckpointConfig(), false, false, new Options()); + catalogLoader, + commitUser, + new CheckpointConfig(), + false, + false, + true, + new Options()); } private static byte[] partition(String dt, int hh) { diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java index 71cf04cf5ef55..4a43e39c31ba1 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java @@ -107,6 +107,7 @@ public class CompactProcedure extends BaseProcedure { new ProcedureParameter[] { ProcedureParameter.required("table", StringType), ProcedureParameter.optional("partitions", StringType), + ProcedureParameter.optional("compact_strategy", StringType), ProcedureParameter.optional("order_strategy", StringType), ProcedureParameter.optional("order_by", StringType), ProcedureParameter.optional("where", StringType), @@ -120,6 +121,9 @@ public class CompactProcedure extends BaseProcedure { new StructField("result", DataTypes.BooleanType, true, Metadata.empty()) }); + private static final String MINOR = "minor"; + private static final String FULL = "full"; + protected CompactProcedure(TableCatalog tableCatalog) { super(tableCatalog); } @@ -138,15 +142,17 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); String partitions = blank(args, 1) ? null : args.getString(1); - String sortType = blank(args, 2) ? TableSorter.OrderType.NONE.name() : args.getString(2); + // make full compact strategy as default. + String compactStrategy = blank(args, 2) ? FULL : args.getString(2); + String sortType = blank(args, 3) ? TableSorter.OrderType.NONE.name() : args.getString(3); List sortColumns = - blank(args, 3) + blank(args, 4) ? Collections.emptyList() - : Arrays.asList(args.getString(3).split(",")); - String where = blank(args, 4) ? null : args.getString(4); - String options = args.isNullAt(5) ? null : args.getString(5); + : Arrays.asList(args.getString(4).split(",")); + String where = blank(args, 5) ? null : args.getString(5); + String options = args.isNullAt(6) ? null : args.getString(6); Duration partitionIdleTime = - blank(args, 6) ? null : TimeUtils.parseDuration(args.getString(6)); + blank(args, 7) ? null : TimeUtils.parseDuration(args.getString(7)); if (TableSorter.OrderType.NONE.name().equals(sortType) && !sortColumns.isEmpty()) { throw new IllegalArgumentException( "order_strategy \"none\" cannot work with order_by columns."); @@ -155,6 +161,14 @@ public InternalRow[] call(InternalRow args) { throw new IllegalArgumentException( "sort compact do not support 'partition_idle_time'."); } + + if (!(compactStrategy.equalsIgnoreCase(FULL) || compactStrategy.equalsIgnoreCase(MINOR))) { + throw new IllegalArgumentException( + String.format( + "The compact strategy only supports 'full' or 'minor', but '%s' is configured.", + compactStrategy)); + } + checkArgument( partitions == null || where == null, "partitions and where cannot be used together."); @@ -192,6 +206,7 @@ public InternalRow[] call(InternalRow args) { newInternalRow( execute( (FileStoreTable) table, + compactStrategy, sortType, sortColumns, relation, @@ -212,6 +227,7 @@ private boolean blank(InternalRow args, int index) { private boolean execute( FileStoreTable table, + String compactStrategy, String sortType, List sortColumns, DataSourceV2Relation relation, @@ -219,6 +235,7 @@ private boolean execute( @Nullable Duration partitionIdleTime) { BucketMode bucketMode = table.bucketMode(); TableSorter.OrderType orderType = TableSorter.OrderType.of(sortType); + boolean fullCompact = compactStrategy.equalsIgnoreCase(FULL); Predicate filter = condition == null ? null @@ -233,7 +250,8 @@ private boolean execute( switch (bucketMode) { case HASH_FIXED: case HASH_DYNAMIC: - compactAwareBucketTable(table, filter, partitionIdleTime, javaSparkContext); + compactAwareBucketTable( + table, fullCompact, filter, partitionIdleTime, javaSparkContext); break; case BUCKET_UNAWARE: compactUnAwareBucketTable(table, filter, partitionIdleTime, javaSparkContext); @@ -259,6 +277,7 @@ private boolean execute( private void compactAwareBucketTable( FileStoreTable table, + boolean fullCompact, @Nullable Predicate filter, @Nullable Duration partitionIdleTime, JavaSparkContext javaSparkContext) { @@ -304,7 +323,7 @@ private void compactAwareBucketTable( SerializationUtils.deserializeBinaryRow( pair.getLeft()), pair.getRight(), - true); + fullCompact); } CommitMessageSerializer serializer = new CommitMessageSerializer(); diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala index 130860c8351e8..31f78f61c20d9 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala @@ -39,6 +39,56 @@ abstract class CompactProcedureTestBase extends PaimonSparkTestBase with StreamT import testImplicits._ + // ----------------------- Minor Compact ----------------------- + + test("Paimon Procedure: compact aware bucket pk table with minor compact strategy") { + withTable("T") { + spark.sql(s""" + |CREATE TABLE T (id INT, value STRING, pt STRING) + |TBLPROPERTIES ('primary-key'='id, pt', 'bucket'='1', 'write-only'='true') + |PARTITIONED BY (pt) + |""".stripMargin) + + val table = loadTable("T") + + spark.sql(s"INSERT INTO T VALUES (1, 'a', 'p1'), (2, 'b', 'p2')") + spark.sql(s"INSERT INTO T VALUES (3, 'c', 'p1'), (4, 'd', 'p2')") + + Assertions.assertThat(lastSnapshotCommand(table).equals(CommitKind.APPEND)).isTrue + Assertions.assertThat(lastSnapshotId(table)).isEqualTo(2) + + spark.sql( + "CALL sys.compact(table => 'T', compact_strategy => 'minor'," + + "options => 'num-sorted-run.compaction-trigger=3')") + + // Due to the limitation of parameter 'num-sorted-run.compaction-trigger' = 3, so compact is not + // performed. + Assertions.assertThat(lastSnapshotCommand(table).equals(CommitKind.APPEND)).isTrue + Assertions.assertThat(lastSnapshotId(table)).isEqualTo(2) + + // Make par-p1 has 3 datafile and par-p2 has 2 datafile, so par-p2 will not be picked out to + // compact. + spark.sql(s"INSERT INTO T VALUES (1, 'a', 'p1')") + + spark.sql( + "CALL sys.compact(table => 'T', compact_strategy => 'minor'," + + "options => 'num-sorted-run.compaction-trigger=3')") + + Assertions.assertThat(lastSnapshotId(table)).isEqualTo(4) + Assertions.assertThat(lastSnapshotCommand(table).equals(CommitKind.COMPACT)).isTrue + + val splits = table.newSnapshotReader.read.dataSplits + splits.forEach( + split => { + Assertions + .assertThat(split.dataFiles.size) + .isEqualTo(if (split.partition().getString(0).toString == "p2") 2 else 1) + }) + } + } + + // ----------------------- Sort Compact ----------------------- + test("Paimon Procedure: sort compact") { failAfter(streamingTimeout) { withTempDir { From cdd4061db4b43393aab6fc5b2ce2c13ed34c69f3 Mon Sep 17 00:00:00 2001 From: xleoken Date: Thu, 28 Nov 2024 14:31:01 +0800 Subject: [PATCH 076/157] [core] Improve the performance of show tables with hive metastore (#4605) --- .../src/main/java/org/apache/paimon/hive/HiveCatalog.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 0ecc78469e154..9a90995f282d2 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -1017,10 +1017,8 @@ public Table getHmsTable(Identifier identifier) throws TableNotExistException { private boolean isPaimonTable(Identifier identifier, Table table) { return isPaimonTable(table) - && tableSchemaInFileSystem( - getTableLocation(identifier, table), - identifier.getBranchNameOrDefault()) - .isPresent(); + && tableExistsInFileSystem( + getTableLocation(identifier, table), identifier.getBranchNameOrDefault()); } private static boolean isPaimonTable(Table table) { From 0fe18e89f04f2c32a7d41560dfa2ac03134904b1 Mon Sep 17 00:00:00 2001 From: lining Date: Thu, 28 Nov 2024 19:58:31 +0800 Subject: [PATCH 077/157] [core] Add basic implementation to support REST Catalog (#4553) --- .../apache/paimon/utils/ThreadPoolUtils.java | 12 +- paimon-core/pom.xml | 57 +++++ .../paimon/rest/DefaultErrorHandler.java | 61 ++++++ .../org/apache/paimon/rest/ErrorHandler.java | 26 +++ .../org/apache/paimon/rest/HttpClient.java | 142 +++++++++++++ .../apache/paimon/rest/HttpClientOptions.java | 74 +++++++ .../org/apache/paimon/rest/RESTCatalog.java | 197 ++++++++++++++++++ .../paimon/rest/RESTCatalogFactory.java | 38 ++++ .../rest/RESTCatalogInternalOptions.java | 31 +++ .../paimon/rest/RESTCatalogOptions.java | 53 +++++ .../org/apache/paimon/rest/RESTClient.java | 31 +++ .../org/apache/paimon/rest/RESTMessage.java | 22 ++ .../apache/paimon/rest/RESTObjectMapper.java | 35 ++++ .../org/apache/paimon/rest/RESTRequest.java | 22 ++ .../org/apache/paimon/rest/RESTResponse.java | 22 ++ .../java/org/apache/paimon/rest/RESTUtil.java | 55 +++++ .../org/apache/paimon/rest/ResourcePaths.java | 34 +++ .../rest/exceptions/BadRequestException.java | 27 +++ .../rest/exceptions/ForbiddenException.java | 26 +++ .../exceptions/NotAuthorizedException.java | 26 +++ .../paimon/rest/exceptions/RESTException.java | 30 +++ .../exceptions/ServiceFailureException.java | 26 +++ .../ServiceUnavailableException.java | 26 +++ .../paimon/rest/responses/ConfigResponse.java | 76 +++++++ .../paimon/rest/responses/ErrorResponse.java | 91 ++++++++ .../src/main/resources/META-INF/NOTICE | 8 + .../org.apache.paimon.factories.Factory | 1 + .../paimon/rest/DefaultErrorHandlerTest.java | 77 +++++++ .../apache/paimon/rest/HttpClientTest.java | 129 ++++++++++++ .../org/apache/paimon/rest/MockRESTData.java | 44 ++++ .../apache/paimon/rest/RESTCatalogTest.java | 86 ++++++++ .../paimon/rest/RESTObjectMapperTest.java | 59 ++++++ paimon-open-api/Makefile | 25 +++ paimon-open-api/README.md | 10 + paimon-open-api/generate.sh | 48 +++++ paimon-open-api/pom.xml | 85 ++++++++ paimon-open-api/rest-catalog-open-api.yaml | 60 ++++++ .../paimon/open/api/OpenApiApplication.java | 31 +++ .../open/api/RESTCatalogController.java | 69 ++++++ .../paimon/open/api/config/OpenAPIConfig.java | 60 ++++++ .../src/main/resources/application.properties | 22 ++ pom.xml | 1 + 42 files changed, 2054 insertions(+), 1 deletion(-) create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java create mode 100644 paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java create mode 100644 paimon-core/src/main/resources/META-INF/NOTICE create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java create mode 100644 paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java create mode 100644 paimon-open-api/Makefile create mode 100644 paimon-open-api/README.md create mode 100755 paimon-open-api/generate.sh create mode 100644 paimon-open-api/pom.xml create mode 100644 paimon-open-api/rest-catalog-open-api.yaml create mode 100644 paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java create mode 100644 paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java create mode 100644 paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java create mode 100644 paimon-open-api/src/main/resources/application.properties diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java index 112b9ad1cda28..f8959def67d13 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/ThreadPoolUtils.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.Queue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -54,13 +55,22 @@ public class ThreadPoolUtils { * is max thread number. */ public static ThreadPoolExecutor createCachedThreadPool(int threadNum, String namePrefix) { + return createCachedThreadPool(threadNum, namePrefix, new LinkedBlockingQueue<>()); + } + + /** + * Create a thread pool with max thread number and define queue. Inactive threads will + * automatically exit. + */ + public static ThreadPoolExecutor createCachedThreadPool( + int threadNum, String namePrefix, BlockingQueue workQueue) { ThreadPoolExecutor executor = new ThreadPoolExecutor( threadNum, threadNum, 1, TimeUnit.MINUTES, - new LinkedBlockingQueue<>(), + workQueue, newDaemonThreadFactory(namePrefix)); executor.allowCoreThreadTimeOut(true); return executor; diff --git a/paimon-core/pom.xml b/paimon-core/pom.xml index 399f0b5d6c19a..e137d57a6db15 100644 --- a/paimon-core/pom.xml +++ b/paimon-core/pom.xml @@ -33,6 +33,7 @@ under the License. 6.20.3-ververica-2.0 + 4.12.0 @@ -63,6 +64,14 @@ under the License. provided + + + + com.squareup.okhttp3 + okhttp + ${okhttp.version} + + @@ -204,6 +213,20 @@ under the License. test + + com.squareup.okhttp3 + mockwebserver + ${okhttp.version} + test + + + org.mockito + mockito-core + ${mockito.version} + jar + test + + @@ -219,6 +242,40 @@ under the License. + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + * + + okhttp3/internal/publicsuffix/NOTICE + + + + + + com.squareup.okhttp3:okhttp + + + + + okhttp3 + org.apache.paimon.shade.okhttp3 + + + + + + diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java new file mode 100644 index 0000000000000..1a8618c1c603f --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java @@ -0,0 +1,61 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.rest.exceptions.BadRequestException; +import org.apache.paimon.rest.exceptions.ForbiddenException; +import org.apache.paimon.rest.exceptions.NotAuthorizedException; +import org.apache.paimon.rest.exceptions.RESTException; +import org.apache.paimon.rest.exceptions.ServiceFailureException; +import org.apache.paimon.rest.exceptions.ServiceUnavailableException; +import org.apache.paimon.rest.responses.ErrorResponse; + +/** Default error handler. */ +public class DefaultErrorHandler extends ErrorHandler { + private static final ErrorHandler INSTANCE = new DefaultErrorHandler(); + + public static ErrorHandler getInstance() { + return INSTANCE; + } + + @Override + public void accept(ErrorResponse error) { + int code = error.code(); + switch (code) { + case 400: + throw new BadRequestException( + String.format("Malformed request: %s", error.message())); + case 401: + throw new NotAuthorizedException("Not authorized: %s", error.message()); + case 403: + throw new ForbiddenException("Forbidden: %s", error.message()); + case 405: + case 406: + break; + case 500: + throw new ServiceFailureException("Server error: %s", error.message()); + case 501: + throw new UnsupportedOperationException(error.message()); + case 503: + throw new ServiceUnavailableException("Service unavailable: %s", error.message()); + } + + throw new RESTException("Unable to process: %s", error.message()); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java b/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java new file mode 100644 index 0000000000000..cdfa4bcdfaac6 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java @@ -0,0 +1,26 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.rest.responses.ErrorResponse; + +import java.util.function.Consumer; + +/** Error handler for REST client. */ +public abstract class ErrorHandler implements Consumer {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java new file mode 100644 index 0000000000000..e092711e5f971 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -0,0 +1,142 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.rest.exceptions.RESTException; +import org.apache.paimon.rest.responses.ErrorResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import okhttp3.Dispatcher; +import okhttp3.Headers; +import okhttp3.MediaType; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.RequestBody; +import okhttp3.Response; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; + +import static okhttp3.ConnectionSpec.CLEARTEXT; +import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; +import static okhttp3.ConnectionSpec.MODERN_TLS; +import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool; + +/** HTTP client for REST catalog. */ +public class HttpClient implements RESTClient { + + private final OkHttpClient okHttpClient; + private final String uri; + private final ObjectMapper mapper; + private final ErrorHandler errorHandler; + + private static final String THREAD_NAME = "REST-CATALOG-HTTP-CLIENT-THREAD-POOL"; + private static final MediaType MEDIA_TYPE = MediaType.parse("application/json"); + + public HttpClient(HttpClientOptions httpClientOptions) { + this.uri = httpClientOptions.uri(); + this.mapper = httpClientOptions.mapper(); + this.okHttpClient = createHttpClient(httpClientOptions); + this.errorHandler = httpClientOptions.errorHandler(); + } + + @Override + public T get( + String path, Class responseType, Map headers) { + try { + Request request = + new Request.Builder() + .url(uri + path) + .get() + .headers(Headers.of(headers)) + .build(); + return exec(request, responseType); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public T post( + String path, RESTRequest body, Class responseType, Map headers) { + try { + RequestBody requestBody = buildRequestBody(body); + Request request = + new Request.Builder() + .url(uri + path) + .post(requestBody) + .headers(Headers.of(headers)) + .build(); + return exec(request, responseType); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() throws IOException { + okHttpClient.dispatcher().cancelAll(); + okHttpClient.connectionPool().evictAll(); + } + + private T exec(Request request, Class responseType) { + try (Response response = okHttpClient.newCall(request).execute()) { + String responseBodyStr = response.body() != null ? response.body().string() : null; + if (!response.isSuccessful()) { + ErrorResponse error = + new ErrorResponse( + responseBodyStr != null ? responseBodyStr : "response body is null", + response.code()); + errorHandler.accept(error); + } + if (responseBodyStr == null) { + throw new RESTException("response body is null."); + } + return mapper.readValue(responseBodyStr, responseType); + } catch (Exception e) { + throw new RESTException(e, "rest exception"); + } + } + + private RequestBody buildRequestBody(RESTRequest body) throws JsonProcessingException { + return RequestBody.create(mapper.writeValueAsBytes(body), MEDIA_TYPE); + } + + private static OkHttpClient createHttpClient(HttpClientOptions httpClientOptions) { + BlockingQueue workQueue = new SynchronousQueue<>(); + ExecutorService executorService = + createCachedThreadPool(httpClientOptions.threadPoolSize(), THREAD_NAME, workQueue); + + OkHttpClient.Builder builder = + new OkHttpClient.Builder() + .dispatcher(new Dispatcher(executorService)) + .retryOnConnectionFailure(true) + .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); + httpClientOptions.connectTimeout().ifPresent(builder::connectTimeout); + httpClientOptions.readTimeout().ifPresent(builder::readTimeout); + + return builder.build(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java new file mode 100644 index 0000000000000..694779cfdb86f --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java @@ -0,0 +1,74 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import java.time.Duration; +import java.util.Optional; + +/** Options for Http Client. */ +public class HttpClientOptions { + + private final String uri; + private final Optional connectTimeout; + private final Optional readTimeout; + private final ObjectMapper mapper; + private final int threadPoolSize; + private final ErrorHandler errorHandler; + + public HttpClientOptions( + String uri, + Optional connectTimeout, + Optional readTimeout, + ObjectMapper mapper, + int threadPoolSize, + ErrorHandler errorHandler) { + this.uri = uri; + this.connectTimeout = connectTimeout; + this.readTimeout = readTimeout; + this.mapper = mapper; + this.threadPoolSize = threadPoolSize; + this.errorHandler = errorHandler; + } + + public String uri() { + return uri; + } + + public Optional connectTimeout() { + return connectTimeout; + } + + public Optional readTimeout() { + return readTimeout; + } + + public ObjectMapper mapper() { + return mapper; + } + + public int threadPoolSize() { + return threadPoolSize; + } + + public ErrorHandler errorHandler() { + return errorHandler; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java new file mode 100644 index 0000000000000..c96400831370c --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -0,0 +1,197 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Database; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; +import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.table.Table; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** A catalog implementation for REST. */ +public class RESTCatalog implements Catalog { + private RESTClient client; + private String token; + private ResourcePaths resourcePaths; + private Map options; + private Map baseHeader; + + private static final ObjectMapper objectMapper = RESTObjectMapper.create(); + static final String AUTH_HEADER = "Authorization"; + static final String AUTH_HEADER_VALUE_FORMAT = "Bearer %s"; + + public RESTCatalog(Options options) { + if (options.getOptional(CatalogOptions.WAREHOUSE).isPresent()) { + throw new IllegalArgumentException("Can not config warehouse in RESTCatalog."); + } + String uri = options.get(RESTCatalogOptions.URI); + token = options.get(RESTCatalogOptions.TOKEN); + Optional connectTimeout = + options.getOptional(RESTCatalogOptions.CONNECTION_TIMEOUT); + Optional readTimeout = options.getOptional(RESTCatalogOptions.READ_TIMEOUT); + Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); + HttpClientOptions httpClientOptions = + new HttpClientOptions( + uri, + connectTimeout, + readTimeout, + objectMapper, + threadPoolSize, + DefaultErrorHandler.getInstance()); + this.client = new HttpClient(httpClientOptions); + Map authHeaders = + ImmutableMap.of(AUTH_HEADER, String.format(AUTH_HEADER_VALUE_FORMAT, token)); + Map initHeaders = + RESTUtil.merge(configHeaders(options.toMap()), authHeaders); + this.options = fetchOptionsFromServer(initHeaders, options.toMap()); + this.baseHeader = configHeaders(this.options()); + this.resourcePaths = + ResourcePaths.forCatalogProperties( + this.options.get(RESTCatalogInternalOptions.PREFIX)); + } + + @Override + public String warehouse() { + throw new UnsupportedOperationException(); + } + + @Override + public Map options() { + return this.options; + } + + @Override + public FileIO fileIO() { + throw new UnsupportedOperationException(); + } + + @Override + public List listDatabases() { + throw new UnsupportedOperationException(); + } + + @Override + public void createDatabase(String name, boolean ignoreIfExists, Map properties) + throws DatabaseAlreadyExistException { + throw new UnsupportedOperationException(); + } + + @Override + public Database getDatabase(String name) throws DatabaseNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException { + throw new UnsupportedOperationException(); + } + + @Override + public Table getTable(Identifier identifier) throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public Path getTableLocation(Identifier identifier) { + throw new UnsupportedOperationException(); + } + + @Override + public List listTables(String databaseName) throws DatabaseNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropTable(Identifier identifier, boolean ignoreIfNotExists) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void createTable(Identifier identifier, Schema schema, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void renameTable(Identifier fromTable, Identifier toTable, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTable( + Identifier identifier, List changes, boolean ignoreIfNotExists) + throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartition(Identifier identifier, Map partitionSpec) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition(Identifier identifier, Map partitions) + throws TableNotExistException, PartitionNotExistException {} + + @Override + public List listPartitions(Identifier identifier) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean allowUpperCase() { + return false; + } + + @Override + public void close() throws Exception {} + + @VisibleForTesting + Map fetchOptionsFromServer( + Map headers, Map clientProperties) { + ConfigResponse response = + client.get(ResourcePaths.V1_CONFIG, ConfigResponse.class, headers); + return response.merge(clientProperties); + } + + private static Map configHeaders(Map properties) { + return RESTUtil.extractPrefixMap(properties, "header."); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java new file mode 100644 index 0000000000000..a5c773cb4bd5e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -0,0 +1,38 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; + +/** Factory to create {@link RESTCatalog}. */ +public class RESTCatalogFactory implements CatalogFactory { + public static final String IDENTIFIER = "rest"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public Catalog create(CatalogContext context) { + return new RESTCatalog(context.options()); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java new file mode 100644 index 0000000000000..cf61caa20e887 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java @@ -0,0 +1,31 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.options.ConfigOptions; + +/** Internal options for REST Catalog. */ +public class RESTCatalogInternalOptions { + public static final ConfigOption PREFIX = + ConfigOptions.key("prefix") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog uri's prefix."); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java new file mode 100644 index 0000000000000..6155b893751b1 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java @@ -0,0 +1,53 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.options.ConfigOptions; + +import java.time.Duration; + +/** Options for REST Catalog. */ +public class RESTCatalogOptions { + public static final ConfigOption URI = + ConfigOptions.key("uri") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog server's uri."); + public static final ConfigOption TOKEN = + ConfigOptions.key("token") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog server's auth token."); + public static final ConfigOption CONNECTION_TIMEOUT = + ConfigOptions.key("rest.client.connection-timeout") + .durationType() + .noDefaultValue() + .withDescription("REST Catalog http client connect timeout."); + public static final ConfigOption READ_TIMEOUT = + ConfigOptions.key("rest.client.read-timeout") + .durationType() + .noDefaultValue() + .withDescription("REST Catalog http client read timeout."); + public static final ConfigOption THREAD_POOL_SIZE = + ConfigOptions.key("rest.client.num-threads") + .intType() + .defaultValue(1) + .withDescription("REST Catalog http client thread num."); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java new file mode 100644 index 0000000000000..feeed06a417ac --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java @@ -0,0 +1,31 @@ +/* + * 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.paimon.rest; + +import java.io.Closeable; +import java.util.Map; + +/** Interface for a basic HTTP Client for interfacing with the REST catalog. */ +public interface RESTClient extends Closeable { + + T get(String path, Class responseType, Map headers); + + T post( + String path, RESTRequest body, Class responseType, Map headers); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java new file mode 100644 index 0000000000000..6cb0b6fa6573b --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java @@ -0,0 +1,22 @@ +/* + * 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.paimon.rest; + +/** Interface to mark both REST requests and responses. */ +public interface RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java new file mode 100644 index 0000000000000..b1c83e90224ad --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java @@ -0,0 +1,35 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.SerializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; + +/** Object mapper for REST request and response. */ +public class RESTObjectMapper { + public static ObjectMapper create() { + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false); + mapper.registerModule(new JavaTimeModule()); + return mapper; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java new file mode 100644 index 0000000000000..9c6758df14f04 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java @@ -0,0 +1,22 @@ +/* + * 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.paimon.rest; + +/** Interface to mark a REST request. */ +public interface RESTRequest extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java new file mode 100644 index 0000000000000..a4149d3fda145 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java @@ -0,0 +1,22 @@ +/* + * 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.paimon.rest; + +/** Interface to mark a REST response. */ +public interface RESTResponse extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java new file mode 100644 index 0000000000000..3d42e99fa6d59 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java @@ -0,0 +1,55 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.utils.Preconditions; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; + +import java.util.Map; + +/** Util for REST. */ +public class RESTUtil { + public static Map extractPrefixMap( + Map properties, String prefix) { + Preconditions.checkNotNull(properties, "Invalid properties map: null"); + Map result = Maps.newHashMap(); + for (Map.Entry entry : properties.entrySet()) { + if (entry.getKey() != null && entry.getKey().startsWith(prefix)) { + result.put( + entry.getKey().substring(prefix.length()), properties.get(entry.getKey())); + } + } + return result; + } + + public static Map merge( + Map target, Map updates) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Map.Entry entry : target.entrySet()) { + if (!updates.containsKey(entry.getKey())) { + builder.put(entry.getKey(), entry.getValue()); + } + } + updates.forEach(builder::put); + + return builder.build(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java new file mode 100644 index 0000000000000..1fad87588a33e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java @@ -0,0 +1,34 @@ +/* + * 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.paimon.rest; + +/** Resource paths for REST catalog. */ +public class ResourcePaths { + public static final String V1_CONFIG = "/api/v1/config"; + + public static ResourcePaths forCatalogProperties(String prefix) { + return new ResourcePaths(prefix); + } + + private final String prefix; + + public ResourcePaths(String prefix) { + this.prefix = prefix; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java new file mode 100644 index 0000000000000..301f3bd63f88d --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java @@ -0,0 +1,27 @@ +/* + * 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.paimon.rest.exceptions; + +/** Exception thrown on HTTP 400 - Bad Request. */ +public class BadRequestException extends RESTException { + + public BadRequestException(String message, Object... args) { + super(message, args); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java new file mode 100644 index 0000000000000..3982e5b704177 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java @@ -0,0 +1,26 @@ +/* + * 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.paimon.rest.exceptions; + +/** Exception thrown on HTTP 403 Forbidden. */ +public class ForbiddenException extends RESTException { + public ForbiddenException(String message, Object... args) { + super(message, args); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java new file mode 100644 index 0000000000000..43c13b1a1c979 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java @@ -0,0 +1,26 @@ +/* + * 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.paimon.rest.exceptions; + +/** Exception thrown on HTTP 401 Unauthorized. */ +public class NotAuthorizedException extends RESTException { + public NotAuthorizedException(String message, Object... args) { + super(String.format(message, args)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java new file mode 100644 index 0000000000000..532936f43032d --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java @@ -0,0 +1,30 @@ +/* + * 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.paimon.rest.exceptions; + +/** Base class for REST client exceptions. */ +public class RESTException extends RuntimeException { + public RESTException(String message, Object... args) { + super(String.format(message, args)); + } + + public RESTException(Throwable cause, String message, Object... args) { + super(String.format(message, args), cause); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java new file mode 100644 index 0000000000000..45c48ec0de094 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java @@ -0,0 +1,26 @@ +/* + * 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.paimon.rest.exceptions; + +/** Exception thrown on HTTP 500 - Bad Request. */ +public class ServiceFailureException extends RESTException { + public ServiceFailureException(String message, Object... args) { + super(String.format(message, args)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java new file mode 100644 index 0000000000000..fb6a05e89f9f3 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java @@ -0,0 +1,26 @@ +/* + * 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.paimon.rest.exceptions; + +/** Exception thrown on HTTP 503 - service is unavailable. */ +public class ServiceUnavailableException extends RESTException { + public ServiceUnavailableException(String message, Object... args) { + super(String.format(message, args)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java new file mode 100644 index 0000000000000..e6bc934703642 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -0,0 +1,76 @@ +/* + * 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.paimon.rest.responses; + +import org.apache.paimon.rest.RESTResponse; +import org.apache.paimon.utils.Preconditions; + +import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; + +import java.beans.ConstructorProperties; +import java.util.Map; +import java.util.Objects; + +/** Response for getting config. */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class ConfigResponse implements RESTResponse { + private static final String FIELD_DEFAULTS = "defaults"; + private static final String FIELD_OVERRIDES = "overrides"; + + @JsonProperty(FIELD_DEFAULTS) + private Map defaults; + + @JsonProperty(FIELD_OVERRIDES) + private Map overrides; + + @ConstructorProperties({FIELD_DEFAULTS, FIELD_OVERRIDES}) + public ConfigResponse(Map defaults, Map overrides) { + this.defaults = defaults; + this.overrides = overrides; + } + + public Map merge(Map clientProperties) { + Preconditions.checkNotNull( + clientProperties, + "Cannot merge client properties with server-provided properties. Invalid client configuration: null"); + Map merged = + defaults != null ? Maps.newHashMap(defaults) : Maps.newHashMap(); + merged.putAll(clientProperties); + + if (overrides != null) { + merged.putAll(overrides); + } + + return ImmutableMap.copyOf(Maps.filterValues(merged, Objects::nonNull)); + } + + @JsonGetter(FIELD_DEFAULTS) + public Map defaults() { + return defaults; + } + + @JsonGetter(FIELD_OVERRIDES) + public Map overrides() { + return overrides; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java new file mode 100644 index 0000000000000..0e4b234867329 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -0,0 +1,91 @@ +/* + * 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.paimon.rest.responses; + +import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; + +import java.beans.ConstructorProperties; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** Response for error. */ +public class ErrorResponse { + private static final String FIELD_MESSAGE = "message"; + private static final String FIELD_CODE = "code"; + private static final String FIELD_STACK = "stack"; + + @JsonProperty(FIELD_MESSAGE) + private final String message; + + @JsonProperty(FIELD_CODE) + private final Integer code; + + @JsonProperty(FIELD_STACK) + private final List stack; + + public ErrorResponse(String message, Integer code) { + this.code = code; + this.message = message; + this.stack = new ArrayList(); + } + + @ConstructorProperties({FIELD_MESSAGE, FIELD_CODE, FIELD_STACK}) + public ErrorResponse(String message, int code, List stack) { + this.message = message; + this.code = code; + this.stack = stack; + } + + public ErrorResponse(String message, int code, Throwable throwable) { + this.message = message; + this.code = code; + this.stack = getStackFromThrowable(throwable); + } + + @JsonGetter(FIELD_MESSAGE) + public String message() { + return message; + } + + @JsonGetter(FIELD_CODE) + public Integer code() { + return code; + } + + @JsonGetter(FIELD_STACK) + public List stack() { + return stack; + } + + private List getStackFromThrowable(Throwable throwable) { + if (throwable == null) { + return new ArrayList(); + } + StringWriter sw = new StringWriter(); + try (PrintWriter pw = new PrintWriter(sw)) { + throwable.printStackTrace(pw); + } + + return Arrays.asList(sw.toString().split("\n")); + } +} diff --git a/paimon-core/src/main/resources/META-INF/NOTICE b/paimon-core/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..dd2479b1d6e7c --- /dev/null +++ b/paimon-core/src/main/resources/META-INF/NOTICE @@ -0,0 +1,8 @@ +paimon-core +Copyright 2023-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) +- com.squareup.okhttp3:okhttp:4.12.0 \ No newline at end of file diff --git a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index ac6cc98fed6b7..3b98eef52c851 100644 --- a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -36,3 +36,4 @@ org.apache.paimon.mergetree.compact.aggregate.factory.FieldRoaringBitmap32AggFac org.apache.paimon.mergetree.compact.aggregate.factory.FieldRoaringBitmap64AggFactory org.apache.paimon.mergetree.compact.aggregate.factory.FieldSumAggFactory org.apache.paimon.mergetree.compact.aggregate.factory.FieldThetaSketchAggFactory +org.apache.paimon.rest.RESTCatalogFactory diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java new file mode 100644 index 0000000000000..1f1b9c01aace0 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java @@ -0,0 +1,77 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.rest.exceptions.BadRequestException; +import org.apache.paimon.rest.exceptions.ForbiddenException; +import org.apache.paimon.rest.exceptions.NotAuthorizedException; +import org.apache.paimon.rest.exceptions.RESTException; +import org.apache.paimon.rest.exceptions.ServiceFailureException; +import org.apache.paimon.rest.exceptions.ServiceUnavailableException; +import org.apache.paimon.rest.responses.ErrorResponse; + +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; + +import static org.junit.Assert.assertThrows; + +/** Test for {@link DefaultErrorHandler}. */ +public class DefaultErrorHandlerTest { + private ErrorHandler defaultErrorHandler; + + @Before + public void setUp() throws IOException { + defaultErrorHandler = DefaultErrorHandler.getInstance(); + } + + @Test + public void testHandleErrorResponse() { + assertThrows( + BadRequestException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(400))); + assertThrows( + NotAuthorizedException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(401))); + assertThrows( + ForbiddenException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(403))); + assertThrows( + RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(405))); + assertThrows( + RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(406))); + assertThrows( + ServiceFailureException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(500))); + assertThrows( + UnsupportedOperationException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(501))); + assertThrows( + RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(502))); + assertThrows( + ServiceUnavailableException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(503))); + } + + private ErrorResponse generateErrorResponse(int code) { + return new ErrorResponse("message", code, new ArrayList()); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java new file mode 100644 index 0000000000000..1140e399824cc --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import okhttp3.mockwebserver.MockResponse; +import okhttp3.mockwebserver.MockWebServer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.apache.paimon.rest.RESTCatalog.AUTH_HEADER; +import static org.apache.paimon.rest.RESTCatalog.AUTH_HEADER_VALUE_FORMAT; +import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** Test for {@link HttpClient}. */ +public class HttpClientTest { + private MockWebServer mockWebServer; + private HttpClient httpClient; + private ObjectMapper objectMapper = RESTObjectMapper.create(); + private ErrorHandler errorHandler; + private MockRESTData mockResponseData; + private String mockResponseDataStr; + private Map headers; + private static final String MOCK_PATH = "/v1/api/mock"; + private static final String TOKEN = "token"; + + @Before + public void setUp() throws IOException { + mockWebServer = new MockWebServer(); + mockWebServer.start(); + String baseUrl = mockWebServer.url("").toString(); + errorHandler = mock(ErrorHandler.class); + HttpClientOptions httpClientOptions = + new HttpClientOptions( + baseUrl, + Optional.of(Duration.ofSeconds(3)), + Optional.of(Duration.ofSeconds(3)), + objectMapper, + 1, + errorHandler); + mockResponseData = new MockRESTData(MOCK_PATH); + mockResponseDataStr = objectMapper.writeValueAsString(mockResponseData); + httpClient = new HttpClient(httpClientOptions); + headers = ImmutableMap.of(AUTH_HEADER, String.format(AUTH_HEADER_VALUE_FORMAT, TOKEN)); + } + + @After + public void tearDown() throws IOException { + mockWebServer.shutdown(); + } + + @Test + public void testGetSuccess() { + mockHttpCallWithCode(mockResponseDataStr, 200); + MockRESTData response = httpClient.get(MOCK_PATH, MockRESTData.class, headers); + verify(errorHandler, times(0)).accept(any()); + assertEquals(mockResponseData.data(), response.data()); + } + + @Test + public void testGetFail() { + mockHttpCallWithCode(mockResponseDataStr, 400); + httpClient.get(MOCK_PATH, MockRESTData.class, headers); + verify(errorHandler, times(1)).accept(any()); + } + + @Test + public void testPostSuccess() { + mockHttpCallWithCode(mockResponseDataStr, 200); + MockRESTData response = + httpClient.post(MOCK_PATH, mockResponseData, MockRESTData.class, headers); + verify(errorHandler, times(0)).accept(any()); + assertEquals(mockResponseData.data(), response.data()); + } + + @Test + public void testPostFail() { + mockHttpCallWithCode(mockResponseDataStr, 400); + httpClient.post(MOCK_PATH, mockResponseData, MockRESTData.class, headers); + verify(errorHandler, times(1)).accept(any()); + } + + private Map headers(String token) { + Map header = new HashMap<>(); + header.put("Authorization", "Bearer " + token); + return header; + } + + private void mockHttpCallWithCode(String body, Integer code) { + MockResponse mockResponseObj = generateMockResponse(body, code); + mockWebServer.enqueue(mockResponseObj); + } + + private MockResponse generateMockResponse(String data, Integer code) { + return new MockResponse() + .setResponseCode(code) + .setBody(data) + .addHeader("Content-Type", "application/json"); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java new file mode 100644 index 0000000000000..55c5165ada48e --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java @@ -0,0 +1,44 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; + +import java.beans.ConstructorProperties; + +/** Mock REST data. */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class MockRESTData implements RESTRequest, RESTResponse { + private static final String FIELD_DATA = "data"; + + @JsonProperty(FIELD_DATA) + private String data; + + @ConstructorProperties({FIELD_DATA}) + public MockRESTData(String data) { + this.data = data; + } + + @JsonGetter(FIELD_DATA) + public String data() { + return data; + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java new file mode 100644 index 0000000000000..3ed8730862ee5 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java @@ -0,0 +1,86 @@ +/* + * 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.paimon.rest; + +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; + +import okhttp3.mockwebserver.MockResponse; +import okhttp3.mockwebserver.MockWebServer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +/** Test for REST Catalog. */ +public class RESTCatalogTest { + private MockWebServer mockWebServer; + private RESTCatalog restCatalog; + private final String initToken = "init_token"; + + @Before + public void setUp() throws IOException { + mockWebServer = new MockWebServer(); + mockWebServer.start(); + String baseUrl = mockWebServer.url("").toString(); + Options options = new Options(); + options.set(RESTCatalogOptions.URI, baseUrl); + options.set(RESTCatalogOptions.TOKEN, initToken); + options.set(RESTCatalogOptions.THREAD_POOL_SIZE, 1); + mockOptions(RESTCatalogInternalOptions.PREFIX.key(), "prefix"); + restCatalog = new RESTCatalog(options); + } + + @After + public void tearDown() throws IOException { + mockWebServer.shutdown(); + } + + @Test + public void testInitFailWhenDefineWarehouse() { + Options options = new Options(); + options.set(CatalogOptions.WAREHOUSE, "/a/b/c"); + assertThrows(IllegalArgumentException.class, () -> new RESTCatalog(options)); + } + + @Test + public void testGetConfig() { + String key = "a"; + String value = "b"; + mockOptions(key, value); + Map header = new HashMap<>(); + Map response = restCatalog.fetchOptionsFromServer(header, new HashMap<>()); + assertEquals(value, response.get(key)); + } + + private void mockOptions(String key, String value) { + String mockResponse = String.format("{\"defaults\": {\"%s\": \"%s\"}}", key, value); + MockResponse mockResponseObj = + new MockResponse() + .setBody(mockResponse) + .addHeader("Content-Type", "application/json"); + mockWebServer.enqueue(mockResponseObj); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java new file mode 100644 index 0000000000000..83a8805d29a0f --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.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.paimon.rest; + +import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.ErrorResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** Test for {@link RESTObjectMapper}. */ +public class RESTObjectMapperTest { + private ObjectMapper mapper = RESTObjectMapper.create(); + + @Test + public void configResponseParseTest() throws Exception { + String confKey = "a"; + Map conf = new HashMap<>(); + conf.put(confKey, "b"); + ConfigResponse response = new ConfigResponse(conf, conf); + String responseStr = mapper.writeValueAsString(response); + ConfigResponse parseData = mapper.readValue(responseStr, ConfigResponse.class); + assertEquals(conf.get(confKey), parseData.defaults().get(confKey)); + } + + @Test + public void errorResponseParseTest() throws Exception { + String message = "message"; + Integer code = 400; + ErrorResponse response = new ErrorResponse(message, code, new ArrayList()); + String responseStr = mapper.writeValueAsString(response); + ErrorResponse parseData = mapper.readValue(responseStr, ErrorResponse.class); + assertEquals(message, parseData.message()); + assertEquals(code, parseData.code()); + } +} diff --git a/paimon-open-api/Makefile b/paimon-open-api/Makefile new file mode 100644 index 0000000000000..c3264c83dbd02 --- /dev/null +++ b/paimon-open-api/Makefile @@ -0,0 +1,25 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# See: https://cwiki.apache.org/confluence/display/INFRA/git+-+.asf.yaml+features + + +install: + brew install yq + +generate: + @sh generate.sh diff --git a/paimon-open-api/README.md b/paimon-open-api/README.md new file mode 100644 index 0000000000000..9d14a7cdd3643 --- /dev/null +++ b/paimon-open-api/README.md @@ -0,0 +1,10 @@ +# Open API spec + +The `rest-catalog-open-api.yaml` defines the REST catalog interface. + +## Generate Open API Spec +```sh +make install +cd paimon-open-api +make generate +``` \ No newline at end of file diff --git a/paimon-open-api/generate.sh b/paimon-open-api/generate.sh new file mode 100755 index 0000000000000..b63aa538abc45 --- /dev/null +++ b/paimon-open-api/generate.sh @@ -0,0 +1,48 @@ +#!/bin/bash +# +# 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. + +# Start the application +cd .. +mvn clean install -DskipTests +cd ./paimon-open-api +mvn spring-boot:run & +SPRING_PID=$! +# Wait for the application to be ready +RETRY_COUNT=0 +MAX_RETRIES=10 +SLEEP_DURATION=5 + +until $(curl -s -o /dev/null -w "%{http_code}" http://localhost:8080/swagger-api-docs | grep -q "200"); do + ((RETRY_COUNT++)) + if [ $RETRY_COUNT -gt $MAX_RETRIES ]; then + echo "Failed to start the application after $MAX_RETRIES retries." + exit 1 + fi + echo "Application not ready yet. Retrying in $SLEEP_DURATION seconds..." + sleep $SLEEP_DURATION +done + +echo "Application is ready". + +# Generate the OpenAPI specification file +curl -s "http://localhost:8080/swagger-api-docs" | jq -M > ./rest-catalog-open-api.json +yq --prettyPrint -o=yaml ./rest-catalog-open-api.json > ./rest-catalog-open-api.yaml +rm -rf ./rest-catalog-open-api.json +mvn spotless:apply +# Stop the application +echo "Stopping application..." +kill $SPRING_PID \ No newline at end of file diff --git a/paimon-open-api/pom.xml b/paimon-open-api/pom.xml new file mode 100644 index 0000000000000..b5cee29fe4e7f --- /dev/null +++ b/paimon-open-api/pom.xml @@ -0,0 +1,85 @@ + + + + 4.0.0 + + org.apache.paimon + paimon-parent + 1.0-SNAPSHOT + + + paimon-open-api + + + 8 + 8 + UTF-8 + + + + org.springframework.boot + spring-boot-starter-web + 2.7.18 + + + ch.qos.logback + logback-classic + + + + + + + org.springdoc + springdoc-openapi-ui + 1.7.0 + + + org.apache.paimon + paimon-core + ${project.version} + + + io.swagger.core.v3 + swagger-annotations + 2.2.20 + + + + + + org.springframework.boot + spring-boot-maven-plugin + 2.7.6 + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + + \ No newline at end of file diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml new file mode 100644 index 0000000000000..432ee123b8d43 --- /dev/null +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -0,0 +1,60 @@ +# +# 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. +# + +--- +openapi: 3.0.1 +info: + title: RESTCatalog API + description: This API exposes endpoints to RESTCatalog. + license: + name: Apache 2.0 + url: https://www.apache.org/licenses/LICENSE-2.0.html + version: "1.0" +servers: + - url: http://localhost:8080 + description: Server URL in Development environment +paths: + /api/v1/config: + get: + tags: + - config + summary: Get Config + operationId: getConfig + responses: + "500": + description: Internal Server Error + "201": + description: Created + content: + application/json: + schema: + $ref: '#/components/schemas/ConfigResponse' +components: + schemas: + ConfigResponse: + type: object + properties: + defaults: + type: object + additionalProperties: + type: string + writeOnly: true + overrides: + type: object + additionalProperties: + type: string + writeOnly: true diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java new file mode 100644 index 0000000000000..76ce4cbf83c61 --- /dev/null +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java @@ -0,0 +1,31 @@ +/* + * 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.paimon.open.api; + +import org.springframework.boot.SpringApplication; +import org.springframework.boot.autoconfigure.SpringBootApplication; + +/** OpenAPI application. */ +@SpringBootApplication +public class OpenApiApplication { + + public static void main(String[] args) { + SpringApplication.run(OpenApiApplication.class, args); + } +} diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java new file mode 100644 index 0000000000000..b475540571054 --- /dev/null +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -0,0 +1,69 @@ +/* + * 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.paimon.open.api; + +import org.apache.paimon.rest.ResourcePaths; +import org.apache.paimon.rest.responses.ConfigResponse; + +import io.swagger.v3.oas.annotations.Operation; +import io.swagger.v3.oas.annotations.media.Content; +import io.swagger.v3.oas.annotations.media.Schema; +import io.swagger.v3.oas.annotations.responses.ApiResponse; +import io.swagger.v3.oas.annotations.responses.ApiResponses; +import org.springframework.http.HttpStatus; +import org.springframework.http.ResponseEntity; +import org.springframework.web.bind.annotation.CrossOrigin; +import org.springframework.web.bind.annotation.GetMapping; +import org.springframework.web.bind.annotation.RestController; + +import java.util.HashMap; +import java.util.Map; + +/** * RESTCatalog management APIs. */ +@CrossOrigin(origins = "http://localhost:8081") +@RestController +public class RESTCatalogController { + + @Operation( + summary = "Get Config", + tags = {"config"}) + @ApiResponses({ + @ApiResponse( + responseCode = "201", + content = { + @Content( + schema = @Schema(implementation = ConfigResponse.class), + mediaType = "application/json") + }), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @GetMapping(ResourcePaths.V1_CONFIG) + public ResponseEntity getConfig() { + try { + Map defaults = new HashMap<>(); + Map overrides = new HashMap<>(); + ConfigResponse response = new ConfigResponse(defaults, overrides); + return new ResponseEntity<>(response, HttpStatus.CREATED); + } catch (Exception e) { + return new ResponseEntity<>(null, HttpStatus.INTERNAL_SERVER_ERROR); + } + } +} diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java new file mode 100644 index 0000000000000..01234c41bbff4 --- /dev/null +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java @@ -0,0 +1,60 @@ +/* + * 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.paimon.open.api.config; + +import io.swagger.v3.oas.models.OpenAPI; +import io.swagger.v3.oas.models.info.Info; +import io.swagger.v3.oas.models.info.License; +import io.swagger.v3.oas.models.servers.Server; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; + +import java.util.ArrayList; +import java.util.List; + +/** Config for OpenAPI. */ +@Configuration +public class OpenAPIConfig { + + @Value("${openapi.url}") + private String devUrl; + + @Bean + public OpenAPI restCatalogOpenAPI() { + Server server = new Server(); + server.setUrl(devUrl); + server.setDescription("Server URL in Development environment"); + + License mitLicense = + new License() + .name("Apache 2.0") + .url("https://www.apache.org/licenses/LICENSE-2.0.html"); + + Info info = + new Info() + .title("RESTCatalog API") + .version("1.0") + .description("This API exposes endpoints to RESTCatalog.") + .license(mitLicense); + List servers = new ArrayList<>(); + servers.add(server); + return new OpenAPI().info(info).servers(servers); + } +} diff --git a/paimon-open-api/src/main/resources/application.properties b/paimon-open-api/src/main/resources/application.properties new file mode 100644 index 0000000000000..58a975161145b --- /dev/null +++ b/paimon-open-api/src/main/resources/application.properties @@ -0,0 +1,22 @@ +# +# 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. +# +springdoc.swagger-ui.path=/swagger-api +springdoc.api-docs.path=/swagger-api-docs +springdoc.swagger-ui.deepLinking=true +springdoc.swagger-ui.tryItOutEnabled=true +springdoc.swagger-ui.filter=true +openapi.url=http://localhost:8080 diff --git a/pom.xml b/pom.xml index 85a880f35158e..904b1c73c7414 100644 --- a/pom.xml +++ b/pom.xml @@ -69,6 +69,7 @@ under the License. paimon-test-utils paimon-arrow tools/ci/paimon-ci-tools + paimon-open-api From 2816811fc69a8209f0674418765a11cd2adf46d5 Mon Sep 17 00:00:00 2001 From: Steven <54518670+yangjf2019@users.noreply.github.com> Date: Thu, 28 Nov 2024 19:59:53 +0800 Subject: [PATCH 078/157] =?UTF-8?q?[doc]=20fix=20the=20=E2=80=98primary-ke?= =?UTF-8?q?y-table=E2=80=99=20link=20(#4606)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/content/flink/action-jars.md | 2 +- docs/content/flink/sql-write.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/flink/action-jars.md b/docs/content/flink/action-jars.md index 34e911ff6feba..69f9dda1fdcbe 100644 --- a/docs/content/flink/action-jars.md +++ b/docs/content/flink/action-jars.md @@ -50,7 +50,7 @@ Paimon supports "MERGE INTO" via submitting the 'merge_into' job through `flink {{< hint info >}} Important table properties setting: -1. Only [primary key table]({{< ref "primary-key-table" >}}) supports this feature. +1. Only [primary key table]({{< ref "primary-key-table/overview" >}}) supports this feature. 2. The action won't produce UPDATE_BEFORE, so it's not recommended to set 'changelog-producer' = 'input'. {{< /hint >}} diff --git a/docs/content/flink/sql-write.md b/docs/content/flink/sql-write.md index 58e90a8f6e3f5..3b6a0498eb279 100644 --- a/docs/content/flink/sql-write.md +++ b/docs/content/flink/sql-write.md @@ -175,7 +175,7 @@ PARTITION (k0 = 0, k1 = 0) SELECT v FROM my_table WHERE false; {{< hint info >}} Important table properties setting: -1. Only [primary key table]({{< ref "primary-key-table" >}}) supports this feature. +1. Only [primary key table]({{< ref "primary-key-table/overview" >}}) supports this feature. 2. [MergeEngine]({{< ref "primary-key-table/merge-engine" >}}) needs to be [deduplicate]({{< ref "primary-key-table/merge-engine#deduplicate" >}}) or [partial-update]({{< ref "primary-key-table/merge-engine#partial-update" >}}) to support this feature. 3. Do not support updating primary keys. From fdcdd09ca09d009b4051fe319cef6050ae58d110 Mon Sep 17 00:00:00 2001 From: WenjunMin Date: Thu, 28 Nov 2024 20:01:37 +0800 Subject: [PATCH 079/157] [core] Avoid extract from file footer when stats mode is none (#4604) --- .../io/StatsCollectingSingleFileWriter.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java index 2f4190a049dc4..67a3fa6d1acea 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java @@ -25,6 +25,7 @@ import org.apache.paimon.format.SimpleStatsExtractor; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.statistics.NoneSimpleColStatsCollector; import org.apache.paimon.statistics.SimpleColStatsCollector; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Preconditions; @@ -32,7 +33,9 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Arrays; import java.util.function.Function; +import java.util.stream.IntStream; /** * A {@link SingleFileWriter} which also produces statistics for each written field. @@ -44,6 +47,8 @@ public abstract class StatsCollectingSingleFileWriter extends SingleFileWr @Nullable private final SimpleStatsExtractor simpleStatsExtractor; @Nullable private SimpleStatsCollector simpleStatsCollector = null; + @Nullable private SimpleColStats[] noneStats = null; + private final boolean isStatsDisabled; public StatsCollectingSingleFileWriter( FileIO fileIO, @@ -63,6 +68,15 @@ public StatsCollectingSingleFileWriter( Preconditions.checkArgument( statsCollectors.length == writeSchema.getFieldCount(), "The stats collector is not aligned to write schema."); + this.isStatsDisabled = + Arrays.stream(SimpleColStatsCollector.create(statsCollectors)) + .allMatch(p -> p instanceof NoneSimpleColStatsCollector); + if (isStatsDisabled) { + this.noneStats = + IntStream.range(0, statsCollectors.length) + .mapToObj(i -> SimpleColStats.NONE) + .toArray(SimpleColStats[]::new); + } } @Override @@ -85,7 +99,11 @@ public void writeBundle(BundleRecords bundle) throws IOException { public SimpleColStats[] fieldStats() throws IOException { Preconditions.checkState(closed, "Cannot access metric unless the writer is closed."); if (simpleStatsExtractor != null) { - return simpleStatsExtractor.extract(fileIO, path); + if (isStatsDisabled) { + return noneStats; + } else { + return simpleStatsExtractor.extract(fileIO, path); + } } else { return simpleStatsCollector.extract(); } From 2f93b7b2dbc3a659f3b25bcff6c23aee4218ebfe Mon Sep 17 00:00:00 2001 From: Gang Yang Date: Thu, 28 Nov 2024 20:02:30 +0800 Subject: [PATCH 080/157] [cdc] Add the latest_schema state at schema evolution operator to reduce the latest schema access frequency (#4535) --- .../apache/paimon/types/FieldIdentifier.java | 53 +++++ .../cdc/UpdatedDataFieldsProcessFunction.java | 41 +++- .../flink/action/cdc/SchemaEvolutionTest.java | 219 ++++++++++++++++++ 3 files changed, 312 insertions(+), 1 deletion(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/types/FieldIdentifier.java create mode 100644 paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java diff --git a/paimon-common/src/main/java/org/apache/paimon/types/FieldIdentifier.java b/paimon-common/src/main/java/org/apache/paimon/types/FieldIdentifier.java new file mode 100644 index 0000000000000..7e9ced7cf95ac --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/types/FieldIdentifier.java @@ -0,0 +1,53 @@ +/* + * 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.paimon.types; + +import java.util.Objects; + +/** Used to indicate the uniqueness of a field. */ +public class FieldIdentifier { + private String name; + private DataType type; + private String description; + + public FieldIdentifier(DataField dataField) { + this.name = dataField.name(); + this.type = dataField.type(); + this.description = dataField.description(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FieldIdentifier field = (FieldIdentifier) o; + return Objects.equals(name, field.name) + && Objects.equals(type, field.type) + && Objects.equals(description, field.description); + } + + @Override + public int hashCode() { + return Objects.hash(name, type, description); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java index 4a33eb1b7ec94..64f00d96b0f51 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java @@ -23,11 +23,18 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.types.DataField; +import org.apache.paimon.types.FieldIdentifier; +import org.apache.paimon.types.RowType; +import org.apache.commons.collections.CollectionUtils; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.util.Collector; +import java.util.HashSet; import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; /** * A {@link ProcessFunction} to handle schema changes. New schema is represented by a list of {@link @@ -43,19 +50,51 @@ public class UpdatedDataFieldsProcessFunction private final Identifier identifier; + private Set latestFields; + public UpdatedDataFieldsProcessFunction( SchemaManager schemaManager, Identifier identifier, Catalog.Loader catalogLoader) { super(catalogLoader); this.schemaManager = schemaManager; this.identifier = identifier; + this.latestFields = new HashSet<>(); } @Override public void processElement( List updatedDataFields, Context context, Collector collector) throws Exception { - for (SchemaChange schemaChange : extractSchemaChanges(schemaManager, updatedDataFields)) { + List actualUpdatedDataFields = + updatedDataFields.stream() + .filter( + dataField -> + !latestDataFieldContain(new FieldIdentifier(dataField))) + .collect(Collectors.toList()); + if (CollectionUtils.isEmpty(actualUpdatedDataFields)) { + return; + } + for (SchemaChange schemaChange : + extractSchemaChanges(schemaManager, actualUpdatedDataFields)) { applySchemaChange(schemaManager, schemaChange, identifier); } + /** + * Here, actualUpdatedDataFields cannot be used to update latestFields because there is a + * non-SchemaChange.AddColumn scenario. Otherwise, the previously existing fields cannot be + * modified again. + */ + updateLatestFields(); + } + + private boolean latestDataFieldContain(FieldIdentifier dataField) { + return latestFields.stream().anyMatch(previous -> Objects.equals(previous, dataField)); + } + + private void updateLatestFields() { + RowType oldRowType = schemaManager.latest().get().logicalRowType(); + Set fieldIdentifiers = + oldRowType.getFields().stream() + .map(item -> new FieldIdentifier(item)) + .collect(Collectors.toSet()); + latestFields = fieldIdentifiers; } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java new file mode 100644 index 0000000000000..9ba18376867ff --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java @@ -0,0 +1,219 @@ +/* + * 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.paimon.flink.action.cdc; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.flink.FlinkCatalogFactory; +import org.apache.paimon.flink.sink.cdc.UpdatedDataFieldsProcessFunction; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.options.Options; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.SchemaUtils; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.TableTestBase; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.DecimalType; +import org.apache.paimon.types.DoubleType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.VarCharType; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +/** Used to test schema evolution related logic. */ +public class SchemaEvolutionTest extends TableTestBase { + + private static List> prepareData() { + List upField1 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new IntType(), "test description."), + new DataField(2, "col_2", new IntType(), "test description."), + new DataField(3, "col_3", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + List upField2 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new BigIntType(), "test description."), + new DataField(2, "col_2", new IntType(), "test description."), + new DataField(3, "col_3", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + List upField3 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new BigIntType(), "test description."), + new DataField(2, "col_2", new IntType(), "test description 2."), + new DataField(3, "col_3", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + List upField4 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new BigIntType(), "test description."), + new DataField(2, "col_2", new IntType(), "test description."), + new DataField(3, "col_3_1", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + List upField5 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new BigIntType(), "test description."), + new DataField(2, "col_2_1", new BigIntType(), "test description 2."), + new DataField(3, "col_3", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + return Arrays.asList(upField1, upField2, upField3, upField4, upField5); + } + + private FileStoreTable table; + private String tableName = "MyTable"; + + @BeforeEach + public void before() throws Exception { + FileIO fileIO = LocalFileIO.create(); + Path tablePath = new Path(String.format("%s/%s.db/%s", warehouse, database, tableName)); + Schema schema = + Schema.newBuilder() + .column("pk", DataTypes.INT()) + .column("pt1", DataTypes.INT()) + .column("pt2", DataTypes.INT()) + .column("col1", DataTypes.INT()) + .partitionKeys("pt1", "pt2") + .primaryKey("pk", "pt1", "pt2") + .option(CoreOptions.CHANGELOG_PRODUCER.key(), "input") + .option(CoreOptions.BUCKET.key(), "2") + .option(CoreOptions.SEQUENCE_FIELD.key(), "col1") + .build(); + TableSchema tableSchema = + SchemaUtils.forceCommit(new SchemaManager(fileIO, tablePath), schema); + table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath, tableSchema); + } + + @Test + public void testSchemaEvolution() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStream> upDataFieldStream = env.fromCollection(prepareData()); + Options options = new Options(); + options.set("warehouse", tempPath.toString()); + final Catalog.Loader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(options); + Identifier identifier = Identifier.create(database, tableName); + DataStream schemaChangeProcessFunction = + upDataFieldStream + .process( + new UpdatedDataFieldsProcessFunction( + new SchemaManager(table.fileIO(), table.location()), + identifier, + catalogLoader)) + .name("Schema Evolution"); + schemaChangeProcessFunction.getTransformation().setParallelism(1); + schemaChangeProcessFunction.getTransformation().setMaxParallelism(1); + env.execute(); + } +} From 9a89def5694b95ebed3dc491b52e10079b601959 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Thu, 28 Nov 2024 20:07:57 +0800 Subject: [PATCH 081/157] [pom] Add name to paimon-open-api --- paimon-open-api/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/paimon-open-api/pom.xml b/paimon-open-api/pom.xml index b5cee29fe4e7f..9422852432705 100644 --- a/paimon-open-api/pom.xml +++ b/paimon-open-api/pom.xml @@ -28,6 +28,7 @@ under the License. paimon-open-api + Paimon : Open API 8 From 475e48791b873c516c4c26774cda3b45a268cd70 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Fri, 29 Nov 2024 20:38:47 +0800 Subject: [PATCH 082/157] [core] Remove all lineage implementation (#4607) --- .../generated/catalog_configuration.html | 6 - .../org/apache/paimon/factories/Factory.java | 2 +- .../paimon/lineage/DataLineageEntity.java | 33 --- .../apache/paimon/lineage/LineageMeta.java | 102 --------- .../paimon/lineage/LineageMetaFactory.java | 37 ---- .../paimon/lineage/TableLineageEntity.java | 32 --- .../lineage/TableLineageEntityImpl.java | 56 ----- .../apache/paimon/options/CatalogOptions.java | 22 -- .../paimon/catalog/AbstractCatalog.java | 26 +-- .../paimon/table/CatalogEnvironment.java | 18 +- .../table/system/SinkTableLineageTable.java | 64 ------ .../table/system/SourceTableLineageTable.java | 64 ------ .../table/system/SystemTableLoader.java | 29 +-- .../table/system/TableLineageTable.java | 168 -------------- .../flink/AbstractFlinkTableFactory.java | 58 +---- .../paimon/flink/CatalogTableITCase.java | 6 +- .../paimon/flink/FlinkLineageITCase.java | 206 ------------------ .../org.apache.paimon.factories.Factory | 3 - .../org/apache/paimon/hive/HiveCatalog.java | 3 +- 19 files changed, 12 insertions(+), 923 deletions(-) delete mode 100644 paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java delete mode 100644 paimon-common/src/main/java/org/apache/paimon/lineage/LineageMeta.java delete mode 100644 paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java delete mode 100644 paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java delete mode 100644 paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java delete mode 100644 paimon-core/src/main/java/org/apache/paimon/table/system/TableLineageTable.java delete mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkLineageITCase.java diff --git a/docs/layouts/shortcodes/generated/catalog_configuration.html b/docs/layouts/shortcodes/generated/catalog_configuration.html index 6706d5c421a1b..63f7adda1e0df 100644 --- a/docs/layouts/shortcodes/generated/catalog_configuration.html +++ b/docs/layouts/shortcodes/generated/catalog_configuration.html @@ -86,12 +86,6 @@ Boolean Whether to support format tables, format table corresponds to a regular csv, parquet or orc table, allowing read and write operations. However, during these processes, it does not connect to the metastore; hence, newly added partitions will not be reflected in the metastore and need to be manually added as separate partition operations. - -
    lineage-meta
    - (none) - String - The lineage meta to store table and data lineage information.

    Possible values:
    • "jdbc": Use standard jdbc to store table and data lineage information.
    • "custom": You can implement LineageMetaFactory and LineageMeta to store lineage information in customized storage.
    -
    lock-acquire-timeout
    8 min diff --git a/paimon-common/src/main/java/org/apache/paimon/factories/Factory.java b/paimon-common/src/main/java/org/apache/paimon/factories/Factory.java index b0f1ec84c1706..74796879ef4bf 100644 --- a/paimon-common/src/main/java/org/apache/paimon/factories/Factory.java +++ b/paimon-common/src/main/java/org/apache/paimon/factories/Factory.java @@ -20,7 +20,7 @@ /** * Base interface for all kind of factories that create object instances from a list of key-value - * pairs in Paimon's catalog, lineage. + * pairs in Paimon's catalog. * *

    A factory is uniquely identified by {@link Class} and {@link #identifier()}. * diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java b/paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java deleted file mode 100644 index e7401a9be3b76..0000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java +++ /dev/null @@ -1,33 +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.paimon.lineage; - -import org.apache.paimon.data.Timestamp; - -/** - * Data lineage entity with table lineage, barrier id and snapshot id for table source and sink - * lineage. - */ -public interface DataLineageEntity extends TableLineageEntity { - long getBarrierId(); - - long getSnapshotId(); - - Timestamp getCreateTime(); -} diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMeta.java b/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMeta.java deleted file mode 100644 index 5d1c42daf6c80..0000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMeta.java +++ /dev/null @@ -1,102 +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.paimon.lineage; - -import org.apache.paimon.predicate.Predicate; - -import javax.annotation.Nullable; - -import java.util.Iterator; - -/** Metadata store will manage table lineage and data lineage information for the catalog. */ -public interface LineageMeta extends AutoCloseable { - /** - * Save the source table and job lineage. - * - * @param entity the table lineage entity - */ - void saveSourceTableLineage(TableLineageEntity entity); - - /** - * Delete the source table lineage for given job. - * - * @param job the job for table lineage - */ - void deleteSourceTableLineage(String job); - - /** - * Get source table and job lineages. - * - * @param predicate the predicate for the table lineages - * @return the iterator for source table and job lineages - */ - Iterator sourceTableLineages(@Nullable Predicate predicate); - - /** - * Save the sink table and job lineage. - * - * @param entity the table lineage entity - */ - void saveSinkTableLineage(TableLineageEntity entity); - - /** - * Get sink table and job lineages. - * - * @param predicate the predicate for the table lineages - * @return the iterator for sink table and job lineages - */ - Iterator sinkTableLineages(@Nullable Predicate predicate); - - /** - * Delete the sink table lineage for given job. - * - * @param job the job for table lineage - */ - void deleteSinkTableLineage(String job); - - /** - * Save the source table and job lineage. - * - * @param entity the data lineage entity - */ - void saveSourceDataLineage(DataLineageEntity entity); - - /** - * Get source data and job lineages. - * - * @param predicate the predicate for the table lineages - * @return the iterator for source table and job lineages - */ - Iterator sourceDataLineages(@Nullable Predicate predicate); - - /** - * Save the sink table and job lineage. - * - * @param entity the data lineage entity - */ - void saveSinkDataLineage(DataLineageEntity entity); - - /** - * Get sink data and job lineages. - * - * @param predicate the predicate for the table lineages - * @return the iterator for sink table and job lineages - */ - Iterator sinkDataLineages(@Nullable Predicate predicate); -} diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java b/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java deleted file mode 100644 index 11c6d3a1173c7..0000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.lineage; - -import org.apache.paimon.factories.Factory; -import org.apache.paimon.options.Options; - -import java.io.Serializable; - -/** Factory to create {@link LineageMeta}. Each factory should have a unique identifier. */ -public interface LineageMetaFactory extends Factory, Serializable { - - LineageMeta create(LineageMetaContext context); - - /** - * Context has all options in a catalog and is used in factory to create {@link LineageMeta}. - */ - interface LineageMetaContext { - Options options(); - } -} diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java b/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java deleted file mode 100644 index c4312c4eb080f..0000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java +++ /dev/null @@ -1,32 +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.paimon.lineage; - -import org.apache.paimon.data.Timestamp; - -/** Table lineage entity with database, table and job for table source and sink lineage. */ -public interface TableLineageEntity { - String getDatabase(); - - String getTable(); - - String getJob(); - - Timestamp getCreateTime(); -} diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java b/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java deleted file mode 100644 index ef11ee87f15c9..0000000000000 --- a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java +++ /dev/null @@ -1,56 +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.paimon.lineage; - -import org.apache.paimon.data.Timestamp; - -/** Default implementation for {@link TableLineageEntity}. */ -public class TableLineageEntityImpl implements TableLineageEntity { - private final String database; - private final String table; - private final String job; - private final Timestamp timestamp; - - public TableLineageEntityImpl(String database, String table, String job, Timestamp timestamp) { - this.database = database; - this.table = table; - this.job = job; - this.timestamp = timestamp; - } - - @Override - public String getDatabase() { - return database; - } - - @Override - public String getTable() { - return table; - } - - @Override - public String getJob() { - return job; - } - - @Override - public Timestamp getCreateTime() { - return timestamp; - } -} diff --git a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java index f69af2d599103..bb8cfae682848 100644 --- a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java @@ -18,8 +18,6 @@ package org.apache.paimon.options; -import org.apache.paimon.options.description.Description; -import org.apache.paimon.options.description.TextElement; import org.apache.paimon.table.CatalogTableType; import java.time.Duration; @@ -130,26 +128,6 @@ public class CatalogOptions { .withDescription( "Controls the max number for snapshots per table in the catalog are cached."); - public static final ConfigOption LINEAGE_META = - key("lineage-meta") - .stringType() - .noDefaultValue() - .withDescription( - Description.builder() - .text( - "The lineage meta to store table and data lineage information.") - .linebreak() - .linebreak() - .text("Possible values:") - .linebreak() - .list( - TextElement.text( - "\"jdbc\": Use standard jdbc to store table and data lineage information.")) - .list( - TextElement.text( - "\"custom\": You can implement LineageMetaFactory and LineageMeta to store lineage information in customized storage.")) - .build()); - public static final ConfigOption ALLOW_UPPER_CASE = ConfigOptions.key("allow-upper-case") .booleanType() diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index 16b76513d7efd..2b277a29b8359 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -24,7 +24,6 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; -import org.apache.paimon.lineage.LineageMetaFactory; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.metastore.MetastoreClient; import org.apache.paimon.operation.FileStoreCommit; @@ -62,7 +61,6 @@ import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.CoreOptions.createCommitUser; import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; -import static org.apache.paimon.options.CatalogOptions.LINEAGE_META; import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; @@ -76,19 +74,14 @@ public abstract class AbstractCatalog implements Catalog { protected final Map tableDefaultOptions; protected final Options catalogOptions; - @Nullable protected final LineageMetaFactory lineageMetaFactory; - protected AbstractCatalog(FileIO fileIO) { this.fileIO = fileIO; - this.lineageMetaFactory = null; this.tableDefaultOptions = new HashMap<>(); this.catalogOptions = new Options(); } protected AbstractCatalog(FileIO fileIO, Options options) { this.fileIO = fileIO; - this.lineageMetaFactory = - findAndCreateLineageMeta(options, AbstractCatalog.class.getClassLoader()); this.tableDefaultOptions = Catalog.tableDefaultOptions(options.toMap()); this.catalogOptions = options; } @@ -377,27 +370,13 @@ public void alterTable( protected abstract void alterTableImpl(Identifier identifier, List changes) throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException; - @Nullable - private LineageMetaFactory findAndCreateLineageMeta(Options options, ClassLoader classLoader) { - return options.getOptional(LINEAGE_META) - .map( - meta -> - FactoryUtil.discoverFactory( - classLoader, LineageMetaFactory.class, meta)) - .orElse(null); - } - @Override public Table getTable(Identifier identifier) throws TableNotExistException { if (isSystemDatabase(identifier.getDatabaseName())) { String tableName = identifier.getTableName(); Table table = SystemTableLoader.loadGlobal( - tableName, - fileIO, - this::allTablePaths, - catalogOptions, - lineageMetaFactory); + tableName, fileIO, this::allTablePaths, catalogOptions); if (table == null) { throw new TableNotExistException(identifier); } @@ -444,8 +423,7 @@ protected Table getDataOrFormatTable(Identifier identifier) throws TableNotExist lockFactory().orElse(null), lockContext().orElse(null), identifier), - metastoreClientFactory(identifier, tableMeta.schema).orElse(null), - lineageMetaFactory)); + metastoreClientFactory(identifier, tableMeta.schema).orElse(null))); CoreOptions options = table.coreOptions(); if (options.type() == TableType.OBJECT_TABLE) { String objectLocation = options.objectLocation(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java b/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java index 9ff5f9b4f6a83..a722d9e21ada3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java @@ -19,7 +19,6 @@ package org.apache.paimon.table; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.lineage.LineageMetaFactory; import org.apache.paimon.metastore.MetastoreClient; import org.apache.paimon.operation.Lock; @@ -27,10 +26,7 @@ import java.io.Serializable; -/** - * Catalog environment in table which contains log factory, metastore client factory and lineage - * meta. - */ +/** Catalog environment in table which contains log factory, metastore client factory. */ public class CatalogEnvironment implements Serializable { private static final long serialVersionUID = 1L; @@ -39,23 +35,20 @@ public class CatalogEnvironment implements Serializable { @Nullable private final String uuid; private final Lock.Factory lockFactory; @Nullable private final MetastoreClient.Factory metastoreClientFactory; - @Nullable private final LineageMetaFactory lineageMetaFactory; public CatalogEnvironment( @Nullable Identifier identifier, @Nullable String uuid, Lock.Factory lockFactory, - @Nullable MetastoreClient.Factory metastoreClientFactory, - @Nullable LineageMetaFactory lineageMetaFactory) { + @Nullable MetastoreClient.Factory metastoreClientFactory) { this.identifier = identifier; this.uuid = uuid; this.lockFactory = lockFactory; this.metastoreClientFactory = metastoreClientFactory; - this.lineageMetaFactory = lineageMetaFactory; } public static CatalogEnvironment empty() { - return new CatalogEnvironment(null, null, Lock.emptyFactory(), null, null); + return new CatalogEnvironment(null, null, Lock.emptyFactory(), null); } @Nullable @@ -76,9 +69,4 @@ public Lock.Factory lockFactory() { public MetastoreClient.Factory metastoreClientFactory() { return metastoreClientFactory; } - - @Nullable - public LineageMetaFactory lineageMetaFactory() { - return lineageMetaFactory; - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java deleted file mode 100644 index 71efce0704713..0000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java +++ /dev/null @@ -1,64 +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.paimon.table.system; - -import org.apache.paimon.lineage.LineageMeta; -import org.apache.paimon.lineage.LineageMetaFactory; -import org.apache.paimon.options.Options; -import org.apache.paimon.table.Table; -import org.apache.paimon.table.source.InnerTableRead; - -import java.util.Map; - -/** - * This is a system table to display all the sink table lineages. - * - *

    - *  For example:
    - *     If we select * from sys.sink_table_lineage, we will get
    - *     database_name       table_name       job_name      create_time
    - *        default            test0            job1    2023-10-22 20:35:12
    - *       database1           test1            job1    2023-10-28 21:35:52
    - *          ...               ...             ...             ...
    - *     We can write sql to fetch the information we need.
    - * 
    - */ -public class SinkTableLineageTable extends TableLineageTable { - - public static final String SINK_TABLE_LINEAGE = "sink_table_lineage"; - - public SinkTableLineageTable(LineageMetaFactory lineageMetaFactory, Options options) { - super(lineageMetaFactory, options); - } - - @Override - public InnerTableRead newRead() { - return new TableLineageRead(lineageMetaFactory, options, LineageMeta::sinkTableLineages); - } - - @Override - public String name() { - return SINK_TABLE_LINEAGE; - } - - @Override - public Table copy(Map dynamicOptions) { - return new SinkTableLineageTable(lineageMetaFactory, options); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java deleted file mode 100644 index 5d9904fa66754..0000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java +++ /dev/null @@ -1,64 +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.paimon.table.system; - -import org.apache.paimon.lineage.LineageMeta; -import org.apache.paimon.lineage.LineageMetaFactory; -import org.apache.paimon.options.Options; -import org.apache.paimon.table.Table; -import org.apache.paimon.table.source.InnerTableRead; - -import java.util.Map; - -/** - * This is a system table to display all the source table lineages. - * - *
    - *  For example:
    - *     If we select * from sys.source_table_lineage, we will get
    - *     database_name       table_name       job_name      create_time
    - *        default            test0            job1    2023-10-22 20:35:12
    - *       database1           test1            job1    2023-10-28 21:35:52
    - *          ...               ...             ...             ...
    - *     We can write sql to fetch the information we need.
    - * 
    - */ -public class SourceTableLineageTable extends TableLineageTable { - - public static final String SOURCE_TABLE_LINEAGE = "source_table_lineage"; - - public SourceTableLineageTable(LineageMetaFactory lineageMetaFactory, Options options) { - super(lineageMetaFactory, options); - } - - @Override - public InnerTableRead newRead() { - return new TableLineageRead(lineageMetaFactory, options, LineageMeta::sourceTableLineages); - } - - @Override - public String name() { - return SOURCE_TABLE_LINEAGE; - } - - @Override - public Table copy(Map dynamicOptions) { - return new SourceTableLineageTable(lineageMetaFactory, options); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java index 3d5b211316ec1..763e4d1216731 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java @@ -20,7 +20,6 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; -import org.apache.paimon.lineage.LineageMetaFactory; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; @@ -37,7 +36,6 @@ import java.util.function.Function; import java.util.function.Supplier; -import static org.apache.paimon.options.CatalogOptions.LINEAGE_META; import static org.apache.paimon.table.system.AggregationFieldsTable.AGGREGATION_FIELDS; import static org.apache.paimon.table.system.AllTableOptionsTable.ALL_TABLE_OPTIONS; import static org.apache.paimon.table.system.AuditLogTable.AUDIT_LOG; @@ -52,12 +50,9 @@ import static org.apache.paimon.table.system.PartitionsTable.PARTITIONS; import static org.apache.paimon.table.system.ReadOptimizedTable.READ_OPTIMIZED; import static org.apache.paimon.table.system.SchemasTable.SCHEMAS; -import static org.apache.paimon.table.system.SinkTableLineageTable.SINK_TABLE_LINEAGE; import static org.apache.paimon.table.system.SnapshotsTable.SNAPSHOTS; -import static org.apache.paimon.table.system.SourceTableLineageTable.SOURCE_TABLE_LINEAGE; import static org.apache.paimon.table.system.StatisticTable.STATISTICS; import static org.apache.paimon.table.system.TagsTable.TAGS; -import static org.apache.paimon.utils.Preconditions.checkNotNull; /** Loader to load system {@link Table}s. */ public class SystemTableLoader { @@ -95,38 +90,18 @@ public static Table loadGlobal( String tableName, FileIO fileIO, Supplier>> allTablePaths, - Options catalogOptions, - @Nullable LineageMetaFactory lineageMetaFactory) { + Options catalogOptions) { switch (tableName.toLowerCase()) { case ALL_TABLE_OPTIONS: return new AllTableOptionsTable(fileIO, allTablePaths.get()); case CATALOG_OPTIONS: return new CatalogOptionsTable(catalogOptions); - case SOURCE_TABLE_LINEAGE: - { - checkNotNull( - lineageMetaFactory, - String.format( - "Lineage meta should be configured for catalog with %s", - LINEAGE_META.key())); - return new SourceTableLineageTable(lineageMetaFactory, catalogOptions); - } - case SINK_TABLE_LINEAGE: - { - checkNotNull( - lineageMetaFactory, - String.format( - "Lineage meta should be configured for catalog with %s", - LINEAGE_META.key())); - return new SinkTableLineageTable(lineageMetaFactory, catalogOptions); - } default: return null; } } public static List loadGlobalTableNames() { - return Arrays.asList( - ALL_TABLE_OPTIONS, CATALOG_OPTIONS, SOURCE_TABLE_LINEAGE, SINK_TABLE_LINEAGE); + return Arrays.asList(ALL_TABLE_OPTIONS, CATALOG_OPTIONS); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/TableLineageTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/TableLineageTable.java deleted file mode 100644 index aeaf3ca3b1330..0000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/TableLineageTable.java +++ /dev/null @@ -1,168 +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.paimon.table.system; - -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.data.GenericRow; -import org.apache.paimon.data.InternalRow; -import org.apache.paimon.disk.IOManager; -import org.apache.paimon.lineage.LineageMeta; -import org.apache.paimon.lineage.LineageMetaFactory; -import org.apache.paimon.lineage.TableLineageEntity; -import org.apache.paimon.options.Options; -import org.apache.paimon.predicate.Predicate; -import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.table.ReadonlyTable; -import org.apache.paimon.table.source.InnerTableRead; -import org.apache.paimon.table.source.InnerTableScan; -import org.apache.paimon.table.source.ReadOnceTableScan; -import org.apache.paimon.table.source.Split; -import org.apache.paimon.table.source.TableRead; -import org.apache.paimon.types.DataField; -import org.apache.paimon.types.RowType; -import org.apache.paimon.types.TimestampType; -import org.apache.paimon.types.VarCharType; -import org.apache.paimon.utils.IteratorRecordReader; -import org.apache.paimon.utils.ProjectedRow; - -import org.apache.paimon.shade.guava30.com.google.common.collect.Iterators; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.BiFunction; - -import static org.apache.paimon.utils.Preconditions.checkNotNull; - -/** Base lineage table for source and sink table lineage. */ -public abstract class TableLineageTable implements ReadonlyTable { - protected final LineageMetaFactory lineageMetaFactory; - protected final Options options; - - public static final RowType TABLE_TYPE = - new RowType( - Arrays.asList( - new DataField( - 0, "database_name", new VarCharType(VarCharType.MAX_LENGTH)), - new DataField(1, "table_name", new VarCharType(VarCharType.MAX_LENGTH)), - new DataField(2, "job_name", new VarCharType(VarCharType.MAX_LENGTH)), - new DataField(3, "create_time", new TimestampType()))); - - protected TableLineageTable(LineageMetaFactory lineageMetaFactory, Options options) { - this.lineageMetaFactory = lineageMetaFactory; - this.options = options; - } - - @Override - public InnerTableScan newScan() { - return new ReadOnceTableScan() { - @Override - public InnerTableScan withFilter(Predicate predicate) { - return this; - } - - @Override - protected Plan innerPlan() { - /// TODO get the real row count for plan. - return () -> Collections.singletonList((Split) () -> 1L); - } - }; - } - - @Override - public RowType rowType() { - return TABLE_TYPE; - } - - @Override - public List primaryKeys() { - return Arrays.asList("database_name", "table_name", "job_name"); - } - - /** Table lineage read with lineage meta query. */ - protected static class TableLineageRead implements InnerTableRead { - private final LineageMetaFactory lineageMetaFactory; - private final Options options; - private final BiFunction> - tableLineageQuery; - @Nullable private Predicate predicate; - private RowType readType; - - protected TableLineageRead( - LineageMetaFactory lineageMetaFactory, - Options options, - BiFunction> - tableLineageQuery) { - this.lineageMetaFactory = lineageMetaFactory; - this.options = options; - this.tableLineageQuery = tableLineageQuery; - this.predicate = null; - } - - @Override - public InnerTableRead withFilter(Predicate predicate) { - this.predicate = predicate; - return this; - } - - @Override - public InnerTableRead withReadType(RowType readType) { - this.readType = readType; - return this; - } - - @Override - public TableRead withIOManager(IOManager ioManager) { - return this; - } - - @Override - public RecordReader createReader(Split split) throws IOException { - try (LineageMeta lineageMeta = lineageMetaFactory.create(() -> options)) { - Iterator sourceTableLineages = - tableLineageQuery.apply(lineageMeta, predicate); - return new IteratorRecordReader<>( - Iterators.transform( - sourceTableLineages, - entity -> { - checkNotNull(entity); - GenericRow row = - GenericRow.of( - BinaryString.fromString(entity.getDatabase()), - BinaryString.fromString(entity.getTable()), - BinaryString.fromString(entity.getJob()), - entity.getCreateTime()); - if (readType != null) { - return ProjectedRow.from( - readType, TableLineageTable.TABLE_TYPE) - .replaceRow(row); - } else { - return row; - } - })); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java index 9f90a2cd01304..6b10dbb84bf46 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java @@ -25,15 +25,10 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.data.Timestamp; import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.flink.sink.FlinkTableSink; import org.apache.paimon.flink.source.DataTableSource; import org.apache.paimon.flink.source.SystemTableSource; -import org.apache.paimon.lineage.LineageMeta; -import org.apache.paimon.lineage.LineageMetaFactory; -import org.apache.paimon.lineage.TableLineageEntity; -import org.apache.paimon.lineage.TableLineageEntityImpl; import org.apache.paimon.options.Options; import org.apache.paimon.options.OptionsUtils; import org.apache.paimon.schema.Schema; @@ -47,7 +42,6 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ExecutionOptions; -import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.ValidationException; @@ -71,7 +65,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.function.BiConsumer; import java.util.regex.Pattern; import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; @@ -109,23 +102,9 @@ public DynamicTableSource createDynamicTableSource(Context context) { isStreamingMode, context.getObjectIdentifier()); } else { - Table table = buildPaimonTable(context); - if (table instanceof FileStoreTable) { - storeTableLineage( - ((FileStoreTable) table).catalogEnvironment().lineageMetaFactory(), - context, - (entity, lineageFactory) -> { - try (LineageMeta lineage = - lineageFactory.create(() -> Options.fromMap(table.options()))) { - lineage.saveSourceTableLineage(entity); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - } return new DataTableSource( context.getObjectIdentifier(), - table, + buildPaimonTable(context), isStreamingMode, context, createOptionalLogStoreFactory(context).orElse(null)); @@ -134,46 +113,13 @@ public DynamicTableSource createDynamicTableSource(Context context) { @Override public DynamicTableSink createDynamicTableSink(Context context) { - Table table = buildPaimonTable(context); - if (table instanceof FileStoreTable) { - storeTableLineage( - ((FileStoreTable) table).catalogEnvironment().lineageMetaFactory(), - context, - (entity, lineageFactory) -> { - try (LineageMeta lineage = - lineageFactory.create(() -> Options.fromMap(table.options()))) { - lineage.saveSinkTableLineage(entity); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - } return new FlinkTableSink( context.getObjectIdentifier(), - table, + buildPaimonTable(context), context, createOptionalLogStoreFactory(context).orElse(null)); } - private void storeTableLineage( - @Nullable LineageMetaFactory lineageMetaFactory, - Context context, - BiConsumer tableLineage) { - if (lineageMetaFactory != null) { - String pipelineName = context.getConfiguration().get(PipelineOptions.NAME); - if (pipelineName == null) { - throw new ValidationException("Cannot get pipeline name for lineage meta."); - } - tableLineage.accept( - new TableLineageEntityImpl( - context.getObjectIdentifier().getDatabaseName(), - context.getObjectIdentifier().getObjectName(), - pipelineName, - Timestamp.fromEpochMillis(System.currentTimeMillis())), - lineageMetaFactory); - } - } - @Override public Set> requiredOptions() { return Collections.emptySet(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java index 96334de3f87b5..10b03b7139ae3 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java @@ -21,8 +21,6 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.table.system.AllTableOptionsTable; import org.apache.paimon.table.system.CatalogOptionsTable; -import org.apache.paimon.table.system.SinkTableLineageTable; -import org.apache.paimon.table.system.SourceTableLineageTable; import org.apache.paimon.utils.BlockingIterator; import org.apache.commons.lang3.StringUtils; @@ -200,9 +198,7 @@ public void testSystemDatabase() { assertThat(sql("SHOW TABLES")) .containsExactlyInAnyOrder( Row.of(AllTableOptionsTable.ALL_TABLE_OPTIONS), - Row.of(CatalogOptionsTable.CATALOG_OPTIONS), - Row.of(SourceTableLineageTable.SOURCE_TABLE_LINEAGE), - Row.of(SinkTableLineageTable.SINK_TABLE_LINEAGE)); + Row.of(CatalogOptionsTable.CATALOG_OPTIONS)); } @Test diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkLineageITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkLineageITCase.java deleted file mode 100644 index 5b61d5272f80e..0000000000000 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkLineageITCase.java +++ /dev/null @@ -1,206 +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.paimon.flink; - -import org.apache.paimon.lineage.DataLineageEntity; -import org.apache.paimon.lineage.LineageMeta; -import org.apache.paimon.lineage.LineageMetaFactory; -import org.apache.paimon.lineage.TableLineageEntity; -import org.apache.paimon.predicate.Predicate; - -import org.apache.flink.configuration.PipelineOptions; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.junit.jupiter.api.Test; - -import javax.annotation.Nullable; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -import static org.apache.paimon.options.CatalogOptions.LINEAGE_META; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** ITCase for flink table and data lineage. */ -public class FlinkLineageITCase extends CatalogITCaseBase { - private static final String THROWING_META = "throwing-meta"; - private static final Map> jobSourceTableLineages = - new HashMap<>(); - private static final Map> jobSinkTableLineages = - new HashMap<>(); - - @Override - protected List ddl() { - return Collections.singletonList("CREATE TABLE IF NOT EXISTS T (a INT, b INT, c INT)"); - } - - @Override - protected Map catalogOptions() { - return Collections.singletonMap(LINEAGE_META.key(), THROWING_META); - } - - @Test - public void testTableLineage() throws Exception { - // Validate for source and sink lineage when pipeline name is null - assertThatThrownBy( - () -> tEnv.executeSql("INSERT INTO T VALUES (1, 2, 3),(4, 5, 6);").await()) - .hasCauseExactlyInstanceOf(ValidationException.class) - .hasRootCauseMessage("Cannot get pipeline name for lineage meta."); - assertThatThrownBy(() -> tEnv.executeSql("SELECT * FROM T").collect().close()) - .hasCauseExactlyInstanceOf(ValidationException.class) - .hasRootCauseMessage("Cannot get pipeline name for lineage meta."); - - // Call storeSinkTableLineage and storeSourceTableLineage methods - tEnv.getConfig().getConfiguration().set(PipelineOptions.NAME, "insert_t_job"); - tEnv.executeSql("INSERT INTO T VALUES (1, 2, 3),(4, 5, 6);").await(); - assertThat(jobSinkTableLineages).isNotEmpty(); - TableLineageEntity sinkTableLineage = - jobSinkTableLineages.get("insert_t_job").get("default.T.insert_t_job"); - assertThat(sinkTableLineage.getTable()).isEqualTo("T"); - - List sinkTableRows = new ArrayList<>(); - try (CloseableIterator iterator = - tEnv.executeSql("SELECT * FROM sys.sink_table_lineage").collect()) { - while (iterator.hasNext()) { - sinkTableRows.add(iterator.next()); - } - } - assertThat(sinkTableRows.size()).isEqualTo(1); - Row sinkTableRow = sinkTableRows.get(0); - assertThat(sinkTableRow.getField("database_name")).isEqualTo("default"); - assertThat(sinkTableRow.getField("table_name")).isEqualTo("T"); - assertThat(sinkTableRow.getField("job_name")).isEqualTo("insert_t_job"); - - tEnv.getConfig().getConfiguration().set(PipelineOptions.NAME, "select_t_job"); - tEnv.executeSql("SELECT * FROM T").collect().close(); - assertThat(jobSourceTableLineages).isNotEmpty(); - TableLineageEntity sourceTableLineage = - jobSourceTableLineages.get("select_t_job").get("default.T.select_t_job"); - assertThat(sourceTableLineage.getTable()).isEqualTo("T"); - - List sourceTableRows = new ArrayList<>(); - try (CloseableIterator iterator = - tEnv.executeSql("SELECT * FROM sys.source_table_lineage").collect()) { - while (iterator.hasNext()) { - sourceTableRows.add(iterator.next()); - } - } - assertThat(sourceTableRows.size()).isEqualTo(1); - Row sourceTableRow = sourceTableRows.get(0); - assertThat(sourceTableRow.getField("database_name")).isEqualTo("default"); - assertThat(sourceTableRow.getField("table_name")).isEqualTo("T"); - assertThat(sourceTableRow.getField("job_name")).isEqualTo("select_t_job"); - } - - private static String getTableLineageKey(TableLineageEntity entity) { - return String.format("%s.%s.%s", entity.getDatabase(), entity.getTable(), entity.getJob()); - } - - /** Factory to create throwing lineage meta. */ - public static class TestingMemoryLineageMetaFactory implements LineageMetaFactory { - private static final long serialVersionUID = 1L; - - @Override - public String identifier() { - return THROWING_META; - } - - @Override - public LineageMeta create(LineageMetaContext context) { - return new TestingMemoryLineageMeta(); - } - } - - /** Throwing specific exception in each method. */ - private static class TestingMemoryLineageMeta implements LineageMeta { - - @Override - public void saveSourceTableLineage(TableLineageEntity entity) { - jobSourceTableLineages - .computeIfAbsent(entity.getJob(), key -> new HashMap<>()) - .put(getTableLineageKey(entity), entity); - } - - @Override - public void deleteSourceTableLineage(String job) { - jobSourceTableLineages.remove(job); - } - - @Override - public Iterator sourceTableLineages(@Nullable Predicate predicate) { - return jobSourceTableLineages.values().stream() - .flatMap(v -> v.values().stream()) - .iterator(); - } - - @Override - public void saveSinkTableLineage(TableLineageEntity entity) { - assertThat(entity.getJob()).isEqualTo("insert_t_job"); - assertThat(entity.getTable()).isEqualTo("T"); - assertThat(entity.getDatabase()).isEqualTo("default"); - jobSinkTableLineages - .computeIfAbsent(entity.getJob(), key -> new HashMap<>()) - .put(getTableLineageKey(entity), entity); - } - - @Override - public Iterator sinkTableLineages(@Nullable Predicate predicate) { - return jobSinkTableLineages.values().stream() - .flatMap(v -> v.values().stream()) - .iterator(); - } - - @Override - public void deleteSinkTableLineage(String job) { - jobSinkTableLineages.remove(job); - } - - @Override - public void saveSourceDataLineage(DataLineageEntity entity) { - assertThat(entity.getJob()).isEqualTo("select_t_job"); - assertThat(entity.getTable()).isEqualTo("T"); - assertThat(entity.getDatabase()).isEqualTo("default"); - throw new UnsupportedOperationException("Method saveSinkTableLineage is not supported"); - } - - @Override - public Iterator sourceDataLineages(@Nullable Predicate predicate) { - throw new UnsupportedOperationException(); - } - - @Override - public void saveSinkDataLineage(DataLineageEntity entity) { - throw new UnsupportedOperationException(); - } - - @Override - public Iterator sinkDataLineages(@Nullable Predicate predicate) { - throw new UnsupportedOperationException(); - } - - @Override - public void close() throws Exception {} - } -} diff --git a/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory index fcb6fe982943f..3c05b5fba3ec9 100644 --- a/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -15,8 +15,5 @@ org.apache.paimon.flink.FlinkCatalogTest$TestingLogSoreRegisterFactory -# Lineage meta factory -org.apache.paimon.flink.FlinkLineageITCase$TestingMemoryLineageMetaFactory - # Catalog lock factory org.apache.paimon.flink.FileSystemCatalogITCase$FileSystemCatalogDummyLockFactory \ No newline at end of file diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 9a90995f282d2..5157e606006ca 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -610,8 +610,7 @@ public org.apache.paimon.table.Table getDataOrFormatTable(Identifier identifier) lockFactory().orElse(null), lockContext().orElse(null), identifier), - metastoreClientFactory(identifier, tableMeta.schema()).orElse(null), - lineageMetaFactory)); + metastoreClientFactory(identifier, tableMeta.schema()).orElse(null))); } catch (TableNotExistException ignore) { } From 4c6c557e6fd70e9ab9e5ebd4292f80340e5803ed Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Sun, 1 Dec 2024 19:38:54 +0800 Subject: [PATCH 083/157] [flink] Avoid deprecated SetupableStreamOperator (#4591) --- .../flink/sink/cdc/CdcDynamicBucketSink.java | 8 +- .../cdc/CdcDynamicBucketWriteOperator.java | 35 ++- .../flink/sink/cdc/CdcFixedBucketSink.java | 6 +- .../cdc/CdcRecordStoreMultiWriteOperator.java | 46 +++- .../sink/cdc/CdcRecordStoreWriteOperator.java | 35 ++- .../flink/sink/cdc/CdcUnawareBucketSink.java | 6 +- .../cdc/CdcUnawareBucketWriteOperator.java | 35 ++- .../sink/cdc/FlinkCdcMultiTableSink.java | 17 +- .../cdc/FlinkCdcSyncDatabaseSinkBuilder.java | 4 +- .../CdcRecordStoreMultiWriteOperatorTest.java | 6 +- .../cdc/CdcRecordStoreWriteOperatorTest.java | 6 +- .../sink/cdc/FlinkCdcMultiTableSinkTest.java | 1 - .../AppendBypassCompactWorkerOperator.java | 35 ++- .../sink/AppendCompactWorkerOperator.java | 22 +- ...nlyMultiTableCompactionWorkerOperator.java | 42 +++- ...lySingleTableCompactionWorkerOperator.java | 34 ++- .../AutoTagForSavepointCommitterOperator.java | 24 +-- ...gForSavepointCommitterOperatorFactory.java | 94 ++++++++ .../sink/BatchWriteGeneratorTagOperator.java | 24 +-- ...BatchWriteGeneratorTagOperatorFactory.java | 60 ++++++ .../sink/CombinedTableCompactorSink.java | 14 +- .../paimon/flink/sink/CommitterOperator.java | 26 +-- .../flink/sink/CommitterOperatorFactory.java | 115 ++++++++++ .../paimon/flink/sink/CompactorSink.java | 6 +- .../sink/DynamicBucketRowWriteOperator.java | 34 ++- .../paimon/flink/sink/FixedBucketSink.java | 7 +- .../apache/paimon/flink/sink/FlinkSink.java | 24 ++- .../paimon/flink/sink/FlinkSinkBuilder.java | 2 +- .../paimon/flink/sink/LocalMergeOperator.java | 34 ++- .../sink/MultiTablesStoreCompactOperator.java | 58 ++++- .../flink/sink/PrepareCommitOperator.java | 19 +- .../flink/sink/RewriteFileIndexSink.java | 56 +++-- .../flink/sink/RowDataStoreWriteOperator.java | 53 +++-- .../flink/sink/RowDynamicBucketSink.java | 8 +- .../flink/sink/RowUnawareBucketSink.java | 46 ++-- .../flink/sink/StoreCompactOperator.java | 50 ++++- .../paimon/flink/sink/TableWriteOperator.java | 23 +- .../sink/UnawareBucketCompactionSink.java | 8 +- .../paimon/flink/sink/UnawareBucketSink.java | 5 +- .../sink/index/GlobalDynamicBucketSink.java | 11 +- .../sink/index/IndexBootstrapOperator.java | 39 +++- .../AppendBypassCoordinateOperator.java | 8 +- ...AppendBypassCoordinateOperatorFactory.java | 6 +- ...ultiTableCompactionWorkerOperatorTest.java | 19 +- ...ngleTableCompactionWorkerOperatorTest.java | 28 ++- ...oTagForSavepointCommitterOperatorTest.java | 35 +-- .../BatchWriteGeneratorTagOperatorTest.java | 39 +++- .../flink/sink/CommitterOperatorTest.java | 110 ++++++---- .../flink/sink/CompactorSinkITCase.java | 8 +- .../paimon/flink/sink/FlinkSinkTest.java | 18 +- .../flink/sink/LocalMergeOperatorTest.java | 18 +- .../flink/sink/StoreCompactOperatorTest.java | 8 +- .../flink/sink/StoreMultiCommitterTest.java | 15 +- .../sink/WriterChainingStrategyTest.java | 203 ++++++++++++++++++ .../paimon/flink/sink/WriterOperatorTest.java | 45 ++-- 55 files changed, 1397 insertions(+), 341 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java create mode 100644 paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java index 574ff685f3fa4..6d9e3a4a7c82f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java @@ -25,7 +25,7 @@ import org.apache.paimon.table.sink.KeyAndBucketExtractor; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; /** {@link CdcDynamicBucketSinkBase} for {@link CdcRecord}. */ public class CdcDynamicBucketSink extends CdcDynamicBucketSinkBase { @@ -42,8 +42,8 @@ protected KeyAndBucketExtractor createExtractor(TableSchema schema) { } @Override - protected OneInputStreamOperator, Committable> createWriteOperator( - StoreSinkWrite.Provider writeProvider, String commitUser) { - return new CdcDynamicBucketWriteOperator(table, writeProvider, commitUser); + protected OneInputStreamOperatorFactory, Committable> + createWriteOperatorFactory(StoreSinkWrite.Provider writeProvider, String commitUser) { + return new CdcDynamicBucketWriteOperator.Factory(table, writeProvider, commitUser); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java index b2fbdc3e93eef..b0b135b3610bb 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.data.GenericRow; +import org.apache.paimon.flink.sink.Committable; import org.apache.paimon.flink.sink.PrepareCommitOperator; import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.flink.sink.TableWriteOperator; @@ -26,6 +27,9 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import java.io.IOException; @@ -43,11 +47,12 @@ public class CdcDynamicBucketWriteOperator extends TableWriteOperator parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); this.retrySleepMillis = table.coreOptions().toConfiguration().get(RETRY_SLEEP_TIME).toMillis(); } @@ -85,4 +90,30 @@ public void processElement(StreamRecord> element) thr throw new IOException(e); } } + + /** {@link StreamOperatorFactory} of {@link CdcDynamicBucketWriteOperator}. */ + public static class Factory extends TableWriteOperator.Factory> { + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CdcDynamicBucketWriteOperator( + parameters, table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CdcDynamicBucketWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java index 59bdb192beea7..bec9508888b4c 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java @@ -24,7 +24,7 @@ import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; /** * A {@link FlinkSink} for fixed-bucket table which accepts {@link CdcRecord} and waits for a schema @@ -39,8 +39,8 @@ public CdcFixedBucketSink(FileStoreTable table) { } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new CdcRecordStoreWriteOperator(table, writeProvider, commitUser); + return new CdcRecordStoreWriteOperator.Factory(table, writeProvider, commitUser); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java index 7d72fe3e801f9..5db111a300474 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java @@ -38,6 +38,9 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import java.io.IOException; @@ -74,12 +77,13 @@ public class CdcRecordStoreMultiWriteOperator private String commitUser; private ExecutorService compactExecutor; - public CdcRecordStoreMultiWriteOperator( + private CdcRecordStoreMultiWriteOperator( + StreamOperatorParameters parameters, Catalog.Loader catalogLoader, StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, String initialCommitUser, Options options) { - super(options); + super(parameters, options); this.catalogLoader = catalogLoader; this.storeSinkWriteProvider = storeSinkWriteProvider; this.initialCommitUser = initialCommitUser; @@ -254,4 +258,42 @@ public Map writes() { public String commitUser() { return commitUser; } + + /** {@link StreamOperatorFactory} of {@link CdcRecordStoreMultiWriteOperator}. */ + public static class Factory + extends PrepareCommitOperator.Factory { + private final StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider; + private final String initialCommitUser; + private final Catalog.Loader catalogLoader; + + public Factory( + Catalog.Loader catalogLoader, + StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, + String initialCommitUser, + Options options) { + super(options); + this.catalogLoader = catalogLoader; + this.storeSinkWriteProvider = storeSinkWriteProvider; + this.initialCommitUser = initialCommitUser; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CdcRecordStoreMultiWriteOperator( + parameters, + catalogLoader, + storeSinkWriteProvider, + initialCommitUser, + options); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CdcRecordStoreMultiWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java index dd0aa2e5622c2..195e683daaf6f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.data.GenericRow; +import org.apache.paimon.flink.sink.Committable; import org.apache.paimon.flink.sink.PrepareCommitOperator; import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.flink.sink.TableWriteOperator; @@ -27,6 +28,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import java.io.IOException; @@ -50,11 +54,12 @@ public class CdcRecordStoreWriteOperator extends TableWriteOperator { private final long retrySleepMillis; - public CdcRecordStoreWriteOperator( + protected CdcRecordStoreWriteOperator( + StreamOperatorParameters parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); this.retrySleepMillis = table.coreOptions().toConfiguration().get(RETRY_SLEEP_TIME).toMillis(); } @@ -92,4 +97,30 @@ public void processElement(StreamRecord element) throws Exception { throw new IOException(e); } } + + /** {@link StreamOperatorFactory} of {@link CdcRecordStoreWriteOperator}. */ + public static class Factory extends TableWriteOperator.Factory { + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CdcRecordStoreWriteOperator( + parameters, table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CdcRecordStoreWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java index 313f4d013ef89..820ef7728f8ce 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java @@ -24,7 +24,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -42,9 +42,9 @@ public CdcUnawareBucketSink(FileStoreTable table, Integer parallelism) { } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new CdcUnawareBucketWriteOperator(table, writeProvider, commitUser); + return new CdcUnawareBucketWriteOperator.Factory(table, writeProvider, commitUser); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketWriteOperator.java index c57a40f3f71da..26f65fdd09ce2 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketWriteOperator.java @@ -18,21 +18,26 @@ package org.apache.paimon.flink.sink.cdc; +import org.apache.paimon.flink.sink.Committable; import org.apache.paimon.flink.sink.PrepareCommitOperator; import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.RowKind; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** A {@link PrepareCommitOperator} to write {@link CdcRecord} to unaware-bucket mode table. */ public class CdcUnawareBucketWriteOperator extends CdcRecordStoreWriteOperator { - public CdcUnawareBucketWriteOperator( + private CdcUnawareBucketWriteOperator( + StreamOperatorParameters parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); } @Override @@ -42,4 +47,30 @@ public void processElement(StreamRecord element) throws Exception { super.processElement(element); } } + + /** {@link StreamOperatorFactory} of {@link CdcUnawareBucketWriteOperator}. */ + public static class Factory extends CdcRecordStoreWriteOperator.Factory { + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CdcUnawareBucketWriteOperator( + parameters, table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CdcUnawareBucketWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java index 55e987c6055fa..f9b7bbc6b9105 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java @@ -21,7 +21,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.sink.CommittableStateManager; import org.apache.paimon.flink.sink.Committer; -import org.apache.paimon.flink.sink.CommitterOperator; +import org.apache.paimon.flink.sink.CommitterOperatorFactory; import org.apache.paimon.flink.sink.FlinkSink; import org.apache.paimon.flink.sink.FlinkStreamPartitioner; import org.apache.paimon.flink.sink.MultiTableCommittable; @@ -41,7 +41,7 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -63,19 +63,16 @@ public class FlinkCdcMultiTableSink implements Serializable { private final Catalog.Loader catalogLoader; private final double commitCpuCores; @Nullable private final MemorySize commitHeapMemory; - private final boolean commitChaining; private final String commitUser; public FlinkCdcMultiTableSink( Catalog.Loader catalogLoader, double commitCpuCores, @Nullable MemorySize commitHeapMemory, - boolean commitChaining, String commitUser) { this.catalogLoader = catalogLoader; this.commitCpuCores = commitCpuCores; this.commitHeapMemory = commitHeapMemory; - this.commitChaining = commitChaining; this.commitUser = commitUser; } @@ -129,10 +126,9 @@ public DataStreamSink sinkFrom( .transform( GLOBAL_COMMITTER_NAME, typeInfo, - new CommitterOperator<>( + new CommitterOperatorFactory<>( true, false, - commitChaining, commitUser, createCommitterFactory(), createCommittableStateManager())) @@ -141,9 +137,10 @@ public DataStreamSink sinkFrom( return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); } - protected OneInputStreamOperator createWriteOperator( - StoreSinkWrite.WithWriteBufferProvider writeProvider, String commitUser) { - return new CdcRecordStoreMultiWriteOperator( + protected OneInputStreamOperatorFactory + createWriteOperator( + StoreSinkWrite.WithWriteBufferProvider writeProvider, String commitUser) { + return new CdcRecordStoreMultiWriteOperator.Factory( catalogLoader, writeProvider, commitUser, new Options()); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java index ed8fdd113389a..a9ad66847b4b5 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java @@ -66,7 +66,6 @@ public class FlinkCdcSyncDatabaseSinkBuilder { @Nullable private Integer parallelism; private double committerCpu; @Nullable private MemorySize committerMemory; - private boolean commitChaining; // Paimon catalog used to check and create tables. There will be two // places where this catalog is used. 1) in processing function, @@ -103,7 +102,6 @@ public FlinkCdcSyncDatabaseSinkBuilder withTableOptions(Options options) { this.parallelism = options.get(FlinkConnectorOptions.SINK_PARALLELISM); this.committerCpu = options.get(FlinkConnectorOptions.SINK_COMMITTER_CPU); this.committerMemory = options.get(FlinkConnectorOptions.SINK_COMMITTER_MEMORY); - this.commitChaining = options.get(FlinkConnectorOptions.SINK_COMMITTER_OPERATOR_CHAINING); this.commitUser = createCommitUser(options); return this; } @@ -169,7 +167,7 @@ private void buildCombinedCdcSink() { FlinkCdcMultiTableSink sink = new FlinkCdcMultiTableSink( - catalogLoader, committerCpu, committerMemory, commitChaining, commitUser); + catalogLoader, committerCpu, committerMemory, commitUser); sink.sinkFrom(partitioned); } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java index 8c78ab853a60a..9f35b25026bbb 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java @@ -689,8 +689,8 @@ public void testUsingTheSameCompactExecutor() throws Exception { private OneInputStreamOperatorTestHarness createTestHarness(Catalog.Loader catalogLoader) throws Exception { - CdcRecordStoreMultiWriteOperator operator = - new CdcRecordStoreMultiWriteOperator( + CdcRecordStoreMultiWriteOperator.Factory operatorFactory = + new CdcRecordStoreMultiWriteOperator.Factory( catalogLoader, (t, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> new StoreSinkWriteImpl( @@ -709,7 +709,7 @@ public void testUsingTheSameCompactExecutor() throws Exception { TypeSerializer outputSerializer = new MultiTableCommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator, inputSerializer); + new OneInputStreamOperatorTestHarness<>(operatorFactory, inputSerializer); harness.setup(outputSerializer); return harness; } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java index f3693fe405ded..f00229d99890e 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java @@ -253,8 +253,8 @@ public void testUpdateColumnType() throws Exception { private OneInputStreamOperatorTestHarness createTestHarness( FileStoreTable table) throws Exception { - CdcRecordStoreWriteOperator operator = - new CdcRecordStoreWriteOperator( + CdcRecordStoreWriteOperator.Factory operatorFactory = + new CdcRecordStoreWriteOperator.Factory( table, (t, commitUser, state, ioManager, memoryPool, metricGroup) -> new StoreSinkWriteImpl( @@ -272,7 +272,7 @@ private OneInputStreamOperatorTestHarness createTestHarn TypeSerializer outputSerializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator, inputSerializer); + new OneInputStreamOperatorTestHarness<>(operatorFactory, inputSerializer); harness.setup(outputSerializer); return harness; } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java index fd23e500d5e55..e1bd112ca751e 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java @@ -60,7 +60,6 @@ public void cancel() {} () -> FlinkCatalogFactory.createPaimonCatalog(new Options()), FlinkConnectorOptions.SINK_COMMITTER_CPU.defaultValue(), null, - true, UUID.randomUUID().toString()); DataStreamSink dataStreamSink = sink.sinkFrom(input); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java index 92cd31ea8aa26..977511920a061 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java @@ -21,7 +21,9 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.types.Either; @@ -29,9 +31,11 @@ public class AppendBypassCompactWorkerOperator extends AppendCompactWorkerOperator> { - public AppendBypassCompactWorkerOperator(FileStoreTable table, String commitUser) { - super(table, commitUser); - this.chainingStrategy = ChainingStrategy.HEAD; + private AppendBypassCompactWorkerOperator( + StreamOperatorParameters parameters, + FileStoreTable table, + String commitUser) { + super(parameters, table, commitUser); } @Override @@ -49,4 +53,27 @@ public void processElement( unawareBucketCompactor.processElement(element.getValue().right()); } } + + /** {@link StreamOperatorFactory} of {@link AppendBypassCompactWorkerOperator}. */ + public static class Factory + extends AppendCompactWorkerOperator.Factory< + Either> { + + public Factory(FileStoreTable table, String initialCommitUser) { + super(table, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) new AppendBypassCompactWorkerOperator(parameters, table, commitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return AppendBypassCompactWorkerOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendCompactWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendCompactWorkerOperator.java index 52ab75de6b2c3..7a3c0231eb65d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendCompactWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendCompactWorkerOperator.java @@ -27,6 +27,8 @@ import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.utils.ExecutorThreadFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,8 +55,11 @@ public abstract class AppendCompactWorkerOperator private transient ExecutorService lazyCompactExecutor; - public AppendCompactWorkerOperator(FileStoreTable table, String commitUser) { - super(Options.fromMap(table.options())); + public AppendCompactWorkerOperator( + StreamOperatorParameters parameters, + FileStoreTable table, + String commitUser) { + super(parameters, Options.fromMap(table.options())); this.table = table; this.commitUser = commitUser; } @@ -101,4 +106,17 @@ public void close() throws Exception { this.unawareBucketCompactor.close(); } } + + /** {@link StreamOperatorFactory} of {@link AppendCompactWorkerOperator}. */ + protected abstract static class Factory + extends PrepareCommitOperator.Factory { + protected final FileStoreTable table; + protected final String commitUser; + + protected Factory(FileStoreTable table, String commitUser) { + super(Options.fromMap(table.options())); + this.table = table; + this.commitUser = commitUser; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java index 15e7b9746fe6c..83d51f302e51f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java @@ -28,6 +28,9 @@ import org.apache.paimon.utils.ExceptionUtils; import org.apache.paimon.utils.ExecutorThreadFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,9 +65,12 @@ public class AppendOnlyMultiTableCompactionWorkerOperator private transient Catalog catalog; - public AppendOnlyMultiTableCompactionWorkerOperator( - Catalog.Loader catalogLoader, String commitUser, Options options) { - super(options); + private AppendOnlyMultiTableCompactionWorkerOperator( + StreamOperatorParameters parameters, + Catalog.Loader catalogLoader, + String commitUser, + Options options) { + super(parameters, options); this.commitUser = commitUser; this.catalogLoader = catalogLoader; } @@ -175,4 +181,34 @@ public void close() throws Exception { ExceptionUtils.throwMultiException(exceptions); } + + /** {@link StreamOperatorFactory} of {@link AppendOnlyMultiTableCompactionWorkerOperator}. */ + public static class Factory + extends PrepareCommitOperator.Factory< + MultiTableUnawareAppendCompactionTask, MultiTableCommittable> { + + private final String commitUser; + private final Catalog.Loader catalogLoader; + + public Factory(Catalog.Loader catalogLoader, String commitUser, Options options) { + super(options); + this.commitUser = commitUser; + this.catalogLoader = catalogLoader; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new AppendOnlyMultiTableCompactionWorkerOperator( + parameters, catalogLoader, commitUser, options); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return AppendOnlyMultiTableCompactionWorkerOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperator.java index 4d0201d324615..917a7f64f1a04 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperator.java @@ -22,6 +22,9 @@ import org.apache.paimon.flink.source.BucketUnawareCompactSource; import org.apache.paimon.table.FileStoreTable; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** @@ -31,12 +34,39 @@ public class AppendOnlySingleTableCompactionWorkerOperator extends AppendCompactWorkerOperator { - public AppendOnlySingleTableCompactionWorkerOperator(FileStoreTable table, String commitUser) { - super(table, commitUser); + private AppendOnlySingleTableCompactionWorkerOperator( + StreamOperatorParameters parameters, + FileStoreTable table, + String commitUser) { + super(parameters, table, commitUser); } @Override public void processElement(StreamRecord element) throws Exception { this.unawareBucketCompactor.processElement(element.getValue()); } + + /** {@link StreamOperatorFactory} of {@link AppendOnlySingleTableCompactionWorkerOperator}. */ + public static class Factory + extends AppendCompactWorkerOperator.Factory { + + public Factory(FileStoreTable table, String initialCommitUser) { + super(table, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new AppendOnlySingleTableCompactionWorkerOperator( + parameters, table, commitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return AppendOnlySingleTableCompactionWorkerOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java index 6d27c60194837..0822f04612413 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java @@ -32,18 +32,13 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CheckpointStreamFactory; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; -import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.operators.SetupableStreamOperator; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import java.time.Duration; @@ -58,9 +53,7 @@ * time, tags are automatically created for each flink savepoint. */ public class AutoTagForSavepointCommitterOperator - implements OneInputStreamOperator, - SetupableStreamOperator, - BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { public static final String SAVEPOINT_TAG_PREFIX = "savepoint-"; private static final long serialVersionUID = 1L; @@ -256,19 +249,4 @@ public void setKeyContextElement(StreamRecord record) throws Exception public void endInput() throws Exception { commitOperator.endInput(); } - - @Override - public void setup(StreamTask containingTask, StreamConfig config, Output output) { - commitOperator.setup(containingTask, config, output); - } - - @Override - public ChainingStrategy getChainingStrategy() { - return commitOperator.getChainingStrategy(); - } - - @Override - public void setChainingStrategy(ChainingStrategy strategy) { - commitOperator.setChainingStrategy(strategy); - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java new file mode 100644 index 0000000000000..1787f8e7adce5 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java @@ -0,0 +1,94 @@ +/* + * 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.paimon.flink.sink; + +import org.apache.paimon.operation.TagDeletion; +import org.apache.paimon.table.sink.TagCallback; +import org.apache.paimon.utils.SerializableSupplier; +import org.apache.paimon.utils.SnapshotManager; +import org.apache.paimon.utils.TagManager; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; + +import java.time.Duration; +import java.util.List; +import java.util.NavigableSet; +import java.util.TreeSet; + +/** + * {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for {@link + * AutoTagForSavepointCommitterOperator}. + */ +public class AutoTagForSavepointCommitterOperatorFactory + extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + + private final CommitterOperatorFactory commitOperatorFactory; + + private final SerializableSupplier snapshotManagerFactory; + + private final SerializableSupplier tagManagerFactory; + + private final SerializableSupplier tagDeletionFactory; + + private final SerializableSupplier> callbacksSupplier; + + private final NavigableSet identifiersForTags; + + private final Duration tagTimeRetained; + + public AutoTagForSavepointCommitterOperatorFactory( + CommitterOperatorFactory commitOperatorFactory, + SerializableSupplier snapshotManagerFactory, + SerializableSupplier tagManagerFactory, + SerializableSupplier tagDeletionFactory, + SerializableSupplier> callbacksSupplier, + Duration tagTimeRetained) { + this.commitOperatorFactory = commitOperatorFactory; + this.tagManagerFactory = tagManagerFactory; + this.snapshotManagerFactory = snapshotManagerFactory; + this.tagDeletionFactory = tagDeletionFactory; + this.callbacksSupplier = callbacksSupplier; + this.identifiersForTags = new TreeSet<>(); + this.tagTimeRetained = tagTimeRetained; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new AutoTagForSavepointCommitterOperator<>( + commitOperatorFactory.createStreamOperator(parameters), + snapshotManagerFactory, + tagManagerFactory, + tagDeletionFactory, + callbacksSupplier, + tagTimeRetained); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return AutoTagForSavepointCommitterOperator.class; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java index 23202b45077ff..1cbcc4b2262fd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java @@ -28,18 +28,13 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CheckpointStreamFactory; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; -import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.operators.SetupableStreamOperator; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import java.time.Instant; @@ -53,9 +48,7 @@ * completed, the corresponding tag is generated. */ public class BatchWriteGeneratorTagOperator - implements OneInputStreamOperator, - SetupableStreamOperator, - BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { private static final String BATCH_WRITE_TAG_PREFIX = "batch-write-"; @@ -250,19 +243,4 @@ public void setKeyContextElement(StreamRecord record) throws Exception public void endInput() throws Exception { commitOperator.endInput(); } - - @Override - public void setup(StreamTask containingTask, StreamConfig config, Output output) { - commitOperator.setup(containingTask, config, output); - } - - @Override - public ChainingStrategy getChainingStrategy() { - return commitOperator.getChainingStrategy(); - } - - @Override - public void setChainingStrategy(ChainingStrategy strategy) { - commitOperator.setChainingStrategy(strategy); - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java new file mode 100644 index 0000000000000..e3c0e5c491680 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java @@ -0,0 +1,60 @@ +/* + * 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.paimon.flink.sink; + +import org.apache.paimon.table.FileStoreTable; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; + +/** + * {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for {@link + * BatchWriteGeneratorTagOperator}. + */ +public class BatchWriteGeneratorTagOperatorFactory + extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final CommitterOperatorFactory commitOperatorFactory; + + protected final FileStoreTable table; + + public BatchWriteGeneratorTagOperatorFactory( + CommitterOperatorFactory commitOperatorFactory, + FileStoreTable table) { + this.table = table; + this.commitOperatorFactory = commitOperatorFactory; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new BatchWriteGeneratorTagOperator<>( + commitOperatorFactory.createStreamOperator(parameters), table); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return BatchWriteGeneratorTagOperator.class; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java index ce4e37305909a..c2b4cc0f87e60 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java @@ -33,7 +33,7 @@ import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.data.RowData; import java.io.Serializable; @@ -119,7 +119,7 @@ public DataStream doWrite( .transform( String.format("%s-%s", "Unaware-Bucket-Table", WRITER_NAME), new MultiTableCommittableTypeInfo(), - new AppendOnlyMultiTableCompactionWorkerOperator( + new AppendOnlyMultiTableCompactionWorkerOperator.Factory( catalogLoader, commitUser, options)) .setParallelism(unawareBucketTableSource.getParallelism()); @@ -160,26 +160,28 @@ protected DataStreamSink doCommit( .transform( GLOBAL_COMMITTER_NAME, new MultiTableCommittableTypeInfo(), - new CommitterOperator<>( + new CommitterOperatorFactory<>( streamingCheckpointEnabled, false, - options.get(SINK_COMMITTER_OPERATOR_CHAINING), commitUser, createCommitterFactory(isStreaming), createCommittableStateManager(), options.get(END_INPUT_WATERMARK))) .setParallelism(written.getParallelism()); + if (!options.get(SINK_COMMITTER_OPERATOR_CHAINING)) { + committed = committed.startNewChain(); + } return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); } // TODO:refactor FlinkSink to adopt this sink - protected OneInputStreamOperator + protected OneInputStreamOperatorFactory combinedMultiComacptionWriteOperator( CheckpointConfig checkpointConfig, boolean isStreaming, boolean fullCompaction, String commitUser) { - return new MultiTablesStoreCompactOperator( + return new MultiTablesStoreCompactOperator.Factory( catalogLoader, commitUser, checkpointConfig, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java index 021a5db413d54..383cbcd6ebf71 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java @@ -25,8 +25,8 @@ import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -91,26 +91,9 @@ public class CommitterOperator extends AbstractStreamOpe private final Long endInputWatermark; public CommitterOperator( + StreamOperatorParameters parameters, boolean streamingCheckpointEnabled, boolean forceSingleParallelism, - boolean chaining, - String initialCommitUser, - Committer.Factory committerFactory, - CommittableStateManager committableStateManager) { - this( - streamingCheckpointEnabled, - forceSingleParallelism, - chaining, - initialCommitUser, - committerFactory, - committableStateManager, - null); - } - - public CommitterOperator( - boolean streamingCheckpointEnabled, - boolean forceSingleParallelism, - boolean chaining, String initialCommitUser, Committer.Factory committerFactory, CommittableStateManager committableStateManager, @@ -122,7 +105,10 @@ public CommitterOperator( this.committerFactory = checkNotNull(committerFactory); this.committableStateManager = committableStateManager; this.endInputWatermark = endInputWatermark; - setChainingStrategy(chaining ? ChainingStrategy.ALWAYS : ChainingStrategy.HEAD); + this.setup( + parameters.getContainingTask(), + parameters.getStreamConfig(), + parameters.getOutput()); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java new file mode 100644 index 0000000000000..cce3d4e176bf8 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java @@ -0,0 +1,115 @@ +/* + * 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.paimon.flink.sink; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; + +import java.util.NavigableMap; +import java.util.TreeMap; + +import static org.apache.paimon.utils.Preconditions.checkNotNull; + +/** + * {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for {@link + * CommitterOperator}. + */ +public class CommitterOperatorFactory + extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + protected final boolean streamingCheckpointEnabled; + + /** Whether to check the parallelism while runtime. */ + protected final boolean forceSingleParallelism; + /** + * This commitUser is valid only for new jobs. After the job starts, this commitUser will be + * recorded into the states of write and commit operators. When the job restarts, commitUser + * will be recovered from states and this value is ignored. + */ + protected final String initialCommitUser; + + /** Group the committable by the checkpoint id. */ + protected final NavigableMap committablesPerCheckpoint; + + protected final Committer.Factory committerFactory; + + protected final CommittableStateManager committableStateManager; + + /** + * Aggregate committables to global committables and commit the global committables to the + * external system. + */ + protected Committer committer; + + protected final Long endInputWatermark; + + public CommitterOperatorFactory( + boolean streamingCheckpointEnabled, + boolean forceSingleParallelism, + String initialCommitUser, + Committer.Factory committerFactory, + CommittableStateManager committableStateManager) { + this( + streamingCheckpointEnabled, + forceSingleParallelism, + initialCommitUser, + committerFactory, + committableStateManager, + null); + } + + public CommitterOperatorFactory( + boolean streamingCheckpointEnabled, + boolean forceSingleParallelism, + String initialCommitUser, + Committer.Factory committerFactory, + CommittableStateManager committableStateManager, + Long endInputWatermark) { + this.streamingCheckpointEnabled = streamingCheckpointEnabled; + this.forceSingleParallelism = forceSingleParallelism; + this.initialCommitUser = initialCommitUser; + this.committablesPerCheckpoint = new TreeMap<>(); + this.committerFactory = checkNotNull(committerFactory); + this.committableStateManager = committableStateManager; + this.endInputWatermark = endInputWatermark; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CommitterOperator<>( + parameters, + streamingCheckpointEnabled, + forceSingleParallelism, + initialCommitUser, + committerFactory, + committableStateManager, + endInputWatermark); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CommitterOperator.class; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java index a0c830d73f582..a9c6031dfa346 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java @@ -21,7 +21,7 @@ import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.data.RowData; /** {@link FlinkSink} for dedicated compact jobs. */ @@ -37,9 +37,9 @@ public CompactorSink(FileStoreTable table, boolean fullCompaction) { } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new StoreCompactOperator(table, writeProvider, commitUser, fullCompaction); + return new StoreCompactOperator.Factory(table, writeProvider, commitUser, fullCompaction); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java index 53b9be457c3d8..b31a1af05224f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java @@ -22,6 +22,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** @@ -32,11 +35,12 @@ public class DynamicBucketRowWriteOperator private static final long serialVersionUID = 1L; - public DynamicBucketRowWriteOperator( + private DynamicBucketRowWriteOperator( + StreamOperatorParameters parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); } @Override @@ -49,4 +53,30 @@ public void processElement(StreamRecord> element) throws Exception { write.write(element.getValue().f0, element.getValue().f1); } + + /** {@link StreamOperatorFactory} of {@link DynamicBucketRowWriteOperator}. */ + public static class Factory extends TableWriteOperator.Factory> { + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new DynamicBucketRowWriteOperator( + parameters, table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return DynamicBucketRowWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java index 613bf369b0524..402abb4d5aac0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java @@ -21,7 +21,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -43,8 +43,9 @@ public FixedBucketSink( } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser); + return new RowDataStoreWriteOperator.Factory( + table, logSinkFunction, writeProvider, commitUser); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java index dd364c196d8bf..8d6c3554c76ff 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java @@ -44,7 +44,7 @@ import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; import javax.annotation.Nullable; @@ -220,7 +220,7 @@ public DataStream doWrite( + " : " + table.name(), new CommittableTypeInfo(), - createWriteOperator( + createWriteOperatorFactory( createWriteProvider( env.getCheckpointConfig(), isStreaming, @@ -268,11 +268,10 @@ protected DataStreamSink doCommit(DataStream written, String com } Options options = Options.fromMap(table.options()); - OneInputStreamOperator committerOperator = - new CommitterOperator<>( + OneInputStreamOperatorFactory committerOperator = + new CommitterOperatorFactory<>( streamingCheckpointEnabled, true, - options.get(SINK_COMMITTER_OPERATOR_CHAINING), commitUser, createCommitterFactory(), createCommittableStateManager(), @@ -280,8 +279,9 @@ protected DataStreamSink doCommit(DataStream written, String com if (options.get(SINK_AUTO_TAG_FOR_SAVEPOINT)) { committerOperator = - new AutoTagForSavepointCommitterOperator<>( - (CommitterOperator) committerOperator, + new AutoTagForSavepointCommitterOperatorFactory<>( + (CommitterOperatorFactory) + committerOperator, table::snapshotManager, table::tagManager, () -> table.store().newTagDeletion(), @@ -291,8 +291,9 @@ protected DataStreamSink doCommit(DataStream written, String com if (conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.BATCH && table.coreOptions().tagCreationMode() == TagCreationMode.BATCH) { committerOperator = - new BatchWriteGeneratorTagOperator<>( - (CommitterOperator) committerOperator, + new BatchWriteGeneratorTagOperatorFactory<>( + (CommitterOperatorFactory) + committerOperator, table); } SingleOutputStreamOperator committed = @@ -310,6 +311,9 @@ protected DataStreamSink doCommit(DataStream written, String com table.name(), options.get(SINK_OPERATOR_UID_SUFFIX))); } + if (!options.get(SINK_COMMITTER_OPERATOR_CHAINING)) { + committed = committed.startNewChain(); + } configureGlobalCommitter( committed, options.get(SINK_COMMITTER_CPU), options.get(SINK_COMMITTER_MEMORY)); return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); @@ -362,7 +366,7 @@ public static void assertBatchAdaptiveParallelism( } } - protected abstract OneInputStreamOperator createWriteOperator( + protected abstract OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser); protected abstract Committer.Factory createCommitterFactory(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java index dcccd0a1a988b..5703c408243bd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java @@ -222,7 +222,7 @@ public DataStreamSink build() { .transform( "local merge", input.getType(), - new LocalMergeOperator(table.schema())) + new LocalMergeOperator.Factory(table.schema())) .setParallelism(input.getParallelism()); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java index 6931fe9072180..070262147643f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java @@ -44,10 +44,15 @@ import org.apache.paimon.utils.UserDefinedSeqComparator; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -76,13 +81,14 @@ public class LocalMergeOperator extends AbstractStreamOperator private transient boolean endOfInput; - public LocalMergeOperator(TableSchema schema) { + private LocalMergeOperator( + StreamOperatorParameters parameters, TableSchema schema) { Preconditions.checkArgument( schema.primaryKeys().size() > 0, "LocalMergeOperator currently only support tables with primary keys"); this.schema = schema; this.ignoreDelete = CoreOptions.fromMap(schema.options()).ignoreDelete(); - setChainingStrategy(ChainingStrategy.ALWAYS); + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); } @Override @@ -235,4 +241,28 @@ LocalMerger merger() { void setOutput(Output> output) { this.output = output; } + + /** {@link StreamOperatorFactory} of {@link LocalMergeOperator}. */ + public static class Factory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final TableSchema schema; + + public Factory(TableSchema schema) { + this.chainingStrategy = ChainingStrategy.ALWAYS; + this.schema = schema; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) new LocalMergeOperator(parameters, schema); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return LocalMergeOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java index 57d2e8413cb5b..58f6a3834096c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java @@ -33,6 +33,9 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; @@ -76,7 +79,8 @@ public class MultiTablesStoreCompactOperator protected Map writes; protected String commitUser; - public MultiTablesStoreCompactOperator( + private MultiTablesStoreCompactOperator( + StreamOperatorParameters parameters, Catalog.Loader catalogLoader, String initialCommitUser, CheckpointConfig checkpointConfig, @@ -84,7 +88,7 @@ public MultiTablesStoreCompactOperator( boolean ignorePreviousFiles, boolean fullCompaction, Options options) { - super(options); + super(parameters, options); this.catalogLoader = catalogLoader; this.initialCommitUser = initialCommitUser; this.checkpointConfig = checkpointConfig; @@ -316,4 +320,54 @@ private StoreSinkWrite.Provider createWriteProvider( memoryPool, metricGroup); } + + /** {@link StreamOperatorFactory} of {@link MultiTablesStoreCompactOperator}. */ + public static class Factory + extends PrepareCommitOperator.Factory { + private final Catalog.Loader catalogLoader; + private final CheckpointConfig checkpointConfig; + private final boolean isStreaming; + private final boolean ignorePreviousFiles; + private final boolean fullCompaction; + private final String initialCommitUser; + + public Factory( + Catalog.Loader catalogLoader, + String initialCommitUser, + CheckpointConfig checkpointConfig, + boolean isStreaming, + boolean ignorePreviousFiles, + boolean fullCompaction, + Options options) { + super(options); + this.catalogLoader = catalogLoader; + this.initialCommitUser = initialCommitUser; + this.checkpointConfig = checkpointConfig; + this.isStreaming = isStreaming; + this.ignorePreviousFiles = ignorePreviousFiles; + this.fullCompaction = fullCompaction; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new MultiTablesStoreCompactOperator( + parameters, + catalogLoader, + initialCommitUser, + checkpointConfig, + isStreaming, + ignorePreviousFiles, + fullCompaction, + options); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return MultiTablesStoreCompactOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java index 3668386ddc2db..8b114d3e492ff 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java @@ -26,10 +26,14 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; @@ -52,9 +56,9 @@ public abstract class PrepareCommitOperator extends AbstractStreamOpera private final Options options; private boolean endOfInput = false; - public PrepareCommitOperator(Options options) { + public PrepareCommitOperator(StreamOperatorParameters parameters, Options options) { this.options = options; - setChainingStrategy(ChainingStrategy.ALWAYS); + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); } @Override @@ -103,4 +107,15 @@ private void emitCommittables(boolean waitCompaction, long checkpointId) throws protected abstract List prepareCommit(boolean waitCompaction, long checkpointId) throws IOException; + + /** {@link StreamOperatorFactory} of {@link PrepareCommitOperator}. */ + protected abstract static class Factory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + protected final Options options; + + protected Factory(Options options) { + this.options = options; + this.chainingStrategy = ChainingStrategy.ALWAYS; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java index 39dcca03c6aab..d9f863c6b9195 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java @@ -45,11 +45,10 @@ import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.Pair; -import org.apache.flink.streaming.api.graph.StreamConfig; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import javax.annotation.Nullable; @@ -76,34 +75,49 @@ public RewriteFileIndexSink(FileStoreTable table) { } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new FileIndexModificationOperator(table.coreOptions().toConfiguration(), table); + return new FileIndexModificationOperatorFactory( + table.coreOptions().toConfiguration(), table); } - /** File index modification operator to rewrite file index. */ - private static class FileIndexModificationOperator - extends PrepareCommitOperator { - - private static final long serialVersionUID = 1L; - + private static class FileIndexModificationOperatorFactory + extends PrepareCommitOperator.Factory { private final FileStoreTable table; - private transient FileIndexProcessor fileIndexProcessor; - private transient List messages; - - public FileIndexModificationOperator(Options options, FileStoreTable table) { + public FileIndexModificationOperatorFactory(Options options, FileStoreTable table) { super(options); this.table = table; } @Override - public void setup( - StreamTask containingTask, - StreamConfig config, - Output> output) { - super.setup(containingTask, config, output); + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) new FileIndexModificationOperator(parameters, options, table); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return FileIndexModificationOperator.class; + } + } + + /** File index modification operator to rewrite file index. */ + private static class FileIndexModificationOperator + extends PrepareCommitOperator { + + private static final long serialVersionUID = 1L; + + private final transient FileIndexProcessor fileIndexProcessor; + private final transient List messages; + private FileIndexModificationOperator( + StreamOperatorParameters parameters, + Options options, + FileStoreTable table) { + super(parameters, options); this.fileIndexProcessor = new FileIndexProcessor(table); this.messages = new ArrayList<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java index 2b25f074667c3..8009bec9677f7 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java @@ -32,13 +32,13 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.InternalTimerService; -import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.util.functions.StreamingFunctionUtils; import javax.annotation.Nullable; @@ -61,21 +61,14 @@ public class RowDataStoreWriteOperator extends TableWriteOperator { /** We listen to this ourselves because we don't have an {@link InternalTimerService}. */ private long currentWatermark = Long.MIN_VALUE; - public RowDataStoreWriteOperator( + protected RowDataStoreWriteOperator( + StreamOperatorParameters parameters, FileStoreTable table, @Nullable LogSinkFunction logSinkFunction, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); this.logSinkFunction = logSinkFunction; - } - - @Override - public void setup( - StreamTask containingTask, - StreamConfig config, - Output> output) { - super.setup(containingTask, config, output); if (logSinkFunction != null) { FunctionUtils.setFunctionRuntimeContext(logSinkFunction, getRuntimeContext()); } @@ -249,4 +242,38 @@ public Long timestamp() { return timestamp; } } + + /** {@link StreamOperatorFactory} of {@link RowDataStoreWriteOperator}. */ + public static class Factory extends TableWriteOperator.Factory { + + @Nullable private final LogSinkFunction logSinkFunction; + + public Factory( + FileStoreTable table, + @Nullable LogSinkFunction logSinkFunction, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + this.logSinkFunction = logSinkFunction; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new RowDataStoreWriteOperator( + parameters, + table, + logSinkFunction, + storeSinkWriteProvider, + initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return RowDataStoreWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java index bf6c70f0aa295..1f7e62d74916e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java @@ -27,7 +27,7 @@ import org.apache.paimon.utils.SerializableFunction; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -60,8 +60,8 @@ protected ChannelComputer> channelComputer2() { } @Override - protected OneInputStreamOperator, Committable> createWriteOperator( - StoreSinkWrite.Provider writeProvider, String commitUser) { - return new DynamicBucketRowWriteOperator(table, writeProvider, commitUser); + protected OneInputStreamOperatorFactory, Committable> + createWriteOperatorFactory(StoreSinkWrite.Provider writeProvider, String commitUser) { + return new DynamicBucketRowWriteOperator.Factory(table, writeProvider, commitUser); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java index 1cd10390c1a0a..fea8a382a954c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java @@ -22,7 +22,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import java.util.Map; @@ -38,25 +40,35 @@ public RowUnawareBucketSink( } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser) { - + return new RowDataStoreWriteOperator.Factory( + table, logSinkFunction, writeProvider, commitUser) { @Override - protected StoreSinkWriteState createState( - StateInitializationContext context, - StoreSinkWriteState.StateValueFilter stateFilter) - throws Exception { - // No conflicts will occur in append only unaware bucket writer, so no state is - // needed. - return new NoopStoreSinkWriteState(stateFilter); - } + public StreamOperator createStreamOperator(StreamOperatorParameters parameters) { + return new RowDataStoreWriteOperator( + parameters, table, logSinkFunction, writeProvider, commitUser) { - @Override - protected String getCommitUser(StateInitializationContext context) throws Exception { - // No conflicts will occur in append only unaware bucket writer, so commitUser does - // not matter. - return commitUser; + @Override + protected StoreSinkWriteState createState( + StateInitializationContext context, + StoreSinkWriteState.StateValueFilter stateFilter) + throws Exception { + // No conflicts will occur in append only unaware bucket writer, so no state + // is + // needed. + return new NoopStoreSinkWriteState(stateFilter); + } + + @Override + protected String getCommitUser(StateInitializationContext context) + throws Exception { + // No conflicts will occur in append only unaware bucket writer, so + // commitUser does + // not matter. + return commitUser; + } + }; } }; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java index ac10345bc4257..1870a0493c2f1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java @@ -31,6 +31,9 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; @@ -60,12 +63,13 @@ public class StoreCompactOperator extends PrepareCommitOperator> waitToCompact; - public StoreCompactOperator( + private StoreCompactOperator( + StreamOperatorParameters parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser, boolean fullCompaction) { - super(Options.fromMap(table.options())); + super(parameters, Options.fromMap(table.options())); Preconditions.checkArgument( !table.coreOptions().writeOnly(), CoreOptions.WRITE_ONLY.key() + " should not be true for StoreCompactOperator."); @@ -163,4 +167,46 @@ public void close() throws Exception { super.close(); write.close(); } + + /** {@link StreamOperatorFactory} of {@link StoreCompactOperator}. */ + public static class Factory extends PrepareCommitOperator.Factory { + private final FileStoreTable table; + private final StoreSinkWrite.Provider storeSinkWriteProvider; + private final String initialCommitUser; + private final boolean fullCompaction; + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser, + boolean fullCompaction) { + super(Options.fromMap(table.options())); + Preconditions.checkArgument( + !table.coreOptions().writeOnly(), + CoreOptions.WRITE_ONLY.key() + " should not be true for StoreCompactOperator."); + this.table = table; + this.storeSinkWriteProvider = storeSinkWriteProvider; + this.initialCommitUser = initialCommitUser; + this.fullCompaction = fullCompaction; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new StoreCompactOperator( + parameters, + table, + storeSinkWriteProvider, + initialCommitUser, + fullCompaction); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return StoreCompactOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java index 32fcdd03bdfd5..fd876698c094a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java @@ -28,6 +28,8 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import java.io.IOException; @@ -45,10 +47,11 @@ public abstract class TableWriteOperator extends PrepareCommitOperator parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(Options.fromMap(table.options())); + super(parameters, Options.fromMap(table.options())); this.table = table; this.storeSinkWriteProvider = storeSinkWriteProvider; this.initialCommitUser = initialCommitUser; @@ -128,4 +131,22 @@ protected List prepareCommit(boolean waitCompaction, long checkpoin public StoreSinkWrite getWrite() { return write; } + + /** {@link StreamOperatorFactory} of {@link TableWriteOperator}. */ + protected abstract static class Factory + extends PrepareCommitOperator.Factory { + protected final FileStoreTable table; + protected final StoreSinkWrite.Provider storeSinkWriteProvider; + protected final String initialCommitUser; + + protected Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(Options.fromMap(table.options())); + this.table = table; + this.storeSinkWriteProvider = storeSinkWriteProvider; + this.initialCommitUser = initialCommitUser; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java index da966d5e51566..7a4095f896cc3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java @@ -24,7 +24,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; /** Compaction Sink for unaware-bucket table. */ public class UnawareBucketCompactionSink extends FlinkSink { @@ -42,9 +42,9 @@ public static DataStreamSink sink( } @Override - protected OneInputStreamOperator createWriteOperator( - StoreSinkWrite.Provider writeProvider, String commitUser) { - return new AppendOnlySingleTableCompactionWorkerOperator(table, commitUser); + protected OneInputStreamOperatorFactory + createWriteOperatorFactory(StoreSinkWrite.Provider writeProvider, String commitUser) { + return new AppendOnlySingleTableCompactionWorkerOperator.Factory(table, commitUser); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java index 98b58aa8e96d3..7bc40d4c2080b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java @@ -74,11 +74,14 @@ public DataStream doWrite( new CommittableTypeInfo(), new CompactionTaskTypeInfo()), new AppendBypassCoordinateOperatorFactory<>(table)) + .startNewChain() .forceNonParallel() .transform( "Compact Worker: " + table.name(), new CommittableTypeInfo(), - new AppendBypassCompactWorkerOperator(table, initialCommitUser)) + new AppendBypassCompactWorkerOperator.Factory( + table, initialCommitUser)) + .startNewChain() .setParallelism(written.getParallelism()); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java index 26e080c32e834..7022002a43ba3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java @@ -39,7 +39,7 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -63,9 +63,9 @@ public GlobalDynamicBucketSink( } @Override - protected OneInputStreamOperator, Committable> createWriteOperator( - StoreSinkWrite.Provider writeProvider, String commitUser) { - return new DynamicBucketRowWriteOperator(table, writeProvider, commitUser); + protected OneInputStreamOperatorFactory, Committable> + createWriteOperatorFactory(StoreSinkWrite.Provider writeProvider, String commitUser) { + return new DynamicBucketRowWriteOperator.Factory(table, writeProvider, commitUser); } public DataStreamSink build(DataStream input, @Nullable Integer parallelism) { @@ -89,7 +89,8 @@ public DataStreamSink build(DataStream input, @Nullable Integer new InternalTypeInfo<>( new KeyWithRowSerializer<>( bootstrapSerializer, rowSerializer)), - new IndexBootstrapOperator<>(new IndexBootstrap(table), r -> r)) + new IndexBootstrapOperator.Factory<>( + new IndexBootstrap(table), r -> r)) .setParallelism(input.getParallelism()); // 1. shuffle by key hash diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java index 5c8ba8f9441f3..8136565f98cf8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java @@ -27,8 +27,13 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** Operator for {@link IndexBootstrap}. */ @@ -40,11 +45,13 @@ public class IndexBootstrapOperator extends AbstractStreamOperator converter; - public IndexBootstrapOperator( - IndexBootstrap bootstrap, SerializableFunction converter) { + private IndexBootstrapOperator( + StreamOperatorParameters> parameters, + IndexBootstrap bootstrap, + SerializableFunction converter) { this.bootstrap = bootstrap; this.converter = converter; - setChainingStrategy(ChainingStrategy.ALWAYS); + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); } @Override @@ -65,4 +72,30 @@ private void collect(InternalRow row) { output.collect( new StreamRecord<>(new Tuple2<>(KeyPartOrRow.KEY_PART, converter.apply(row)))); } + + /** {@link StreamOperatorFactory} of {@link IndexBootstrapOperator}. */ + public static class Factory extends AbstractStreamOperatorFactory> + implements OneInputStreamOperatorFactory> { + private final IndexBootstrap bootstrap; + private final SerializableFunction converter; + + public Factory(IndexBootstrap bootstrap, SerializableFunction converter) { + this.chainingStrategy = ChainingStrategy.ALWAYS; + this.bootstrap = bootstrap; + this.converter = converter; + } + + @Override + @SuppressWarnings("unchecked") + public >> OP createStreamOperator( + StreamOperatorParameters> parameters) { + return (OP) new IndexBootstrapOperator<>(parameters, bootstrap, converter); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return IndexBootstrapOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java index 45090f7b68b40..b8b0d61e10a96 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java @@ -26,8 +26,8 @@ import org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.types.Either; @@ -58,10 +58,12 @@ public class AppendBypassCoordinateOperator private transient LinkedBlockingQueue compactTasks; public AppendBypassCoordinateOperator( - FileStoreTable table, ProcessingTimeService processingTimeService) { + StreamOperatorParameters> parameters, + FileStoreTable table, + ProcessingTimeService processingTimeService) { this.table = table; this.processingTimeService = processingTimeService; - this.chainingStrategy = ChainingStrategy.HEAD; + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java index 7c53e01b47e65..a4c51e5b5a9be 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java @@ -45,11 +45,7 @@ T createStreamOperator( StreamOperatorParameters> parameters) { AppendBypassCoordinateOperator operator = - new AppendBypassCoordinateOperator<>(table, processingTimeService); - operator.setup( - parameters.getContainingTask(), - parameters.getStreamConfig(), - parameters.getOutput()); + new AppendBypassCoordinateOperator<>(parameters, table, processingTimeService); return (T) operator; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java index d589459d9b969..949c2c7a66a36 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java @@ -25,7 +25,13 @@ import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -45,8 +51,17 @@ public class AppendOnlyMultiTableCompactionWorkerOperatorTest extends TableTestB public void testAsyncCompactionWorks() throws Exception { AppendOnlyMultiTableCompactionWorkerOperator workerOperator = - new AppendOnlyMultiTableCompactionWorkerOperator( - () -> catalog, "user", new Options()); + new AppendOnlyMultiTableCompactionWorkerOperator.Factory( + () -> catalog, "user", new Options()) + .createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask( + new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); List> records = new ArrayList<>(); // create table and write diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java index d04032817cf01..6238a9cbf3ea0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java @@ -32,7 +32,13 @@ import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.types.DataTypes; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -49,7 +55,16 @@ public class AppendOnlySingleTableCompactionWorkerOperatorTest extends TableTest public void testAsyncCompactionWorks() throws Exception { createTableDefault(); AppendOnlySingleTableCompactionWorkerOperator workerOperator = - new AppendOnlySingleTableCompactionWorkerOperator(getTableDefault(), "user"); + new AppendOnlySingleTableCompactionWorkerOperator.Factory(getTableDefault(), "user") + .createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask( + new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); // write 200 files List commitMessages = writeDataDefault(200, 20); @@ -102,7 +117,16 @@ public void testAsyncCompactionWorks() throws Exception { public void testAsyncCompactionFileDeletedWhenShutdown() throws Exception { createTableDefault(); AppendOnlySingleTableCompactionWorkerOperator workerOperator = - new AppendOnlySingleTableCompactionWorkerOperator(getTableDefault(), "user"); + new AppendOnlySingleTableCompactionWorkerOperator.Factory(getTableDefault(), "user") + .createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask( + new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); // write 200 files List commitMessages = writeDataDefault(200, 40); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java index 3b58c24d16b18..ee930a06fc3d0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.SavepointType; import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.jupiter.api.Test; @@ -198,13 +198,15 @@ private void processCommittable( } @Override - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager) { - return new AutoTagForSavepointCommitterOperator<>( - (CommitterOperator) - super.createCommitterOperator(table, commitUser, committableStateManager), + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager) { + return new AutoTagForSavepointCommitterOperatorFactory<>( + (CommitterOperatorFactory) + super.createCommitterOperatorFactory( + table, commitUser, committableStateManager), table::snapshotManager, table::tagManager, () -> table.store().newTagDeletion(), @@ -213,14 +215,15 @@ protected OneInputStreamOperator createCommitterOperat } @Override - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager, - ThrowingConsumer initializeFunction) { - return new AutoTagForSavepointCommitterOperator<>( - (CommitterOperator) - super.createCommitterOperator( + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager, + ThrowingConsumer initializeFunction) { + return new AutoTagForSavepointCommitterOperatorFactory<>( + (CommitterOperatorFactory) + super.createCommitterOperatorFactory( table, commitUser, committableStateManager, initializeFunction), table::snapshotManager, table::tagManager, diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java index 147110637aef6..68162832eac9c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java @@ -27,13 +27,21 @@ import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.TagManager; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.junit.jupiter.api.Test; import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; import java.time.format.DateTimeFormatter; +import java.util.ArrayList; import java.util.HashMap; import java.util.Objects; @@ -54,12 +62,23 @@ public void testBatchWriteGeneratorTag() throws Exception { StreamTableWrite write = table.newStreamWriteBuilder().withCommitUser(initialCommitUser).newWrite(); - OneInputStreamOperator committerOperator = - createCommitterOperator( + OneInputStreamOperatorFactory committerOperatorFactory = + createCommitterOperatorFactory( table, initialCommitUser, new RestoreAndFailCommittableStateManager<>( ManifestCommittableSerializer::new)); + + OneInputStreamOperator committerOperator = + committerOperatorFactory.createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask(new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); + committerOperator.open(); TableCommitImpl tableCommit = table.newCommit(initialCommitUser); @@ -106,13 +125,15 @@ public void testBatchWriteGeneratorTag() throws Exception { } @Override - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager) { - return new BatchWriteGeneratorTagOperator<>( - (CommitterOperator) - super.createCommitterOperator(table, commitUser, committableStateManager), + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager) { + return new BatchWriteGeneratorTagOperatorFactory<>( + (CommitterOperatorFactory) + super.createCommitterOperatorFactory( + table, commitUser, committableStateManager), table); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java index 668d651236fdc..28c93ca79be02 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java @@ -51,10 +51,13 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.util.Preconditions; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -259,8 +262,8 @@ public void testRestoreCommitUser() throws Exception { // 3. Check whether success List actual = new ArrayList<>(); - OneInputStreamOperator operator = - createCommitterOperator( + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( table, initialCommitUser, new NoopCommittableStateManager(), @@ -274,7 +277,7 @@ public void testRestoreCommitUser() throws Exception { }); OneInputStreamOperatorTestHarness testHarness1 = - createTestHarness(operator); + createTestHarness(operatorFactory); testHarness1.initializeState(snapshot); testHarness1.close(); @@ -315,10 +318,11 @@ public void testRestoreEmptyMarkDoneState() throws Exception { public void testCommitInputEnd() throws Exception { FileStoreTable table = createFileStoreTable(); String commitUser = UUID.randomUUID().toString(); - OneInputStreamOperator operator = - createCommitterOperator(table, commitUser, new NoopCommittableStateManager()); + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( + table, commitUser, new NoopCommittableStateManager()); OneInputStreamOperatorTestHarness testHarness = - createTestHarness(operator); + createTestHarness(operatorFactory); testHarness.open(); Assertions.assertThatCode( () -> { @@ -378,10 +382,10 @@ public void testCommitInputEnd() throws Exception { }) .doesNotThrowAnyException(); - if (operator instanceof CommitterOperator) { + if (operatorFactory instanceof CommitterOperator) { Assertions.assertThat( ((ManifestCommittable) - ((CommitterOperator) operator) + ((CommitterOperator) operatorFactory) .committablesPerCheckpoint.get(Long.MAX_VALUE)) .fileCommittables() .size()) @@ -604,14 +608,14 @@ public void testCalcDataBytesSend() throws Exception { public void testCommitMetrics() throws Exception { FileStoreTable table = createFileStoreTable(); - OneInputStreamOperator operator = - createCommitterOperator( + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( table, null, new RestoreAndFailCommittableStateManager<>( ManifestCommittableSerializer::new)); OneInputStreamOperatorTestHarness testHarness = - createTestHarness(operator); + createTestHarness(operatorFactory); testHarness.open(); long timestamp = 0; StreamTableWrite write = @@ -627,7 +631,9 @@ public void testCommitMetrics() throws Exception { testHarness.notifyOfCompletedCheckpoint(cpId); MetricGroup commitMetricGroup = - operator.getMetricGroup() + testHarness + .getOneInputOperator() + .getMetricGroup() .addGroup("paimon") .addGroup("table", table.name()) .addGroup("commit"); @@ -685,10 +691,11 @@ public void testCommitMetrics() throws Exception { public void testParallelism() throws Exception { FileStoreTable table = createFileStoreTable(); String commitUser = UUID.randomUUID().toString(); - OneInputStreamOperator operator = - createCommitterOperator(table, commitUser, new NoopCommittableStateManager()); + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( + table, commitUser, new NoopCommittableStateManager()); try (OneInputStreamOperatorTestHarness testHarness = - createTestHarness(operator, 10, 10, 3)) { + createTestHarness(operatorFactory, 10, 10, 3)) { Assertions.assertThatCode(testHarness::open) .hasMessage("Committer Operator parallelism in paimon MUST be one."); } @@ -700,13 +707,13 @@ public void testParallelism() throws Exception { protected OneInputStreamOperatorTestHarness createRecoverableTestHarness(FileStoreTable table) throws Exception { - OneInputStreamOperator operator = - createCommitterOperator( + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( table, null, new RestoreAndFailCommittableStateManager<>( ManifestCommittableSerializer::new)); - return createTestHarness(operator); + return createTestHarness(operatorFactory); } private OneInputStreamOperatorTestHarness createLossyTestHarness( @@ -716,18 +723,20 @@ private OneInputStreamOperatorTestHarness createLossyT private OneInputStreamOperatorTestHarness createLossyTestHarness( FileStoreTable table, String commitUser) throws Exception { - OneInputStreamOperator operator = - createCommitterOperator(table, commitUser, new NoopCommittableStateManager()); - return createTestHarness(operator); + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( + table, commitUser, new NoopCommittableStateManager()); + return createTestHarness(operatorFactory); } private OneInputStreamOperatorTestHarness createTestHarness( - OneInputStreamOperator operator) throws Exception { - return createTestHarness(operator, 1, 1, 0); + OneInputStreamOperatorFactory operatorFactory) + throws Exception { + return createTestHarness(operatorFactory, 1, 1, 0); } private OneInputStreamOperatorTestHarness createTestHarness( - OneInputStreamOperator operator, + OneInputStreamOperatorFactory operatorFactory, int maxParallelism, int parallelism, int subTaskIndex) @@ -736,22 +745,23 @@ private OneInputStreamOperatorTestHarness createTestHa new CommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>( - operator, + operatorFactory, maxParallelism, parallelism, subTaskIndex, - serializer, new OperatorID()); + harness.getStreamConfig().setupNetworkInputs(Preconditions.checkNotNull(serializer)); + harness.getStreamConfig().serializeAllConfigs(); harness.setup(serializer); return harness; } - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager) { - return new CommitterOperator<>( - true, + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager) { + return new CommitterOperatorFactory<>( true, true, commitUser == null ? initialCommitUser : commitUser, @@ -765,13 +775,13 @@ protected OneInputStreamOperator createCommitterOperat committableStateManager); } - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager, - ThrowingConsumer initializeFunction) { - return new CommitterOperator( - true, + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager, + ThrowingConsumer initializeFunction) { + return new CommitterOperatorFactory( true, true, commitUser == null ? initialCommitUser : commitUser, @@ -784,8 +794,24 @@ protected OneInputStreamOperator createCommitterOperat context), committableStateManager) { @Override - public void initializeState(StateInitializationContext context) throws Exception { - initializeFunction.accept(context); + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CommitterOperator( + parameters, + streamingCheckpointEnabled, + forceSingleParallelism, + initialCommitUser, + committerFactory, + committableStateManager, + endInputWatermark) { + @Override + public void initializeState(StateInitializationContext context) + throws Exception { + initializeFunction.accept(context); + } + }; } }; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java index 42293ca2842e1..d487d75925eb9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java @@ -254,8 +254,8 @@ private OneInputStreamOperatorTestHarness createTestHarnes return harness; } - protected StoreCompactOperator createCompactOperator(FileStoreTable table) { - return new StoreCompactOperator( + protected StoreCompactOperator.Factory createCompactOperator(FileStoreTable table) { + return new StoreCompactOperator.Factory( table, (t, commitUser, state, ioManager, memoryPool, metricGroup) -> new StoreSinkWriteImpl( @@ -272,9 +272,9 @@ protected StoreCompactOperator createCompactOperator(FileStoreTable table) { true); } - protected MultiTablesStoreCompactOperator createMultiTablesCompactOperator( + protected MultiTablesStoreCompactOperator.Factory createMultiTablesCompactOperator( Catalog.Loader catalogLoader) throws Exception { - return new MultiTablesStoreCompactOperator( + return new MultiTablesStoreCompactOperator.Factory( catalogLoader, commitUser, new CheckpointConfig(), diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java index c335568344b32..5f21858e61a58 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java @@ -42,7 +42,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.jupiter.api.Test; @@ -82,20 +82,22 @@ private boolean testSpillable( Collections.singletonList(GenericRow.of(1, 1))); FlinkSink flinkSink = new FixedBucketSink(fileStoreTable, null, null); DataStream written = flinkSink.doWrite(source, "123", 1); - RowDataStoreWriteOperator operator = - ((RowDataStoreWriteOperator) - ((SimpleOperatorFactory) - ((OneInputTransformation) written.getTransformation()) - .getOperatorFactory()) - .getOperator()); + OneInputStreamOperatorFactory operatorFactory = + (OneInputStreamOperatorFactory) + ((OneInputTransformation) + written.getTransformation()) + .getOperatorFactory(); TypeSerializer serializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator); + new OneInputStreamOperatorTestHarness<>(operatorFactory); harness.setup(serializer); harness.initializeEmptyState(); + RowDataStoreWriteOperator operator = + (RowDataStoreWriteOperator) harness.getOneInputOperator(); + return ((KeyValueFileStoreWrite) ((StoreSinkWriteImpl) operator.write).write.getWrite()) .bufferSpillable(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LocalMergeOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LocalMergeOperatorTest.java index 1162e20b155b2..fc45eceb3fd5b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LocalMergeOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LocalMergeOperatorTest.java @@ -26,12 +26,18 @@ import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.apache.flink.util.OutputTag; import org.junit.jupiter.api.Test; @@ -151,7 +157,17 @@ private void prepareHashOperator(Map options) throws Exception { Collections.singletonList("f0"), options, null); - operator = new LocalMergeOperator(schema); + operator = + new LocalMergeOperator.Factory(schema) + .createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask( + new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); operator.open(); assertThat(operator.merger()).isInstanceOf(HashMapLocalMerger.class); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java index f8387e1fc41a0..3740033e025e9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java @@ -48,8 +48,8 @@ public void testCompactExactlyOnce(boolean streamingMode) throws Exception { CompactRememberStoreWrite compactRememberStoreWrite = new CompactRememberStoreWrite(streamingMode); - StoreCompactOperator operator = - new StoreCompactOperator( + StoreCompactOperator.Factory operatorFactory = + new StoreCompactOperator.Factory( getTableDefault(), (table, commitUser, state, ioManager, memoryPool, metricGroup) -> compactRememberStoreWrite, @@ -59,7 +59,7 @@ public void testCompactExactlyOnce(boolean streamingMode) throws Exception { TypeSerializer serializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator); + new OneInputStreamOperatorTestHarness<>(operatorFactory); harness.setup(serializer); harness.initializeEmptyState(); harness.open(); @@ -70,7 +70,7 @@ public void testCompactExactlyOnce(boolean streamingMode) throws Exception { harness.processElement(new StreamRecord<>(data(1))); harness.processElement(new StreamRecord<>(data(2))); - operator.prepareCommit(true, 1); + ((StoreCompactOperator) harness.getOneInputOperator()).prepareCommit(true, 1); Assertions.assertThat(compactRememberStoreWrite.compactTime).isEqualTo(3); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java index 10e432f3c8c21..752679fb59037 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java @@ -645,11 +645,10 @@ public void testCommitMetrics() throws Exception { private OneInputStreamOperatorTestHarness createRecoverableTestHarness() throws Exception { - CommitterOperator operator = - new CommitterOperator<>( + CommitterOperatorFactory operator = + new CommitterOperatorFactory<>( true, false, - true, initialCommitUser, context -> new StoreMultiCommitter(catalogLoader, context), new RestoreAndFailCommittableStateManager<>( @@ -659,11 +658,10 @@ public void testCommitMetrics() throws Exception { private OneInputStreamOperatorTestHarness createLossyTestHarness() throws Exception { - CommitterOperator operator = - new CommitterOperator<>( + CommitterOperatorFactory operator = + new CommitterOperatorFactory<>( true, false, - true, initialCommitUser, context -> new StoreMultiCommitter(catalogLoader, context), new CommittableStateManager() { @@ -682,12 +680,13 @@ public void snapshotState( private OneInputStreamOperatorTestHarness createTestHarness( - CommitterOperator operator) + CommitterOperatorFactory + operatorFactory) throws Exception { TypeSerializer serializer = new MultiTableCommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator, serializer); + new OneInputStreamOperatorTestHarness<>(operatorFactory, serializer); harness.setup(serializer); return harness; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java new file mode 100644 index 0000000000000..a4605b830918e --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java @@ -0,0 +1,203 @@ +/* + * 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.paimon.flink.sink; + +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.CompiledPlan; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.CompiledPlanUtils; +import org.apache.flink.util.TimeUtils; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Tests for {@link org.apache.flink.streaming.api.operators.ChainingStrategy} of writer operators. + */ +public class WriterChainingStrategyTest { + private static final String TABLE_NAME = "paimon_table"; + + @TempDir java.nio.file.Path tempDir; + + private StreamTableEnvironment tEnv; + + @BeforeEach + public void beforeEach() { + Configuration config = new Configuration(); + config.setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit(Duration.ofMillis(500))); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + tEnv = StreamTableEnvironment.create(env); + + String catalog = "PAIMON"; + Map options = new HashMap<>(); + options.put("type", "paimon"); + options.put("warehouse", tempDir.toString()); + tEnv.executeSql( + String.format( + "CREATE CATALOG %s WITH ( %s )", + catalog, + options.entrySet().stream() + .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) + .collect(Collectors.joining(",")))); + tEnv.useCatalog(catalog); + } + + @Test + public void testAppendTable() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING) " + + "WITH ('bucket' = '1', 'bucket-key'='id', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + verifyChaining(false, true); + } + + @Test + public void testAppendTableWithUnawareBucket() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING) " + + "WITH ('bucket' = '-1', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + verifyChaining(true, true); + } + + @Test + public void testPrimaryKeyTable() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "WITH ('bucket' = '1', 'bucket-key'='id', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + verifyChaining(false, true); + } + + @Test + public void testPrimaryKeyTableWithDynamicBucket() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "WITH ('bucket' = '-1', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + verifyChaining(false, true); + } + + @Test + public void testPrimaryKeyTableWithMultipleWriter() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "WITH ('bucket' = '1', 'bucket-key'='id', 'write-only' = 'true', 'sink.parallelism' = '2')", + TABLE_NAME)) + .await(); + + verifyChaining(false, false); + } + + @Test + public void testPrimaryKeyTableWithCrossPartitionUpdate() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "PARTITIONED BY ( dt ) WITH ('bucket' = '-1', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + List vertices = verifyChaining(false, true); + JobVertex vertex = findVertex(vertices, "INDEX_BOOTSTRAP"); + assertThat(vertex.toString()).contains("Source"); + } + + @Test + public void testPrimaryKeyTableWithLocalMerge() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "WITH ('bucket' = '-1', 'write-only' = 'true', 'local-merge-buffer-size' = '1MB')", + TABLE_NAME)) + .await(); + + List vertices = verifyChaining(false, true); + JobVertex vertex = findVertex(vertices, "local merge"); + assertThat(vertex.toString()).contains("Source"); + } + + private List verifyChaining( + boolean isWriterChainedWithUpstream, boolean isWriterChainedWithDownStream) { + CompiledPlan plan = + tEnv.compilePlanSql( + String.format( + "INSERT INTO %s VALUES (1, 'AAA', ''), (2, 'BBB', '')", + TABLE_NAME)); + List> transformations = CompiledPlanUtils.toTransformations(tEnv, plan); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + transformations.forEach(env::addOperator); + + List vertices = new ArrayList<>(); + env.getStreamGraph().getJobGraph().getVertices().forEach(vertices::add); + JobVertex vertex = findVertex(vertices, "Writer"); + + if (isWriterChainedWithUpstream) { + assertThat(vertex.toString()).contains("Source"); + } else { + assertThat(vertex.toString()).doesNotContain("Source"); + } + + if (isWriterChainedWithDownStream) { + assertThat(vertex.toString()).contains("Committer"); + } else { + assertThat(vertex.toString()).doesNotContain("Committer"); + } + + return vertices; + } + + private JobVertex findVertex(List vertices, String key) { + for (JobVertex vertex : vertices) { + if (vertex.toString().contains(key)) { + return vertex; + } + } + throw new IllegalStateException( + String.format( + "Cannot find vertex with keyword %s among job vertices %s", key, vertices)); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java index 3a8c1557122f7..83af157450780 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java @@ -115,9 +115,10 @@ public void testAppendOnlyTableMetrics() throws Exception { private void testMetricsImpl(FileStoreTable fileStoreTable) throws Exception { String tableName = tablePath.getName(); - RowDataStoreWriteOperator operator = getStoreSinkWriteOperator(fileStoreTable); + RowDataStoreWriteOperator.Factory operatorFactory = + getStoreSinkWriteOperatorFactory(fileStoreTable); OneInputStreamOperatorTestHarness harness = - createHarness(operator); + createHarness(operatorFactory); TypeSerializer serializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); @@ -133,7 +134,7 @@ private void testMetricsImpl(FileStoreTable fileStoreTable) throws Exception { harness.snapshot(1, 2); harness.notifyOfCompletedCheckpoint(1); - OperatorMetricGroup metricGroup = operator.getMetricGroup(); + OperatorMetricGroup metricGroup = harness.getOneInputOperator().getMetricGroup(); MetricGroup writerBufferMetricGroup = metricGroup .addGroup("paimon") @@ -173,9 +174,10 @@ public void testAsyncLookupWithFailure() throws Exception { rowType, Arrays.asList("pt", "k"), Collections.singletonList("k"), options); // we don't wait for compaction because this is async lookup test - RowDataStoreWriteOperator operator = getAsyncLookupWriteOperator(fileStoreTable, false); + RowDataStoreWriteOperator.Factory operatorFactory = + getAsyncLookupWriteOperatorFactory(fileStoreTable, false); OneInputStreamOperatorTestHarness harness = - createHarness(operator); + createHarness(operatorFactory); TableCommitImpl commit = fileStoreTable.newCommit(commitUser); @@ -205,8 +207,8 @@ public void testAsyncLookupWithFailure() throws Exception { harness.close(); // re-create operator from state, this time wait for compaction to check result - operator = getAsyncLookupWriteOperator(fileStoreTable, true); - harness = createHarness(operator); + operatorFactory = getAsyncLookupWriteOperatorFactory(fileStoreTable, true); + harness = createHarness(operatorFactory); harness.setup(serializer); harness.initializeState(state); harness.open(); @@ -263,9 +265,10 @@ private void testChangelog(boolean insertOnly) throws Exception { FileStoreTable fileStoreTable = createFileStoreTable( rowType, Arrays.asList("pt", "k"), Collections.singletonList("k"), options); - RowDataStoreWriteOperator operator = getStoreSinkWriteOperator(fileStoreTable); + RowDataStoreWriteOperator.Factory operatorFactory = + getStoreSinkWriteOperatorFactory(fileStoreTable); OneInputStreamOperatorTestHarness harness = - createHarness(operator); + createHarness(operatorFactory); TableCommitImpl commit = fileStoreTable.newCommit(commitUser); @@ -277,7 +280,7 @@ private void testChangelog(boolean insertOnly) throws Exception { if (insertOnly) { Field field = TableWriteOperator.class.getDeclaredField("write"); field.setAccessible(true); - StoreSinkWrite write = (StoreSinkWrite) field.get(operator); + StoreSinkWrite write = (StoreSinkWrite) field.get(harness.getOneInputOperator()); write.withInsertOnly(true); } @@ -339,17 +342,17 @@ public void testNumWritersMetric() throws Exception { options); TableCommitImpl commit = fileStoreTable.newCommit(commitUser); - RowDataStoreWriteOperator rowDataStoreWriteOperator = - getStoreSinkWriteOperator(fileStoreTable); + RowDataStoreWriteOperator.Factory operatorFactory = + getStoreSinkWriteOperatorFactory(fileStoreTable); OneInputStreamOperatorTestHarness harness = - createHarness(rowDataStoreWriteOperator); + createHarness(operatorFactory); TypeSerializer serializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); harness.setup(serializer); harness.open(); - OperatorMetricGroup metricGroup = rowDataStoreWriteOperator.getMetricGroup(); + OperatorMetricGroup metricGroup = harness.getOneInputOperator().getMetricGroup(); MetricGroup writerBufferMetricGroup = metricGroup .addGroup("paimon") @@ -408,8 +411,9 @@ public void testNumWritersMetric() throws Exception { // Test utils // ------------------------------------------------------------------------ - private RowDataStoreWriteOperator getStoreSinkWriteOperator(FileStoreTable fileStoreTable) { - return new RowDataStoreWriteOperator( + private RowDataStoreWriteOperator.Factory getStoreSinkWriteOperatorFactory( + FileStoreTable fileStoreTable) { + return new RowDataStoreWriteOperator.Factory( fileStoreTable, null, (table, commitUser, state, ioManager, memoryPool, metricGroup) -> @@ -426,9 +430,9 @@ private RowDataStoreWriteOperator getStoreSinkWriteOperator(FileStoreTable fileS commitUser); } - private RowDataStoreWriteOperator getAsyncLookupWriteOperator( + private RowDataStoreWriteOperator.Factory getAsyncLookupWriteOperatorFactory( FileStoreTable fileStoreTable, boolean waitCompaction) { - return new RowDataStoreWriteOperator( + return new RowDataStoreWriteOperator.Factory( fileStoreTable, null, (table, commitUser, state, ioManager, memoryPool, metricGroup) -> @@ -471,10 +475,11 @@ private FileStoreTable createFileStoreTable( } private OneInputStreamOperatorTestHarness createHarness( - RowDataStoreWriteOperator operator) throws Exception { + RowDataStoreWriteOperator.Factory operatorFactory) throws Exception { InternalTypeInfo internalRowInternalTypeInfo = new InternalTypeInfo<>(new InternalRowTypeSerializer(RowType.builder().build())); return new OneInputStreamOperatorTestHarness<>( - operator, internalRowInternalTypeInfo.createSerializer(new ExecutionConfig())); + operatorFactory, + internalRowInternalTypeInfo.createSerializer(new ExecutionConfig())); } } From db37d46f835c4a591492409483b5027ea395ba64 Mon Sep 17 00:00:00 2001 From: herefree <841043203@qq.com> Date: Sun, 1 Dec 2024 19:39:28 +0800 Subject: [PATCH 084/157] [rest] change JsonProperty import path (#4609) --- .../java/org/apache/paimon/rest/responses/ConfigResponse.java | 2 +- .../java/org/apache/paimon/rest/responses/ErrorResponse.java | 2 +- .../src/test/java/org/apache/paimon/rest/MockRESTData.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java index e6bc934703642..903cfc84b46d8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java @@ -21,11 +21,11 @@ import org.apache.paimon.rest.RESTResponse; import org.apache.paimon.utils.Preconditions; -import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.beans.ConstructorProperties; import java.util.Map; diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java index 0e4b234867329..685fe53071b66 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -18,8 +18,8 @@ package org.apache.paimon.rest.responses; -import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.beans.ConstructorProperties; import java.io.PrintWriter; diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java index 55c5165ada48e..9b7f1003e76fd 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java @@ -18,9 +18,9 @@ package org.apache.paimon.rest; -import org.apache.paimon.shade.com.fasterxml.jackson.annotation.JsonProperty; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.beans.ConstructorProperties; From e72c06ce489e54272f02dbaa555a5c31d7c36c40 Mon Sep 17 00:00:00 2001 From: Steven <54518670+yangjf2019@users.noreply.github.com> Date: Sun, 1 Dec 2024 19:40:14 +0800 Subject: [PATCH 085/157] [doc] fix url links in documentation (#4610) --- docs/content/concepts/table-types.md | 2 +- docs/content/engines/doris.md | 4 ++-- docs/content/engines/starrocks.md | 2 +- docs/content/primary-key-table/overview.md | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/content/concepts/table-types.md b/docs/content/concepts/table-types.md index 58199031b6e7d..b5a1fafa3da1f 100644 --- a/docs/content/concepts/table-types.md +++ b/docs/content/concepts/table-types.md @@ -33,7 +33,7 @@ Paimon supports table types: 3. view: metastore required, views in SQL are a kind of virtual table 4. format-table: file format table refers to a directory that contains multiple files of the same format, where operations on this table allow for reading or writing to these files, compatible with Hive tables -5. object table: provides metadata indexes for unstructured data objects in the specified Object Storage storage directory. +5. object table: provides metadata indexes for unstructured data objects in the specified Object Storage directory. 6. materialized-table: aimed at simplifying both batch and stream data pipelines, providing a consistent development experience, see [Flink Materialized Table](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/materialized-table/overview/) diff --git a/docs/content/engines/doris.md b/docs/content/engines/doris.md index cd778cd577979..6d22bc376a885 100644 --- a/docs/content/engines/doris.md +++ b/docs/content/engines/doris.md @@ -89,11 +89,11 @@ See [Apache Doris Website](https://doris.apache.org/docs/lakehouse/datalake-anal - Read optimized for Primary Key Table - Doris can utilize the [Read optimized](https://paimon.apache.org/releases/release-0.6/#read-optimized) feature for Primary Key Table(release in Paimon 0.6), by reading base data files using native Parquet/ORC reader and delta file using JNI. + Doris can utilize the [Read optimized](https://paimon.apache.org/docs/0.8/primary-key-table/read-optimized/) feature for Primary Key Table(release in Paimon 0.6), by reading base data files using native Parquet/ORC reader and delta file using JNI. - Deletion Vectors - Doris(2.1.4+) natively supports [Deletion Vectors](https://paimon.apache.org/releases/release-0.8/#deletion-vectors)(released in Paimon 0.8). + Doris(2.1.4+) natively supports [Deletion Vectors](https://paimon.apache.org/docs/0.8/primary-key-table/deletion-vectors/)(released in Paimon 0.8). ## Doris to Paimon type mapping diff --git a/docs/content/engines/starrocks.md b/docs/content/engines/starrocks.md index 1ab821a9a1033..dda22d35f76a3 100644 --- a/docs/content/engines/starrocks.md +++ b/docs/content/engines/starrocks.md @@ -81,7 +81,7 @@ SELECT * FROM paimon_catalog.test_db.partition_tbl$partitions; ## StarRocks to Paimon type mapping This section lists all supported type conversion between StarRocks and Paimon. -All StarRocks’s data types can be found in this doc [StarRocks Data type overview](https://docs.starrocks.io/docs/sql-reference/data-types/data-type-list/). +All StarRocks’s data types can be found in this doc [StarRocks Data type overview](https://docs.starrocks.io/docs/sql-reference/data-types/). diff --git a/docs/content/primary-key-table/overview.md b/docs/content/primary-key-table/overview.md index 508b990abc596..552d60eff6deb 100644 --- a/docs/content/primary-key-table/overview.md +++ b/docs/content/primary-key-table/overview.md @@ -56,6 +56,6 @@ Records within a data file are sorted by their primary keys. Within a sorted run {{< img src="/img/sorted-runs.png">}} -As you can see, different sorted runs may have overlapping primary key ranges, and may even contain the same primary key. When querying the LSM tree, all sorted runs must be combined and all records with the same primary key must be merged according to the user-specified [merge engine]({{< ref "primary-key-table/merge-engine" >}}) and the timestamp of each record. +As you can see, different sorted runs may have overlapping primary key ranges, and may even contain the same primary key. When querying the LSM tree, all sorted runs must be combined and all records with the same primary key must be merged according to the user-specified [merge engine]({{< ref "primary-key-table/merge-engine/overview" >}}) and the timestamp of each record. New records written into the LSM tree will be first buffered in memory. When the memory buffer is full, all records in memory will be sorted and flushed to disk. A new sorted run is now created. From 77b7d8d4dc11cdb5b4b86712faa1e8d37e30af86 Mon Sep 17 00:00:00 2001 From: tsreaper Date: Mon, 2 Dec 2024 15:41:03 +0800 Subject: [PATCH 086/157] [spark] Support changing column types in array or map (#4618) --- .../apache/paimon/schema/SchemaManager.java | 23 +- .../paimon/schema/SchemaManagerTest.java | 8 +- .../org/apache/paimon/flink/FlinkCatalog.java | 10 +- .../spark/SparkSchemaEvolutionITCase.java | 224 ++++++++++++++++++ 4 files changed, 253 insertions(+), 12 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index d827ffd0fb66b..83ddbccfef987 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -639,9 +639,10 @@ public void updateIntermediateColumn(List newFields, int depth) String fullFieldName = String.join(".", Arrays.asList(updateFieldNames).subList(0, depth + 1)); - List nestedFields = - new ArrayList<>(extractRowType(field.type(), fullFieldName).getFields()); - updateIntermediateColumn(nestedFields, depth + 1); + List nestedFields = new ArrayList<>(); + int newDepth = + depth + extractRowDataFields(field.type(), fullFieldName, nestedFields); + updateIntermediateColumn(nestedFields, newDepth); newFields.set( i, new DataField( @@ -657,14 +658,22 @@ public void updateIntermediateColumn(List newFields, int depth) String.join(".", Arrays.asList(updateFieldNames).subList(0, depth + 1))); } - private RowType extractRowType(DataType type, String fullFieldName) { + private int extractRowDataFields( + DataType type, String fullFieldName, List nestedFields) { switch (type.getTypeRoot()) { case ROW: - return (RowType) type; + nestedFields.addAll(((RowType) type).getFields()); + return 1; case ARRAY: - return extractRowType(((ArrayType) type).getElementType(), fullFieldName); + return extractRowDataFields( + ((ArrayType) type).getElementType(), + fullFieldName, + nestedFields) + + 1; case MAP: - return extractRowType(((MapType) type).getValueType(), fullFieldName); + return extractRowDataFields( + ((MapType) type).getValueType(), fullFieldName, nestedFields) + + 1; default: throw new IllegalArgumentException( fullFieldName + " is not a structured type."); diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java index f0d6543699659..c8b102b3584dc 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java @@ -738,13 +738,15 @@ public void testUpdateRowTypeInArrayAndMap() throws Exception { SchemaChange addColumn = SchemaChange.addColumn( - new String[] {"v", "f3"}, + new String[] {"v", "element", "value", "f3"}, DataTypes.STRING(), null, SchemaChange.Move.first("f3")); - SchemaChange dropColumn = SchemaChange.dropColumn(new String[] {"v", "f2"}); + SchemaChange dropColumn = + SchemaChange.dropColumn(new String[] {"v", "element", "value", "f2"}); SchemaChange updateColumnType = - SchemaChange.updateColumnType(new String[] {"v", "f1"}, DataTypes.BIGINT(), false); + SchemaChange.updateColumnType( + new String[] {"v", "element", "value", "f1"}, DataTypes.BIGINT(), false); manager.commitChanges(addColumn, dropColumn, updateColumnType); innerType = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index 09fc0328ef655..c67e79c1c06b7 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -756,8 +756,11 @@ private void generateNestedColumnUpdates( "Column %s can only be updated to array type, and cannot be updated to %s type", joinedNames, newType); + List fullFieldNames = new ArrayList<>(fieldNames); + // add a dummy column name indicating the element of array + fullFieldNames.add("element"); generateNestedColumnUpdates( - fieldNames, + fullFieldNames, ((org.apache.paimon.types.ArrayType) oldType).getElementType(), ((org.apache.paimon.types.ArrayType) newType).getElementType(), schemaChanges); @@ -775,8 +778,11 @@ private void generateNestedColumnUpdates( joinedNames, oldMapType.getKeyType(), newMapType.getKeyType()); + List fullFieldNames = new ArrayList<>(fieldNames); + // add a dummy column name indicating the value of map + fullFieldNames.add("value"); generateNestedColumnUpdates( - fieldNames, + fullFieldNames, oldMapType.getValueType(), newMapType.getValueType(), schemaChanges); diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java index 771ddc62878d1..fb4dab38ed940 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java @@ -789,6 +789,89 @@ public void testAddAndDropNestedColumn(String formatType) { "[5,[53,[503,500.03,5003],five]]"); } + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testAddAndDropNestedColumnInArray(String formatType) { + String tableName = "testAddNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v ARRAY>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT('apple', 100), STRUCT('banana', 101))), " + + "(2, ARRAY(STRUCT('cat', 200), STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,WrappedArray([apple,100], [banana,101])]", + "[2,WrappedArray([cat,200], [dog,201])]"); + + spark.sql( + "ALTER TABLE paimon.default." + + tableName + + " ADD COLUMN v.element.f3 STRING AFTER f2"); + spark.sql("ALTER TABLE paimon.default." + tableName + " DROP COLUMN v.element.f1"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT(110, 'APPLE'), STRUCT(111, 'BANANA'))), " + + "(3, ARRAY(STRUCT(310, 'FLOWER')))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,WrappedArray([110,APPLE], [111,BANANA])]", + "[2,WrappedArray([200,null], [201,null])]", + "[3,WrappedArray([310,FLOWER])]"); + } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testAddAndDropNestedColumnInMap(String formatType) { + String tableName = "testAddNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v MAP>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT('apple', 100), 20, STRUCT('banana', 101))), " + + "(2, MAP(10, STRUCT('cat', 200), 20, STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT k, v[10].f1, v[10].f2 FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[1,apple,100]", "[2,cat,200]"); + + spark.sql( + "ALTER TABLE paimon.default." + + tableName + + " ADD COLUMN v.value.f3 STRING AFTER f2"); + spark.sql("ALTER TABLE paimon.default." + tableName + " DROP COLUMN v.value.f1"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT(110, 'APPLE'), 20, STRUCT(111, 'BANANA'))), " + + "(3, MAP(10, STRUCT(310, 'FLOWER')))"); + assertThat( + spark.sql("SELECT k, v[10].f2, v[10].f3 FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[1,110,APPLE]", "[2,200,null]", "[3,310,FLOWER]"); + } + @ParameterizedTest() @ValueSource(strings = {"orc", "avro", "parquet"}) public void testRenameNestedColumn(String formatType) { @@ -818,6 +901,67 @@ public void testRenameNestedColumn(String formatType) { .containsExactlyInAnyOrder("[apple,1]", "[banana,2]"); } + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testRenameNestedColumnInArray(String formatType) { + String tableName = "testRenameNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v ARRAY>) " + + "TBLPROPERTIES ('file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT('apple', 100), STRUCT('banana', 101))), " + + "(2, ARRAY(STRUCT('cat', 200), STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT v[0].f1, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[cat,2]"); + + spark.sql( + "ALTER TABLE paimon.default." + tableName + " RENAME COLUMN v.element.f1 to f100"); + assertThat( + spark.sql("SELECT v[0].f100, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[cat,2]"); + } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testRenameNestedColumnInMap(String formatType) { + String tableName = "testRenameNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v MAP>) " + + "TBLPROPERTIES ('file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT('apple', 100), 20, STRUCT('banana', 101))), " + + "(2, MAP(10, STRUCT('cat', 200), 20, STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT v[10].f1, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[cat,2]"); + + spark.sql("ALTER TABLE paimon.default." + tableName + " RENAME COLUMN v.value.f1 to f100"); + assertThat( + spark.sql("SELECT v[10].f100, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[cat,2]"); + } + @ParameterizedTest() @ValueSource(strings = {"orc", "avro", "parquet"}) public void testUpdateNestedColumnType(String formatType) { @@ -850,4 +994,84 @@ public void testUpdateNestedColumnType(String formatType) { .map(Row::toString)) .containsExactlyInAnyOrder("[101,1]", "[200,2]", "[3000000000000,3]"); } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testUpdateNestedColumnTypeInArray(String formatType) { + String tableName = "testRenameNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v ARRAY>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT('apple', 100), STRUCT('banana', 101))), " + + "(2, ARRAY(STRUCT('cat', 200), STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,WrappedArray([apple,100], [banana,101])]", + "[2,WrappedArray([cat,200], [dog,201])]"); + + spark.sql( + "ALTER TABLE paimon.default." + + tableName + + " CHANGE COLUMN v.element.f2 f2 BIGINT"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT('APPLE', 1000000000000), STRUCT('BANANA', 111))), " + + "(3, ARRAY(STRUCT('FLOWER', 3000000000000)))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,WrappedArray([APPLE,1000000000000], [BANANA,111])]", + "[2,WrappedArray([cat,200], [dog,201])]", + "[3,WrappedArray([FLOWER,3000000000000])]"); + } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testUpdateNestedColumnTypeInMap(String formatType) { + String tableName = "testRenameNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v MAP>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT('apple', 100), 20, STRUCT('banana', 101))), " + + "(2, MAP(10, STRUCT('cat', 200), 20, STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT k, v[10].f1, v[10].f2 FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[1,apple,100]", "[2,cat,200]"); + + spark.sql( + "ALTER TABLE paimon.default." + tableName + " CHANGE COLUMN v.value.f2 f2 BIGINT"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT('APPLE', 1000000000000), 20, STRUCT('BANANA', 111))), " + + "(3, MAP(10, STRUCT('FLOWER', 3000000000000)))"); + assertThat( + spark.sql("SELECT k, v[10].f1, v[10].f2 FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,APPLE,1000000000000]", "[2,cat,200]", "[3,FLOWER,3000000000000]"); + } } From 6f967b4db5ca69d9ca530195bfc56e461b1719c6 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 2 Dec 2024 15:43:53 +0800 Subject: [PATCH 087/157] [core] Make metadata.stats-dense-store default value is true (#4617) --- docs/content/flink/sql-ddl.md | 4 +- .../generated/core_configuration.html | 4 +- .../java/org/apache/paimon/CoreOptions.java | 6 +-- .../apache/paimon/stats/StatsTableTest.java | 52 ++++++++++++++++++- .../table/AppendOnlyFileStoreTableTest.java | 1 - .../table/PrimaryKeyFileStoreTableTest.java | 2 - 6 files changed, 58 insertions(+), 11 deletions(-) diff --git a/docs/content/flink/sql-ddl.md b/docs/content/flink/sql-ddl.md index 0324e6655689a..8b8c069229aae 100644 --- a/docs/content/flink/sql-ddl.md +++ b/docs/content/flink/sql-ddl.md @@ -203,8 +203,8 @@ Paimon will automatically collect the statistics of the data file for speeding u The statistics collector mode can be configured by `'metadata.stats-mode'`, by default is `'truncate(16)'`. You can configure the field level by setting `'fields.{field_name}.stats-mode'`. -For the stats mode of `none`, we suggest that you configure `metadata.stats-dense-store` = `true`, which will -significantly reduce the storage size of the manifest. +For the stats mode of `none`, by default `metadata.stats-dense-store` is `true`, which will significantly reduce the +storage size of the manifest. But the Paimon sdk in reading engine requires at least version 0.9.1 or 1.0.0 or higher. ### Field Default Value diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index a38f07784fd38..fad1f4907e5a9 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -485,9 +485,9 @@ - + - + diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index fce09357f07ae..b9b5675f1d2d5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -1101,7 +1101,7 @@ public class CoreOptions implements Serializable { public static final ConfigOption METADATA_STATS_DENSE_STORE = key("metadata.stats-dense-store") .booleanType() - .defaultValue(false) + .defaultValue(true) .withDescription( Description.builder() .text( @@ -1110,8 +1110,8 @@ public class CoreOptions implements Serializable { + " none statistic mode is set.") .linebreak() .text( - "Note, when this mode is enabled, the Paimon sdk in reading engine requires" - + " at least version 0.9.1 or 1.0.0 or higher.") + "Note, when this mode is enabled with 'metadata.stats-mode:none', the Paimon sdk in" + + " reading engine requires at least version 0.9.1 or 1.0.0 or higher.") .build()); public static final ConfigOption COMMIT_CALLBACKS = diff --git a/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java b/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java index 494b2e28e4593..25282d898a3dd 100644 --- a/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java @@ -35,6 +35,7 @@ import org.junit.jupiter.api.Test; +import static org.apache.paimon.CoreOptions.METADATA_STATS_DENSE_STORE; import static org.apache.paimon.CoreOptions.METADATA_STATS_MODE; import static org.assertj.core.api.Assertions.assertThat; @@ -42,10 +43,11 @@ public class StatsTableTest extends TableTestBase { @Test - public void testPartitionStats() throws Exception { + public void testPartitionStatsNotDense() throws Exception { Identifier identifier = identifier("T"); Options options = new Options(); options.set(METADATA_STATS_MODE, "NONE"); + options.set(METADATA_STATS_DENSE_STORE, false); options.set(CoreOptions.BUCKET, 1); Schema schema = Schema.newBuilder() @@ -90,4 +92,52 @@ public void testPartitionStats() throws Exception { assertThat(recordStats.maxValues().isNullAt(1)).isTrue(); assertThat(recordStats.maxValues().isNullAt(2)).isTrue(); } + + @Test + public void testPartitionStatsDenseMode() throws Exception { + Identifier identifier = identifier("T"); + Options options = new Options(); + options.set(METADATA_STATS_MODE, "NONE"); + options.set(CoreOptions.BUCKET, 1); + Schema schema = + Schema.newBuilder() + .column("pt", DataTypes.INT()) + .column("pk", DataTypes.INT()) + .column("col1", DataTypes.INT()) + .partitionKeys("pt") + .primaryKey("pk", "pt") + .options(options.toMap()) + .build(); + catalog.createTable(identifier, schema, true); + Table table = catalog.getTable(identifier); + + write( + table, + GenericRow.of(1, 1, 1), + GenericRow.of(1, 2, 1), + GenericRow.of(1, 3, 1), + GenericRow.of(2, 1, 1)); + + FileStoreTable storeTable = (FileStoreTable) table; + FileStore store = storeTable.store(); + String manifestListFile = storeTable.snapshotManager().latestSnapshot().deltaManifestList(); + + ManifestList manifestList = store.manifestListFactory().create(); + ManifestFileMeta manifest = manifestList.read(manifestListFile).get(0); + + // should have partition stats + SimpleStats partitionStats = manifest.partitionStats(); + assertThat(partitionStats.minValues().getInt(0)).isEqualTo(1); + assertThat(partitionStats.maxValues().getInt(0)).isEqualTo(2); + + // should not have record stats because of NONE mode + ManifestFile manifestFile = store.manifestFileFactory().create(); + DataFileMeta file = + manifestFile.read(manifest.fileName(), manifest.fileSize()).get(0).file(); + SimpleStats recordStats = file.valueStats(); + assertThat(file.valueStatsCols()).isEmpty(); + assertThat(recordStats.minValues().getFieldCount()).isEqualTo(0); + assertThat(recordStats.maxValues().getFieldCount()).isEqualTo(0); + assertThat(recordStats.nullCounts().size()).isEqualTo(0); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java index 0328cc6bada34..922221bb8dc99 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java @@ -230,7 +230,6 @@ public void testBatchProjection() throws Exception { public void testBatchFilter(boolean statsDenseStore) throws Exception { Consumer optionsSetter = options -> { - options.set(CoreOptions.METADATA_STATS_DENSE_STORE, statsDenseStore); if (statsDenseStore) { options.set(CoreOptions.METADATA_STATS_MODE, "none"); options.set("fields.b.stats-mode", "full"); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java index 51c8b328dfc6a..46b85223bc2fa 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java @@ -348,7 +348,6 @@ public void testBatchProjection() throws Exception { public void testBatchFilter(boolean statsDenseStore) throws Exception { Consumer optionsSetter = options -> { - options.set(CoreOptions.METADATA_STATS_DENSE_STORE, statsDenseStore); if (statsDenseStore) { // pk table doesn't need value stats options.set(CoreOptions.METADATA_STATS_MODE, "none"); @@ -1664,7 +1663,6 @@ public void testReadDeletionVectorTable(boolean statsDenseStore) throws Exceptio options.set(TARGET_FILE_SIZE, new MemorySize(1)); options.set(DELETION_VECTORS_ENABLED, true); - options.set(CoreOptions.METADATA_STATS_DENSE_STORE, statsDenseStore); if (statsDenseStore) { options.set(CoreOptions.METADATA_STATS_MODE, "none"); options.set("fields.b.stats-mode", "full"); From 39b82ec160414f7ca2cdafed0fc453f3fc7a7529 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 2 Dec 2024 15:46:40 +0800 Subject: [PATCH 088/157] [flink] Replace legacy SinkFunction with v2 Sink (#4612) --- .../connector/sink2/WriterInitContext.java | 22 +++++ .../api/functions/sink/v2/DiscardingSink.java | 58 +++++++++++++ .../connector/sink2/WriterInitContext.java | 22 +++++ .../api/functions/sink/v2/DiscardingSink.java | 58 +++++++++++++ .../connector/sink2/WriterInitContext.java | 25 ++++++ .../api/functions/sink/v2/DiscardingSink.java | 59 +++++++++++++ .../connector/sink2/WriterInitContext.java | 22 +++++ .../api/functions/sink/v2/DiscardingSink.java | 59 +++++++++++++ .../sink/cdc/FlinkCdcMultiTableSink.java | 4 +- .../sink/cdc/FlinkCdcMultiTableSinkTest.java | 5 +- .../paimon/flink/action/CloneAction.java | 4 +- .../flink/service/QueryAddressRegister.java | 84 +++++++++++-------- .../paimon/flink/service/QueryService.java | 2 +- .../sink/CombinedTableCompactorSink.java | 4 +- .../apache/paimon/flink/sink/FlinkSink.java | 4 +- .../sink/WriterChainingStrategyTest.java | 2 +- 16 files changed, 384 insertions(+), 50 deletions(-) create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java create mode 100644 paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java create mode 100644 paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java create mode 100644 paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java create mode 100644 paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java new file mode 100644 index 0000000000000..563dbbe75e7e6 --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.connector.sink2; + +/** Placeholder class to resolve compatibility issues. */ +public interface WriterInitContext extends org.apache.flink.api.connector.sink2.Sink.InitContext {} diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java new file mode 100644 index 0000000000000..98aaf6418ff7f --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java @@ -0,0 +1,58 @@ +/* + * 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.flink.streaming.api.functions.sink.v2; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; + +import java.io.IOException; + +/** + * A special sink that ignores all elements. + * + * @param The type of elements received by the sink. + */ +@PublicEvolving +public class DiscardingSink implements Sink { + private static final long serialVersionUID = 1L; + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new DiscardingElementWriter(); + } + + private class DiscardingElementWriter implements SinkWriter { + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // discard it. + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + // this writer has no pending data. + } + + @Override + public void close() throws Exception { + // do nothing. + } + } +} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java new file mode 100644 index 0000000000000..563dbbe75e7e6 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.connector.sink2; + +/** Placeholder class to resolve compatibility issues. */ +public interface WriterInitContext extends org.apache.flink.api.connector.sink2.Sink.InitContext {} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java new file mode 100644 index 0000000000000..98aaf6418ff7f --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java @@ -0,0 +1,58 @@ +/* + * 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.flink.streaming.api.functions.sink.v2; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; + +import java.io.IOException; + +/** + * A special sink that ignores all elements. + * + * @param The type of elements received by the sink. + */ +@PublicEvolving +public class DiscardingSink implements Sink { + private static final long serialVersionUID = 1L; + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new DiscardingElementWriter(); + } + + private class DiscardingElementWriter implements SinkWriter { + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // discard it. + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + // this writer has no pending data. + } + + @Override + public void close() throws Exception { + // do nothing. + } + } +} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java new file mode 100644 index 0000000000000..db45000425723 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.connector.sink2; + +import org.apache.flink.annotation.Public; + +/** Placeholder class to resolve compatibility issues. */ +@Public +public interface WriterInitContext extends org.apache.flink.api.connector.sink2.Sink.InitContext {} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java new file mode 100644 index 0000000000000..fc7eb0d483569 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.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.flink.streaming.api.functions.sink.v2; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; + +import java.io.IOException; + +/** + * A special sink that ignores all elements. + * + * @param The type of elements received by the sink. + */ +@PublicEvolving +public class DiscardingSink implements Sink, SupportsConcurrentExecutionAttempts { + private static final long serialVersionUID = 1L; + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new DiscardingElementWriter(); + } + + private class DiscardingElementWriter implements SinkWriter { + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // discard it. + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + // this writer has no pending data. + } + + @Override + public void close() throws Exception { + // do nothing. + } + } +} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java new file mode 100644 index 0000000000000..563dbbe75e7e6 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java @@ -0,0 +1,22 @@ +/* + * 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.flink.api.connector.sink2; + +/** Placeholder class to resolve compatibility issues. */ +public interface WriterInitContext extends org.apache.flink.api.connector.sink2.Sink.InitContext {} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java new file mode 100644 index 0000000000000..fc7eb0d483569 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.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.flink.streaming.api.functions.sink.v2; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; + +import java.io.IOException; + +/** + * A special sink that ignores all elements. + * + * @param The type of elements received by the sink. + */ +@PublicEvolving +public class DiscardingSink implements Sink, SupportsConcurrentExecutionAttempts { + private static final long serialVersionUID = 1L; + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new DiscardingElementWriter(); + } + + private class DiscardingElementWriter implements SinkWriter { + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // discard it. + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + // this writer has no pending data. + } + + @Override + public void close() throws Exception { + // do nothing. + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java index f9b7bbc6b9105..1688d4deb0884 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java @@ -40,7 +40,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -134,7 +134,7 @@ public DataStreamSink sinkFrom( createCommittableStateManager())) .setParallelism(input.getParallelism()); configureGlobalCommitter(committed, commitCpuCores, commitHeapMemory); - return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + return committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } protected OneInputStreamOperatorFactory diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java index e1bd112ca751e..723f57a30e3f7 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java @@ -22,11 +22,11 @@ import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.options.Options; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; -import org.apache.flink.streaming.api.transformations.LegacySinkTransformation; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.junit.jupiter.api.Test; @@ -64,8 +64,7 @@ public void cancel() {} DataStreamSink dataStreamSink = sink.sinkFrom(input); // check the transformation graph - LegacySinkTransformation end = - (LegacySinkTransformation) dataStreamSink.getTransformation(); + Transformation end = dataStreamSink.getTransformation(); assertThat(end.getName()).isEqualTo("end"); OneInputTransformation committer = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java index 2f90147eeb2a3..bac030dd04966 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java @@ -32,7 +32,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import java.util.HashMap; import java.util.Map; @@ -141,7 +141,7 @@ copyFiles, new SnapshotHintChannelComputer(), parallelism) new SnapshotHintOperator(targetCatalogConfig)) .setParallelism(parallelism); - snapshotHintOperator.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + snapshotHintOperator.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java index 524f2e5f01c17..00d527506cfef 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java @@ -23,10 +23,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; -import org.apache.flink.api.common.functions.OpenContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.WriterInitContext; import java.net.InetSocketAddress; import java.util.TreeMap; @@ -34,57 +33,68 @@ import static org.apache.paimon.service.ServiceManager.PRIMARY_KEY_LOOKUP; /** Operator for address server to register addresses to {@link ServiceManager}. */ -public class QueryAddressRegister extends RichSinkFunction { - +public class QueryAddressRegister implements Sink { private final ServiceManager serviceManager; - private transient int numberExecutors; - private transient TreeMap executors; - public QueryAddressRegister(Table table) { this.serviceManager = ((FileStoreTable) table).store().newServiceManager(); } /** - * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. */ - public void open(OpenContext openContext) throws Exception { - open(new Configuration()); + public SinkWriter createWriter(InitContext context) { + return new QueryAddressRegisterSinkWriter(serviceManager); } /** - * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. */ - public void open(Configuration parameters) throws Exception { - this.executors = new TreeMap<>(); + public SinkWriter createWriter(WriterInitContext context) { + return new QueryAddressRegisterSinkWriter(serviceManager); } - @Override - public void invoke(InternalRow row, SinkFunction.Context context) { - int numberExecutors = row.getInt(0); - if (this.numberExecutors != 0 && this.numberExecutors != numberExecutors) { - throw new IllegalArgumentException( - String.format( - "Number Executors can not be changed! Old %s , New %s .", - this.numberExecutors, numberExecutors)); - } - this.numberExecutors = numberExecutors; + private static class QueryAddressRegisterSinkWriter implements SinkWriter { + private final ServiceManager serviceManager; - int executorId = row.getInt(1); - String hostname = row.getString(2).toString(); - int port = row.getInt(3); + private final TreeMap executors; - executors.put(executorId, new InetSocketAddress(hostname, port)); + private int numberExecutors; - if (executors.size() == numberExecutors) { - serviceManager.resetService( - PRIMARY_KEY_LOOKUP, executors.values().toArray(new InetSocketAddress[0])); + private QueryAddressRegisterSinkWriter(ServiceManager serviceManager) { + this.serviceManager = serviceManager; + this.executors = new TreeMap<>(); } - } - @Override - public void close() throws Exception { - super.close(); - serviceManager.deleteService(PRIMARY_KEY_LOOKUP); + @Override + public void write(InternalRow row, Context context) { + int numberExecutors = row.getInt(0); + if (this.numberExecutors != 0 && this.numberExecutors != numberExecutors) { + throw new IllegalArgumentException( + String.format( + "Number Executors can not be changed! Old %s , New %s .", + this.numberExecutors, numberExecutors)); + } + this.numberExecutors = numberExecutors; + + int executorId = row.getInt(1); + String hostname = row.getString(2).toString(); + int port = row.getInt(3); + + executors.put(executorId, new InetSocketAddress(hostname, port)); + + if (executors.size() == numberExecutors) { + serviceManager.resetService( + PRIMARY_KEY_LOOKUP, executors.values().toArray(new InetSocketAddress[0])); + } + } + + @Override + public void flush(boolean endOfInput) {} + + @Override + public void close() { + serviceManager.deleteService(PRIMARY_KEY_LOOKUP); + } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java index bd433fe0f00d0..752d54cff5a0f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java @@ -62,7 +62,7 @@ public static void build(StreamExecutionEnvironment env, Table table, int parall InternalTypeInfo.fromRowType(QueryExecutorOperator.outputType()), executorOperator) .setParallelism(parallelism) - .addSink(new QueryAddressRegister(table)) + .sinkTo(new QueryAddressRegister(table)) .setParallelism(1); sink.getTransformation().setMaxParallelism(1); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java index c2b4cc0f87e60..25f76ce976833 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java @@ -32,7 +32,7 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.data.RowData; @@ -171,7 +171,7 @@ protected DataStreamSink doCommit( if (!options.get(SINK_COMMITTER_OPERATOR_CHAINING)) { committed = committed.startNewChain(); } - return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + return committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } // TODO:refactor FlinkSink to adopt this sink diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java index 8d6c3554c76ff..002f5887b5f08 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java @@ -43,7 +43,7 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -316,7 +316,7 @@ protected DataStreamSink doCommit(DataStream written, String com } configureGlobalCommitter( committed, options.get(SINK_COMMITTER_CPU), options.get(SINK_COMMITTER_MEMORY)); - return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + return committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } public static void configureGlobalCommitter( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java index a4605b830918e..24fb529b59ea2 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java @@ -173,7 +173,7 @@ private List verifyChaining( List vertices = new ArrayList<>(); env.getStreamGraph().getJobGraph().getVertices().forEach(vertices::add); - JobVertex vertex = findVertex(vertices, "Writer"); + JobVertex vertex = findVertex(vertices, "Writer(write-only)"); if (isWriterChainedWithUpstream) { assertThat(vertex.toString()).contains("Source"); From 271de7deac24b5ecc6306d345dcb5607d9f4a35a Mon Sep 17 00:00:00 2001 From: tsreaper Date: Mon, 2 Dec 2024 15:59:39 +0800 Subject: [PATCH 089/157] [docs] Add document for nested column evolution (#4619) --- docs/content/flink/sql-alter.md | 16 +++++++ docs/content/spark/sql-alter.md | 84 +++++++++++++++++++++++++++++++++ 2 files changed, 100 insertions(+) diff --git a/docs/content/flink/sql-alter.md b/docs/content/flink/sql-alter.md index bee616f047d42..877995cc631bb 100644 --- a/docs/content/flink/sql-alter.md +++ b/docs/content/flink/sql-alter.md @@ -78,6 +78,10 @@ If you use object storage, such as S3 or OSS, please use this syntax carefully, The following SQL adds two columns `c1` and `c2` to table `my_table`. +{{< hint info >}} +To add a column in a row type, see [Changing Column Type](#changing-column-type). +{{< /hint >}} + ```sql ALTER TABLE my_table ADD (c1 INT, c2 STRING); ``` @@ -99,6 +103,10 @@ otherwise this operation may fail, throws an exception like `The following colum ALTER TABLE my_table DROP (c1, c2); ``` +{{< hint info >}} +To drop a column in a row type, see [Changing Column Type](#changing-column-type). +{{< /hint >}} + ## Dropping Partitions The following SQL drops the partitions of the paimon table. @@ -185,6 +193,14 @@ The following SQL changes type of column `col_a` to `DOUBLE`. ALTER TABLE my_table MODIFY col_a DOUBLE; ``` +Paimon also supports changing columns of row type, array type, and map type. + +```sql +-- col_a previously has type ARRAY> +-- the following SQL changes f1 to BIGINT, drops f2, and adds f3 +ALTER TABLE my_table MODIFY col_a ARRAY>; +``` + ## Adding watermark The following SQL adds a computed column `ts` from existing column `log_ts`, and a watermark with strategy `ts - INTERVAL '1' HOUR` on column `ts` which is marked as event time attribute of table `my_table`. diff --git a/docs/content/spark/sql-alter.md b/docs/content/spark/sql-alter.md index 11af186e62131..3ad72048029b0 100644 --- a/docs/content/spark/sql-alter.md +++ b/docs/content/spark/sql-alter.md @@ -95,6 +95,27 @@ ALTER TABLE my_table ADD COLUMNS ( ); ``` +The following SQL adds a nested column `f3` to a struct type. + +```sql +-- column v previously has type STRUCT +ALTER TABLE my_table ADD COLUMN v.f3 STRING; +``` + +The following SQL adds a nested column `f3` to a struct type, which is the element type of an array type. + +```sql +-- column v previously has type ARRAY> +ALTER TABLE my_table ADD COLUMN v.element.f3 STRING; +``` + +The following SQL adds a nested column `f3` to a struct type, which is the value type of a map type. + +```sql +-- column v previously has type MAP> +ALTER TABLE my_table ADD COLUMN v.value.f3 STRING; +``` + ## Renaming Column Name The following SQL renames column `c0` in table `my_table` to `c1`. @@ -103,6 +124,27 @@ The following SQL renames column `c0` in table `my_table` to `c1`. ALTER TABLE my_table RENAME COLUMN c0 TO c1; ``` +The following SQL renames a nested column `f1` to `f100` in a struct type. + +```sql +-- column v previously has type STRUCT +ALTER TABLE my_table RENAME COLUMN v.f1 to f100; +``` + +The following SQL renames a nested column `f1` to `f100` in a struct type, which is the element type of an array type. + +```sql +-- column v previously has type ARRAY> +ALTER TABLE my_table RENAME COLUMN v.element.f1 to f100; +``` + +The following SQL renames a nested column `f1` to `f100` in a struct type, which is the value type of a map type. + +```sql +-- column v previously has type MAP> +ALTER TABLE my_table RENAME COLUMN v.value.f1 to f100; +``` + ## Dropping Columns The following SQL drops two columns `c1` and `c2` from table `my_table`. @@ -111,6 +153,27 @@ The following SQL drops two columns `c1` and `c2` from table `my_table`. ALTER TABLE my_table DROP COLUMNS (c1, c2); ``` +The following SQL drops a nested column `f2` from a struct type. + +```sql +-- column v previously has type STRUCT +ALTER TABLE my_table DROP COLUMN v.f2; +``` + +The following SQL drops a nested column `f2` from a struct type, which is the element type of an array type. + +```sql +-- column v previously has type ARRAY> +ALTER TABLE my_table DROP COLUMN v.element.f2; +``` + +The following SQL drops a nested column `f2` from a struct type, which is the value type of a map type. + +```sql +-- column v previously has type MAP> +ALTER TABLE my_table DROP COLUMN v.value.f2; +``` + ## Dropping Partitions The following SQL drops the partitions of the paimon table. For spark sql, you need to specify all the partition columns. @@ -156,3 +219,24 @@ ALTER TABLE my_table ALTER COLUMN col_a AFTER col_b; ```sql ALTER TABLE my_table ALTER COLUMN col_a TYPE DOUBLE; ``` + +The following SQL changes the type of a nested column `f2` to `BIGINT` in a struct type. + +```sql +-- column v previously has type STRUCT +ALTER TABLE my_table ALTER COLUMN v.f2 TYPE BIGINT; +``` + +The following SQL changes the type of a nested column `f2` to `BIGINT` in a struct type, which is the element type of an array type. + +```sql +-- column v previously has type ARRAY> +ALTER TABLE my_table ALTER COLUMN v.element.f2 TYPE BIGINT; +``` + +The following SQL changes the type of a nested column `f2` to `BIGINT` in a struct type, which is the value type of a map type. + +```sql +-- column v previously has type MAP> +ALTER TABLE my_table ALTER COLUMN v.value.f2 TYPE BIGINT; +``` From 4e1b74976590fbe5d885614b65e038217c9266df Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 2 Dec 2024 18:13:33 +0800 Subject: [PATCH 090/157] [iceberg] Use gzip by default to iceberg avro writer (#4620) --- docs/content/migration/iceberg-compatibility.md | 6 ++++++ .../org/apache/paimon/iceberg/IcebergOptions.java | 7 +++++++ .../iceberg/manifest/IcebergManifestFile.java | 15 +++++++++++---- .../iceberg/manifest/IcebergManifestList.java | 15 +++++++++++---- .../paimon/iceberg/IcebergCompatibilityTest.java | 6 ++++++ 5 files changed, 41 insertions(+), 8 deletions(-) diff --git a/docs/content/migration/iceberg-compatibility.md b/docs/content/migration/iceberg-compatibility.md index f07f78cb201df..7b83936b53208 100644 --- a/docs/content/migration/iceberg-compatibility.md +++ b/docs/content/migration/iceberg-compatibility.md @@ -371,6 +371,12 @@ you also need to set some (or all) of the following table options when creating + + + + + +
    metadata.stats-dense-store
    falsetrue BooleanWhether to store statistic densely in metadata (manifest files), which will significantly reduce the storage size of metadata when the none statistic mode is set.
    Note, when this mode is enabled, the Paimon sdk in reading engine requires at least version 0.9.1 or 1.0.0 or higher.
    Whether to store statistic densely in metadata (manifest files), which will significantly reduce the storage size of metadata when the none statistic mode is set.
    Note, when this mode is enabled with 'metadata.stats-mode:none', the Paimon sdk in reading engine requires at least version 0.9.1 or 1.0.0 or higher.
    metadata.stats-mode
    String hadoop-conf-dir for Iceberg Hive catalog.
    metadata.iceberg.manifest-compression
    gzipStringCompression for Iceberg manifest files.
    diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java index 769ce6b1614af..3900233d21cea 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java @@ -70,6 +70,13 @@ public class IcebergOptions { .noDefaultValue() .withDescription("hadoop-conf-dir for Iceberg Hive catalog."); + public static final ConfigOption MANIFEST_COMPRESSION = + key("metadata.iceberg.manifest-compression") + .stringType() + .defaultValue( + "gzip") // some Iceberg reader cannot support zstd, for example DuckDB + .withDescription("Compression for Iceberg manifest files."); + /** Where to store Iceberg metadata. */ public enum StorageType implements DescribedEnum { DISABLED("disabled", "Disable Iceberg compatibility support."), diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java index d04cf3576a111..57484a1f3ff92 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java @@ -19,6 +19,7 @@ package org.apache.paimon.iceberg.manifest; import org.apache.paimon.CoreOptions; +import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.FormatWriterFactory; @@ -26,6 +27,7 @@ import org.apache.paimon.format.SimpleStatsCollector; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.iceberg.IcebergOptions; import org.apache.paimon.iceberg.IcebergPathFactory; import org.apache.paimon.iceberg.manifest.IcebergManifestFileMeta.Content; import org.apache.paimon.iceberg.metadata.IcebergPartitionSpec; @@ -82,23 +84,28 @@ public IcebergManifestFile( this.targetFileSize = targetFileSize; } + @VisibleForTesting + public String compression() { + return compression; + } + public static IcebergManifestFile create(FileStoreTable table, IcebergPathFactory pathFactory) { RowType partitionType = table.schema().logicalPartitionType(); RowType entryType = IcebergManifestEntry.schema(partitionType); - Options manifestFileAvroOptions = Options.fromMap(table.options()); + Options avroOptions = Options.fromMap(table.options()); // https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/ManifestReader.java - manifestFileAvroOptions.set( + avroOptions.set( "avro.row-name-mapping", "org.apache.paimon.avro.generated.record:manifest_entry," + "manifest_entry_data_file:r2," + "r2_partition:r102"); - FileFormat manifestFileAvro = FileFormat.fromIdentifier("avro", manifestFileAvroOptions); + FileFormat manifestFileAvro = FileFormat.fromIdentifier("avro", avroOptions); return new IcebergManifestFile( table.fileIO(), partitionType, manifestFileAvro.createReaderFactory(entryType), manifestFileAvro.createWriterFactory(entryType), - table.coreOptions().manifestCompression(), + avroOptions.get(IcebergOptions.MANIFEST_COMPRESSION), pathFactory.manifestFileFactory(), table.coreOptions().manifestTargetSize()); } diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestList.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestList.java index 911ebf954ce53..ab5cc926cd961 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestList.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestList.java @@ -18,10 +18,12 @@ package org.apache.paimon.iceberg.manifest; +import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.FormatWriterFactory; import org.apache.paimon.fs.FileIO; +import org.apache.paimon.iceberg.IcebergOptions; import org.apache.paimon.iceberg.IcebergPathFactory; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; @@ -51,19 +53,24 @@ public IcebergManifestList( null); } + @VisibleForTesting + public String compression() { + return compression; + } + public static IcebergManifestList create(FileStoreTable table, IcebergPathFactory pathFactory) { - Options manifestListAvroOptions = Options.fromMap(table.options()); + Options avroOptions = Options.fromMap(table.options()); // https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/ManifestLists.java - manifestListAvroOptions.set( + avroOptions.set( "avro.row-name-mapping", "org.apache.paimon.avro.generated.record:manifest_file," + "manifest_file_partitions:r508"); - FileFormat manifestListAvro = FileFormat.fromIdentifier("avro", manifestListAvroOptions); + FileFormat manifestListAvro = FileFormat.fromIdentifier("avro", avroOptions); return new IcebergManifestList( table.fileIO(), manifestListAvro.createReaderFactory(IcebergManifestFileMeta.schema()), manifestListAvro.createWriterFactory(IcebergManifestFileMeta.schema()), - table.coreOptions().manifestCompression(), + avroOptions.get(IcebergOptions.MANIFEST_COMPRESSION), pathFactory.manifestListFactory()); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java index 9a27d5618459e..45cfe109b9869 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java @@ -30,6 +30,7 @@ import org.apache.paimon.disk.IOManagerImpl; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.iceberg.manifest.IcebergManifestFile; import org.apache.paimon.iceberg.manifest.IcebergManifestFileMeta; import org.apache.paimon.iceberg.manifest.IcebergManifestList; import org.apache.paimon.iceberg.metadata.IcebergMetadata; @@ -302,6 +303,11 @@ public void testIcebergSnapshotExpire() throws Exception { IcebergPathFactory pathFactory = new IcebergPathFactory(new Path(table.location(), "metadata")); IcebergManifestList manifestList = IcebergManifestList.create(table, pathFactory); + assertThat(manifestList.compression()).isEqualTo("gzip"); + + IcebergManifestFile manifestFile = IcebergManifestFile.create(table, pathFactory); + assertThat(manifestFile.compression()).isEqualTo("gzip"); + Set usingManifests = new HashSet<>(); for (IcebergManifestFileMeta fileMeta : manifestList.read(new Path(metadata.currentSnapshot().manifestList()).getName())) { From 512e2ce05c15c388f81bf2b645546c9a29071a7b Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 2 Dec 2024 19:06:20 +0800 Subject: [PATCH 091/157] [iceberg] Introduce metadata.iceberg.manifest-legacy-version (#4621) --- .../migration/iceberg-compatibility.md | 11 +++++++ .../apache/paimon/iceberg/IcebergOptions.java | 7 +++++ .../manifest/IcebergManifestFileMeta.java | 29 ++++++++++++++++++- .../IcebergManifestFileMetaSerializer.java | 5 ++-- .../iceberg/manifest/IcebergManifestList.java | 24 ++++++++------- .../iceberg/IcebergCompatibilityTest.java | 15 ++++++++-- 6 files changed, 75 insertions(+), 16 deletions(-) diff --git a/docs/content/migration/iceberg-compatibility.md b/docs/content/migration/iceberg-compatibility.md index 7b83936b53208..d745607148643 100644 --- a/docs/content/migration/iceberg-compatibility.md +++ b/docs/content/migration/iceberg-compatibility.md @@ -377,9 +377,20 @@ you also need to set some (or all) of the following table options when creating String Compression for Iceberg manifest files. + +
    metadata.iceberg.manifest-legacy-version
    + false + Boolean + Should use the legacy manifest version to generate Iceberg's 1.4 manifest files. + +## AWS Athena + +AWS Athena may use old manifest reader to read Iceberg manifest by names, we should let Paimon producing legacy Iceberg +manifest list file, you can enable: `'metadata.iceberg.manifest-legacy-version'`. + ## Trino Iceberg In this example, we use Trino Iceberg connector to access Paimon table through Iceberg Hive catalog. diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java index 3900233d21cea..c0ceed97ba8cf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java @@ -77,6 +77,13 @@ public class IcebergOptions { "gzip") // some Iceberg reader cannot support zstd, for example DuckDB .withDescription("Compression for Iceberg manifest files."); + public static final ConfigOption MANIFEST_LEGACY_VERSION = + key("metadata.iceberg.manifest-legacy-version") + .booleanType() + .defaultValue(false) + .withDescription( + "Should use the legacy manifest version to generate Iceberg's 1.4 manifest files."); + /** Where to store Iceberg metadata. */ public enum StorageType implements DescribedEnum { DISABLED("disabled", "Disable Iceberg compatibility support."), diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMeta.java index fd9c2daf7e208..c5fcb6005fcb8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMeta.java @@ -165,7 +165,11 @@ public List partitions() { return partitions; } - public static RowType schema() { + public static RowType schema(boolean legacyVersion) { + return legacyVersion ? schemaForIceberg1_4() : schemaForIcebergNew(); + } + + private static RowType schemaForIcebergNew() { List fields = new ArrayList<>(); fields.add(new DataField(500, "manifest_path", DataTypes.STRING().notNull())); fields.add(new DataField(501, "manifest_length", DataTypes.BIGINT().notNull())); @@ -186,6 +190,29 @@ public static RowType schema() { return new RowType(false, fields); } + private static RowType schemaForIceberg1_4() { + // see https://github.com/apache/iceberg/pull/5338 + // some reader still want old schema, for example, AWS athena + List fields = new ArrayList<>(); + fields.add(new DataField(500, "manifest_path", DataTypes.STRING().notNull())); + fields.add(new DataField(501, "manifest_length", DataTypes.BIGINT().notNull())); + fields.add(new DataField(502, "partition_spec_id", DataTypes.INT().notNull())); + fields.add(new DataField(517, "content", DataTypes.INT().notNull())); + fields.add(new DataField(515, "sequence_number", DataTypes.BIGINT().notNull())); + fields.add(new DataField(516, "min_sequence_number", DataTypes.BIGINT().notNull())); + fields.add(new DataField(503, "added_snapshot_id", DataTypes.BIGINT())); + fields.add(new DataField(504, "added_data_files_count", DataTypes.INT().notNull())); + fields.add(new DataField(505, "existing_data_files_count", DataTypes.INT().notNull())); + fields.add(new DataField(506, "deleted_data_files_count", DataTypes.INT().notNull())); + fields.add(new DataField(512, "added_rows_count", DataTypes.BIGINT().notNull())); + fields.add(new DataField(513, "existing_rows_count", DataTypes.BIGINT().notNull())); + fields.add(new DataField(514, "deleted_rows_count", DataTypes.BIGINT().notNull())); + fields.add( + new DataField( + 508, "partitions", DataTypes.ARRAY(IcebergPartitionSummary.schema()))); + return new RowType(false, fields); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMetaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMetaSerializer.java index c40a26e8fdf8f..2b4c9b771c599 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMetaSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMetaSerializer.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalRow; import org.apache.paimon.iceberg.manifest.IcebergManifestFileMeta.Content; +import org.apache.paimon.types.RowType; import org.apache.paimon.utils.ObjectSerializer; import java.util.ArrayList; @@ -36,8 +37,8 @@ public class IcebergManifestFileMetaSerializer extends ObjectSerializer { public IcebergManifestList( FileIO fileIO, - FormatReaderFactory readerFactory, - FormatWriterFactory writerFactory, + FileFormat fileFormat, + RowType manifestType, String compression, PathFactory pathFactory) { super( fileIO, - new IcebergManifestFileMetaSerializer(), - IcebergManifestFileMeta.schema(), - readerFactory, - writerFactory, + new IcebergManifestFileMetaSerializer(manifestType), + manifestType, + fileFormat.createReaderFactory(manifestType), + fileFormat.createWriterFactory(manifestType), compression, pathFactory, null); @@ -65,11 +64,14 @@ public static IcebergManifestList create(FileStoreTable table, IcebergPathFactor "avro.row-name-mapping", "org.apache.paimon.avro.generated.record:manifest_file," + "manifest_file_partitions:r508"); - FileFormat manifestListAvro = FileFormat.fromIdentifier("avro", avroOptions); + FileFormat fileFormat = FileFormat.fromIdentifier("avro", avroOptions); + RowType manifestType = + IcebergManifestFileMeta.schema( + avroOptions.get(IcebergOptions.MANIFEST_LEGACY_VERSION)); return new IcebergManifestList( table.fileIO(), - manifestListAvro.createReaderFactory(IcebergManifestFileMeta.schema()), - manifestListAvro.createWriterFactory(IcebergManifestFileMeta.schema()), + fileFormat, + manifestType, avroOptions.get(IcebergOptions.MANIFEST_COMPRESSION), pathFactory.manifestListFactory()); } diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java index 45cfe109b9869..b069ac031d38b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java @@ -78,6 +78,7 @@ import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for Iceberg compatibility. */ public class IcebergCompatibilityTest { @@ -309,11 +310,21 @@ public void testIcebergSnapshotExpire() throws Exception { assertThat(manifestFile.compression()).isEqualTo("gzip"); Set usingManifests = new HashSet<>(); - for (IcebergManifestFileMeta fileMeta : - manifestList.read(new Path(metadata.currentSnapshot().manifestList()).getName())) { + String manifestListFile = new Path(metadata.currentSnapshot().manifestList()).getName(); + for (IcebergManifestFileMeta fileMeta : manifestList.read(manifestListFile)) { usingManifests.add(fileMeta.manifestPath()); } + IcebergManifestList legacyManifestList = + IcebergManifestList.create( + table.copy( + Collections.singletonMap( + IcebergOptions.MANIFEST_LEGACY_VERSION.key(), "true")), + pathFactory); + assertThatThrownBy(() -> legacyManifestList.read(manifestListFile)) + .rootCause() + .isInstanceOf(NullPointerException.class); + Set unusedFiles = new HashSet<>(); for (int i = 0; i < 2; i++) { unusedFiles.add(metadata.snapshots().get(i).manifestList()); From a5cb687755bcb02780e291f9d58538eb9a9d23e0 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Mon, 2 Dec 2024 19:12:55 +0800 Subject: [PATCH 092/157] [doc] Update Python API for pypaimon 0.2 (#4623) --- docs/content/program-api/python-api.md | 256 +++++++++++++++++-------- 1 file changed, 179 insertions(+), 77 deletions(-) diff --git a/docs/content/program-api/python-api.md b/docs/content/program-api/python-api.md index ff457cc55f446..86acee06d3a86 100644 --- a/docs/content/program-api/python-api.md +++ b/docs/content/program-api/python-api.md @@ -34,9 +34,9 @@ Java-based implementation will launch a JVM and use `py4j` to execute Java code ### SDK Installing -SDK is published at [paimon-python](https://pypi.org/project/paimon-python/). You can install by +SDK is published at [pypaimon](https://pypi.org/project/pypaimon/). You can install by ```shell -pip install paimon-python +pip install pypaimon ``` ### Java Runtime Environment @@ -67,7 +67,7 @@ classpath via one of the following ways: ```python import os -from paimon_python_java import constants +from pypaimon.py4j import constants os.environ[constants.PYPAIMON_JAVA_CLASSPATH] = '/path/to/jars/*' ``` @@ -81,7 +81,7 @@ You can set JVM args via one of the following ways: ```python import os -from paimon_python_java import constants +from pypaimon.py4j import constants os.environ[constants.PYPAIMON_JVM_ARGS] = 'arg1 arg2 ...' ``` @@ -98,7 +98,7 @@ Otherwise, you should set hadoop classpath via one of the following ways: ```python import os -from paimon_python_java import constants +from pypaimon.py4j import constants os.environ[constants.PYPAIMON_HADOOP_CLASSPATH] = '/path/to/jars/*' ``` @@ -111,7 +111,7 @@ If you just want to test codes in local, we recommend to use [Flink Pre-bundled Before coming into contact with the Table, you need to create a Catalog. ```python -from paimon_python_java import Catalog +from pypaimon.py4j import Catalog # Note that keys and values are all string catalog_options = { @@ -138,12 +138,17 @@ catalog.create_database( ### Create Schema -Table schema contains fields definition, partition keys, primary keys, table options and comment. For example: +Table schema contains fields definition, partition keys, primary keys, table options and comment. +The field definition is described by `pyarrow.Schema`. All arguments except fields definition are optional. + +Generally, there are two ways to build `pyarrow.Schema`. + +First, you can use `pyarrow.schema` method directly, for example: ```python import pyarrow as pa -from paimon_python_api import Schema +from pypaimon import Schema pa_schema = pa.schema([ ('dt', pa.string()), @@ -161,13 +166,13 @@ schema = Schema( ) ``` -All arguments except `pa_schema` is optional. If you have some Pandas data, the `pa_schema` can be extracted from `DataFrame`: +Second, if you have some Pandas data, the `pa_schema` can be extracted from `DataFrame`: ```python import pandas as pd import pyarrow as pa -from paimon_python_api import Schema +from pypaimon import Schema # Example DataFrame data data = { @@ -184,10 +189,15 @@ schema = Schema( pa_schema=record_batch.schema, partition_keys=['dt', 'hh'], primary_keys=['dt', 'hh', 'pk'], - options={'bucket': '2'}) + options={'bucket': '2'}, + comment='my test table' +) ``` ### Create Table + +After building table schema, you can create corresponding table: + ```python schema = ... catalog.create_table( @@ -207,30 +217,165 @@ table = catalog.get_table('database_name.table_name') ## Batch Read -TableRead interface provides parallelly reading for multiple splits. You can set `'max-workers': 'N'` in `catalog_options` -to set thread numbers when reading splits. `max-workers` is 1 by default, that means TableRead will read splits sequentially -if you doesn't set `max-workers`. +The reading is divided into Scan Plan and Read Splits stages. A `ReadBuilder` is used to create utils for these stages. ```python table = catalog.get_table('database_name.table_name') - -# 1. Create table scan and read read_builder = table.new_read_builder() +``` + +You can use `PredicateBuilder` to build filters and pushdown them by `ReadBuilder`: + +```python +# Example filter: ('f0' < 3 OR 'f1' > 6) AND 'f3' = 'A' + +predicate_builder = read_builder.new_predicate_builder() + +predicate1 = predicate_builder.less_than('f0', 3) +predicate2 = predicate_builder.greater_than('f1', 6) +predicate3 = predicate_builder.or_predicates([predicate1, predicate2]) + +predicate4 = predicate_builder.equal('f3', 'A') +predicate_5 = predicate_builder.and_predicates([predicate3, predicate4]) + +read_builder = read_builder.with_filter(predicate_5) +``` + + +| Predicate kind | Predicate method | +|:----------------------|:----------------------------------------------| +| p1 and p2 | PredicateBuilder.and_predicates([p1, p2]) | +| p1 or p2 | PredicateBuilder.or_predicates([p1, p2]) | +| f = literal | PredicateBuilder.equal(f, literal) | +| f != literal | PredicateBuilder.not_equal(f, literal) | +| f < literal | PredicateBuilder.less_than(f, literal) | +| f <= literal | PredicateBuilder.less_or_equal(f, literal) | +| f > literal | PredicateBuilder.greater_than(f, literal) | +| f >= literal | PredicateBuilder.greater_or_equal(f, literal) | +| f is null | PredicateBuilder.is_null(f) | +| f is not null | PredicateBuilder.is_not_null(f) | +| f.startswith(literal) | PredicateBuilder.startswith(f, literal) | +| f.endswith(literal) | PredicateBuilder.endswith(f, literal) | +| f.contains(literal) | PredicateBuilder.contains(f, literal) | +| f is in [l1, l2] | PredicateBuilder.is_in(f, [l1, l2]) | +| f is not in [l1, l2] | PredicateBuilder.is_not_in(f, [l1, l2]) | +| lower <= f <= upper | PredicateBuilder.between(f, lower, upper) | + +You can also pushdown projection by `ReadBuilder`: + +```python +# select f3 and f2 columns +read_builder = read_builder.with_projection(['f3', 'f2']) +``` + +Then you can scan plan: + +```python table_scan = read_builder.new_scan() -table_read = read_builder.new_read() +splits = table_scan.splits() +``` + +Finally, you can read data from the `splits` to various data format. -# 2. Get splits -splits = table_scan.plan().splits() +### Apache Arrow -# 3. Read splits. Support 3 methods: -# 3.1 Read as pandas.DataFrame -dataframe = table_read.to_pandas(splits) +This requires `pyarrow` to be installed. -# 3.2 Read as pyarrow.Table +You can read all the data into a `pyarrow.Table`: + +```python +table_read = read_builder.new_read() pa_table = table_read.to_arrow(splits) +print(pa_table) + +# pyarrow.Table +# f0: int32 +# f1: string +# ---- +# f0: [[1,2,3],[4,5,6],...] +# f1: [["a","b","c"],["d","e","f"],...] +``` + +You can also read data into a `pyarrow.RecordBatchReader` and iterate record batches: -# 3.3 Read as pyarrow.RecordBatchReader -record_batch_reader = table_read.to_arrow_batch_reader(splits) +```python +table_read = read_builder.new_read() +for batch in table_read.to_arrow_batch_reader(splits) + print(batch) + +# pyarrow.RecordBatch +# f0: int32 +# f1: string +# ---- +# f0: [1,2,3] +# f1: ["a","b","c"] +``` + +### Pandas + +This requires `pandas` to be installed. + +You can read all the data into a `pandas.DataFrame`: + +```python +table_read = read_builder.new_read() +df = table_read.to_pandas(splits) +print(df) + +# f0 f1 +# 0 1 a +# 1 2 b +# 2 3 c +# 3 4 d +# ... +``` + +### DuckDB + +This requires `duckdb` to be installed. + +You can convert the splits into a in-memory DuckDB table and query it: + +```python +table_read = read_builder.new_read() +duckdb_con = table_read.to_duckdb(splits, 'duckdb_table') + +print(duckdb_con.query("SELECT * FROM duckdb_table").fetchdf()) +# f0 f1 +# 0 1 a +# 1 2 b +# 2 3 c +# 3 4 d +# ... + +print(duckdb_con.query("SELECT * FROM duckdb_table WHERE f0 = 1").fetchdf()) +# f0 f1 +# 0 1 a +``` + +### Ray + +This requires `ray` to be installed. + +You can convert the splits into a Ray dataset and handle it by Ray API: + +```python +table_read = read_builder.new_read() +ray_dataset = table_read.to_ray(splits) + +print(ray_dataset) +# MaterializedDataset(num_blocks=1, num_rows=9, schema={f0: int32, f1: string}) + +print(ray_dataset.take(3)) +# [{'f0': 1, 'f1': 'a'}, {'f0': 2, 'f1': 'b'}, {'f0': 3, 'f1': 'c'}] + +print(ray_dataset.to_pandas()) +# f0 f1 +# 0 1 a +# 1 2 b +# 2 3 c +# 3 4 d +# ... ``` ## Batch Write @@ -246,12 +391,6 @@ table = catalog.get_table('database_name.table_name') # 1. Create table write and commit write_builder = table.new_batch_write_builder() -# By default, write data will be appended to table. -# If you want to overwrite table: -# write_builder.overwrite() -# If you want to overwrite partition 'dt=2024-01-01': -# write_builder.overwrite({'dt': '2024-01-01'}) - table_write = write_builder.new_write() table_commit = write_builder.new_commit() @@ -275,7 +414,16 @@ table_commit.commit(commit_messages) # 4. Close resources table_write.close() table_commit.close() +``` + +By default, the data will be appended to table. If you want to overwrite table, you should use `TableWrite#overwrite` API: + +```python +# overwrite whole table +write_builder.overwrite() +# overwrite partition 'dt=2024-01-01' +write_builder.overwrite({'dt': '2024-01-01'}) ``` ## Data Types @@ -290,49 +438,3 @@ table_commit.close() | pyarrow.float64() | DOUBLE | | pyarrow.string() | STRING | | pyarrow.boolean() | BOOLEAN | - -## Predicate - -You can use predicate to filter data when reading. Example: - -```python -# table data: -# f0: 0 1 2 3 4 -# f1: 5 6 7 8 9 -read_builder = table.new_read_builder() -predicate_builder = read_builder.new_predicate_builder() - -# build predicate: f0 < 3 && f1 > 5 -predicate1 = predicate_builder.less_than('f0', 1); -predicate2 = predicate_builder.greater_than('f1', 5); -predicate = predicate_builder.and_predicates([predicate1, predicate2]) - -read_builder = read_builder.with_filter(predicate) -table_scan = read_builder.new_scan() -table_read = read_builder.new_read() -splits = table_scan.plan().splits() -dataframe = table_read.to_pandas(splits) - -# result: -# f0: 1 2 -# f1: 6 7 -``` - -| Predicate kind | Predicate method | -|:----------------------|:----------------------------------------------| -| p1 and p2 | PredicateBuilder.and_predicates([p1, p2]) | -| p1 or p2 | PredicateBuilder.or_predicates([p1, p2]) | -| f = literal | PredicateBuilder.equal(f, literal) | -| f != literal | PredicateBuilder.not_equal(f, literal) | -| f < literal | PredicateBuilder.less_than(f, literal) | -| f <= literal | PredicateBuilder.less_or_equal(f, literal) | -| f > literal | PredicateBuilder.greater_than(f, literal) | -| f >= literal | PredicateBuilder.greater_or_equal(f, literal) | -| f is null | PredicateBuilder.is_null(f) | -| f is not null | PredicateBuilder.is_not_null(f) | -| f.startswith(literal) | PredicateBuilder.startswith(f, literal) | -| f.endswith(literal) | PredicateBuilder.endswith(f, literal) | -| f.contains(literal) | PredicateBuilder.contains(f, literal) | -| f is in [l1, l2] | PredicateBuilder.is_in(f, [l1, l2]) | -| f is not in [l1, l2] | PredicateBuilder.is_not_in(f, [l1, l2]) | -| lower <= f <= upper | PredicateBuilder.between(f, lower, upper) | From 419b02a836da34e2050a1d6c56a57e3ea32d7e99 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 2 Dec 2024 21:09:48 +0800 Subject: [PATCH 093/157] [iceberg] Introduce integration for AWS Glue (#4624) --- .../migration/iceberg-compatibility.md | 11 ++++++ .../AbstractIcebergCommitCallback.java | 35 +++++++++--------- .../apache/paimon/iceberg/IcebergOptions.java | 6 ++++ .../iceberg/IcebergHiveMetadataCommitter.java | 8 ++--- .../IcebergHive23MetadataCommitterITCase.java | 9 ++++- .../IcebergHive31MetadataCommitterITCase.java | 9 ++++- ...cebergHiveMetadataCommitterITCaseBase.java | 36 +++++++++++++++++++ 7 files changed, 92 insertions(+), 22 deletions(-) diff --git a/docs/content/migration/iceberg-compatibility.md b/docs/content/migration/iceberg-compatibility.md index d745607148643..01a03a45264dd 100644 --- a/docs/content/migration/iceberg-compatibility.md +++ b/docs/content/migration/iceberg-compatibility.md @@ -383,9 +383,20 @@ you also need to set some (or all) of the following table options when creating Boolean Should use the legacy manifest version to generate Iceberg's 1.4 manifest files. + +
    metadata.iceberg.hive-client-class
    + org.apache.hadoop.hive.metastore.HiveMetaStoreClient + String + Hive client class name for Iceberg Hive Catalog. + +## AWS Glue Catalog + +You can use Hive Catalog to connect AWS Glue metastore, you can use set `'metadata.iceberg.hive-client-class'` to +`'com.amazonaws.glue.catalog.metastore.AWSCatalogMetastoreClient'`. + ## AWS Athena AWS Athena may use old manifest reader to read Iceberg manifest by names, we should let Paimon producing legacy Iceberg diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java index 1b952c1716cf9..7ea6cbe05777e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java @@ -112,22 +112,7 @@ public AbstractIcebergCommitCallback(FileStoreTable table, String commitUser) { break; case HADOOP_CATALOG: case HIVE_CATALOG: - Path dbPath = table.location().getParent(); - final String dbSuffix = ".db"; - if (dbPath.getName().endsWith(dbSuffix)) { - String dbName = - dbPath.getName() - .substring(0, dbPath.getName().length() - dbSuffix.length()); - String tableName = table.location().getName(); - Path separatePath = - new Path( - dbPath.getParent(), - String.format("iceberg/%s/%s/metadata", dbName, tableName)); - this.pathFactory = new IcebergPathFactory(separatePath); - } else { - throw new UnsupportedOperationException( - "Storage type ICEBERG_WAREHOUSE can only be used on Paimon tables in a Paimon warehouse."); - } + this.pathFactory = new IcebergPathFactory(catalogTableMetadataPath(table)); break; default: throw new UnsupportedOperationException( @@ -152,6 +137,24 @@ public AbstractIcebergCommitCallback(FileStoreTable table, String commitUser) { this.manifestList = IcebergManifestList.create(table, pathFactory); } + public static Path catalogTableMetadataPath(FileStoreTable table) { + Path icebergDBPath = catalogDatabasePath(table); + return new Path(icebergDBPath, String.format("%s/metadata", table.location().getName())); + } + + public static Path catalogDatabasePath(FileStoreTable table) { + Path dbPath = table.location().getParent(); + final String dbSuffix = ".db"; + if (dbPath.getName().endsWith(dbSuffix)) { + String dbName = + dbPath.getName().substring(0, dbPath.getName().length() - dbSuffix.length()); + return new Path(dbPath.getParent(), String.format("iceberg/%s/", dbName)); + } else { + throw new UnsupportedOperationException( + "Storage type ICEBERG_WAREHOUSE can only be used on Paimon tables in a Paimon warehouse."); + } + } + @Override public void close() throws Exception {} diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java index c0ceed97ba8cf..4b59e29c8c333 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java @@ -84,6 +84,12 @@ public class IcebergOptions { .withDescription( "Should use the legacy manifest version to generate Iceberg's 1.4 manifest files."); + public static final ConfigOption HIVE_CLIENT_CLASS = + key("metadata.iceberg.hive-client-class") + .stringType() + .defaultValue("org.apache.hadoop.hive.metastore.HiveMetaStoreClient") + .withDescription("Hive client class name for Iceberg Hive Catalog."); + /** Where to store Iceberg metadata. */ public enum StorageType implements DescribedEnum { DISABLED("disabled", "Disable Iceberg compatibility support."), diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitter.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitter.java index d913f729e3518..ddd21384cbc83 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitter.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitter.java @@ -22,7 +22,6 @@ import org.apache.paimon.client.ClientPool; import org.apache.paimon.fs.Path; import org.apache.paimon.hive.HiveCatalog; -import org.apache.paimon.hive.HiveCatalogFactory; import org.apache.paimon.hive.HiveTypeUtils; import org.apache.paimon.hive.pool.CachedClientPool; import org.apache.paimon.options.Options; @@ -49,6 +48,8 @@ import java.util.HashMap; import java.util.stream.Collectors; +import static org.apache.paimon.iceberg.AbstractIcebergCommitCallback.catalogDatabasePath; + /** * {@link IcebergMetadataCommitter} to commit Iceberg metadata to Hive metastore, so the table can * be visited by Iceberg's Hive catalog. @@ -98,9 +99,7 @@ public IcebergHiveMetadataCommitter(FileStoreTable table) { this.clients = new CachedClientPool( - hiveConf, - options, - HiveCatalogFactory.METASTORE_CLIENT_CLASS.defaultValue()); + hiveConf, options, options.getString(IcebergOptions.HIVE_CLIENT_CLASS)); } @Override @@ -158,6 +157,7 @@ private boolean databaseExists(String databaseName) throws Exception { private void createDatabase(String databaseName) throws Exception { Database database = new Database(); database.setName(databaseName); + database.setLocationUri(catalogDatabasePath(table).toString()); clients.execute(client -> client.createDatabase(database)); } diff --git a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/iceberg/IcebergHive23MetadataCommitterITCase.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/iceberg/IcebergHive23MetadataCommitterITCase.java index a9e4ba945440e..7d726e75a17d2 100644 --- a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/iceberg/IcebergHive23MetadataCommitterITCase.java +++ b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/iceberg/IcebergHive23MetadataCommitterITCase.java @@ -18,5 +18,12 @@ package org.apache.paimon.iceberg; +import org.apache.paimon.hive.CreateFailHiveMetaStoreClient; + /** IT cases for {@link IcebergHiveMetadataCommitter} in Hive 2.3. */ -public class IcebergHive23MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase {} +public class IcebergHive23MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase { + @Override + protected String createFailHiveMetaStoreClient() { + return CreateFailHiveMetaStoreClient.class.getName(); + } +} diff --git a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/iceberg/IcebergHive31MetadataCommitterITCase.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/iceberg/IcebergHive31MetadataCommitterITCase.java index 6f4b0afd1ae12..0634adfad3576 100644 --- a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/iceberg/IcebergHive31MetadataCommitterITCase.java +++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/iceberg/IcebergHive31MetadataCommitterITCase.java @@ -18,5 +18,12 @@ package org.apache.paimon.iceberg; +import org.apache.paimon.hive.CreateFailHiveMetaStoreClient; + /** IT cases for {@link IcebergHiveMetadataCommitter} in Hive 3.1. */ -public class IcebergHive31MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase {} +public class IcebergHive31MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase { + @Override + protected String createFailHiveMetaStoreClient() { + return CreateFailHiveMetaStoreClient.class.getName(); + } +} diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterITCaseBase.java index fab22775751b0..d0c64c5d3b7fc 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterITCaseBase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterITCaseBase.java @@ -104,6 +104,12 @@ public void testPrimaryKeyTable() throws Exception { Row.of(2, 1, "cat"), Row.of(2, 2, "elephant")), collect(tEnv.executeSql("SELECT * FROM my_iceberg.test_db.t ORDER BY pt, id"))); + + Assert.assertTrue( + hiveShell + .executeQuery("DESC DATABASE EXTENDED test_db") + .toString() + .contains("iceberg/test_db")); } @Test @@ -150,6 +156,36 @@ public void testAppendOnlyTable() throws Exception { "SELECT data, id, pt FROM my_iceberg.test_db.t WHERE id > 1 ORDER BY pt, id"))); } + @Test + public void testCustomMetastoreClass() { + TableEnvironment tEnv = + TableEnvironmentImpl.create( + EnvironmentSettings.newInstance().inBatchMode().build()); + tEnv.executeSql( + "CREATE CATALOG my_paimon WITH ( 'type' = 'paimon', 'warehouse' = '" + + path + + "' )"); + tEnv.executeSql("CREATE DATABASE my_paimon.test_db"); + tEnv.executeSql( + String.format( + "CREATE TABLE my_paimon.test_db.t ( pt INT, id INT, data STRING ) PARTITIONED BY (pt) WITH " + + "( " + + "'metadata.iceberg.storage' = 'hive-catalog', " + + "'metadata.iceberg.uri' = '', " + + "'file.format' = 'avro', " + + "'metadata.iceberg.hive-client-class' = '%s')", + createFailHiveMetaStoreClient())); + Assert.assertThrows( + Exception.class, + () -> + tEnv.executeSql( + "INSERT INTO my_paimon.test_db.t VALUES " + + "(1, 1, 'apple'), (1, 2, 'pear'), (2, 1, 'cat'), (2, 2, 'dog')") + .await()); + } + + protected abstract String createFailHiveMetaStoreClient(); + private List collect(TableResult result) throws Exception { List rows = new ArrayList<>(); try (CloseableIterator it = result.collect()) { From 6fb887f47f2e79f6b3142f094b20b6d7a3f86846 Mon Sep 17 00:00:00 2001 From: yunfengzhou-hub Date: Mon, 2 Dec 2024 21:11:23 +0800 Subject: [PATCH 094/157] [flink] Avoid deprecated usage on TableSchema, DataType and DescriptorProperties (#4611) --- .../apache/paimon/flink/DataCatalogTable.java | 115 +++++++++++----- .../org/apache/paimon/flink/FlinkCatalog.java | 55 ++++---- .../paimon/flink/FlinkGenericCatalog.java | 6 - .../paimon/flink/SystemCatalogTable.java | 12 +- .../utils/FlinkCatalogPropertiesUtil.java | 102 ++++---------- .../utils/FlinkDescriptorProperties.java | 99 +++++++++++++ .../flink/FlinkCatalogPropertiesUtilTest.java | 130 +++++++++++++----- .../apache/paimon/flink/FlinkCatalogTest.java | 9 +- 8 files changed, 342 insertions(+), 186 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java index 019d7bd6892fe..e141581b476ba 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java @@ -23,33 +23,55 @@ import org.apache.paimon.types.DataField; import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.constraints.UniqueConstraint; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; -/** A {@link CatalogTableImpl} to wrap {@link FileStoreTable}. */ -public class DataCatalogTable extends CatalogTableImpl { +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A {@link CatalogTable} to wrap {@link FileStoreTable}. */ +public class DataCatalogTable implements CatalogTable { + // Schema of the table (column names and types) + private final Schema schema; + + // Partition keys if this is a partitioned table. It's an empty set if the table is not + // partitioned + private final List partitionKeys; + + // Properties of the table + private final Map options; + + // Comment of the table + private final String comment; private final Table table; private final Map nonPhysicalColumnComments; public DataCatalogTable( Table table, - TableSchema tableSchema, + Schema resolvedSchema, List partitionKeys, - Map properties, + Map options, String comment, Map nonPhysicalColumnComments) { - super(tableSchema, partitionKeys, properties, comment); + this.schema = resolvedSchema; + this.partitionKeys = checkNotNull(partitionKeys, "partitionKeys cannot be null"); + this.options = checkNotNull(options, "options cannot be null"); + + checkArgument( + options.entrySet().stream() + .allMatch(e -> e.getKey() != null && e.getValue() != null), + "properties cannot have null keys or values"); + + this.comment = comment; + this.table = table; this.nonPhysicalColumnComments = nonPhysicalColumnComments; } @@ -66,32 +88,30 @@ public Schema getUnresolvedSchema() { .filter(dataField -> dataField.description() != null) .collect(Collectors.toMap(DataField::name, DataField::description)); - return toSchema(getSchema(), columnComments); + return toSchema(schema, columnComments); } - /** Copied from {@link TableSchema#toSchema(Map)} to support versions lower than 1.17. */ - private Schema toSchema(TableSchema tableSchema, Map comments) { + private Schema toSchema(Schema tableSchema, Map comments) { final Schema.Builder builder = Schema.newBuilder(); - tableSchema - .getTableColumns() + .getColumns() .forEach( column -> { - if (column instanceof TableColumn.PhysicalColumn) { - final TableColumn.PhysicalColumn c = - (TableColumn.PhysicalColumn) column; - builder.column(c.getName(), c.getType()); - } else if (column instanceof TableColumn.MetadataColumn) { - final TableColumn.MetadataColumn c = - (TableColumn.MetadataColumn) column; + if (column instanceof Schema.UnresolvedPhysicalColumn) { + final Schema.UnresolvedPhysicalColumn c = + (Schema.UnresolvedPhysicalColumn) column; + builder.column(c.getName(), c.getDataType()); + } else if (column instanceof Schema.UnresolvedMetadataColumn) { + final Schema.UnresolvedMetadataColumn c = + (Schema.UnresolvedMetadataColumn) column; builder.columnByMetadata( c.getName(), - c.getType(), - c.getMetadataAlias().orElse(null), + c.getDataType(), + c.getMetadataKey(), c.isVirtual()); - } else if (column instanceof TableColumn.ComputedColumn) { - final TableColumn.ComputedColumn c = - (TableColumn.ComputedColumn) column; + } else if (column instanceof Schema.UnresolvedComputedColumn) { + final Schema.UnresolvedComputedColumn c = + (Schema.UnresolvedComputedColumn) column; builder.columnByExpression(c.getName(), c.getExpression()); } else { throw new IllegalArgumentException( @@ -104,19 +124,16 @@ private Schema toSchema(TableSchema tableSchema, Map comments) { builder.withComment(nonPhysicalColumnComments.get(colName)); } }); - tableSchema .getWatermarkSpecs() .forEach( spec -> builder.watermark( - spec.getRowtimeAttribute(), spec.getWatermarkExpr())); - + spec.getColumnName(), spec.getWatermarkExpression())); if (tableSchema.getPrimaryKey().isPresent()) { - UniqueConstraint primaryKey = tableSchema.getPrimaryKey().get(); - builder.primaryKeyNamed(primaryKey.getName(), primaryKey.getColumns()); + Schema.UnresolvedPrimaryKey primaryKey = tableSchema.getPrimaryKey().get(); + builder.primaryKeyNamed(primaryKey.getConstraintName(), primaryKey.getColumnNames()); } - return builder.build(); } @@ -124,7 +141,7 @@ private Schema toSchema(TableSchema tableSchema, Map comments) { public CatalogBaseTable copy() { return new DataCatalogTable( table, - getSchema().copy(), + schema, new ArrayList<>(getPartitionKeys()), new HashMap<>(getOptions()), getComment(), @@ -135,10 +152,40 @@ public CatalogBaseTable copy() { public CatalogTable copy(Map options) { return new DataCatalogTable( table, - getSchema(), + schema, getPartitionKeys(), options, getComment(), nonPhysicalColumnComments); } + + @Override + public Optional getDescription() { + return Optional.of(getComment()); + } + + @Override + public Optional getDetailedDescription() { + return Optional.of("This is a catalog table in an im-memory catalog"); + } + + @Override + public boolean isPartitioned() { + return !partitionKeys.isEmpty(); + } + + @Override + public List getPartitionKeys() { + return partitionKeys; + } + + @Override + public Map getOptions() { + return options; + } + + @Override + public String getComment() { + return comment != null ? comment : ""; + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index c67e79c1c06b7..3a7f9790ccca1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -24,6 +24,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.procedure.ProcedureUtil; import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; +import org.apache.paimon.flink.utils.FlinkDescriptorProperties; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.operation.FileStoreCommit; @@ -46,7 +47,6 @@ import org.apache.paimon.view.View; import org.apache.paimon.view.ViewImpl; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.AbstractCatalog; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabase; @@ -96,7 +96,6 @@ import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; -import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.factories.Factory; import org.apache.flink.table.procedures.Procedure; @@ -121,13 +120,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; -import static org.apache.flink.table.descriptors.DescriptorProperties.NAME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; import static org.apache.flink.table.utils.EncodingUtils.decodeBase64ToBytes; @@ -152,11 +144,18 @@ import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; import static org.apache.paimon.flink.log.LogStoreRegister.registerLogSystem; import static org.apache.paimon.flink.log.LogStoreRegister.unRegisterLogSystem; +import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeWatermarkSpec; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.nonPhysicalColumnsCount; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.serializeNewWatermarkSpec; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.COMMENT; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.NAME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_EXPR; import static org.apache.paimon.flink.utils.TableStatsUtil.createTableColumnStats; import static org.apache.paimon.flink.utils.TableStatsUtil.createTableStats; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -1008,18 +1007,18 @@ private static void validateAlterTable(CatalogBaseTable ct1, CatalogBaseTable ct } // materialized table is not resolved at this time. if (!table1IsMaterialized) { - org.apache.flink.table.api.TableSchema ts1 = ct1.getSchema(); - org.apache.flink.table.api.TableSchema ts2 = ct2.getSchema(); + org.apache.flink.table.api.Schema ts1 = ct1.getUnresolvedSchema(); + org.apache.flink.table.api.Schema ts2 = ct2.getUnresolvedSchema(); boolean pkEquality = false; if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { pkEquality = Objects.equals( - ts1.getPrimaryKey().get().getType(), - ts2.getPrimaryKey().get().getType()) + ts1.getPrimaryKey().get().getConstraintName(), + ts2.getPrimaryKey().get().getConstraintName()) && Objects.equals( - ts1.getPrimaryKey().get().getColumns(), - ts2.getPrimaryKey().get().getColumns()); + ts1.getPrimaryKey().get().getColumnNames(), + ts2.getPrimaryKey().get().getColumnNames()); } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { pkEquality = true; } @@ -1063,7 +1062,8 @@ public final void close() throws CatalogException { private CatalogBaseTable toCatalogTable(Table table) { Map newOptions = new HashMap<>(table.options()); - TableSchema.Builder builder = TableSchema.builder(); + org.apache.flink.table.api.Schema.Builder builder = + org.apache.flink.table.api.Schema.newBuilder(); Map nonPhysicalColumnComments = new HashMap<>(); // add columns @@ -1078,10 +1078,10 @@ private CatalogBaseTable toCatalogTable(Table table) { if (optionalName == null || physicalColumns.contains(optionalName)) { // build physical column from table row field RowType.RowField field = physicalRowFields.get(physicalColumnIndex++); - builder.field(field.getName(), fromLogicalToDataType(field.getType())); + builder.column(field.getName(), fromLogicalToDataType(field.getType())); } else { // build non-physical column from options - builder.add(deserializeNonPhysicalColumn(newOptions, i)); + deserializeNonPhysicalColumn(newOptions, i, builder); if (newOptions.containsKey(compoundKey(SCHEMA, i, COMMENT))) { nonPhysicalColumnComments.put( optionalName, newOptions.get(compoundKey(SCHEMA, i, COMMENT))); @@ -1093,22 +1093,18 @@ private CatalogBaseTable toCatalogTable(Table table) { // extract watermark information if (newOptions.keySet().stream() .anyMatch(key -> key.startsWith(compoundKey(SCHEMA, WATERMARK)))) { - builder.watermark(deserializeWatermarkSpec(newOptions)); + deserializeWatermarkSpec(newOptions, builder); } // add primary keys if (table.primaryKeys().size() > 0) { - builder.primaryKey( - table.primaryKeys().stream().collect(Collectors.joining("_", "PK_", "")), - table.primaryKeys().toArray(new String[0])); + builder.primaryKey(table.primaryKeys()); } - TableSchema schema = builder.build(); + org.apache.flink.table.api.Schema schema = builder.build(); // remove schema from options - DescriptorProperties removeProperties = new DescriptorProperties(false); - removeProperties.putTableSchema(SCHEMA, schema); - removeProperties.asMap().keySet().forEach(newOptions::remove); + FlinkDescriptorProperties.removeSchemaKeys(SCHEMA, schema, newOptions); Options options = Options.fromMap(newOptions); if (TableType.MATERIALIZED_TABLE == options.get(CoreOptions.TYPE)) { @@ -1124,7 +1120,10 @@ private CatalogBaseTable toCatalogTable(Table table) { } private CatalogMaterializedTable buildMaterializedTable( - Table table, Map newOptions, TableSchema schema, Options options) { + Table table, + Map newOptions, + org.apache.flink.table.api.Schema schema, + Options options) { String definitionQuery = options.get(MATERIALIZED_TABLE_DEFINITION_QUERY); IntervalFreshness freshness = IntervalFreshness.of( @@ -1148,7 +1147,7 @@ private CatalogMaterializedTable buildMaterializedTable( // remove materialized table related options allMaterializedTableAttributes().forEach(newOptions::remove); return CatalogMaterializedTable.newBuilder() - .schema(schema.toSchema()) + .schema(schema) .comment(table.comment().orElse("")) .partitionKeys(table.partitionKeys()) .options(newOptions) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java index 37bed2d0480f5..75af5917bb492 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java @@ -48,7 +48,6 @@ import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.factories.Factory; import org.apache.flink.table.factories.FunctionDefinitionFactory; -import org.apache.flink.table.factories.TableFactory; import org.apache.flink.table.procedures.Procedure; import java.util.List; @@ -86,11 +85,6 @@ public Optional getFactory() { new FlinkGenericTableFactory(paimon.getFactory().get(), flink.getFactory().get())); } - @Override - public Optional getTableFactory() { - return flink.getTableFactory(); - } - @Override public Optional getFunctionDefinitionFactory() { return flink.getFunctionDefinitionFactory(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java index d5d843d91bb10..f88a808713c24 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java @@ -22,7 +22,6 @@ import org.apache.paimon.table.system.AuditLogTable; import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.WatermarkSpec; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.types.utils.TypeConversions; @@ -32,11 +31,11 @@ import java.util.Map; import java.util.Optional; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; +import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeWatermarkSpec; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; /** A {@link CatalogTable} to represent system table. */ public class SystemCatalogTable implements CatalogTable { @@ -60,11 +59,8 @@ public Schema getUnresolvedSchema() { Map newOptions = new HashMap<>(table.options()); if (newOptions.keySet().stream() .anyMatch(key -> key.startsWith(compoundKey(SCHEMA, WATERMARK)))) { - WatermarkSpec watermarkSpec = deserializeWatermarkSpec(newOptions); - return builder.watermark( - watermarkSpec.getRowtimeAttribute(), - watermarkSpec.getWatermarkExpr()) - .build(); + deserializeWatermarkSpec(newOptions, builder); + return builder.build(); } } return builder.build(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java index b0f99a6e89e43..fa84a1ca070d0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java @@ -20,8 +20,7 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableSet; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.WatermarkSpec; +import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.types.DataType; @@ -36,48 +35,23 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; -import static org.apache.flink.table.descriptors.DescriptorProperties.DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; -import static org.apache.flink.table.descriptors.DescriptorProperties.METADATA; -import static org.apache.flink.table.descriptors.DescriptorProperties.NAME; -import static org.apache.flink.table.descriptors.DescriptorProperties.VIRTUAL; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.COMMENT; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.EXPR; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.METADATA; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.NAME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.VIRTUAL; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_EXPR; /** * Utilities for ser/deserializing non-physical columns and watermark into/from a map of string * properties. */ public class FlinkCatalogPropertiesUtil { - - public static Map serializeNonPhysicalColumns( - Map indexMap, List nonPhysicalColumns) { - Map serialized = new HashMap<>(); - for (TableColumn c : nonPhysicalColumns) { - int index = indexMap.get(c.getName()); - serialized.put(compoundKey(SCHEMA, index, NAME), c.getName()); - serialized.put( - compoundKey(SCHEMA, index, DATA_TYPE), - c.getType().getLogicalType().asSerializableString()); - if (c instanceof TableColumn.ComputedColumn) { - TableColumn.ComputedColumn computedColumn = (TableColumn.ComputedColumn) c; - serialized.put(compoundKey(SCHEMA, index, EXPR), computedColumn.getExpression()); - } else { - TableColumn.MetadataColumn metadataColumn = (TableColumn.MetadataColumn) c; - serialized.put( - compoundKey(SCHEMA, index, METADATA), - metadataColumn.getMetadataAlias().orElse(metadataColumn.getName())); - serialized.put( - compoundKey(SCHEMA, index, VIRTUAL), - Boolean.toString(metadataColumn.isVirtual())); - } - } - return serialized; - } + public static final String SCHEMA = "schema"; /** Serialize non-physical columns of new api. */ public static Map serializeNonPhysicalNewColumns(ResolvedSchema schema) { @@ -119,22 +93,6 @@ public static Map serializeNonPhysicalNewColumns(ResolvedSchema return serialized; } - public static Map serializeWatermarkSpec(WatermarkSpec watermarkSpec) { - Map serializedWatermarkSpec = new HashMap<>(); - String watermarkPrefix = compoundKey(SCHEMA, WATERMARK, 0); - serializedWatermarkSpec.put( - compoundKey(watermarkPrefix, WATERMARK_ROWTIME), - watermarkSpec.getRowtimeAttribute()); - serializedWatermarkSpec.put( - compoundKey(watermarkPrefix, WATERMARK_STRATEGY_EXPR), - watermarkSpec.getWatermarkExpr()); - serializedWatermarkSpec.put( - compoundKey(watermarkPrefix, WATERMARK_STRATEGY_DATA_TYPE), - watermarkSpec.getWatermarkExprOutputType().getLogicalType().asSerializableString()); - - return serializedWatermarkSpec; - } - public static Map serializeNewWatermarkSpec( org.apache.flink.table.catalog.WatermarkSpec watermarkSpec) { Map serializedWatermarkSpec = new HashMap<>(); @@ -219,7 +177,8 @@ private static boolean isColumnNameKey(String key) { && SCHEMA_COLUMN_NAME_SUFFIX.matcher(key.substring(SCHEMA.length() + 1)).matches(); } - public static TableColumn deserializeNonPhysicalColumn(Map options, int index) { + public static void deserializeNonPhysicalColumn( + Map options, int index, Schema.Builder builder) { String nameKey = compoundKey(SCHEMA, index, NAME); String dataTypeKey = compoundKey(SCHEMA, index, DATA_TYPE); String exprKey = compoundKey(SCHEMA, index, EXPR); @@ -227,45 +186,42 @@ public static TableColumn deserializeNonPhysicalColumn(Map optio String virtualKey = compoundKey(SCHEMA, index, VIRTUAL); String name = options.get(nameKey); - DataType dataType = - TypeConversions.fromLogicalToDataType( - LogicalTypeParser.parse(options.get(dataTypeKey))); - TableColumn column; if (options.containsKey(exprKey)) { - column = TableColumn.computed(name, dataType, options.get(exprKey)); + final String expr = options.get(exprKey); + builder.columnByExpression(name, expr); } else if (options.containsKey(metadataKey)) { String metadataAlias = options.get(metadataKey); boolean isVirtual = Boolean.parseBoolean(options.get(virtualKey)); - column = - metadataAlias.equals(name) - ? TableColumn.metadata(name, dataType, isVirtual) - : TableColumn.metadata(name, dataType, metadataAlias, isVirtual); + DataType dataType = + TypeConversions.fromLogicalToDataType( + LogicalTypeParser.parse( + options.get(dataTypeKey), + Thread.currentThread().getContextClassLoader())); + if (metadataAlias.equals(name)) { + builder.columnByMetadata(name, dataType, isVirtual); + } else { + builder.columnByMetadata(name, dataType, metadataAlias, isVirtual); + } } else { throw new RuntimeException( String.format( "Failed to build non-physical column. Current index is %s, options are %s", index, options)); } - - return column; } - public static WatermarkSpec deserializeWatermarkSpec(Map options) { + public static void deserializeWatermarkSpec( + Map options, Schema.Builder builder) { String watermarkPrefixKey = compoundKey(SCHEMA, WATERMARK); String rowtimeKey = compoundKey(watermarkPrefixKey, 0, WATERMARK_ROWTIME); String exprKey = compoundKey(watermarkPrefixKey, 0, WATERMARK_STRATEGY_EXPR); - String dataTypeKey = compoundKey(watermarkPrefixKey, 0, WATERMARK_STRATEGY_DATA_TYPE); String rowtimeAttribute = options.get(rowtimeKey); String watermarkExpressionString = options.get(exprKey); - DataType watermarkExprOutputType = - TypeConversions.fromLogicalToDataType( - LogicalTypeParser.parse(options.get(dataTypeKey))); - return new WatermarkSpec( - rowtimeAttribute, watermarkExpressionString, watermarkExprOutputType); + builder.watermark(rowtimeAttribute, watermarkExpressionString); } public static String compoundKey(Object... components) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java new file mode 100644 index 0000000000000..edc73ca7bf41f --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.utils; + +import org.apache.flink.table.api.Schema; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Utility class for having a unified string-based representation of Table API related classes such + * as Schema, TypeInformation, etc. + * + *

    Note to implementers: Please try to reuse key names as much as possible. Key-names should be + * hierarchical and lower case. Use "-" instead of dots or camel case. E.g., + * connector.schema.start-from = from-earliest. Try not to use the higher level in a key-name. E.g., + * instead of connector.kafka.kafka-version use connector.kafka.version. + * + *

    Properties with key normalization enabled contain only lower-case keys. + */ +public class FlinkDescriptorProperties { + + public static final String NAME = "name"; + + public static final String DATA_TYPE = "data-type"; + + public static final String EXPR = "expr"; + + public static final String METADATA = "metadata"; + + public static final String VIRTUAL = "virtual"; + + public static final String WATERMARK = "watermark"; + + public static final String WATERMARK_ROWTIME = "rowtime"; + + public static final String WATERMARK_STRATEGY = "strategy"; + + public static final String WATERMARK_STRATEGY_EXPR = WATERMARK_STRATEGY + '.' + EXPR; + + public static final String WATERMARK_STRATEGY_DATA_TYPE = WATERMARK_STRATEGY + '.' + DATA_TYPE; + + public static final String PRIMARY_KEY_NAME = "primary-key.name"; + + public static final String PRIMARY_KEY_COLUMNS = "primary-key.columns"; + + public static final String COMMENT = "comment"; + + public static void removeSchemaKeys(String key, Schema schema, Map options) { + checkNotNull(key); + checkNotNull(schema); + + List subKeys = Arrays.asList(NAME, DATA_TYPE, EXPR, METADATA, VIRTUAL); + for (int idx = 0; idx < schema.getColumns().size(); idx++) { + for (String subKey : subKeys) { + options.remove(key + '.' + idx + '.' + subKey); + } + } + + if (!schema.getWatermarkSpecs().isEmpty()) { + subKeys = + Arrays.asList( + WATERMARK_ROWTIME, + WATERMARK_STRATEGY_EXPR, + WATERMARK_STRATEGY_DATA_TYPE); + for (int idx = 0; idx < schema.getWatermarkSpecs().size(); idx++) { + for (String subKey : subKeys) { + options.remove(key + '.' + WATERMARK + '.' + idx + '.' + subKey); + } + } + } + + schema.getPrimaryKey() + .ifPresent( + pk -> { + options.remove(key + '.' + PRIMARY_KEY_NAME); + options.remove(key + '.' + PRIMARY_KEY_COLUMNS); + }); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java index 9268a236b6cb6..e32150b1fe822 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java @@ -21,27 +21,35 @@ import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.WatermarkSpec; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.WatermarkSpec; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionVisitor; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.SqlCallExpression; +import org.apache.flink.table.types.DataType; import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import static org.apache.flink.table.descriptors.DescriptorProperties.DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; -import static org.apache.flink.table.descriptors.DescriptorProperties.METADATA; -import static org.apache.flink.table.descriptors.DescriptorProperties.NAME; -import static org.apache.flink.table.descriptors.DescriptorProperties.VIRTUAL; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.EXPR; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.METADATA; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.NAME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.VIRTUAL; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_EXPR; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link FlinkCatalogPropertiesUtil}. */ @@ -49,18 +57,27 @@ public class FlinkCatalogPropertiesUtilTest { @Test public void testSerDeNonPhysicalColumns() { - Map indexMap = new HashMap<>(); - indexMap.put("comp", 2); - indexMap.put("meta1", 3); - indexMap.put("meta2", 5); - List columns = new ArrayList<>(); - columns.add(TableColumn.computed("comp", DataTypes.INT(), "`k` * 2")); - columns.add(TableColumn.metadata("meta1", DataTypes.VARCHAR(10))); - columns.add(TableColumn.metadata("meta2", DataTypes.BIGINT().notNull(), "price", true)); + List columns = new ArrayList<>(); + columns.add(new Schema.UnresolvedComputedColumn("comp", new SqlCallExpression("`k` * 2"))); + columns.add( + new Schema.UnresolvedMetadataColumn("meta1", DataTypes.VARCHAR(10), null, false)); + columns.add( + new Schema.UnresolvedMetadataColumn( + "meta2", DataTypes.BIGINT().notNull(), "price", true, null)); + + List resolvedColumns = new ArrayList<>(); + resolvedColumns.add(Column.physical("phy1", DataTypes.INT())); + resolvedColumns.add(Column.physical("phy2", DataTypes.INT())); + resolvedColumns.add( + Column.computed("comp", new TestResolvedExpression("`k` * 2", DataTypes.INT()))); + resolvedColumns.add(Column.metadata("meta1", DataTypes.VARCHAR(10), null, false)); + resolvedColumns.add(Column.physical("phy3", DataTypes.INT())); + resolvedColumns.add(Column.metadata("meta2", DataTypes.BIGINT().notNull(), "price", true)); // validate serialization Map serialized = - FlinkCatalogPropertiesUtil.serializeNonPhysicalColumns(indexMap, columns); + FlinkCatalogPropertiesUtil.serializeNonPhysicalNewColumns( + new ResolvedSchema(resolvedColumns, Collections.emptyList(), null)); Map expected = new HashMap<>(); expected.put(compoundKey(SCHEMA, 2, NAME), "comp"); @@ -80,27 +97,26 @@ public void testSerDeNonPhysicalColumns() { assertThat(serialized).containsExactlyInAnyOrderEntriesOf(expected); // validate deserialization - List deserialized = new ArrayList<>(); - deserialized.add(FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 2)); - deserialized.add(FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 3)); - deserialized.add(FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 5)); + Schema.Builder builder = Schema.newBuilder(); + FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 2, builder); + FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 3, builder); + FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 5, builder); - assertThat(deserialized).isEqualTo(columns); - - // validate that + assertThat(builder.build().getColumns()) + .containsExactly(columns.toArray(new Schema.UnresolvedColumn[0])); } @Test public void testSerDeWatermarkSpec() { WatermarkSpec watermarkSpec = - new WatermarkSpec( + WatermarkSpec.of( "test_time", - "`test_time` - INTERVAL '0.001' SECOND", - DataTypes.TIMESTAMP(3)); + new TestResolvedExpression( + "`test_time` - INTERVAL '0.001' SECOND", DataTypes.TIMESTAMP(3))); // validate serialization Map serialized = - FlinkCatalogPropertiesUtil.serializeWatermarkSpec(watermarkSpec); + FlinkCatalogPropertiesUtil.serializeNewWatermarkSpec(watermarkSpec); Map expected = new HashMap<>(); String watermarkPrefix = compoundKey(SCHEMA, WATERMARK, 0); @@ -113,9 +129,13 @@ public void testSerDeWatermarkSpec() { assertThat(serialized).containsExactlyInAnyOrderEntriesOf(expected); // validate serialization - WatermarkSpec deserialized = - FlinkCatalogPropertiesUtil.deserializeWatermarkSpec(serialized); - assertThat(deserialized).isEqualTo(watermarkSpec); + Schema.Builder builder = Schema.newBuilder(); + FlinkCatalogPropertiesUtil.deserializeWatermarkSpec(serialized, builder); + assertThat(builder.build().getWatermarkSpecs()).hasSize(1); + Schema.UnresolvedWatermarkSpec actual = builder.build().getWatermarkSpecs().get(0); + assertThat(actual.getColumnName()).isEqualTo(watermarkSpec.getRowtimeAttribute()); + assertThat(actual.getWatermarkExpression().asSummaryString()) + .isEqualTo(watermarkSpec.getWatermarkExpression().asSummaryString()); } @Test @@ -150,4 +170,44 @@ public void testNonPhysicalColumnsCount() { oldStyleOptions, Arrays.asList("phy1", "phy2"))) .isEqualTo(3); } + + private static class TestResolvedExpression implements ResolvedExpression { + private final String name; + private final DataType outputDataType; + + private TestResolvedExpression(String name, DataType outputDataType) { + this.name = name; + this.outputDataType = outputDataType; + } + + @Override + public DataType getOutputDataType() { + return outputDataType; + } + + @Override + public List getResolvedChildren() { + return Collections.emptyList(); + } + + @Override + public String asSummaryString() { + return new SqlCallExpression(name).asSummaryString(); + } + + @Override + public String asSerializableString() { + return name; + } + + @Override + public List getChildren() { + return Collections.emptyList(); + } + + @Override + public R accept(ExpressionVisitor expressionVisitor) { + return null; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index 27a89510975f3..e4286eb18172d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -850,7 +850,7 @@ private static void checkEquals(CatalogBaseTable t1, CatalogBaseTable t2) { assertThat(t2.getComment()).isEqualTo(t1.getComment()); assertThat(t2.getOptions()).isEqualTo(t1.getOptions()); if (t1.getTableKind() == CatalogBaseTable.TableKind.TABLE) { - assertThat(t2.getSchema()).isEqualTo(t1.getSchema()); + assertThat(t2.getUnresolvedSchema()).isEqualTo(t1.getUnresolvedSchema()); assertThat(((CatalogTable) (t2)).getPartitionKeys()) .isEqualTo(((CatalogTable) (t1)).getPartitionKeys()); assertThat(((CatalogTable) (t2)).isPartitioned()) @@ -864,7 +864,12 @@ private static void checkEquals(CatalogBaseTable t1, CatalogBaseTable t2) { t2.getUnresolvedSchema() .resolve(new TestSchemaResolver())) .build()) - .isEqualTo(t1.getSchema().toSchema()); + .isEqualTo( + Schema.newBuilder() + .fromResolvedSchema( + t1.getUnresolvedSchema() + .resolve(new TestSchemaResolver())) + .build()); assertThat(mt2.getPartitionKeys()).isEqualTo(mt1.getPartitionKeys()); assertThat(mt2.isPartitioned()).isEqualTo(mt1.isPartitioned()); // validate definition query From 3c820828062abef86b278f5a0334b6e65570c54b Mon Sep 17 00:00:00 2001 From: WenjunMin Date: Mon, 2 Dec 2024 21:42:22 +0800 Subject: [PATCH 095/157] [core] Make default of 'lookup.local-file-type' to sort (#4622) --- docs/layouts/shortcodes/generated/core_configuration.html | 2 +- .../paimon/benchmark/lookup/AbstractLookupBenchmark.java | 5 ++++- .../paimon/benchmark/lookup/LookupReaderBenchmark.java | 2 +- .../src/main/java/org/apache/paimon/CoreOptions.java | 2 +- .../main/java/org/apache/paimon/io/cache/CacheBuilder.java | 4 ++++ 5 files changed, 11 insertions(+), 4 deletions(-) diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index fad1f4907e5a9..2ad5db28b9ba1 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -443,7 +443,7 @@

    lookup.local-file-type
    - hash + sort

    Enum

    The local file type for lookup.

    Possible values:
    • "sort": Construct a sorted file for lookup.
    • "hash": Construct a hash file for lookup.
    diff --git a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/lookup/AbstractLookupBenchmark.java b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/lookup/AbstractLookupBenchmark.java index 635d876f7a985..653bfee6cc00f 100644 --- a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/lookup/AbstractLookupBenchmark.java +++ b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/lookup/AbstractLookupBenchmark.java @@ -102,7 +102,10 @@ protected Pair writeData( new CacheManager(MemorySize.ofMebiBytes(10)), keySerializer.createSliceComparator()); - File file = new File(tempDir.toFile(), UUID.randomUUID().toString()); + String name = + String.format( + "%s-%s-%s", options.lookupLocalFileType(), valueLength, bloomFilterEnabled); + File file = new File(tempDir.toFile(), UUID.randomUUID() + "-" + name); LookupStoreWriter writer = factory.createWriter(file, createBloomFiler(bloomFilterEnabled)); int i = 0; for (byte[] input : inputs) { diff --git a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/lookup/LookupReaderBenchmark.java b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/lookup/LookupReaderBenchmark.java index 9947b54a70f0c..2d8de84327d46 100644 --- a/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/lookup/LookupReaderBenchmark.java +++ b/paimon-benchmark/paimon-micro-benchmarks/src/test/java/org/apache/paimon/benchmark/lookup/LookupReaderBenchmark.java @@ -128,7 +128,7 @@ private void readData( LookupStoreFactory factory = LookupStoreFactory.create( options, - new CacheManager(MemorySize.ofMebiBytes(10)), + new CacheManager(MemorySize.ofMebiBytes(20), 0.5), new RowCompactedSerializer(RowType.of(new IntType())) .createSliceComparator()); diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index b9b5675f1d2d5..cddef33c276e5 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -891,7 +891,7 @@ public class CoreOptions implements Serializable { public static final ConfigOption LOOKUP_LOCAL_FILE_TYPE = key("lookup.local-file-type") .enumType(LookupLocalFileType.class) - .defaultValue(LookupLocalFileType.HASH) + .defaultValue(LookupLocalFileType.SORT) .withDescription("The local file type for lookup."); public static final ConfigOption LOOKUP_HASH_LOAD_FACTOR = diff --git a/paimon-common/src/main/java/org/apache/paimon/io/cache/CacheBuilder.java b/paimon-common/src/main/java/org/apache/paimon/io/cache/CacheBuilder.java index 4660343d45e1e..402f21f06264d 100644 --- a/paimon-common/src/main/java/org/apache/paimon/io/cache/CacheBuilder.java +++ b/paimon-common/src/main/java/org/apache/paimon/io/cache/CacheBuilder.java @@ -72,6 +72,10 @@ public Cache build() { org.apache.paimon.shade.guava30.com.google.common.cache.CacheBuilder .newBuilder() .weigher(CacheBuilder::weigh) + // The concurrency level determines the number of segment caches in + // Guava,limiting the maximum block entries held in cache. Since we do + // not access this cache concurrently, it is set to 1. + .concurrencyLevel(1) .maximumWeight(memorySize.getBytes()) .removalListener(this::onRemoval) .build()); From 039046a0d4d4aa4195f9187b2d0214f277316ce8 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Mon, 2 Dec 2024 22:52:10 +0800 Subject: [PATCH 096/157] [core] Extract decompressBlock method in SortLookupStoreReader --- .../lookup/sort/SortLookupStoreReader.java | 65 +++++++++---------- 1 file changed, 32 insertions(+), 33 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/lookup/sort/SortLookupStoreReader.java b/paimon-common/src/main/java/org/apache/paimon/lookup/sort/SortLookupStoreReader.java index 39997888ce92f..6dbfe130e3bb0 100644 --- a/paimon-common/src/main/java/org/apache/paimon/lookup/sort/SortLookupStoreReader.java +++ b/paimon-common/src/main/java/org/apache/paimon/lookup/sort/SortLookupStoreReader.java @@ -106,7 +106,7 @@ public byte[] lookup(byte[] key) throws IOException { return null; } - private BlockIterator getNextBlock() throws IOException { + private BlockIterator getNextBlock() { // index block handle, point to the key, value position. MemorySlice blockHandle = indexBlockIterator.next().getValue(); BlockReader dataBlock = @@ -134,42 +134,41 @@ private BlockReader readBlock(BlockHandle blockHandle, boolean index) { blockCache.getBlock( blockHandle.offset(), blockHandle.size(), - bytes -> { - MemorySegment block = MemorySegment.wrap(bytes); - int crc32cCode = crc32c(block, blockTrailer.getCompressionType()); - checkArgument( - blockTrailer.getCrc32c() == crc32cCode, - String.format( - "Expected CRC32C(%d) but found CRC32C(%d) for file(%s)", - blockTrailer.getCrc32c(), crc32cCode, filePath)); - - // decompress data - BlockCompressionFactory compressionFactory = - BlockCompressionFactory.create( - blockTrailer.getCompressionType()); - if (compressionFactory == null) { - return bytes; - } else { - MemorySliceInput compressedInput = - MemorySlice.wrap(block).toInput(); - byte[] uncompressed = new byte[compressedInput.readVarLenInt()]; - BlockDecompressor decompressor = - compressionFactory.getDecompressor(); - int uncompressedLength = - decompressor.decompress( - block.getHeapMemory(), - compressedInput.position(), - compressedInput.available(), - uncompressed, - 0); - checkArgument(uncompressedLength == uncompressed.length); - return uncompressed; - } - }, + bytes -> decompressBlock(bytes, blockTrailer), index); return new BlockReader(MemorySlice.wrap(unCompressedBlock), comparator); } + private byte[] decompressBlock(byte[] compressedBytes, BlockTrailer blockTrailer) { + MemorySegment compressed = MemorySegment.wrap(compressedBytes); + int crc32cCode = crc32c(compressed, blockTrailer.getCompressionType()); + checkArgument( + blockTrailer.getCrc32c() == crc32cCode, + String.format( + "Expected CRC32C(%d) but found CRC32C(%d) for file(%s)", + blockTrailer.getCrc32c(), crc32cCode, filePath)); + + // decompress data + BlockCompressionFactory compressionFactory = + BlockCompressionFactory.create(blockTrailer.getCompressionType()); + if (compressionFactory == null) { + return compressedBytes; + } else { + MemorySliceInput compressedInput = MemorySlice.wrap(compressed).toInput(); + byte[] uncompressed = new byte[compressedInput.readVarLenInt()]; + BlockDecompressor decompressor = compressionFactory.getDecompressor(); + int uncompressedLength = + decompressor.decompress( + compressed.getHeapMemory(), + compressedInput.position(), + compressedInput.available(), + uncompressed, + 0); + checkArgument(uncompressedLength == uncompressed.length); + return uncompressed; + } + } + @Override public void close() throws IOException { if (bloomFilter != null) { From 300cc67c208c4b86e2edf58ad1981b86649fe892 Mon Sep 17 00:00:00 2001 From: Yann Byron Date: Tue, 3 Dec 2024 11:35:16 +0800 Subject: [PATCH 097/157] [spark] show table extended (#4603) * [spark] show table extended * update * [update] doc --- docs/content/spark/auxiliary.md | 11 ++ .../spark/PaimonPartitionManagement.scala | 2 +- .../analysis/PaimonResolvePartitionSpec.scala | 75 +++++++++++ .../PaimonSparkSessionExtensions.scala | 2 + .../org/apache/spark/sql/PaimonUtils.scala | 19 +++ .../catalog/PaimonCatalogImplicits.scala | 30 +++++ .../catalog/PaimonCatalogUtils.scala | 3 + .../spark/sql/paimon/shims/SparkShim.scala | 4 + .../paimon/spark/sql/DescribeTableTest.scala | 70 ++++++++++ .../analysis/Spark3ResolutionRules.scala | 56 ++++++++ .../PaimonShowTablePartitionCommand.scala | 96 ++++++++++++++ .../PaimonShowTablesExtendedCommand.scala | 123 ++++++++++++++++++ .../spark/sql/paimon/shims/Spark3Shim.scala | 8 +- .../analysis/Spark4ResolutionRules.scala | 27 ++++ .../spark/sql/paimon/shims/Spark4Shim.scala | 9 +- 15 files changed, 532 insertions(+), 3 deletions(-) create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonResolvePartitionSpec.scala create mode 100644 paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogImplicits.scala create mode 100644 paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark3ResolutionRules.scala create mode 100644 paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablePartitionCommand.scala create mode 100644 paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablesExtendedCommand.scala create mode 100644 paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark4ResolutionRules.scala diff --git a/docs/content/spark/auxiliary.md b/docs/content/spark/auxiliary.md index 6330ca27ce312..5de0289565f2f 100644 --- a/docs/content/spark/auxiliary.md +++ b/docs/content/spark/auxiliary.md @@ -96,6 +96,17 @@ SHOW PARTITIONS my_table; SHOW PARTITIONS my_table PARTITION (dt=20230817); ``` +## Show Table Extended +The SHOW TABLE EXTENDED statement is used to list table or partition information. + +```sql +-- Lists tables that satisfy regular expressions +SHOW TABLE EXTENDED IN db_name LIKE 'test*'; + +-- Lists the specified partition information for the table +SHOW TABLE EXTENDED IN db_name LIKE 'table_name' PARTITION(pt = '2024'); +``` + ## Analyze table The ANALYZE TABLE statement collects statistics about the table, that are to be used by the query optimizer to find a better query execution plan. diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala index 9a305ca59a0f1..840f1341a69d7 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala @@ -100,7 +100,7 @@ trait PaimonPartitionManagement extends SupportsAtomicPartitionManagement { } override def loadPartitionMetadata(ident: InternalRow): JMap[String, String] = { - throw new UnsupportedOperationException("Load partition is not supported") + Map.empty[String, String].asJava } override def listPartitionIdentifiers( diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonResolvePartitionSpec.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonResolvePartitionSpec.scala new file mode 100644 index 0000000000000..5d6a5a063c06d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonResolvePartitionSpec.scala @@ -0,0 +1,75 @@ +/* + * 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.paimon.spark.catalyst.analysis + +import org.apache.spark.sql.PaimonUtils.{normalizePartitionSpec, requireExactMatchedPartitionSpec} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.{PartitionSpec, ResolvedPartitionSpec, UnresolvedPartitionSpec} +import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec.conf +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} +import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +object PaimonResolvePartitionSpec { + + def resolve( + catalog: TableCatalog, + tableIndent: Identifier, + partitionSpec: PartitionSpec): ResolvedPartitionSpec = { + val table = catalog.loadTable(tableIndent).asPartitionable + partitionSpec match { + case u: UnresolvedPartitionSpec => + val partitionSchema = table.partitionSchema() + resolvePartitionSpec(table.name(), u, partitionSchema, allowPartitionSpec = false) + case o => o.asInstanceOf[ResolvedPartitionSpec] + } + } + + private def resolvePartitionSpec( + tableName: String, + partSpec: UnresolvedPartitionSpec, + partSchema: StructType, + allowPartitionSpec: Boolean): ResolvedPartitionSpec = { + val normalizedSpec = normalizePartitionSpec(partSpec.spec, partSchema, tableName, conf.resolver) + if (!allowPartitionSpec) { + requireExactMatchedPartitionSpec(tableName, normalizedSpec, partSchema.fieldNames) + } + val partitionNames = normalizedSpec.keySet + val requestedFields = partSchema.filter(field => partitionNames.contains(field.name)) + ResolvedPartitionSpec( + requestedFields.map(_.name), + convertToPartIdent(normalizedSpec, requestedFields), + partSpec.location) + } + + def convertToPartIdent( + partitionSpec: TablePartitionSpec, + schema: Seq[StructField]): InternalRow = { + val partValues = schema.map { + part => + val raw = partitionSpec.get(part.name).orNull + val dt = CharVarcharUtils.replaceCharVarcharWithString(part.dataType) + Cast(Literal.create(raw, StringType), dt, Some(conf.sessionLocalTimeZone)).eval() + } + InternalRow.fromSeq(partValues) + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala index e8f75d394a818..f73df64fb8ab6 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala @@ -40,6 +40,8 @@ class PaimonSparkSessionExtensions extends (SparkSessionExtensions => Unit) { extensions.injectResolutionRule(spark => new PaimonAnalysis(spark)) extensions.injectResolutionRule(spark => PaimonProcedureResolver(spark)) extensions.injectResolutionRule(spark => PaimonViewResolver(spark)) + extensions.injectResolutionRule( + spark => SparkShimLoader.getSparkShim.createCustomResolution(spark)) extensions.injectResolutionRule(spark => PaimonIncompatibleResolutionRules(spark)) extensions.injectPostHocResolutionRule(spark => PaimonPostHocResolutionRules(spark)) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala index 4492d856ad509..cc49e787dc81c 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala @@ -20,11 +20,15 @@ package org.apache.spark.sql import org.apache.spark.executor.OutputMetrics import org.apache.spark.rdd.InputFileBlockHolder +import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.expressions.{FieldReference, NamedReference} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.PartitioningUtils import org.apache.spark.util.{Utils => SparkUtils} /** @@ -87,4 +91,19 @@ object PaimonUtils { outputMetrics.setBytesWritten(bytesWritten) outputMetrics.setRecordsWritten(recordsWritten) } + + def normalizePartitionSpec[T]( + partitionSpec: Map[String, T], + partCols: StructType, + tblName: String, + resolver: Resolver): Map[String, T] = { + PartitioningUtils.normalizePartitionSpec(partitionSpec, partCols, tblName, resolver) + } + + def requireExactMatchedPartitionSpec( + tableName: String, + spec: TablePartitionSpec, + partitionColumnNames: Seq[String]): Unit = { + PartitioningUtils.requireExactMatchedPartitionSpec(tableName, spec, partitionColumnNames) + } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogImplicits.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogImplicits.scala new file mode 100644 index 0000000000000..f1f20fb6fb31d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogImplicits.scala @@ -0,0 +1,30 @@ +/* + * 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.spark.sql.connector.catalog + +object PaimonCatalogImplicits { + + import CatalogV2Implicits._ + + implicit class PaimonCatalogHelper(plugin: CatalogPlugin) extends CatalogHelper(plugin) + + implicit class PaimonNamespaceHelper(namespace: Array[String]) extends NamespaceHelper(namespace) + +// implicit class PaimonTableHelper(table: Table) extends TableHelper(table) +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala index 2ab3dc4945245..5db6894ba093b 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.ExternalCatalog +import org.apache.spark.sql.connector.catalog.CatalogV2Util import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.paimon.ReflectUtils @@ -40,4 +41,6 @@ object PaimonCatalogUtils { hadoopConf) } + val TABLE_RESERVED_PROPERTIES: Seq[String] = CatalogV2Util.TABLE_RESERVED_PROPERTIES + } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala index bd85282737e99..334bd6e93180e 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala @@ -24,6 +24,8 @@ import org.apache.paimon.types.{DataType, RowType} import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType @@ -39,6 +41,8 @@ trait SparkShim { def createSparkParser(delegate: ParserInterface): ParserInterface + def createCustomResolution(spark: SparkSession): Rule[LogicalPlan] + def createSparkInternalRow(rowType: RowType): SparkInternalRow def createSparkArrayData(elementType: DataType): SparkArrayData diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala index 528dcd3cd107a..ae538fa48c4e2 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala @@ -27,6 +27,76 @@ import java.util.Objects class DescribeTableTest extends PaimonSparkTestBase { + test("Paimon show: show table extended") { + val testDB = "test_show" + withDatabase(testDB) { + spark.sql("CREATE TABLE s1 (id INT)") + + spark.sql(s"CREATE DATABASE $testDB") + spark.sql(s"USE $testDB") + spark.sql("CREATE TABLE s2 (id INT, pt STRING) PARTITIONED BY (pt)") + spark.sql("CREATE TABLE s3 (id INT, pt1 STRING, pt2 STRING) PARTITIONED BY (pt1, pt2)") + + spark.sql("INSERT INTO s2 VALUES (1, '2024'), (2, '2024'), (3, '2025'), (4, '2026')") + spark.sql(""" + |INSERT INTO s3 + |VALUES + |(1, '2024', '11'), (2, '2024', '12'), (3, '2025', '11'), (4, '2025', '12') + |""".stripMargin) + + // SHOW TABL EXTENDED will give four columns: namespace, tableName, isTemporary, information. + checkAnswer( + sql(s"SHOW TABLE EXTENDED IN $dbName0 LIKE '*'") + .select("namespace", "tableName", "isTemporary"), + Row("test", "s1", false)) + checkAnswer( + sql(s"SHOW TABLE EXTENDED IN $testDB LIKE '*'") + .select("namespace", "tableName", "isTemporary"), + Row(testDB, "s2", false) :: Row(testDB, "s3", false) :: Nil + ) + + // check table s1 + val res1 = spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's2'").select("information") + Assertions.assertEquals(1, res1.count()) + val information1 = res1 + .collect() + .head + .getString(0) + .split("\n") + .map { + line => + val kv = line.split(": ", 2) + kv(0) -> kv(1) + } + .toMap + Assertions.assertEquals(information1("Catalog"), "paimon") + Assertions.assertEquals(information1("Namespace"), testDB) + Assertions.assertEquals(information1("Table"), "s2") + Assertions.assertEquals(information1("Provider"), "paimon") + Assertions.assertEquals(information1("Location"), loadTable(testDB, "s2").location().toString) + + // check table s2 partition info + val error1 = intercept[Exception] { + spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's2' PARTITION(pt='2022')") + }.getMessage + assert(error1.contains("PARTITIONS_NOT_FOUND")) + + val error2 = intercept[Exception] { + spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's3' PARTITION(pt1='2024')") + }.getMessage + assert(error2.contains("Partition spec is invalid")) + + val res2 = + spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's3' PARTITION(pt1 = '2024', pt2 = 11)") + checkAnswer( + res2.select("namespace", "tableName", "isTemporary"), + Row(testDB, "s3", false) + ) + Assertions.assertTrue( + res2.select("information").collect().head.getString(0).contains("Partition Values")) + } + } + test(s"Paimon describe: describe table comment") { var comment = "test comment" spark.sql(s""" diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark3ResolutionRules.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark3ResolutionRules.scala new file mode 100644 index 0000000000000..924df2d1e3206 --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark3ResolutionRules.scala @@ -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.paimon.spark.catalyst.analysis + +import org.apache.paimon.spark.commands.{PaimonShowTablePartitionCommand, PaimonShowTablesExtendedCommand} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.{PartitionSpec, ResolvedNamespace, UnresolvedPartitionSpec} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ShowTableExtended} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.Identifier + +case class Spark3ResolutionRules(session: SparkSession) + extends Rule[LogicalPlan] + with SQLConfHelper { + + import org.apache.spark.sql.connector.catalog.PaimonCatalogImplicits._ + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown { + case ShowTableExtended( + ResolvedNamespace(catalog, ns), + pattern, + partitionSpec @ (None | Some(UnresolvedPartitionSpec(_, _))), + output) => + partitionSpec + .map { + spec: PartitionSpec => + val table = Identifier.of(ns.toArray, pattern) + val resolvedSpec = + PaimonResolvePartitionSpec.resolve(catalog.asTableCatalog, table, spec) + PaimonShowTablePartitionCommand(output, catalog.asTableCatalog, table, resolvedSpec) + } + .getOrElse { + PaimonShowTablesExtendedCommand(catalog.asTableCatalog, ns, pattern, output) + } + + } + +} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablePartitionCommand.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablePartitionCommand.scala new file mode 100644 index 0000000000000..32f94985859cf --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablePartitionCommand.scala @@ -0,0 +1,96 @@ +/* + * 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.paimon.spark.commands + +import org.apache.paimon.spark.leafnode.PaimonLeafRunnableCommand + +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName +import org.apache.spark.sql.catalyst.expressions.{Attribute, ToPrettyString} +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.connector.catalog.{Identifier, SupportsPartitionManagement, TableCatalog} +import org.apache.spark.sql.connector.catalog.PaimonCatalogImplicits._ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +case class PaimonShowTablePartitionCommand( + override val output: Seq[Attribute], + catalog: TableCatalog, + tableIndent: Identifier, + partSpec: ResolvedPartitionSpec) + extends PaimonLeafRunnableCommand { + override def run(sparkSession: SparkSession): Seq[Row] = { + val rows = new mutable.ArrayBuffer[Row]() + val table = catalog.loadTable(tableIndent) + val information = getTablePartitionDetails(tableIndent, table.asPartitionable, partSpec) + rows += Row(tableIndent.namespace.quoted, tableIndent.name(), false, s"$information\n") + + rows.toSeq + } + + private def getTablePartitionDetails( + tableIdent: Identifier, + partitionTable: SupportsPartitionManagement, + partSpec: ResolvedPartitionSpec): String = { + val results = new mutable.LinkedHashMap[String, String]() + + // "Partition Values" + val partitionSchema = partitionTable.partitionSchema() + val (names, ident) = (partSpec.names, partSpec.ident) + val partitionIdentifiers = partitionTable.listPartitionIdentifiers(names.toArray, ident) + if (partitionIdentifiers.isEmpty) { + val part = ident + .toSeq(partitionSchema) + .zip(partitionSchema.map(_.name)) + .map(kv => s"${kv._2}" + s" = ${kv._1}") + .mkString(", ") + throw new RuntimeException( + s""" + |[PARTITIONS_NOT_FOUND] The partition(s) PARTITION ($part) cannot be found in table ${tableIdent.toString}. + |Verify the partition specification and table name. + |""".stripMargin) + } + assert(partitionIdentifiers.length == 1) + val row = partitionIdentifiers.head + val len = partitionSchema.length + val partitions = new Array[String](len) + val timeZoneId = conf.sessionLocalTimeZone + for (i <- 0 until len) { + val dataType = partitionSchema(i).dataType + val partValueUTF8String = + ToPrettyString(Literal(row.get(i, dataType), dataType), Some(timeZoneId)).eval(null) + val partValueStr = if (partValueUTF8String == null) "null" else partValueUTF8String.toString + partitions(i) = escapePathName(partitionSchema(i).name) + "=" + escapePathName(partValueStr) + } + val partitionValues = partitions.mkString("[", ", ", "]") + results.put("Partition Values", s"$partitionValues") + + // TODO "Partition Parameters", "Created Time", "Last Access", "Partition Statistics" + + results + .map { + case (key, value) => + if (value.isEmpty) key else s"$key: $value" + } + .mkString("", "\n", "") + } +} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablesExtendedCommand.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablesExtendedCommand.scala new file mode 100644 index 0000000000000..b393982e25d3e --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablesExtendedCommand.scala @@ -0,0 +1,123 @@ +/* + * 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.paimon.spark.commands + +import org.apache.paimon.spark.leafnode.PaimonLeafRunnableCommand + +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.{QuotingUtils, StringUtils} +import org.apache.spark.sql.connector.catalog.{Identifier, PaimonCatalogUtils, SupportsPartitionManagement, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.PaimonCatalogImplicits._ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +case class PaimonShowTablesExtendedCommand( + catalog: TableCatalog, + namespace: Seq[String], + pattern: String, + override val output: Seq[Attribute], + isExtended: Boolean = false, + partitionSpec: Option[TablePartitionSpec] = None) + extends PaimonLeafRunnableCommand { + + override def run(spark: SparkSession): Seq[Row] = { + val rows = new mutable.ArrayBuffer[Row]() + + val tables = catalog.listTables(namespace.toArray) + tables.map { + tableIdent: Identifier => + if (StringUtils.filterPattern(Seq(tableIdent.name()), pattern).nonEmpty) { + val table = catalog.loadTable(tableIdent) + val information = getTableDetails(catalog.name, tableIdent, table) + rows += Row(tableIdent.namespace().quoted, tableIdent.name(), false, s"$information\n") + } + } + + // TODO: view + + rows.toSeq + } + + private def getTableDetails(catalogName: String, identifier: Identifier, table: Table): String = { + val results = new mutable.LinkedHashMap[String, String]() + + results.put("Catalog", catalogName) + results.put("Namespace", identifier.namespace().quoted) + results.put("Table", identifier.name()) + val tableType = if (table.properties().containsKey(TableCatalog.PROP_EXTERNAL)) { + CatalogTableType.EXTERNAL + } else { + CatalogTableType.MANAGED + } + results.put("Type", tableType.name) + + PaimonCatalogUtils.TABLE_RESERVED_PROPERTIES + .filterNot(_ == TableCatalog.PROP_EXTERNAL) + .foreach( + propKey => { + if (table.properties.containsKey(propKey)) { + results.put(propKey.capitalize, table.properties.get(propKey)) + } + }) + + val properties: Seq[String] = + conf + .redactOptions(table.properties.asScala.toMap) + .toList + .filter(kv => !PaimonCatalogUtils.TABLE_RESERVED_PROPERTIES.contains(kv._1)) + .sortBy(_._1) + .map { case (key, value) => key + "=" + value } + if (!table.properties().isEmpty) { + results.put("Table Properties", properties.mkString("[", ", ", "]")) + } + + // Partition Provider & Partition Columns + if (supportsPartitions(table) && table.asPartitionable.partitionSchema().nonEmpty) { + results.put("Partition Provider", "Catalog") + results.put( + "Partition Columns", + table.asPartitionable + .partitionSchema() + .map(field => QuotingUtils.quoteIdentifier(field.name)) + .mkString("[", ", ", "]")) + } + + if (table.schema().nonEmpty) { + results.put("Schema", table.schema().treeString) + } + + results + .map { + case (key, value) => + if (value.isEmpty) key else s"$key: $value" + } + .mkString("", "\n", "") + } + + private def supportsPartitions(table: Table): Boolean = table match { + case _: SupportsPartitionManagement => true + case _ => false + } + +} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala index 57d79d6474e9a..f508e2605cbc1 100644 --- a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.paimon.shims +import org.apache.paimon.spark.catalyst.analysis.Spark3ResolutionRules import org.apache.paimon.spark.catalyst.parser.extensions.PaimonSpark3SqlExtensionsParser import org.apache.paimon.spark.data.{Spark3ArrayData, Spark3InternalRow, SparkArrayData, SparkInternalRow} import org.apache.paimon.types.{DataType, RowType} @@ -25,7 +26,8 @@ import org.apache.paimon.types.{DataType, RowType} import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.parser.ParserInterface -import org.apache.spark.sql.catalyst.plans.logical.Aggregate +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType @@ -38,6 +40,10 @@ class Spark3Shim extends SparkShim { new PaimonSpark3SqlExtensionsParser(delegate) } + override def createCustomResolution(spark: SparkSession): Rule[LogicalPlan] = { + Spark3ResolutionRules(spark) + } + override def createSparkInternalRow(rowType: RowType): SparkInternalRow = { new Spark3InternalRow(rowType) } diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark4ResolutionRules.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark4ResolutionRules.scala new file mode 100644 index 0000000000000..461cbd0c938ac --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark4ResolutionRules.scala @@ -0,0 +1,27 @@ +/* + * 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.paimon.spark.catalyst.analysis + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule + +case class Spark4ResolutionRules(session: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan +} diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala index dfec4eb71f4f1..eefddafdbfb89 100644 --- a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.paimon.shims +import org.apache.paimon.spark.catalyst.analysis.Spark4ResolutionRules import org.apache.paimon.spark.catalyst.parser.extensions.PaimonSpark4SqlExtensionsParser import org.apache.paimon.spark.data.{Spark4ArrayData, Spark4InternalRow, SparkArrayData, SparkInternalRow} import org.apache.paimon.types.{DataType, RowType} @@ -25,7 +26,8 @@ import org.apache.paimon.types.{DataType, RowType} import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.parser.ParserInterface -import org.apache.spark.sql.catalyst.plans.logical.Aggregate +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.internal.ExpressionUtils @@ -38,6 +40,11 @@ class Spark4Shim extends SparkShim { override def createSparkParser(delegate: ParserInterface): ParserInterface = { new PaimonSpark4SqlExtensionsParser(delegate) } + + override def createCustomResolution(spark: SparkSession): Rule[LogicalPlan] = { + Spark4ResolutionRules(spark) + } + override def createSparkInternalRow(rowType: RowType): SparkInternalRow = { new Spark4InternalRow(rowType) } From d33b8711fc6b4e1f35ba7d85336be4ff3baa956d Mon Sep 17 00:00:00 2001 From: "aiden.dong" <782112163@qq.com> Date: Tue, 3 Dec 2024 13:33:44 +0800 Subject: [PATCH 098/157] [core] Optimization of Parquet Predicate Pushdown Capability (#4608) --- .../table/PrimaryKeyFileStoreTableTest.java | 63 ++++++ .../format/parquet/ParquetReaderFactory.java | 66 +++++- .../parquet/reader/AbstractColumnReader.java | 204 +++++++++++++----- .../parquet/reader/BooleanColumnReader.java | 36 +++- .../parquet/reader/ByteColumnReader.java | 39 +++- .../parquet/reader/BytesColumnReader.java | 41 +++- .../parquet/reader/DoubleColumnReader.java | 38 +++- .../reader/FixedLenBytesColumnReader.java | 36 +++- .../parquet/reader/FloatColumnReader.java | 38 +++- .../parquet/reader/IntColumnReader.java | 39 +++- .../parquet/reader/LongColumnReader.java | 39 +++- .../parquet/reader/NestedColumnReader.java | 2 +- .../reader/NestedPrimitiveColumnReader.java | 141 +++++++----- .../parquet/reader/ParquetReadState.java | 148 +++++++++++++ .../reader/ParquetSplitReaderUtil.java | 41 ++-- .../parquet/reader/RunLengthDecoder.java | 45 ++++ .../parquet/reader/ShortColumnReader.java | 38 +++- .../parquet/reader/TimestampColumnReader.java | 15 +- 18 files changed, 898 insertions(+), 171 deletions(-) create mode 100644 paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReadState.java diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java index 46b85223bc2fa..e80b49a0f05df 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java @@ -84,6 +84,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; @@ -809,6 +810,68 @@ public void testDeletionVectorsWithFileIndexInFile() throws Exception { "1|4|500|binary|varbinary|mapKey:mapVal|multiset")); } + @Test + public void testDeletionVectorsWithParquetFilter() throws Exception { + FileStoreTable table = + createFileStoreTable( + conf -> { + conf.set(BUCKET, 1); + conf.set(DELETION_VECTORS_ENABLED, true); + conf.set(FILE_FORMAT, "parquet"); + conf.set("parquet.block.size", "1048576"); + conf.set("parquet.page.size", "1024"); + }); + + BatchWriteBuilder writeBuilder = table.newBatchWriteBuilder(); + + BatchTableWrite write = + (BatchTableWrite) + writeBuilder + .newWrite() + .withIOManager(new IOManagerImpl(tempDir.toString())); + + for (int i = 0; i < 200000; i++) { + write.write(rowData(1, i, i * 100L)); + } + + List messages = write.prepareCommit(); + BatchTableCommit commit = writeBuilder.newCommit(); + commit.commit(messages); + write = + (BatchTableWrite) + writeBuilder + .newWrite() + .withIOManager(new IOManagerImpl(tempDir.toString())); + for (int i = 180000; i < 200000; i++) { + write.write(rowDataWithKind(RowKind.DELETE, 1, i, i * 100L)); + } + + messages = write.prepareCommit(); + commit = writeBuilder.newCommit(); + commit.commit(messages); + + PredicateBuilder builder = new PredicateBuilder(ROW_TYPE); + List splits = toSplits(table.newSnapshotReader().read().dataSplits()); + Random random = new Random(); + + for (int i = 0; i < 10; i++) { + int value = random.nextInt(180000); + TableRead read = table.newRead().withFilter(builder.equal(1, value)).executeFilter(); + assertThat(getResult(read, splits, BATCH_ROW_TO_STRING)) + .isEqualTo( + Arrays.asList( + String.format( + "%d|%d|%d|binary|varbinary|mapKey:mapVal|multiset", + 1, value, value * 100L))); + } + + for (int i = 0; i < 10; i++) { + int value = 180000 + random.nextInt(20000); + TableRead read = table.newRead().withFilter(builder.equal(1, value)).executeFilter(); + assertThat(getResult(read, splits, BATCH_ROW_TO_STRING)).isEmpty(); + } + } + @Test public void testDeletionVectorsWithFileIndexInMeta() throws Exception { FileStoreTable table = diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java index f0151d6f3d8fb..0c996531201a5 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java @@ -28,6 +28,7 @@ import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.parquet.reader.ColumnReader; import org.apache.paimon.format.parquet.reader.ParquetDecimalVector; +import org.apache.paimon.format.parquet.reader.ParquetReadState; import org.apache.paimon.format.parquet.reader.ParquetTimestampVector; import org.apache.paimon.format.parquet.type.ParquetField; import org.apache.paimon.fs.Path; @@ -130,7 +131,7 @@ public FileRecordReader createReader(FormatReaderFactory.Context co buildFieldsList(projectedType.getFields(), projectedType.getFieldNames(), columnIO); return new ParquetReader( - reader, requestedSchema, reader.getRecordCount(), poolOfBatches, fields); + reader, requestedSchema, reader.getFilteredRecordCount(), poolOfBatches, fields); } private void setReadOptions(ParquetReadOptions.Builder builder) { @@ -336,6 +337,10 @@ private class ParquetReader implements FileRecordReader { private long nextRowPosition; + private ParquetReadState currentRowGroupReadState; + + private long currentRowGroupFirstRowIndex; + /** * For each request column, the reader to read this column. This is NULL if this column is * missing from the file, in which case we populate the attribute with NULL. @@ -359,6 +364,7 @@ private ParquetReader( this.totalCountLoadedSoFar = 0; this.currentRowPosition = 0; this.nextRowPosition = 0; + this.currentRowGroupFirstRowIndex = 0; this.fields = fields; } @@ -390,7 +396,8 @@ private boolean nextBatch(ParquetReaderBatch batch) throws IOException { currentRowPosition = nextRowPosition; } - int num = (int) Math.min(batchSize, totalCountLoadedSoFar - rowsReturned); + int num = getBachSize(); + for (int i = 0; i < columnReaders.length; ++i) { if (columnReaders[i] == null) { batch.writableVectors[i].fillWithNulls(); @@ -400,13 +407,13 @@ private boolean nextBatch(ParquetReaderBatch batch) throws IOException { } } rowsReturned += num; - nextRowPosition = currentRowPosition + num; + nextRowPosition = getNextRowPosition(num); batch.columnarBatch.setNumRows(num); return true; } private void readNextRowGroup() throws IOException { - PageReadStore rowGroup = reader.readNextRowGroup(); + PageReadStore rowGroup = reader.readNextFilteredRowGroup(); if (rowGroup == null) { throw new IOException( "expecting more rows but reached last block. Read " @@ -415,6 +422,9 @@ private void readNextRowGroup() throws IOException { + totalRowCount); } + this.currentRowGroupReadState = + new ParquetReadState(rowGroup.getRowIndexes().orElse(null)); + List types = requestedSchema.getFields(); columnReaders = new ColumnReader[types.size()]; for (int i = 0; i < types.size(); ++i) { @@ -429,18 +439,62 @@ private void readNextRowGroup() throws IOException { 0); } } + totalCountLoadedSoFar += rowGroup.getRowCount(); - if (rowGroup.getRowIndexOffset().isPresent()) { - currentRowPosition = rowGroup.getRowIndexOffset().get(); + + if (rowGroup.getRowIndexOffset().isPresent()) { // filter + currentRowGroupFirstRowIndex = rowGroup.getRowIndexOffset().get(); + long pageIndex = 0; + if (!this.currentRowGroupReadState.isMaxRange()) { + pageIndex = this.currentRowGroupReadState.currentRangeStart(); + } + currentRowPosition = currentRowGroupFirstRowIndex + pageIndex; } else { if (reader.rowGroupsFiltered()) { throw new RuntimeException( "There is a bug, rowIndexOffset must be present when row groups are filtered."); } + currentRowGroupFirstRowIndex = nextRowPosition; currentRowPosition = nextRowPosition; } } + private int getBachSize() throws IOException { + + long rangeBatchSize = Long.MAX_VALUE; + if (this.currentRowGroupReadState.isFinished()) { + throw new IOException( + "expecting more rows but reached last page block. Read " + + rowsReturned + + " out of " + + totalRowCount); + } else if (!this.currentRowGroupReadState.isMaxRange()) { + long pageIndex = this.currentRowPosition - this.currentRowGroupFirstRowIndex; + rangeBatchSize = this.currentRowGroupReadState.currentRangeEnd() - pageIndex + 1; + } + + return (int) + Math.min( + batchSize, + Math.min(rangeBatchSize, totalCountLoadedSoFar - rowsReturned)); + } + + private long getNextRowPosition(int num) { + if (this.currentRowGroupReadState.isMaxRange()) { + return this.currentRowPosition + num; + } else { + long pageIndex = this.currentRowPosition - this.currentRowGroupFirstRowIndex; + long nextIndex = pageIndex + num; + + if (this.currentRowGroupReadState.currentRangeEnd() < nextIndex) { + this.currentRowGroupReadState.nextRange(); + nextIndex = this.currentRowGroupReadState.currentRangeStart(); + } + + return nextIndex; + } + } + private ParquetReaderBatch getCachedEntry() throws IOException { try { return pool.pollEntry(); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/AbstractColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/AbstractColumnReader.java index 7e2ab6d5e7f05..5e3f4a7e6a336 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/AbstractColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/AbstractColumnReader.java @@ -32,6 +32,7 @@ import org.apache.parquet.column.page.DataPageV1; import org.apache.parquet.column.page.DataPageV2; import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.column.page.PageReader; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.io.ParquetDecodingException; @@ -65,20 +66,16 @@ public abstract class AbstractColumnReader protected final ColumnDescriptor descriptor; - /** Total number of values read. */ - private long valuesRead; - - /** - * value that indicates the end of the current page. That is, if valuesRead == - * endOfPageValueCount, we are at the end of the page. - */ - private long endOfPageValueCount; - /** If true, the current page is dictionary encoded. */ private boolean isCurrentPageDictionaryEncoded; /** Total values in the current page. */ - private int pageValueCount; + // private int pageValueCount; + + /** + * Helper struct to track intermediate states while reading Parquet pages in the column chunk. + */ + private final ParquetReadState readState; /* * Input streams: @@ -101,12 +98,14 @@ public abstract class AbstractColumnReader /** Dictionary decoder to wrap dictionary ids input stream. */ private RunLengthDecoder dictionaryIdsDecoder; - public AbstractColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public AbstractColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { this.descriptor = descriptor; - this.pageReader = pageReader; + this.pageReader = pageReadStore.getPageReader(descriptor); this.maxDefLevel = descriptor.getMaxDefinitionLevel(); + this.readState = new ParquetReadState(pageReadStore.getRowIndexes().orElse(null)); + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); if (dictionaryPage != null) { try { @@ -147,56 +146,136 @@ public final void readToVector(int readNumber, VECTOR vector) throws IOException if (dictionary != null) { dictionaryIds = vector.reserveDictionaryIds(readNumber); } - while (readNumber > 0) { + + readState.resetForNewBatch(readNumber); + + while (readState.rowsToReadInBatch > 0) { // Compute the number of values we want to read in this page. - int leftInPage = (int) (endOfPageValueCount - valuesRead); - if (leftInPage == 0) { - DataPage page = pageReader.readPage(); - if (page instanceof DataPageV1) { - readPageV1((DataPageV1) page); - } else if (page instanceof DataPageV2) { - readPageV2((DataPageV2) page); - } else { - throw new RuntimeException("Unsupported page type: " + page.getClass()); + if (readState.valuesToReadInPage == 0) { + int pageValueCount = readPage(); + if (pageValueCount < 0) { + // we've read all the pages; this could happen when we're reading a repeated + // list and we + // don't know where the list will end until we've seen all the pages. + break; } - leftInPage = (int) (endOfPageValueCount - valuesRead); } - int num = Math.min(readNumber, leftInPage); - if (isCurrentPageDictionaryEncoded) { - // Read and decode dictionary ids. - runLenDecoder.readDictionaryIds( - num, dictionaryIds, vector, rowId, maxDefLevel, this.dictionaryIdsDecoder); - - if (vector.hasDictionary() || (rowId == 0 && supportLazyDecode())) { - // Column vector supports lazy decoding of dictionary values so just set the - // dictionary. - // We can't do this if rowId != 0 AND the column doesn't have a dictionary (i.e. - // some - // non-dictionary encoded values have already been added). - vector.setDictionary(new ParquetDictionary(dictionary)); + + if (readState.isFinished()) { + break; + } + + long pageRowId = readState.rowId; + int leftInBatch = readState.rowsToReadInBatch; + int leftInPage = readState.valuesToReadInPage; + + int readBatch = Math.min(leftInBatch, leftInPage); + + long rangeStart = readState.currentRangeStart(); + long rangeEnd = readState.currentRangeEnd(); + + if (pageRowId < rangeStart) { + int toSkip = (int) (rangeStart - pageRowId); + if (toSkip >= leftInPage) { // drop page + pageRowId += leftInPage; + leftInPage = 0; } else { - readBatchFromDictionaryIds(rowId, num, vector, dictionaryIds); + if (isCurrentPageDictionaryEncoded) { + runLenDecoder.skipDictionaryIds( + toSkip, maxDefLevel, this.dictionaryIdsDecoder); + pageRowId += toSkip; + leftInPage -= toSkip; + } else { + skipBatch(toSkip); + pageRowId += toSkip; + leftInPage -= toSkip; + } } + } else if (pageRowId > rangeEnd) { + readState.nextRange(); } else { - if (vector.hasDictionary() && rowId != 0) { - // This batch already has dictionary encoded values but this new page is not. - // The batch - // does not support a mix of dictionary and not so we will decode the - // dictionary. - readBatchFromDictionaryIds(0, rowId, vector, vector.getDictionaryIds()); + long start = pageRowId; + long end = Math.min(rangeEnd, pageRowId + readBatch - 1); + int num = (int) (end - start + 1); + + if (isCurrentPageDictionaryEncoded) { + // Read and decode dictionary ids. + runLenDecoder.readDictionaryIds( + num, + dictionaryIds, + vector, + rowId, + maxDefLevel, + this.dictionaryIdsDecoder); + + if (vector.hasDictionary() || (rowId == 0 && supportLazyDecode())) { + // Column vector supports lazy decoding of dictionary values so just set the + // dictionary. + // We can't do this if rowId != 0 AND the column doesn't have a dictionary + // (i.e. + // some + // non-dictionary encoded values have already been added). + vector.setDictionary(new ParquetDictionary(dictionary)); + } else { + readBatchFromDictionaryIds(rowId, num, vector, dictionaryIds); + } + } else { + if (vector.hasDictionary() && rowId != 0) { + // This batch already has dictionary encoded values but this new page is + // not. + // The batch + // does not support a mix of dictionary and not so we will decode the + // dictionary. + readBatchFromDictionaryIds(0, rowId, vector, vector.getDictionaryIds()); + } + vector.setDictionary(null); + readBatch(rowId, num, vector); } - vector.setDictionary(null); - readBatch(rowId, num, vector); + leftInBatch -= num; + pageRowId += num; + leftInPage -= num; + rowId += num; } + readState.rowsToReadInBatch = leftInBatch; + readState.valuesToReadInPage = leftInPage; + readState.rowId = pageRowId; + } + } - valuesRead += num; - rowId += num; - readNumber -= num; + private int readPage() { + DataPage page = pageReader.readPage(); + if (page == null) { + return -1; } + long pageFirstRowIndex = page.getFirstRowIndex().orElse(0L); + + int pageValueCount = + page.accept( + new DataPage.Visitor() { + @Override + public Integer visit(DataPageV1 dataPageV1) { + try { + return readPageV1(dataPageV1); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Integer visit(DataPageV2 dataPageV2) { + try { + return readPageV2(dataPageV2); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + readState.resetForNewPage(pageValueCount, pageFirstRowIndex); + return pageValueCount; } - private void readPageV1(DataPageV1 page) throws IOException { - this.pageValueCount = page.getValueCount(); + private int readPageV1(DataPageV1 page) throws IOException { + int pageValueCount = page.getValueCount(); ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); // Initialize the decoders. @@ -211,30 +290,31 @@ private void readPageV1(DataPageV1 page) throws IOException { ByteBufferInputStream in = bytes.toInputStream(); rlReader.initFromPage(pageValueCount, in); this.runLenDecoder.initFromStream(pageValueCount, in); - prepareNewPage(page.getValueEncoding(), in); + prepareNewPage(page.getValueEncoding(), in, pageValueCount); + return pageValueCount; } catch (IOException e) { throw new IOException("could not read page " + page + " in col " + descriptor, e); } } - private void readPageV2(DataPageV2 page) throws IOException { - this.pageValueCount = page.getValueCount(); + private int readPageV2(DataPageV2 page) throws IOException { + int pageValueCount = page.getValueCount(); int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); // do not read the length from the stream. v2 pages handle dividing the page bytes. this.runLenDecoder = new RunLengthDecoder(bitWidth, false); this.runLenDecoder.initFromStream( - this.pageValueCount, page.getDefinitionLevels().toInputStream()); + pageValueCount, page.getDefinitionLevels().toInputStream()); try { - prepareNewPage(page.getDataEncoding(), page.getData().toInputStream()); + prepareNewPage(page.getDataEncoding(), page.getData().toInputStream(), pageValueCount); + return pageValueCount; } catch (IOException e) { throw new IOException("could not read page " + page + " in col " + descriptor, e); } } - private void prepareNewPage(Encoding dataEncoding, ByteBufferInputStream in) + private void prepareNewPage(Encoding dataEncoding, ByteBufferInputStream in, int pageValueCount) throws IOException { - this.endOfPageValueCount = valuesRead + pageValueCount; if (dataEncoding.usesDictionary()) { if (dictionary == null) { throw new IOException( @@ -269,6 +349,14 @@ private void prepareNewPage(Encoding dataEncoding, ByteBufferInputStream in) afterReadPage(); } + final void skipDataBuffer(int length) { + try { + dataInputStream.skipFully(length); + } catch (IOException e) { + throw new ParquetDecodingException("Failed to skip " + length + " bytes", e); + } + } + final ByteBuffer readDataBuffer(int length) { try { return dataInputStream.slice(length).order(ByteOrder.LITTLE_ENDIAN); @@ -291,6 +379,8 @@ protected boolean supportLazyDecode() { /** Read batch from {@link #runLenDecoder} and {@link #dataInputStream}. */ protected abstract void readBatch(int rowId, int num, VECTOR column); + protected abstract void skipBatch(int num); + /** * Decode dictionary ids to data. From {@link #runLenDecoder} and {@link #dictionaryIdsDecoder}. */ diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BooleanColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BooleanColumnReader.java index d5dc231d8436d..83d3c5a07d4b6 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BooleanColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BooleanColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.schema.PrimitiveType; @@ -36,9 +36,9 @@ public class BooleanColumnReader extends AbstractColumnReader 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + for (int i = 0; i < n; i++) { + readBoolean(); + } + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + readBoolean(); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + private boolean readBoolean() { if (bitOffset == 0) { try { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ByteColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ByteColumnReader.java index bed9923d9be35..804b8bc0275ea 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ByteColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ByteColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,8 +31,9 @@ /** Byte {@link ColumnReader}. Using INT32 to store byte, so just cast int to byte. */ public class ByteColumnReader extends AbstractColumnReader { - public ByteColumnReader(ColumnDescriptor descriptor, PageReader pageReader) throws IOException { - super(descriptor, pageReader); + public ByteColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) + throws IOException { + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.INT32); } @@ -69,6 +70,38 @@ protected void readBatch(int rowId, int num, WritableByteVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipByte(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipByte(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipByte(int num) { + skipDataBuffer(4 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableByteVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BytesColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BytesColumnReader.java index e83115c8a69f4..6ee395e585684 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BytesColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BytesColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,9 +31,9 @@ /** Bytes {@link ColumnReader}. A int length and bytes data. */ public class BytesColumnReader extends AbstractColumnReader { - public BytesColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public BytesColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { - super(descriptor, pageReader); + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.BINARY); } @@ -70,6 +70,41 @@ protected void readBatch(int rowId, int num, WritableBytesVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipBinary(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipBinary(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipBinary(int num) { + for (int i = 0; i < num; i++) { + int len = readDataBuffer(4).getInt(); + skipDataBuffer(len); + } + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableBytesVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/DoubleColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/DoubleColumnReader.java index d6d8aa2bbb22b..2cffd406248e5 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/DoubleColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/DoubleColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,9 +31,9 @@ /** Double {@link ColumnReader}. */ public class DoubleColumnReader extends AbstractColumnReader { - public DoubleColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public DoubleColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { - super(descriptor, pageReader); + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.DOUBLE); } @@ -70,6 +70,38 @@ protected void readBatch(int rowId, int num, WritableDoubleVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipDouble(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipDouble(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipDouble(int num) { + skipDataBuffer(8 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableDoubleVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FixedLenBytesColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FixedLenBytesColumnReader.java index afce717a67197..25e1b466e4658 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FixedLenBytesColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FixedLenBytesColumnReader.java @@ -25,7 +25,7 @@ import org.apache.paimon.format.parquet.ParquetSchemaConverter; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.PrimitiveType; @@ -39,8 +39,9 @@ public class FixedLenBytesColumnReader private final int precision; public FixedLenBytesColumnReader( - ColumnDescriptor descriptor, PageReader pageReader, int precision) throws IOException { - super(descriptor, pageReader); + ColumnDescriptor descriptor, PageReadStore pageReadStore, int precision) + throws IOException { + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY); this.precision = precision; } @@ -79,6 +80,35 @@ protected void readBatch(int rowId, int num, VECTOR column) { } } + @Override + protected void skipBatch(int num) { + int bytesLen = descriptor.getPrimitiveType().getTypeLength(); + if (ParquetSchemaConverter.is32BitDecimal(precision)) { + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + skipDataBinary(bytesLen); + } + } + } else if (ParquetSchemaConverter.is64BitDecimal(precision)) { + + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + skipDataBinary(bytesLen); + } + } + } else { + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + skipDataBinary(bytesLen); + } + } + } + } + + private void skipDataBinary(int len) { + skipDataBuffer(len); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, VECTOR column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FloatColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FloatColumnReader.java index 1f4adfa4b9c8c..e9eec13df5fc5 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FloatColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FloatColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,9 +31,9 @@ /** Float {@link ColumnReader}. */ public class FloatColumnReader extends AbstractColumnReader { - public FloatColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public FloatColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { - super(descriptor, pageReader); + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.FLOAT); } @@ -70,6 +70,38 @@ protected void readBatch(int rowId, int num, WritableFloatVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipFloat(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipFloat(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipFloat(int num) { + skipDataBuffer(4 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableFloatVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/IntColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/IntColumnReader.java index e38e916d187e6..521ad998f6f1f 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/IntColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/IntColumnReader.java @@ -21,7 +21,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -30,8 +30,9 @@ /** Int {@link ColumnReader}. */ public class IntColumnReader extends AbstractColumnReader { - public IntColumnReader(ColumnDescriptor descriptor, PageReader pageReader) throws IOException { - super(descriptor, pageReader); + public IntColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) + throws IOException { + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.INT32); } @@ -68,6 +69,38 @@ protected void readBatch(int rowId, int num, WritableIntVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipInteger(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipInteger(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipInteger(int num) { + skipDataBuffer(4 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableIntVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/LongColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/LongColumnReader.java index a8e04eae673af..c4af086a7026a 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/LongColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/LongColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableLongVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,8 +31,9 @@ /** Long {@link ColumnReader}. */ public class LongColumnReader extends AbstractColumnReader { - public LongColumnReader(ColumnDescriptor descriptor, PageReader pageReader) throws IOException { - super(descriptor, pageReader); + public LongColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) + throws IOException { + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.INT64); } @@ -69,6 +70,38 @@ protected void readBatch(int rowId, int num, WritableLongVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipValue(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipValue(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipValue(int num) { + skipDataBuffer(num * 8); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableLongVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java index 68225fbd13207..8f20be2754471 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java @@ -279,7 +279,7 @@ private Pair readPrimitive( reader = new NestedPrimitiveColumnReader( descriptor, - pages.getPageReader(descriptor), + pages, isUtcTimestamp, descriptor.getPrimitiveType(), field.getType(), diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java index 7d00ff79234ae..7db7aedbf6aee 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java @@ -44,6 +44,7 @@ import org.apache.parquet.column.page.DataPageV1; import org.apache.parquet.column.page.DataPageV2; import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.column.page.PageReader; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder; @@ -82,15 +83,6 @@ public class NestedPrimitiveColumnReader implements ColumnReader valueList = new ArrayList<>(); + int valueIndex = collectDataFromParquetPage(readNumber, valueList); + + return fillColumnVector(valueIndex, valueList); + } + + private int collectDataFromParquetPage(int total, List valueList) throws IOException { + int valueIndex = 0; // repeated type need two loops to read data. - while (!eof && index < readNumber) { + + readState.resetForNewBatch(total); + + while (!eof && readState.rowsToReadInBatch > 0) { + + if (readState.isFinished()) { // finished to read + eof = true; + break; + } + + long pageRowId = readState.rowId; + long rangeStart = readState.currentRangeStart(); + long rangeEnd = readState.currentRangeEnd(); + + if (pageRowId > rangeEnd) { + readState.nextRange(); + continue; + } + + boolean needFilterSkip = pageRowId < rangeStart; + do { - if (!lastValue.shouldSkip) { + + if (!lastValue.shouldSkip && !needFilterSkip) { valueList.add(lastValue.value); valueIndex++; } } while (readValue() && (repetitionLevel != 0)); - index++; + + if (pageRowId == readState.rowId) { + readState.rowId = readState.rowId + 1; + } + + if (!needFilterSkip) { + readState.rowsToReadInBatch = readState.rowsToReadInBatch - 1; + } } - return fillColumnVector(valueIndex, valueList); + return valueIndex; } public LevelDelegation getLevelDelegation() { @@ -255,20 +287,24 @@ private void readAndSaveRepetitionAndDefinitionLevels() { // get the values of repetition and definitionLevel repetitionLevel = repetitionLevelColumn.nextInt(); definitionLevel = definitionLevelColumn.nextInt(); - valuesRead++; + readState.valuesToReadInPage = readState.valuesToReadInPage - 1; repetitionLevelList.add(repetitionLevel); definitionLevelList.add(definitionLevel); } private int readPageIfNeed() throws IOException { // Compute the number of values we want to read in this page. - int leftInPage = (int) (endOfPageValueCount - valuesRead); - if (leftInPage == 0) { - // no data left in current page, load data from new page - readPage(); - leftInPage = (int) (endOfPageValueCount - valuesRead); + if (readState.valuesToReadInPage == 0) { + int pageValueCount = readPage(); + // 返回当前 page 的数据量 + if (pageValueCount < 0) { + // we've read all the pages; this could happen when we're reading a repeated list + // and we + // don't know where the list will end until we've seen all the pages. + return -1; + } } - return leftInPage; + return readState.valuesToReadInPage; } private Object readPrimitiveTypedRow(DataType category) { @@ -528,33 +564,36 @@ private static HeapBytesVector getHeapBytesVector(int total, List valueList) { return phbv; } - protected void readPage() { + protected int readPage() { DataPage page = pageReader.readPage(); if (page == null) { - return; + return -1; } - page.accept( - new DataPage.Visitor() { - @Override - public Void visit(DataPageV1 dataPageV1) { - readPageV1(dataPageV1); - return null; - } + long pageFirstRowIndex = page.getFirstRowIndex().orElse(0L); - @Override - public Void visit(DataPageV2 dataPageV2) { - readPageV2(dataPageV2); - return null; - } - }); + int pageValueCount = + page.accept( + new DataPage.Visitor() { + @Override + public Integer visit(DataPageV1 dataPageV1) { + return readPageV1(dataPageV1); + } + + @Override + public Integer visit(DataPageV2 dataPageV2) { + return readPageV2(dataPageV2); + } + }); + readState.resetForNewPage(pageValueCount, pageFirstRowIndex); + return pageValueCount; } private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) throws IOException { - this.pageValueCount = valueCount; - this.endOfPageValueCount = valuesRead + pageValueCount; + // this.pageValueCount = valueCount; + // this.endOfPageValueCount = valuesRead + pageValueCount; if (dataEncoding.usesDictionary()) { this.dataColumn = null; if (dictionary == null) { @@ -577,13 +616,14 @@ private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int } try { - dataColumn.initFromPage(pageValueCount, in); + dataColumn.initFromPage(valueCount, in); } catch (IOException e) { throw new IOException(String.format("Could not read page in col %s.", descriptor), e); } } - private void readPageV1(DataPageV1 page) { + private int readPageV1(DataPageV1 page) { + int pageValueCount = page.getValueCount(); ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); ValuesReader dlReader = page.getDlEncoding().getValuesReader(descriptor, DEFINITION_LEVEL); this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader); @@ -597,15 +637,16 @@ private void readPageV1(DataPageV1 page) { LOG.debug("Reading definition levels at {}.", in.position()); dlReader.initFromPage(pageValueCount, in); LOG.debug("Reading data at {}.", in.position()); - initDataReader(page.getValueEncoding(), in, page.getValueCount()); + initDataReader(page.getValueEncoding(), in, pageValueCount); + return pageValueCount; } catch (IOException e) { throw new ParquetDecodingException( String.format("Could not read page %s in col %s.", page, descriptor), e); } } - private void readPageV2(DataPageV2 page) { - this.pageValueCount = page.getValueCount(); + private int readPageV2(DataPageV2 page) { + int pageValueCount = page.getValueCount(); this.repetitionLevelColumn = newRLEIterator(descriptor.getMaxRepetitionLevel(), page.getRepetitionLevels()); this.definitionLevelColumn = @@ -615,8 +656,8 @@ private void readPageV2(DataPageV2 page) { "Page data size {} bytes and {} records.", page.getData().size(), pageValueCount); - initDataReader( - page.getDataEncoding(), page.getData().toInputStream(), page.getValueCount()); + initDataReader(page.getDataEncoding(), page.getData().toInputStream(), pageValueCount); + return pageValueCount; } catch (IOException e) { throw new ParquetDecodingException( String.format("Could not read page %s in col %s.", page, descriptor), e); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReadState.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReadState.java new file mode 100644 index 0000000000000..a6003676825a1 --- /dev/null +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReadState.java @@ -0,0 +1,148 @@ +/* + * 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.paimon.format.parquet.reader; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.PrimitiveIterator; + +/** Parquet reader state for column index. */ +public class ParquetReadState { + /** A special row range used when there is no row indexes (hence all rows must be included). */ + private static final RowRange MAX_ROW_RANGE = new RowRange(Long.MIN_VALUE, Long.MAX_VALUE); + + /** + * A special row range used when the row indexes are present AND all the row ranges have been + * processed. This serves as a sentinel at the end indicating that all rows come after the last + * row range should be skipped. + */ + private static final RowRange END_ROW_RANGE = new RowRange(Long.MAX_VALUE, Long.MIN_VALUE); + + private final Iterator rowRanges; + + private RowRange currentRange; + + /** row index for the next read. */ + long rowId; + + int valuesToReadInPage; + int rowsToReadInBatch; + + public ParquetReadState(PrimitiveIterator.OfLong rowIndexes) { + this.rowRanges = constructRanges(rowIndexes); + nextRange(); + } + + /** + * Construct a list of row ranges from the given `rowIndexes`. For example, suppose the + * `rowIndexes` are `[0, 1, 2, 4, 5, 7, 8, 9]`, it will be converted into 3 row ranges: `[0-2], + * [4-5], [7-9]`. + */ + private Iterator constructRanges(PrimitiveIterator.OfLong rowIndexes) { + if (rowIndexes == null) { + return null; + } + + List rowRanges = new ArrayList<>(); + long currentStart = Long.MIN_VALUE; + long previous = Long.MIN_VALUE; + + while (rowIndexes.hasNext()) { + long idx = rowIndexes.nextLong(); + if (currentStart == Long.MIN_VALUE) { + currentStart = idx; + } else if (previous + 1 != idx) { + RowRange range = new RowRange(currentStart, previous); + rowRanges.add(range); + currentStart = idx; + } + previous = idx; + } + + if (previous != Long.MIN_VALUE) { + rowRanges.add(new RowRange(currentStart, previous)); + } + + return rowRanges.iterator(); + } + + /** Must be called at the beginning of reading a new batch. */ + void resetForNewBatch(int batchSize) { + this.rowsToReadInBatch = batchSize; + } + + /** Must be called at the beginning of reading a new page. */ + void resetForNewPage(int totalValuesInPage, long pageFirstRowIndex) { + this.valuesToReadInPage = totalValuesInPage; + this.rowId = pageFirstRowIndex; + } + + /** Returns the start index of the current row range. */ + public long currentRangeStart() { + return currentRange.start; + } + + /** Returns the end index of the current row range. */ + public long currentRangeEnd() { + return currentRange.end; + } + + public boolean isFinished() { + return this.currentRange.equals(this.END_ROW_RANGE); + } + + public boolean isMaxRange() { + return this.currentRange.equals(this.MAX_ROW_RANGE); + } + + public RowRange getCurrentRange() { + return currentRange; + } + + /** Advance to the next range. */ + public void nextRange() { + if (rowRanges == null) { + currentRange = MAX_ROW_RANGE; + } else if (!rowRanges.hasNext()) { + currentRange = END_ROW_RANGE; + } else { + currentRange = rowRanges.next(); + } + } + + /** Helper struct to represent a range of row indexes `[start, end]`. */ + public static class RowRange { + final long start; + final long end; + + RowRange(long start, long end) { + this.start = start; + this.end = end; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RowRange)) { + return false; + } + return ((RowRange) obj).start == this.start && ((RowRange) obj).end == this.end; + } + } +} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetSplitReaderUtil.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetSplitReaderUtil.java index 860ec54fa88b0..a2be77414d5ac 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetSplitReaderUtil.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetSplitReaderUtil.java @@ -87,58 +87,45 @@ public static ColumnReader createColumnReader( getAllColumnDescriptorByType(depth, type, columnDescriptors); switch (fieldType.getTypeRoot()) { case BOOLEAN: - return new BooleanColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new BooleanColumnReader(descriptors.get(0), pages); case TINYINT: - return new ByteColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new ByteColumnReader(descriptors.get(0), pages); case DOUBLE: - return new DoubleColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new DoubleColumnReader(descriptors.get(0), pages); case FLOAT: - return new FloatColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new FloatColumnReader(descriptors.get(0), pages); case INTEGER: case DATE: case TIME_WITHOUT_TIME_ZONE: - return new IntColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new IntColumnReader(descriptors.get(0), pages); case BIGINT: - return new LongColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new LongColumnReader(descriptors.get(0), pages); case SMALLINT: - return new ShortColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new ShortColumnReader(descriptors.get(0), pages); case CHAR: case VARCHAR: case BINARY: case VARBINARY: - return new BytesColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new BytesColumnReader(descriptors.get(0), pages); case TIMESTAMP_WITHOUT_TIME_ZONE: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: if (descriptors.get(0).getPrimitiveType().getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT64) { - return new LongColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new LongColumnReader(descriptors.get(0), pages); } - return new TimestampColumnReader( - true, descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new TimestampColumnReader(true, descriptors.get(0), pages); case DECIMAL: switch (descriptors.get(0).getPrimitiveType().getPrimitiveTypeName()) { case INT32: - return new IntColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new IntColumnReader(descriptors.get(0), pages); case INT64: - return new LongColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new LongColumnReader(descriptors.get(0), pages); case BINARY: - return new BytesColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new BytesColumnReader(descriptors.get(0), pages); case FIXED_LEN_BYTE_ARRAY: return new FixedLenBytesColumnReader( descriptors.get(0), - pages.getPageReader(descriptors.get(0)), + pages, ((DecimalType) fieldType).getPrecision()); } case ARRAY: diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RunLengthDecoder.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RunLengthDecoder.java index 2dd1655d571f0..ebb8f28fa1eee 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RunLengthDecoder.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RunLengthDecoder.java @@ -194,6 +194,51 @@ private void readDictionaryIdData(int total, WritableIntVector c, int rowId) { } } + void skipDictionaryIds(int total, int level, RunLengthDecoder data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) { + this.readNextGroup(); + } + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.skipDictionaryIdData(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + data.readInteger(); + } + } + break; + } + left -= n; + currentCount -= n; + } + } + + private void skipDictionaryIdData(int total) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) { + this.readNextGroup(); + } + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + break; + case PACKED: + currentBufferIdx += n; + break; + } + left -= n; + currentCount -= n; + } + } + /** Reads the next varint encoded int. */ private int readUnsignedVarInt() throws IOException { int value = 0; diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ShortColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ShortColumnReader.java index 7b32232261a7b..bdb2f401fa3f3 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ShortColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ShortColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableShortVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -30,9 +30,9 @@ /** Short {@link ColumnReader}. Using INT32 to store short, so just cast int to short. */ public class ShortColumnReader extends AbstractColumnReader { - public ShortColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public ShortColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { - super(descriptor, pageReader); + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.INT32); } @@ -71,6 +71,38 @@ protected void readBatch(int rowId, int num, WritableShortVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipShot(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipShot(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipShot(int num) { + skipDataBuffer(4 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableShortVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/TimestampColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/TimestampColumnReader.java index 4a279ff90e15f..8767173315c24 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/TimestampColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/TimestampColumnReader.java @@ -23,7 +23,7 @@ import org.apache.paimon.data.columnar.writable.WritableTimestampVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.PrimitiveType; @@ -49,9 +49,9 @@ public class TimestampColumnReader extends AbstractColumnReader Date: Tue, 3 Dec 2024 23:20:15 +0800 Subject: [PATCH 099/157] [doc] Add data types in concept (#4625) --- docs/content/concepts/data-types.md | 179 ++++++++++++++++++ docs/content/concepts/spec/_index.md | 2 +- .../generated/format_table_configuration.html | 36 ++++ .../ConfigOptionsDocGenerator.java | 1 + 4 files changed, 217 insertions(+), 1 deletion(-) create mode 100644 docs/content/concepts/data-types.md create mode 100644 docs/layouts/shortcodes/generated/format_table_configuration.html diff --git a/docs/content/concepts/data-types.md b/docs/content/concepts/data-types.md new file mode 100644 index 0000000000000..b33dcd428399b --- /dev/null +++ b/docs/content/concepts/data-types.md @@ -0,0 +1,179 @@ +--- +title: "Data Types" +weight: 7 +type: docs +aliases: +- /concepts/data-types.html +--- + + +# Data Types + +A data type describes the logical type of a value in the table ecosystem. It can be used to declare input and/or output types of operations. + +All data types supported by Paimon are as follows: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    DataTypeDescription
    BOOLEANData type of a boolean with a (possibly) three-valued logic of TRUE, FALSE, and UNKNOWN.
    CHAR
    + CHAR(n) +
    Data type of a fixed-length character string.

    + The type can be declared using CHAR(n) where n is the number of code points. n must have a value between 1 and 2,147,483,647 (both inclusive). If no length is specified, n is equal to 1. +
    VARCHAR
    + VARCHAR(n)

    + STRING +
    Data type of a variable-length character string.

    + The type can be declared using VARCHAR(n) where n is the maximum number of code points. n must have a value between 1 and 2,147,483,647 (both inclusive). If no length is specified, n is equal to 1.

    + STRING is a synonym for VARCHAR(2147483647). +
    BINARY
    + BINARY(n)

    +
    Data type of a fixed-length binary string (=a sequence of bytes).

    + The type can be declared using BINARY(n) where n is the number of bytes. n must have a value between 1 and 2,147,483,647 (both inclusive). If no length is specified, n is equal to 1. +
    VARBINARY
    + VARBINARY(n)

    + BYTES +
    Data type of a variable-length binary string (=a sequence of bytes).

    + The type can be declared using VARBINARY(n) where n is the maximum number of bytes. n must have a value between 1 and 2,147,483,647 (both inclusive). If no length is specified, n is equal to 1.

    + BYTES is a synonym for VARBINARY(2147483647). +
    DECIMAL
    + DECIMAL(p)
    + DECIMAL(p, s) +
    Data type of a decimal number with fixed precision and scale.

    + The type can be declared using DECIMAL(p, s) where p is the number of digits in a number (precision) and s is the number of digits to the right of the decimal point in a number (scale). p must have a value between 1 and 38 (both inclusive). s must have a value between 0 and p (both inclusive). The default value for p is 10. The default value for s is 0. +
    TINYINTData type of a 1-byte signed integer with values from -128 to 127.
    SMALLINTData type of a 2-byte signed integer with values from -32,768 to 32,767.
    INTData type of a 4-byte signed integer with values from -2,147,483,648 to 2,147,483,647.
    BIGINTData type of an 8-byte signed integer with values from -9,223,372,036,854,775,808 to 9,223,372,036,854,775,807.
    FLOATData type of a 4-byte single precision floating point number.

    + Compared to the SQL standard, the type does not take parameters. +
    DOUBLEData type of an 8-byte double precision floating point number.
    DATEData type of a date consisting of year-month-day with values ranging from 0000-01-01 to 9999-12-31.

    + Compared to the SQL standard, the range starts at year 0000. +
    TIME
    + TIME(p) +
    Data type of a time without time zone consisting of hour:minute:second[.fractional] with up to nanosecond precision and values ranging from 00:00:00.000000000 to 23:59:59.999999999.

    + The type can be declared using TIME(p) where p is the number of digits of fractional seconds (precision). p must have a value between 0 and 9 (both inclusive). If no precision is specified, p is equal to 0. +
    TIMESTAMP
    + TIMESTAMP(p) +
    Data type of a timestamp without time zone consisting of year-month-day hour:minute:second[.fractional] with up to nanosecond precision and values ranging from 0000-01-01 00:00:00.000000000 to 9999-12-31 23:59:59.999999999.

    + The type can be declared using TIMESTAMP(p) where p is the number of digits of fractional seconds (precision). p must have a value between 0 and 9 (both inclusive). If no precision is specified, p is equal to 6. +
    TIMESTAMP WITH TIME ZONE
    + TIMESTAMP(p) WITH TIME ZONE +
    Data type of a timestamp with time zone consisting of year-month-day hour:minute:second[.fractional] zone with up to nanosecond precision and values ranging from 0000-01-01 00:00:00.000000000 +14:59 to 9999-12-31 23:59:59.999999999 -14:59.

    + This type fills the gap between time zone free and time zone mandatory timestamp types by allowing the interpretation of UTC timestamps according to the configured session time zone. A conversion from and to int describes the number of seconds since epoch. A conversion from and to long describes the number of milliseconds since epoch. +
    ARRAY<t>Data type of an array of elements with same subtype.

    + Compared to the SQL standard, the maximum cardinality of an array cannot be specified but is fixed at 2,147,483,647. Also, any valid type is supported as a subtype.

    + The type can be declared using ARRAY<t> where t is the data type of the contained elements. +
    MAP<kt, vt>Data type of an associative array that maps keys (including NULL) to values (including NULL). A map cannot contain duplicate keys; each key can map to at most one value.

    + There is no restriction of element types; it is the responsibility of the user to ensure uniqueness.

    + The type can be declared using MAP<kt, vt> where kt is the data type of the key elements and vt is the data type of the value elements. +
    MULTISET<t>Data type of a multiset (=bag). Unlike a set, it allows for multiple instances for each of its elements with a common subtype. Each unique value (including NULL) is mapped to some multiplicity.

    + There is no restriction of element types; it is the responsibility of the user to ensure uniqueness.

    + The type can be declared using MULTISET<t> where t is the data type of the contained elements. +
    ROW<n0 t0, n1 t1, ...>
    + ROW<n0 t0 'd0', n1 t1 'd1', ...> +
    Data type of a sequence of fields.

    + A field consists of a field name, field type, and an optional description. The most specific type of a row of a table is a row type. In this case, each column of the row corresponds to the field of the row type that has the same ordinal position as the column.

    + Compared to the SQL standard, an optional field description simplifies the handling with complex structures.

    + A row type is similar to the STRUCT type known from other non-standard-compliant frameworks.

    + The type can be declared using ROW<n0 t0 'd0', n1 t1 'd1', ...> where n is the unique name of a field, t is the logical type of a field, d is the description of a field. +
    diff --git a/docs/content/concepts/spec/_index.md b/docs/content/concepts/spec/_index.md index ef5f03098e20d..cc148d6a8b533 100644 --- a/docs/content/concepts/spec/_index.md +++ b/docs/content/concepts/spec/_index.md @@ -1,7 +1,7 @@ --- title: Specification bookCollapseSection: true -weight: 7 +weight: 8 --- - + From b6fb4715021a2261230773645f114fb7140dcc08 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 10 Dec 2024 14:01:22 +0800 Subject: [PATCH 129/157] [core] Introduce data-file.path-directory (#4672) --- .../migration/iceberg-compatibility.md | 6 ++++++ .../generated/core_configuration.html | 6 ++++++ .../java/org/apache/paimon/CoreOptions.java | 11 ++++++++++ .../org/apache/paimon/AbstractFileStore.java | 9 ++++++-- .../org/apache/paimon/KeyValueFileStore.java | 15 +------------ .../paimon/utils/FileStorePathFactory.java | 13 ++++++++++-- .../paimon/io/KeyValueFileReadWriteTest.java | 6 ++++-- .../manifest/ManifestFileMetaTestBase.java | 3 ++- .../paimon/manifest/ManifestFileTest.java | 3 ++- .../paimon/manifest/ManifestListTest.java | 3 ++- .../table/AppendOnlyFileStoreTableTest.java | 21 +++++++++++++++++++ .../utils/FileStorePathFactoryTest.java | 6 ++++-- .../source/TestChangelogDataReadWrite.java | 3 ++- .../paimon/spark/SparkFileIndexITCase.java | 3 ++- 14 files changed, 81 insertions(+), 27 deletions(-) diff --git a/docs/content/migration/iceberg-compatibility.md b/docs/content/migration/iceberg-compatibility.md index 8a1637d7e96e8..54018b073f91f 100644 --- a/docs/content/migration/iceberg-compatibility.md +++ b/docs/content/migration/iceberg-compatibility.md @@ -402,6 +402,12 @@ You can use Hive Catalog to connect AWS Glue metastore, you can use set `'metada AWS Athena may use old manifest reader to read Iceberg manifest by names, we should let Paimon producing legacy Iceberg manifest list file, you can enable: `'metadata.iceberg.manifest-legacy-version'`. +## DuckDB + +Duckdb may rely on files placed in the `root/data` directory, while Paimon is usually placed directly in the `root` +directory, so you can configure this parameter for the table to achieve compatibility: +`'data-file.path-directory' = 'data'`. + ## Trino Iceberg In this example, we use Trino Iceberg connector to access Paimon table through Iceberg Hive catalog. diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 52b64a3a565db..b2bd3a976d66d 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -218,6 +218,12 @@ Duration The TTL in rocksdb index for cross partition upsert (primary keys not contain all partition fields), this can avoid maintaining too many indexes and lead to worse and worse performance, but please note that this may also cause data duplication. + +
    data-file.path-directory
    + (none) + String + Specify the path directory of data files. +
    data-file.prefix
    "data-" diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index f42bb8aeca985..5db809cff1d1f 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -184,6 +184,12 @@ public class CoreOptions implements Serializable { .defaultValue("data-") .withDescription("Specify the file name prefix of data files."); + public static final ConfigOption DATA_FILE_PATH_DIRECTORY = + key("data-file.path-directory") + .stringType() + .noDefaultValue() + .withDescription("Specify the path directory of data files."); + public static final ConfigOption CHANGELOG_FILE_PREFIX = key("changelog-file.prefix") .stringType() @@ -1638,6 +1644,11 @@ public String dataFilePrefix() { return options.get(DATA_FILE_PREFIX); } + @Nullable + public String dataFilePathDirectory() { + return options.get(DATA_FILE_PATH_DIRECTORY); + } + public String changelogFilePrefix() { return options.get(CHANGELOG_FILE_PREFIX); } diff --git a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java index e6d63149149c4..1caff252a6547 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java @@ -105,16 +105,21 @@ protected AbstractFileStore( @Override public FileStorePathFactory pathFactory() { + return pathFactory(options.fileFormat().getFormatIdentifier()); + } + + protected FileStorePathFactory pathFactory(String format) { return new FileStorePathFactory( options.path(), partitionType, options.partitionDefaultName(), - options.fileFormat().getFormatIdentifier(), + format, options.dataFilePrefix(), options.changelogFilePrefix(), options.legacyPartitionName(), options.fileSuffixIncludeCompression(), - options.fileCompression()); + options.fileCompression(), + options.dataFilePathDirectory()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java index 1c87a087e44eb..8cf45105c01b8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java @@ -194,20 +194,7 @@ private Map format2PathFactory() { Map pathFactoryMap = new HashMap<>(); Set formats = new HashSet<>(options.fileFormatPerLevel().values()); formats.add(options.fileFormat().getFormatIdentifier()); - formats.forEach( - format -> - pathFactoryMap.put( - format, - new FileStorePathFactory( - options.path(), - partitionType, - options.partitionDefaultName(), - format, - options.dataFilePrefix(), - options.changelogFilePrefix(), - options.legacyPartitionName(), - options.fileSuffixIncludeCompression(), - options.fileCompression()))); + formats.forEach(format -> pathFactoryMap.put(format, pathFactory(format))); return pathFactoryMap; } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index fcdc4634d74ab..8896ec3286802 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -24,6 +24,7 @@ import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.types.RowType; +import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; import java.util.List; @@ -46,6 +47,8 @@ public class FileStorePathFactory { private final boolean fileSuffixIncludeCompression; private final String fileCompression; + @Nullable private final String dataFilePathDirectory; + private final AtomicInteger manifestFileCount; private final AtomicInteger manifestListCount; private final AtomicInteger indexManifestCount; @@ -61,8 +64,10 @@ public FileStorePathFactory( String changelogFilePrefix, boolean legacyPartitionName, boolean fileSuffixIncludeCompression, - String fileCompression) { + String fileCompression, + @Nullable String dataFilePathDirectory) { this.root = root; + this.dataFilePathDirectory = dataFilePathDirectory; this.uuid = UUID.randomUUID().toString(); this.partitionComputer = @@ -125,7 +130,11 @@ public DataFilePathFactory createDataFilePathFactory(BinaryRow partition, int bu } public Path bucketPath(BinaryRow partition, int bucket) { - return new Path(root + "/" + relativePartitionAndBucketPath(partition, bucket)); + Path dataFileRoot = this.root; + if (dataFilePathDirectory != null) { + dataFileRoot = new Path(dataFileRoot, dataFilePathDirectory); + } + return new Path(dataFileRoot + "/" + relativePartitionAndBucketPath(partition, bucket)); } public Path relativePartitionAndBucketPath(BinaryRow partition, int bucket) { diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index 52d56afad8134..b648e2af89727 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -231,7 +231,8 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); Options options = new Options(); @@ -250,7 +251,8 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue())); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null)); return KeyValueFileWriterFactory.builder( fileIO, diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java index 5e69035ca8348..52d82e76be2ab 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java @@ -149,7 +149,8 @@ protected ManifestFile createManifestFile(String pathStr) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()), + CoreOptions.FILE_COMPRESSION.defaultValue(), + null), Long.MAX_VALUE, null) .create(); diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java index 34cca41e61a1f..089e11656a995 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java @@ -105,7 +105,8 @@ private ManifestFile createManifestFile(String pathStr) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); return new ManifestFile.Factory( diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java index ce4f7b8072d2a..5bf01f32cb07d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java @@ -109,7 +109,8 @@ private ManifestList createManifestList(String pathStr) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); return new ManifestList.Factory(FileIOFinder.find(path), avro, "zstd", pathFactory, null) .create(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java index 922221bb8dc99..01d4e89af95d6 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java @@ -78,6 +78,7 @@ import static org.apache.paimon.CoreOptions.BUCKET; import static org.apache.paimon.CoreOptions.BUCKET_KEY; +import static org.apache.paimon.CoreOptions.DATA_FILE_PATH_DIRECTORY; import static org.apache.paimon.CoreOptions.FILE_INDEX_IN_MANIFEST_THRESHOLD; import static org.apache.paimon.CoreOptions.METADATA_STATS_MODE; import static org.apache.paimon.io.DataFileTestUtils.row; @@ -143,6 +144,26 @@ public void testBatchReadWrite() throws Exception { "2|21|201|binary|varbinary|mapKey:mapVal|multiset")); } + @Test + public void testReadWriteWithDataDirectory() throws Exception { + Consumer optionsSetter = options -> options.set(DATA_FILE_PATH_DIRECTORY, "data"); + writeData(optionsSetter); + FileStoreTable table = createFileStoreTable(optionsSetter); + + assertThat(table.fileIO().exists(new Path(tablePath, "data/pt=1"))).isTrue(); + + List splits = toSplits(table.newSnapshotReader().read().dataSplits()); + TableRead read = table.newRead(); + assertThat(getResult(read, splits, binaryRow(1), 0, BATCH_ROW_TO_STRING)) + .hasSameElementsAs( + Arrays.asList( + "1|10|100|binary|varbinary|mapKey:mapVal|multiset", + "1|11|101|binary|varbinary|mapKey:mapVal|multiset", + "1|12|102|binary|varbinary|mapKey:mapVal|multiset", + "1|11|101|binary|varbinary|mapKey:mapVal|multiset", + "1|12|102|binary|varbinary|mapKey:mapVal|multiset")); + } + @Test public void testBatchRecordsWrite() throws Exception { FileStoreTable table = createFileStoreTable(); diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java index d4d45b3129614..6ca15cf1503d7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java @@ -90,7 +90,8 @@ public void testCreateDataFilePathFactoryWithPartition() { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); assertPartition("20211224", 16, pathFactory, "/dt=20211224/hr=16"); assertPartition("20211224", null, pathFactory, "/dt=20211224/hr=default"); @@ -130,6 +131,7 @@ public static FileStorePathFactory createNonPartFactory(Path root) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index 85679e5fd30af..17e63700faac7 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -110,7 +110,8 @@ public TestChangelogDataReadWrite(String root) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); this.snapshotManager = new SnapshotManager(LocalFileIO.create(), new Path(root)); this.commitUser = UUID.randomUUID().toString(); } diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java index 7da8f2e6b7fe0..0360def685b6b 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java @@ -160,7 +160,8 @@ protected void foreachIndexReader(Consumer consumer) CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); Table table = fileSystemCatalog.getTable(Identifier.create("db", "T")); ReadBuilder readBuilder = table.newReadBuilder(); From 369141963500bb938cf40559e6418abf0f41cedd Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Tue, 10 Dec 2024 14:05:17 +0800 Subject: [PATCH 130/157] [core] Trim key field in reading, map it to value field (#4651) This closes #4651. --------- Co-authored-by: tsreaper --- .../data/columnar/ColumnarRowIterator.java | 2 +- .../apache/paimon/table/SpecialFields.java | 4 + .../paimon/utils/VectorMappingUtils.java | 3 +- .../paimon/utils/VectorMappingUtilsTest.java | 2 +- .../paimon/io/DataFileRecordReader.java | 1 + .../paimon/utils/BulkFormatMapping.java | 109 +++++++++++-- .../paimon/manifest/ManifestFileMetaTest.java | 2 +- .../paimon/utils/BulkFormatMappingTest.java | 147 ++++++++++++++++++ .../source/TestChangelogDataReadWrite.java | 4 +- 9 files changed, 257 insertions(+), 17 deletions(-) create mode 100644 paimon-core/src/test/java/org/apache/paimon/utils/BulkFormatMappingTest.java diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java index 27e3d1c1ddadf..874c221348646 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java @@ -95,7 +95,7 @@ public ColumnarRowIterator mapping( vectors = VectorMappingUtils.createPartitionMappedVectors(partitionInfo, vectors); } if (indexMapping != null) { - vectors = VectorMappingUtils.createIndexMappedVectors(indexMapping, vectors); + vectors = VectorMappingUtils.createMappedVectors(indexMapping, vectors); } return copy(vectors); } diff --git a/paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java b/paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java index d438bfb0ffe9e..3288276a1f649 100644 --- a/paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java +++ b/paimon-common/src/main/java/org/apache/paimon/table/SpecialFields.java @@ -99,6 +99,10 @@ public static boolean isSystemField(String field) { return field.startsWith(KEY_FIELD_PREFIX) || SYSTEM_FIELD_NAMES.contains(field); } + public static boolean isKeyField(String field) { + return field.startsWith(KEY_FIELD_PREFIX); + } + // ---------------------------------------------------------------------------------------- // Structured type fields // ---------------------------------------------------------------------------------------- diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java index 8b01e644de576..02b011a2f1cf3 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/VectorMappingUtils.java @@ -97,8 +97,7 @@ public static ColumnVector createFixedVector( return dataType.accept(visitor); } - public static ColumnVector[] createIndexMappedVectors( - int[] indexMapping, ColumnVector[] vectors) { + public static ColumnVector[] createMappedVectors(int[] indexMapping, ColumnVector[] vectors) { ColumnVector[] newVectors = new ColumnVector[indexMapping.length]; for (int i = 0; i < indexMapping.length; i++) { int realIndex = indexMapping[i]; diff --git a/paimon-common/src/test/java/org/apache/paimon/utils/VectorMappingUtilsTest.java b/paimon-common/src/test/java/org/apache/paimon/utils/VectorMappingUtilsTest.java index c5fac9c880db9..571a0d7189d6c 100644 --- a/paimon-common/src/test/java/org/apache/paimon/utils/VectorMappingUtilsTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/utils/VectorMappingUtilsTest.java @@ -82,7 +82,7 @@ public void testCreateIndexMappedVectors() { int[] mapping = new int[] {0, 2, 1, 3, 2, 3, 1, 0, 4}; ColumnVector[] newColumnVectors = - VectorMappingUtils.createIndexMappedVectors(mapping, columnVectors); + VectorMappingUtils.createMappedVectors(mapping, columnVectors); for (int i = 0; i < mapping.length; i++) { Assertions.assertThat(newColumnVectors[i]).isEqualTo(columnVectors[mapping[i]]); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java index d2559fe6240b2..16fad55a49a27 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java @@ -76,6 +76,7 @@ public FileRecordIterator readBatch() throws IOException { PartitionSettedRow.from(partitionInfo); iterator = iterator.transform(partitionSettedRow::replaceRow); } + if (indexMapping != null) { final ProjectedRow projectedRow = ProjectedRow.from(indexMapping); iterator = iterator.transform(projectedRow::replaceRow); diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BulkFormatMapping.java b/paimon-core/src/main/java/org/apache/paimon/utils/BulkFormatMapping.java index 037622f95f1e6..58ef924df1786 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/BulkFormatMapping.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/BulkFormatMapping.java @@ -26,6 +26,7 @@ import org.apache.paimon.schema.IndexCastMapping; import org.apache.paimon.schema.SchemaEvolutionUtil; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; @@ -35,17 +36,25 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.function.Function; import static org.apache.paimon.predicate.PredicateBuilder.excludePredicateWithFields; +import static org.apache.paimon.table.SpecialFields.KEY_FIELD_ID_START; /** Class with index mapping and bulk format. */ public class BulkFormatMapping { + // Index mapping from data schema fields to table schema fields, this is used to realize paimon + // schema evolution. And it combines trimeedKeyMapping, which maps key fields to the value + // fields @Nullable private final int[] indexMapping; + // help indexMapping to cast different data type @Nullable private final CastFieldGetter[] castMapping; + // partition fields mapping, add partition fields to the read fields @Nullable private final Pair partitionPair; private final FormatReaderFactory bulkFormat; private final TableSchema dataSchema; @@ -54,11 +63,12 @@ public class BulkFormatMapping { public BulkFormatMapping( @Nullable int[] indexMapping, @Nullable CastFieldGetter[] castMapping, + @Nullable int[] trimmedKeyMapping, @Nullable Pair partitionPair, FormatReaderFactory bulkFormat, TableSchema dataSchema, List dataFilters) { - this.indexMapping = indexMapping; + this.indexMapping = combine(indexMapping, trimmedKeyMapping); this.castMapping = castMapping; this.bulkFormat = bulkFormat; this.partitionPair = partitionPair; @@ -66,6 +76,26 @@ public BulkFormatMapping( this.dataFilters = dataFilters; } + private int[] combine(@Nullable int[] indexMapping, @Nullable int[] trimmedKeyMapping) { + if (indexMapping == null) { + return trimmedKeyMapping; + } + if (trimmedKeyMapping == null) { + return indexMapping; + } + + int[] combined = new int[indexMapping.length]; + + for (int i = 0; i < indexMapping.length; i++) { + if (indexMapping[i] < 0) { + combined[i] = indexMapping[i]; + } else { + combined[i] = trimmedKeyMapping[indexMapping[i]]; + } + } + return combined; + } + @Nullable public int[] getIndexMapping() { return indexMapping; @@ -112,24 +142,46 @@ public BulkFormatMappingBuilder( this.filters = filters; } + /** + * There are three steps here to build BulkFormatMapping: + * + *

    1. Calculate the readDataFields, which is what we intend to read from the data schema. + * Meanwhile, generate the indexCastMapping, which is used to map the index of the + * readDataFields to the index of the data schema. + * + *

    2. Calculate the mapping to trim _KEY_ fields. For example: we want _KEY_a, _KEY_b, + * _FIELD_SEQUENCE, _ROW_KIND, a, b, c, d, e, f, g from the data, but actually we don't need + * to read _KEY_a and a, _KEY_b and b the same time, so we need to trim them. So we mapping + * it: read before: _KEY_a, _KEY_b, _FIELD_SEQUENCE, _ROW_KIND, a, b, c, d, e, f, g read + * after: a, b, _FIELD_SEQUENCE, _ROW_KIND, c, d, e, f, g and the mapping is + * [0,1,2,3,0,1,4,5,6,7,8], it converts the [read after] columns to [read before] columns. + * + *

    3. We want read much fewer fields than readDataFields, so we kick out the partition + * fields. We generate the partitionMappingAndFieldsWithoutPartitionPair which helps reduce + * the real read fields and tell us how to map it back. + */ public BulkFormatMapping build( String formatIdentifier, TableSchema tableSchema, TableSchema dataSchema) { - List readDataFields = readDataFields(dataSchema); - + // extract the whole data fields in logic. + List allDataFields = fieldsExtractor.apply(dataSchema); + List readDataFields = readDataFields(allDataFields); // build index cast mapping IndexCastMapping indexCastMapping = SchemaEvolutionUtil.createIndexCastMapping(readTableFields, readDataFields); + // map from key fields reading to value fields reading + Pair trimmedKeyPair = trimKeyFields(readDataFields, allDataFields); + // build partition mapping and filter partition fields Pair, List> partitionMappingAndFieldsWithoutPartitionPair = - PartitionUtils.constructPartitionMapping(dataSchema, readDataFields); + PartitionUtils.constructPartitionMapping( + dataSchema, trimmedKeyPair.getRight().getFields()); Pair partitionMapping = partitionMappingAndFieldsWithoutPartitionPair.getLeft(); - // build read row type - RowType readDataRowType = + RowType readRowType = new RowType(partitionMappingAndFieldsWithoutPartitionPair.getRight()); // build read filters @@ -138,18 +190,55 @@ public BulkFormatMapping build( return new BulkFormatMapping( indexCastMapping.getIndexMapping(), indexCastMapping.getCastMapping(), + trimmedKeyPair.getLeft(), partitionMapping, formatDiscover .discover(formatIdentifier) - .createReaderFactory(readDataRowType, readFilters), + .createReaderFactory(readRowType, readFilters), dataSchema, readFilters); } - private List readDataFields(TableSchema dataSchema) { - List dataFields = fieldsExtractor.apply(dataSchema); + static Pair trimKeyFields( + List fieldsWithoutPartition, List fields) { + int[] map = new int[fieldsWithoutPartition.size()]; + List trimmedFields = new ArrayList<>(); + Map fieldMap = new HashMap<>(); + Map positionMap = new HashMap<>(); + + for (DataField field : fields) { + fieldMap.put(field.id(), field); + } + + for (int i = 0; i < fieldsWithoutPartition.size(); i++) { + DataField field = fieldsWithoutPartition.get(i); + boolean keyField = SpecialFields.isKeyField(field.name()); + int id = keyField ? field.id() - KEY_FIELD_ID_START : field.id(); + // field in data schema + DataField f = fieldMap.get(id); + + if (f != null) { + if (positionMap.containsKey(id)) { + map[i] = positionMap.get(id); + } else { + map[i] = positionMap.computeIfAbsent(id, k -> trimmedFields.size()); + // If the target field is not key field, we remain what it is, because it + // may be projected. Example: the target field is a row type, but only read + // the few fields in it. If we simply trimmedFields.add(f), we will read + // more fields than we need. + trimmedFields.add(keyField ? f : field); + } + } else { + throw new RuntimeException("Can't find field with id: " + id + " in fields."); + } + } + + return Pair.of(map, new RowType(trimmedFields)); + } + + private List readDataFields(List allDataFields) { List readDataFields = new ArrayList<>(); - for (DataField dataField : dataFields) { + for (DataField dataField : allDataFields) { readTableFields.stream() .filter(f -> f.id() == dataField.id()) .findFirst() diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java index 9384eb88824d1..1be5993fb0d08 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java @@ -452,7 +452,7 @@ public void testIdentifierAfterFullCompaction() throws Exception { containSameIdentifyEntryFile(fullCompacted, entryIdentifierExpected); } - @RepeatedTest(1000) + @RepeatedTest(10) public void testRandomFullCompaction() throws Exception { List input = new ArrayList<>(); Set manifestEntrySet = new HashSet<>(); diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/BulkFormatMappingTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/BulkFormatMappingTest.java new file mode 100644 index 0000000000000..4d5d6e32e85dc --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/utils/BulkFormatMappingTest.java @@ -0,0 +1,147 @@ +/* + * 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.paimon.utils; + +import org.apache.paimon.schema.IndexCastMapping; +import org.apache.paimon.schema.SchemaEvolutionUtil; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +/** Test for {@link BulkFormatMapping.BulkFormatMappingBuilder}. */ +public class BulkFormatMappingTest { + + @Test + public void testTrimKeyFields() { + List keyFields = new ArrayList<>(); + List allFields = new ArrayList<>(); + List testFields = new ArrayList<>(); + + for (int i = 0; i < 10; i++) { + keyFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + i, + SpecialFields.KEY_FIELD_PREFIX + i, + DataTypes.STRING())); + } + + allFields.addAll(keyFields); + for (int i = 0; i < 20; i++) { + allFields.add(new DataField(i, String.valueOf(i), DataTypes.STRING())); + } + + testFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 1, + SpecialFields.KEY_FIELD_PREFIX + 1, + DataTypes.STRING())); + testFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 3, + SpecialFields.KEY_FIELD_PREFIX + 3, + DataTypes.STRING())); + testFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 5, + SpecialFields.KEY_FIELD_PREFIX + 5, + DataTypes.STRING())); + testFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 7, + SpecialFields.KEY_FIELD_PREFIX + 7, + DataTypes.STRING())); + testFields.add(new DataField(3, String.valueOf(3), DataTypes.STRING())); + testFields.add(new DataField(4, String.valueOf(4), DataTypes.STRING())); + testFields.add(new DataField(5, String.valueOf(5), DataTypes.STRING())); + testFields.add(new DataField(1, String.valueOf(1), DataTypes.STRING())); + testFields.add(new DataField(6, String.valueOf(6), DataTypes.STRING())); + + Pair res = + BulkFormatMapping.BulkFormatMappingBuilder.trimKeyFields(testFields, allFields); + + Assertions.assertThat(res.getKey()).containsExactly(0, 1, 2, 3, 1, 4, 2, 0, 5); + + List fields = res.getRight().getFields(); + Assertions.assertThat(fields.size()).isEqualTo(6); + Assertions.assertThat(fields.get(0).id()).isEqualTo(1); + Assertions.assertThat(fields.get(1).id()).isEqualTo(3); + Assertions.assertThat(fields.get(2).id()).isEqualTo(5); + Assertions.assertThat(fields.get(3).id()).isEqualTo(7); + Assertions.assertThat(fields.get(4).id()).isEqualTo(4); + Assertions.assertThat(fields.get(5).id()).isEqualTo(6); + } + + @Test + public void testTrimKeyWithIndexMapping() { + List readTableFields = new ArrayList<>(); + List readDataFields = new ArrayList<>(); + + readTableFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 1, + SpecialFields.KEY_FIELD_PREFIX + "a", + DataTypes.STRING())); + readTableFields.add(new DataField(0, "0", DataTypes.STRING())); + readTableFields.add(new DataField(1, "a", DataTypes.STRING())); + readTableFields.add(new DataField(2, "2", DataTypes.STRING())); + readTableFields.add(new DataField(3, "3", DataTypes.STRING())); + + readDataFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 1, + SpecialFields.KEY_FIELD_PREFIX + "a", + DataTypes.STRING())); + readDataFields.add(new DataField(0, "0", DataTypes.STRING())); + readDataFields.add(new DataField(1, "a", DataTypes.STRING())); + readDataFields.add(new DataField(3, "3", DataTypes.STRING())); + + // build index cast mapping + IndexCastMapping indexCastMapping = + SchemaEvolutionUtil.createIndexCastMapping(readTableFields, readDataFields); + + // map from key fields reading to value fields reading + Pair trimmedKeyPair = + BulkFormatMapping.BulkFormatMappingBuilder.trimKeyFields( + readDataFields, readDataFields); + + BulkFormatMapping bulkFormatMapping = + new BulkFormatMapping( + indexCastMapping.getIndexMapping(), + indexCastMapping.getCastMapping(), + trimmedKeyPair.getLeft(), + null, + null, + null, + null); + + Assertions.assertThat(bulkFormatMapping.getIndexMapping()).containsExactly(0, 1, 0, -1, 2); + List trimmed = trimmedKeyPair.getRight().getFields(); + Assertions.assertThat(trimmed.get(0).id()).isEqualTo(1); + Assertions.assertThat(trimmed.get(1).id()).isEqualTo(0); + Assertions.assertThat(trimmed.get(2).id()).isEqualTo(3); + Assertions.assertThat(trimmed.size()).isEqualTo(3); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index 17e63700faac7..d2bb9eb982741 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -69,7 +69,7 @@ public class TestChangelogDataReadWrite { private static final RowType KEY_TYPE = new RowType(singletonList(new DataField(0, "k", new BigIntType()))); private static final RowType VALUE_TYPE = - new RowType(singletonList(new DataField(0, "v", new BigIntType()))); + new RowType(singletonList(new DataField(1, "v", new BigIntType()))); private static final RowType PARTITION_TYPE = new RowType(singletonList(new DataField(0, "p", new IntType()))); private static final Comparator COMPARATOR = @@ -87,7 +87,7 @@ public List keyFields(TableSchema schema) { @Override public List valueFields(TableSchema schema) { return Collections.singletonList( - new DataField(0, "v", new org.apache.paimon.types.BigIntType(false))); + new DataField(1, "v", new org.apache.paimon.types.BigIntType(false))); } }; From c6387a669d6cd0611c352dbfe4d241fea6a6484c Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Tue, 10 Dec 2024 14:41:37 +0800 Subject: [PATCH 131/157] [hive][spark] Support creating external table without schema when the table already exists (#4638) --- docs/content/spark/sql-ddl.md | 29 ++++++- docs/content/spark/sql-write.md | 31 +++++--- .../apache/paimon/schema/SchemaManager.java | 54 +++++++++---- .../org/apache/paimon/hive/HiveCatalog.java | 10 +-- .../org/apache/paimon/spark/SparkCatalog.java | 30 ++------ .../org/apache/paimon/spark/SparkSource.scala | 2 +- .../apache/paimon/spark/sql/DDLTestBase.scala | 2 +- .../sql/DDLWithHiveCatalogTestBase.scala | 75 ++++++++++++++----- 8 files changed, 157 insertions(+), 76 deletions(-) diff --git a/docs/content/spark/sql-ddl.md b/docs/content/spark/sql-ddl.md index 638a21a7042aa..cfe105f6ac006 100644 --- a/docs/content/spark/sql-ddl.md +++ b/docs/content/spark/sql-ddl.md @@ -156,6 +156,33 @@ CREATE TABLE my_table ( ); ``` +### Create External Table + +When the catalog's `metastore` type is `hive`, if the `location` is specified when creating a table, that table will be considered an external table; otherwise, it will be a managed table. + +When you drop an external table, only the metadata in Hive will be removed, and the actual data files will not be deleted; whereas dropping a managed table will also delete the data. + +```sql +CREATE TABLE my_table ( + user_id BIGINT, + item_id BIGINT, + behavior STRING, + dt STRING, + hh STRING +) PARTITIONED BY (dt, hh) TBLPROPERTIES ( + 'primary-key' = 'dt,hh,user_id' +) LOCATION '/path/to/table'; +``` + +Furthermore, if there is already data stored in the specified location, you can create the table without explicitly specifying the fields, partitions and props or other information. +In this case, the new table will inherit them all from the existing table’s metadata. + +However, if you manually specify them, you need to ensure that they are consistent with those of the existing table (props can be a subset). Therefore, it is strongly recommended not to specify them. + +```sql +CREATE TABLE my_table LOCATION '/path/to/table'; +``` + ### Create Table As Select Table can be created and populated by the results of a query, for example, we have a sql like this: `CREATE TABLE table_b AS SELECT id, name FORM table_a`, @@ -241,7 +268,7 @@ DROP VIEW v1; ``` ## Tag -### Create or Replace Tag +### Create Or Replace Tag Create or replace a tag syntax with the following options. - Create a tag with or without the snapshot id and time retention. - Create an existed tag is not failed if using `IF NOT EXISTS` syntax. diff --git a/docs/content/spark/sql-write.md b/docs/content/spark/sql-write.md index 5f4fa2dabc9f6..c3afcd3754c80 100644 --- a/docs/content/spark/sql-write.md +++ b/docs/content/spark/sql-write.md @@ -120,7 +120,17 @@ TRUNCATE TABLE my_table; ## Update Table -spark supports update PrimitiveType and StructType, for example: +Updates the column values for the rows that match a predicate. When no predicate is provided, update the column values for all rows. + +Note: + +{{< hint info >}} + +Update primary key columns is not supported when the target table is a primary key table. + +{{< /hint >}} + +Spark supports update PrimitiveType and StructType, for example: ```sql -- Syntax @@ -142,17 +152,22 @@ UPDATE t SET s.c2 = 'a_new' WHERE s.c1 = 1; ## Delete From Table +Deletes the rows that match a predicate. When no predicate is provided, deletes all rows. + ```sql DELETE FROM my_table WHERE currency = 'UNKNOWN'; ``` ## Merge Into Table -Paimon currently supports Merge Into syntax in Spark 3+, which allow a set of updates, insertions and deletions based on a source table in a single commit. +Merges a set of updates, insertions and deletions based on a source table into a target table. + +Note: + +{{< hint info >}} + +In update clause, to update primary key columns is not supported when the target table is a primary key table. -{{< hint into >}} -1. In update clause, to update primary key columns is not supported. -2. `WHEN NOT MATCHED BY SOURCE` syntax is not supported. {{< /hint >}} **Example: One** @@ -160,7 +175,6 @@ Paimon currently supports Merge Into syntax in Spark 3+, which allow a set of up This is a simple demo that, if a row exists in the target table update it, else insert it. ```sql - -- Here both source and target tables have the same schema: (a INT, b INT, c STRING), and a is a primary key. MERGE INTO target @@ -170,7 +184,6 @@ WHEN MATCHED THEN UPDATE SET * WHEN NOT MATCHED THEN INSERT * - ``` **Example: Two** @@ -178,7 +191,6 @@ THEN INSERT * This is a demo with multiple, conditional clauses. ```sql - -- Here both source and target tables have the same schema: (a INT, b INT, c STRING), and a is a primary key. MERGE INTO target @@ -194,15 +206,12 @@ WHEN NOT MATCHED AND c > 'c9' THEN INSERT (a, b, c) VALUES (a, b * 1.1, c) -- when not matched but meet the condition 3, then transform and insert this row; WHEN NOT MATCHED THEN INSERT * -- when not matched, insert this row without any transformation; - ``` ## Streaming Write {{< hint info >}} -Paimon currently supports Spark 3+ for streaming write. - Paimon Structured Streaming only supports the two `append` and `complete` modes. {{< /hint >}} diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index 83ddbccfef987..2139dca4a9907 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -208,24 +208,18 @@ public TableSchema createTable(Schema schema) throws Exception { return createTable(schema, false); } - public TableSchema createTable(Schema schema, boolean ignoreIfExistsSame) throws Exception { + public TableSchema createTable(Schema schema, boolean externalTable) throws Exception { while (true) { Optional latest = latest(); if (latest.isPresent()) { - TableSchema oldSchema = latest.get(); - boolean isSame = - Objects.equals(oldSchema.fields(), schema.fields()) - && Objects.equals(oldSchema.partitionKeys(), schema.partitionKeys()) - && Objects.equals(oldSchema.primaryKeys(), schema.primaryKeys()) - && Objects.equals(oldSchema.options(), schema.options()); - if (ignoreIfExistsSame && isSame) { - return oldSchema; + TableSchema latestSchema = latest.get(); + if (externalTable) { + checkSchemaForExternalTable(latestSchema, schema); + return latestSchema; + } else { + throw new IllegalStateException( + "Schema in filesystem exists, creation is not allowed."); } - - throw new IllegalStateException( - "Schema in filesystem exists, please use updating," - + " latest schema is: " - + oldSchema); } List fields = schema.fields(); @@ -254,6 +248,38 @@ public TableSchema createTable(Schema schema, boolean ignoreIfExistsSame) throws } } + private void checkSchemaForExternalTable(TableSchema existsSchema, Schema newSchema) { + // When creating an external table, if the table already exists in the location, we can + // choose not to specify the fields. + if (newSchema.fields().isEmpty() + // When the fields are explicitly specified, we need check for consistency. + || (Objects.equals(existsSchema.fields(), newSchema.fields()) + && Objects.equals(existsSchema.partitionKeys(), newSchema.partitionKeys()) + && Objects.equals(existsSchema.primaryKeys(), newSchema.primaryKeys()))) { + // check for options + Map existsOptions = existsSchema.options(); + Map newOptions = newSchema.options(); + newOptions.forEach( + (key, value) -> { + if (!key.equals(Catalog.OWNER_PROP) + && (!existsOptions.containsKey(key) + || !existsOptions.get(key).equals(value))) { + throw new RuntimeException( + "New schema's options are not equal to the exists schema's, new schema: " + + newOptions + + ", exists schema: " + + existsOptions); + } + }); + } else { + throw new RuntimeException( + "New schema is not equal to exists schema, new schema: " + + newSchema + + ", exists schema: " + + existsSchema); + } + } + /** Update {@link SchemaChange}s. */ public TableSchema commitChanges(SchemaChange... changes) throws Exception { return commitChanges(Arrays.asList(changes)); diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 151e2b4d2c076..c74ede9815465 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -720,11 +720,7 @@ protected void createTableImpl(Identifier identifier, Schema schema) { try { tableSchema = schemaManager(identifier, location).createTable(schema, externalTable); } catch (Exception e) { - throw new RuntimeException( - "Failed to commit changes of table " - + identifier.getFullName() - + " to underlying files.", - e); + throw new RuntimeException("Failed to create table " + identifier.getFullName(), e); } try { @@ -735,7 +731,9 @@ protected void createTableImpl(Identifier identifier, Schema schema) { identifier, tableSchema, location, externalTable))); } catch (Exception e) { try { - fileIO.deleteDirectoryQuietly(location); + if (!externalTable) { + fileIO.deleteDirectoryQuietly(location); + } } catch (Exception ee) { LOG.error("Delete directory[{}] fail for table {}", location, identifier, ee); } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index 5ad1b13b7c7b6..d6318c723fe0e 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -75,7 +75,6 @@ import static org.apache.paimon.spark.util.OptionUtils.copyWithSQLConf; import static org.apache.paimon.spark.utils.CatalogUtils.checkNamespace; import static org.apache.paimon.spark.utils.CatalogUtils.toIdentifier; -import static org.apache.paimon.utils.Preconditions.checkArgument; /** Spark {@link TableCatalog} for paimon. */ public class SparkCatalog extends SparkBaseCatalog implements SupportFunction, SupportView { @@ -298,26 +297,8 @@ public org.apache.spark.sql.connector.catalog.Table createTable( Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException { try { - String provider = properties.get(TableCatalog.PROP_PROVIDER); - if ((!usePaimon(provider)) - && SparkSource.FORMAT_NAMES().contains(provider.toLowerCase())) { - Map newProperties = new HashMap<>(properties); - newProperties.put(TYPE.key(), FORMAT_TABLE.toString()); - newProperties.put(FILE_FORMAT.key(), provider.toLowerCase()); - catalog.createTable( - toIdentifier(ident), - toInitialSchema(schema, partitions, newProperties), - false); - } else { - checkArgument( - usePaimon(provider), - "SparkCatalog can only create paimon table, but current provider is %s", - provider); - catalog.createTable( - toIdentifier(ident), - toInitialSchema(schema, partitions, properties), - false); - } + catalog.createTable( + toIdentifier(ident), toInitialSchema(schema, partitions, properties), false); return loadTable(ident); } catch (Catalog.TableAlreadyExistException e) { throw new TableAlreadyExistsException(ident); @@ -406,9 +387,12 @@ private static SchemaChange.Move getMove( private Schema toInitialSchema( StructType schema, Transform[] partitions, Map properties) { Map normalizedProperties = new HashMap<>(properties); - if (!normalizedProperties.containsKey(TableCatalog.PROP_PROVIDER)) { - normalizedProperties.put(TableCatalog.PROP_PROVIDER, SparkSource.NAME()); + String provider = properties.get(TableCatalog.PROP_PROVIDER); + if (!usePaimon(provider) && SparkSource.FORMAT_NAMES().contains(provider.toLowerCase())) { + normalizedProperties.put(TYPE.key(), FORMAT_TABLE.toString()); + normalizedProperties.put(FILE_FORMAT.key(), provider.toLowerCase()); } + normalizedProperties.remove(TableCatalog.PROP_PROVIDER); normalizedProperties.remove(PRIMARY_KEY_IDENTIFIER); normalizedProperties.remove(TableCatalog.PROP_COMMENT); if (normalizedProperties.containsKey(TableCatalog.PROP_LOCATION)) { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala index 0170a29f68d36..d80d7350a6554 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala @@ -118,7 +118,7 @@ object SparkSource { val NAME = "paimon" - val FORMAT_NAMES = Seq("csv", "orc", "parquet") + val FORMAT_NAMES: Seq[String] = Seq("csv", "orc", "parquet") def toBaseRelation(table: FileStoreTable, _sqlContext: SQLContext): BaseRelation = { new BaseRelation { diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala index 6ad5274496a90..3ed2c98306fb3 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala @@ -161,7 +161,7 @@ abstract class DDLTestBase extends PaimonSparkTestBase { test("Paimon DDL: create table without using paimon") { withTable("paimon_tbl") { sql("CREATE TABLE paimon_tbl (id int)") - assert(loadTable("paimon_tbl").options().get("provider").equals("paimon")) + assert(!loadTable("paimon_tbl").options().containsKey("provider")) } } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index e99e4434ef7f1..1189f1f2906b3 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -326,13 +326,7 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { spark.sql( s"CREATE TABLE external_tbl (id INT) USING paimon LOCATION '$expertTbLocation'") checkAnswer(spark.sql("SELECT * FROM external_tbl"), Row(1)) - assert( - loadTable("paimon_db", "external_tbl") - .location() - .toString - .split(':') - .apply(1) - .equals(expertTbLocation)) + assert(getActualTableLocation("paimon_db", "external_tbl").equals(expertTbLocation)) // create managed table spark.sql(s"CREATE TABLE managed_tbl (id INT) USING paimon") @@ -373,12 +367,8 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { spark.sql("ALTER TABLE external_tbl RENAME TO external_tbl_renamed") checkAnswer(spark.sql("SELECT * FROM external_tbl_renamed"), Row(1)) assert( - loadTable("paimon_db", "external_tbl_renamed") - .location() - .toString - .split(':') - .apply(1) - .equals(expertTbLocation)) + getActualTableLocation("paimon_db", "external_tbl_renamed").equals( + expertTbLocation)) // create managed table spark.sql(s"CREATE TABLE managed_tbl (id INT) USING paimon") @@ -389,12 +379,55 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { spark.sql("ALTER TABLE managed_tbl RENAME TO managed_tbl_renamed") checkAnswer(spark.sql("SELECT * FROM managed_tbl_renamed"), Row(1)) assert( - !loadTable("paimon_db", "managed_tbl_renamed") - .location() - .toString - .split(':') - .apply(1) - .equals(managedTbLocation.toString)) + !getActualTableLocation("paimon_db", "managed_tbl_renamed").equals( + managedTbLocation.toString)) + } + } + } + } + } + + test("Paimon DDL with hive catalog: create external table without schema") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + withTempDir { + tbLocation => + withDatabase("paimon_db") { + spark.sql(s"CREATE DATABASE IF NOT EXISTS paimon_db") + spark.sql(s"USE paimon_db") + withTable("t1", "t2", "t3", "t4", "t5") { + val expertTbLocation = tbLocation.getCanonicalPath + spark.sql(s""" + |CREATE TABLE t1 (id INT, pt INT) USING paimon + |PARTITIONED BY (pt) + |TBLPROPERTIES('primary-key' = 'id', 'k1' = 'v1') + |LOCATION '$expertTbLocation' + |""".stripMargin) + spark.sql("INSERT INTO t1 VALUES (1, 1)") + + // create table without schema + spark.sql(s"CREATE TABLE t2 USING paimon LOCATION '$expertTbLocation'") + checkAnswer(spark.sql("SELECT * FROM t2"), Row(1, 1)) + assert(getActualTableLocation("paimon_db", "t2").equals(expertTbLocation)) + + // create table with wrong schema + intercept[Exception] { + spark.sql( + s"CREATE TABLE t3 (fake_col INT) USING paimon LOCATION '$expertTbLocation'") + } + + // create table with exists props + spark.sql( + s"CREATE TABLE t4 USING paimon TBLPROPERTIES ('k1' = 'v1') LOCATION '$expertTbLocation'") + checkAnswer(spark.sql("SELECT * FROM t4"), Row(1, 1)) + assert(getActualTableLocation("paimon_db", "t4").equals(expertTbLocation)) + + // create table with new props + intercept[Exception] { + spark.sql( + s"CREATE TABLE t5 USING paimon TBLPROPERTIES ('k2' = 'v2') LOCATION '$expertTbLocation'") + } } } } @@ -445,4 +478,8 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { .toMap tableProps("path").split(":")(1) } + + def getActualTableLocation(dbName: String, tblName: String): String = { + loadTable(dbName, tblName).location().toString.split(':').apply(1) + } } From f8c33c5d72cbe16977ca842f41bfb8a3e32285f7 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Tue, 10 Dec 2024 15:01:43 +0800 Subject: [PATCH 132/157] [hotfix] Fix flaky test of orc tests in ArrowBatchConverterTest (#4673) --- .../paimon/arrow/converter/ArrowBatchConverterTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowBatchConverterTest.java b/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowBatchConverterTest.java index c726283f0044b..aef589d912423 100644 --- a/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowBatchConverterTest.java +++ b/paimon-arrow/src/test/java/org/apache/paimon/arrow/converter/ArrowBatchConverterTest.java @@ -910,8 +910,9 @@ private boolean isVectorizedWithDv(RecordReader.RecordIterator iter private Object[] randomRowValues(boolean[] nullable) { Object[] values = new Object[18]; - values[0] = BinaryString.fromString(StringUtils.getRandomString(RND, 10, 10)); - values[1] = BinaryString.fromString(StringUtils.getRandomString(RND, 1, 20)); + // The orc char reader will trim the string. See TreeReaderFactory.CharTreeReader + values[0] = BinaryString.fromString(StringUtils.getRandomString(RND, 9, 9) + "A"); + values[1] = BinaryString.fromString(StringUtils.getRandomString(RND, 1, 19) + "A"); values[2] = RND.nextBoolean(); values[3] = randomBytes(10, 10); values[4] = randomBytes(1, 20); From 7400979e0a3988fc72298c1bbcf762921c307e78 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Tue, 10 Dec 2024 16:00:42 +0800 Subject: [PATCH 133/157] [orc] Row group filter push down cannot work with bitmap index --- .../apache/paimon/format/FormatReaderContext.java | 1 + .../apache/paimon/format/FormatReaderFactory.java | 3 +++ .../java/org/apache/orc/impl/RecordReaderImpl.java | 4 +++- .../apache/paimon/format/orc/OrcReaderFactory.java | 13 ++++++++----- 4 files changed, 15 insertions(+), 6 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java index 0d3dd7c79ff31..cae6a977e6152 100644 --- a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java +++ b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderContext.java @@ -60,6 +60,7 @@ public long fileSize() { return fileSize; } + @Nullable @Override public FileIndexResult fileIndex() { return fileIndexResult; diff --git a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java index d8af3e2fe37ca..5ef084ec4d344 100644 --- a/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java +++ b/paimon-common/src/main/java/org/apache/paimon/format/FormatReaderFactory.java @@ -25,6 +25,8 @@ import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader; +import javax.annotation.Nullable; + import java.io.IOException; /** A factory to create {@link RecordReader} for file. */ @@ -41,6 +43,7 @@ interface Context { long fileSize(); + @Nullable FileIndexResult fileIndex(); } } diff --git a/paimon-format/src/main/java/org/apache/orc/impl/RecordReaderImpl.java b/paimon-format/src/main/java/org/apache/orc/impl/RecordReaderImpl.java index 6c3af4e50043f..93aa0719caea5 100644 --- a/paimon-format/src/main/java/org/apache/orc/impl/RecordReaderImpl.java +++ b/paimon-format/src/main/java/org/apache/orc/impl/RecordReaderImpl.java @@ -62,6 +62,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; import java.math.BigDecimal; import java.sql.Timestamp; @@ -127,7 +129,7 @@ public class RecordReaderImpl implements RecordReader { private final boolean noSelectedVector; // identifies whether the file has bad bloom filters that we should not use. private final boolean skipBloomFilters; - private final FileIndexResult fileIndexResult; + @Nullable private final FileIndexResult fileIndexResult; static final String[] BAD_CPP_BLOOM_FILTER_VERSIONS = { "1.6.0", "1.6.1", "1.6.2", "1.6.3", "1.6.4", "1.6.5", "1.6.6", "1.6.7", "1.6.8", "1.6.9", "1.6.10", "1.6.11", "1.7.0" diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java index ee0f8a55c0346..db17357bfd705 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.data.columnar.VectorizedColumnBatch; import org.apache.paimon.fileindex.FileIndexResult; +import org.apache.paimon.fileindex.bitmap.BitmapIndexResult; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.OrcFormatReaderContext; import org.apache.paimon.format.fs.HadoopReadOnlyFileSystem; @@ -258,7 +259,7 @@ private static RecordReader createRecordReader( org.apache.paimon.fs.Path path, long splitStart, long splitLength, - FileIndexResult fileIndexResult, + @Nullable FileIndexResult fileIndexResult, boolean deletionVectorsEnabled) throws IOException { org.apache.orc.Reader orcReader = createReader(conf, fileIO, path, fileIndexResult); @@ -276,9 +277,11 @@ private static RecordReader createRecordReader( .skipCorruptRecords(OrcConf.SKIP_CORRUPT_DATA.getBoolean(conf)) .tolerateMissingSchema( OrcConf.TOLERATE_MISSING_SCHEMA.getBoolean(conf)); - if (!conjunctPredicates.isEmpty() && !deletionVectorsEnabled) { - // deletion vectors can not enable this feature, cased by getRowNumber would be - // changed. + if (!conjunctPredicates.isEmpty() + && !deletionVectorsEnabled + && !(fileIndexResult instanceof BitmapIndexResult)) { + // row group filter push down will make row number change incorrect + // so deletion vectors mode and bitmap index cannot work with row group push down options.useSelected(OrcConf.READER_USE_SELECTED.getBoolean(conf)); options.allowSARGToFilter(OrcConf.ALLOW_SARG_TO_FILTER.getBoolean(conf)); } @@ -342,7 +345,7 @@ public static org.apache.orc.Reader createReader( org.apache.hadoop.conf.Configuration conf, FileIO fileIO, org.apache.paimon.fs.Path path, - FileIndexResult fileIndexResult) + @Nullable FileIndexResult fileIndexResult) throws IOException { // open ORC file and create reader org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(path.toUri()); From 84eadce2b95eec0c36fe13dde10958080cd80f28 Mon Sep 17 00:00:00 2001 From: Giannis Polyzos Date: Tue, 10 Dec 2024 10:12:30 +0200 Subject: [PATCH 134/157] [docs] add glue metastore repo for iceberg compability (#4674) --- docs/content/migration/iceberg-compatibility.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/migration/iceberg-compatibility.md b/docs/content/migration/iceberg-compatibility.md index 54018b073f91f..b6fcaa2826155 100644 --- a/docs/content/migration/iceberg-compatibility.md +++ b/docs/content/migration/iceberg-compatibility.md @@ -397,6 +397,7 @@ you also need to set some (or all) of the following table options when creating You can use Hive Catalog to connect AWS Glue metastore, you can use set `'metadata.iceberg.hive-client-class'` to `'com.amazonaws.glue.catalog.metastore.AWSCatalogMetastoreClient'`. +> **Note:** You can use this [repo](https://github.com/promotedai/aws-glue-data-catalog-client-for-apache-hive-metastore) to build the required jar, include it in your path and configure the AWSCatalogMetastoreClient. ## AWS Athena AWS Athena may use old manifest reader to read Iceberg manifest by names, we should let Paimon producing legacy Iceberg From 90a37c8b81ed8c2ec38703909c3559ae83630466 Mon Sep 17 00:00:00 2001 From: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com> Date: Tue, 10 Dec 2024 16:15:16 +0800 Subject: [PATCH 135/157] [core] Improve fault tolerance for data spill to disk. (#4675) --- .../paimon/disk/FileChannelManagerImpl.java | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/disk/FileChannelManagerImpl.java b/paimon-core/src/main/java/org/apache/paimon/disk/FileChannelManagerImpl.java index ce175e90bbd1e..99690d426fb52 100644 --- a/paimon-core/src/main/java/org/apache/paimon/disk/FileChannelManagerImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/disk/FileChannelManagerImpl.java @@ -29,7 +29,9 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.Random; import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; @@ -63,24 +65,32 @@ public FileChannelManagerImpl(String[] tempDirs, String prefix) { } private static File[] createFiles(String[] tempDirs, String prefix) { - File[] files = new File[tempDirs.length]; + List filesList = new ArrayList<>(); for (int i = 0; i < tempDirs.length; i++) { File baseDir = new File(tempDirs[i]); String subfolder = String.format("paimon-%s-%s", prefix, UUID.randomUUID()); File storageDir = new File(baseDir, subfolder); if (!storageDir.exists() && !storageDir.mkdirs()) { - throw new RuntimeException( - "Could not create storage directory for FileChannelManager: " - + storageDir.getAbsolutePath()); + LOG.warn( + "Failed to create directory {}, temp directory {} will not be used", + storageDir.getAbsolutePath(), + tempDirs[i]); + continue; } - files[i] = storageDir; + + filesList.add(storageDir); LOG.debug( "FileChannelManager uses directory {} for spill files.", storageDir.getAbsolutePath()); } - return files; + + if (filesList.isEmpty()) { + throw new RuntimeException("No available temporary directories"); + } + + return filesList.toArray(new File[0]); } @Override From 2eeed7d00102e765721464795bf786b8d0b506c5 Mon Sep 17 00:00:00 2001 From: yangjf2019 <54518670+yangjf2019@users.noreply.github.com> Date: Tue, 10 Dec 2024 18:58:03 +0800 Subject: [PATCH 136/157] [doc] updated url links in documentation (#4679) --- docs/content/flink/sql-write.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/content/flink/sql-write.md b/docs/content/flink/sql-write.md index 60c0f5c36c529..408c8e209e65f 100644 --- a/docs/content/flink/sql-write.md +++ b/docs/content/flink/sql-write.md @@ -176,8 +176,8 @@ PARTITION (k0 = 0, k1 = 0) SELECT v FROM my_table WHERE false; {{< hint info >}} Important table properties setting: 1. Only [primary key table]({{< ref "primary-key-table/overview" >}}) supports this feature. -2. [MergeEngine]({{< ref "primary-key-table/merge-engine" >}}) needs to be [deduplicate]({{< ref "primary-key-table/merge-engine#deduplicate" >}}) - or [partial-update]({{< ref "primary-key-table/merge-engine#partial-update" >}}) to support this feature. +2. [MergeEngine]({{< ref "primary-key-table/merge-engine" >}}) needs to be [deduplicate]({{< ref "primary-key-table/merge-engine/overview/#deduplicate" >}}) + or [partial-update]({{< ref "primary-key-table/merge-engine/partial-update/" >}}) to support this feature. 3. Do not support updating primary keys. {{< /hint >}} @@ -211,9 +211,9 @@ UPDATE my_table SET b = 1, c = 2 WHERE a = 'myTable'; {{< hint info >}} Important table properties setting: 1. Only primary key tables support this feature. -2. If the table has primary keys, the following [MergeEngine]({{< ref "primary-key-table/merge-engine" >}}) support this feature: - * [deduplicate]({{< ref "primary-key-table/merge-engine#deduplicate" >}}). - * [partial-update]({{< ref "primary-key-table/merge-engine#partial-update" >}}) with option 'partial-update.remove-record-on-delete' enabled. +2. If the table has primary keys, the following [MergeEngine]({{< ref "primary-key-table/merge-engine/overview/" >}}) support this feature: + * [deduplicate]({{< ref "primary-key-table/merge-engine/overview/#deduplicate" >}}). + * [partial-update]({{< ref "primary-key-table/merge-engine/partial-update/" >}}) with option 'partial-update.remove-record-on-delete' enabled. 3. Do not support deleting from table in streaming mode. {{< /hint >}} From 6010e615fdc99320b8dd6e9f52d70bfdc98174a5 Mon Sep 17 00:00:00 2001 From: yuzelin <33053040+yuzelin@users.noreply.github.com> Date: Tue, 10 Dec 2024 21:35:56 +0800 Subject: [PATCH 137/157] [hotfix] CloneAction throw more clear exception when no table in source catalog (#4682) --- .../flink/clone/CloneSourceBuilder.java | 3 ++ .../flink/action/CloneActionITCase.java | 43 +++++++++++++++++++ 2 files changed, 46 insertions(+) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneSourceBuilder.java index a0f4ef33dee28..585c73cb952cc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneSourceBuilder.java @@ -34,6 +34,7 @@ import java.util.Map; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkState; /** * Pick the tables to be cloned based on the user input parameters. The record type of the build @@ -114,6 +115,8 @@ private DataStream> build(Catalog sourceCatalog) throws E database + "." + tableName, targetDatabase + "." + targetTableName)); } + checkState(!result.isEmpty(), "Didn't find any table in source catalog."); + if (LOG.isDebugEnabled()) { LOG.debug("The clone identifiers of source table and target table are: {}", result); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java index 71672551abcba..a55b01cc203be 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java @@ -32,6 +32,7 @@ import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -44,8 +45,10 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; import static org.apache.paimon.utils.Preconditions.checkState; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** IT cases for {@link CloneAction}. */ public class CloneActionITCase extends ActionITCaseBase { @@ -640,6 +643,46 @@ public void testCloneTableWithExpiration(String invoker) throws Exception { .isEqualTo(Collections.singletonList("+I[1]")); } + // ------------------------------------------------------------------------ + // Negative Tests + // ------------------------------------------------------------------------ + + @Test + public void testEmptySourceCatalog() { + String sourceWarehouse = getTempDirPath("source-ware"); + + TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().parallelism(1).build(); + tEnv.executeSql( + "CREATE CATALOG sourcecat WITH (\n" + + " 'type' = 'paimon',\n" + + String.format(" 'warehouse' = '%s'\n", sourceWarehouse) + + ")"); + + String targetWarehouse = getTempDirPath("target-ware"); + + String[] args = + new String[] { + "clone", + "--warehouse", + sourceWarehouse, + "--target_warehouse", + targetWarehouse, + "--parallelism", + "1" + }; + CloneAction action = (CloneAction) ActionFactory.createAction(args).get(); + + StreamExecutionEnvironment env = + streamExecutionEnvironmentBuilder().streamingMode().allowRestart().build(); + action.withStreamExecutionEnvironment(env); + + assertThatThrownBy(action::run) + .satisfies( + anyCauseMatches( + IllegalStateException.class, + "Didn't find any table in source catalog.")); + } + // ------------------------------------------------------------------------ // Utils // ------------------------------------------------------------------------ From 5cf4c93791c460cb35bdd53a5e0b512b964781a5 Mon Sep 17 00:00:00 2001 From: YeJunHao <41894543+leaves12138@users.noreply.github.com> Date: Wed, 11 Dec 2024 15:14:03 +0800 Subject: [PATCH 138/157] [core] Check file size after write bundle of records (#4685) --- .../java/org/apache/paimon/io/RollingFileWriter.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/io/RollingFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/RollingFileWriter.java index 109b7574304ed..29b9223b9a372 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/RollingFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/RollingFileWriter.java @@ -64,10 +64,9 @@ public long targetFileSize() { return targetFileSize; } - @VisibleForTesting - boolean rollingFile() throws IOException { + private boolean rollingFile(boolean forceCheck) throws IOException { return currentWriter.reachTargetSize( - recordCount % CHECK_ROLLING_RECORD_CNT == 0, targetFileSize); + forceCheck || recordCount % CHECK_ROLLING_RECORD_CNT == 0, targetFileSize); } @Override @@ -81,7 +80,7 @@ public void write(T row) throws IOException { currentWriter.write(row); recordCount += 1; - if (rollingFile()) { + if (rollingFile(false)) { closeCurrentWriter(); } } catch (Throwable e) { @@ -105,7 +104,7 @@ public void writeBundle(BundleRecords bundle) throws IOException { currentWriter.writeBundle(bundle); recordCount += bundle.rowCount(); - if (rollingFile()) { + if (rollingFile(true)) { closeCurrentWriter(); } } catch (Throwable e) { From b339693d9a53d680c383cfdfd9075464f31b1261 Mon Sep 17 00:00:00 2001 From: Jingsong Date: Wed, 11 Dec 2024 16:29:03 +0800 Subject: [PATCH 139/157] [doc] Fix links in sql-write --- docs/content/flink/sql-write.md | 10 +++++----- .../content/primary-key-table/merge-engine/overview.md | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/content/flink/sql-write.md b/docs/content/flink/sql-write.md index 408c8e209e65f..6abbfa01756ce 100644 --- a/docs/content/flink/sql-write.md +++ b/docs/content/flink/sql-write.md @@ -176,8 +176,8 @@ PARTITION (k0 = 0, k1 = 0) SELECT v FROM my_table WHERE false; {{< hint info >}} Important table properties setting: 1. Only [primary key table]({{< ref "primary-key-table/overview" >}}) supports this feature. -2. [MergeEngine]({{< ref "primary-key-table/merge-engine" >}}) needs to be [deduplicate]({{< ref "primary-key-table/merge-engine/overview/#deduplicate" >}}) - or [partial-update]({{< ref "primary-key-table/merge-engine/partial-update/" >}}) to support this feature. +2. [MergeEngine]({{< ref "primary-key-table/merge-engine" >}}) needs to be [deduplicate]({{< ref "primary-key-table/merge-engine/overview#deduplicate" >}}) + or [partial-update]({{< ref "primary-key-table/merge-engine/partial-update" >}}) to support this feature. 3. Do not support updating primary keys. {{< /hint >}} @@ -211,9 +211,9 @@ UPDATE my_table SET b = 1, c = 2 WHERE a = 'myTable'; {{< hint info >}} Important table properties setting: 1. Only primary key tables support this feature. -2. If the table has primary keys, the following [MergeEngine]({{< ref "primary-key-table/merge-engine/overview/" >}}) support this feature: - * [deduplicate]({{< ref "primary-key-table/merge-engine/overview/#deduplicate" >}}). - * [partial-update]({{< ref "primary-key-table/merge-engine/partial-update/" >}}) with option 'partial-update.remove-record-on-delete' enabled. +2. If the table has primary keys, the following [MergeEngine]({{< ref "primary-key-table/merge-engine/overview" >}}) support this feature: + * [deduplicate]({{< ref "primary-key-table/merge-engine/overview#deduplicate" >}}). + * [partial-update]({{< ref "primary-key-table/merge-engine/partial-update" >}}) with option 'partial-update.remove-record-on-delete' enabled. 3. Do not support deleting from table in streaming mode. {{< /hint >}} diff --git a/docs/content/primary-key-table/merge-engine/overview.md b/docs/content/primary-key-table/merge-engine/overview.md index 9f3b50c398bdc..4d192283aa688 100644 --- a/docs/content/primary-key-table/merge-engine/overview.md +++ b/docs/content/primary-key-table/merge-engine/overview.md @@ -3,7 +3,7 @@ title: "Overview" weight: 1 type: docs aliases: -- /primary-key-table/merge-engin/overview.html +- /primary-key-table/merge-engine/overview.html --- paimon-common/src/main/antlr4/** + paimon-core/src/test/resources/compatibility/** From 72e71503394dbf1269560cf373f5ea6186d134db Mon Sep 17 00:00:00 2001 From: WenjunMin Date: Sun, 15 Dec 2024 22:33:17 +0800 Subject: [PATCH 156/157] [core] Tolerate the NoSuchObjectException when report the partition statistic (#4708) --- .../paimon/metastore/MetastoreClient.java | 3 ++- .../flink/sink/partition/HmsReporter.java | 2 +- .../partition/AddDonePartitionActionTest.java | 3 ++- .../flink/sink/partition/HmsReporterTest.java | 3 ++- .../paimon/hive/HiveMetastoreClient.java | 27 +++++++++++++------ 5 files changed, 26 insertions(+), 12 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java index 60e28c59f45dc..75f7af5abbdc1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java @@ -55,7 +55,8 @@ default void addPartitionsSpec(List> partitionSpec default void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { throw new UnsupportedOperationException(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java index eb965aa3a3188..853dc52c20bfc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java @@ -93,7 +93,7 @@ public void report(String partition, long modifyTime) throws Exception { statistic.put(HIVE_LAST_UPDATE_TIME_PROP, String.valueOf(modifyTime / 1000)); LOG.info("alter partition {} with statistic {}.", partitionSpec, statistic); - metastoreClient.alterPartition(partitionSpec, statistic, modifyTime); + metastoreClient.alterPartition(partitionSpec, statistic, modifyTime, true); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java index 19c22d137c7f2..fca5dcf0ed69d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java @@ -66,7 +66,8 @@ public void markDone(LinkedHashMap partitionSpec) public void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { throw new UnsupportedOperationException(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java index 0050f3083a8ec..f245940da57d8 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java @@ -117,7 +117,8 @@ public void markDone(LinkedHashMap partitionSpec) public void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { partitionParams.put( PartitionPathUtils.generatePartitionPath(partitionSpec), diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java index 3793c86f82693..885fa463e5a7d 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java @@ -119,17 +119,28 @@ public void addPartitionsSpec(List> partitionSpecs public void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { List partitionValues = new ArrayList<>(partitionSpec.values()); int currentTime = (int) (modifyTime / 1000); - Partition hivePartition = - clients.run( - client -> - client.getPartition( - identifier.getDatabaseName(), - identifier.getObjectName(), - partitionValues)); + Partition hivePartition; + try { + hivePartition = + clients.run( + client -> + client.getPartition( + identifier.getDatabaseName(), + identifier.getObjectName(), + partitionValues)); + } catch (NoSuchObjectException e) { + if (ignoreIfNotExist) { + return; + } else { + throw e; + } + } + hivePartition.setValues(partitionValues); hivePartition.setLastAccessTime(currentTime); hivePartition.getParameters().putAll(parameters); From 9179d654b61d933551baffe36cbcd675869f1ebc Mon Sep 17 00:00:00 2001 From: Zouxxyy Date: Sun, 15 Dec 2024 22:36:47 +0800 Subject: [PATCH 157/157] [core][spark] Enable limit pushdown and count optimization for dv table (#4709) --- .../apache/paimon/table/source/DataSplit.java | 40 +++++++++++ .../table/source/DataTableBatchScan.java | 19 +----- .../table/source/SplitGeneratorTest.java | 4 +- .../apache/paimon/table/source/SplitTest.java | 67 +++++++++++++++++++ .../paimon/spark/PaimonScanBuilder.scala | 12 ++-- .../spark/aggregate/LocalAggregator.scala | 22 +++--- .../paimon/spark/sql/PaimonPushDownTest.scala | 52 ++++++++++---- .../spark/sql/PushDownAggregatesTest.scala | 66 +++++++++++++----- 8 files changed, 216 insertions(+), 66 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java index 29405466b93f1..b9460f28b4e79 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java @@ -44,6 +44,7 @@ import static org.apache.paimon.io.DataFilePathFactory.INDEX_PATH_SUFFIX; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkState; /** Input splits. Needed by most batch computation engines. */ public class DataSplit implements Split { @@ -126,6 +127,45 @@ public long rowCount() { return rowCount; } + /** Whether it is possible to calculate the merged row count. */ + public boolean mergedRowCountAvailable() { + return rawConvertible + && (dataDeletionFiles == null + || dataDeletionFiles.stream() + .allMatch(f -> f == null || f.cardinality() != null)); + } + + public long mergedRowCount() { + checkState(mergedRowCountAvailable()); + return partialMergedRowCount(); + } + + /** + * Obtain merged row count as much as possible. There are two scenarios where accurate row count + * can be calculated: + * + *

    1. raw file and no deletion file. + * + *

    2. raw file + deletion file with cardinality. + */ + public long partialMergedRowCount() { + long sum = 0L; + if (rawConvertible) { + List rawFiles = convertToRawFiles().orElse(null); + if (rawFiles != null) { + for (int i = 0; i < rawFiles.size(); i++) { + RawFile rawFile = rawFiles.get(i); + if (dataDeletionFiles == null || dataDeletionFiles.get(i) == null) { + sum += rawFile.rowCount(); + } else if (dataDeletionFiles.get(i).cardinality() != null) { + sum += rawFile.rowCount() - dataDeletionFiles.get(i).cardinality(); + } + } + } + } + return sum; + } + @Override public Optional> convertToRawFiles() { if (rawConvertible) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java index 635802cc9dcbb..a4fe6d73bba19 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java @@ -28,7 +28,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Objects; import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; @@ -103,9 +102,9 @@ private StartingScanner.Result applyPushDownLimit(StartingScanner.Result result) List limitedSplits = new ArrayList<>(); for (DataSplit dataSplit : splits) { if (dataSplit.rawConvertible()) { - long splitRowCount = getRowCountForSplit(dataSplit); + long partialMergedRowCount = dataSplit.partialMergedRowCount(); limitedSplits.add(dataSplit); - scannedRowCount += splitRowCount; + scannedRowCount += partialMergedRowCount; if (scannedRowCount >= pushDownLimit) { SnapshotReader.Plan newPlan = new PlanImpl(plan.watermark(), plan.snapshotId(), limitedSplits); @@ -117,20 +116,6 @@ private StartingScanner.Result applyPushDownLimit(StartingScanner.Result result) return result; } - /** - * 0 represents that we can't compute the row count of this split: 1. the split needs to be - * merged; 2. the table enabled deletion vector and there are some deletion files. - */ - private long getRowCountForSplit(DataSplit split) { - if (split.deletionFiles().isPresent() - && split.deletionFiles().get().stream().anyMatch(Objects::nonNull)) { - return 0L; - } - return split.convertToRawFiles() - .map(files -> files.stream().map(RawFile::rowCount).reduce(Long::sum).orElse(0L)) - .orElse(0L); - } - @Override public DataTableScan withShard(int indexOfThisSubtask, int numberOfParallelSubtasks) { snapshotReader.withShard(indexOfThisSubtask, numberOfParallelSubtasks); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java index a9e093dab124e..a1f7d69e28779 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java @@ -43,10 +43,10 @@ public class SplitGeneratorTest { public static DataFileMeta newFileFromSequence( - String name, int rowCount, long minSequence, long maxSequence) { + String name, int fileSize, long minSequence, long maxSequence) { return new DataFileMeta( name, - rowCount, + fileSize, 1, EMPTY_ROW, EMPTY_ROW, diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java index 359d38c973db1..0219941a0ac0b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java @@ -49,6 +49,41 @@ /** Test for {@link DataSplit}. */ public class SplitTest { + @Test + public void testSplitMergedRowCount() { + // not rawConvertible + List dataFiles = + Arrays.asList(newDataFile(1000L), newDataFile(2000L), newDataFile(3000L)); + DataSplit split = newDataSplit(false, dataFiles, null); + assertThat(split.partialMergedRowCount()).isEqualTo(0L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(false); + + // rawConvertible without deletion files + split = newDataSplit(true, dataFiles, null); + assertThat(split.partialMergedRowCount()).isEqualTo(6000L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(true); + assertThat(split.mergedRowCount()).isEqualTo(6000L); + + // rawConvertible with deletion files without cardinality + ArrayList deletionFiles = new ArrayList<>(); + deletionFiles.add(null); + deletionFiles.add(new DeletionFile("p", 1, 2, null)); + deletionFiles.add(new DeletionFile("p", 1, 2, 100L)); + split = newDataSplit(true, dataFiles, deletionFiles); + assertThat(split.partialMergedRowCount()).isEqualTo(3900L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(false); + + // rawConvertible with deletion files with cardinality + deletionFiles = new ArrayList<>(); + deletionFiles.add(null); + deletionFiles.add(new DeletionFile("p", 1, 2, 200L)); + deletionFiles.add(new DeletionFile("p", 1, 2, 100L)); + split = newDataSplit(true, dataFiles, deletionFiles); + assertThat(split.partialMergedRowCount()).isEqualTo(5700L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(true); + assertThat(split.mergedRowCount()).isEqualTo(5700L); + } + @Test public void testSerializer() throws IOException { DataFileTestDataGenerator gen = DataFileTestDataGenerator.builder().build(); @@ -311,4 +346,36 @@ public void testSerializerCompatibleV3() throws Exception { InstantiationUtil.deserializeObject(v2Bytes, DataSplit.class.getClassLoader()); assertThat(actual).isEqualTo(split); } + + private DataFileMeta newDataFile(long rowCount) { + return DataFileMeta.forAppend( + "my_data_file.parquet", + 1024 * 1024, + rowCount, + null, + 0L, + rowCount, + 1, + Collections.emptyList(), + null, + null, + null); + } + + private DataSplit newDataSplit( + boolean rawConvertible, + List dataFiles, + List deletionFiles) { + DataSplit.Builder builder = DataSplit.builder(); + builder.withSnapshot(1) + .withPartition(BinaryRow.EMPTY_ROW) + .withBucket(1) + .withBucketPath("my path") + .rawConvertible(rawConvertible) + .withDataFiles(dataFiles); + if (deletionFiles != null) { + builder.withDataDeletionFiles(deletionFiles); + } + return builder.build(); + } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala index d8b66e1cd1e03..0393a1cd15786 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala @@ -21,6 +21,7 @@ package org.apache.paimon.spark import org.apache.paimon.predicate.PredicateBuilder import org.apache.paimon.spark.aggregate.LocalAggregator import org.apache.paimon.table.Table +import org.apache.paimon.table.source.DataSplit import org.apache.spark.sql.connector.expressions.aggregate.Aggregation import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownAggregates, SupportsPushDownLimit} @@ -36,12 +37,12 @@ class PaimonScanBuilder(table: Table) override def pushLimit(limit: Int): Boolean = { // It is safe, since we will do nothing if it is the primary table and the split is not `rawConvertible` pushDownLimit = Some(limit) - // just make a best effort to push down limit + // just make the best effort to push down limit false } override def supportCompletePushDown(aggregation: Aggregation): Boolean = { - // for now we only support complete push down, so there is no difference with `pushAggregation` + // for now, we only support complete push down, so there is no difference with `pushAggregation` pushAggregation(aggregation) } @@ -66,8 +67,11 @@ class PaimonScanBuilder(table: Table) val pushedPartitionPredicate = PredicateBuilder.and(pushedPredicates.map(_._2): _*) readBuilder.withFilter(pushedPartitionPredicate) } - val scan = readBuilder.newScan() - scan.listPartitionEntries.asScala.foreach(aggregator.update) + val dataSplits = readBuilder.newScan().plan().splits().asScala.map(_.asInstanceOf[DataSplit]) + if (!dataSplits.forall(_.mergedRowCountAvailable())) { + return false + } + dataSplits.foreach(aggregator.update) localScan = Some( PaimonLocalScan( aggregator.result(), diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala index 41e7fd3c3ce90..8988e7218d1f8 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala @@ -19,10 +19,10 @@ package org.apache.paimon.spark.aggregate import org.apache.paimon.data.BinaryRow -import org.apache.paimon.manifest.PartitionEntry import org.apache.paimon.spark.SparkTypeUtils import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.table.{DataTable, Table} +import org.apache.paimon.table.source.DataSplit import org.apache.paimon.utils.{InternalRowUtils, ProjectedRow} import org.apache.spark.sql.catalyst.InternalRow @@ -78,13 +78,7 @@ class LocalAggregator(table: Table) { } def pushAggregation(aggregation: Aggregation): Boolean = { - if ( - !table.isInstanceOf[DataTable] || - !table.primaryKeys.isEmpty - ) { - return false - } - if (table.asInstanceOf[DataTable].coreOptions.deletionVectorsEnabled) { + if (!table.isInstanceOf[DataTable]) { return false } @@ -108,12 +102,12 @@ class LocalAggregator(table: Table) { SparkInternalRow.create(partitionType).replace(genericRow) } - def update(partitionEntry: PartitionEntry): Unit = { + def update(dataSplit: DataSplit): Unit = { assert(isInitialized) - val groupByRow = requiredGroupByRow(partitionEntry.partition()) + val groupByRow = requiredGroupByRow(dataSplit.partition()) val aggFuncEvaluator = groupByEvaluatorMap.getOrElseUpdate(groupByRow, aggFuncEvaluatorGetter()) - aggFuncEvaluator.foreach(_.update(partitionEntry)) + aggFuncEvaluator.foreach(_.update(dataSplit)) } def result(): Array[InternalRow] = { @@ -147,7 +141,7 @@ class LocalAggregator(table: Table) { } trait AggFuncEvaluator[T] { - def update(partitionEntry: PartitionEntry): Unit + def update(dataSplit: DataSplit): Unit def result(): T def resultType: DataType def prettyName: String @@ -156,8 +150,8 @@ trait AggFuncEvaluator[T] { class CountStarEvaluator extends AggFuncEvaluator[Long] { private var _result: Long = 0L - override def update(partitionEntry: PartitionEntry): Unit = { - _result += partitionEntry.recordCount() + override def update(dataSplit: DataSplit): Unit = { + _result += dataSplit.mergedRowCount() } override def result(): Long = _result diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala index ba314e3afa816..503f1c8e3e9d6 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala @@ -18,7 +18,7 @@ package org.apache.paimon.spark.sql -import org.apache.paimon.spark.{PaimonBatch, PaimonInputPartition, PaimonScan, PaimonSparkTestBase, SparkTable} +import org.apache.paimon.spark.{PaimonScan, PaimonSparkTestBase, SparkTable} import org.apache.paimon.table.source.DataSplit import org.apache.spark.sql.Row @@ -29,8 +29,6 @@ import org.apache.spark.sql.connector.read.{ScanBuilder, SupportsPushDownLimit} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.junit.jupiter.api.Assertions -import scala.collection.JavaConverters._ - class PaimonPushDownTest extends PaimonSparkTestBase { import testImplicits._ @@ -64,7 +62,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { checkAnswer(spark.sql(q), Row(1, "a", "p1") :: Nil) // case 2 - // filter "id = '1' or pt = 'p1'" can't push down completely, it still need to be evaluated after scanning + // filter "id = '1' or pt = 'p1'" can't push down completely, it still needs to be evaluated after scanning q = "SELECT * FROM T WHERE id = '1' or pt = 'p1'" Assertions.assertTrue(checkEqualToFilterExists(q, "pt", Literal("p1"))) checkAnswer(spark.sql(q), Row(1, "a", "p1") :: Row(2, "b", "p1") :: Nil) @@ -121,7 +119,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { val dataSplitsWithoutLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits Assertions.assertTrue(dataSplitsWithoutLimit.length >= 2) - // It still return false even it can push down limit. + // It still returns false even it can push down limit. Assertions.assertFalse(scanBuilder.asInstanceOf[SupportsPushDownLimit].pushLimit(1)) val dataSplitsWithLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits Assertions.assertEquals(1, dataSplitsWithLimit.length) @@ -169,12 +167,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { // Now, we have 4 dataSplits, and 2 dataSplit is nonRawConvertible, 2 dataSplit is rawConvertible. Assertions.assertEquals( 2, - dataSplitsWithoutLimit2 - .filter( - split => { - split.asInstanceOf[DataSplit].rawConvertible() - }) - .length) + dataSplitsWithoutLimit2.count(split => { split.asInstanceOf[DataSplit].rawConvertible() })) // Return 2 dataSplits. Assertions.assertFalse(scanBuilder2.asInstanceOf[SupportsPushDownLimit].pushLimit(2)) @@ -206,7 +199,40 @@ class PaimonPushDownTest extends PaimonSparkTestBase { // Need to scan all dataSplits. Assertions.assertEquals(4, dataSplitsWithLimit3.length) Assertions.assertEquals(1, spark.sql("SELECT * FROM T LIMIT 1").count()) + } + test("Paimon pushDown: limit for table with deletion vector") { + Seq(true, false).foreach( + deletionVectorsEnabled => { + Seq(true, false).foreach( + primaryKeyTable => { + withTable("T") { + sql(s""" + |CREATE TABLE T (id INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = $deletionVectorsEnabled, + | '${if (primaryKeyTable) "primary-key" else "bucket-key"}' = 'id', + | 'bucket' = '10' + |) + |""".stripMargin) + + sql("INSERT INTO T SELECT id FROM range (1, 50000)") + sql("DELETE FROM T WHERE id % 13 = 0") + + val withoutLimit = getScanBuilder().build().asInstanceOf[PaimonScan].getOriginSplits + assert(withoutLimit.length == 10) + + val scanBuilder = getScanBuilder().asInstanceOf[SupportsPushDownLimit] + scanBuilder.pushLimit(1) + val withLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits + if (deletionVectorsEnabled || !primaryKeyTable) { + assert(withLimit.length == 1) + } else { + assert(withLimit.length == 10) + } + } + }) + }) } test("Paimon pushDown: runtime filter") { @@ -250,8 +276,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { } private def getScanBuilder(tableName: String = "T"): ScanBuilder = { - new SparkTable(loadTable(tableName)) - .newScanBuilder(CaseInsensitiveStringMap.empty()) + SparkTable(loadTable(tableName)).newScanBuilder(CaseInsensitiveStringMap.empty()) } private def checkFilterExists(sql: String): Boolean = { @@ -272,5 +297,4 @@ class PaimonPushDownTest extends PaimonSparkTestBase { case _ => false } } - } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala index 501e7bfb4a515..78c02644a7ced 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala @@ -117,22 +117,58 @@ class PushDownAggregatesTest extends PaimonSparkTestBase with AdaptiveSparkPlanH } } - test("Push down aggregate - primary table") { - withTable("T") { - spark.sql("CREATE TABLE T (c1 INT, c2 STRING) TBLPROPERTIES ('primary-key' = 'c1')") - runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 2) - spark.sql("INSERT INTO T VALUES(1, 'x'), (2, 'x'), (3, 'x'), (3, 'x')") - runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 2) - } + test("Push down aggregate - primary key table with deletion vector") { + Seq(true, false).foreach( + deletionVectorsEnabled => { + withTable("T") { + spark.sql(s""" + |CREATE TABLE T (c1 INT, c2 STRING) + |TBLPROPERTIES ( + |'primary-key' = 'c1', + |'deletion-vectors.enabled' = $deletionVectorsEnabled + |) + |""".stripMargin) + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 0) + + spark.sql("INSERT INTO T VALUES(1, 'x'), (2, 'x'), (3, 'x'), (3, 'x')") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 0) + + spark.sql("INSERT INTO T VALUES(1, 'x_1')") + if (deletionVectorsEnabled) { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 0) + } else { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 2) + } + } + }) } - test("Push down aggregate - enable deletion vector") { - withTable("T") { - spark.sql( - "CREATE TABLE T (c1 INT, c2 STRING) TBLPROPERTIES('deletion-vectors.enabled' = 'true')") - runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 2) - spark.sql("INSERT INTO T VALUES(1, 'x'), (2, 'x'), (3, 'x'), (3, 'x')") - runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(4) :: Nil, 2) - } + test("Push down aggregate - table with deletion vector") { + Seq(true, false).foreach( + deletionVectorsEnabled => { + Seq(true, false).foreach( + primaryKeyTable => { + withTable("T") { + sql(s""" + |CREATE TABLE T (id INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = $deletionVectorsEnabled, + | '${if (primaryKeyTable) "primary-key" else "bucket-key"}' = 'id', + | 'bucket' = '1' + |) + |""".stripMargin) + + sql("INSERT INTO T SELECT id FROM range (0, 5000)") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Seq(Row(5000)), 0) + + sql("DELETE FROM T WHERE id > 100 and id <= 400") + if (deletionVectorsEnabled || !primaryKeyTable) { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Seq(Row(4700)), 0) + } else { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Seq(Row(4700)), 2) + } + } + }) + }) } }