Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

support clone latest snapshot #3287

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
82 changes: 82 additions & 0 deletions docs/content/migration/clone-tables.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
---
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 cloning tables for data migration.
Currently, only table files used by the latest snapshot will be cloned.

To clone a table, run the following command to submit a clone job.
If the table you clone is not modified at the same time, it is recommended to submit a Flink batch job for better performance.
However, if you want to clone the table while writing it at the same time, submit a Flink streaming job for automatic failure recovery.

```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 <parallelism>]
```

{{< hint info >}}
1. If `database` is not specified, all tables in all databases of the specified warehouse will be cloned.
2. If `table` is not specified, all tables of the specified database will be cloned.
{{< /hint >}}

Example: Clone `test_db.test_table` from source warehouse to target warehouse.

```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
```
Original file line number Diff line number Diff line change
Expand Up @@ -329,4 +329,13 @@ public Schema build() {
return new Schema(columns, partitionKeys, primaryKeys, options, comment);
}
}

public static Schema fromTableSchema(TableSchema tableSchema) {
return new Schema(
tableSchema.fields(),
tableSchema.partitionKeys(),
tableSchema.primaryKeys(),
tableSchema.options(),
tableSchema.comment());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,150 @@
/*
* 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;

/** The Latest Snapshot clone action for Flink. */
public class CloneAction extends ActionBase {

private final int parallelism;

private Map<String, String> sourceCatalogConfig;
private final String database;
private final String tableName;

private Map<String, String> targetCatalogConfig;
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);

this.sourceCatalogConfig = new HashMap<>();
if (!sourceCatalogConfig.isEmpty()) {
this.sourceCatalogConfig = sourceCatalogConfig;
}
wwj6591812 marked this conversation as resolved.
Show resolved Hide resolved
this.sourceCatalogConfig.put(CatalogOptions.WAREHOUSE.key(), warehouse);
this.database = database;
this.tableName = tableName;

this.targetCatalogConfig = new HashMap<>();
if (!targetCatalogConfig.isEmpty()) {
this.targetCatalogConfig = targetCatalogConfig;
}
wwj6591812 marked this conversation as resolved.
Show resolved Hide resolved
this.targetCatalogConfig.put(CatalogOptions.WAREHOUSE.key(), targetWarehouse);
this.targetDatabase = targetDatabase;
this.targetTableName = targetTableName;
}

// ------------------------------------------------------------------------
// Java API
// ------------------------------------------------------------------------

@Override
public void build() {
try {
buildCloneFlinkJob(env);
} catch (Exception e) {
throw new RuntimeException(e);
}
}

private void buildCloneFlinkJob(StreamExecutionEnvironment env) throws Exception {
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");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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> ...]] "
+ "[--parallelism <parallelism>]");

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=***** ");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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.clone;

/** The information of copy file. */
public class CloneFileInfo {

private final String filePathExcludeTableRoot;
private final String sourceIdentifier;
private final String targetIdentifier;

public CloneFileInfo(
String filePathExcludeTableRoot, String sourceIdentifier, String targetIdentifier) {
this.filePathExcludeTableRoot = filePathExcludeTableRoot;
this.sourceIdentifier = sourceIdentifier;
this.targetIdentifier = targetIdentifier;
}

public String getFilePathExcludeTableRoot() {
return filePathExcludeTableRoot;
}

public String getSourceIdentifier() {
return sourceIdentifier;
}

public String getTargetIdentifier() {
return targetIdentifier;
}

@Override
public String toString() {
return String.format(
"{ filePath: %s, sourceIdentifier: %s, targetIdentifier: %s }",
filePathExcludeTableRoot, sourceIdentifier, targetIdentifier);
}
}