Skip to content

Commit

Permalink
support clone latest snapshot
Browse files Browse the repository at this point in the history
  • Loading branch information
wwj6591812 committed Apr 30, 2024
1 parent c796439 commit bd32ebd
Show file tree
Hide file tree
Showing 21 changed files with 1,624 additions and 50 deletions.
116 changes: 116 additions & 0 deletions docs/content/migration/clone-tables.md
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 >}}
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.clone;

import org.apache.paimon.fs.Path;

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

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

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

public Path getFilePathExcludeTableRoot() {
return filePathExcludeTableRoot;
}

public String sourceIdentifier() {
return sourceIdentifier;
}

public String targetIdentifier() {
return targetIdentifier;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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.clone;

import org.apache.paimon.fs.FileIO;
import org.apache.paimon.fs.Path;
import org.apache.paimon.utils.IOUtils;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;

/** Utility class for copy file. */
public class CopyFileUtils {

private static final Logger LOG = LoggerFactory.getLogger(CopyFileUtils.class);

public static void copyFile(
CloneFileInfo cloneFileInfo,
FileIO sourceTableFileIO,
FileIO targetTableFileIO,
Path sourceTableRootPath,
Path targetTableRootPath)
throws IOException {
Path filePathExcludeTableRoot = cloneFileInfo.getFilePathExcludeTableRoot();
Path sourcePath = new Path(sourceTableRootPath.toString() + filePathExcludeTableRoot);
Path targetPath = new Path(targetTableRootPath.toString() + filePathExcludeTableRoot);

if (targetTableFileIO.exists(targetPath)
&& targetTableFileIO.getFileSize(targetPath)
== sourceTableFileIO.getFileSize(sourcePath)) {
LOG.info(
"Skipping target file {} because it already exists and has the same size.",
targetPath);
return;
}

LOG.debug("Begin copy file from {} to {}.", sourcePath, targetPath);
IOUtils.copyBytes(
sourceTableFileIO.newInputStream(sourcePath),
targetTableFileIO.newOutputStream(targetPath, true));
LOG.debug("End copy file from {} to {}.", sourcePath, targetPath);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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.clone;

import org.apache.paimon.table.sink.ChannelComputer;

/** SnapshotHintChannelComputer. */
public class SnapshotHintChannelComputer implements ChannelComputer<CloneFileInfo> {

private static final long serialVersionUID = 1L;

private int numChannels;

@Override
public void setup(int numChannels) {
this.numChannels = numChannels;
}

@Override
public int channel(CloneFileInfo record) {
int hsh = 0;
for (int i = 0; i < record.targetIdentifier().length(); i++) {
hsh = (hsh * 131 + record.targetIdentifier().charAt(i)) % numChannels;
}
return hsh;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -499,8 +499,7 @@ private void deleteFileOrDirQuietly(Path path) {

/** A helper functional interface for method {@link #retryReadingFiles}. */
@FunctionalInterface
private interface ReaderWithIOException<T> {

interface ReaderWithIOException<T> {
T read() throws IOException;
}
}

0 comments on commit bd32ebd

Please sign in to comment.