This is an automated email from the ASF dual-hosted git repository.

lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git


The following commit(s) were added to refs/heads/master by this push:
     new c455db2bc0 [clone] support meta only in clone action (#6967)
c455db2bc0 is described below

commit c455db2bc0a2ed60d8e4e75cc778210e3820ffa9
Author: shyjsarah <[email protected]>
AuthorDate: Thu Jan 8 09:22:31 2026 +0800

    [clone] support meta only in clone action (#6967)
---
 .../apache/paimon/flink/action/CloneAction.java    | 11 ++--
 .../paimon/flink/action/CloneActionFactory.java    | 10 +++-
 .../paimon/flink/clone/CloneHiveTableUtils.java    |  9 +++-
 .../paimon/flink/clone/ClonePaimonTableUtils.java  |  9 +++-
 .../paimon/flink/procedure/CloneProcedure.java     | 10 +++-
 .../paimon/hive/procedure/CloneActionITCase.java   | 58 ++++++++++++++++++++++
 6 files changed, 99 insertions(+), 8 deletions(-)

diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java
index 0280e9badf..fe283fe8f4 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java
@@ -48,6 +48,7 @@ public class CloneAction extends ActionBase {
     @Nullable private final List<String> excludedTables;
     @Nullable private final String preferFileFormat;
     private final String cloneFrom;
+    private final boolean metaOnly;
 
     public CloneAction(
             String sourceDatabase,
@@ -61,7 +62,8 @@ public class CloneAction extends ActionBase {
             @Nullable List<String> includedTables,
             @Nullable List<String> excludedTables,
             @Nullable String preferFileFormat,
-            String cloneFrom) {
+            String cloneFrom,
+            boolean metaOnly) {
         super(sourceCatalogConfig);
 
         if (cloneFrom.equalsIgnoreCase("hive")) {
@@ -94,6 +96,7 @@ public class CloneAction extends ActionBase {
                         ? preferFileFormat
                         : preferFileFormat.toLowerCase();
         this.cloneFrom = cloneFrom;
+        this.metaOnly = metaOnly;
     }
 
     @Override
@@ -113,7 +116,8 @@ public class CloneAction extends ActionBase {
                         whereSql,
                         includedTables,
                         excludedTables,
-                        preferFileFormat);
+                        preferFileFormat,
+                        metaOnly);
                 break;
             case "paimon":
                 ClonePaimonTableUtils.build(
@@ -129,7 +133,8 @@ public class CloneAction extends ActionBase {
                         whereSql,
                         includedTables,
                         excludedTables,
-                        preferFileFormat);
+                        preferFileFormat,
+                        metaOnly);
                 break;
         }
     }
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneActionFactory.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneActionFactory.java
index c0258ec70f..5d884bd2d9 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneActionFactory.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneActionFactory.java
@@ -40,6 +40,7 @@ public class CloneActionFactory implements ActionFactory {
     private static final String EXCLUDED_TABLES = "excluded_tables";
     private static final String PREFER_FILE_FORMAT = "prefer_file_format";
     private static final String CLONE_FROM = "clone_from";
+    private static final String META_ONLY = "meta_only";
 
     @Override
     public String identifier() {
@@ -76,6 +77,12 @@ public class CloneActionFactory implements ActionFactory {
             cloneFrom = "hive";
         }
         String preferFileFormat = params.get(PREFER_FILE_FORMAT);
+
+        String metaOnlyStr = params.get(META_ONLY);
+        boolean metaOnly =
+                !StringUtils.isNullOrWhitespaceOnly(metaOnlyStr)
+                        && Boolean.parseBoolean(metaOnlyStr);
+
         CloneAction cloneAction =
                 new CloneAction(
                         params.get(DATABASE),
@@ -89,7 +96,8 @@ public class CloneActionFactory implements ActionFactory {
                         includedTables,
                         excludedTables,
                         preferFileFormat,
-                        cloneFrom);
+                        cloneFrom,
+                        metaOnly);
 
         return Optional.of(cloneAction);
     }
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneHiveTableUtils.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneHiveTableUtils.java
index c32c9b53c5..b6298404a3 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneHiveTableUtils.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneHiveTableUtils.java
@@ -156,7 +156,8 @@ public class CloneHiveTableUtils {
             @Nullable String whereSql,
             @Nullable List<String> includedTables,
             @Nullable List<String> excludedTables,
-            @Nullable String preferFileFormat)
+            @Nullable String preferFileFormat,
+            boolean metaOnly)
             throws Exception {
         // list source tables
         DataStream<Tuple2<Identifier, Identifier>> source =
@@ -183,6 +184,12 @@ public class CloneHiveTableUtils {
                         .name("Clone Schema")
                         .setParallelism(parallelism);
 
+        // if metaOnly is true, only clone schema and skip data cloning
+        if (metaOnly) {
+            schemaInfos.sinkTo(new 
DiscardingSink<>()).name("end").setParallelism(1);
+            return;
+        }
+
         buildForCloneSplits(
                 sourceCatalogConfig, targetCatalogConfig, parallelism, 
whereSql, schemaInfos);
 
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/ClonePaimonTableUtils.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/ClonePaimonTableUtils.java
index a4984e69ed..1bce445a45 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/ClonePaimonTableUtils.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/ClonePaimonTableUtils.java
@@ -138,7 +138,8 @@ public class ClonePaimonTableUtils {
             @Nullable String whereSql,
             @Nullable List<String> includedTables,
             @Nullable List<String> excludedTables,
-            @Nullable String preferFileFormat)
+            @Nullable String preferFileFormat,
+            boolean metaOnly)
             throws Exception {
         // list source tables
         DataStream<Tuple2<Identifier, Identifier>> source =
@@ -165,6 +166,12 @@ public class ClonePaimonTableUtils {
                         .name("Clone Schema")
                         .setParallelism(parallelism);
 
+        // if metaOnly is true, only clone schema and skip data cloning
+        if (metaOnly) {
+            schemaInfos.sinkTo(new 
DiscardingSink<>()).name("end").setParallelism(1);
+            return;
+        }
+
         // list splits
         DataStream<CloneSplitInfo> splits =
                 schemaInfos
diff --git 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CloneProcedure.java
 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CloneProcedure.java
index 1f60707c7c..4daeb2fef7 100644
--- 
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CloneProcedure.java
+++ 
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CloneProcedure.java
@@ -73,6 +73,10 @@ public class CloneProcedure extends ProcedureBase {
                 @ArgumentHint(
                         name = "clone_from",
                         type = @DataTypeHint("STRING"),
+                        isOptional = true),
+                @ArgumentHint(
+                        name = "meta_only",
+                        type = @DataTypeHint("BOOLEAN"),
                         isOptional = true)
             })
     public String[] call(
@@ -88,7 +92,8 @@ public class CloneProcedure extends ProcedureBase {
             String includedTablesStr,
             String excludedTablesStr,
             String preferFileFormat,
-            String cloneFrom)
+            String cloneFrom,
+            Boolean metaOnly)
             throws Exception {
         Map<String, String> sourceCatalogConfig =
                 new HashMap<>(optionalConfigMap(sourceCatalogConfigStr));
@@ -118,7 +123,8 @@ public class CloneProcedure extends ProcedureBase {
                         includedTables,
                         excludedTables,
                         preferFileFormat,
-                        cloneFrom);
+                        cloneFrom,
+                        metaOnly != null && metaOnly);
         return execute(procedureContext, action, "Clone Job");
     }
 
diff --git 
a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/CloneActionITCase.java
 
b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/CloneActionITCase.java
index 01640c6623..943732f256 100644
--- 
a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/CloneActionITCase.java
+++ 
b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/CloneActionITCase.java
@@ -1012,6 +1012,64 @@ public class CloneActionITCase extends ActionITCaseBase {
         Assertions.assertThatList(r1).containsExactlyInAnyOrderElementsOf(r2);
     }
 
+    @Test
+    public void testMigrateWithMetaOnly() throws Exception {
+        String format = "avro";
+        String dbName = "hivedb" + StringUtils.randomNumericString(10);
+        String tableName = "hivetable" + StringUtils.randomNumericString(10);
+
+        TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().build();
+        tEnv.executeSql("CREATE CATALOG HIVE WITH ('type'='hive')");
+        tEnv.useCatalog("HIVE");
+        tEnv.getConfig().setSqlDialect(SqlDialect.HIVE);
+        tEnv.executeSql("CREATE DATABASE " + dbName);
+        sql(
+                tEnv,
+                "CREATE TABLE %s.%s (id STRING) PARTITIONED BY (id2 INT, id3 
INT)"
+                        + "STORED AS %s ",
+                dbName,
+                tableName,
+                format);
+        sql(tEnv, "INSERT INTO %s.%s VALUES %s", dbName, tableName, data(100));
+
+        tEnv.getConfig().setSqlDialect(SqlDialect.DEFAULT);
+        tEnv.executeSql("CREATE CATALOG PAIMON_GE WITH 
('type'='paimon-generic')");
+        tEnv.useCatalog("PAIMON_GE");
+
+        tEnv.executeSql(
+                "CREATE CATALOG PAIMON WITH ('type'='paimon', 'warehouse' = '" 
+ warehouse + "')");
+        tEnv.useCatalog("PAIMON");
+        tEnv.executeSql("CREATE DATABASE test");
+
+        List<String> args =
+                new ArrayList<>(
+                        Arrays.asList(
+                                "clone",
+                                "--database",
+                                dbName,
+                                "--table",
+                                tableName,
+                                "--catalog_conf",
+                                "metastore=hive",
+                                "--catalog_conf",
+                                "uri=thrift://localhost:" + PORT,
+                                "--target_database",
+                                "test",
+                                "--target_table",
+                                "test_table",
+                                "--target_catalog_conf",
+                                "warehouse=" + warehouse,
+                                "--meta_only",
+                                "true"));
+
+        createAction(CloneAction.class, args).run();
+        FileStoreTable paimonTable =
+                paimonTable(tEnv, "PAIMON", Identifier.create("test", 
"test_table"));
+        // table exists but no data
+        assertThat(paimonTable.schema().fieldNames()).containsExactly("id", 
"id2", "id3");
+        assertThat(paimonTable.snapshotManager().earliestSnapshot()).isNull();
+    }
+
     private String[] ddls(String format) {
         // has primary key
         String ddl0 =

Reply via email to