diff --git a/.github/workflows/stage.sh b/.github/workflows/stage.sh
index 49a3ed57e9..18587dc292 100755
--- a/.github/workflows/stage.sh
+++ b/.github/workflows/stage.sh
@@ -24,7 +24,7 @@ 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-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.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/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