Skip to content

Commit

Permalink
[flink] Support compact procedure (apache#2013)
Browse files Browse the repository at this point in the history
(cherry picked from commit f31cd18)
  • Loading branch information
yuzelin authored and pongandnoon committed Oct 25, 2023
1 parent a650c89 commit 81b3699
Show file tree
Hide file tree
Showing 35 changed files with 542 additions and 282 deletions.
4 changes: 2 additions & 2 deletions .github/workflows/e2e-tests-1.16-jdk11.yml
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ jobs:
. .github/workflows/utils.sh
jvm_timezone=$(random_timezone)
echo "JVM timezone is set to $jvm_timezone"
mvn -T 1C -B clean install -DskipTests
mvn -T 1C -B test -pl paimon-e2e-tests -Duser.timezone=$jvm_timezone
mvn -T 1C -B clean install -DskipTests -Pflink-1.16
mvn -T 1C -B test -pl paimon-e2e-tests -Duser.timezone=$jvm_timezone -Pflink-1.16
env:
MAVEN_OPTS: -Xmx4096m
Original file line number Diff line number Diff line change
Expand Up @@ -157,7 +157,11 @@ Map<String, Map<String, Path>> allTablePaths() {
}
}

protected abstract String warehouse();
public abstract String warehouse();

public Map<String, String> options() {
return catalogOptions;
}

