From 16c230fcd1d98563391f8ee0458f87213ddb0176 Mon Sep 17 00:00:00 2001 From: vamossagar12 Date: Sun, 14 Dec 2025 17:40:41 +0530 Subject: [PATCH 01/14] Adding changes for flink 2.2 --- fluss-flink/fluss-flink-2.2/pom.xml | 268 +++++++++++++++ .../fluss/flink/catalog/Flink22Catalog.java | 106 ++++++ .../flink/catalog/Flink22CatalogFactory.java | 34 ++ .../apache/fluss/flink/sink/FlinkSink.java | 213 ++++++++++++ .../org.apache.flink.table.factories.Factory | 19 ++ .../flink/catalog/Flink22CatalogITCase.java | 193 +++++++++++ .../flink/metrics/Flink22MetricsITCase.java | 21 ++ .../procedure/Flink22ProcedureITCase.java | 21 ++ .../acl/Flink22AuthorizationITCase.java | 21 ++ .../flink/sink/Flink22ComplexTypeITCase.java | 21 ++ .../flink/sink/Flink22TableSinkITCase.java | 21 ++ .../source/Flink22TableSourceBatchITCase.java | 21 ++ .../Flink22TableSourceFailOverITCase.java | 21 ++ .../source/Flink22TableSourceITCase.java | 309 ++++++++++++++++++ .../apache/fluss/flink/sink/FlinkSink.java | 9 +- fluss-flink/pom.xml | 1 + 16 files changed, 1294 insertions(+), 5 deletions(-) create mode 100644 fluss-flink/fluss-flink-2.2/pom.xml create mode 100644 fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java create mode 100644 fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java create mode 100644 fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java create mode 100644 fluss-flink/fluss-flink-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22ComplexTypeITCase.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java diff --git a/fluss-flink/fluss-flink-2.2/pom.xml b/fluss-flink/fluss-flink-2.2/pom.xml new file mode 100644 index 0000000000..c6527e2822 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/pom.xml @@ -0,0 +1,268 @@ + + + + 4.0.0 + + org.apache.fluss + fluss-flink + 0.9-SNAPSHOT + + + fluss-flink-2.2 + Fluss : Engine Flink : 2.2 + + 2.2 + 2.2.0 + + + + + + org.apache.fluss + fluss-client + ${project.version} + + + + org.apache.fluss + fluss-flink-common + ${project.version} + + + * + * + + + + + + + org.apache.flink + flink-core + ${flink.minor.version} + provided + + + + org.apache.flink + flink-table-common + ${flink.minor.version} + provided + + + + org.apache.flink + flink-streaming-java + ${flink.minor.version} + provided + + + + org.apache.flink + flink-runtime + ${flink.minor.version} + provided + + + + + org.apache.fluss + fluss-flink-common + ${project.version} + test + test-jar + + + + org.apache.fluss + fluss-server + ${project.version} + test + + + + org.apache.flink + flink-table-test-utils + ${flink.minor.version} + test + + + + org.apache.flink + flink-connector-base + ${flink.minor.version} + test + + + + org.apache.fluss + fluss-server + ${project.version} + test + test-jar + + + + org.apache.fluss + fluss-rpc + test + test-jar + ${project.version} + + + + + org.apache.curator + curator-test + ${curator.version} + test + + + + org.apache.flink + flink-table-common + ${flink.minor.version} + test + test-jar + + + + org.apache.flink + flink-connector-test-utils + ${flink.minor.version} + test + + + + + org.apache.fluss + fluss-test-utils + + + + org.apache.fluss + fluss-common + ${project.version} + test + test-jar + + + + org.apache.flink + flink-clients + ${flink.minor.version} + test + + + + org.apache.flink + flink-sql-gateway + ${flink.minor.version} + test + + + + org.apache.flink + flink-sql-gateway + ${flink.minor.version} + test-jar + test + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + ${skip.on.java8} + + ${skip.on.java8} + + + + + org.apache.maven.plugins + maven-surefire-plugin + 3.0.0-M5 + + + + integration-tests + integration-test + false + + test + + + ${skip.on.java8} + + **/*ITCase.* + + + 1 + + + + + default-test + test + false + + test + + + ${skip.on.java8} + + **/*ITCase.* + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-fluss + package + + shade + + + + + org.apache.fluss:fluss-flink-common + org.apache.fluss:fluss-client + + + + + + + + + + diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java new file mode 100644 index 0000000000..3bdc5746a1 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java @@ -0,0 +1,106 @@ +/* + * 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.fluss.flink.catalog; + +import org.apache.fluss.metadata.TableInfo; + +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** A {@link FlinkCatalog} used for Flink 2.2. */ +public class Flink22Catalog extends FlinkCatalog { + + public Flink22Catalog( + String name, + String defaultDatabase, + String bootstrapServers, + ClassLoader classLoader, + Map securityConfigs) { + super(name, defaultDatabase, bootstrapServers, classLoader, securityConfigs); + } + + @Override + public CatalogBaseTable getTable(ObjectPath objectPath) + throws TableNotExistException, CatalogException { + CatalogBaseTable catalogBaseTable = super.getTable(objectPath); + if (!(catalogBaseTable instanceof CatalogTable)) { + return catalogBaseTable; + } + + CatalogTable table = (CatalogTable) catalogBaseTable; + Optional pkOp = table.getUnresolvedSchema().getPrimaryKey(); + // If there is no pk, return directly. + if (pkOp.isEmpty()) { + return table; + } + + Schema.Builder newSchemaBuilder = + Schema.newBuilder().fromSchema(table.getUnresolvedSchema()); + // Pk is always an index. + newSchemaBuilder.index(pkOp.get().getColumnNames()); + + // Judge whether we can do prefix lookup. + TableInfo tableInfo = connection.getTable(toTablePath(objectPath)).getTableInfo(); + List bucketKeys = tableInfo.getBucketKeys(); + // For partition table, the physical primary key is the primary key that excludes the + // partition key + List physicalPrimaryKeys = tableInfo.getPhysicalPrimaryKeys(); + List indexKeys = new ArrayList<>(); + if (isPrefixList(physicalPrimaryKeys, bucketKeys)) { + indexKeys.addAll(bucketKeys); + if (tableInfo.isPartitioned()) { + indexKeys.addAll(tableInfo.getPartitionKeys()); + } + } + + if (!indexKeys.isEmpty()) { + newSchemaBuilder.index(indexKeys); + } + return CatalogTable.newBuilder() + .schema(newSchemaBuilder.build()) + .comment(table.getComment()) + .partitionKeys(table.getPartitionKeys()) + .options(table.getOptions()) + .snapshot(table.getSnapshot().orElse(null)) + .distribution(table.getDistribution().orElse(null)) + .build(); + } + + private static boolean isPrefixList(List fullList, List prefixList) { + if (fullList.size() <= prefixList.size()) { + return false; + } + + for (int i = 0; i < prefixList.size(); i++) { + if (!fullList.get(i).equals(prefixList.get(i))) { + return false; + } + } + return true; + } +} diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java new file mode 100644 index 0000000000..e198a9aee4 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.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.fluss.flink.catalog; + +/** A {@link FlinkCatalogFactory} used for Flink 2.2. */ +public class Flink22CatalogFactory extends FlinkCatalogFactory { + + @Override + public FlinkCatalog createCatalog(Context context) { + FlinkCatalog catalog = super.createCatalog(context); + return new Flink22Catalog( + catalog.catalogName, + catalog.defaultDatabase, + catalog.bootstrapServers, + catalog.classLoader, + catalog.securityConfigs); + } +} diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java new file mode 100644 index 0000000000..39ef016f04 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.sink; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.sink.serializer.FlussSerializationSchema; +import org.apache.fluss.flink.sink.writer.AppendSinkWriter; +import org.apache.fluss.flink.sink.writer.FlinkSinkWriter; +import org.apache.fluss.flink.sink.writer.UpsertSinkWriter; +import org.apache.fluss.metadata.DataLakeFormat; +import org.apache.fluss.metadata.TablePath; + +import org.apache.flink.api.common.operators.MailboxExecutor; +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 org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.types.logical.RowType; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; + +import static org.apache.fluss.flink.sink.FlinkStreamPartitioner.partition; +import static org.apache.fluss.flink.utils.FlinkConversions.toFlussRowType; + +/** Flink sink for Fluss. */ +class FlinkSink implements Sink, SupportsPreWriteTopology { + + private static final long serialVersionUID = 1L; + + private final SinkWriterBuilder builder; + + FlinkSink(SinkWriterBuilder builder) { + this.builder = builder; + } + + @Override + public SinkWriter createWriter(WriterInitContext context) throws IOException { + FlinkSinkWriter flinkSinkWriter = + builder.createWriter(context.getMailboxExecutor()); + flinkSinkWriter.initialize(InternalSinkWriterMetricGroup.wrap(context.metricGroup())); + return flinkSinkWriter; + } + + @Override + public DataStream addPreWriteTopology(DataStream input) { + return builder.addPreWriteTopology(input); + } + + @Internal + interface SinkWriterBuilder, InputT> extends Serializable { + W createWriter(MailboxExecutor mailboxExecutor); + + DataStream addPreWriteTopology(DataStream input); + } + + @Internal + static class AppendSinkWriterBuilder + implements SinkWriterBuilder, InputT> { + + private static final long serialVersionUID = 1L; + + private final TablePath tablePath; + private final Configuration flussConfig; + private final RowType tableRowType; + private final int numBucket; + private final List bucketKeys; + private final List partitionKeys; + private final @Nullable DataLakeFormat lakeFormat; + private final boolean shuffleByBucketId; + private final FlussSerializationSchema flussSerializationSchema; + + public AppendSinkWriterBuilder( + TablePath tablePath, + Configuration flussConfig, + RowType tableRowType, + int numBucket, + List bucketKeys, + List partitionKeys, + @Nullable DataLakeFormat lakeFormat, + boolean shuffleByBucketId, + FlussSerializationSchema flussSerializationSchema) { + this.tablePath = tablePath; + this.flussConfig = flussConfig; + this.tableRowType = tableRowType; + this.numBucket = numBucket; + this.bucketKeys = bucketKeys; + this.partitionKeys = partitionKeys; + this.lakeFormat = lakeFormat; + this.shuffleByBucketId = shuffleByBucketId; + this.flussSerializationSchema = flussSerializationSchema; + } + + @Override + public AppendSinkWriter createWriter(MailboxExecutor mailboxExecutor) { + return new AppendSinkWriter<>( + tablePath, + flussConfig, + tableRowType, + mailboxExecutor, + flussSerializationSchema); + } + + @Override + public DataStream addPreWriteTopology(DataStream input) { + // For append only sink, we will do bucket shuffle only if bucket keys are not + // empty. + if (!bucketKeys.isEmpty() && shuffleByBucketId) { + return partition( + input, + new FlinkRowDataChannelComputer<>( + toFlussRowType(tableRowType), + bucketKeys, + partitionKeys, + lakeFormat, + numBucket, + flussSerializationSchema), + input.getParallelism()); + } else { + return input; + } + } + } + + @Internal + static class UpsertSinkWriterBuilder + implements SinkWriterBuilder, InputT> { + + private static final long serialVersionUID = 1L; + + private final TablePath tablePath; + private final Configuration flussConfig; + private final RowType tableRowType; + private final @Nullable int[] targetColumnIndexes; + private final int numBucket; + private final List bucketKeys; + private final List partitionKeys; + private final @Nullable DataLakeFormat lakeFormat; + private final boolean shuffleByBucketId; + private final FlussSerializationSchema flussSerializationSchema; + + UpsertSinkWriterBuilder( + TablePath tablePath, + Configuration flussConfig, + RowType tableRowType, + @Nullable int[] targetColumnIndexes, + int numBucket, + List bucketKeys, + List partitionKeys, + @Nullable DataLakeFormat lakeFormat, + boolean shuffleByBucketId, + FlussSerializationSchema flussSerializationSchema) { + this.tablePath = tablePath; + this.flussConfig = flussConfig; + this.tableRowType = tableRowType; + this.targetColumnIndexes = targetColumnIndexes; + this.numBucket = numBucket; + this.bucketKeys = bucketKeys; + this.partitionKeys = partitionKeys; + this.lakeFormat = lakeFormat; + this.shuffleByBucketId = shuffleByBucketId; + this.flussSerializationSchema = flussSerializationSchema; + } + + @Override + public UpsertSinkWriter createWriter(MailboxExecutor mailboxExecutor) { + return new UpsertSinkWriter<>( + tablePath, + flussConfig, + tableRowType, + targetColumnIndexes, + mailboxExecutor, + flussSerializationSchema); + } + + @Override + public DataStream addPreWriteTopology(DataStream input) { + return shuffleByBucketId + ? partition( + input, + new FlinkRowDataChannelComputer<>( + toFlussRowType(tableRowType), + bucketKeys, + partitionKeys, + lakeFormat, + numBucket, + flussSerializationSchema), + input.getParallelism()) + : input; + } + } +} diff --git a/fluss-flink/fluss-flink-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/fluss-flink/fluss-flink-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000000..369e7d0877 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,19 @@ +# +# 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.fluss.flink.catalog.Flink22CatalogFactory diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java new file mode 100644 index 0000000000..05f2089717 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java @@ -0,0 +1,193 @@ +/* + * 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.fluss.flink.catalog; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT case for catalog in Flink 2.1. */ +public class Flink22CatalogITCase extends FlinkCatalogITCase { + + @BeforeAll + static void beforeAll() { + FlinkCatalogITCase.beforeAll(); + + // close the old one and open a new one later + catalog.close(); + + catalog = + new Flink22Catalog( + catalog.catalogName, + catalog.defaultDatabase, + catalog.bootstrapServers, + catalog.classLoader, + catalog.securityConfigs); + catalog.open(); + } + + @Test + void testGetTableWithIndex() throws Exception { + String tableName = "table_with_pk_only"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " primary key (a, b) NOT ENFORCED" + + ") with ( " + + " 'connector' = 'fluss' " + + ")", + tableName)); + CatalogTable table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + Schema expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .primaryKey("a", "b") + .index("a", "b") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_prefix_bucket_key"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " primary key (a, b) NOT ENFORCED" + + ") with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'a'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .primaryKey("a", "b") + .index("a", "b") + .index("a") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_bucket_key_is_not_prefix_pk"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " primary key (a, b) NOT ENFORCED" + + ") with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'b'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .primaryKey("a", "b") + .index("a", "b") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_partition_1"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " dt varchar, " + + " primary key (a, b, dt) NOT ENFORCED " + + ") " + + " partitioned by (dt) " + + " with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'a'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .column("dt", DataTypes.STRING().notNull()) + .primaryKey("a", "b", "dt") + .index("a", "b", "dt") + .index("a", "dt") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_partition_2"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " dt varchar, " + + " primary key (dt, a, b) NOT ENFORCED " + + ") " + + " partitioned by (dt) " + + " with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'a'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .column("dt", DataTypes.STRING().notNull()) + .primaryKey("dt", "a", "b") + .index("dt", "a", "b") + .index("a", "dt") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + } + + @Override + protected void addDefaultIndexKey(Schema.Builder schemaBuilder) { + super.addDefaultIndexKey(schemaBuilder); + + Schema currentSchema = schemaBuilder.build(); + currentSchema.getPrimaryKey().ifPresent(pk -> schemaBuilder.index(pk.getColumnNames())); + } +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java new file mode 100644 index 0000000000..de728a24d4 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.metrics; + +/** IT case for metrics in Flink 2.1. */ +public class Flink22MetricsITCase extends FlinkMetricsITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java new file mode 100644 index 0000000000..c37d48000e --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.procedure; + +/** IT case for procedure in Flink 2.1. */ +public class Flink22ProcedureITCase extends FlinkProcedureITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java new file mode 100644 index 0000000000..8e0979f35d --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.security.acl; + +/** IT case for authorization in Flink 2.1. */ +public class Flink22AuthorizationITCase extends FlinkAuthorizationITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22ComplexTypeITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22ComplexTypeITCase.java new file mode 100644 index 0000000000..9afc956caa --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22ComplexTypeITCase.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.sink; + +/** Integration tests for Array type support in Flink 2.1. */ +public class Flink22ComplexTypeITCase extends FlinkComplexTypeITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java new file mode 100644 index 0000000000..22c9db1618 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.sink; + +/** IT case for {@link FlinkTableSink} in Flink 2.1. */ +public class Flink22TableSinkITCase extends FlinkTableSinkITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java new file mode 100644 index 0000000000..a7cffa1abb --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.source; + +/** IT case for batch source in Flink 2.1. */ +public class Flink22TableSourceBatchITCase extends FlinkTableSourceBatchITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java new file mode 100644 index 0000000000..96c2527029 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.source; + +/** IT case for source failover and recovery in Flink 2.1. */ +public class Flink22TableSourceFailOverITCase extends FlinkTableSourceFailOverITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java new file mode 100644 index 0000000000..bf643d1de6 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java @@ -0,0 +1,309 @@ +/* + * 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.fluss.flink.source; + +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.InternalRow; + +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.fluss.flink.source.testutils.FlinkRowAssertionsUtils.assertResultsIgnoreOrder; +import static org.apache.fluss.flink.utils.FlinkTestBase.writeRows; +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.assertj.core.api.Assertions.assertThat; + +/** IT case for {@link FlinkTableSource} in Flink 2.1. */ +public class Flink22TableSourceITCase extends FlinkTableSourceITCase { + + @Test + void testDeltaJoin() throws Exception { + // start two jobs for this test: one for DML involving the delta join, and the other for DQL + // to query the results of the sink table + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + String leftTableName = "left_table"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " b1 varchar, " + + " c1 bigint, " + + " d1 int, " + + " e1 bigint, " + + " primary key (c1, d1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c1', " + // currently, delta join only support append-only source + + " 'table.merge-engine' = 'first_row' " + + ")", + leftTableName)); + List rows1 = + Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v2", 200L, 2, 20000L), + row(3, "v1", 300L, 3, 30000L), + row(4, "v4", 400L, 4, 40000L)); + // write records + TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); + writeRows(conn, leftTablePath, rows1, false); + + String rightTableName = "right_table"; + tEnv.executeSql( + String.format( + "create table %s (" + + " a2 int, " + + " b2 varchar, " + + " c2 bigint, " + + " d2 int, " + + " e2 bigint, " + + " primary key (c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c2', " + // currently, delta join only support append-only source + + " 'table.merge-engine' = 'first_row' " + + ")", + rightTableName)); + List rows2 = + Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v3", 200L, 2, 20000L), + row(3, "v4", 300L, 4, 30000L), + row(4, "v4", 500L, 4, 50000L)); + // write records + TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); + writeRows(conn, rightTablePath, rows2, false); + + String sinkTableName = "sink_table"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " b1 varchar, " + + " c1 bigint, " + + " d1 int, " + + " e1 bigint, " + + " a2 int, " + + " b2 varchar, " + + " c2 bigint, " + + " d2 int, " + + " e2 bigint, " + + " primary key (c1, d1, c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss' " + + ")", + sinkTableName)); + + tEnv.getConfig() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE); + + String sql = + String.format( + "INSERT INTO %s SELECT * FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2", + sinkTableName, leftTableName, rightTableName); + + assertThat(tEnv.explainSql(sql)) + .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); + + tEnv.executeSql(sql); + + CloseableIterator collected = + tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); + List expected = + Arrays.asList( + "+I[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", + "-U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", + "+U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", + "+I[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", + "-U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", + "+U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]"); + assertResultsIgnoreOrder(collected, expected, true); + } + + @Test + void testDeltaJoinWithProjectionAndFilter() throws Exception { + // start two jobs for this test: one for DML involving the delta join, and the other for DQL + // to query the results of the sink table + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + String leftTableName = "left_table_proj"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " b1 varchar, " + + " c1 bigint, " + + " d1 int, " + + " e1 bigint, " + + " primary key (c1, d1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c1', " + + " 'table.merge-engine' = 'first_row' " + + ")", + leftTableName)); + List rows1 = + Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v2", 200L, 2, 20000L), + row(3, "v1", 300L, 3, 30000L)); + TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); + writeRows(conn, leftTablePath, rows1, false); + + String rightTableName = "right_table_proj"; + tEnv.executeSql( + String.format( + "create table %s (" + + " a2 int, " + + " b2 varchar, " + + " c2 bigint, " + + " d2 int, " + + " e2 bigint, " + + " primary key (c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c2', " + + " 'table.merge-engine' = 'first_row' " + + ")", + rightTableName)); + List rows2 = + Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v3", 200L, 2, 20000L), + row(3, "v4", 300L, 4, 30000L)); + TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); + writeRows(conn, rightTablePath, rows2, false); + + String sinkTableName = "sink_table_proj"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " c1 bigint, " + + " a2 int, " + + " primary key (c1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss' " + + ")", + sinkTableName)); + + tEnv.getConfig() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE); + + // Test with projection and filter + String sql = + String.format( + "INSERT INTO %s SELECT a1, c1, a2 FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2 WHERE a1 > 1", + sinkTableName, leftTableName, rightTableName); + + assertThat(tEnv.explainSql(sql)) + .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); + + tEnv.executeSql(sql); + + CloseableIterator collected = + tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); + List expected = Arrays.asList("+I[2, 200, 2]", "-U[2, 200, 2]", "+U[2, 200, 2]"); + assertResultsIgnoreOrder(collected, expected, true); + } + + @Test + void testDeltaJoinWithLookupCache() throws Exception { + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + String leftTableName = "left_table_cache"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " c1 bigint, " + + " d1 int, " + + " primary key (c1, d1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c1', " + + " 'table.merge-engine' = 'first_row' " + + ")", + leftTableName)); + List rows1 = Arrays.asList(row(1, 100L, 1)); + writeRows(conn, TablePath.of(DEFAULT_DB, leftTableName), rows1, false); + + String rightTableName = "right_table_cache"; + tEnv.executeSql( + String.format( + "create table %s (" + + " a2 int, " + + " c2 bigint, " + + " d2 int, " + + " primary key (c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c2', " + + " 'table.merge-engine' = 'first_row', " + + " 'lookup.cache' = 'partial', " + + " 'lookup.partial-cache.max-rows' = '100' " + + ")", + rightTableName)); + List rows2 = Arrays.asList(row(1, 100L, 1)); + writeRows(conn, TablePath.of(DEFAULT_DB, rightTableName), rows2, false); + + String sinkTableName = "sink_table_cache"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " a2 int, " + + " primary key (a1) NOT ENFORCED" // Dummy PK + + ") with (" + + " 'connector' = 'fluss' " + + ")", + sinkTableName)); + + tEnv.getConfig() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE); + + String sql = + String.format( + "INSERT INTO %s SELECT T1.a1, T2.a2 FROM %s AS T1 INNER JOIN %s AS T2 ON T1.c1 = T2.c2 AND T1.d1 = T2.d2", + sinkTableName, leftTableName, rightTableName); + + assertThat(tEnv.explainSql(sql)) + .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); + + tEnv.executeSql(sql); + + CloseableIterator collected = + tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); + List expected = Arrays.asList("+I[1, 1]", "-U[1, 1]", "+U[1, 1]"); + assertResultsIgnoreOrder(collected, expected, true); + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java index a35d481176..efea5013c7 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java @@ -58,16 +58,14 @@ class FlinkSink implements Sink, SupportsPreWriteTopology createWriter(InitContext context) throws IOException { - FlinkSinkWriter flinkSinkWriter = - builder.createWriter(context.getMailboxExecutor()); + FlinkSinkWriter flinkSinkWriter = builder.createWriter(context.getMailboxExecutor()); flinkSinkWriter.initialize(InternalSinkWriterMetricGroup.wrap(context.metricGroup())); return flinkSinkWriter; } @Override public SinkWriter createWriter(WriterInitContext context) throws IOException { - FlinkSinkWriter flinkSinkWriter = - builder.createWriter(context.getMailboxExecutor()); + FlinkSinkWriter flinkSinkWriter = builder.createWriter(context.getMailboxExecutor()); flinkSinkWriter.initialize(InternalSinkWriterMetricGroup.wrap(context.metricGroup())); return flinkSinkWriter; } @@ -133,7 +131,8 @@ public AppendSinkWriter createWriter(MailboxExecutor mailboxExecutor) { @Override public DataStream addPreWriteTopology(DataStream input) { - // For append only sink, we will do bucket shuffle only if bucket keys are not empty. + // For append only sink, we will do bucket shuffle only if bucket keys are not + // empty. if (!bucketKeys.isEmpty() && shuffleByBucketId) { return partition( input, diff --git a/fluss-flink/pom.xml b/fluss-flink/pom.xml index 8bd4c11ff5..6802286e5a 100644 --- a/fluss-flink/pom.xml +++ b/fluss-flink/pom.xml @@ -37,6 +37,7 @@ fluss-flink-1.19 fluss-flink-1.18 fluss-flink-2.1 + fluss-flink-2.2 fluss-flink-tiering From a7cb45e1a0ee4e6b2ff19e333827aba8e5aab3d7 Mon Sep 17 00:00:00 2001 From: vamossagar12 Date: Sat, 20 Dec 2025 21:09:32 +0530 Subject: [PATCH 02/14] spot check apply --- .../main/java/org/apache/fluss/flink/sink/FlinkSink.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java index efea5013c7..8f6c277b3c 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java @@ -58,14 +58,16 @@ class FlinkSink implements Sink, SupportsPreWriteTopology createWriter(InitContext context) throws IOException { - FlinkSinkWriter flinkSinkWriter = builder.createWriter(context.getMailboxExecutor()); + FlinkSinkWriter flinkSinkWriter = + builder.createWriter(context.getMailboxExecutor()); flinkSinkWriter.initialize(InternalSinkWriterMetricGroup.wrap(context.metricGroup())); return flinkSinkWriter; } @Override public SinkWriter createWriter(WriterInitContext context) throws IOException { - FlinkSinkWriter flinkSinkWriter = builder.createWriter(context.getMailboxExecutor()); + FlinkSinkWriter flinkSinkWriter = + builder.createWriter(context.getMailboxExecutor()); flinkSinkWriter.initialize(InternalSinkWriterMetricGroup.wrap(context.metricGroup())); return flinkSinkWriter; } From 8fb1eb5fb0444f44c51a5865af72932f7269945e Mon Sep 17 00:00:00 2001 From: vamossagar12 Date: Sat, 20 Dec 2025 21:46:05 +0530 Subject: [PATCH 03/14] Fix build errors --- .../fluss/flink/catalog/Flink22Catalog.java | 39 ++++++++++-- .../flink/catalog/Flink22CatalogFactory.java | 3 +- .../flink/catalog/Flink22CatalogITCase.java | 3 +- .../flink/catalog/FlinkCatalog22Test.java | 62 +++++++++++++++++++ 4 files changed, 101 insertions(+), 6 deletions(-) create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java index 3bdc5746a1..d10e38628f 100644 --- a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java +++ b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java @@ -18,6 +18,8 @@ package org.apache.fluss.flink.catalog; +import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.flink.lake.LakeFlinkCatalog; import org.apache.fluss.metadata.TableInfo; import org.apache.flink.table.api.Schema; @@ -31,6 +33,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Supplier; /** A {@link FlinkCatalog} used for Flink 2.2. */ public class Flink22Catalog extends FlinkCatalog { @@ -40,15 +43,42 @@ public Flink22Catalog( String defaultDatabase, String bootstrapServers, ClassLoader classLoader, - Map securityConfigs) { - super(name, defaultDatabase, bootstrapServers, classLoader, securityConfigs); + Map securityConfigs, + Supplier> lakeCatalogPropertiesSupplier) { + super( + name, + defaultDatabase, + bootstrapServers, + classLoader, + securityConfigs, + lakeCatalogPropertiesSupplier); + } + + @VisibleForTesting + public Flink22Catalog( + String name, + String defaultDatabase, + String bootstrapServers, + ClassLoader classLoader, + Map securityConfigs, + Supplier> lakeCatalogPropertiesSupplier, + LakeFlinkCatalog lakeFlinkCatalog) { + super( + name, + defaultDatabase, + bootstrapServers, + classLoader, + securityConfigs, + lakeCatalogPropertiesSupplier, + lakeFlinkCatalog); } @Override public CatalogBaseTable getTable(ObjectPath objectPath) throws TableNotExistException, CatalogException { CatalogBaseTable catalogBaseTable = super.getTable(objectPath); - if (!(catalogBaseTable instanceof CatalogTable)) { + if (!(catalogBaseTable instanceof CatalogTable) + || objectPath.getObjectName().contains(LAKE_TABLE_SPLITTER)) { return catalogBaseTable; } @@ -67,7 +97,8 @@ public CatalogBaseTable getTable(ObjectPath objectPath) // Judge whether we can do prefix lookup. TableInfo tableInfo = connection.getTable(toTablePath(objectPath)).getTableInfo(); List bucketKeys = tableInfo.getBucketKeys(); - // For partition table, the physical primary key is the primary key that excludes the + // For partition table, the physical primary key is the primary key that + // excludes the // partition key List physicalPrimaryKeys = tableInfo.getPhysicalPrimaryKeys(); List indexKeys = new ArrayList<>(); diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java index e198a9aee4..2d930c164d 100644 --- a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java +++ b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java @@ -29,6 +29,7 @@ public FlinkCatalog createCatalog(Context context) { catalog.defaultDatabase, catalog.bootstrapServers, catalog.classLoader, - catalog.securityConfigs); + catalog.securityConfigs, + catalog.lakeCatalogPropertiesSupplier); } } diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java index 05f2089717..90f1d2a76c 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java @@ -42,7 +42,8 @@ static void beforeAll() { catalog.defaultDatabase, catalog.bootstrapServers, catalog.classLoader, - catalog.securityConfigs); + catalog.securityConfigs, + catalog.lakeCatalogPropertiesSupplier); catalog.open(); } diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java new file mode 100644 index 0000000000..c36f0a9230 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java @@ -0,0 +1,62 @@ +/* + * 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.fluss.flink.catalog; + +import org.apache.fluss.flink.lake.LakeFlinkCatalog; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.DefaultIndex; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; + +import java.util.Arrays; +import java.util.Collections; + +/** Test for {@link Flink21Catalog}. */ +public class FlinkCatalog22Test extends FlinkCatalogTest { + + @Override + protected FlinkCatalog initCatalog( + String catalogName, + String databaseName, + String bootstrapServers, + LakeFlinkCatalog lakeFlinkCatalog) { + return new Flink22Catalog( + catalogName, + databaseName, + bootstrapServers, + Thread.currentThread().getContextClassLoader(), + Collections.emptyMap(), + Collections::emptyMap, + lakeFlinkCatalog); + } + + protected ResolvedSchema createSchema() { + return new ResolvedSchema( + Arrays.asList( + Column.physical("first", DataTypes.STRING().notNull()), + Column.physical("second", DataTypes.INT()), + Column.physical("third", DataTypes.STRING().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("PK_first_third", Arrays.asList("first", "third")), + Collections.singletonList( + DefaultIndex.newIndex( + "INDEX_first_third", Arrays.asList("first", "third")))); + } +} From fe1f3ab5a6879212ee6e14e6f4396194b9cef63b Mon Sep 17 00:00:00 2001 From: vamossagar12 Date: Sun, 21 Dec 2025 07:07:37 +0530 Subject: [PATCH 04/14] Correcting header file name --- .../flink/catalog/Flink22CatalogITCase.java | 320 +++++------ .../flink/catalog/FlinkCatalog22Test.java | 2 +- .../flink/metrics/Flink22MetricsITCase.java | 5 +- .../procedure/Flink22ProcedureITCase.java | 5 +- .../acl/Flink22AuthorizationITCase.java | 5 +- .../flink/sink/Flink22TableSinkITCase.java | 5 +- .../source/Flink22TableSourceBatchITCase.java | 5 +- .../Flink22TableSourceFailOverITCase.java | 5 +- .../source/Flink22TableSourceITCase.java | 534 +++++++++--------- 9 files changed, 440 insertions(+), 446 deletions(-) diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java index 90f1d2a76c..3341bbb649 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java @@ -26,169 +26,163 @@ import static org.assertj.core.api.Assertions.assertThat; -/** IT case for catalog in Flink 2.1. */ +/** IT case for catalog in Flink 2.2. */ public class Flink22CatalogITCase extends FlinkCatalogITCase { - @BeforeAll - static void beforeAll() { - FlinkCatalogITCase.beforeAll(); - - // close the old one and open a new one later - catalog.close(); - - catalog = - new Flink22Catalog( - catalog.catalogName, - catalog.defaultDatabase, - catalog.bootstrapServers, - catalog.classLoader, - catalog.securityConfigs, - catalog.lakeCatalogPropertiesSupplier); - catalog.open(); - } - - @Test - void testGetTableWithIndex() throws Exception { - String tableName = "table_with_pk_only"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " primary key (a, b) NOT ENFORCED" - + ") with ( " - + " 'connector' = 'fluss' " - + ")", - tableName)); - CatalogTable table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - Schema expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .primaryKey("a", "b") - .index("a", "b") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_prefix_bucket_key"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " primary key (a, b) NOT ENFORCED" - + ") with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'a'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .primaryKey("a", "b") - .index("a", "b") - .index("a") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_bucket_key_is_not_prefix_pk"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " primary key (a, b) NOT ENFORCED" - + ") with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'b'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .primaryKey("a", "b") - .index("a", "b") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_partition_1"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " dt varchar, " - + " primary key (a, b, dt) NOT ENFORCED " - + ") " - + " partitioned by (dt) " - + " with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'a'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .column("dt", DataTypes.STRING().notNull()) - .primaryKey("a", "b", "dt") - .index("a", "b", "dt") - .index("a", "dt") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_partition_2"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " dt varchar, " - + " primary key (dt, a, b) NOT ENFORCED " - + ") " - + " partitioned by (dt) " - + " with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'a'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .column("dt", DataTypes.STRING().notNull()) - .primaryKey("dt", "a", "b") - .index("dt", "a", "b") - .index("a", "dt") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - } - - @Override - protected void addDefaultIndexKey(Schema.Builder schemaBuilder) { - super.addDefaultIndexKey(schemaBuilder); - - Schema currentSchema = schemaBuilder.build(); - currentSchema.getPrimaryKey().ifPresent(pk -> schemaBuilder.index(pk.getColumnNames())); - } + @BeforeAll + static void beforeAll() { + FlinkCatalogITCase.beforeAll(); + + // close the old one and open a new one later + catalog.close(); + + catalog = new Flink22Catalog( + catalog.catalogName, + catalog.defaultDatabase, + catalog.bootstrapServers, + catalog.classLoader, + catalog.securityConfigs, + catalog.lakeCatalogPropertiesSupplier); + catalog.open(); + } + + @Test + void testGetTableWithIndex() throws Exception { + String tableName = "table_with_pk_only"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " primary key (a, b) NOT ENFORCED" + + ") with ( " + + " 'connector' = 'fluss' " + + ")", + tableName)); + CatalogTable table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + Schema expectedSchema = Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .primaryKey("a", "b") + .index("a", "b") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_prefix_bucket_key"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " primary key (a, b) NOT ENFORCED" + + ") with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'a'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .primaryKey("a", "b") + .index("a", "b") + .index("a") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_bucket_key_is_not_prefix_pk"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " primary key (a, b) NOT ENFORCED" + + ") with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'b'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .primaryKey("a", "b") + .index("a", "b") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_partition_1"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " dt varchar, " + + " primary key (a, b, dt) NOT ENFORCED " + + ") " + + " partitioned by (dt) " + + " with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'a'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .column("dt", DataTypes.STRING().notNull()) + .primaryKey("a", "b", "dt") + .index("a", "b", "dt") + .index("a", "dt") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_partition_2"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " dt varchar, " + + " primary key (dt, a, b) NOT ENFORCED " + + ") " + + " partitioned by (dt) " + + " with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'a'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .column("dt", DataTypes.STRING().notNull()) + .primaryKey("dt", "a", "b") + .index("dt", "a", "b") + .index("a", "dt") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + } + + @Override + protected void addDefaultIndexKey(Schema.Builder schemaBuilder) { + super.addDefaultIndexKey(schemaBuilder); + + Schema currentSchema = schemaBuilder.build(); + currentSchema.getPrimaryKey().ifPresent(pk -> schemaBuilder.index(pk.getColumnNames())); + } } diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java index c36f0a9230..4e977eb188 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java @@ -28,7 +28,7 @@ import java.util.Arrays; import java.util.Collections; -/** Test for {@link Flink21Catalog}. */ +/** Test for {@link Flink22Catalog}. */ public class FlinkCatalog22Test extends FlinkCatalogTest { @Override diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java index de728a24d4..6f66b88b23 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java @@ -17,5 +17,6 @@ package org.apache.fluss.flink.metrics; -/** IT case for metrics in Flink 2.1. */ -public class Flink22MetricsITCase extends FlinkMetricsITCase {} +/** IT case for metrics in Flink 2.2. */ +public class Flink22MetricsITCase extends FlinkMetricsITCase { +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java index c37d48000e..85cb87346c 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java @@ -17,5 +17,6 @@ package org.apache.fluss.flink.procedure; -/** IT case for procedure in Flink 2.1. */ -public class Flink22ProcedureITCase extends FlinkProcedureITCase {} +/** IT case for procedure in Flink 2.2. */ +public class Flink22ProcedureITCase extends FlinkProcedureITCase { +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java index 8e0979f35d..e2ad912698 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java @@ -17,5 +17,6 @@ package org.apache.fluss.flink.security.acl; -/** IT case for authorization in Flink 2.1. */ -public class Flink22AuthorizationITCase extends FlinkAuthorizationITCase {} +/** IT case for authorization in Flink 2.2. */ +public class Flink22AuthorizationITCase extends FlinkAuthorizationITCase { +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java index 22c9db1618..9c0b9548db 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java @@ -17,5 +17,6 @@ package org.apache.fluss.flink.sink; -/** IT case for {@link FlinkTableSink} in Flink 2.1. */ -public class Flink22TableSinkITCase extends FlinkTableSinkITCase {} +/** IT case for {@link FlinkTableSink} in Flink 2.2. */ +public class Flink22TableSinkITCase extends FlinkTableSinkITCase { +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java index a7cffa1abb..e6acb5731e 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java @@ -17,5 +17,6 @@ package org.apache.fluss.flink.source; -/** IT case for batch source in Flink 2.1. */ -public class Flink22TableSourceBatchITCase extends FlinkTableSourceBatchITCase {} +/** IT case for batch source in Flink 2.2. */ +public class Flink22TableSourceBatchITCase extends FlinkTableSourceBatchITCase { +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java index 96c2527029..bd0cd4ea96 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java @@ -17,5 +17,6 @@ package org.apache.fluss.flink.source; -/** IT case for source failover and recovery in Flink 2.1. */ -public class Flink22TableSourceFailOverITCase extends FlinkTableSourceFailOverITCase {} +/** IT case for source failover and recovery in Flink 2.2. */ +public class Flink22TableSourceFailOverITCase extends FlinkTableSourceFailOverITCase { +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java index bf643d1de6..3f0677d2b9 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java @@ -34,276 +34,270 @@ import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; -/** IT case for {@link FlinkTableSource} in Flink 2.1. */ +/** IT case for {@link FlinkTableSource} in Flink 2.2. */ public class Flink22TableSourceITCase extends FlinkTableSourceITCase { - @Test - void testDeltaJoin() throws Exception { - // start two jobs for this test: one for DML involving the delta join, and the other for DQL - // to query the results of the sink table - tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); - - String leftTableName = "left_table"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " b1 varchar, " - + " c1 bigint, " - + " d1 int, " - + " e1 bigint, " - + " primary key (c1, d1) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c1', " - // currently, delta join only support append-only source - + " 'table.merge-engine' = 'first_row' " - + ")", - leftTableName)); - List rows1 = - Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v2", 200L, 2, 20000L), - row(3, "v1", 300L, 3, 30000L), - row(4, "v4", 400L, 4, 40000L)); - // write records - TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); - writeRows(conn, leftTablePath, rows1, false); - - String rightTableName = "right_table"; - tEnv.executeSql( - String.format( - "create table %s (" - + " a2 int, " - + " b2 varchar, " - + " c2 bigint, " - + " d2 int, " - + " e2 bigint, " - + " primary key (c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c2', " - // currently, delta join only support append-only source - + " 'table.merge-engine' = 'first_row' " - + ")", - rightTableName)); - List rows2 = - Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v3", 200L, 2, 20000L), - row(3, "v4", 300L, 4, 30000L), - row(4, "v4", 500L, 4, 50000L)); - // write records - TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); - writeRows(conn, rightTablePath, rows2, false); - - String sinkTableName = "sink_table"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " b1 varchar, " - + " c1 bigint, " - + " d1 int, " - + " e1 bigint, " - + " a2 int, " - + " b2 varchar, " - + " c2 bigint, " - + " d2 int, " - + " e2 bigint, " - + " primary key (c1, d1, c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss' " - + ")", - sinkTableName)); - - tEnv.getConfig() - .set( - OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, - OptimizerConfigOptions.DeltaJoinStrategy.FORCE); - - String sql = - String.format( - "INSERT INTO %s SELECT * FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2", - sinkTableName, leftTableName, rightTableName); - - assertThat(tEnv.explainSql(sql)) - .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); - - tEnv.executeSql(sql); - - CloseableIterator collected = - tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); - List expected = - Arrays.asList( - "+I[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", - "-U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", - "+U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", - "+I[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", - "-U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", - "+U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]"); - assertResultsIgnoreOrder(collected, expected, true); - } - - @Test - void testDeltaJoinWithProjectionAndFilter() throws Exception { - // start two jobs for this test: one for DML involving the delta join, and the other for DQL - // to query the results of the sink table - tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); - - String leftTableName = "left_table_proj"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " b1 varchar, " - + " c1 bigint, " - + " d1 int, " - + " e1 bigint, " - + " primary key (c1, d1) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c1', " - + " 'table.merge-engine' = 'first_row' " - + ")", - leftTableName)); - List rows1 = - Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v2", 200L, 2, 20000L), - row(3, "v1", 300L, 3, 30000L)); - TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); - writeRows(conn, leftTablePath, rows1, false); - - String rightTableName = "right_table_proj"; - tEnv.executeSql( - String.format( - "create table %s (" - + " a2 int, " - + " b2 varchar, " - + " c2 bigint, " - + " d2 int, " - + " e2 bigint, " - + " primary key (c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c2', " - + " 'table.merge-engine' = 'first_row' " - + ")", - rightTableName)); - List rows2 = - Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v3", 200L, 2, 20000L), - row(3, "v4", 300L, 4, 30000L)); - TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); - writeRows(conn, rightTablePath, rows2, false); - - String sinkTableName = "sink_table_proj"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " c1 bigint, " - + " a2 int, " - + " primary key (c1) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss' " - + ")", - sinkTableName)); - - tEnv.getConfig() - .set( - OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, - OptimizerConfigOptions.DeltaJoinStrategy.FORCE); - - // Test with projection and filter - String sql = - String.format( - "INSERT INTO %s SELECT a1, c1, a2 FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2 WHERE a1 > 1", - sinkTableName, leftTableName, rightTableName); - - assertThat(tEnv.explainSql(sql)) - .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); - - tEnv.executeSql(sql); - - CloseableIterator collected = - tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); - List expected = Arrays.asList("+I[2, 200, 2]", "-U[2, 200, 2]", "+U[2, 200, 2]"); - assertResultsIgnoreOrder(collected, expected, true); - } - - @Test - void testDeltaJoinWithLookupCache() throws Exception { - tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); - - String leftTableName = "left_table_cache"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " c1 bigint, " - + " d1 int, " - + " primary key (c1, d1) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c1', " - + " 'table.merge-engine' = 'first_row' " - + ")", - leftTableName)); - List rows1 = Arrays.asList(row(1, 100L, 1)); - writeRows(conn, TablePath.of(DEFAULT_DB, leftTableName), rows1, false); - - String rightTableName = "right_table_cache"; - tEnv.executeSql( - String.format( - "create table %s (" - + " a2 int, " - + " c2 bigint, " - + " d2 int, " - + " primary key (c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c2', " - + " 'table.merge-engine' = 'first_row', " - + " 'lookup.cache' = 'partial', " - + " 'lookup.partial-cache.max-rows' = '100' " - + ")", - rightTableName)); - List rows2 = Arrays.asList(row(1, 100L, 1)); - writeRows(conn, TablePath.of(DEFAULT_DB, rightTableName), rows2, false); - - String sinkTableName = "sink_table_cache"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " a2 int, " - + " primary key (a1) NOT ENFORCED" // Dummy PK - + ") with (" - + " 'connector' = 'fluss' " - + ")", - sinkTableName)); - - tEnv.getConfig() - .set( - OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, - OptimizerConfigOptions.DeltaJoinStrategy.FORCE); - - String sql = - String.format( - "INSERT INTO %s SELECT T1.a1, T2.a2 FROM %s AS T1 INNER JOIN %s AS T2 ON T1.c1 = T2.c2 AND T1.d1 = T2.d2", - sinkTableName, leftTableName, rightTableName); - - assertThat(tEnv.explainSql(sql)) - .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); - - tEnv.executeSql(sql); - - CloseableIterator collected = - tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); - List expected = Arrays.asList("+I[1, 1]", "-U[1, 1]", "+U[1, 1]"); - assertResultsIgnoreOrder(collected, expected, true); - } + @Test + void testDeltaJoin() throws Exception { + // start two jobs for this test: one for DML involving the delta join, and the + // other for DQL + // to query the results of the sink table + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + String leftTableName = "left_table"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " b1 varchar, " + + " c1 bigint, " + + " d1 int, " + + " e1 bigint, " + + " primary key (c1, d1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c1', " + // currently, delta join only support append-only source + + " 'table.merge-engine' = 'first_row' " + + ")", + leftTableName)); + List rows1 = Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v2", 200L, 2, 20000L), + row(3, "v1", 300L, 3, 30000L), + row(4, "v4", 400L, 4, 40000L)); + // write records + TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); + writeRows(conn, leftTablePath, rows1, false); + + String rightTableName = "right_table"; + tEnv.executeSql( + String.format( + "create table %s (" + + " a2 int, " + + " b2 varchar, " + + " c2 bigint, " + + " d2 int, " + + " e2 bigint, " + + " primary key (c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c2', " + // currently, delta join only support append-only source + + " 'table.merge-engine' = 'first_row' " + + ")", + rightTableName)); + List rows2 = Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v3", 200L, 2, 20000L), + row(3, "v4", 300L, 4, 30000L), + row(4, "v4", 500L, 4, 50000L)); + // write records + TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); + writeRows(conn, rightTablePath, rows2, false); + + String sinkTableName = "sink_table"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " b1 varchar, " + + " c1 bigint, " + + " d1 int, " + + " e1 bigint, " + + " a2 int, " + + " b2 varchar, " + + " c2 bigint, " + + " d2 int, " + + " e2 bigint, " + + " primary key (c1, d1, c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss' " + + ")", + sinkTableName)); + + tEnv.getConfig() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE); + + String sql = String.format( + "INSERT INTO %s SELECT * FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2", + sinkTableName, leftTableName, rightTableName); + + assertThat(tEnv.explainSql(sql)) + .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); + + tEnv.executeSql(sql); + + CloseableIterator collected = tEnv.executeSql(String.format("select * from %s", sinkTableName)) + .collect(); + List expected = Arrays.asList( + "+I[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", + "-U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", + "+U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", + "+I[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", + "-U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", + "+U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]"); + assertResultsIgnoreOrder(collected, expected, true); + } + + @Test + void testDeltaJoinWithProjectionAndFilter() throws Exception { + // start two jobs for this test: one for DML involving the delta join, and the + // other for DQL + // to query the results of the sink table + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + String leftTableName = "left_table_proj"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " b1 varchar, " + + " c1 bigint, " + + " d1 int, " + + " e1 bigint, " + + " primary key (c1, d1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c1', " + + " 'table.merge-engine' = 'first_row' " + + ")", + leftTableName)); + List rows1 = Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v2", 200L, 2, 20000L), + row(3, "v1", 300L, 3, 30000L)); + TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); + writeRows(conn, leftTablePath, rows1, false); + + String rightTableName = "right_table_proj"; + tEnv.executeSql( + String.format( + "create table %s (" + + " a2 int, " + + " b2 varchar, " + + " c2 bigint, " + + " d2 int, " + + " e2 bigint, " + + " primary key (c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c2', " + + " 'table.merge-engine' = 'first_row' " + + ")", + rightTableName)); + List rows2 = Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v3", 200L, 2, 20000L), + row(3, "v4", 300L, 4, 30000L)); + TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); + writeRows(conn, rightTablePath, rows2, false); + + String sinkTableName = "sink_table_proj"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " c1 bigint, " + + " a2 int, " + + " primary key (c1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss' " + + ")", + sinkTableName)); + + tEnv.getConfig() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE); + + // Test with projection and filter + String sql = String.format( + "INSERT INTO %s SELECT a1, c1, a2 FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2 WHERE a1 > 1", + sinkTableName, leftTableName, rightTableName); + + assertThat(tEnv.explainSql(sql)) + .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); + + tEnv.executeSql(sql); + + CloseableIterator collected = tEnv.executeSql(String.format("select * from %s", sinkTableName)) + .collect(); + List expected = Arrays.asList("+I[2, 200, 2]", "-U[2, 200, 2]", "+U[2, 200, 2]"); + assertResultsIgnoreOrder(collected, expected, true); + } + + @Test + void testDeltaJoinWithLookupCache() throws Exception { + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + String leftTableName = "left_table_cache"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " c1 bigint, " + + " d1 int, " + + " primary key (c1, d1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c1', " + + " 'table.merge-engine' = 'first_row' " + + ")", + leftTableName)); + List rows1 = Arrays.asList(row(1, 100L, 1)); + writeRows(conn, TablePath.of(DEFAULT_DB, leftTableName), rows1, false); + + String rightTableName = "right_table_cache"; + tEnv.executeSql( + String.format( + "create table %s (" + + " a2 int, " + + " c2 bigint, " + + " d2 int, " + + " primary key (c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c2', " + + " 'table.merge-engine' = 'first_row', " + + " 'lookup.cache' = 'partial', " + + " 'lookup.partial-cache.max-rows' = '100' " + + ")", + rightTableName)); + List rows2 = Arrays.asList(row(1, 100L, 1)); + writeRows(conn, TablePath.of(DEFAULT_DB, rightTableName), rows2, false); + + String sinkTableName = "sink_table_cache"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " a2 int, " + + " primary key (a1) NOT ENFORCED" // Dummy PK + + ") with (" + + " 'connector' = 'fluss' " + + ")", + sinkTableName)); + + tEnv.getConfig() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE); + + String sql = String.format( + "INSERT INTO %s SELECT T1.a1, T2.a2 FROM %s AS T1 INNER JOIN %s AS T2 ON T1.c1 = T2.c2 AND T1.d1 = T2.d2", + sinkTableName, leftTableName, rightTableName); + + assertThat(tEnv.explainSql(sql)) + .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); + + tEnv.executeSql(sql); + + CloseableIterator collected = tEnv.executeSql(String.format("select * from %s", sinkTableName)) + .collect(); + List expected = Arrays.asList("+I[1, 1]", "-U[1, 1]", "+U[1, 1]"); + assertResultsIgnoreOrder(collected, expected, true); + } } From cfeafc886455875e36fa4a8c42136f29c6d93203 Mon Sep 17 00:00:00 2001 From: vamossagar12 Date: Mon, 22 Dec 2025 09:28:56 +0530 Subject: [PATCH 05/14] more spotless checks --- .../flink/catalog/Flink22CatalogITCase.java | 318 ++++++----- .../flink/metrics/Flink22MetricsITCase.java | 3 +- .../procedure/Flink22ProcedureITCase.java | 3 +- .../acl/Flink22AuthorizationITCase.java | 3 +- .../flink/sink/Flink22TableSinkITCase.java | 3 +- .../source/Flink22TableSourceBatchITCase.java | 3 +- .../Flink22TableSourceFailOverITCase.java | 3 +- .../source/Flink22TableSourceITCase.java | 534 +++++++++--------- 8 files changed, 439 insertions(+), 431 deletions(-) diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java index 3341bbb649..e51a38f29f 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java @@ -29,160 +29,166 @@ /** IT case for catalog in Flink 2.2. */ public class Flink22CatalogITCase extends FlinkCatalogITCase { - @BeforeAll - static void beforeAll() { - FlinkCatalogITCase.beforeAll(); - - // close the old one and open a new one later - catalog.close(); - - catalog = new Flink22Catalog( - catalog.catalogName, - catalog.defaultDatabase, - catalog.bootstrapServers, - catalog.classLoader, - catalog.securityConfigs, - catalog.lakeCatalogPropertiesSupplier); - catalog.open(); - } - - @Test - void testGetTableWithIndex() throws Exception { - String tableName = "table_with_pk_only"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " primary key (a, b) NOT ENFORCED" - + ") with ( " - + " 'connector' = 'fluss' " - + ")", - tableName)); - CatalogTable table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - Schema expectedSchema = Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .primaryKey("a", "b") - .index("a", "b") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_prefix_bucket_key"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " primary key (a, b) NOT ENFORCED" - + ") with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'a'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .primaryKey("a", "b") - .index("a", "b") - .index("a") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_bucket_key_is_not_prefix_pk"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " primary key (a, b) NOT ENFORCED" - + ") with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'b'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .primaryKey("a", "b") - .index("a", "b") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_partition_1"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " dt varchar, " - + " primary key (a, b, dt) NOT ENFORCED " - + ") " - + " partitioned by (dt) " - + " with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'a'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .column("dt", DataTypes.STRING().notNull()) - .primaryKey("a", "b", "dt") - .index("a", "b", "dt") - .index("a", "dt") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_partition_2"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " dt varchar, " - + " primary key (dt, a, b) NOT ENFORCED " - + ") " - + " partitioned by (dt) " - + " with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'a'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .column("dt", DataTypes.STRING().notNull()) - .primaryKey("dt", "a", "b") - .index("dt", "a", "b") - .index("a", "dt") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - } - - @Override - protected void addDefaultIndexKey(Schema.Builder schemaBuilder) { - super.addDefaultIndexKey(schemaBuilder); - - Schema currentSchema = schemaBuilder.build(); - currentSchema.getPrimaryKey().ifPresent(pk -> schemaBuilder.index(pk.getColumnNames())); - } + @BeforeAll + static void beforeAll() { + FlinkCatalogITCase.beforeAll(); + + // close the old one and open a new one later + catalog.close(); + + catalog = + new Flink22Catalog( + catalog.catalogName, + catalog.defaultDatabase, + catalog.bootstrapServers, + catalog.classLoader, + catalog.securityConfigs, + catalog.lakeCatalogPropertiesSupplier); + catalog.open(); + } + + @Test + void testGetTableWithIndex() throws Exception { + String tableName = "table_with_pk_only"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " primary key (a, b) NOT ENFORCED" + + ") with ( " + + " 'connector' = 'fluss' " + + ")", + tableName)); + CatalogTable table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + Schema expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .primaryKey("a", "b") + .index("a", "b") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_prefix_bucket_key"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " primary key (a, b) NOT ENFORCED" + + ") with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'a'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .primaryKey("a", "b") + .index("a", "b") + .index("a") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_bucket_key_is_not_prefix_pk"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " primary key (a, b) NOT ENFORCED" + + ") with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'b'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .primaryKey("a", "b") + .index("a", "b") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_partition_1"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " dt varchar, " + + " primary key (a, b, dt) NOT ENFORCED " + + ") " + + " partitioned by (dt) " + + " with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'a'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .column("dt", DataTypes.STRING().notNull()) + .primaryKey("a", "b", "dt") + .index("a", "b", "dt") + .index("a", "dt") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + + tableName = "table_with_partition_2"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a int, " + + " b varchar, " + + " c bigint, " + + " dt varchar, " + + " primary key (dt, a, b) NOT ENFORCED " + + ") " + + " partitioned by (dt) " + + " with ( " + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'a'" + + ")", + tableName)); + + table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); + expectedSchema = + Schema.newBuilder() + .column("a", DataTypes.INT().notNull()) + .column("b", DataTypes.STRING().notNull()) + .column("c", DataTypes.BIGINT()) + .column("dt", DataTypes.STRING().notNull()) + .primaryKey("dt", "a", "b") + .index("dt", "a", "b") + .index("a", "dt") + .build(); + assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); + } + + @Override + protected void addDefaultIndexKey(Schema.Builder schemaBuilder) { + super.addDefaultIndexKey(schemaBuilder); + + Schema currentSchema = schemaBuilder.build(); + currentSchema.getPrimaryKey().ifPresent(pk -> schemaBuilder.index(pk.getColumnNames())); + } } diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java index 6f66b88b23..37d795e0bd 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/metrics/Flink22MetricsITCase.java @@ -18,5 +18,4 @@ package org.apache.fluss.flink.metrics; /** IT case for metrics in Flink 2.2. */ -public class Flink22MetricsITCase extends FlinkMetricsITCase { -} +public class Flink22MetricsITCase extends FlinkMetricsITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java index 85cb87346c..84612dc94b 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/procedure/Flink22ProcedureITCase.java @@ -18,5 +18,4 @@ package org.apache.fluss.flink.procedure; /** IT case for procedure in Flink 2.2. */ -public class Flink22ProcedureITCase extends FlinkProcedureITCase { -} +public class Flink22ProcedureITCase extends FlinkProcedureITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java index e2ad912698..192be6b902 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/security/acl/Flink22AuthorizationITCase.java @@ -18,5 +18,4 @@ package org.apache.fluss.flink.security.acl; /** IT case for authorization in Flink 2.2. */ -public class Flink22AuthorizationITCase extends FlinkAuthorizationITCase { -} +public class Flink22AuthorizationITCase extends FlinkAuthorizationITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java index 9c0b9548db..9aa830096a 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22TableSinkITCase.java @@ -18,5 +18,4 @@ package org.apache.fluss.flink.sink; /** IT case for {@link FlinkTableSink} in Flink 2.2. */ -public class Flink22TableSinkITCase extends FlinkTableSinkITCase { -} +public class Flink22TableSinkITCase extends FlinkTableSinkITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java index e6acb5731e..f65966ea6f 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java @@ -18,5 +18,4 @@ package org.apache.fluss.flink.source; /** IT case for batch source in Flink 2.2. */ -public class Flink22TableSourceBatchITCase extends FlinkTableSourceBatchITCase { -} +public class Flink22TableSourceBatchITCase extends FlinkTableSourceBatchITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java index bd0cd4ea96..d909bc5365 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceFailOverITCase.java @@ -18,5 +18,4 @@ package org.apache.fluss.flink.source; /** IT case for source failover and recovery in Flink 2.2. */ -public class Flink22TableSourceFailOverITCase extends FlinkTableSourceFailOverITCase { -} +public class Flink22TableSourceFailOverITCase extends FlinkTableSourceFailOverITCase {} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java index 3f0677d2b9..24d8157b52 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java @@ -37,267 +37,275 @@ /** IT case for {@link FlinkTableSource} in Flink 2.2. */ public class Flink22TableSourceITCase extends FlinkTableSourceITCase { - @Test - void testDeltaJoin() throws Exception { - // start two jobs for this test: one for DML involving the delta join, and the - // other for DQL - // to query the results of the sink table - tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); - - String leftTableName = "left_table"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " b1 varchar, " - + " c1 bigint, " - + " d1 int, " - + " e1 bigint, " - + " primary key (c1, d1) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c1', " - // currently, delta join only support append-only source - + " 'table.merge-engine' = 'first_row' " - + ")", - leftTableName)); - List rows1 = Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v2", 200L, 2, 20000L), - row(3, "v1", 300L, 3, 30000L), - row(4, "v4", 400L, 4, 40000L)); - // write records - TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); - writeRows(conn, leftTablePath, rows1, false); - - String rightTableName = "right_table"; - tEnv.executeSql( - String.format( - "create table %s (" - + " a2 int, " - + " b2 varchar, " - + " c2 bigint, " - + " d2 int, " - + " e2 bigint, " - + " primary key (c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c2', " - // currently, delta join only support append-only source - + " 'table.merge-engine' = 'first_row' " - + ")", - rightTableName)); - List rows2 = Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v3", 200L, 2, 20000L), - row(3, "v4", 300L, 4, 30000L), - row(4, "v4", 500L, 4, 50000L)); - // write records - TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); - writeRows(conn, rightTablePath, rows2, false); - - String sinkTableName = "sink_table"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " b1 varchar, " - + " c1 bigint, " - + " d1 int, " - + " e1 bigint, " - + " a2 int, " - + " b2 varchar, " - + " c2 bigint, " - + " d2 int, " - + " e2 bigint, " - + " primary key (c1, d1, c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss' " - + ")", - sinkTableName)); - - tEnv.getConfig() - .set( - OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, - OptimizerConfigOptions.DeltaJoinStrategy.FORCE); - - String sql = String.format( - "INSERT INTO %s SELECT * FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2", - sinkTableName, leftTableName, rightTableName); - - assertThat(tEnv.explainSql(sql)) - .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); - - tEnv.executeSql(sql); - - CloseableIterator collected = tEnv.executeSql(String.format("select * from %s", sinkTableName)) - .collect(); - List expected = Arrays.asList( - "+I[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", - "-U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", - "+U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", - "+I[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", - "-U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", - "+U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]"); - assertResultsIgnoreOrder(collected, expected, true); - } - - @Test - void testDeltaJoinWithProjectionAndFilter() throws Exception { - // start two jobs for this test: one for DML involving the delta join, and the - // other for DQL - // to query the results of the sink table - tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); - - String leftTableName = "left_table_proj"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " b1 varchar, " - + " c1 bigint, " - + " d1 int, " - + " e1 bigint, " - + " primary key (c1, d1) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c1', " - + " 'table.merge-engine' = 'first_row' " - + ")", - leftTableName)); - List rows1 = Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v2", 200L, 2, 20000L), - row(3, "v1", 300L, 3, 30000L)); - TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); - writeRows(conn, leftTablePath, rows1, false); - - String rightTableName = "right_table_proj"; - tEnv.executeSql( - String.format( - "create table %s (" - + " a2 int, " - + " b2 varchar, " - + " c2 bigint, " - + " d2 int, " - + " e2 bigint, " - + " primary key (c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c2', " - + " 'table.merge-engine' = 'first_row' " - + ")", - rightTableName)); - List rows2 = Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v3", 200L, 2, 20000L), - row(3, "v4", 300L, 4, 30000L)); - TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); - writeRows(conn, rightTablePath, rows2, false); - - String sinkTableName = "sink_table_proj"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " c1 bigint, " - + " a2 int, " - + " primary key (c1) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss' " - + ")", - sinkTableName)); - - tEnv.getConfig() - .set( - OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, - OptimizerConfigOptions.DeltaJoinStrategy.FORCE); - - // Test with projection and filter - String sql = String.format( - "INSERT INTO %s SELECT a1, c1, a2 FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2 WHERE a1 > 1", - sinkTableName, leftTableName, rightTableName); - - assertThat(tEnv.explainSql(sql)) - .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); - - tEnv.executeSql(sql); - - CloseableIterator collected = tEnv.executeSql(String.format("select * from %s", sinkTableName)) - .collect(); - List expected = Arrays.asList("+I[2, 200, 2]", "-U[2, 200, 2]", "+U[2, 200, 2]"); - assertResultsIgnoreOrder(collected, expected, true); - } - - @Test - void testDeltaJoinWithLookupCache() throws Exception { - tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); - - String leftTableName = "left_table_cache"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " c1 bigint, " - + " d1 int, " - + " primary key (c1, d1) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c1', " - + " 'table.merge-engine' = 'first_row' " - + ")", - leftTableName)); - List rows1 = Arrays.asList(row(1, 100L, 1)); - writeRows(conn, TablePath.of(DEFAULT_DB, leftTableName), rows1, false); - - String rightTableName = "right_table_cache"; - tEnv.executeSql( - String.format( - "create table %s (" - + " a2 int, " - + " c2 bigint, " - + " d2 int, " - + " primary key (c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c2', " - + " 'table.merge-engine' = 'first_row', " - + " 'lookup.cache' = 'partial', " - + " 'lookup.partial-cache.max-rows' = '100' " - + ")", - rightTableName)); - List rows2 = Arrays.asList(row(1, 100L, 1)); - writeRows(conn, TablePath.of(DEFAULT_DB, rightTableName), rows2, false); - - String sinkTableName = "sink_table_cache"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " a2 int, " - + " primary key (a1) NOT ENFORCED" // Dummy PK - + ") with (" - + " 'connector' = 'fluss' " - + ")", - sinkTableName)); - - tEnv.getConfig() - .set( - OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, - OptimizerConfigOptions.DeltaJoinStrategy.FORCE); - - String sql = String.format( - "INSERT INTO %s SELECT T1.a1, T2.a2 FROM %s AS T1 INNER JOIN %s AS T2 ON T1.c1 = T2.c2 AND T1.d1 = T2.d2", - sinkTableName, leftTableName, rightTableName); - - assertThat(tEnv.explainSql(sql)) - .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); - - tEnv.executeSql(sql); - - CloseableIterator collected = tEnv.executeSql(String.format("select * from %s", sinkTableName)) - .collect(); - List expected = Arrays.asList("+I[1, 1]", "-U[1, 1]", "+U[1, 1]"); - assertResultsIgnoreOrder(collected, expected, true); - } + @Test + void testDeltaJoin() throws Exception { + // start two jobs for this test: one for DML involving the delta join, and the + // other for DQL + // to query the results of the sink table + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + String leftTableName = "left_table"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " b1 varchar, " + + " c1 bigint, " + + " d1 int, " + + " e1 bigint, " + + " primary key (c1, d1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c1', " + // currently, delta join only support append-only source + + " 'table.merge-engine' = 'first_row' " + + ")", + leftTableName)); + List rows1 = + Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v2", 200L, 2, 20000L), + row(3, "v1", 300L, 3, 30000L), + row(4, "v4", 400L, 4, 40000L)); + // write records + TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); + writeRows(conn, leftTablePath, rows1, false); + + String rightTableName = "right_table"; + tEnv.executeSql( + String.format( + "create table %s (" + + " a2 int, " + + " b2 varchar, " + + " c2 bigint, " + + " d2 int, " + + " e2 bigint, " + + " primary key (c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c2', " + // currently, delta join only support append-only source + + " 'table.merge-engine' = 'first_row' " + + ")", + rightTableName)); + List rows2 = + Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v3", 200L, 2, 20000L), + row(3, "v4", 300L, 4, 30000L), + row(4, "v4", 500L, 4, 50000L)); + // write records + TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); + writeRows(conn, rightTablePath, rows2, false); + + String sinkTableName = "sink_table"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " b1 varchar, " + + " c1 bigint, " + + " d1 int, " + + " e1 bigint, " + + " a2 int, " + + " b2 varchar, " + + " c2 bigint, " + + " d2 int, " + + " e2 bigint, " + + " primary key (c1, d1, c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss' " + + ")", + sinkTableName)); + + tEnv.getConfig() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE); + + String sql = + String.format( + "INSERT INTO %s SELECT * FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2", + sinkTableName, leftTableName, rightTableName); + + assertThat(tEnv.explainSql(sql)) + .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); + + tEnv.executeSql(sql); + + CloseableIterator collected = + tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); + List expected = + Arrays.asList( + "+I[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", + "-U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", + "+U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", + "+I[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", + "-U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", + "+U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]"); + assertResultsIgnoreOrder(collected, expected, true); + } + + @Test + void testDeltaJoinWithProjectionAndFilter() throws Exception { + // start two jobs for this test: one for DML involving the delta join, and the + // other for DQL + // to query the results of the sink table + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + String leftTableName = "left_table_proj"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " b1 varchar, " + + " c1 bigint, " + + " d1 int, " + + " e1 bigint, " + + " primary key (c1, d1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c1', " + + " 'table.merge-engine' = 'first_row' " + + ")", + leftTableName)); + List rows1 = + Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v2", 200L, 2, 20000L), + row(3, "v1", 300L, 3, 30000L)); + TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); + writeRows(conn, leftTablePath, rows1, false); + + String rightTableName = "right_table_proj"; + tEnv.executeSql( + String.format( + "create table %s (" + + " a2 int, " + + " b2 varchar, " + + " c2 bigint, " + + " d2 int, " + + " e2 bigint, " + + " primary key (c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c2', " + + " 'table.merge-engine' = 'first_row' " + + ")", + rightTableName)); + List rows2 = + Arrays.asList( + row(1, "v1", 100L, 1, 10000L), + row(2, "v3", 200L, 2, 20000L), + row(3, "v4", 300L, 4, 30000L)); + TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); + writeRows(conn, rightTablePath, rows2, false); + + String sinkTableName = "sink_table_proj"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " c1 bigint, " + + " a2 int, " + + " primary key (c1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss' " + + ")", + sinkTableName)); + + tEnv.getConfig() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE); + + // Test with projection and filter + String sql = + String.format( + "INSERT INTO %s SELECT a1, c1, a2 FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2 WHERE a1 > 1", + sinkTableName, leftTableName, rightTableName); + + assertThat(tEnv.explainSql(sql)) + .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); + + tEnv.executeSql(sql); + + CloseableIterator collected = + tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); + List expected = Arrays.asList("+I[2, 200, 2]", "-U[2, 200, 2]", "+U[2, 200, 2]"); + assertResultsIgnoreOrder(collected, expected, true); + } + + @Test + void testDeltaJoinWithLookupCache() throws Exception { + tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + + String leftTableName = "left_table_cache"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " c1 bigint, " + + " d1 int, " + + " primary key (c1, d1) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c1', " + + " 'table.merge-engine' = 'first_row' " + + ")", + leftTableName)); + List rows1 = Arrays.asList(row(1, 100L, 1)); + writeRows(conn, TablePath.of(DEFAULT_DB, leftTableName), rows1, false); + + String rightTableName = "right_table_cache"; + tEnv.executeSql( + String.format( + "create table %s (" + + " a2 int, " + + " c2 bigint, " + + " d2 int, " + + " primary key (c2, d2) NOT ENFORCED" + + ") with (" + + " 'connector' = 'fluss', " + + " 'bucket.key' = 'c2', " + + " 'table.merge-engine' = 'first_row', " + + " 'lookup.cache' = 'partial', " + + " 'lookup.partial-cache.max-rows' = '100' " + + ")", + rightTableName)); + List rows2 = Arrays.asList(row(1, 100L, 1)); + writeRows(conn, TablePath.of(DEFAULT_DB, rightTableName), rows2, false); + + String sinkTableName = "sink_table_cache"; + tEnv.executeSql( + String.format( + "create table %s ( " + + " a1 int, " + + " a2 int, " + + " primary key (a1) NOT ENFORCED" // Dummy PK + + ") with (" + + " 'connector' = 'fluss' " + + ")", + sinkTableName)); + + tEnv.getConfig() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE); + + String sql = + String.format( + "INSERT INTO %s SELECT T1.a1, T2.a2 FROM %s AS T1 INNER JOIN %s AS T2 ON T1.c1 = T2.c2 AND T1.d1 = T2.d2", + sinkTableName, leftTableName, rightTableName); + + assertThat(tEnv.explainSql(sql)) + .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); + + tEnv.executeSql(sql); + + CloseableIterator collected = + tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); + List expected = Arrays.asList("+I[1, 1]", "-U[1, 1]", "+U[1, 1]"); + assertResultsIgnoreOrder(collected, expected, true); + } } From b31ca5646be61cdef18a2de371eb0a3d064f50af Mon Sep 17 00:00:00 2001 From: vamossagar12 Date: Tue, 23 Dec 2025 12:52:33 +0530 Subject: [PATCH 06/14] fixing spotless checks + tests --- .../flink/catalog/FlinkCatalog22Test.java | 42 +++++++++++++++++++ .../flink/sink/Flink22ComplexTypeITCase.java | 2 +- .../fluss/flink/catalog/FlinkCatalogTest.java | 19 +++++---- fluss-test-coverage/pom.xml | 4 ++ 4 files changed, 59 insertions(+), 8 deletions(-) diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java index 4e977eb188..be59dfe59e 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java @@ -59,4 +59,46 @@ protected ResolvedSchema createSchema() { DefaultIndex.newIndex( "INDEX_first_third", Arrays.asList("first", "third")))); } + + @Override + protected org.apache.flink.table.catalog.CatalogMaterializedTable newCatalogMaterializedTable( + ResolvedSchema resolvedSchema, + org.apache.flink.table.catalog.CatalogMaterializedTable.RefreshMode refreshMode, + java.util.Map options) { + org.apache.flink.table.catalog.CatalogMaterializedTable origin = + org.apache.flink.table.catalog.CatalogMaterializedTable.newBuilder() + .schema( + org.apache.flink.table.api.Schema.newBuilder() + .fromResolvedSchema(resolvedSchema) + .build()) + .comment("test comment") + .options(options) + .partitionKeys(Collections.emptyList()) + .definitionQuery("select first, second, third from t") + .freshness( + org.apache.flink.table.catalog.IntervalFreshness.of( + "5", + org.apache.flink.table.catalog.IntervalFreshness.TimeUnit + .SECOND)) + .logicalRefreshMode( + refreshMode + == org.apache.flink.table.catalog + .CatalogMaterializedTable.RefreshMode + .CONTINUOUS + ? org.apache.flink.table.catalog.CatalogMaterializedTable + .LogicalRefreshMode.CONTINUOUS + : org.apache.flink.table.catalog.CatalogMaterializedTable + .LogicalRefreshMode.FULL) + .refreshMode(refreshMode) + .refreshStatus( + org.apache.flink.table.catalog.CatalogMaterializedTable + .RefreshStatus.INITIALIZING) + .build(); + return new org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable( + origin, + resolvedSchema, + refreshMode, + org.apache.flink.table.catalog.IntervalFreshness.of( + "5", org.apache.flink.table.catalog.IntervalFreshness.TimeUnit.SECOND)); + } } diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22ComplexTypeITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22ComplexTypeITCase.java index 9afc956caa..cbb1f9a966 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22ComplexTypeITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/sink/Flink22ComplexTypeITCase.java @@ -17,5 +17,5 @@ package org.apache.fluss.flink.sink; -/** Integration tests for Array type support in Flink 2.1. */ +/** Integration tests for Array type support in Flink 2.2. */ public class Flink22ComplexTypeITCase extends FlinkComplexTypeITCase {} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java index 9b915a74d5..41176f3884 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java @@ -139,7 +139,7 @@ private CatalogTable newCatalogTable( return new ResolvedCatalogTable(origin, resolvedSchema); } - private CatalogMaterializedTable newCatalogMaterializedTable( + protected CatalogMaterializedTable newCatalogMaterializedTable( ResolvedSchema resolvedSchema, CatalogMaterializedTable.RefreshMode refreshMode, Map options) { @@ -306,7 +306,8 @@ void testCreateTable() throws Exception { assertThatThrownBy(() -> catalog.renameTable(this.tableInDefaultDb, "newName", false)) .isInstanceOf(UnsupportedOperationException.class); - // Test lake table handling - should throw TableNotExistException for non-existent lake + // Test lake table handling - should throw TableNotExistException for + // non-existent lake // table ObjectPath lakePath = new ObjectPath(DEFAULT_DB, "regularTable$lake"); assertThatThrownBy(() -> catalog.getTable(lakePath)) @@ -332,7 +333,8 @@ void testCreateAlreadyExistsLakeTable() throws Exception { // drop fluss table catalog.dropTable(lakeTablePath, false); assertThat(catalog.tableExists(lakeTablePath)).isFalse(); - // create the table again should be ok, because the existing lake table is matched + // create the table again should be ok, because the existing lake table is + // matched catalog.createTable(lakeTablePath, table, false); } @@ -547,7 +549,8 @@ void testCreateUnsupportedMaterializedTable() { this.createSchema(), CatalogMaterializedTable.RefreshMode.FULL, Collections.emptyMap()); - // Fluss doesn't support insert overwrite in batch mode now, so full refresh mode is not + // Fluss doesn't support insert overwrite in batch mode now, so full refresh + // mode is not // supported now. assertThatThrownBy( () -> @@ -615,7 +618,8 @@ void testDatabase() throws Exception { .hasMessage("Database %s in catalog %s is not empty.", "db1", CATALOG_NAME); // should be ok since we set cascade = true catalog.dropDatabase("db1", false, true); - // drop it again, should throw exception since db1 is not exist and we set ignoreIfNotExists + // drop it again, should throw exception since db1 is not exist and we set + // ignoreIfNotExists // = false assertThatThrownBy(() -> catalog.dropDatabase("db1", false, true)) .isInstanceOf(DatabaseNotExistException.class) @@ -627,7 +631,8 @@ void testDatabase() throws Exception { catalog.dropDatabase("db2", false, true); // should be empty assertThat(catalog.listDatabases()).isEqualTo(Collections.singletonList(DEFAULT_DB)); - // should throw exception since the db is not exist and we set ignoreIfNotExists = false + // should throw exception since the db is not exist and we set ignoreIfNotExists + // = false assertThatThrownBy(() -> catalog.listTables("unknown")) .isInstanceOf(DatabaseNotExistException.class) .hasMessage("Database %s does not exist in Catalog %s.", "unknown", CATALOG_NAME); @@ -846,7 +851,7 @@ void testConnectionFailureHandling() { @Test void testStatisticsOperations() throws Exception { - // Statistics testing + // Statistics testing CatalogTable table = newCatalogTable(Collections.emptyMap()); ObjectPath tablePath = new ObjectPath(DEFAULT_DB, "statsTable"); catalog.createTable(tablePath, table, false); diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index 46c34da57a..f9e24f24c2 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -166,6 +166,8 @@ fluss-test-utils/** fluss-flink/**/target/classes/org/apache/fluss/flink/adapter/** + + fluss-flink/fluss-flink-2.2/target/classes/org/apache/fluss/flink/sink/FlinkSink* @@ -208,6 +210,8 @@ fluss-test-utils/** fluss-flink/**/target/classes/org/apache/fluss/flink/adapter/** + + fluss-flink/fluss-flink-2.2/target/classes/org/apache/fluss/flink/sink/FlinkSink* From 9e505e52f5cee4e7df2bdcf94503a40d2a1eae24 Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Tue, 23 Dec 2025 15:49:25 +0800 Subject: [PATCH 07/14] hongshun's advice: 1. merge flink21Catalog and flink22Catalog to FlinkCatalog. 2. add flink-2.2 to ci tests of flink --- .github/workflows/stage.sh | 1 + .../flink/api/connector/sink2/Sink.java | 61 ----- .../fluss/flink/adapter/FlinkSinkAdapter.java | 43 ++++ .../fluss/flink/adapter/SchemaAdapter.java | 43 ++++ .../fluss/flink/catalog/Flink21Catalog.java | 136 ----------- .../flink/catalog/Flink21CatalogFactory.java | 35 --- .../org.apache.flink.table.factories.Factory | 2 +- .../flink/catalog/Flink21CatalogITCase.java | 19 -- .../flink/catalog/FlinkCatalog21Test.java | 20 +- .../fluss/flink/adapter/FlinkSinkAdapter.java | 43 ++++ .../adapter/MultipleParameterToolAdapter.java | 46 ++++ .../fluss/flink/adapter/SchemaAdapter.java | 43 ++++ .../fluss/flink/catalog/Flink22Catalog.java | 137 ----------- .../flink/catalog/Flink22CatalogFactory.java | 35 --- .../apache/fluss/flink/sink/FlinkSink.java | 213 ------------------ .../org.apache.flink.table.factories.Factory | 2 +- .../flink/catalog/Flink22CatalogITCase.java | 19 -- .../flink/catalog/FlinkCatalog22Test.java | 67 ++---- .../source/Flink22TableSourceBatchITCase.java | 59 ++++- .../org.junit.jupiter.api.extension.Extension | 19 ++ .../src/test/resources/log4j2-test.properties | 32 +++ .../fluss/flink/adapter/FlinkSinkAdapter.java | 48 ++++ .../fluss/flink/adapter/SchemaAdapter.java | 43 ++++ ...hreadMultiplexSourceReaderBaseAdapter.java | 2 + .../fluss/flink/catalog/FlinkCatalog.java | 61 ++++- .../apache/fluss/flink/sink/FlinkSink.java | 25 +- .../fluss/flink/catalog/FlinkCatalogTest.java | 12 +- .../source/FlinkTableSourceBatchITCase.java | 6 +- fluss-test-coverage/pom.xml | 4 - 29 files changed, 518 insertions(+), 758 deletions(-) delete mode 100644 fluss-flink/fluss-flink-2.1/src/main/java/org/apache/flink/api/connector/sink2/Sink.java create mode 100644 fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java create mode 100644 fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/catalog/Flink21Catalog.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/catalog/Flink21CatalogFactory.java create mode 100644 fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java create mode 100644 fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/MultipleParameterToolAdapter.java create mode 100644 fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java delete mode 100644 fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java delete mode 100644 fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java delete mode 100644 fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension create mode 100644 fluss-flink/fluss-flink-2.2/src/test/resources/log4j2-test.properties create mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java create mode 100644 fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java diff --git a/.github/workflows/stage.sh b/.github/workflows/stage.sh index 49a3ed57e9..3f2b3f3215 100755 --- a/.github/workflows/stage.sh +++ b/.github/workflows/stage.sh @@ -25,6 +25,7 @@ MODULES_FLINK="\ fluss-flink,\ fluss-flink/fluss-flink-common,\ fluss-flink/fluss-flink-2.1,\ +fluss-flink/fluss-flink-2.2,\ fluss-flink/fluss-flink-1.20,\ " diff --git a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/flink/api/connector/sink2/Sink.java b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/flink/api/connector/sink2/Sink.java deleted file mode 100644 index 8cab1cb533..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/flink/api/connector/sink2/Sink.java +++ /dev/null @@ -1,61 +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.api.connector.sink2; - -import org.apache.flink.api.common.operators.MailboxExecutor; -import org.apache.flink.metrics.groups.SinkWriterMetricGroup; - -import java.io.IOException; -import java.io.Serializable; - -/** - * Placeholder class to resolve compatibility issues. This placeholder class can be removed once we - * drop the support of Flink v1.18 which requires the legacy InitContext interface. - */ -public interface Sink extends Serializable { - - /** - * Creates a {@link SinkWriter}. - * - * @param context the runtime context. - * @return A sink writer. - * @throws IOException for any failure during creation. - */ - SinkWriter createWriter(WriterInitContext context) throws IOException; - - /** The interface exposes some runtime info for creating a {@link SinkWriter}. */ - interface InitContext { - - /** - * Returns the mailbox executor that allows to execute {@link Runnable}s inside the task - * thread in between record processing. - * - *

Note that this method should not be used per-record for performance reasons in the - * same way as records should not be sent to the external system individually. Rather, - * implementers are expected to batch records and only enqueue a single {@link Runnable} per - * batch to handle the result. - */ - MailboxExecutor getMailboxExecutor(); - - /** - * @return The metric group this writer belongs to. - */ - SinkWriterMetricGroup metricGroup(); - } -} diff --git a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java new file mode 100644 index 0000000000..bb518ce861 --- /dev/null +++ b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java @@ -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.fluss.flink.adapter; + +import org.apache.flink.api.common.operators.MailboxExecutor; +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 org.apache.flink.metrics.groups.SinkWriterMetricGroup; + +import java.io.IOException; + +/** + * Flink sink adapter which hide the different version of createWriter method. + * + *

TODO: remove this class when no longer support all the Flink 1.x series. + */ +public abstract class FlinkSinkAdapter implements Sink { + + @Override + public SinkWriter createWriter(WriterInitContext writerInitContext) throws IOException { + return createWriter( + writerInitContext.getMailboxExecutor(), writerInitContext.metricGroup()); + } + + protected abstract SinkWriter createWriter( + MailboxExecutor mailboxExecutor, SinkWriterMetricGroup metricGroup); +} diff --git a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java new file mode 100644 index 0000000000..8f6492bd16 --- /dev/null +++ b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java @@ -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.fluss.flink.adapter; + +import org.apache.flink.table.api.Schema; + +import java.util.List; + +/** + * An adapter for the schema with Index. + * + *

TODO: remove this class when no longer support all the Flink 1.x series. + */ +public class SchemaAdapter { + private SchemaAdapter() {} + + public static Schema withIndex(Schema unresolvedSchema, List> indexes) { + Schema.Builder newSchemaBuilder = Schema.newBuilder().fromSchema(unresolvedSchema); + for (List index : indexes) { + newSchemaBuilder.index(index); + } + return newSchemaBuilder.build(); + } + + public static boolean supportIndex() { + return true; + } +} diff --git a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/catalog/Flink21Catalog.java b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/catalog/Flink21Catalog.java deleted file mode 100644 index d14e3af6fb..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/catalog/Flink21Catalog.java +++ /dev/null @@ -1,136 +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.fluss.flink.catalog; - -import org.apache.fluss.flink.lake.LakeFlinkCatalog; -import org.apache.fluss.metadata.TableInfo; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.Supplier; - -/** A {@link FlinkCatalog} used for Flink 2.1. */ -public class Flink21Catalog extends FlinkCatalog { - - public Flink21Catalog( - String name, - String defaultDatabase, - String bootstrapServers, - ClassLoader classLoader, - Map securityConfigs, - Supplier> lakeCatalogPropertiesSupplier) { - super( - name, - defaultDatabase, - bootstrapServers, - classLoader, - securityConfigs, - lakeCatalogPropertiesSupplier); - } - - @VisibleForTesting - public Flink21Catalog( - String name, - String defaultDatabase, - String bootstrapServers, - ClassLoader classLoader, - Map securityConfigs, - Supplier> lakeCatalogPropertiesSupplier, - LakeFlinkCatalog lakeFlinkCatalog) { - super( - name, - defaultDatabase, - bootstrapServers, - classLoader, - securityConfigs, - lakeCatalogPropertiesSupplier, - lakeFlinkCatalog); - } - - @Override - public CatalogBaseTable getTable(ObjectPath objectPath) - throws TableNotExistException, CatalogException { - CatalogBaseTable catalogBaseTable = super.getTable(objectPath); - if (!(catalogBaseTable instanceof CatalogTable) - || objectPath.getObjectName().contains(LAKE_TABLE_SPLITTER)) { - return catalogBaseTable; - } - - CatalogTable table = (CatalogTable) catalogBaseTable; - Optional pkOp = table.getUnresolvedSchema().getPrimaryKey(); - // If there is no pk, return directly. - if (pkOp.isEmpty()) { - return table; - } - - Schema.Builder newSchemaBuilder = - Schema.newBuilder().fromSchema(table.getUnresolvedSchema()); - // Pk is always an index. - newSchemaBuilder.index(pkOp.get().getColumnNames()); - - // Judge whether we can do prefix lookup. - TableInfo tableInfo = connection.getTable(toTablePath(objectPath)).getTableInfo(); - List bucketKeys = tableInfo.getBucketKeys(); - // For partition table, the physical primary key is the primary key that excludes the - // partition key - List physicalPrimaryKeys = tableInfo.getPhysicalPrimaryKeys(); - List indexKeys = new ArrayList<>(); - if (isPrefixList(physicalPrimaryKeys, bucketKeys)) { - indexKeys.addAll(bucketKeys); - if (tableInfo.isPartitioned()) { - indexKeys.addAll(tableInfo.getPartitionKeys()); - } - } - - if (!indexKeys.isEmpty()) { - newSchemaBuilder.index(indexKeys); - } - return CatalogTable.newBuilder() - .schema(newSchemaBuilder.build()) - .comment(table.getComment()) - .partitionKeys(table.getPartitionKeys()) - .options(table.getOptions()) - .snapshot(table.getSnapshot().orElse(null)) - .distribution(table.getDistribution().orElse(null)) - .build(); - } - - private static boolean isPrefixList(List fullList, List prefixList) { - if (fullList.size() <= prefixList.size()) { - return false; - } - - for (int i = 0; i < prefixList.size(); i++) { - if (!fullList.get(i).equals(prefixList.get(i))) { - return false; - } - } - return true; - } -} diff --git a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/catalog/Flink21CatalogFactory.java b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/catalog/Flink21CatalogFactory.java deleted file mode 100644 index cff44ab866..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/catalog/Flink21CatalogFactory.java +++ /dev/null @@ -1,35 +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.fluss.flink.catalog; - -/** A {@link FlinkCatalogFactory} used for Flink 2.1. */ -public class Flink21CatalogFactory extends FlinkCatalogFactory { - - @Override - public FlinkCatalog createCatalog(Context context) { - FlinkCatalog catalog = super.createCatalog(context); - return new Flink21Catalog( - catalog.catalogName, - catalog.defaultDatabase, - catalog.bootstrapServers, - catalog.classLoader, - catalog.securityConfigs, - catalog.lakeCatalogPropertiesSupplier); - } -} diff --git a/fluss-flink/fluss-flink-2.1/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/fluss-flink/fluss-flink-2.1/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index f13f71331e..6544cb5347 100644 --- a/fluss-flink/fluss-flink-2.1/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/fluss-flink/fluss-flink-2.1/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -16,4 +16,4 @@ # limitations under the License. # -org.apache.fluss.flink.catalog.Flink21CatalogFactory \ No newline at end of file +org.apache.fluss.flink.catalog.FlinkCatalogFactory \ No newline at end of file diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21CatalogITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21CatalogITCase.java index 62bf5b9aa0..bfccfa2d06 100644 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21CatalogITCase.java +++ b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21CatalogITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ObjectPath; -import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -29,24 +28,6 @@ /** IT case for catalog in Flink 2.1. */ public class Flink21CatalogITCase extends FlinkCatalogITCase { - @BeforeAll - static void beforeAll() { - FlinkCatalogITCase.beforeAll(); - - // close the old one and open a new one later - catalog.close(); - - catalog = - new Flink21Catalog( - catalog.catalogName, - catalog.defaultDatabase, - catalog.bootstrapServers, - catalog.classLoader, - catalog.securityConfigs, - catalog.lakeCatalogPropertiesSupplier); - catalog.open(); - } - @Test void testGetTableWithIndex() throws Exception { String tableName = "table_with_pk_only"; diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog21Test.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog21Test.java index d6aa6ef564..0c89ded816 100644 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog21Test.java +++ b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog21Test.java @@ -17,8 +17,6 @@ package org.apache.fluss.flink.catalog; -import org.apache.fluss.flink.lake.LakeFlinkCatalog; - import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.DefaultIndex; @@ -28,25 +26,9 @@ import java.util.Arrays; import java.util.Collections; -/** Test for {@link Flink21Catalog}. */ +/** Test for {@link FlinkCatalog}. */ public class FlinkCatalog21Test extends FlinkCatalogTest { - @Override - protected FlinkCatalog initCatalog( - String catalogName, - String databaseName, - String bootstrapServers, - LakeFlinkCatalog lakeFlinkCatalog) { - return new Flink21Catalog( - catalogName, - databaseName, - bootstrapServers, - Thread.currentThread().getContextClassLoader(), - Collections.emptyMap(), - Collections::emptyMap, - lakeFlinkCatalog); - } - protected ResolvedSchema createSchema() { return new ResolvedSchema( Arrays.asList( diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java new file mode 100644 index 0000000000..bb518ce861 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java @@ -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.fluss.flink.adapter; + +import org.apache.flink.api.common.operators.MailboxExecutor; +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 org.apache.flink.metrics.groups.SinkWriterMetricGroup; + +import java.io.IOException; + +/** + * Flink sink adapter which hide the different version of createWriter method. + * + *

TODO: remove this class when no longer support all the Flink 1.x series. + */ +public abstract class FlinkSinkAdapter implements Sink { + + @Override + public SinkWriter createWriter(WriterInitContext writerInitContext) throws IOException { + return createWriter( + writerInitContext.getMailboxExecutor(), writerInitContext.metricGroup()); + } + + protected abstract SinkWriter createWriter( + MailboxExecutor mailboxExecutor, SinkWriterMetricGroup metricGroup); +} diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/MultipleParameterToolAdapter.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/MultipleParameterToolAdapter.java new file mode 100644 index 0000000000..076dcb86c8 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/MultipleParameterToolAdapter.java @@ -0,0 +1,46 @@ +/* + * 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.fluss.flink.adapter; + +import org.apache.flink.util.MultipleParameterTool; + +import java.util.Map; + +/** + * An adapter for Flink {@link MultipleParameterTool} class. The {@link MultipleParameterTool} is + * moved to a new package since Flink 2.x, so this adapter helps to bridge compatibility for + * different Flink versions. + * + *

TODO: remove this class when no longer support all the Flink 1.x series. + */ +public class MultipleParameterToolAdapter { + + private MultipleParameterToolAdapter() {} + + private MultipleParameterTool multipleParameterTool; + + public static MultipleParameterToolAdapter fromArgs(String[] args) { + MultipleParameterToolAdapter adapter = new MultipleParameterToolAdapter(); + adapter.multipleParameterTool = MultipleParameterTool.fromArgs(args); + return adapter; + } + + public Map toMap() { + return this.multipleParameterTool.toMap(); + } +} diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java new file mode 100644 index 0000000000..8f6492bd16 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java @@ -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.fluss.flink.adapter; + +import org.apache.flink.table.api.Schema; + +import java.util.List; + +/** + * An adapter for the schema with Index. + * + *

TODO: remove this class when no longer support all the Flink 1.x series. + */ +public class SchemaAdapter { + private SchemaAdapter() {} + + public static Schema withIndex(Schema unresolvedSchema, List> indexes) { + Schema.Builder newSchemaBuilder = Schema.newBuilder().fromSchema(unresolvedSchema); + for (List index : indexes) { + newSchemaBuilder.index(index); + } + return newSchemaBuilder.build(); + } + + public static boolean supportIndex() { + return true; + } +} diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java deleted file mode 100644 index d10e38628f..0000000000 --- a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22Catalog.java +++ /dev/null @@ -1,137 +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.fluss.flink.catalog; - -import org.apache.fluss.annotation.VisibleForTesting; -import org.apache.fluss.flink.lake.LakeFlinkCatalog; -import org.apache.fluss.metadata.TableInfo; - -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.Supplier; - -/** A {@link FlinkCatalog} used for Flink 2.2. */ -public class Flink22Catalog extends FlinkCatalog { - - public Flink22Catalog( - String name, - String defaultDatabase, - String bootstrapServers, - ClassLoader classLoader, - Map securityConfigs, - Supplier> lakeCatalogPropertiesSupplier) { - super( - name, - defaultDatabase, - bootstrapServers, - classLoader, - securityConfigs, - lakeCatalogPropertiesSupplier); - } - - @VisibleForTesting - public Flink22Catalog( - String name, - String defaultDatabase, - String bootstrapServers, - ClassLoader classLoader, - Map securityConfigs, - Supplier> lakeCatalogPropertiesSupplier, - LakeFlinkCatalog lakeFlinkCatalog) { - super( - name, - defaultDatabase, - bootstrapServers, - classLoader, - securityConfigs, - lakeCatalogPropertiesSupplier, - lakeFlinkCatalog); - } - - @Override - public CatalogBaseTable getTable(ObjectPath objectPath) - throws TableNotExistException, CatalogException { - CatalogBaseTable catalogBaseTable = super.getTable(objectPath); - if (!(catalogBaseTable instanceof CatalogTable) - || objectPath.getObjectName().contains(LAKE_TABLE_SPLITTER)) { - return catalogBaseTable; - } - - CatalogTable table = (CatalogTable) catalogBaseTable; - Optional pkOp = table.getUnresolvedSchema().getPrimaryKey(); - // If there is no pk, return directly. - if (pkOp.isEmpty()) { - return table; - } - - Schema.Builder newSchemaBuilder = - Schema.newBuilder().fromSchema(table.getUnresolvedSchema()); - // Pk is always an index. - newSchemaBuilder.index(pkOp.get().getColumnNames()); - - // Judge whether we can do prefix lookup. - TableInfo tableInfo = connection.getTable(toTablePath(objectPath)).getTableInfo(); - List bucketKeys = tableInfo.getBucketKeys(); - // For partition table, the physical primary key is the primary key that - // excludes the - // partition key - List physicalPrimaryKeys = tableInfo.getPhysicalPrimaryKeys(); - List indexKeys = new ArrayList<>(); - if (isPrefixList(physicalPrimaryKeys, bucketKeys)) { - indexKeys.addAll(bucketKeys); - if (tableInfo.isPartitioned()) { - indexKeys.addAll(tableInfo.getPartitionKeys()); - } - } - - if (!indexKeys.isEmpty()) { - newSchemaBuilder.index(indexKeys); - } - return CatalogTable.newBuilder() - .schema(newSchemaBuilder.build()) - .comment(table.getComment()) - .partitionKeys(table.getPartitionKeys()) - .options(table.getOptions()) - .snapshot(table.getSnapshot().orElse(null)) - .distribution(table.getDistribution().orElse(null)) - .build(); - } - - private static boolean isPrefixList(List fullList, List prefixList) { - if (fullList.size() <= prefixList.size()) { - return false; - } - - for (int i = 0; i < prefixList.size(); i++) { - if (!fullList.get(i).equals(prefixList.get(i))) { - return false; - } - } - return true; - } -} diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java deleted file mode 100644 index 2d930c164d..0000000000 --- a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/catalog/Flink22CatalogFactory.java +++ /dev/null @@ -1,35 +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.fluss.flink.catalog; - -/** A {@link FlinkCatalogFactory} used for Flink 2.2. */ -public class Flink22CatalogFactory extends FlinkCatalogFactory { - - @Override - public FlinkCatalog createCatalog(Context context) { - FlinkCatalog catalog = super.createCatalog(context); - return new Flink22Catalog( - catalog.catalogName, - catalog.defaultDatabase, - catalog.bootstrapServers, - catalog.classLoader, - catalog.securityConfigs, - catalog.lakeCatalogPropertiesSupplier); - } -} diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java deleted file mode 100644 index 39ef016f04..0000000000 --- a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java +++ /dev/null @@ -1,213 +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.fluss.flink.sink; - -import org.apache.fluss.annotation.Internal; -import org.apache.fluss.config.Configuration; -import org.apache.fluss.flink.sink.serializer.FlussSerializationSchema; -import org.apache.fluss.flink.sink.writer.AppendSinkWriter; -import org.apache.fluss.flink.sink.writer.FlinkSinkWriter; -import org.apache.fluss.flink.sink.writer.UpsertSinkWriter; -import org.apache.fluss.metadata.DataLakeFormat; -import org.apache.fluss.metadata.TablePath; - -import org.apache.flink.api.common.operators.MailboxExecutor; -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 org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; -import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.table.types.logical.RowType; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.io.Serializable; -import java.util.List; - -import static org.apache.fluss.flink.sink.FlinkStreamPartitioner.partition; -import static org.apache.fluss.flink.utils.FlinkConversions.toFlussRowType; - -/** Flink sink for Fluss. */ -class FlinkSink implements Sink, SupportsPreWriteTopology { - - private static final long serialVersionUID = 1L; - - private final SinkWriterBuilder builder; - - FlinkSink(SinkWriterBuilder builder) { - this.builder = builder; - } - - @Override - public SinkWriter createWriter(WriterInitContext context) throws IOException { - FlinkSinkWriter flinkSinkWriter = - builder.createWriter(context.getMailboxExecutor()); - flinkSinkWriter.initialize(InternalSinkWriterMetricGroup.wrap(context.metricGroup())); - return flinkSinkWriter; - } - - @Override - public DataStream addPreWriteTopology(DataStream input) { - return builder.addPreWriteTopology(input); - } - - @Internal - interface SinkWriterBuilder, InputT> extends Serializable { - W createWriter(MailboxExecutor mailboxExecutor); - - DataStream addPreWriteTopology(DataStream input); - } - - @Internal - static class AppendSinkWriterBuilder - implements SinkWriterBuilder, InputT> { - - private static final long serialVersionUID = 1L; - - private final TablePath tablePath; - private final Configuration flussConfig; - private final RowType tableRowType; - private final int numBucket; - private final List bucketKeys; - private final List partitionKeys; - private final @Nullable DataLakeFormat lakeFormat; - private final boolean shuffleByBucketId; - private final FlussSerializationSchema flussSerializationSchema; - - public AppendSinkWriterBuilder( - TablePath tablePath, - Configuration flussConfig, - RowType tableRowType, - int numBucket, - List bucketKeys, - List partitionKeys, - @Nullable DataLakeFormat lakeFormat, - boolean shuffleByBucketId, - FlussSerializationSchema flussSerializationSchema) { - this.tablePath = tablePath; - this.flussConfig = flussConfig; - this.tableRowType = tableRowType; - this.numBucket = numBucket; - this.bucketKeys = bucketKeys; - this.partitionKeys = partitionKeys; - this.lakeFormat = lakeFormat; - this.shuffleByBucketId = shuffleByBucketId; - this.flussSerializationSchema = flussSerializationSchema; - } - - @Override - public AppendSinkWriter createWriter(MailboxExecutor mailboxExecutor) { - return new AppendSinkWriter<>( - tablePath, - flussConfig, - tableRowType, - mailboxExecutor, - flussSerializationSchema); - } - - @Override - public DataStream addPreWriteTopology(DataStream input) { - // For append only sink, we will do bucket shuffle only if bucket keys are not - // empty. - if (!bucketKeys.isEmpty() && shuffleByBucketId) { - return partition( - input, - new FlinkRowDataChannelComputer<>( - toFlussRowType(tableRowType), - bucketKeys, - partitionKeys, - lakeFormat, - numBucket, - flussSerializationSchema), - input.getParallelism()); - } else { - return input; - } - } - } - - @Internal - static class UpsertSinkWriterBuilder - implements SinkWriterBuilder, InputT> { - - private static final long serialVersionUID = 1L; - - private final TablePath tablePath; - private final Configuration flussConfig; - private final RowType tableRowType; - private final @Nullable int[] targetColumnIndexes; - private final int numBucket; - private final List bucketKeys; - private final List partitionKeys; - private final @Nullable DataLakeFormat lakeFormat; - private final boolean shuffleByBucketId; - private final FlussSerializationSchema flussSerializationSchema; - - UpsertSinkWriterBuilder( - TablePath tablePath, - Configuration flussConfig, - RowType tableRowType, - @Nullable int[] targetColumnIndexes, - int numBucket, - List bucketKeys, - List partitionKeys, - @Nullable DataLakeFormat lakeFormat, - boolean shuffleByBucketId, - FlussSerializationSchema flussSerializationSchema) { - this.tablePath = tablePath; - this.flussConfig = flussConfig; - this.tableRowType = tableRowType; - this.targetColumnIndexes = targetColumnIndexes; - this.numBucket = numBucket; - this.bucketKeys = bucketKeys; - this.partitionKeys = partitionKeys; - this.lakeFormat = lakeFormat; - this.shuffleByBucketId = shuffleByBucketId; - this.flussSerializationSchema = flussSerializationSchema; - } - - @Override - public UpsertSinkWriter createWriter(MailboxExecutor mailboxExecutor) { - return new UpsertSinkWriter<>( - tablePath, - flussConfig, - tableRowType, - targetColumnIndexes, - mailboxExecutor, - flussSerializationSchema); - } - - @Override - public DataStream addPreWriteTopology(DataStream input) { - return shuffleByBucketId - ? partition( - input, - new FlinkRowDataChannelComputer<>( - toFlussRowType(tableRowType), - bucketKeys, - partitionKeys, - lakeFormat, - numBucket, - flussSerializationSchema), - input.getParallelism()) - : input; - } - } -} diff --git a/fluss-flink/fluss-flink-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/fluss-flink/fluss-flink-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 369e7d0877..d5aca2d53b 100644 --- a/fluss-flink/fluss-flink-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/fluss-flink/fluss-flink-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -16,4 +16,4 @@ # limitations under the License. # -org.apache.fluss.flink.catalog.Flink22CatalogFactory +org.apache.fluss.flink.catalog.FlinkCatalogFactory diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java index e51a38f29f..b87965727a 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ObjectPath; -import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -29,24 +28,6 @@ /** IT case for catalog in Flink 2.2. */ public class Flink22CatalogITCase extends FlinkCatalogITCase { - @BeforeAll - static void beforeAll() { - FlinkCatalogITCase.beforeAll(); - - // close the old one and open a new one later - catalog.close(); - - catalog = - new Flink22Catalog( - catalog.catalogName, - catalog.defaultDatabase, - catalog.bootstrapServers, - catalog.classLoader, - catalog.securityConfigs, - catalog.lakeCatalogPropertiesSupplier); - catalog.open(); - } - @Test void testGetTableWithIndex() throws Exception { String tableName = "table_with_pk_only"; diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java index be59dfe59e..e742bfb037 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java @@ -17,36 +17,23 @@ package org.apache.fluss.flink.catalog; -import org.apache.fluss.flink.lake.LakeFlinkCatalog; - import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogMaterializedTable; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.DefaultIndex; +import org.apache.flink.table.catalog.IntervalFreshness; +import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; import java.util.Arrays; import java.util.Collections; +import java.util.Map; -/** Test for {@link Flink22Catalog}. */ +/** Test for {@link FlinkCatalog}. */ public class FlinkCatalog22Test extends FlinkCatalogTest { - @Override - protected FlinkCatalog initCatalog( - String catalogName, - String databaseName, - String bootstrapServers, - LakeFlinkCatalog lakeFlinkCatalog) { - return new Flink22Catalog( - catalogName, - databaseName, - bootstrapServers, - Thread.currentThread().getContextClassLoader(), - Collections.emptyMap(), - Collections::emptyMap, - lakeFlinkCatalog); - } - protected ResolvedSchema createSchema() { return new ResolvedSchema( Arrays.asList( @@ -60,45 +47,29 @@ protected ResolvedSchema createSchema() { "INDEX_first_third", Arrays.asList("first", "third")))); } - @Override - protected org.apache.flink.table.catalog.CatalogMaterializedTable newCatalogMaterializedTable( + protected CatalogMaterializedTable newCatalogMaterializedTable( ResolvedSchema resolvedSchema, - org.apache.flink.table.catalog.CatalogMaterializedTable.RefreshMode refreshMode, - java.util.Map options) { - org.apache.flink.table.catalog.CatalogMaterializedTable origin = - org.apache.flink.table.catalog.CatalogMaterializedTable.newBuilder() - .schema( - org.apache.flink.table.api.Schema.newBuilder() - .fromResolvedSchema(resolvedSchema) - .build()) + CatalogMaterializedTable.RefreshMode refreshMode, + Map options) { + CatalogMaterializedTable origin = + CatalogMaterializedTable.newBuilder() + .schema(Schema.newBuilder().fromResolvedSchema(resolvedSchema).build()) .comment("test comment") .options(options) .partitionKeys(Collections.emptyList()) .definitionQuery("select first, second, third from t") - .freshness( - org.apache.flink.table.catalog.IntervalFreshness.of( - "5", - org.apache.flink.table.catalog.IntervalFreshness.TimeUnit - .SECOND)) + .freshness(IntervalFreshness.of("5", IntervalFreshness.TimeUnit.SECOND)) .logicalRefreshMode( - refreshMode - == org.apache.flink.table.catalog - .CatalogMaterializedTable.RefreshMode - .CONTINUOUS - ? org.apache.flink.table.catalog.CatalogMaterializedTable - .LogicalRefreshMode.CONTINUOUS - : org.apache.flink.table.catalog.CatalogMaterializedTable - .LogicalRefreshMode.FULL) + refreshMode == CatalogMaterializedTable.RefreshMode.CONTINUOUS + ? CatalogMaterializedTable.LogicalRefreshMode.CONTINUOUS + : CatalogMaterializedTable.LogicalRefreshMode.FULL) .refreshMode(refreshMode) - .refreshStatus( - org.apache.flink.table.catalog.CatalogMaterializedTable - .RefreshStatus.INITIALIZING) + .refreshStatus(CatalogMaterializedTable.RefreshStatus.INITIALIZING) .build(); - return new org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable( + return new ResolvedCatalogMaterializedTable( origin, resolvedSchema, refreshMode, - org.apache.flink.table.catalog.IntervalFreshness.of( - "5", org.apache.flink.table.catalog.IntervalFreshness.TimeUnit.SECOND)); + IntervalFreshness.of("5", IntervalFreshness.TimeUnit.SECOND)); } } diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java index f65966ea6f..0f667ff81a 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java @@ -17,5 +17,62 @@ package org.apache.fluss.flink.source; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.Collections; +import java.util.List; + +import static org.apache.fluss.flink.source.testutils.FlinkRowAssertionsUtils.collectRowsWithTimeout; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + /** IT case for batch source in Flink 2.2. */ -public class Flink22TableSourceBatchITCase extends FlinkTableSourceBatchITCase {} +public class Flink22TableSourceBatchITCase extends FlinkTableSourceBatchITCase { + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + @Override + void testCountPushDown(boolean partitionTable) throws Exception { + String tableName = partitionTable ? preparePartitionedLogTable() : prepareLogTable(); + int expectedRows = partitionTable ? 10 : 5; + // normal scan + String query = String.format("SELECT COUNT(*) FROM %s", tableName); + assertThat(tEnv.explainSql(query)) + .contains( + String.format( + "TableSourceScan(table=[[testcatalog, defaultdb, %s, " + + "aggregates=[grouping=[], aggFunctions=[Count1AggFunction()]]]], " + + "fields=[count1$0])", + tableName)); + CloseableIterator iterRows = tEnv.executeSql(query).collect(); + List collected = collectRowsWithTimeout(iterRows, 1); + List expected = Collections.singletonList(String.format("+I[%s]", expectedRows)); + assertThat(collected).isEqualTo(expected); + + // test not push down grouping count. + assertThatThrownBy( + () -> + tEnv.explainSql( + String.format( + "SELECT COUNT(*) FROM %s group by id", + tableName)) + .wait()) + .hasMessageContaining( + "Currently, Fluss only support queries on table with datalake enabled or point queries on primary key when it's in batch execution mode."); + + // test not support primary key now + String primaryTableName = prepareSourceTable(new String[] {"id"}, null); + assertThatThrownBy( + () -> + tEnv.explainSql( + String.format( + "SELECT COUNT(*) FROM %s ", + primaryTableName)) + .wait()) + .hasMessageContaining( + "Currently, Fluss only support queries on table with datalake enabled or point queries on primary key when it's in batch execution mode."); + } +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/fluss-flink/fluss-flink-2.2/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension new file mode 100644 index 0000000000..ca0e907f6d --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension @@ -0,0 +1,19 @@ +# +# 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.fluss.testutils.common.TestLoggerExtension \ No newline at end of file diff --git a/fluss-flink/fluss-flink-2.2/src/test/resources/log4j2-test.properties b/fluss-flink/fluss-flink-2.2/src/test/resources/log4j2-test.properties new file mode 100644 index 0000000000..12b05f1867 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/resources/log4j2-test.properties @@ -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. +# + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n + +# suppress the duplicated logger extension +logger.flink.name = org.apache.flink.util.TestLoggerExtension +logger.flink.level = OFF diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java new file mode 100644 index 0000000000..ba869dbcdb --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java @@ -0,0 +1,48 @@ +/* + * 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.fluss.flink.adapter; + +import org.apache.flink.api.common.operators.MailboxExecutor; +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 org.apache.flink.metrics.groups.SinkWriterMetricGroup; + +import java.io.IOException; + +/** + * Flink sink adapter which hide the different version of createWriter method. + * + *

TODO: remove this class when no longer support all the Flink 1.x series. + */ +public abstract class FlinkSinkAdapter implements Sink { + + @Override + public SinkWriter createWriter(InitContext initContext) throws IOException { + return createWriter(initContext.getMailboxExecutor(), initContext.metricGroup()); + } + + @Override + public SinkWriter createWriter(WriterInitContext writerInitContext) throws IOException { + return createWriter( + writerInitContext.getMailboxExecutor(), writerInitContext.metricGroup()); + } + + protected abstract SinkWriter createWriter( + MailboxExecutor mailboxExecutor, SinkWriterMetricGroup metricGroup); +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java new file mode 100644 index 0000000000..ad48cc99e7 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java @@ -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.fluss.flink.adapter; + +import org.apache.flink.table.api.Schema; + +import java.util.List; + +/** + * An adapter for the schema with Index. + * + *

TODO: remove this class when no longer support all the Flink 1.x series. + */ +public class SchemaAdapter { + private SchemaAdapter() {} + + public static Schema withIndex(Schema unresolvedSchema, List> indexes) { + Schema.Builder newSchemaBuilder = Schema.newBuilder().fromSchema(unresolvedSchema); + if (!indexes.isEmpty()) { + throw new UnsupportedOperationException("Index is not supported."); + } + return newSchemaBuilder.build(); + } + + public static boolean supportIndex() { + return false; + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SingleThreadMultiplexSourceReaderBaseAdapter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SingleThreadMultiplexSourceReaderBaseAdapter.java index 72a4540eab..d5a4ff0aa5 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SingleThreadMultiplexSourceReaderBaseAdapter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SingleThreadMultiplexSourceReaderBaseAdapter.java @@ -29,6 +29,8 @@ /** * Adapter for {@link SingleThreadMultiplexSourceReaderBase}.TODO: remove it until not supported in * flink 1.18. + * + *

TODO: remove this class when no longer support all the Flink 1.x series. */ public abstract class SingleThreadMultiplexSourceReaderBaseAdapter< E, T, SplitT extends SourceSplit, SplitStateT> diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkCatalog.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkCatalog.java index fb7a41157c..e2556c913b 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkCatalog.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkCatalog.java @@ -38,6 +38,7 @@ import org.apache.fluss.utils.IOUtils; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.AbstractCatalog; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabase; @@ -86,6 +87,8 @@ import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.fluss.config.ConfigOptions.BOOTSTRAP_SERVERS; import static org.apache.fluss.flink.FlinkConnectorOptions.ALTER_DISALLOW_OPTIONS; +import static org.apache.fluss.flink.adapter.SchemaAdapter.supportIndex; +import static org.apache.fluss.flink.adapter.SchemaAdapter.withIndex; import static org.apache.fluss.flink.utils.CatalogExceptionUtils.isPartitionAlreadyExists; import static org.apache.fluss.flink.utils.CatalogExceptionUtils.isPartitionInvalid; import static org.apache.fluss.flink.utils.CatalogExceptionUtils.isPartitionNotExist; @@ -344,7 +347,11 @@ public CatalogBaseTable getTable(ObjectPath objectPath) } } if (CatalogBaseTable.TableKind.TABLE == catalogBaseTable.getTableKind()) { - return ((CatalogTable) catalogBaseTable).copy(newOptions); + CatalogTable table = ((CatalogTable) catalogBaseTable).copy(newOptions); + if (supportIndex()) { + table = wrapWithIndexes(table, tableInfo); + } + return table; } else if (CatalogBaseTable.TableKind.MATERIALIZED_TABLE == catalogBaseTable.getTableKind()) { return ((CatalogMaterializedTable) catalogBaseTable).copy(newOptions); @@ -806,4 +813,56 @@ public Map getLakeCatalogProperties() { } return lakeCatalogProperties; } + + private CatalogTable wrapWithIndexes(CatalogTable table, TableInfo tableInfo) { + + Optional pkOp = table.getUnresolvedSchema().getPrimaryKey(); + // If there is no pk, return directly. + if (!pkOp.isPresent()) { + return table; + } + + List> indexes = new ArrayList<>(); + // Pk is always an index. + indexes.add(pkOp.get().getColumnNames()); + + // Judge whether we can do prefix lookup. + List bucketKeys = tableInfo.getBucketKeys(); + // For partition table, the physical primary key is the primary key that + // excludes the + // partition key + List physicalPrimaryKeys = tableInfo.getPhysicalPrimaryKeys(); + List indexKeys = new ArrayList<>(); + if (isPrefixList(physicalPrimaryKeys, bucketKeys)) { + indexKeys.addAll(bucketKeys); + if (tableInfo.isPartitioned()) { + indexKeys.addAll(tableInfo.getPartitionKeys()); + } + } + + if (!indexKeys.isEmpty()) { + indexes.add(indexKeys); + } + return CatalogTable.newBuilder() + .schema(withIndex(table.getUnresolvedSchema(), indexes)) + .comment(table.getComment()) + .partitionKeys(table.getPartitionKeys()) + .options(table.getOptions()) + .snapshot(table.getSnapshot().orElse(null)) + .distribution(table.getDistribution().orElse(null)) + .build(); + } + + private static boolean isPrefixList(List fullList, List prefixList) { + if (fullList.size() <= prefixList.size()) { + return false; + } + + for (int i = 0; i < prefixList.size(); i++) { + if (!fullList.get(i).equals(prefixList.get(i))) { + return false; + } + } + return true; + } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java index 8f6c277b3c..8ce3f6a46a 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java @@ -19,6 +19,7 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.config.Configuration; +import org.apache.fluss.flink.adapter.FlinkSinkAdapter; import org.apache.fluss.flink.sink.serializer.FlussSerializationSchema; import org.apache.fluss.flink.sink.writer.AppendSinkWriter; import org.apache.fluss.flink.sink.writer.FlinkSinkWriter; @@ -27,9 +28,8 @@ import org.apache.fluss.metadata.TablePath; import org.apache.flink.api.common.operators.MailboxExecutor; -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 org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; import org.apache.flink.streaming.api.datastream.DataStream; @@ -37,7 +37,6 @@ import javax.annotation.Nullable; -import java.io.IOException; import java.io.Serializable; import java.util.List; @@ -45,7 +44,8 @@ import static org.apache.fluss.flink.utils.FlinkConversions.toFlussRowType; /** Flink sink for Fluss. */ -class FlinkSink implements Sink, SupportsPreWriteTopology { +class FlinkSink extends FlinkSinkAdapter + implements SupportsPreWriteTopology { private static final long serialVersionUID = 1L; @@ -55,20 +55,11 @@ class FlinkSink implements Sink, SupportsPreWriteTopology createWriter(InitContext context) throws IOException { - FlinkSinkWriter flinkSinkWriter = - builder.createWriter(context.getMailboxExecutor()); - flinkSinkWriter.initialize(InternalSinkWriterMetricGroup.wrap(context.metricGroup())); - return flinkSinkWriter; - } - - @Override - public SinkWriter createWriter(WriterInitContext context) throws IOException { - FlinkSinkWriter flinkSinkWriter = - builder.createWriter(context.getMailboxExecutor()); - flinkSinkWriter.initialize(InternalSinkWriterMetricGroup.wrap(context.metricGroup())); + protected SinkWriter createWriter( + MailboxExecutor mailboxExecutor, SinkWriterMetricGroup metricGroup) { + FlinkSinkWriter flinkSinkWriter = builder.createWriter(mailboxExecutor); + flinkSinkWriter.initialize(InternalSinkWriterMetricGroup.wrap(metricGroup)); return flinkSinkWriter; } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java index 41176f3884..541d9ce636 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java @@ -306,8 +306,7 @@ void testCreateTable() throws Exception { assertThatThrownBy(() -> catalog.renameTable(this.tableInDefaultDb, "newName", false)) .isInstanceOf(UnsupportedOperationException.class); - // Test lake table handling - should throw TableNotExistException for - // non-existent lake + // Test lake table handling - should throw TableNotExistException for non-existent lake // table ObjectPath lakePath = new ObjectPath(DEFAULT_DB, "regularTable$lake"); assertThatThrownBy(() -> catalog.getTable(lakePath)) @@ -549,8 +548,7 @@ void testCreateUnsupportedMaterializedTable() { this.createSchema(), CatalogMaterializedTable.RefreshMode.FULL, Collections.emptyMap()); - // Fluss doesn't support insert overwrite in batch mode now, so full refresh - // mode is not + // Fluss doesn't support insert overwrite in batch mode now, so full refresh mode is not // supported now. assertThatThrownBy( () -> @@ -618,8 +616,7 @@ void testDatabase() throws Exception { .hasMessage("Database %s in catalog %s is not empty.", "db1", CATALOG_NAME); // should be ok since we set cascade = true catalog.dropDatabase("db1", false, true); - // drop it again, should throw exception since db1 is not exist and we set - // ignoreIfNotExists + // drop it again, should throw exception since db1 is not exist and we set ignoreIfNotExists // = false assertThatThrownBy(() -> catalog.dropDatabase("db1", false, true)) .isInstanceOf(DatabaseNotExistException.class) @@ -631,8 +628,7 @@ void testDatabase() throws Exception { catalog.dropDatabase("db2", false, true); // should be empty assertThat(catalog.listDatabases()).isEqualTo(Collections.singletonList(DEFAULT_DB)); - // should throw exception since the db is not exist and we set ignoreIfNotExists - // = false + // should throw exception since the db is not exist and we set ignoreIfNotExists = false assertThatThrownBy(() -> catalog.listTables("unknown")) .isInstanceOf(DatabaseNotExistException.class) .hasMessage("Database %s does not exist in Catalog %s.", "unknown", CATALOG_NAME); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java index 7f3894f965..6aea7a1b83 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java @@ -315,7 +315,7 @@ void testCountPushDown(boolean partitionTable) throws Exception { "Currently, Fluss only support queries on table with datalake enabled or point queries on primary key when it's in batch execution mode."); } - private String prepareSourceTable(String[] keys, String partitionedKey) throws Exception { + protected String prepareSourceTable(String[] keys, String partitionedKey) throws Exception { String tableName = String.format("test_%s_%s", String.join("_", keys), RandomUtils.nextInt()); if (partitionedKey == null) { @@ -371,7 +371,7 @@ private String prepareSourceTable(String[] keys, String partitionedKey) throws E return tableName; } - private String prepareLogTable() throws Exception { + protected String prepareLogTable() throws Exception { String tableName = String.format("test_log_table_%s", RandomUtils.nextInt()); tEnv.executeSql( String.format( @@ -401,7 +401,7 @@ private String prepareLogTable() throws Exception { return tableName; } - private String preparePartitionedLogTable() throws Exception { + protected String preparePartitionedLogTable() throws Exception { String tableName = String.format("test_partitioned_log_table_%s", RandomUtils.nextInt()); tEnv.executeSql( String.format( diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index f9e24f24c2..46c34da57a 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -166,8 +166,6 @@ fluss-test-utils/** fluss-flink/**/target/classes/org/apache/fluss/flink/adapter/** - - fluss-flink/fluss-flink-2.2/target/classes/org/apache/fluss/flink/sink/FlinkSink* @@ -210,8 +208,6 @@ fluss-test-utils/** fluss-flink/**/target/classes/org/apache/fluss/flink/adapter/** - - fluss-flink/fluss-flink-2.2/target/classes/org/apache/fluss/flink/sink/FlinkSink* From e987ea1d3e1315a3fc4535a22abcf19bf31b6007 Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Tue, 23 Dec 2025 20:33:45 +0800 Subject: [PATCH 08/14] remove flink 2.1 --- .github/workflows/stage.sh | 1 - fluss-flink/fluss-flink-2.1/pom.xml | 254 ------------------ .../fluss/flink/adapter/FlinkSinkAdapter.java | 43 --- .../adapter/MultipleParameterToolAdapter.java | 46 ---- .../fluss/flink/adapter/SchemaAdapter.java | 43 --- .../org.apache.flink.table.factories.Factory | 19 -- .../Flink21MultipleParameterToolTest.java | 21 -- .../flink/catalog/Flink21CatalogITCase.java | 175 ------------ .../Flink21MaterializedTableITCase.java | 21 -- .../flink/catalog/FlinkCatalog21Test.java | 44 --- .../flink/metrics/Flink21MetricsITCase.java | 21 -- .../procedure/Flink21ProcedureITCase.java | 21 -- .../acl/Flink21AuthorizationITCase.java | 21 -- .../flink/sink/Flink21ComplexTypeITCase.java | 21 -- .../flink/sink/Flink21TableSinkITCase.java | 21 -- .../source/Flink21TableSourceBatchITCase.java | 21 -- .../Flink21TableSourceFailOverITCase.java | 21 -- .../source/Flink21TableSourceITCase.java | 147 ---------- .../org.junit.jupiter.api.extension.Extension | 19 -- .../src/test/resources/log4j2-test.properties | 32 --- fluss-flink/pom.xml | 1 - fluss-test-coverage/pom.xml | 6 - website/src/pages/downloads.md | 34 +-- 23 files changed, 17 insertions(+), 1036 deletions(-) delete mode 100644 fluss-flink/fluss-flink-2.1/pom.xml delete mode 100644 fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/MultipleParameterToolAdapter.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/adapter/Flink21MultipleParameterToolTest.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21CatalogITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21MaterializedTableITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog21Test.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/metrics/Flink21MetricsITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/procedure/Flink21ProcedureITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/security/acl/Flink21AuthorizationITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/sink/Flink21ComplexTypeITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/sink/Flink21TableSinkITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceBatchITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceFailOverITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceITCase.java delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension delete mode 100644 fluss-flink/fluss-flink-2.1/src/test/resources/log4j2-test.properties diff --git a/.github/workflows/stage.sh b/.github/workflows/stage.sh index 3f2b3f3215..18587dc292 100755 --- a/.github/workflows/stage.sh +++ b/.github/workflows/stage.sh @@ -24,7 +24,6 @@ STAGE_LAKE="lake" MODULES_FLINK="\ fluss-flink,\ fluss-flink/fluss-flink-common,\ -fluss-flink/fluss-flink-2.1,\ fluss-flink/fluss-flink-2.2,\ fluss-flink/fluss-flink-1.20,\ " diff --git a/fluss-flink/fluss-flink-2.1/pom.xml b/fluss-flink/fluss-flink-2.1/pom.xml deleted file mode 100644 index 5b5475f3df..0000000000 --- a/fluss-flink/fluss-flink-2.1/pom.xml +++ /dev/null @@ -1,254 +0,0 @@ - - - - 4.0.0 - - org.apache.fluss - fluss-flink - 0.9-SNAPSHOT - - - fluss-flink-2.1 - Fluss : Engine Flink : 2.1 - - 2.1 - 2.1.1 - - - - - - org.apache.fluss - fluss-client - ${project.version} - - - - org.apache.fluss - fluss-flink-common - ${project.version} - - - * - * - - - - - - - org.apache.flink - flink-core - ${flink.minor.version} - provided - - - - org.apache.flink - flink-table-common - ${flink.minor.version} - provided - - - - - org.apache.fluss - fluss-flink-common - ${project.version} - test - test-jar - - - - org.apache.fluss - fluss-server - ${project.version} - test - - - - org.apache.flink - flink-table-test-utils - ${flink.minor.version} - test - - - - org.apache.flink - flink-connector-base - ${flink.minor.version} - test - - - - org.apache.fluss - fluss-server - ${project.version} - test - test-jar - - - - org.apache.fluss - fluss-rpc - test - test-jar - ${project.version} - - - - - org.apache.curator - curator-test - ${curator.version} - test - - - - org.apache.flink - flink-table-common - ${flink.minor.version} - test - test-jar - - - - org.apache.flink - flink-connector-test-utils - ${flink.minor.version} - test - - - - - org.apache.fluss - fluss-test-utils - - - - org.apache.fluss - fluss-common - ${project.version} - test - test-jar - - - - org.apache.flink - flink-clients - ${flink.minor.version} - test - - - - org.apache.flink - flink-sql-gateway - ${flink.minor.version} - test - - - - org.apache.flink - flink-sql-gateway - ${flink.minor.version} - test-jar - test - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - ${skip.on.java8} - - ${skip.on.java8} - - - - - org.apache.maven.plugins - maven-surefire-plugin - 3.0.0-M5 - - - - integration-tests - integration-test - false - - test - - - ${skip.on.java8} - - **/*ITCase.* - - - 1 - - - - - default-test - test - false - - test - - - ${skip.on.java8} - - **/*ITCase.* - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-fluss - package - - shade - - - - - org.apache.fluss:fluss-flink-common - org.apache.fluss:fluss-client - - - - - - - - - - diff --git a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java deleted file mode 100644 index bb518ce861..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java +++ /dev/null @@ -1,43 +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.fluss.flink.adapter; - -import org.apache.flink.api.common.operators.MailboxExecutor; -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 org.apache.flink.metrics.groups.SinkWriterMetricGroup; - -import java.io.IOException; - -/** - * Flink sink adapter which hide the different version of createWriter method. - * - *

TODO: remove this class when no longer support all the Flink 1.x series. - */ -public abstract class FlinkSinkAdapter implements Sink { - - @Override - public SinkWriter createWriter(WriterInitContext writerInitContext) throws IOException { - return createWriter( - writerInitContext.getMailboxExecutor(), writerInitContext.metricGroup()); - } - - protected abstract SinkWriter createWriter( - MailboxExecutor mailboxExecutor, SinkWriterMetricGroup metricGroup); -} diff --git a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/MultipleParameterToolAdapter.java b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/MultipleParameterToolAdapter.java deleted file mode 100644 index 076dcb86c8..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/MultipleParameterToolAdapter.java +++ /dev/null @@ -1,46 +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.fluss.flink.adapter; - -import org.apache.flink.util.MultipleParameterTool; - -import java.util.Map; - -/** - * An adapter for Flink {@link MultipleParameterTool} class. The {@link MultipleParameterTool} is - * moved to a new package since Flink 2.x, so this adapter helps to bridge compatibility for - * different Flink versions. - * - *

TODO: remove this class when no longer support all the Flink 1.x series. - */ -public class MultipleParameterToolAdapter { - - private MultipleParameterToolAdapter() {} - - private MultipleParameterTool multipleParameterTool; - - public static MultipleParameterToolAdapter fromArgs(String[] args) { - MultipleParameterToolAdapter adapter = new MultipleParameterToolAdapter(); - adapter.multipleParameterTool = MultipleParameterTool.fromArgs(args); - return adapter; - } - - public Map toMap() { - return this.multipleParameterTool.toMap(); - } -} diff --git a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java b/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java deleted file mode 100644 index 8f6492bd16..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/main/java/org/apache/fluss/flink/adapter/SchemaAdapter.java +++ /dev/null @@ -1,43 +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.fluss.flink.adapter; - -import org.apache.flink.table.api.Schema; - -import java.util.List; - -/** - * An adapter for the schema with Index. - * - *

TODO: remove this class when no longer support all the Flink 1.x series. - */ -public class SchemaAdapter { - private SchemaAdapter() {} - - public static Schema withIndex(Schema unresolvedSchema, List> indexes) { - Schema.Builder newSchemaBuilder = Schema.newBuilder().fromSchema(unresolvedSchema); - for (List index : indexes) { - newSchemaBuilder.index(index); - } - return newSchemaBuilder.build(); - } - - public static boolean supportIndex() { - return true; - } -} diff --git a/fluss-flink/fluss-flink-2.1/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/fluss-flink/fluss-flink-2.1/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index 6544cb5347..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,19 +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. -# - -org.apache.fluss.flink.catalog.FlinkCatalogFactory \ No newline at end of file diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/adapter/Flink21MultipleParameterToolTest.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/adapter/Flink21MultipleParameterToolTest.java deleted file mode 100644 index f3922bcbda..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/adapter/Flink21MultipleParameterToolTest.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.adapter; - -/** Test for {@link MultipleParameterToolAdapter} in flink 2.1. */ -public class Flink21MultipleParameterToolTest extends FlinkMultipleParameterToolTest {} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21CatalogITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21CatalogITCase.java deleted file mode 100644 index bfccfa2d06..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21CatalogITCase.java +++ /dev/null @@ -1,175 +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.fluss.flink.catalog; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.ObjectPath; -import org.junit.jupiter.api.Test; - -import static org.assertj.core.api.Assertions.assertThat; - -/** IT case for catalog in Flink 2.1. */ -public class Flink21CatalogITCase extends FlinkCatalogITCase { - - @Test - void testGetTableWithIndex() throws Exception { - String tableName = "table_with_pk_only"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " primary key (a, b) NOT ENFORCED" - + ") with ( " - + " 'connector' = 'fluss' " - + ")", - tableName)); - CatalogTable table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - Schema expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .primaryKey("a", "b") - .index("a", "b") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_prefix_bucket_key"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " primary key (a, b) NOT ENFORCED" - + ") with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'a'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .primaryKey("a", "b") - .index("a", "b") - .index("a") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_bucket_key_is_not_prefix_pk"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " primary key (a, b) NOT ENFORCED" - + ") with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'b'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .primaryKey("a", "b") - .index("a", "b") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_partition_1"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " dt varchar, " - + " primary key (a, b, dt) NOT ENFORCED " - + ") " - + " partitioned by (dt) " - + " with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'a'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .column("dt", DataTypes.STRING().notNull()) - .primaryKey("a", "b", "dt") - .index("a", "b", "dt") - .index("a", "dt") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - - tableName = "table_with_partition_2"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a int, " - + " b varchar, " - + " c bigint, " - + " dt varchar, " - + " primary key (dt, a, b) NOT ENFORCED " - + ") " - + " partitioned by (dt) " - + " with ( " - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'a'" - + ")", - tableName)); - - table = (CatalogTable) catalog.getTable(new ObjectPath(DEFAULT_DB, tableName)); - expectedSchema = - Schema.newBuilder() - .column("a", DataTypes.INT().notNull()) - .column("b", DataTypes.STRING().notNull()) - .column("c", DataTypes.BIGINT()) - .column("dt", DataTypes.STRING().notNull()) - .primaryKey("dt", "a", "b") - .index("dt", "a", "b") - .index("a", "dt") - .build(); - assertThat(table.getUnresolvedSchema()).isEqualTo(expectedSchema); - } - - @Override - protected void addDefaultIndexKey(Schema.Builder schemaBuilder) { - super.addDefaultIndexKey(schemaBuilder); - - Schema currentSchema = schemaBuilder.build(); - currentSchema.getPrimaryKey().ifPresent(pk -> schemaBuilder.index(pk.getColumnNames())); - } -} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21MaterializedTableITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21MaterializedTableITCase.java deleted file mode 100644 index ee30419960..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/Flink21MaterializedTableITCase.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.catalog; - -/** IT case for materialized table in Flink 2.1. */ -public class Flink21MaterializedTableITCase extends MaterializedTableITCase {} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog21Test.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog21Test.java deleted file mode 100644 index 0c89ded816..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog21Test.java +++ /dev/null @@ -1,44 +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.fluss.flink.catalog; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.DefaultIndex; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.catalog.UniqueConstraint; - -import java.util.Arrays; -import java.util.Collections; - -/** Test for {@link FlinkCatalog}. */ -public class FlinkCatalog21Test extends FlinkCatalogTest { - - protected ResolvedSchema createSchema() { - return new ResolvedSchema( - Arrays.asList( - Column.physical("first", DataTypes.STRING().notNull()), - Column.physical("second", DataTypes.INT()), - Column.physical("third", DataTypes.STRING().notNull())), - Collections.emptyList(), - UniqueConstraint.primaryKey("PK_first_third", Arrays.asList("first", "third")), - Collections.singletonList( - DefaultIndex.newIndex( - "INDEX_first_third", Arrays.asList("first", "third")))); - } -} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/metrics/Flink21MetricsITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/metrics/Flink21MetricsITCase.java deleted file mode 100644 index 961c69d2fb..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/metrics/Flink21MetricsITCase.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.metrics; - -/** IT case for metrics in Flink 2.1. */ -public class Flink21MetricsITCase extends FlinkMetricsITCase {} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/procedure/Flink21ProcedureITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/procedure/Flink21ProcedureITCase.java deleted file mode 100644 index c95c47f0ff..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/procedure/Flink21ProcedureITCase.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.procedure; - -/** IT case for procedure in Flink 2.1. */ -public class Flink21ProcedureITCase extends FlinkProcedureITCase {} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/security/acl/Flink21AuthorizationITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/security/acl/Flink21AuthorizationITCase.java deleted file mode 100644 index 66aefab8e0..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/security/acl/Flink21AuthorizationITCase.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.security.acl; - -/** IT case for authorization in Flink 2.1. */ -public class Flink21AuthorizationITCase extends FlinkAuthorizationITCase {} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/sink/Flink21ComplexTypeITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/sink/Flink21ComplexTypeITCase.java deleted file mode 100644 index 7b2ed44b2b..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/sink/Flink21ComplexTypeITCase.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.sink; - -/** Integration tests for Array type support in Flink 2.1. */ -public class Flink21ComplexTypeITCase extends FlinkComplexTypeITCase {} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/sink/Flink21TableSinkITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/sink/Flink21TableSinkITCase.java deleted file mode 100644 index b040476f4d..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/sink/Flink21TableSinkITCase.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.sink; - -/** IT case for {@link FlinkTableSink} in Flink 2.1. */ -public class Flink21TableSinkITCase extends FlinkTableSinkITCase {} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceBatchITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceBatchITCase.java deleted file mode 100644 index a881b6b31c..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceBatchITCase.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.source; - -/** IT case for batch source in Flink 2.1. */ -public class Flink21TableSourceBatchITCase extends FlinkTableSourceBatchITCase {} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceFailOverITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceFailOverITCase.java deleted file mode 100644 index 630470f288..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceFailOverITCase.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.fluss.flink.source; - -/** IT case for source failover and recovery in Flink 2.1. */ -public class Flink21TableSourceFailOverITCase extends FlinkTableSourceFailOverITCase {} diff --git a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceITCase.java b/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceITCase.java deleted file mode 100644 index 9b1e908daf..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/java/org/apache/fluss/flink/source/Flink21TableSourceITCase.java +++ /dev/null @@ -1,147 +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.fluss.flink.source; - -import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.row.InternalRow; - -import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.flink.table.api.config.OptimizerConfigOptions; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.junit.jupiter.api.Test; - -import java.util.Arrays; -import java.util.List; - -import static org.apache.fluss.flink.source.testutils.FlinkRowAssertionsUtils.assertResultsIgnoreOrder; -import static org.apache.fluss.flink.utils.FlinkTestBase.writeRows; -import static org.apache.fluss.testutils.DataTestUtils.row; -import static org.assertj.core.api.Assertions.assertThat; - -/** IT case for {@link FlinkTableSource} in Flink 2.1. */ -public class Flink21TableSourceITCase extends FlinkTableSourceITCase { - - @Test - void testDeltaJoin() throws Exception { - // start two jobs for this test: one for DML involving the delta join, and the other for DQL - // to query the results of the sink table - tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); - - String leftTableName = "left_table"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " b1 varchar, " - + " c1 bigint, " - + " d1 int, " - + " e1 bigint, " - + " primary key (c1, d1) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c1', " - // currently, delta join only support append-only source - + " 'table.merge-engine' = 'first_row' " - + ")", - leftTableName)); - List rows1 = - Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v2", 200L, 2, 20000L), - row(3, "v1", 300L, 3, 30000L), - row(4, "v4", 400L, 4, 40000L)); - // write records - TablePath leftTablePath = TablePath.of(DEFAULT_DB, leftTableName); - writeRows(conn, leftTablePath, rows1, false); - - String rightTableName = "right_table"; - tEnv.executeSql( - String.format( - "create table %s (" - + " a2 int, " - + " b2 varchar, " - + " c2 bigint, " - + " d2 int, " - + " e2 bigint, " - + " primary key (c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss', " - + " 'bucket.key' = 'c2', " - // currently, delta join only support append-only source - + " 'table.merge-engine' = 'first_row' " - + ")", - rightTableName)); - List rows2 = - Arrays.asList( - row(1, "v1", 100L, 1, 10000L), - row(2, "v3", 200L, 2, 20000L), - row(3, "v4", 300L, 4, 30000L), - row(4, "v4", 500L, 4, 50000L)); - // write records - TablePath rightTablePath = TablePath.of(DEFAULT_DB, rightTableName); - writeRows(conn, rightTablePath, rows2, false); - - String sinkTableName = "sink_table"; - tEnv.executeSql( - String.format( - "create table %s ( " - + " a1 int, " - + " b1 varchar, " - + " c1 bigint, " - + " d1 int, " - + " e1 bigint, " - + " a2 int, " - + " b2 varchar, " - + " c2 bigint, " - + " d2 int, " - + " e2 bigint, " - + " primary key (c1, d1, c2, d2) NOT ENFORCED" - + ") with (" - + " 'connector' = 'fluss' " - + ")", - sinkTableName)); - - tEnv.getConfig() - .set( - OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, - OptimizerConfigOptions.DeltaJoinStrategy.FORCE); - - String sql = - String.format( - "INSERT INTO %s SELECT * FROM %s INNER JOIN %s ON c1 = c2 AND d1 = d2", - sinkTableName, leftTableName, rightTableName); - - assertThat(tEnv.explainSql(sql)) - .contains("DeltaJoin(joinType=[InnerJoin], where=[((c1 = c2) AND (d1 = d2))]"); - - tEnv.executeSql(sql); - - CloseableIterator collected = - tEnv.executeSql(String.format("select * from %s", sinkTableName)).collect(); - List expected = - Arrays.asList( - "+I[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", - "-U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", - "+U[1, v1, 100, 1, 10000, 1, v1, 100, 1, 10000]", - "+I[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", - "-U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]", - "+U[2, v2, 200, 2, 20000, 2, v3, 200, 2, 20000]"); - assertResultsIgnoreOrder(collected, expected, true); - } -} diff --git a/fluss-flink/fluss-flink-2.1/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/fluss-flink/fluss-flink-2.1/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension deleted file mode 100644 index ca0e907f6d..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension +++ /dev/null @@ -1,19 +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. -# - -org.apache.fluss.testutils.common.TestLoggerExtension \ No newline at end of file diff --git a/fluss-flink/fluss-flink-2.1/src/test/resources/log4j2-test.properties b/fluss-flink/fluss-flink-2.1/src/test/resources/log4j2-test.properties deleted file mode 100644 index 12b05f1867..0000000000 --- a/fluss-flink/fluss-flink-2.1/src/test/resources/log4j2-test.properties +++ /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. -# - -# Set root logger level to OFF to not flood build logs -# set manually to INFO for debugging purposes -rootLogger.level = OFF -rootLogger.appenderRef.test.ref = TestLogger - -appender.testlogger.name = TestLogger -appender.testlogger.type = CONSOLE -appender.testlogger.target = SYSTEM_ERR -appender.testlogger.layout.type = PatternLayout -appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n - -# suppress the duplicated logger extension -logger.flink.name = org.apache.flink.util.TestLoggerExtension -logger.flink.level = OFF diff --git a/fluss-flink/pom.xml b/fluss-flink/pom.xml index 6802286e5a..d759f53563 100644 --- a/fluss-flink/pom.xml +++ b/fluss-flink/pom.xml @@ -36,7 +36,6 @@ fluss-flink-1.20 fluss-flink-1.19 fluss-flink-1.18 - fluss-flink-2.1 fluss-flink-2.2 fluss-flink-tiering diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index 46c34da57a..7b058a3dd8 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -68,12 +68,6 @@ compile - - org.apache.fluss - fluss-flink-2.1 - ${project.version} - compile - org.apache.fluss diff --git a/website/src/pages/downloads.md b/website/src/pages/downloads.md index 69624e1d48..72a7a4eadc 100644 --- a/website/src/pages/downloads.md +++ b/website/src/pages/downloads.md @@ -9,24 +9,24 @@ Repo: https://downloads.apache.org/incubator/fluss/helm-chart ## Apache Fluss 0.8.0 -| Artifact | Signature | SHA | -|------------------------------------------------------------------------------------------------------------------------------------------------------------|---------|---------| -| [Fluss Binary Release](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz.sha512) | -| [Fluss Source Release](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz.sha512) | -| [Fluss Helm Chart](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz.sha512) | +| Artifact | Signature | SHA | +|------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------| +| [Fluss Binary Release](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz.sha512) | +| [Fluss Source Release](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz.sha512) | +| [Fluss Helm Chart](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz.sha512) | | [Fluss Tiering Service Jar](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-tiering/0.8.0-incubating/fluss-flink-tiering-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-tiering/0.8.0-incubating/fluss-flink-tiering-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-tiering/0.8.0-incubating/fluss-flink-tiering-0.8.0-incubating.jar.sha1) | -| [Connector Jar for Flink 2.1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.1/0.8.0-incubating/fluss-flink-2.1-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.1/0.8.0-incubating/fluss-flink-2.1-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.1/0.8.0-incubating/fluss-flink-2.1-0.8.0-incubating.jar.sha1) | -| [Connector Jar for Flink 1.20](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar.sha1) | -| [Connector Jar for Flink 1.19](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar.sha1) | -| [Connector Jar for Flink 1.18](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar.sha1) | -| [Lake Jar for Paimon](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar.sha1) | -| [Lake Jar for Iceberg](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar.sha1) | -| [Lake Jar for Lance](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for AWS S3](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for Aliyun OSS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for HDFS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for Google Cloud Storage](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for Huawei OBS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar.sha1) | +| [Connector Jar for Flink 2.2](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.2/0.8.0-incubating/fluss-flink-2.2-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.2/0.8.0-incubating/fluss-flink-2.2-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.2/0.8.0-incubating/fluss-flink-2.2-0.8.0-incubating.jar.sha1) | +| [Connector Jar for Flink 1.20](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar.sha1) | +| [Connector Jar for Flink 1.19](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar.sha1) | +| [Connector Jar for Flink 1.18](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar.sha1) | +| [Lake Jar for Paimon](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar.sha1) | +| [Lake Jar for Iceberg](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar.sha1) | +| [Lake Jar for Lance](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for AWS S3](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for Aliyun OSS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for HDFS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for Google Cloud Storage](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for Huawei OBS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar.sha1) | Read the [release blog](/blog/releases/0.8/) about the new features and significant improvements in the Apache Fluss 0.8.0 release. From 375106dc14bcc9090d0d37c5bc742b07090e6b16 Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Wed, 24 Dec 2025 10:51:21 +0800 Subject: [PATCH 09/14] Revert changes to the download page. --- website/src/pages/downloads.md | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/website/src/pages/downloads.md b/website/src/pages/downloads.md index 72a7a4eadc..69624e1d48 100644 --- a/website/src/pages/downloads.md +++ b/website/src/pages/downloads.md @@ -9,24 +9,24 @@ Repo: https://downloads.apache.org/incubator/fluss/helm-chart ## Apache Fluss 0.8.0 -| Artifact | Signature | SHA | -|------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------| -| [Fluss Binary Release](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz.sha512) | -| [Fluss Source Release](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz.sha512) | -| [Fluss Helm Chart](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz.sha512) | +| Artifact | Signature | SHA | +|------------------------------------------------------------------------------------------------------------------------------------------------------------|---------|---------| +| [Fluss Binary Release](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-bin.tgz.sha512) | +| [Fluss Source Release](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/fluss-0.8.0-incubating/fluss-0.8.0-incubating-src.tgz.sha512) | +| [Fluss Helm Chart](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz) | [.asc](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz.asc) | [.sha512](https://downloads.apache.org/incubator/fluss/helm-chart/0.8.0-incubating/fluss-0.8.0-incubating.tgz.sha512) | | [Fluss Tiering Service Jar](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-tiering/0.8.0-incubating/fluss-flink-tiering-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-tiering/0.8.0-incubating/fluss-flink-tiering-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-tiering/0.8.0-incubating/fluss-flink-tiering-0.8.0-incubating.jar.sha1) | -| [Connector Jar for Flink 2.2](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.2/0.8.0-incubating/fluss-flink-2.2-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.2/0.8.0-incubating/fluss-flink-2.2-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.2/0.8.0-incubating/fluss-flink-2.2-0.8.0-incubating.jar.sha1) | -| [Connector Jar for Flink 1.20](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar.sha1) | -| [Connector Jar for Flink 1.19](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar.sha1) | -| [Connector Jar for Flink 1.18](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar.sha1) | -| [Lake Jar for Paimon](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar.sha1) | -| [Lake Jar for Iceberg](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar.sha1) | -| [Lake Jar for Lance](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for AWS S3](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for Aliyun OSS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for HDFS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for Google Cloud Storage](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar.sha1) | -| [Filesystem Jar for Huawei OBS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar.sha1) | +| [Connector Jar for Flink 2.1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.1/0.8.0-incubating/fluss-flink-2.1-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.1/0.8.0-incubating/fluss-flink-2.1-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-2.1/0.8.0-incubating/fluss-flink-2.1-0.8.0-incubating.jar.sha1) | +| [Connector Jar for Flink 1.20](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.20/0.8.0-incubating/fluss-flink-1.20-0.8.0-incubating.jar.sha1) | +| [Connector Jar for Flink 1.19](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.19/0.8.0-incubating/fluss-flink-1.19-0.8.0-incubating.jar.sha1) | +| [Connector Jar for Flink 1.18](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-flink-1.18/0.8.0-incubating/fluss-flink-1.18-0.8.0-incubating.jar.sha1) | +| [Lake Jar for Paimon](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-paimon/0.8.0-incubating/fluss-lake-paimon-0.8.0-incubating.jar.sha1) | +| [Lake Jar for Iceberg](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-iceberg/0.8.0-incubating/fluss-lake-iceberg-0.8.0-incubating.jar.sha1) | +| [Lake Jar for Lance](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-lake-lance/0.8.0-incubating/fluss-lake-lance-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for AWS S3](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-s3/0.8.0-incubating/fluss-fs-s3-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for Aliyun OSS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-oss/0.8.0-incubating/fluss-fs-oss-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for HDFS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-hdfs/0.8.0-incubating/fluss-fs-hdfs-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for Google Cloud Storage](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-gs/0.8.0-incubating/fluss-fs-gs-0.8.0-incubating.jar.sha1) | +| [Filesystem Jar for Huawei OBS](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar) | [.asc](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar.asc) | [.sha1](https://repo1.maven.org/maven2/org/apache/fluss/fluss-fs-obs/0.8.0-incubating/fluss-fs-obs-0.8.0-incubating.jar.sha1) | Read the [release blog](/blog/releases/0.8/) about the new features and significant improvements in the Apache Fluss 0.8.0 release. From 3318b48e782e2596cff0297a8b8ec4ce649b690b Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Wed, 24 Dec 2025 11:40:53 +0800 Subject: [PATCH 10/14] fix comment format --- .../apache/fluss/flink/source/Flink22TableSourceITCase.java | 6 ++---- .../java/org/apache/fluss/flink/catalog/FlinkCatalog.java | 3 +-- .../main/java/org/apache/fluss/flink/sink/FlinkSink.java | 3 +-- .../org/apache/fluss/flink/catalog/FlinkCatalogTest.java | 5 ++--- 4 files changed, 6 insertions(+), 11 deletions(-) diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java index 24d8157b52..7253720481 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceITCase.java @@ -39,8 +39,7 @@ public class Flink22TableSourceITCase extends FlinkTableSourceITCase { @Test void testDeltaJoin() throws Exception { - // start two jobs for this test: one for DML involving the delta join, and the - // other for DQL + // start two jobs for this test: one for DML involving the delta join, and the other for DQL // to query the results of the sink table tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); @@ -148,8 +147,7 @@ void testDeltaJoin() throws Exception { @Test void testDeltaJoinWithProjectionAndFilter() throws Exception { - // start two jobs for this test: one for DML involving the delta join, and the - // other for DQL + // start two jobs for this test: one for DML involving the delta join, and the other for DQL // to query the results of the sink table tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkCatalog.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkCatalog.java index e2556c913b..37e9f7c5e8 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkCatalog.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/catalog/FlinkCatalog.java @@ -828,8 +828,7 @@ private CatalogTable wrapWithIndexes(CatalogTable table, TableInfo tableInfo) { // Judge whether we can do prefix lookup. List bucketKeys = tableInfo.getBucketKeys(); - // For partition table, the physical primary key is the primary key that - // excludes the + // For partition table, the physical primary key is the primary key that excludes the // partition key List physicalPrimaryKeys = tableInfo.getPhysicalPrimaryKeys(); List indexKeys = new ArrayList<>(); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java index 8ce3f6a46a..c509ecbfc7 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java @@ -124,8 +124,7 @@ public AppendSinkWriter createWriter(MailboxExecutor mailboxExecutor) { @Override public DataStream addPreWriteTopology(DataStream input) { - // For append only sink, we will do bucket shuffle only if bucket keys are not - // empty. + // For append only sink, we will do bucket shuffle only if bucket keys are not empty. if (!bucketKeys.isEmpty() && shuffleByBucketId) { return partition( input, diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java index 541d9ce636..0197e5f18b 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java @@ -332,8 +332,7 @@ void testCreateAlreadyExistsLakeTable() throws Exception { // drop fluss table catalog.dropTable(lakeTablePath, false); assertThat(catalog.tableExists(lakeTablePath)).isFalse(); - // create the table again should be ok, because the existing lake table is - // matched + // create the table again should be ok, because the existing lake table is matched catalog.createTable(lakeTablePath, table, false); } @@ -847,7 +846,7 @@ void testConnectionFailureHandling() { @Test void testStatisticsOperations() throws Exception { - // Statistics testing + // Statistics testing CatalogTable table = newCatalogTable(Collections.emptyMap()); ObjectPath tablePath = new ObjectPath(DEFAULT_DB, "statsTable"); catalog.createTable(tablePath, table, false); From f483517c6eb2bf2b00e7aa94ecb4e47b7aa53b17 Mon Sep 17 00:00:00 2001 From: vamossagar12 Date: Wed, 24 Dec 2025 13:17:49 +0530 Subject: [PATCH 11/14] Handling review comments --- ...FlinkSinkAdapter.java => SinkAdapter.java} | 2 +- .../adapter/Flink22MultipleParameterToolTest | 21 ++++++++++++ ...log22Test.java => Flink22CatalogTest.java} | 33 +------------------ .../Flink22MaterializedTableITCase.java | 21 ++++++++++++ fluss-test-coverage/pom.xml | 7 ++++ 5 files changed, 51 insertions(+), 33 deletions(-) rename fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/{FlinkSinkAdapter.java => SinkAdapter.java} (95%) create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest rename fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/{FlinkCatalog22Test.java => Flink22CatalogTest.java} (51%) create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22MaterializedTableITCase.java diff --git a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.java similarity index 95% rename from fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java rename to fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.java index bb518ce861..8d6e512501 100644 --- a/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java +++ b/fluss-flink/fluss-flink-2.2/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.java @@ -30,7 +30,7 @@ * *

TODO: remove this class when no longer support all the Flink 1.x series. */ -public abstract class FlinkSinkAdapter implements Sink { +public abstract class SinkAdapter implements Sink { @Override public SinkWriter createWriter(WriterInitContext writerInitContext) throws IOException { diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest new file mode 100644 index 0000000000..db6a0117ee --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest @@ -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.fluss.flink.adapter; + +/** Test for {@link MultipleParameterToolAdapter} in flink 2.2. */ +public class Flink22MultipleParameterToolTest extends FlinkMultipleParameterToolTest {} \ No newline at end of file diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogTest.java similarity index 51% rename from fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java rename to fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogTest.java index e742bfb037..bff2b77d60 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalog22Test.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22CatalogTest.java @@ -18,21 +18,16 @@ package org.apache.fluss.flink.catalog; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.catalog.CatalogMaterializedTable; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.DefaultIndex; -import org.apache.flink.table.catalog.IntervalFreshness; -import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; import java.util.Arrays; import java.util.Collections; -import java.util.Map; /** Test for {@link FlinkCatalog}. */ -public class FlinkCatalog22Test extends FlinkCatalogTest { +public class Flink22CatalogTest extends FlinkCatalogTest { protected ResolvedSchema createSchema() { return new ResolvedSchema( @@ -46,30 +41,4 @@ protected ResolvedSchema createSchema() { DefaultIndex.newIndex( "INDEX_first_third", Arrays.asList("first", "third")))); } - - protected CatalogMaterializedTable newCatalogMaterializedTable( - ResolvedSchema resolvedSchema, - CatalogMaterializedTable.RefreshMode refreshMode, - Map options) { - CatalogMaterializedTable origin = - CatalogMaterializedTable.newBuilder() - .schema(Schema.newBuilder().fromResolvedSchema(resolvedSchema).build()) - .comment("test comment") - .options(options) - .partitionKeys(Collections.emptyList()) - .definitionQuery("select first, second, third from t") - .freshness(IntervalFreshness.of("5", IntervalFreshness.TimeUnit.SECOND)) - .logicalRefreshMode( - refreshMode == CatalogMaterializedTable.RefreshMode.CONTINUOUS - ? CatalogMaterializedTable.LogicalRefreshMode.CONTINUOUS - : CatalogMaterializedTable.LogicalRefreshMode.FULL) - .refreshMode(refreshMode) - .refreshStatus(CatalogMaterializedTable.RefreshStatus.INITIALIZING) - .build(); - return new ResolvedCatalogMaterializedTable( - origin, - resolvedSchema, - refreshMode, - IntervalFreshness.of("5", IntervalFreshness.TimeUnit.SECOND)); - } } diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22MaterializedTableITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22MaterializedTableITCase.java new file mode 100644 index 0000000000..36240466c4 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/catalog/Flink22MaterializedTableITCase.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.flink.catalog; + +/** IT case for materialized table in Flink 2.2. */ +public class Flink22MaterializedTableITCase extends MaterializedTableITCase {} diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index 7b058a3dd8..de2800d5e2 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -68,6 +68,13 @@ compile + + org.apache.fluss + fluss-flink-2.2 + ${project.version} + compile + + org.apache.fluss From 63ab6b443650b8372cf66879bb933f62e7fc7660 Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Wed, 24 Dec 2025 16:12:11 +0800 Subject: [PATCH 12/14] fix comment format --- .../connector/sink2/WriterInitContext.java | 21 ---------- .../fluss/flink/adapter/SinkAdapter.java | 41 +++++++++++++++++++ ...solvedCatalogMaterializedTableAdapter.java | 41 +++++++++++++++++++ ...FlinkSinkAdapter.java => SinkAdapter.java} | 2 +- .../apache/fluss/flink/sink/FlinkSink.java | 5 +-- ...solvedCatalogMaterializedTableAdapter.java | 41 +++++++++++++++++++ .../fluss/flink/catalog/FlinkCatalogTest.java | 8 +++- 7 files changed, 132 insertions(+), 27 deletions(-) delete mode 100644 fluss-flink/fluss-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java create mode 100644 fluss-flink/fluss-flink-1.18/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.java create mode 100644 fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/ResolvedCatalogMaterializedTableAdapter.java rename fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/{FlinkSinkAdapter.java => SinkAdapter.java} (96%) create mode 100644 fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/adapter/ResolvedCatalogMaterializedTableAdapter.java diff --git a/fluss-flink/fluss-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/fluss-flink/fluss-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java deleted file mode 100644 index 6caa32d7c6..0000000000 --- a/fluss-flink/fluss-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.api.connector.sink2; - -/** Placeholder class to resolve compatibility issues. */ -public interface WriterInitContext extends Sink.InitContext {} diff --git a/fluss-flink/fluss-flink-1.18/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.java b/fluss-flink/fluss-flink-1.18/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.java new file mode 100644 index 0000000000..575bfdfbd7 --- /dev/null +++ b/fluss-flink/fluss-flink-1.18/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.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.fluss.flink.adapter; + +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; + +import java.io.IOException; + +/** + * Flink sink adapter which hide the different version of createWriter method. + * + *

TODO: remove this class when no longer support all the Flink 1.x series. + */ +public abstract class SinkAdapter implements Sink { + + @Override + public SinkWriter createWriter(InitContext initContext) throws IOException { + return createWriter(initContext.getMailboxExecutor(), initContext.metricGroup()); + } + + protected abstract SinkWriter createWriter( + MailboxExecutor mailboxExecutor, SinkWriterMetricGroup metricGroup); +} diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/ResolvedCatalogMaterializedTableAdapter.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/ResolvedCatalogMaterializedTableAdapter.java new file mode 100644 index 0000000000..f868a84db6 --- /dev/null +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/ResolvedCatalogMaterializedTableAdapter.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.fluss.flink.adapter; + +import org.apache.flink.table.catalog.CatalogMaterializedTable; +import org.apache.flink.table.catalog.IntervalFreshness; +import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; +import org.apache.flink.table.catalog.ResolvedSchema; + +/** + * Adapter for {@link ResolvedCatalogMaterializedTable} because the constructor is compatibility in + * flink 2.2. However, this constructor only used in test. + * + *

TODO: remove it until ... is + * fixed. + */ +public class ResolvedCatalogMaterializedTableAdapter { + + public static ResolvedCatalogMaterializedTable create( + CatalogMaterializedTable origin, + ResolvedSchema resolvedSchema, + CatalogMaterializedTable.RefreshMode refreshMode, + IntervalFreshness freshness) { + return new ResolvedCatalogMaterializedTable(origin, resolvedSchema, refreshMode, freshness); + } +} diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.java similarity index 96% rename from fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java rename to fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.java index ba869dbcdb..5f7e9e76c3 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/FlinkSinkAdapter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/adapter/SinkAdapter.java @@ -30,7 +30,7 @@ * *

TODO: remove this class when no longer support all the Flink 1.x series. */ -public abstract class FlinkSinkAdapter implements Sink { +public abstract class SinkAdapter implements Sink { @Override public SinkWriter createWriter(InitContext initContext) throws IOException { diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java index c509ecbfc7..fcdf9ab42e 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/sink/FlinkSink.java @@ -19,7 +19,7 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.config.Configuration; -import org.apache.fluss.flink.adapter.FlinkSinkAdapter; +import org.apache.fluss.flink.adapter.SinkAdapter; import org.apache.fluss.flink.sink.serializer.FlussSerializationSchema; import org.apache.fluss.flink.sink.writer.AppendSinkWriter; import org.apache.fluss.flink.sink.writer.FlinkSinkWriter; @@ -44,8 +44,7 @@ import static org.apache.fluss.flink.utils.FlinkConversions.toFlussRowType; /** Flink sink for Fluss. */ -class FlinkSink extends FlinkSinkAdapter - implements SupportsPreWriteTopology { +class FlinkSink extends SinkAdapter implements SupportsPreWriteTopology { private static final long serialVersionUID = 1L; diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/adapter/ResolvedCatalogMaterializedTableAdapter.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/adapter/ResolvedCatalogMaterializedTableAdapter.java new file mode 100644 index 0000000000..7a018bc027 --- /dev/null +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/adapter/ResolvedCatalogMaterializedTableAdapter.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.fluss.flink.adapter; + +import org.apache.flink.table.catalog.CatalogMaterializedTable; +import org.apache.flink.table.catalog.IntervalFreshness; +import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; +import org.apache.flink.table.catalog.ResolvedSchema; + +/** + * Adapter for {@link ResolvedCatalogMaterializedTable} because the constructor is compatibility in + * flink 2.2. However, this constructor only used in test. + * + *

TODO: remove it until ... is + * fixed. + */ +public class ResolvedCatalogMaterializedTableAdapter { + + public static ResolvedCatalogMaterializedTable create( + CatalogMaterializedTable origin, + ResolvedSchema resolvedSchema, + CatalogMaterializedTable.RefreshMode refreshMode, + IntervalFreshness freshness) { + return new ResolvedCatalogMaterializedTable(origin, resolvedSchema); + } +} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java index 0197e5f18b..85b8b3d8b6 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java @@ -22,6 +22,7 @@ import org.apache.fluss.exception.IllegalConfigurationException; import org.apache.fluss.exception.InvalidPartitionException; import org.apache.fluss.exception.InvalidTableException; +import org.apache.fluss.flink.adapter.ResolvedCatalogMaterializedTableAdapter; import org.apache.fluss.flink.lake.LakeFlinkCatalog; import org.apache.fluss.flink.utils.FlinkConversionsTest; import org.apache.fluss.server.testutils.FlussClusterExtension; @@ -40,7 +41,6 @@ import org.apache.flink.table.catalog.GenericInMemoryCatalog; import org.apache.flink.table.catalog.IntervalFreshness; import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.TableChange; @@ -158,7 +158,11 @@ protected CatalogMaterializedTable newCatalogMaterializedTable( .refreshMode(refreshMode) .refreshStatus(CatalogMaterializedTable.RefreshStatus.INITIALIZING) .build(); - return new ResolvedCatalogMaterializedTable(origin, resolvedSchema); + return ResolvedCatalogMaterializedTableAdapter.create( + origin, + resolvedSchema, + refreshMode, + IntervalFreshness.of("5", IntervalFreshness.TimeUnit.SECOND)); } protected FlinkCatalog initCatalog( From fb8d8032d2fdfd074dfd5cb8e43f7496af88723b Mon Sep 17 00:00:00 2001 From: Hongshun Wang Date: Wed, 24 Dec 2025 16:51:22 +0800 Subject: [PATCH 13/14] remove override of testCountPushDown --- .../source/Flink22TableSourceBatchITCase.java | 59 +------------------ .../source/FlinkTableSourceBatchITCase.java | 6 +- 2 files changed, 2 insertions(+), 63 deletions(-) diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java index 0f667ff81a..f65966ea6f 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/source/Flink22TableSourceBatchITCase.java @@ -17,62 +17,5 @@ package org.apache.fluss.flink.source; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; - -import java.util.Collections; -import java.util.List; - -import static org.apache.fluss.flink.source.testutils.FlinkRowAssertionsUtils.collectRowsWithTimeout; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - /** IT case for batch source in Flink 2.2. */ -public class Flink22TableSourceBatchITCase extends FlinkTableSourceBatchITCase { - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - @Override - void testCountPushDown(boolean partitionTable) throws Exception { - String tableName = partitionTable ? preparePartitionedLogTable() : prepareLogTable(); - int expectedRows = partitionTable ? 10 : 5; - // normal scan - String query = String.format("SELECT COUNT(*) FROM %s", tableName); - assertThat(tEnv.explainSql(query)) - .contains( - String.format( - "TableSourceScan(table=[[testcatalog, defaultdb, %s, " - + "aggregates=[grouping=[], aggFunctions=[Count1AggFunction()]]]], " - + "fields=[count1$0])", - tableName)); - CloseableIterator iterRows = tEnv.executeSql(query).collect(); - List collected = collectRowsWithTimeout(iterRows, 1); - List expected = Collections.singletonList(String.format("+I[%s]", expectedRows)); - assertThat(collected).isEqualTo(expected); - - // test not push down grouping count. - assertThatThrownBy( - () -> - tEnv.explainSql( - String.format( - "SELECT COUNT(*) FROM %s group by id", - tableName)) - .wait()) - .hasMessageContaining( - "Currently, Fluss only support queries on table with datalake enabled or point queries on primary key when it's in batch execution mode."); - - // test not support primary key now - String primaryTableName = prepareSourceTable(new String[] {"id"}, null); - assertThatThrownBy( - () -> - tEnv.explainSql( - String.format( - "SELECT COUNT(*) FROM %s ", - primaryTableName)) - .wait()) - .hasMessageContaining( - "Currently, Fluss only support queries on table with datalake enabled or point queries on primary key when it's in batch execution mode."); - } -} +public class Flink22TableSourceBatchITCase extends FlinkTableSourceBatchITCase {} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java index 6aea7a1b83..e38eda29b0 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java @@ -281,11 +281,7 @@ void testCountPushDown(boolean partitionTable) throws Exception { String query = String.format("SELECT COUNT(*) FROM %s", tableName); assertThat(tEnv.explainSql(query)) .contains( - String.format( - "TableSourceScan(table=[[testcatalog, defaultdb, %s, project=[id], " - + "aggregates=[grouping=[], aggFunctions=[Count1AggFunction()]]]], " - + "fields=[count1$0])", - tableName)); + "aggregates=[grouping=[], aggFunctions=[Count1AggFunction()]]]], fields=[count1$0]"); CloseableIterator iterRows = tEnv.executeSql(query).collect(); List collected = collectRowsWithTimeout(iterRows, 1); List expected = Collections.singletonList(String.format("+I[%s]", expectedRows)); From 4d22a4c7bb832565f687334729fb0d1a2549f050 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 24 Dec 2025 18:00:28 +0800 Subject: [PATCH 14/14] fix --- ...arameterToolTest => Flink22MultipleParameterToolTest.java} | 2 +- .../java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java | 2 +- .../fluss/flink/source/FlinkTableSourceBatchITCase.java | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) rename fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/{Flink22MultipleParameterToolTest => Flink22MultipleParameterToolTest.java} (97%) diff --git a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest.java similarity index 97% rename from fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest rename to fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest.java index db6a0117ee..4d06fe901f 100644 --- a/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest +++ b/fluss-flink/fluss-flink-2.2/src/test/java/org/apache/fluss/flink/adapter/Flink22MultipleParameterToolTest.java @@ -18,4 +18,4 @@ package org.apache.fluss.flink.adapter; /** Test for {@link MultipleParameterToolAdapter} in flink 2.2. */ -public class Flink22MultipleParameterToolTest extends FlinkMultipleParameterToolTest {} \ No newline at end of file +public class Flink22MultipleParameterToolTest extends FlinkMultipleParameterToolTest {} diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java index 85b8b3d8b6..6c65544c4a 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/catalog/FlinkCatalogTest.java @@ -139,7 +139,7 @@ private CatalogTable newCatalogTable( return new ResolvedCatalogTable(origin, resolvedSchema); } - protected CatalogMaterializedTable newCatalogMaterializedTable( + private CatalogMaterializedTable newCatalogMaterializedTable( ResolvedSchema resolvedSchema, CatalogMaterializedTable.RefreshMode refreshMode, Map options) { diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java index e38eda29b0..f295fff5f6 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/FlinkTableSourceBatchITCase.java @@ -311,7 +311,7 @@ void testCountPushDown(boolean partitionTable) throws Exception { "Currently, Fluss only support queries on table with datalake enabled or point queries on primary key when it's in batch execution mode."); } - protected String prepareSourceTable(String[] keys, String partitionedKey) throws Exception { + private String prepareSourceTable(String[] keys, String partitionedKey) throws Exception { String tableName = String.format("test_%s_%s", String.join("_", keys), RandomUtils.nextInt()); if (partitionedKey == null) { @@ -367,7 +367,7 @@ protected String prepareSourceTable(String[] keys, String partitionedKey) throws return tableName; } - protected String prepareLogTable() throws Exception { + private String prepareLogTable() throws Exception { String tableName = String.format("test_log_table_%s", RandomUtils.nextInt()); tEnv.executeSql( String.format(