Skip to content

Commit

Permalink
[flink][procedure] Support named argument in procedures without compa…
Browse files Browse the repository at this point in the history
…tibility issue
  • Loading branch information
yunfengzhou-hub committed Aug 28, 2024
1 parent 4d237c5 commit 5d623fd
Show file tree
Hide file tree
Showing 31 changed files with 1,307 additions and 512 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ public DataStream<Tuple2<String, String>> build() throws Exception {
private DataStream<Tuple2<String, String>> build(Catalog sourceCatalog) throws Exception {
List<Tuple2<String, String>> result = new ArrayList<>();

if (database == null) {
if (StringUtils.isBlank(database)) {
checkArgument(
StringUtils.isBlank(tableName),
"tableName must be blank when database is null.");
Expand All @@ -92,7 +92,7 @@ private DataStream<Tuple2<String, String>> build(Catalog sourceCatalog) throws E
result.add(new Tuple2<>(s, s));
}
}
} else if (tableName == null) {
} else if (StringUtils.isBlank(tableName)) {
checkArgument(
!StringUtils.isBlank(targetDatabase),
"targetDatabase must not be blank when clone all tables in a database.");
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* 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.procedure;

import org.apache.paimon.flink.action.CloneAction;

import org.apache.flink.table.annotation.ArgumentHint;
import org.apache.flink.table.annotation.DataTypeHint;
import org.apache.flink.table.annotation.ProcedureHint;
import org.apache.flink.table.procedure.ProcedureContext;

/** Clone Procedure. */
public class CloneProcedure extends ProcedureBase {
public static final String IDENTIFIER = "clone";

@ProcedureHint(
argument = {
@ArgumentHint(name = "warehouse", type = @DataTypeHint("STRING")),
@ArgumentHint(name = "database", type = @DataTypeHint("STRING"), isOptional = true),
@ArgumentHint(name = "table", type = @DataTypeHint("STRING"), isOptional = true),
@ArgumentHint(
name = "catalog_conf",
type = @DataTypeHint("STRING"),
isOptional = true),
@ArgumentHint(name = "target_warehouse", type = @DataTypeHint("STRING")),
@ArgumentHint(
name = "target_database",
type = @DataTypeHint("STRING"),
isOptional = true),
@ArgumentHint(
name = "target_table",
type = @DataTypeHint("STRING"),
isOptional = true),
@ArgumentHint(
name = "target_catalog_conf",
type = @DataTypeHint("STRING"),
isOptional = true),
@ArgumentHint(name = "parallelism", type = @DataTypeHint("INT"), isOptional = true)
})
public String[] call(
ProcedureContext procedureContext,
String warehouse,
String database,
String tableName,
String sourceCatalogConfigStr,
String targetWarehouse,
String targetDatabase,
String targetTableName,
String targetCatalogConfigStr,
Integer parallelismStr)
throws Exception {
CloneAction cloneAction =
new CloneAction(
warehouse,
database,
tableName,
optionalConfigMap(sourceCatalogConfigStr),
targetWarehouse,
targetDatabase,
targetTableName,
optionalConfigMap(targetCatalogConfigStr),
parallelismStr == null ? null : Integer.toString(parallelismStr));
return execute(procedureContext, cloneAction, "Clone Job");
}

@Override
public String identifier() {
return IDENTIFIER;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,9 @@
import org.apache.paimon.flink.action.CompactDatabaseAction;
import org.apache.paimon.utils.StringUtils;

import org.apache.flink.table.annotation.ArgumentHint;
import org.apache.flink.table.annotation.DataTypeHint;
import org.apache.flink.table.annotation.ProcedureHint;
import org.apache.flink.table.procedure.ProcedureContext;

import java.util.Map;
Expand Down Expand Up @@ -56,40 +59,26 @@ public class CompactDatabaseProcedure extends ProcedureBase {

public static final String IDENTIFIER = "compact_database";

public String[] call(ProcedureContext procedureContext) throws Exception {
return call(procedureContext, "");
}

public String[] call(ProcedureContext procedureContext, String includingDatabases)
throws Exception {
return call(procedureContext, includingDatabases, "");
}

public String[] call(ProcedureContext procedureContext, String includingDatabases, String mode)
throws Exception {
return call(procedureContext, includingDatabases, mode, "");
}

public String[] call(
ProcedureContext procedureContext,
String includingDatabases,
String mode,
String includingTables)
throws Exception {
return call(procedureContext, includingDatabases, mode, includingTables, "");
}

public String[] call(
ProcedureContext procedureContext,
String includingDatabases,
String mode,
String includingTables,
String excludingTables)
throws Exception {
return call(
procedureContext, includingDatabases, mode, includingTables, excludingTables, "");
}

@ProcedureHint(
argument = {
@ArgumentHint(
name = "including_databases",
type = @DataTypeHint("STRING"),
isOptional = true),
@ArgumentHint(name = "mode", type = @DataTypeHint("STRING"), isOptional = true),
@ArgumentHint(
name = "including_tables",
type = @DataTypeHint("STRING"),
isOptional = true),
@ArgumentHint(
name = "excluding_tables",
type = @DataTypeHint("STRING"),
isOptional = true),
@ArgumentHint(
name = "table_options",
type = @DataTypeHint("STRING"),
isOptional = true)
})
public String[] call(
ProcedureContext procedureContext,
String includingDatabases,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,9 @@
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.table.Table;

import org.apache.flink.table.annotation.ArgumentHint;
import org.apache.flink.table.annotation.DataTypeHint;
import org.apache.flink.table.annotation.ProcedureHint;
import org.apache.flink.table.procedure.ProcedureContext;

/**
Expand All @@ -40,6 +43,11 @@ public String identifier() {
return IDENTIFIER;
}

@ProcedureHint(
argument = {
@ArgumentHint(name = "table", type = @DataTypeHint("STRING")),
@ArgumentHint(name = "branch", type = @DataTypeHint("STRING"))
})
public String[] call(ProcedureContext procedureContext, String tableId, String branchName)
throws Catalog.TableNotExistException {
return innerCall(tableId, branchName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@
import org.apache.paimon.migrate.Migrator;
import org.apache.paimon.utils.ParameterUtils;

import org.apache.flink.table.annotation.ArgumentHint;
import org.apache.flink.table.annotation.DataTypeHint;
import org.apache.flink.table.annotation.ProcedureHint;
import org.apache.flink.table.procedure.ProcedureContext;

import java.util.List;
Expand All @@ -35,18 +38,19 @@ public String identifier() {
return "migrate_database";
}

public String[] call(
ProcedureContext procedureContext, String connector, String sourceDatabasePath)
throws Exception {
return call(procedureContext, connector, sourceDatabasePath, "");
}

@ProcedureHint(
argument = {
@ArgumentHint(name = "connector", type = @DataTypeHint("STRING")),
@ArgumentHint(name = "source_database", type = @DataTypeHint("STRING")),
@ArgumentHint(name = "options", type = @DataTypeHint("STRING"), isOptional = true)
})
public String[] call(
ProcedureContext procedureContext,
String connector,
String sourceDatabasePath,
String properties)
throws Exception {
properties = notnull(properties);
if (!(catalog instanceof HiveCatalog)) {
throw new IllegalArgumentException("Only support Hive Catalog");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@
import org.apache.paimon.hive.HiveCatalog;
import org.apache.paimon.migrate.Migrator;

import org.apache.flink.table.annotation.ArgumentHint;
import org.apache.flink.table.annotation.DataTypeHint;
import org.apache.flink.table.annotation.ProcedureHint;
import org.apache.flink.table.procedure.ProcedureContext;

import java.util.Collections;
Expand All @@ -35,23 +38,26 @@ public String identifier() {
return "migrate_file";
}

public String[] call(
ProcedureContext procedureContext,
String connector,
String sourceTablePath,
String targetPaimonTablePath)
throws Exception {
call(procedureContext, connector, sourceTablePath, targetPaimonTablePath, true);
return new String[] {"Success"};
}

@ProcedureHint(
argument = {
@ArgumentHint(name = "connector", type = @DataTypeHint("STRING")),
@ArgumentHint(name = "source_table", type = @DataTypeHint("STRING")),
@ArgumentHint(name = "target_table", type = @DataTypeHint("STRING")),
@ArgumentHint(
name = "delete_origin",
type = @DataTypeHint("BOOLEAN"),
isOptional = true)
})
public String[] call(
ProcedureContext procedureContext,
String connector,
String sourceTablePath,
String targetPaimonTablePath,
boolean deleteOrigin)
Boolean deleteOrigin)
throws Exception {
if (deleteOrigin == null) {
deleteOrigin = true;
}
migrateHandle(connector, sourceTablePath, targetPaimonTablePath, deleteOrigin);
return new String[] {"Success"};
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@
import org.apache.paimon.hive.HiveCatalog;
import org.apache.paimon.utils.ParameterUtils;

import org.apache.flink.table.annotation.ArgumentHint;
import org.apache.flink.table.annotation.DataTypeHint;
import org.apache.flink.table.annotation.ProcedureHint;
import org.apache.flink.table.procedure.ProcedureContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -39,18 +42,20 @@ public String identifier() {
return "migrate_table";
}

public String[] call(
ProcedureContext procedureContext, String connector, String sourceTablePath)
throws Exception {
return call(procedureContext, connector, sourceTablePath, "");
}

@ProcedureHint(
argument = {
@ArgumentHint(name = "connector", type = @DataTypeHint("STRING")),
@ArgumentHint(name = "source_table", type = @DataTypeHint("STRING")),
@ArgumentHint(name = "options", type = @DataTypeHint("STRING"), isOptional = true)
})
public String[] call(
ProcedureContext procedureContext,
String connector,
String sourceTablePath,
String properties)
throws Exception {
properties = notnull(properties);

if (!(catalog instanceof HiveCatalog)) {
throw new IllegalArgumentException("Only support Hive Catalog");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,12 @@

import javax.annotation.Nullable;

import java.util.Collections;
import java.util.HashMap;
import java.util.Map;

import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYNC;
import static org.apache.paimon.utils.ParameterUtils.parseKeyValueString;

/** Base implementation for flink {@link Procedure}. */
public abstract class ProcedureBase implements Procedure, Factory {
Expand All @@ -51,6 +56,10 @@ protected Table table(String tableId) throws Catalog.TableNotExistException {
return catalog.getTable(Identifier.fromString(tableId));
}

protected String notnull(@Nullable String arg) {
return arg == null ? "" : arg;
}

@Nullable
protected String nullable(String arg) {
return StringUtils.isBlank(arg) ? null : arg;
Expand Down Expand Up @@ -92,4 +101,16 @@ private String[] execute(JobClient jobClient, boolean dmlSync) {
return new String[] {"JobID=" + jobId};
}
}

protected Map<String, String> optionalConfigMap(String configStr) {
if (StringUtils.isBlank(configStr)) {
return Collections.emptyMap();
}

Map<String, String> config = new HashMap<>();
for (String kvString : configStr.split(";")) {
parseKeyValueString(config, kvString);
}
return config;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@
import org.apache.paimon.table.Table;

import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.annotation.ArgumentHint;
import org.apache.flink.table.annotation.DataTypeHint;
import org.apache.flink.table.annotation.ProcedureHint;
import org.apache.flink.table.procedure.ProcedureContext;

/**
Expand All @@ -41,6 +44,11 @@ public String identifier() {
return IDENTIFIER;
}

@ProcedureHint(
argument = {
@ArgumentHint(name = "table", type = @DataTypeHint("STRING")),
@ArgumentHint(name = "parallelism", type = @DataTypeHint("INT"))
})
public String[] call(ProcedureContext procedureContext, String tableId, int parallelism)
throws Exception {
Table table = catalog.getTable(Identifier.fromString(tableId));
Expand Down
Loading

0 comments on commit 5d623fd

Please sign in to comment.