diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml
index 8a20a0a2cb06..f095d499034c 100644
--- a/backends-clickhouse/pom.xml
+++ b/backends-clickhouse/pom.xml
@@ -43,11 +43,189 @@
+
+ spark-3.3
+
+ false
+
+
+
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+
+
+ add-spark33-sources
+ generate-sources
+
+ add-source
+
+
+
+ ${project.basedir}/src-spark33/main/scala
+ ${project.basedir}src-spark33/main/java
+
+
+
+
+ add-spark33-resources
+ generate-resources
+
+ add-resource
+
+
+
+
+ ${project.basedir}/src-spark33/main/resources
+
+
+
+
+
+ add-spark33-test-sources
+ generate-test-sources
+
+ add-test-source
+
+
+
+ ${project.basedir}/src-spark33/test/scala
+ ${project.basedir}/src-spark33/test/java
+
+
+
+
+ add-spark33-test-resources
+ generate-test-resources
+
+ add-test-resource
+
+
+
+
+ ${project.basedir}/src-spark33/test/resources
+
+
+
+
+
+
+
+
+
+
+ spark-3.5
+
+ 1.7.36
+
+
+ false
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ ${hadoop.version}
+ provided
+
+
+ org.apache.logging.log4j
+ log4j-slf4j-impl
+
+
+
+
+ org.apache.hadoop
+ hadoop-client
+ ${hadoop.version}
+ test
+
+
+ org.apache.logging.log4j
+ log4j-slf4j-impl
+
+
+
+
+ com.google.guava
+ guava
+ ${guava.version}
+ provided
+
+
+
+
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+
+
+ add-spark35-sources
+ generate-sources
+
+ add-source
+
+
+
+ ${project.basedir}/src-spark35/main/scala
+ ${project.basedir}src-spark35/main/java
+
+
+
+
+ add-spark35-resources
+ generate-resources
+
+ add-resource
+
+
+
+
+ ${project.basedir}/src-spark35/main/resources
+
+
+
+
+
+ add-spark35-test-sources
+ generate-test-sources
+
+ add-test-source
+
+
+
+ ${project.basedir}/src-spark35/test/scala
+ ${project.basedir}/src-spark35/test/java
+
+
+
+
+ add-spark35-test-resources
+ generate-test-resources
+
+ add-test-resource
+
+
+
+
+ ${project.basedir}/src-spark35/test/resources
+
+
+
+
+
+
+
+
+
iceberg
false
+
+ 1.16.2
+
org.apache.gluten
@@ -64,6 +242,133 @@
org.scala-lang.modules
scala-collection-compat_${scala.binary.version}
+
+ org.apache.iceberg
+ iceberg-api
+ ${iceberg.version}
+ test-jar
+ test
+
+
+ org.apache.iceberg
+ iceberg-api
+ ${iceberg.version}
+ test
+
+
+ org.apache.iceberg
+ iceberg-flink-runtime-1.16
+ ${iceberg.version}
+ provided
+
+
+ org.apache.iceberg
+ iceberg-flink-1.16
+ ${iceberg.version}
+ test-jar
+ test
+
+
+ org.apache.iceberg
+ iceberg-flink-1.16
+ ${iceberg.version}
+ test
+
+
+ org.apache.iceberg
+ iceberg-hive-metastore
+ ${iceberg.version}
+ test-jar
+ test
+
+
+ org.apache.iceberg
+ iceberg-hive-metastore
+ ${iceberg.version}
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter
+ test
+ 5.10.1
+
+
+ org.junit.jupiter
+ junit-jupiter-engine
+ test
+ 5.10.1
+
+
+ org.apache.flink
+ flink-streaming-java
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-table-api-bridge-base
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-table-api-java
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-table-api-java-bridge
+ ${flink.version}
+ provided
+
+
+ org.apache.flink
+ flink-test-utils
+ ${flink.version}
+ provided
+
+
+
+ org.apache.flink
+ flink-table-planner_2.12
+ ${flink.version}
+ provided
+
+
+ org.scala-lang
+ scala-library
+
+
+
+
+ org.apache.iceberg
+ iceberg-spark-${sparkbundle.version}_${scala.binary.version}
+ ${iceberg.version}
+ test-jar
+ test
+
+
+ org.apache.iceberg
+ iceberg-spark-${sparkbundle.version}_${scala.binary.version}
+ ${iceberg.version}
+ test
+
+
+ org.apache.iceberg
+ iceberg-data
+ ${iceberg.version}
+ test-jar
+ test
+
+
+ org.apache.iceberg
+ iceberg-data
+ ${iceberg.version}
+ test
+
@@ -116,6 +421,10 @@
+
+ true
+
+
org.apache.gluten
@@ -240,44 +549,44 @@
test
- org.apache.hive.hcatalog
- hive-hcatalog-core
- 2.3.9
- test
-
-
- org.pentaho
- pentaho-aggdesigner-algorithm
-
-
- net.minidev
- json-smart
-
-
- org.apache.hive
- hive-exec
-
-
- guava
- com.google.guava
-
-
- hadoop-common
- org.apache.hadoop
-
-
- hadoop-hdfs
- org.apache.hadoop
-
-
- protobuf-java
- com.google.protobuf
-
-
- jdk.tools
- jdk.tools
-
-
+ org.apache.hive.hcatalog
+ hive-hcatalog-core
+ 2.3.9
+ test
+
+
+ org.pentaho
+ pentaho-aggdesigner-algorithm
+
+
+ net.minidev
+ json-smart
+
+
+ org.apache.hive
+ hive-exec
+
+
+ guava
+ com.google.guava
+
+
+ hadoop-common
+ org.apache.hadoop
+
+
+ hadoop-hdfs
+ org.apache.hadoop
+
+
+ protobuf-java
+ com.google.protobuf
+
+
+ jdk.tools
+ jdk.tools
+
+
org.apache.hadoop
@@ -551,6 +860,17 @@
true
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+
+ **/*Test.java
+ **/Test*.java
+
+ ${surefire.skipTests}
+
+
diff --git a/backends-clickhouse/src-spark33/test/java/org/apache/gluten/execution/iceberg/TestFlinkUpsert.java b/backends-clickhouse/src-spark33/test/java/org/apache/gluten/execution/iceberg/TestFlinkUpsert.java
new file mode 100644
index 000000000000..429feab086f3
--- /dev/null
+++ b/backends-clickhouse/src-spark33/test/java/org/apache/gluten/execution/iceberg/TestFlinkUpsert.java
@@ -0,0 +1,538 @@
+/*
+ * 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.gluten.execution.iceberg;
+
+import java.time.LocalDate;
+import java.time.ZoneId;
+import java.util.Date;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Parameter;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.flink.CatalogTestBase;
+import org.apache.iceberg.flink.MiniClusterResource;
+import org.apache.iceberg.flink.TestHelpers;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.SparkSession;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+
+public class TestFlinkUpsert extends CatalogTestBase {
+
+ @Parameter(index = 2)
+ private FileFormat format;
+
+ @Parameter(index = 3)
+ private boolean isStreamingJob;
+
+ private final Map tableUpsertProps = Maps.newHashMap();
+ private TableEnvironment tEnv;
+ private SparkSession spark;
+ private ClickHouseIcebergHiveTableSupport hiveTableSupport;
+
+ @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}")
+ public static List