[vitess] Add Vitess CDC connector (#456)
Co-authored-by: gintarasm <gintaras.matulas@gmail.com>pull/2153/head
parent
2b017346c3
commit
e349cb1a83
@ -0,0 +1,166 @@
|
|||||||
|
<?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">
|
||||||
|
<parent>
|
||||||
|
<artifactId>flink-cdc-connectors</artifactId>
|
||||||
|
<groupId>com.ververica</groupId>
|
||||||
|
<version>2.4-SNAPSHOT</version>
|
||||||
|
</parent>
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
|
<artifactId>flink-connector-vitess-cdc</artifactId>
|
||||||
|
<name>flink-connector-vitess-cdc</name>
|
||||||
|
<packaging>jar</packaging>
|
||||||
|
|
||||||
|
<dependencies>
|
||||||
|
|
||||||
|
<!-- Debezium dependencies -->
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.ververica</groupId>
|
||||||
|
<artifactId>flink-connector-debezium</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
<exclusions>
|
||||||
|
<exclusion>
|
||||||
|
<artifactId>kafka-log4j-appender</artifactId>
|
||||||
|
<groupId>org.apache.kafka</groupId>
|
||||||
|
</exclusion>
|
||||||
|
</exclusions>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>io.debezium</groupId>
|
||||||
|
<artifactId>debezium-connector-vitess</artifactId>
|
||||||
|
<version>${debezium.version}</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<!-- test dependencies on Debezium -->
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.ververica</groupId>
|
||||||
|
<artifactId>flink-connector-test-util</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>io.debezium</groupId>
|
||||||
|
<artifactId>debezium-core</artifactId>
|
||||||
|
<version>${debezium.version}</version>
|
||||||
|
<type>test-jar</type>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>mysql</groupId>
|
||||||
|
<artifactId>mysql-connector-java</artifactId>
|
||||||
|
<version>8.0.26</version>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
|
||||||
|
<!-- test dependencies on Flink -->
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.flink</groupId>
|
||||||
|
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
|
||||||
|
<version>${flink.version}</version>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.flink</groupId>
|
||||||
|
<artifactId>flink-table-runtime</artifactId>
|
||||||
|
<version>${flink.version}</version>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.flink</groupId>
|
||||||
|
<artifactId>flink-test-utils</artifactId>
|
||||||
|
<version>${flink.version}</version>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.flink</groupId>
|
||||||
|
<artifactId>flink-core</artifactId>
|
||||||
|
<version>${flink.version}</version>
|
||||||
|
<type>test-jar</type>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.flink</groupId>
|
||||||
|
<artifactId>flink-streaming-java</artifactId>
|
||||||
|
<version>${flink.version}</version>
|
||||||
|
<type>test-jar</type>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.flink</groupId>
|
||||||
|
<artifactId>flink-table-common</artifactId>
|
||||||
|
<version>${flink.version}</version>
|
||||||
|
<type>test-jar</type>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.flink</groupId>
|
||||||
|
<artifactId>flink-tests</artifactId>
|
||||||
|
<version>${flink.version}</version>
|
||||||
|
<type>test-jar</type>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.flink</groupId>
|
||||||
|
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
|
||||||
|
<version>${flink.version}</version>
|
||||||
|
<type>test-jar</type>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<!-- test dependencies on TestContainers -->
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.testcontainers</groupId>
|
||||||
|
<artifactId>testcontainers</artifactId>
|
||||||
|
<version>${testcontainers.version}</version>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.testcontainers</groupId>
|
||||||
|
<artifactId>jdbc</artifactId>
|
||||||
|
<version>${testcontainers.version}</version>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.jayway.jsonpath</groupId>
|
||||||
|
<artifactId>json-path</artifactId>
|
||||||
|
<version>2.4.0</version>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
|
</dependencies>
|
||||||
|
|
||||||
|
</project>
|
@ -0,0 +1,46 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.ververica.cdc.connectors.vitess;
|
||||||
|
|
||||||
|
import org.apache.flink.shaded.guava30.com.google.common.collect.Maps;
|
||||||
|
|
||||||
|
import com.ververica.cdc.debezium.Validator;
|
||||||
|
import io.debezium.connector.vitess.VitessConnector;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
|
/** The validator for Vitess. */
|
||||||
|
public class VitessValidator implements Validator, Serializable {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
private final Map<String, String> configuration;
|
||||||
|
|
||||||
|
public VitessValidator(Properties properties) {
|
||||||
|
this.configuration = Maps.fromProperties(properties);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void validate() {
|
||||||
|
VitessConnector c = new VitessConnector();
|
||||||
|
c.validate(configuration);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,41 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.ververica.cdc.connectors.vitess.config;
|
||||||
|
|
||||||
|
/** The type of Tablet (hence MySQL) from which to stream the changes. */
|
||||||
|
public enum TabletType {
|
||||||
|
/** Streaming from the master MySQL instance. */
|
||||||
|
MASTER,
|
||||||
|
/** Streaming from the replica slave MySQL instance. */
|
||||||
|
REPLICA,
|
||||||
|
/** Streaming from the read-only slave MySQL instance. */
|
||||||
|
RDONLY;
|
||||||
|
|
||||||
|
public static TabletType master() {
|
||||||
|
return TabletType.MASTER;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static TabletType replica() {
|
||||||
|
return TabletType.REPLICA;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static TabletType rdonly() {
|
||||||
|
return TabletType.RDONLY;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,134 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.ververica.cdc.connectors.vitess.config;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
/** VTCtld server configuration options. */
|
||||||
|
public class VtctldConfig {
|
||||||
|
|
||||||
|
private String hostname;
|
||||||
|
private int port = 15999; // default 15999 port
|
||||||
|
private String username;
|
||||||
|
private String password;
|
||||||
|
|
||||||
|
public String getHostname() {
|
||||||
|
return hostname;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getPort() {
|
||||||
|
return port;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getUsername() {
|
||||||
|
return username;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getPassword() {
|
||||||
|
return password;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static VtctldConfig.Builder builder() {
|
||||||
|
return new VtctldConfig.Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Builder class of {@link VtctldConfig}. */
|
||||||
|
public static final class Builder {
|
||||||
|
private String hostname;
|
||||||
|
private int port = 15999; // default 15999 port
|
||||||
|
private String username;
|
||||||
|
private String password;
|
||||||
|
|
||||||
|
/** IP address or hostname of the VTCtld server. */
|
||||||
|
public Builder hostname(String hostname) {
|
||||||
|
this.hostname = hostname;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Integer port number of the VTCtld server. */
|
||||||
|
public Builder port(int port) {
|
||||||
|
this.port = port;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An optional username of the VTCtld server. If not configured, unauthenticated VTCtld gRPC
|
||||||
|
* is used.
|
||||||
|
*/
|
||||||
|
public Builder username(String username) {
|
||||||
|
this.username = username;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An optional password of the VTCtld server. If not configured, unauthenticated VTCtld gRPC
|
||||||
|
* is used.
|
||||||
|
*/
|
||||||
|
public Builder password(String password) {
|
||||||
|
this.password = password;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public VtctldConfig build() {
|
||||||
|
VtctldConfig vtctldConfig = new VtctldConfig();
|
||||||
|
vtctldConfig.password = this.password;
|
||||||
|
vtctldConfig.username = this.username;
|
||||||
|
vtctldConfig.hostname = this.hostname;
|
||||||
|
vtctldConfig.port = this.port;
|
||||||
|
return vtctldConfig;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
VtctldConfig that = (VtctldConfig) o;
|
||||||
|
return port == that.port
|
||||||
|
&& Objects.equals(hostname, that.hostname)
|
||||||
|
&& Objects.equals(username, that.username)
|
||||||
|
&& Objects.equals(password, that.password);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(hostname, port, username, password);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "VtctldConfig{"
|
||||||
|
+ "hostname='"
|
||||||
|
+ hostname
|
||||||
|
+ '\''
|
||||||
|
+ ", port="
|
||||||
|
+ port
|
||||||
|
+ ", username='"
|
||||||
|
+ username
|
||||||
|
+ '\''
|
||||||
|
+ ", password='"
|
||||||
|
+ password
|
||||||
|
+ '\''
|
||||||
|
+ '}';
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,230 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.ververica.cdc.connectors.vitess.table;
|
||||||
|
|
||||||
|
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||||
|
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||||
|
import org.apache.flink.table.connector.ChangelogMode;
|
||||||
|
import org.apache.flink.table.connector.source.DynamicTableSource;
|
||||||
|
import org.apache.flink.table.connector.source.ScanTableSource;
|
||||||
|
import org.apache.flink.table.connector.source.SourceFunctionProvider;
|
||||||
|
import org.apache.flink.table.data.RowData;
|
||||||
|
import org.apache.flink.table.types.logical.RowType;
|
||||||
|
import org.apache.flink.types.RowKind;
|
||||||
|
|
||||||
|
import com.ververica.cdc.connectors.vitess.VitessSource;
|
||||||
|
import com.ververica.cdc.connectors.vitess.config.TabletType;
|
||||||
|
import com.ververica.cdc.connectors.vitess.config.VtctldConfig;
|
||||||
|
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
|
||||||
|
import com.ververica.cdc.debezium.DebeziumSourceFunction;
|
||||||
|
import com.ververica.cdc.debezium.table.RowDataDebeziumDeserializeSchema;
|
||||||
|
|
||||||
|
import java.time.ZoneId;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
|
import static org.apache.flink.util.Preconditions.checkNotNull;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A {@link DynamicTableSource} that describes how to create a Vitess source from a logical
|
||||||
|
* description.
|
||||||
|
*/
|
||||||
|
public class VitessTableSource implements ScanTableSource {
|
||||||
|
|
||||||
|
private final ResolvedSchema physicalSchema;
|
||||||
|
private final String pluginName;
|
||||||
|
private final String name;
|
||||||
|
private final int port;
|
||||||
|
private final String hostname;
|
||||||
|
private final String keyspace;
|
||||||
|
private final String username;
|
||||||
|
private final String password;
|
||||||
|
private final String tableName;
|
||||||
|
private final VtctldConfig vtctldConfig;
|
||||||
|
private final TabletType tabletType;
|
||||||
|
private final Properties dbzProperties;
|
||||||
|
|
||||||
|
public VitessTableSource(
|
||||||
|
ResolvedSchema physicalSchema,
|
||||||
|
int port,
|
||||||
|
String hostname,
|
||||||
|
String keyspace,
|
||||||
|
String tableName,
|
||||||
|
String username,
|
||||||
|
String password,
|
||||||
|
VtctldConfig vtctldConfig,
|
||||||
|
TabletType tabletType,
|
||||||
|
String pluginName,
|
||||||
|
String name,
|
||||||
|
Properties dbzProperties) {
|
||||||
|
this.physicalSchema = physicalSchema;
|
||||||
|
this.port = port;
|
||||||
|
this.hostname = checkNotNull(hostname);
|
||||||
|
this.keyspace = checkNotNull(keyspace);
|
||||||
|
this.tableName = checkNotNull(tableName);
|
||||||
|
this.username = username;
|
||||||
|
this.password = password;
|
||||||
|
this.vtctldConfig = checkNotNull(vtctldConfig);
|
||||||
|
this.tabletType = checkNotNull(tabletType);
|
||||||
|
this.pluginName = checkNotNull(pluginName);
|
||||||
|
this.name = name;
|
||||||
|
this.dbzProperties = dbzProperties;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ChangelogMode getChangelogMode() {
|
||||||
|
return ChangelogMode.newBuilder()
|
||||||
|
.addContainedKind(RowKind.INSERT)
|
||||||
|
.addContainedKind(RowKind.UPDATE_BEFORE)
|
||||||
|
.addContainedKind(RowKind.UPDATE_AFTER)
|
||||||
|
.addContainedKind(RowKind.DELETE)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
|
||||||
|
RowType physicalDataType =
|
||||||
|
(RowType) physicalSchema.toPhysicalRowDataType().getLogicalType();
|
||||||
|
TypeInformation<RowData> typeInfo =
|
||||||
|
scanContext.createTypeInformation(physicalSchema.toPhysicalRowDataType());
|
||||||
|
|
||||||
|
DebeziumDeserializationSchema<RowData> deserializer =
|
||||||
|
RowDataDebeziumDeserializeSchema.newBuilder()
|
||||||
|
.setPhysicalRowType(physicalDataType)
|
||||||
|
.setResultTypeInfo(typeInfo)
|
||||||
|
.setServerTimeZone(ZoneId.of("UTC"))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
DebeziumSourceFunction<RowData> sourceFunction =
|
||||||
|
VitessSource.<RowData>builder()
|
||||||
|
.hostname(hostname)
|
||||||
|
.port(port)
|
||||||
|
.keyspace(keyspace)
|
||||||
|
.tableIncludeList(tableName)
|
||||||
|
.username(username)
|
||||||
|
.password(password)
|
||||||
|
.tabletType(tabletType)
|
||||||
|
.decodingPluginName(pluginName)
|
||||||
|
.vtctldConfig(vtctldConfig)
|
||||||
|
.name(name)
|
||||||
|
.debeziumProperties(dbzProperties)
|
||||||
|
.deserializer(deserializer)
|
||||||
|
.build();
|
||||||
|
return SourceFunctionProvider.of(sourceFunction, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DynamicTableSource copy() {
|
||||||
|
return new VitessTableSource(
|
||||||
|
physicalSchema,
|
||||||
|
port,
|
||||||
|
hostname,
|
||||||
|
keyspace,
|
||||||
|
tableName,
|
||||||
|
username,
|
||||||
|
password,
|
||||||
|
vtctldConfig,
|
||||||
|
tabletType,
|
||||||
|
pluginName,
|
||||||
|
name,
|
||||||
|
dbzProperties);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
VitessTableSource that = (VitessTableSource) o;
|
||||||
|
return port == that.port
|
||||||
|
&& Objects.equals(physicalSchema, that.physicalSchema)
|
||||||
|
&& Objects.equals(pluginName, that.pluginName)
|
||||||
|
&& Objects.equals(name, that.name)
|
||||||
|
&& Objects.equals(hostname, that.hostname)
|
||||||
|
&& Objects.equals(keyspace, that.keyspace)
|
||||||
|
&& Objects.equals(username, that.username)
|
||||||
|
&& Objects.equals(password, that.password)
|
||||||
|
&& Objects.equals(tableName, that.tableName)
|
||||||
|
&& Objects.equals(vtctldConfig, that.vtctldConfig)
|
||||||
|
&& tabletType == that.tabletType
|
||||||
|
&& Objects.equals(dbzProperties, that.dbzProperties);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(
|
||||||
|
physicalSchema,
|
||||||
|
pluginName,
|
||||||
|
name,
|
||||||
|
port,
|
||||||
|
hostname,
|
||||||
|
keyspace,
|
||||||
|
username,
|
||||||
|
password,
|
||||||
|
tableName,
|
||||||
|
vtctldConfig,
|
||||||
|
tabletType,
|
||||||
|
dbzProperties);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "VitessTableSource{"
|
||||||
|
+ "physicalSchema="
|
||||||
|
+ physicalSchema
|
||||||
|
+ ", pluginName='"
|
||||||
|
+ pluginName
|
||||||
|
+ '\''
|
||||||
|
+ ", name='"
|
||||||
|
+ name
|
||||||
|
+ '\''
|
||||||
|
+ ", port="
|
||||||
|
+ port
|
||||||
|
+ ", hostname='"
|
||||||
|
+ hostname
|
||||||
|
+ '\''
|
||||||
|
+ ", keyspace='"
|
||||||
|
+ keyspace
|
||||||
|
+ '\''
|
||||||
|
+ ", username='"
|
||||||
|
+ username
|
||||||
|
+ '\''
|
||||||
|
+ ", password='"
|
||||||
|
+ password
|
||||||
|
+ '\''
|
||||||
|
+ ", tableName='"
|
||||||
|
+ tableName
|
||||||
|
+ '\''
|
||||||
|
+ ", vtctldConfig="
|
||||||
|
+ vtctldConfig
|
||||||
|
+ ", tabletType="
|
||||||
|
+ tabletType
|
||||||
|
+ ", dbzProperties="
|
||||||
|
+ dbzProperties
|
||||||
|
+ '}';
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String asSummaryString() {
|
||||||
|
return "Vitess-CDC";
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,16 @@
|
|||||||
|
# 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.
|
||||||
|
|
||||||
|
com.ververica.cdc.connectors.vitess.table.VitessTableFactory
|
@ -0,0 +1,319 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.vervetica.cdc.connectors.vitess;
|
||||||
|
|
||||||
|
import org.apache.flink.api.common.state.BroadcastState;
|
||||||
|
import org.apache.flink.api.common.state.KeyedStateStore;
|
||||||
|
import org.apache.flink.api.common.state.ListState;
|
||||||
|
import org.apache.flink.api.common.state.ListStateDescriptor;
|
||||||
|
import org.apache.flink.api.common.state.MapStateDescriptor;
|
||||||
|
import org.apache.flink.api.common.state.OperatorStateStore;
|
||||||
|
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||||
|
import org.apache.flink.configuration.Configuration;
|
||||||
|
import org.apache.flink.core.testutils.CheckedThread;
|
||||||
|
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||||
|
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||||
|
import org.apache.flink.streaming.util.MockStreamingRuntimeContext;
|
||||||
|
import org.apache.flink.util.Collector;
|
||||||
|
|
||||||
|
import com.ververica.cdc.connectors.utils.TestSourceContext;
|
||||||
|
import com.ververica.cdc.connectors.vitess.VitessSource;
|
||||||
|
import com.ververica.cdc.connectors.vitess.config.TabletType;
|
||||||
|
import com.ververica.cdc.connectors.vitess.config.VtctldConfig;
|
||||||
|
import com.ververica.cdc.debezium.DebeziumDeserializationSchema;
|
||||||
|
import com.ververica.cdc.debezium.DebeziumSourceFunction;
|
||||||
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.Statement;
|
||||||
|
import java.time.Duration;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.OptionalLong;
|
||||||
|
import java.util.Properties;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
import static com.ververica.cdc.connectors.utils.AssertUtils.assertDelete;
|
||||||
|
import static com.ververica.cdc.connectors.utils.AssertUtils.assertInsert;
|
||||||
|
import static com.ververica.cdc.connectors.utils.AssertUtils.assertUpdate;
|
||||||
|
|
||||||
|
/** Tests for {@link VitessSource} which also heavily tests {@link DebeziumSourceFunction}. */
|
||||||
|
public class VitessSourceTest extends VitessTestBase {
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void before() {
|
||||||
|
initializeTable("inventory");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConsumingAllEvents() throws Exception {
|
||||||
|
DebeziumSourceFunction<SourceRecord> source = createVitessSqlSource(0);
|
||||||
|
TestSourceContext<SourceRecord> sourceContext = new TestSourceContext<>();
|
||||||
|
|
||||||
|
setupSource(source);
|
||||||
|
|
||||||
|
try (Connection connection = getJdbcConnection();
|
||||||
|
Statement statement = connection.createStatement()) {
|
||||||
|
|
||||||
|
// start the source
|
||||||
|
final CheckedThread runThread =
|
||||||
|
new CheckedThread() {
|
||||||
|
@Override
|
||||||
|
public void go() throws Exception {
|
||||||
|
source.run(sourceContext);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
runThread.start();
|
||||||
|
|
||||||
|
waitForSourceToStart(Duration.ofSeconds(60), source);
|
||||||
|
List<SourceRecord> records;
|
||||||
|
|
||||||
|
statement.execute(
|
||||||
|
"INSERT INTO test.products VALUES (default,'robot','Toy robot',1.304)"); // 110
|
||||||
|
records = drain(sourceContext, 1);
|
||||||
|
assertInsert(records.get(0), "id", 101);
|
||||||
|
|
||||||
|
statement.execute(
|
||||||
|
"INSERT INTO test.products VALUES (1001,'roy','old robot',1234.56)"); // 1001
|
||||||
|
records = drain(sourceContext, 1);
|
||||||
|
assertInsert(records.get(0), "id", 1001);
|
||||||
|
|
||||||
|
// ---------------------------------------------------------------------------------------------------------------
|
||||||
|
// Changing the primary key of a row should result in 2 events: INSERT, DELETE
|
||||||
|
// (TOMBSTONE is dropped)
|
||||||
|
// ---------------------------------------------------------------------------------------------------------------
|
||||||
|
statement.execute(
|
||||||
|
"UPDATE test.products SET id=2001, description='really old robot' WHERE id=1001");
|
||||||
|
records = drain(sourceContext, 2);
|
||||||
|
assertDelete(records.get(0), "id", 1001);
|
||||||
|
assertInsert(records.get(1), "id", 2001);
|
||||||
|
|
||||||
|
// ---------------------------------------------------------------------------------------------------------------
|
||||||
|
// Simple UPDATE (with no schema changes)
|
||||||
|
// ---------------------------------------------------------------------------------------------------------------
|
||||||
|
statement.execute("UPDATE test.products SET weight=1345.67 WHERE id=2001");
|
||||||
|
records = drain(sourceContext, 1);
|
||||||
|
assertUpdate(records.get(0), "id", 2001);
|
||||||
|
|
||||||
|
// ---------------------------------------------------------------------------------------------------------------
|
||||||
|
// Change our schema with a fully-qualified name; we should still see this event
|
||||||
|
// ---------------------------------------------------------------------------------------------------------------
|
||||||
|
// Add a column with default to the 'products' table and explicitly update one record
|
||||||
|
// ...
|
||||||
|
statement.execute(
|
||||||
|
"ALTER TABLE test.products ADD COLUMN volume FLOAT, ADD COLUMN alias VARCHAR(30) NULL");
|
||||||
|
|
||||||
|
// Vitess schema change has eventual consistency, wait few seconds.
|
||||||
|
Thread.sleep(5000);
|
||||||
|
statement.execute("UPDATE test.products SET volume=13.5 WHERE id=2001");
|
||||||
|
records = drain(sourceContext, 1);
|
||||||
|
assertUpdate(records.get(0), "id", 2001);
|
||||||
|
|
||||||
|
// cleanup
|
||||||
|
source.cancel();
|
||||||
|
source.close();
|
||||||
|
runThread.sync();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// ------------------------------------------------------------------------------------------
|
||||||
|
// Utilities
|
||||||
|
// ------------------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
private DebeziumSourceFunction<SourceRecord> createVitessSqlSource(int heartbeatInterval) {
|
||||||
|
Properties properties = new Properties();
|
||||||
|
properties.setProperty("heartbeat.interval.ms", String.valueOf(heartbeatInterval));
|
||||||
|
return VitessSource.<SourceRecord>builder()
|
||||||
|
.hostname(VITESS_CONTAINER.getHost())
|
||||||
|
.port(VITESS_CONTAINER.getGrpcPort())
|
||||||
|
.keyspace(VITESS_CONTAINER.getKeyspace())
|
||||||
|
.tabletType(TabletType.MASTER)
|
||||||
|
.tableIncludeList("test.products")
|
||||||
|
.vtctldConfig(
|
||||||
|
VtctldConfig.builder()
|
||||||
|
.hostname(VITESS_CONTAINER.getHost())
|
||||||
|
.port(VITESS_CONTAINER.getVtctldGrpcPort())
|
||||||
|
.build())
|
||||||
|
.deserializer(new ForwardDeserializeSchema())
|
||||||
|
.debeziumProperties(properties)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private <T> List<T> drain(TestSourceContext<T> sourceContext, int expectedRecordCount)
|
||||||
|
throws Exception {
|
||||||
|
List<T> allRecords = new ArrayList<>();
|
||||||
|
LinkedBlockingQueue<StreamRecord<T>> queue = sourceContext.getCollectedOutputs();
|
||||||
|
while (allRecords.size() < expectedRecordCount) {
|
||||||
|
StreamRecord<T> record = queue.poll(1000, TimeUnit.SECONDS);
|
||||||
|
if (record != null) {
|
||||||
|
allRecords.add(record.getValue());
|
||||||
|
} else {
|
||||||
|
throw new RuntimeException(
|
||||||
|
"Can't receive " + expectedRecordCount + " elements before timeout.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return allRecords;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean waitForSourceToStart(
|
||||||
|
Duration timeout, DebeziumSourceFunction<SourceRecord> source)
|
||||||
|
throws InterruptedException {
|
||||||
|
long now = System.currentTimeMillis();
|
||||||
|
long stop = now + timeout.toMillis();
|
||||||
|
while (System.currentTimeMillis() < stop) {
|
||||||
|
if (source.getDebeziumStarted()) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
Thread.sleep(10); // save CPU
|
||||||
|
}
|
||||||
|
Thread.sleep(10000); // Wait for full start
|
||||||
|
return source.getDebeziumStarted();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <T> void setupSource(DebeziumSourceFunction<T> source) throws Exception {
|
||||||
|
setupSource(
|
||||||
|
source, false, null, null,
|
||||||
|
true, // enable checkpointing; auto commit should be ignored
|
||||||
|
0, 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <T, S1, S2> void setupSource(
|
||||||
|
DebeziumSourceFunction<T> source,
|
||||||
|
boolean isRestored,
|
||||||
|
ListState<S1> restoredOffsetState,
|
||||||
|
ListState<S2> restoredHistoryState,
|
||||||
|
boolean isCheckpointingEnabled,
|
||||||
|
int subtaskIndex,
|
||||||
|
int totalNumSubtasks)
|
||||||
|
throws Exception {
|
||||||
|
|
||||||
|
// run setup procedure in operator life cycle
|
||||||
|
source.setRuntimeContext(
|
||||||
|
new MockStreamingRuntimeContext(
|
||||||
|
isCheckpointingEnabled, totalNumSubtasks, subtaskIndex));
|
||||||
|
source.initializeState(
|
||||||
|
new MockFunctionInitializationContext(
|
||||||
|
isRestored,
|
||||||
|
new MockOperatorStateStore(restoredOffsetState, restoredHistoryState)));
|
||||||
|
source.open(new Configuration());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class ForwardDeserializeSchema
|
||||||
|
implements DebeziumDeserializationSchema<SourceRecord> {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 2975058057832211228L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void deserialize(SourceRecord record, Collector<SourceRecord> out) throws Exception {
|
||||||
|
out.collect(record);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TypeInformation<SourceRecord> getProducedType() {
|
||||||
|
return TypeInformation.of(SourceRecord.class);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class MockOperatorStateStore implements OperatorStateStore {
|
||||||
|
|
||||||
|
private final ListState<?> restoredOffsetListState;
|
||||||
|
private final ListState<?> restoredHistoryListState;
|
||||||
|
|
||||||
|
private MockOperatorStateStore(
|
||||||
|
ListState<?> restoredOffsetListState, ListState<?> restoredHistoryListState) {
|
||||||
|
this.restoredOffsetListState = restoredOffsetListState;
|
||||||
|
this.restoredHistoryListState = restoredHistoryListState;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public <S> ListState<S> getUnionListState(ListStateDescriptor<S> stateDescriptor)
|
||||||
|
throws Exception {
|
||||||
|
if (stateDescriptor.getName().equals(DebeziumSourceFunction.OFFSETS_STATE_NAME)) {
|
||||||
|
return (ListState<S>) restoredOffsetListState;
|
||||||
|
} else if (stateDescriptor
|
||||||
|
.getName()
|
||||||
|
.equals(DebeziumSourceFunction.HISTORY_RECORDS_STATE_NAME)) {
|
||||||
|
return (ListState<S>) restoredHistoryListState;
|
||||||
|
} else {
|
||||||
|
throw new IllegalStateException("Unknown state.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public <K, V> BroadcastState<K, V> getBroadcastState(
|
||||||
|
MapStateDescriptor<K, V> stateDescriptor) throws Exception {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public <S> ListState<S> getListState(ListStateDescriptor<S> stateDescriptor)
|
||||||
|
throws Exception {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getRegisteredStateNames() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getRegisteredBroadcastStateNames() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class MockFunctionInitializationContext
|
||||||
|
implements FunctionInitializationContext {
|
||||||
|
|
||||||
|
private final boolean isRestored;
|
||||||
|
private final OperatorStateStore operatorStateStore;
|
||||||
|
|
||||||
|
private MockFunctionInitializationContext(
|
||||||
|
boolean isRestored, OperatorStateStore operatorStateStore) {
|
||||||
|
this.isRestored = isRestored;
|
||||||
|
this.operatorStateStore = operatorStateStore;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isRestored() {
|
||||||
|
return isRestored;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public OptionalLong getRestoredCheckpointId() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public OperatorStateStore getOperatorStateStore() {
|
||||||
|
return operatorStateStore;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public KeyedStateStore getKeyedStateStore() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,106 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.vervetica.cdc.connectors.vitess;
|
||||||
|
|
||||||
|
import org.apache.flink.test.util.AbstractTestBase;
|
||||||
|
|
||||||
|
import com.vervetica.cdc.connectors.vitess.container.VitessContainer;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
import org.testcontainers.containers.output.Slf4jLogConsumer;
|
||||||
|
import org.testcontainers.lifecycle.Startables;
|
||||||
|
|
||||||
|
import java.net.URL;
|
||||||
|
import java.nio.file.Files;
|
||||||
|
import java.nio.file.Paths;
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.DriverManager;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.sql.Statement;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.regex.Matcher;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static com.vervetica.cdc.connectors.vitess.container.VitessContainer.GRPC_PORT;
|
||||||
|
import static com.vervetica.cdc.connectors.vitess.container.VitessContainer.MYSQL_PORT;
|
||||||
|
import static com.vervetica.cdc.connectors.vitess.container.VitessContainer.VTCTLD_GRPC_PORT;
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
|
||||||
|
/** Basic class for testing Vitess source, this contains a Vitess container. */
|
||||||
|
public abstract class VitessTestBase extends AbstractTestBase {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(VitessTestBase.class);
|
||||||
|
private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$");
|
||||||
|
|
||||||
|
protected static final VitessContainer VITESS_CONTAINER =
|
||||||
|
(VitessContainer)
|
||||||
|
new VitessContainer()
|
||||||
|
.withKeyspace("test")
|
||||||
|
.withUsername("flinkuser")
|
||||||
|
.withPassword("flinkpwd")
|
||||||
|
.withExposedPorts(MYSQL_PORT, GRPC_PORT, VTCTLD_GRPC_PORT)
|
||||||
|
.withLogConsumer(new Slf4jLogConsumer(LOG));
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void startContainers() {
|
||||||
|
LOG.info("Starting containers...");
|
||||||
|
Startables.deepStart(Stream.of(VITESS_CONTAINER)).join();
|
||||||
|
LOG.info("Containers are started.");
|
||||||
|
}
|
||||||
|
|
||||||
|
public Connection getJdbcConnection() throws SQLException {
|
||||||
|
return DriverManager.getConnection(VITESS_CONTAINER.getJdbcUrl());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Executes a JDBC statement using the default jdbc config without autocommitting the
|
||||||
|
* connection.
|
||||||
|
*/
|
||||||
|
protected void initializeTable(String sqlFile) {
|
||||||
|
final String ddlFile = String.format("ddl/%s.sql", sqlFile);
|
||||||
|
final URL ddlTestFile = VitessTestBase.class.getClassLoader().getResource(ddlFile);
|
||||||
|
assertNotNull("Cannot locate " + ddlFile, ddlTestFile);
|
||||||
|
try (Connection connection = getJdbcConnection();
|
||||||
|
Statement statement = connection.createStatement()) {
|
||||||
|
final List<String> statements =
|
||||||
|
Arrays.stream(
|
||||||
|
Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream()
|
||||||
|
.map(String::trim)
|
||||||
|
.filter(x -> !x.startsWith("--") && !x.isEmpty())
|
||||||
|
.map(
|
||||||
|
x -> {
|
||||||
|
final Matcher m =
|
||||||
|
COMMENT_PATTERN.matcher(x);
|
||||||
|
return m.matches() ? m.group(1) : x;
|
||||||
|
})
|
||||||
|
.collect(Collectors.joining("\n"))
|
||||||
|
.split(";"))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
for (String stmt : statements) {
|
||||||
|
statement.execute(stmt);
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,121 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.vervetica.cdc.connectors.vitess.container;
|
||||||
|
|
||||||
|
import org.testcontainers.containers.JdbcDatabaseContainer;
|
||||||
|
|
||||||
|
/** Vitess container. */
|
||||||
|
public class VitessContainer extends JdbcDatabaseContainer {
|
||||||
|
|
||||||
|
public static final String IMAGE = "vitess/vttestserver";
|
||||||
|
public static final String DEFAULT_TAG = "mysql80";
|
||||||
|
private static final Integer VITESS_PORT = 15991;
|
||||||
|
public static final Integer GRPC_PORT = VITESS_PORT + 1;
|
||||||
|
public static final Integer VTCTLD_GRPC_PORT = VITESS_PORT + 8;
|
||||||
|
public static final Integer MYSQL_PORT = VITESS_PORT + 3;
|
||||||
|
|
||||||
|
private String keyspaces = "test";
|
||||||
|
private String username = "flinkuser";
|
||||||
|
private String password = "flinkpwd";
|
||||||
|
|
||||||
|
public VitessContainer() {
|
||||||
|
this(DEFAULT_TAG);
|
||||||
|
}
|
||||||
|
|
||||||
|
public VitessContainer(String tag) {
|
||||||
|
super(IMAGE + ":" + tag);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configure() {
|
||||||
|
addEnv("PORT", VITESS_PORT.toString());
|
||||||
|
addEnv("KEYSPACES", getKeyspace());
|
||||||
|
addEnv("NUM_SHARDS", "1");
|
||||||
|
addEnv("MYSQL_BIND_HOST", "0.0.0.0");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getDriverClassName() {
|
||||||
|
try {
|
||||||
|
Class.forName("com.mysql.cj.jdbc.Driver");
|
||||||
|
return "com.mysql.cj.jdbc.Driver";
|
||||||
|
} catch (ClassNotFoundException e) {
|
||||||
|
return "com.mysql.jdbc.Driver";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getJdbcUrl() {
|
||||||
|
return "jdbc:mysql://" + getHost() + ":" + getMysqlPort() + "/" + getKeyspace();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getUsername() {
|
||||||
|
return username;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getPassword() {
|
||||||
|
return password;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getKeyspace() {
|
||||||
|
return keyspaces;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Integer getMysqlPort() {
|
||||||
|
return this.getMappedPort(MYSQL_PORT);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Integer getGrpcPort() {
|
||||||
|
return this.getMappedPort(GRPC_PORT);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Integer getVtctldGrpcPort() {
|
||||||
|
return this.getMappedPort(VTCTLD_GRPC_PORT);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getTestQueryString() {
|
||||||
|
return "SELECT 1";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public VitessContainer withDatabaseName(final String keyspace) {
|
||||||
|
this.keyspaces = keyspace;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public VitessContainer withKeyspace(String keyspace) {
|
||||||
|
this.keyspaces = keyspace;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public VitessContainer withUsername(final String username) {
|
||||||
|
this.username = username;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public VitessContainer withPassword(final String password) {
|
||||||
|
this.password = password;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,264 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.vervetica.cdc.connectors.vitess.table;
|
||||||
|
|
||||||
|
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||||
|
import org.apache.flink.table.api.EnvironmentSettings;
|
||||||
|
import org.apache.flink.table.api.TableResult;
|
||||||
|
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
|
||||||
|
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
|
||||||
|
import org.apache.flink.types.Row;
|
||||||
|
import org.apache.flink.util.CloseableIterator;
|
||||||
|
|
||||||
|
import com.vervetica.cdc.connectors.vitess.VitessTestBase;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.sql.Connection;
|
||||||
|
import java.sql.SQLException;
|
||||||
|
import java.sql.Statement;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
/** Integration tests for MySQL binlog SQL source. */
|
||||||
|
public class VitessConnectorITCase extends VitessTestBase {
|
||||||
|
|
||||||
|
private final StreamExecutionEnvironment env =
|
||||||
|
StreamExecutionEnvironment.getExecutionEnvironment();
|
||||||
|
private final StreamTableEnvironment tEnv =
|
||||||
|
StreamTableEnvironment.create(
|
||||||
|
env, EnvironmentSettings.newInstance().inStreamingMode().build());
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void before() {
|
||||||
|
TestValuesTableFactory.clearAllData();
|
||||||
|
env.setParallelism(1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConsumingAllEvents()
|
||||||
|
throws SQLException, ExecutionException, InterruptedException {
|
||||||
|
initializeTable("inventory");
|
||||||
|
String sourceDDL =
|
||||||
|
String.format(
|
||||||
|
"CREATE TABLE debezium_source ("
|
||||||
|
+ " `id` INT NOT NULL,"
|
||||||
|
+ " name STRING,"
|
||||||
|
+ " description STRING,"
|
||||||
|
+ " weight DECIMAL(10,3),"
|
||||||
|
+ " primary key (`id`) not enforced"
|
||||||
|
+ ") WITH ("
|
||||||
|
+ " 'connector' = 'vitess-cdc',"
|
||||||
|
+ " 'tablet-type' = 'MASTER',"
|
||||||
|
+ " 'hostname' = '%s',"
|
||||||
|
+ " 'port' = '%s',"
|
||||||
|
+ " 'vtctl.hostname' = '%s',"
|
||||||
|
+ " 'vtctl.port' = '%s',"
|
||||||
|
+ " 'keyspace' = '%s',"
|
||||||
|
+ " 'table-name' = '%s'"
|
||||||
|
+ ")",
|
||||||
|
VITESS_CONTAINER.getHost(),
|
||||||
|
VITESS_CONTAINER.getGrpcPort(),
|
||||||
|
VITESS_CONTAINER.getHost(),
|
||||||
|
VITESS_CONTAINER.getVtctldGrpcPort(),
|
||||||
|
VITESS_CONTAINER.getKeyspace(),
|
||||||
|
"test.products");
|
||||||
|
String sinkDDL =
|
||||||
|
"CREATE TABLE sink ("
|
||||||
|
+ " name STRING,"
|
||||||
|
+ " weightSum DECIMAL(10,3),"
|
||||||
|
+ " PRIMARY KEY (name) NOT ENFORCED"
|
||||||
|
+ ") WITH ("
|
||||||
|
+ " 'connector' = 'values',"
|
||||||
|
+ " 'sink-insert-only' = 'false',"
|
||||||
|
+ " 'sink-expected-messages-num' = '20'"
|
||||||
|
+ ")";
|
||||||
|
tEnv.executeSql(sourceDDL);
|
||||||
|
tEnv.executeSql(sinkDDL);
|
||||||
|
|
||||||
|
// async submit job
|
||||||
|
TableResult result =
|
||||||
|
tEnv.executeSql(
|
||||||
|
"INSERT INTO sink SELECT name, SUM(weight) FROM debezium_source GROUP BY name");
|
||||||
|
|
||||||
|
// Vitess source doesn't read snapshot data. Source will be empty at first.
|
||||||
|
// There's no way knowing if it's started, using sleep here.
|
||||||
|
Thread.sleep(10000);
|
||||||
|
|
||||||
|
try (Connection connection = getJdbcConnection();
|
||||||
|
Statement statement = connection.createStatement()) {
|
||||||
|
statement.execute(
|
||||||
|
"INSERT INTO test.products \n"
|
||||||
|
+ "VALUES (default,'scooter','Small 2-wheel scooter',3.14),\n"
|
||||||
|
+ " (default,'car battery','12V car battery',8.1),\n"
|
||||||
|
+ " (default,'12-pack drill bits','12-pack of drill bits with sizes ranging from #40 to #3',0.8),\n"
|
||||||
|
+ " (default,'hammer','12oz carpenters hammer',0.75),\n"
|
||||||
|
+ " (default,'hammer','14oz carpenters hammer',0.875),\n"
|
||||||
|
+ " (default,'hammer','16oz carpenters hammer',1.0),\n"
|
||||||
|
+ " (default,'rocks','box of assorted rocks',5.3),\n"
|
||||||
|
+ " (default,'jacket','water resistent black wind breaker',0.1),\n"
|
||||||
|
+ " (default,'spare tire','24 inch spare tire',22.2);");
|
||||||
|
statement.execute(
|
||||||
|
"UPDATE test.products SET description='18oz carpenter hammer' WHERE id=106;");
|
||||||
|
statement.execute("UPDATE test.products SET weight='5.1' WHERE id=107;");
|
||||||
|
statement.execute(
|
||||||
|
"INSERT INTO test.products VALUES (default,'jacket','water resistent white wind breaker',0.2);"); // 110
|
||||||
|
statement.execute(
|
||||||
|
"INSERT INTO test.products VALUES (default,'scooter','Big 2-wheel scooter ',5.18);");
|
||||||
|
statement.execute(
|
||||||
|
"UPDATE test.products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;");
|
||||||
|
statement.execute("UPDATE test.products SET weight='5.17' WHERE id=111;");
|
||||||
|
statement.execute("DELETE FROM test.products WHERE id=111;");
|
||||||
|
}
|
||||||
|
|
||||||
|
waitForSinkSize("sink", 20);
|
||||||
|
|
||||||
|
List<String> expected =
|
||||||
|
Arrays.asList(
|
||||||
|
"+I[scooter, 3.140]",
|
||||||
|
"+I[car battery, 8.100]",
|
||||||
|
"+I[12-pack drill bits, 0.800]",
|
||||||
|
"+I[hammer, 2.625]",
|
||||||
|
"+I[rocks, 5.100]",
|
||||||
|
"+I[jacket, 0.600]",
|
||||||
|
"+I[spare tire, 22.200]");
|
||||||
|
|
||||||
|
List<String> actual = TestValuesTableFactory.getResults("sink");
|
||||||
|
assertEqualsInAnyOrder(expected, actual);
|
||||||
|
result.getJobClient().get().cancel().get();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAllTypes() throws Throwable {
|
||||||
|
initializeTable("column_type_test");
|
||||||
|
String sourceDDL =
|
||||||
|
String.format(
|
||||||
|
"CREATE TABLE full_types (\n"
|
||||||
|
+ " `id` INT NOT NULL,\n"
|
||||||
|
+ " tiny_c TINYINT,\n"
|
||||||
|
+ " tiny_un_c SMALLINT ,\n"
|
||||||
|
+ " small_c SMALLINT,\n"
|
||||||
|
+ " small_un_c INT,\n"
|
||||||
|
+ " int_c INT ,\n"
|
||||||
|
+ " int_un_c BIGINT,\n"
|
||||||
|
+ " int11_c BIGINT,\n"
|
||||||
|
+ " big_c BIGINT,\n"
|
||||||
|
+ " varchar_c STRING,\n"
|
||||||
|
+ " char_c STRING,\n"
|
||||||
|
+ " float_c FLOAT,\n"
|
||||||
|
+ " double_c DOUBLE,\n"
|
||||||
|
+ " decimal_c DECIMAL(8, 4),\n"
|
||||||
|
+ " numeric_c DECIMAL(6, 0),\n"
|
||||||
|
+ " boolean_c BOOLEAN,\n"
|
||||||
|
+ " primary key (`id`) not enforced"
|
||||||
|
+ ") WITH ("
|
||||||
|
+ " 'connector' = 'vitess-cdc',"
|
||||||
|
+ " 'tablet-type' = 'MASTER',"
|
||||||
|
+ " 'hostname' = '%s',"
|
||||||
|
+ " 'port' = '%s',"
|
||||||
|
+ " 'vtctl.hostname' = '%s',"
|
||||||
|
+ " 'vtctl.port' = '%s',"
|
||||||
|
+ " 'keyspace' = '%s',"
|
||||||
|
+ " 'table-name' = '%s'"
|
||||||
|
+ ")",
|
||||||
|
VITESS_CONTAINER.getHost(),
|
||||||
|
VITESS_CONTAINER.getGrpcPort(),
|
||||||
|
VITESS_CONTAINER.getHost(),
|
||||||
|
VITESS_CONTAINER.getVtctldGrpcPort(),
|
||||||
|
VITESS_CONTAINER.getKeyspace(),
|
||||||
|
"test.full_types");
|
||||||
|
tEnv.executeSql(sourceDDL);
|
||||||
|
|
||||||
|
// async submit job
|
||||||
|
TableResult result = tEnv.executeSql("SELECT * FROM full_types");
|
||||||
|
|
||||||
|
// Vitess source doesn't read snapshot data. Source will be empty at first.
|
||||||
|
// There's no way knowing if it's started, using sleep here.
|
||||||
|
Thread.sleep(10000);
|
||||||
|
|
||||||
|
try (Connection connection = getJdbcConnection();
|
||||||
|
Statement statement = connection.createStatement()) {
|
||||||
|
statement.execute(
|
||||||
|
"INSERT INTO test.full_types VALUES (\n"
|
||||||
|
+ " DEFAULT, 127, 255, 32767, 65535, 2147483647, 4294967295, 2147483647, 9223372036854775807,\n"
|
||||||
|
+ " 'Hello World', 'abc', 123.102, 404.4443, 123.4567, 345.6, true);");
|
||||||
|
statement.execute("UPDATE test.full_types SET varchar_c = 'Bye World' WHERE id=1;");
|
||||||
|
}
|
||||||
|
|
||||||
|
waitForSnapshotStarted(result.collect());
|
||||||
|
|
||||||
|
List<String> expected =
|
||||||
|
Arrays.asList(
|
||||||
|
"+I[1, 127, 255, 32767, 65535, 2147483647, 4294967295, 2147483647, 9223372036854775807, Hello World, abc, 123.102, 404.4443, 123.4567, 346, true]",
|
||||||
|
"-U[1, 127, 255, 32767, 65535, 2147483647, 4294967295, 2147483647, 9223372036854775807, Hello World, abc, 123.102, 404.4443, 123.4567, 346, true]",
|
||||||
|
"+U[1, 127, 255, 32767, 65535, 2147483647, 4294967295, 2147483647, 9223372036854775807, Bye World, abc, 123.102, 404.4443, 123.4567, 346, true]");
|
||||||
|
|
||||||
|
List<String> actual = fetchRows(result.collect(), expected.size());
|
||||||
|
assertEquals(expected, actual);
|
||||||
|
result.getJobClient().get().cancel().get();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static List<String> fetchRows(Iterator<Row> iter, int size) {
|
||||||
|
List<String> rows = new ArrayList<>(size);
|
||||||
|
while (size > 0 && iter.hasNext()) {
|
||||||
|
Row row = iter.next();
|
||||||
|
rows.add(row.toString());
|
||||||
|
size--;
|
||||||
|
}
|
||||||
|
return rows;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void assertEqualsInAnyOrder(List<String> actual, List<String> expected) {
|
||||||
|
assertTrue(actual != null && expected != null);
|
||||||
|
assertEquals(
|
||||||
|
actual.stream().sorted().collect(Collectors.toList()),
|
||||||
|
expected.stream().sorted().collect(Collectors.toList()));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void waitForSnapshotStarted(CloseableIterator<Row> iterator) throws Exception {
|
||||||
|
while (!iterator.hasNext()) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void waitForSinkSize(String sinkName, int expectedSize)
|
||||||
|
throws InterruptedException {
|
||||||
|
while (sinkSize(sinkName) < expectedSize) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int sinkSize(String sinkName) {
|
||||||
|
synchronized (TestValuesTableFactory.class) {
|
||||||
|
try {
|
||||||
|
return TestValuesTableFactory.getRawResults(sinkName).size();
|
||||||
|
} catch (IllegalArgumentException e) {
|
||||||
|
// job is not started yet
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,202 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.vervetica.cdc.connectors.vitess.table;
|
||||||
|
|
||||||
|
import org.apache.flink.configuration.ConfigOption;
|
||||||
|
import org.apache.flink.configuration.Configuration;
|
||||||
|
import org.apache.flink.table.api.DataTypes;
|
||||||
|
import org.apache.flink.table.catalog.CatalogTable;
|
||||||
|
import org.apache.flink.table.catalog.Column;
|
||||||
|
import org.apache.flink.table.catalog.ObjectIdentifier;
|
||||||
|
import org.apache.flink.table.catalog.ResolvedCatalogTable;
|
||||||
|
import org.apache.flink.table.catalog.ResolvedSchema;
|
||||||
|
import org.apache.flink.table.catalog.UniqueConstraint;
|
||||||
|
import org.apache.flink.table.connector.source.DynamicTableSource;
|
||||||
|
import org.apache.flink.table.factories.Factory;
|
||||||
|
import org.apache.flink.table.factories.FactoryUtil;
|
||||||
|
import org.apache.flink.util.ExceptionUtils;
|
||||||
|
|
||||||
|
import com.ververica.cdc.connectors.vitess.config.TabletType;
|
||||||
|
import com.ververica.cdc.connectors.vitess.config.VtctldConfig;
|
||||||
|
import com.ververica.cdc.connectors.vitess.table.VitessTableFactory;
|
||||||
|
import com.ververica.cdc.connectors.vitess.table.VitessTableSource;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
|
import static org.apache.flink.table.api.TableSchema.fromResolvedSchema;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
/** Test for {@link VitessTableSource} created by {@link VitessTableFactory}. */
|
||||||
|
public class VitessTableFactoryTest {
|
||||||
|
|
||||||
|
private static final ResolvedSchema SCHEMA =
|
||||||
|
new ResolvedSchema(
|
||||||
|
Arrays.asList(
|
||||||
|
Column.physical("aaa", DataTypes.INT().notNull()),
|
||||||
|
Column.physical("bbb", DataTypes.STRING().notNull()),
|
||||||
|
Column.physical("ccc", DataTypes.DOUBLE()),
|
||||||
|
Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
|
||||||
|
Column.physical("eee", DataTypes.TIMESTAMP(3))),
|
||||||
|
new ArrayList<>(),
|
||||||
|
UniqueConstraint.primaryKey("pk", Arrays.asList("bbb", "aaa")));
|
||||||
|
|
||||||
|
private static final String MY_SCHEMA = "public";
|
||||||
|
private static final String MY_LOCALHOST = "localhost";
|
||||||
|
private static final String MY_USERNAME = "flinkuser";
|
||||||
|
private static final String MY_PASSWORD = "flinkpw";
|
||||||
|
private static final String MY_KEYSPACE = "myDB";
|
||||||
|
private static final String MY_TABLE = "myTable";
|
||||||
|
private static final Properties PROPERTIES = new Properties();
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommonProperties() {
|
||||||
|
Map<String, String> properties = getAllOptions();
|
||||||
|
|
||||||
|
// validation for source
|
||||||
|
DynamicTableSource actualSource = createTableSource(properties);
|
||||||
|
VitessTableSource expectedSource =
|
||||||
|
new VitessTableSource(
|
||||||
|
SCHEMA,
|
||||||
|
15991,
|
||||||
|
MY_LOCALHOST,
|
||||||
|
MY_KEYSPACE,
|
||||||
|
MY_TABLE,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
VtctldConfig.builder().hostname(MY_LOCALHOST).port(15999).build(),
|
||||||
|
TabletType.RDONLY,
|
||||||
|
"decoderbufs",
|
||||||
|
"flink",
|
||||||
|
PROPERTIES);
|
||||||
|
assertEquals(expectedSource, actualSource);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOptionalProperties() {
|
||||||
|
Map<String, String> options = getAllOptions();
|
||||||
|
options.put("port", "5444");
|
||||||
|
options.put("vtctl.port", "5445");
|
||||||
|
options.put("decoding.plugin.name", "wal2json");
|
||||||
|
options.put("debezium.snapshot.mode", "never");
|
||||||
|
options.put("name", "flink");
|
||||||
|
options.put("tablet-type", "MASTER");
|
||||||
|
options.put("username", MY_USERNAME);
|
||||||
|
options.put("password", MY_PASSWORD);
|
||||||
|
|
||||||
|
DynamicTableSource actualSource = createTableSource(options);
|
||||||
|
Properties dbzProperties = new Properties();
|
||||||
|
dbzProperties.put("snapshot.mode", "never");
|
||||||
|
VitessTableSource expectedSource =
|
||||||
|
new VitessTableSource(
|
||||||
|
SCHEMA,
|
||||||
|
5444,
|
||||||
|
MY_LOCALHOST,
|
||||||
|
MY_KEYSPACE,
|
||||||
|
MY_TABLE,
|
||||||
|
MY_USERNAME,
|
||||||
|
MY_PASSWORD,
|
||||||
|
VtctldConfig.builder().hostname(MY_LOCALHOST).port(5445).build(),
|
||||||
|
TabletType.MASTER,
|
||||||
|
"wal2json",
|
||||||
|
"flink",
|
||||||
|
dbzProperties);
|
||||||
|
assertEquals(expectedSource, actualSource);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testValidation() {
|
||||||
|
// validate illegal port
|
||||||
|
try {
|
||||||
|
Map<String, String> properties = getAllOptions();
|
||||||
|
properties.put("port", "123b");
|
||||||
|
|
||||||
|
createTableSource(properties);
|
||||||
|
fail("exception expected");
|
||||||
|
} catch (Throwable t) {
|
||||||
|
assertTrue(
|
||||||
|
ExceptionUtils.findThrowableWithMessage(
|
||||||
|
t, "Could not parse value '123b' for key 'port'.")
|
||||||
|
.isPresent());
|
||||||
|
}
|
||||||
|
|
||||||
|
// validate missing required
|
||||||
|
Factory factory = new VitessTableFactory();
|
||||||
|
for (ConfigOption<?> requiredOption : factory.requiredOptions()) {
|
||||||
|
Map<String, String> properties = getAllOptions();
|
||||||
|
properties.remove(requiredOption.key());
|
||||||
|
|
||||||
|
try {
|
||||||
|
createTableSource(properties);
|
||||||
|
fail("exception expected");
|
||||||
|
} catch (Throwable t) {
|
||||||
|
assertTrue(
|
||||||
|
ExceptionUtils.findThrowableWithMessage(
|
||||||
|
t,
|
||||||
|
"Missing required options are:\n\n" + requiredOption.key())
|
||||||
|
.isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// validate unsupported option
|
||||||
|
try {
|
||||||
|
Map<String, String> properties = getAllOptions();
|
||||||
|
properties.put("unknown", "abc");
|
||||||
|
|
||||||
|
createTableSource(properties);
|
||||||
|
fail("exception expected");
|
||||||
|
} catch (Throwable t) {
|
||||||
|
assertTrue(
|
||||||
|
ExceptionUtils.findThrowableWithMessage(t, "Unsupported options:\n\nunknown")
|
||||||
|
.isPresent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Map<String, String> getAllOptions() {
|
||||||
|
Map<String, String> options = new HashMap<>();
|
||||||
|
options.put("connector", "vitess-cdc");
|
||||||
|
options.put("hostname", MY_LOCALHOST);
|
||||||
|
options.put("keyspace", MY_KEYSPACE);
|
||||||
|
options.put("vtctl.hostname", MY_LOCALHOST);
|
||||||
|
options.put("table-name", MY_TABLE);
|
||||||
|
return options;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static DynamicTableSource createTableSource(Map<String, String> options) {
|
||||||
|
return FactoryUtil.createTableSource(
|
||||||
|
null,
|
||||||
|
ObjectIdentifier.of("default", "default", "t1"),
|
||||||
|
new ResolvedCatalogTable(
|
||||||
|
CatalogTable.of(
|
||||||
|
fromResolvedSchema(SCHEMA).toSchema(),
|
||||||
|
"mock source",
|
||||||
|
new ArrayList<>(),
|
||||||
|
options),
|
||||||
|
SCHEMA),
|
||||||
|
new Configuration(),
|
||||||
|
VitessTableFactoryTest.class.getClassLoader(),
|
||||||
|
false);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,44 @@
|
|||||||
|
-- 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.
|
||||||
|
|
||||||
|
|
||||||
|
USE test;
|
||||||
|
DROP TABLE IF EXISTS full_types;
|
||||||
|
-- TODO add DATE, DATETIME, TIMESTAMP, TIME type mapping
|
||||||
|
CREATE TABLE full_types (
|
||||||
|
id INT AUTO_INCREMENT NOT NULL,
|
||||||
|
tiny_c TINYINT,
|
||||||
|
tiny_un_c TINYINT UNSIGNED,
|
||||||
|
small_c SMALLINT,
|
||||||
|
small_un_c SMALLINT UNSIGNED,
|
||||||
|
int_c INTEGER ,
|
||||||
|
int_un_c INTEGER UNSIGNED,
|
||||||
|
int11_c INT(11) ,
|
||||||
|
big_c BIGINT,
|
||||||
|
varchar_c VARCHAR(255),
|
||||||
|
char_c CHAR(3),
|
||||||
|
float_c FLOAT,
|
||||||
|
double_c DOUBLE,
|
||||||
|
decimal_c DECIMAL(8, 4),
|
||||||
|
numeric_c NUMERIC(6, 0),
|
||||||
|
boolean_c BOOLEAN,
|
||||||
|
-- date_c DATE,
|
||||||
|
-- time_c TIME(0),
|
||||||
|
-- datetime3_c DATETIME(3),
|
||||||
|
-- datetime6_c DATETIME(6),
|
||||||
|
-- timestamp_c TIMESTAMP,
|
||||||
|
-- file_uuid BINARY(16),
|
||||||
|
PRIMARY KEY (id)
|
||||||
|
) DEFAULT CHARSET=utf8;
|
@ -0,0 +1,24 @@
|
|||||||
|
-- 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.
|
||||||
|
|
||||||
|
USE test;
|
||||||
|
DROP TABLE IF EXISTS products;
|
||||||
|
CREATE TABLE products (
|
||||||
|
id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
|
||||||
|
name VARCHAR(255) NOT NULL DEFAULT 'flink',
|
||||||
|
description VARCHAR(512),
|
||||||
|
weight FLOAT
|
||||||
|
);
|
||||||
|
ALTER TABLE products AUTO_INCREMENT = 101;
|
@ -0,0 +1,28 @@
|
|||||||
|
################################################################################
|
||||||
|
# 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.
|
||||||
|
################################################################################
|
||||||
|
|
||||||
|
# Set root logger level to OFF to not flood build logs
|
||||||
|
# set manually to INFO for debugging purposes
|
||||||
|
rootLogger.level=INFO
|
||||||
|
rootLogger.appenderRef.test.ref = TestLogger
|
||||||
|
|
||||||
|
appender.testlogger.name = TestLogger
|
||||||
|
appender.testlogger.type = CONSOLE
|
||||||
|
appender.testlogger.target = SYSTEM_ERR
|
||||||
|
appender.testlogger.layout.type = PatternLayout
|
||||||
|
appender.testlogger.layout.pattern = %-4r [%t] %-5p %c - %m%n
|
@ -0,0 +1,95 @@
|
|||||||
|
<?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">
|
||||||
|
<parent>
|
||||||
|
<artifactId>flink-cdc-connectors</artifactId>
|
||||||
|
<groupId>com.ververica</groupId>
|
||||||
|
<version>2.4-SNAPSHOT</version>
|
||||||
|
</parent>
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
|
<artifactId>flink-sql-connector-vitess-cdc</artifactId>
|
||||||
|
|
||||||
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.ververica</groupId>
|
||||||
|
<artifactId>flink-connector-vitess-cdc</artifactId>
|
||||||
|
<version>${project.version}</version>
|
||||||
|
</dependency>
|
||||||
|
</dependencies>
|
||||||
|
|
||||||
|
<build>
|
||||||
|
<plugins>
|
||||||
|
<plugin>
|
||||||
|
<groupId>org.apache.maven.plugins</groupId>
|
||||||
|
<artifactId>maven-shade-plugin</artifactId>
|
||||||
|
<version>3.2.4</version>
|
||||||
|
<executions>
|
||||||
|
<execution>
|
||||||
|
<id>shade-flink</id>
|
||||||
|
<phase>package</phase>
|
||||||
|
<goals>
|
||||||
|
<goal>shade</goal>
|
||||||
|
</goals>
|
||||||
|
<configuration>
|
||||||
|
<shadeTestJar>false</shadeTestJar>
|
||||||
|
<transformers>
|
||||||
|
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
|
||||||
|
</transformers>
|
||||||
|
<artifactSet>
|
||||||
|
<includes>
|
||||||
|
<include>*:*</include>
|
||||||
|
</includes>
|
||||||
|
</artifactSet>
|
||||||
|
<relocations>
|
||||||
|
<relocation>
|
||||||
|
<pattern>io.grpc</pattern>
|
||||||
|
<shadedPattern>com.ververica.cdc.connectors.vitess.shaded.io.grpc</shadedPattern>
|
||||||
|
</relocation>
|
||||||
|
<relocation>
|
||||||
|
<pattern>io.netty</pattern>
|
||||||
|
<shadedPattern>com.ververica.cdc.connectors.vitess.shaded.io.netty</shadedPattern>
|
||||||
|
</relocation>
|
||||||
|
<relocation>
|
||||||
|
<pattern>com.google</pattern>
|
||||||
|
<shadedPattern>com.ververica.cdc.connectors.vitess.shaded.com.google</shadedPattern>
|
||||||
|
</relocation>
|
||||||
|
</relocations>
|
||||||
|
<filters>
|
||||||
|
<filter>
|
||||||
|
<artifact>org.apache.kafka:*</artifact>
|
||||||
|
<excludes>
|
||||||
|
<exclude>kafka/kafka-version.properties</exclude>
|
||||||
|
<exclude>LICENSE</exclude>
|
||||||
|
<!-- Does not contain anything relevant.
|
||||||
|
Cites a binary dependency on jersey, but this is neither reflected in the
|
||||||
|
dependency graph, nor are any jersey files bundled. -->
|
||||||
|
<exclude>NOTICE</exclude>
|
||||||
|
<exclude>common/**</exclude>
|
||||||
|
</excludes>
|
||||||
|
</filter>
|
||||||
|
</filters>
|
||||||
|
</configuration>
|
||||||
|
</execution>
|
||||||
|
</executions>
|
||||||
|
</plugin>
|
||||||
|
</plugins>
|
||||||
|
</build>
|
||||||
|
</project>
|
@ -0,0 +1,22 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.ververica.cdc.connectors.vitess;
|
||||||
|
|
||||||
|
/** This is used to generate a dummy docs jar for this module to pass OSS repository rule. */
|
||||||
|
public class DummyDocs {}
|
Loading…
Reference in New Issue