-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
1 parent
3a7d030
commit 034753f
Showing
19 changed files
with
1,483 additions
and
50 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,116 @@ | ||
--- | ||
title: "Clone Tables" | ||
weight: 3 | ||
type: docs | ||
aliases: | ||
- /migration/clone-tables.html | ||
--- | ||
<!-- | ||
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. | ||
--> | ||
|
||
# Clone Tables | ||
|
||
Paimon supports clone tables of the latest Snapshot for data migration. | ||
|
||
{{< hint info >}} | ||
1、Clone Tables only support batch mode yet. Please use -D execution.runtime-mode=batch or -yD execution.runtime-mode=batch (for the ON-YARN scenario) to run clone job. | ||
|
||
2、If you want clone job runs quickly, you can add parameter parallelism. | ||
|
||
3、Only support Flink now. | ||
{{< /hint >}} | ||
|
||
## Clone Table | ||
The target table needs to be a non-existent table, and it will have the exact same schema (only the schema for current snapshot) as the source table. | ||
|
||
To run a Flink batch job for clone, follow these instructions. | ||
|
||
### LatestSnapshot | ||
Clone the latest snapshot of the source table, copying all the files required for the snapshot to the new target table. | ||
|
||
{{< tabs "clone-tables" >}} | ||
|
||
{{< tab "Flink" >}} | ||
|
||
Flink SQL currently does not support statements related to clone, so we have to submit the clone job through `flink run`. | ||
|
||
Run the following command to submit a clone job for the table's latest Snapshot. | ||
|
||
```bash | ||
<FLINK_HOME>/bin/flink run \ | ||
/path/to/paimon-flink-action-{{< version >}}.jar \ | ||
clone \ | ||
--warehouse <source-warehouse-path> \ | ||
[--database <source-database-name>] \ | ||
[--table <source-table-name>] \ | ||
[--catalog_conf <source-paimon-catalog-conf> [--catalog_conf <source-paimon-catalog-conf> ...]] \ | ||
--target_warehouse <target-warehouse-path> \ | ||
--target_database <target-database> \ | ||
--target_table <target-table-name> \ | ||
[--target_catalog_conf <target-paimon-catalog-conf> [--target_catalog_conf <target-paimon-catalog-conf> ...]] | ||
[--parallelism 128 ] | ||
``` | ||
{{< hint info >}} | ||
1、If the database parameter is not passed, then all tables of all databases will be cloned. | ||
2、If the table parameter is not passed, then all tables of the database will be cloned. | ||
{{< /hint >}} | ||
Example: clone table latest Snapshot. | ||
```bash | ||
<FLINK_HOME>/bin/flink run \ | ||
/path/to/paimon-flink-action-{{< version >}}.jar \ | ||
clone \ | ||
--warehouse s3:///path/to/warehouse_source \ | ||
--database test_db \ | ||
--table test_table \ | ||
--catalog_conf s3.endpoint=https://****.com \ | ||
--catalog_conf s3.access-key=***** \ | ||
--catalog_conf s3.secret-key=***** \ | ||
--target_warehouse s3:///path/to/warehouse_target \ | ||
--target_database test_db \ | ||
--target_table test_table \ | ||
--target_catalog_conf s3.endpoint=https://****.com \ | ||
--target_catalog_conf s3.access-key=***** \ | ||
--target_catalog_conf s3.secret-key=***** | ||
``` | ||
For more usage of the clone action, see | ||
```bash | ||
<FLINK_HOME>/bin/flink run \ | ||
/path/to/paimon-flink-action-{{< version >}}.jar \ | ||
clone --help | ||
``` | ||
{{< /tab >}} | ||
{{< tab "Flink Procedure" >}} | ||
Run the following command to submit a clone job for the table's latest Snapshot. | ||
```bash | ||
CALL sys.clone('source_warehouse', 'source_database', 'source_table', '', 'target_warehouse', 'target_database', 'target_table', '', '') | ||
``` | ||
{{< /tab >}} | ||
{{< /tabs >}} | ||
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
144 changes: 144 additions & 0 deletions
144
...n-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,144 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.paimon.flink.action; | ||
|
||
import org.apache.paimon.flink.clone.CloneFileInfo; | ||
import org.apache.paimon.flink.clone.CloneSourceBuilder; | ||
import org.apache.paimon.flink.clone.CopyFileOperator; | ||
import org.apache.paimon.flink.clone.PickFilesForCloneOperator; | ||
import org.apache.paimon.flink.clone.SnapshotHintChannelComputer; | ||
import org.apache.paimon.flink.clone.SnapshotHintOperator; | ||
import org.apache.paimon.flink.sink.FlinkStreamPartitioner; | ||
import org.apache.paimon.options.CatalogOptions; | ||
|
||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
import org.apache.flink.api.java.tuple.Tuple2; | ||
import org.apache.flink.streaming.api.datastream.DataStream; | ||
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; | ||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
import org.apache.flink.streaming.api.functions.sink.DiscardingSink; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
|
||
import static org.apache.paimon.utils.Preconditions.checkNotNull; | ||
import static org.apache.paimon.utils.StringUtils.isBlank; | ||
|
||
/** Snapshot/Tag/Table clone action for Flink. */ | ||
public class CloneAction extends ActionBase { | ||
|
||
private final int parallelism; | ||
|
||
private Map<String, String> sourceCatalogConfig = new HashMap<>(); | ||
private final String database; | ||
private final String tableName; | ||
|
||
private Map<String, String> targetCatalogConfig = new HashMap<>(); | ||
private final String targetDatabase; | ||
private final String targetTableName; | ||
|
||
public CloneAction( | ||
String warehouse, | ||
String database, | ||
String tableName, | ||
Map<String, String> sourceCatalogConfig, | ||
String targetWarehouse, | ||
String targetDatabase, | ||
String targetTableName, | ||
Map<String, String> targetCatalogConfig, | ||
String parallelismStr) { | ||
super(warehouse, sourceCatalogConfig); | ||
|
||
checkNotNull(warehouse, "warehouse must not be null."); | ||
checkNotNull(targetWarehouse, "targetWarehouse must not be null."); | ||
|
||
this.parallelism = | ||
isBlank(parallelismStr) ? env.getParallelism() : Integer.parseInt(parallelismStr); | ||
|
||
if (!sourceCatalogConfig.isEmpty()) { | ||
this.sourceCatalogConfig = sourceCatalogConfig; | ||
} | ||
this.sourceCatalogConfig.put(CatalogOptions.WAREHOUSE.key(), warehouse); | ||
this.database = database; | ||
this.tableName = tableName; | ||
|
||
if (!targetCatalogConfig.isEmpty()) { | ||
this.targetCatalogConfig = targetCatalogConfig; | ||
} | ||
this.targetCatalogConfig.put(CatalogOptions.WAREHOUSE.key(), targetWarehouse); | ||
this.targetDatabase = targetDatabase; | ||
this.targetTableName = targetTableName; | ||
} | ||
|
||
// ------------------------------------------------------------------------ | ||
// Java API | ||
// ------------------------------------------------------------------------ | ||
|
||
@Override | ||
public void build() { | ||
buildCloneFlinkJob(env); | ||
} | ||
|
||
private void buildCloneFlinkJob(StreamExecutionEnvironment env) { | ||
DataStream<Tuple2<String, String>> cloneSource = | ||
new CloneSourceBuilder( | ||
env, | ||
sourceCatalogConfig, | ||
database, | ||
tableName, | ||
targetDatabase, | ||
targetTableName) | ||
.build(); | ||
|
||
SingleOutputStreamOperator<CloneFileInfo> pickFilesForClone = | ||
cloneSource | ||
.transform( | ||
"Pick Files", | ||
TypeInformation.of(CloneFileInfo.class), | ||
new PickFilesForCloneOperator( | ||
sourceCatalogConfig, targetCatalogConfig)) | ||
.forceNonParallel(); | ||
|
||
SingleOutputStreamOperator<CloneFileInfo> copyFiles = | ||
pickFilesForClone | ||
.rebalance() | ||
.transform( | ||
"Copy Files", | ||
TypeInformation.of(CloneFileInfo.class), | ||
new CopyFileOperator(sourceCatalogConfig, targetCatalogConfig)) | ||
.setParallelism(parallelism); | ||
|
||
SingleOutputStreamOperator<CloneFileInfo> snapshotHintOperator = | ||
FlinkStreamPartitioner.partition( | ||
copyFiles, new SnapshotHintChannelComputer(), parallelism) | ||
.transform( | ||
"Recreate Snapshot Hint", | ||
TypeInformation.of(CloneFileInfo.class), | ||
new SnapshotHintOperator(targetCatalogConfig)) | ||
.setParallelism(parallelism); | ||
|
||
snapshotHintOperator.addSink(new DiscardingSink<>()).name("end").setParallelism(1); | ||
} | ||
|
||
@Override | ||
public void run() throws Exception { | ||
build(); | ||
execute("Clone job"); | ||
} | ||
} |
88 changes: 88 additions & 0 deletions
88
.../paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneActionFactory.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,88 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.paimon.flink.action; | ||
|
||
import java.util.Optional; | ||
|
||
/** Factory to create {@link CloneAction}. */ | ||
public class CloneActionFactory implements ActionFactory { | ||
|
||
private static final String IDENTIFIER = "clone"; | ||
private static final String PARALLELISM = "parallelism"; | ||
private static final String TARGET_WAREHOUSE = "target_warehouse"; | ||
private static final String TARGET_DATABASE = "target_database"; | ||
private static final String TARGET_TABLE = "target_table"; | ||
private static final String TARGET_CATALOG_CONF = "target_catalog_conf"; | ||
|
||
@Override | ||
public String identifier() { | ||
return IDENTIFIER; | ||
} | ||
|
||
@Override | ||
public Optional<Action> create(MultipleParameterToolAdapter params) { | ||
CloneAction cloneAction = | ||
new CloneAction( | ||
params.get(WAREHOUSE), | ||
params.get(DATABASE), | ||
params.get(TABLE), | ||
optionalConfigMap(params, CATALOG_CONF), | ||
params.get(TARGET_WAREHOUSE), | ||
params.get(TARGET_DATABASE), | ||
params.get(TARGET_TABLE), | ||
optionalConfigMap(params, TARGET_CATALOG_CONF), | ||
params.get(PARALLELISM)); | ||
|
||
return Optional.of(cloneAction); | ||
} | ||
|
||
@Override | ||
public void printHelp() { | ||
System.out.println("Action \"clone\" runs a batch job for clone the latest Snapshot."); | ||
System.out.println(); | ||
|
||
System.out.println("Syntax:"); | ||
System.out.println( | ||
" clone --warehouse <warehouse_path>" | ||
+ "[--database <database_name>]" | ||
+ "[--table <table_name>]" | ||
+ "[--catalog_conf <source-paimon-catalog-conf> [--catalog_conf <source-paimon-catalog-conf> ...]]" | ||
+ "--target_warehouse <target_warehouse_path>" | ||
+ "--target_database <target_database_name> " | ||
+ "--target_table <target_table_name> " | ||
+ "[--target_catalog_conf <target-paimon-catalog-conf> [--target_catalog_conf <target-paimon-catalog-conf> ...]]"); | ||
|
||
System.out.println(); | ||
|
||
System.out.println("Examples:"); | ||
System.out.println( | ||
" clone --warehouse s3:///path1/from/warehouse " | ||
+ "--database test_db " | ||
+ "--table test_table " | ||
+ "--catalog_conf s3.endpoint=https://****.com " | ||
+ "--catalog_conf s3.access-key=***** " | ||
+ "--catalog_conf s3.secret-key=***** " | ||
+ "--target_warehouse s3:///path2/to/warehouse " | ||
+ "--target_database test_db_copy " | ||
+ "--target_table test_table_copy " | ||
+ "--target_catalog_conf s3.endpoint=https://****.com " | ||
+ "--target_catalog_conf s3.access-key=***** " | ||
+ "--target_catalog_conf s3.secret-key=***** "); | ||
} | ||
} |
Oops, something went wrong.