|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.paimon.flink.clone; |
| 20 | + |
| 21 | +import org.apache.paimon.catalog.Catalog; |
| 22 | +import org.apache.paimon.catalog.DelegateCatalog; |
| 23 | +import org.apache.paimon.catalog.Identifier; |
| 24 | +import org.apache.paimon.flink.action.CloneAction; |
| 25 | +import org.apache.paimon.flink.sink.FlinkStreamPartitioner; |
| 26 | +import org.apache.paimon.hive.HiveCatalog; |
| 27 | +import org.apache.paimon.utils.StringUtils; |
| 28 | + |
| 29 | +import org.apache.commons.collections.CollectionUtils; |
| 30 | +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; |
| 31 | +import org.apache.flink.api.java.tuple.Tuple2; |
| 32 | +import org.apache.flink.streaming.api.datastream.DataStream; |
| 33 | +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; |
| 34 | +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; |
| 35 | +import org.slf4j.Logger; |
| 36 | +import org.slf4j.LoggerFactory; |
| 37 | + |
| 38 | +import javax.annotation.Nullable; |
| 39 | + |
| 40 | +import java.util.ArrayList; |
| 41 | +import java.util.List; |
| 42 | +import java.util.Map; |
| 43 | + |
| 44 | +import static org.apache.paimon.utils.Preconditions.checkArgument; |
| 45 | +import static org.apache.paimon.utils.Preconditions.checkState; |
| 46 | + |
| 47 | +/** Utils for building {@link CloneAction} for append tables. */ |
| 48 | +public class CloneHiveTableUtils { |
| 49 | + |
| 50 | + private static final Logger LOG = LoggerFactory.getLogger(CloneHiveTableUtils.class); |
| 51 | + |
| 52 | + public static DataStream<Tuple2<Identifier, Identifier>> buildSource( |
| 53 | + String sourceDatabase, |
| 54 | + String sourceTableName, |
| 55 | + String targetDatabase, |
| 56 | + String targetTableName, |
| 57 | + Catalog sourceCatalog, |
| 58 | + @Nullable List<String> excludedTables, |
| 59 | + StreamExecutionEnvironment env) |
| 60 | + throws Exception { |
| 61 | + List<Tuple2<Identifier, Identifier>> result = new ArrayList<>(); |
| 62 | + HiveCatalog hiveCatalog = getRootHiveCatalog(sourceCatalog); |
| 63 | + if (StringUtils.isNullOrWhitespaceOnly(sourceDatabase)) { |
| 64 | + checkArgument( |
| 65 | + StringUtils.isNullOrWhitespaceOnly(sourceTableName), |
| 66 | + "sourceTableName must be blank when database is null."); |
| 67 | + checkArgument( |
| 68 | + StringUtils.isNullOrWhitespaceOnly(targetDatabase), |
| 69 | + "targetDatabase must be blank when clone all tables in a catalog."); |
| 70 | + checkArgument( |
| 71 | + StringUtils.isNullOrWhitespaceOnly(targetTableName), |
| 72 | + "targetTableName must be blank when clone all tables in a catalog."); |
| 73 | + |
| 74 | + for (Identifier identifier : |
| 75 | + org.apache.paimon.hive.clone.HiveCloneUtils.listTables( |
| 76 | + hiveCatalog, excludedTables)) { |
| 77 | + result.add(new Tuple2<>(identifier, identifier)); |
| 78 | + } |
| 79 | + } else if (StringUtils.isNullOrWhitespaceOnly(sourceTableName)) { |
| 80 | + checkArgument( |
| 81 | + !StringUtils.isNullOrWhitespaceOnly(targetDatabase), |
| 82 | + "targetDatabase must not be blank when clone all tables in a database."); |
| 83 | + checkArgument( |
| 84 | + StringUtils.isNullOrWhitespaceOnly(targetTableName), |
| 85 | + "targetTableName must be blank when clone all tables in a catalog."); |
| 86 | + |
| 87 | + for (Identifier identifier : |
| 88 | + org.apache.paimon.hive.clone.HiveCloneUtils.listTables( |
| 89 | + hiveCatalog, sourceDatabase, excludedTables)) { |
| 90 | + result.add( |
| 91 | + new Tuple2<>( |
| 92 | + identifier, |
| 93 | + Identifier.create(targetDatabase, identifier.getObjectName()))); |
| 94 | + } |
| 95 | + } else { |
| 96 | + checkArgument( |
| 97 | + !StringUtils.isNullOrWhitespaceOnly(targetDatabase), |
| 98 | + "targetDatabase must not be blank when clone a table."); |
| 99 | + checkArgument( |
| 100 | + !StringUtils.isNullOrWhitespaceOnly(targetTableName), |
| 101 | + "targetTableName must not be blank when clone a table."); |
| 102 | + checkArgument( |
| 103 | + CollectionUtils.isEmpty(excludedTables), |
| 104 | + "excludedTables must be empty when clone a single table."); |
| 105 | + result.add( |
| 106 | + new Tuple2<>( |
| 107 | + Identifier.create(sourceDatabase, sourceTableName), |
| 108 | + Identifier.create(targetDatabase, targetTableName))); |
| 109 | + } |
| 110 | + |
| 111 | + checkState(!result.isEmpty(), "Didn't find any table in source catalog."); |
| 112 | + |
| 113 | + if (LOG.isDebugEnabled()) { |
| 114 | + LOG.debug("The clone identifiers of source table and target table are: {}", result); |
| 115 | + } |
| 116 | + return env.fromCollection(result).forceNonParallel(); |
| 117 | + } |
| 118 | + |
| 119 | + public static HiveCatalog getRootHiveCatalog(Catalog catalog) { |
| 120 | + Catalog rootCatalog = DelegateCatalog.rootCatalog(catalog); |
| 121 | + checkArgument( |
| 122 | + rootCatalog instanceof HiveCatalog, |
| 123 | + "Only support HiveCatalog now but found %s.", |
| 124 | + rootCatalog.getClass().getName()); |
| 125 | + return (HiveCatalog) rootCatalog; |
| 126 | + } |
| 127 | + |
| 128 | + public static void build( |
| 129 | + StreamExecutionEnvironment env, |
| 130 | + Catalog sourceCatalog, |
| 131 | + String sourceDatabase, |
| 132 | + String sourceTableName, |
| 133 | + Map<String, String> sourceCatalogConfig, |
| 134 | + String targetDatabase, |
| 135 | + String targetTableName, |
| 136 | + Map<String, String> targetCatalogConfig, |
| 137 | + int parallelism, |
| 138 | + @Nullable String whereSql, |
| 139 | + @Nullable List<String> excludedTables) |
| 140 | + throws Exception { |
| 141 | + // list source tables |
| 142 | + DataStream<Tuple2<Identifier, Identifier>> source = |
| 143 | + buildSource( |
| 144 | + sourceDatabase, |
| 145 | + sourceTableName, |
| 146 | + targetDatabase, |
| 147 | + targetTableName, |
| 148 | + sourceCatalog, |
| 149 | + excludedTables, |
| 150 | + env); |
| 151 | + |
| 152 | + DataStream<Tuple2<Identifier, Identifier>> partitionedSource = |
| 153 | + FlinkStreamPartitioner.partition( |
| 154 | + source, new ShuffleIdentifierByTableComputer(), parallelism); |
| 155 | + |
| 156 | + // create target table, list files and group by <table, partition> |
| 157 | + DataStream<CloneFileInfo> files = |
| 158 | + partitionedSource |
| 159 | + .process( |
| 160 | + new ListCloneFilesFunction( |
| 161 | + sourceCatalogConfig, targetCatalogConfig, whereSql)) |
| 162 | + .name("List Files") |
| 163 | + .setParallelism(parallelism); |
| 164 | + |
| 165 | + // copy files and commit |
| 166 | + DataStream<DataFileInfo> dataFile = |
| 167 | + files.rebalance() |
| 168 | + .process(new CloneFilesFunction(sourceCatalogConfig, targetCatalogConfig)) |
| 169 | + .name("Copy Files") |
| 170 | + .setParallelism(parallelism); |
| 171 | + |
| 172 | + DataStream<DataFileInfo> partitionedDataFile = |
| 173 | + FlinkStreamPartitioner.partition( |
| 174 | + dataFile, new ShuffleDataFileByTableComputer(), parallelism); |
| 175 | + |
| 176 | + DataStream<Long> committed = |
| 177 | + partitionedDataFile |
| 178 | + .transform( |
| 179 | + "Commit table", |
| 180 | + BasicTypeInfo.LONG_TYPE_INFO, |
| 181 | + new CommitTableOperator(targetCatalogConfig)) |
| 182 | + .setParallelism(parallelism); |
| 183 | + committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); |
| 184 | + } |
| 185 | +} |
0 commit comments