protected abstract TableSchema getDataTableSchema(Identifier identifier)
throws TableNotExistException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -231,7 +231,7 @@ private static String database(Path path) {
public void close() throws Exception {}

@Override
protected String warehouse() {
public String warehouse() {
return warehouse.toString();
}
}
85 changes: 85 additions & 0 deletions paimon-flink/paimon-flink-1.18/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.paimon</groupId>
<artifactId>paimon-flink</artifactId>
<version>0.6-SNAPSHOT</version>
</parent>

<packaging>jar</packaging>

<artifactId>paimon-flink-1.18</artifactId>
<name>Paimon : Flink : 1.18</name>

<properties>
<flink.version>1.18-SNAPSHOT</flink.version>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.paimon</groupId>
<artifactId>paimon-flink-common</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.paimon</groupId>
<artifactId>paimon-flink-cdc</artifactId>
<version>${project.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<id>shade-paimon</id>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<artifactSet>
<includes combine.children="append">
<include>org.apache.paimon:paimon-flink-common</include>
<include>org.apache.paimon:paimon-flink-cdc</include>
</includes>
</artifactSet>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
Expand Up @@ -202,11 +202,6 @@ public Map<String, String> tableConfig() {
return tableConfig;
}

@VisibleForTesting
public Map<String, String> catalogConfig() {
return catalogConfig;
}

// ------------------------------------------------------------------------
// Flink run methods
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -203,11 +203,6 @@ public Map<String, String> tableConfig() {
return tableConfig;
}

@VisibleForTesting
public Map<String, String> catalogConfig() {
return catalogConfig;
}

// ------------------------------------------------------------------------
// Flink run methods
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,11 +184,6 @@ public Map<String, String> tableConfig() {
return tableConfig;
}

@VisibleForTesting
public Map<String, String> catalogConfig() {
return catalogConfig;
}

// ------------------------------------------------------------------------
// Flink run methods
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -201,11 +201,6 @@ public Map<String, String> tableConfig() {
return tableConfig;
}

@VisibleForTesting
public Map<String, String> catalogConfig() {
return catalogConfig;
}

// ------------------------------------------------------------------------
// Flink run methods
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -372,11 +372,6 @@ public List<Identifier> excludedTables() {
return excludedTables;
}

@VisibleForTesting
public Map<String, String> catalogConfig() {
return catalogConfig;
}

@VisibleForTesting
public Map<String, String> tableConfig() {
return tableConfig;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -280,11 +280,6 @@ public Map<String, String> tableConfig() {
return tableConfig;
}

@VisibleForTesting
public Map<String, String> catalogConfig() {
return catalogConfig;
}

// ------------------------------------------------------------------------
// Flink run methods
// ------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package org.apache.paimon.flink.action.cdc;

import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.flink.action.ActionBase;
import org.apache.paimon.flink.action.ActionITCaseBase;
import org.apache.paimon.table.FileStoreTable;
Expand All @@ -28,7 +27,11 @@
import org.apache.paimon.types.RowType;

import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -50,9 +53,19 @@ public class CdcActionITCaseBase extends ActionITCaseBase {

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

protected FileStoreTable getFileStoreTable(String tableName) throws Exception {
Identifier identifier = Identifier.create(database, tableName);
return (FileStoreTable) catalog.getTable(identifier);
protected StreamExecutionEnvironment env;

@BeforeEach
public void setEnv() {
env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(2);
env.enableCheckpointing(1000);
env.setRestartStrategy(RestartStrategies.noRestart());
}

@AfterEach
public void closeEnv() throws Exception {
env.close();
}

protected void waitingTables(String... tables) throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -566,8 +566,7 @@ public void testCatalogAndTableConfig() {
.withTableConfig(Collections.singletonMap("table-key", "table-value"))
.build();

assertThat(action.catalogConfig())
.containsExactlyEntriesOf(Collections.singletonMap("catalog-key", "catalog-value"));
assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value");
assertThat(action.tableConfig())
.containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value"));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -953,8 +953,7 @@ public void testCatalogAndTableConfig() {
.withTableConfig(Collections.singletonMap("table-key", "table-value"))
.build();

assertThat(action.catalogConfig())
.containsExactlyEntriesOf(Collections.singletonMap("catalog-key", "catalog-value"));
assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value");
assertThat(action.tableConfig())
.containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value"));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,8 +126,7 @@ public void testCatalogAndTableConfig() {
.withTableConfig(Collections.singletonMap("table-key", "table-value"))
.build();

assertThat(action.catalogConfig())
.containsExactlyEntriesOf(Collections.singletonMap("catalog-key", "catalog-value"));
assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value");
assertThat(action.tableConfig())
.containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value"));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -158,8 +158,7 @@ public void testCatalogAndTableConfig() {
.withTableConfig(Collections.singletonMap("table-key", "table-value"))
.build();

assertThat(action.catalogConfig())
.containsExactlyEntriesOf(Collections.singletonMap("catalog-key", "catalog-value"));
assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value");
assertThat(action.tableConfig())
.containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value"));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1190,8 +1190,7 @@ public void testCatalogAndTableConfig() {
.withTableConfig(Collections.singletonMap("table-key", "table-value"))
.build();

assertThat(action.catalogConfig())
.containsExactlyEntriesOf(Collections.singletonMap("catalog-key", "catalog-value"));
assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value");
assertThat(action.tableConfig())
.containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value"));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -851,8 +851,7 @@ public void testCatalogAndTableConfig() {
.withTableConfig(Collections.singletonMap("table-key", "table-value"))
.build();

assertThat(action.catalogConfig())
.containsExactlyEntriesOf(Collections.singletonMap("catalog-key", "catalog-value"));
assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value");
assertThat(action.tableConfig())
.containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value"));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -889,7 +889,7 @@ public List<String> listProcedures(String dbName)
*/
public Procedure getProcedure(ObjectPath procedurePath)
throws ProcedureNotExistException, CatalogException {
return ProcedureUtil.getProcedure(procedurePath.getObjectName())
return ProcedureUtil.getProcedure(catalog, procedurePath.getObjectName())
.orElseThrow(() -> new ProcedureNotExistException(name, procedurePath));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.paimon.flink.action;

import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.flink.FlinkCatalog;
import org.apache.paimon.flink.FlinkCatalogFactory;
Expand All @@ -43,17 +44,14 @@
/** Abstract base of {@link Action} for table. */
public abstract class ActionBase implements Action {

private final Options catalogOptions;

protected final Map<String, String> catalogConfig;
protected final Options catalogOptions;
protected final Catalog catalog;
protected final FlinkCatalog flinkCatalog;
protected final String catalogName = "paimon-" + UUID.randomUUID();
protected final StreamExecutionEnvironment env;
protected final StreamTableEnvironment batchTEnv;

public ActionBase(String warehouse, Map<String, String> catalogConfig) {
this.catalogConfig = catalogConfig;
catalogOptions = Options.fromMap(catalogConfig);
catalogOptions.set(CatalogOptions.WAREHOUSE, warehouse);

Expand Down Expand Up @@ -110,4 +108,9 @@ protected boolean compatibleCheck(List<DataType> actualTypes, List<DataType> exp

return true;
}

@VisibleForTesting
public Map<String, String> catalogConfig() {
return catalogOptions.toMap();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.utils.MultipleParameterTool;

import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Optional;
Expand All @@ -44,24 +43,10 @@ public Optional<Action> create(MultipleParameterTool params) {

CompactAction action;
if (params.has("order-strategy")) {
SortCompactAction sortCompactAction =
new SortCompactAction(tablePath.f0, tablePath.f1, tablePath.f2, catalogConfig);

String strategy = params.get("order-strategy");
sortCompactAction.withOrderStrategy(strategy);

if (params.has("order-by")) {
String sqlOrderBy = params.get("order-by");
if (sqlOrderBy == null) {
throw new IllegalArgumentException("Please specify \"order-by\".");
}
sortCompactAction.withOrderColumns(Arrays.asList(sqlOrderBy.split(",")));
} else {
throw new IllegalArgumentException(
"Please specify order columns in parameter --order-by.");
}

action = sortCompactAction;
action =
new SortCompactAction(tablePath.f0, tablePath.f1, tablePath.f2, catalogConfig)
.withOrderStrategy(params.get("order-strategy"))
.withOrderColumns(getRequiredValue(params, "order-by").split(","));
} else {
action = new CompactAction(tablePath.f0, tablePath.f1, tablePath.f2, catalogConfig);
}
Expand Down
Loading

0 comments on commit 81b3699

Please sign in to comment.