Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[api-draft][connector] Add SeaTunnel jdbc sink (#1946) #2009

Merged
merged 4 commits into from
Jun 13, 2022
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,11 @@
<artifactId>seatunnel-connector-seatunnel-hive</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-connector-seatunnel-jdbc</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-connector-seatunnel-socket</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
<module>seatunnel-connector-seatunnel-console</module>
<module>seatunnel-connector-seatunnel-fake</module>
<module>seatunnel-connector-seatunnel-kafka</module>
<module>seatunnel-connector-seatunnel-jdbc</module>
<module>seatunnel-connector-seatunnel-socket</module>
</modules>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
<?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>seatunnel-connectors-seatunnel</artifactId>
<groupId>org.apache.seatunnel</groupId>
<version>${revision}</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>seatunnel-connector-seatunnel-jdbc</artifactId>

<dependencies>

<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-api</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>mysql</groupId>
<artifactId>mysql-connector-java</artifactId>
</dependency>

<dependency>
<groupId>org.postgresql</groupId>
<artifactId>postgresql</artifactId>
</dependency>

</dependencies>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,257 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.seatunnel.connectors.seatunnel.jdbc.internal;

import static com.google.common.base.Preconditions.checkNotNull;

import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider;
import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.executor.JdbcBatchStatementExecutor;
import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.options.JdbcConnectorOptions;
import org.apache.seatunnel.connectors.seatunnel.jdbc.utils.ExceptionUtils;

import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.io.Serializable;
import java.sql.Connection;
import java.sql.SQLException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;

/**
* A JDBC outputFormat
*/
public class JdbcOutputFormat<I, E extends JdbcBatchStatementExecutor<I>>
implements Serializable {

protected final JdbcConnectionProvider connectionProvider;

private static final long serialVersionUID = 1L;

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

private final JdbcConnectorOptions jdbcConnectorOptions;
private final StatementExecutorFactory<E> statementExecutorFactory;

private transient E jdbcStatementExecutor;
private transient int batchCount = 0;
private transient volatile boolean closed = false;

private transient ScheduledExecutorService scheduler;
private transient ScheduledFuture<?> scheduledFuture;
private transient volatile Exception flushException;

public JdbcOutputFormat(
JdbcConnectionProvider connectionProvider,
JdbcConnectorOptions jdbcConnectorOptions,
StatementExecutorFactory<E> statementExecutorFactory) {
this.connectionProvider = checkNotNull(connectionProvider);
this.jdbcConnectorOptions = checkNotNull(jdbcConnectorOptions);
this.statementExecutorFactory = checkNotNull(statementExecutorFactory);
}

/**
* Connects to the target database and initializes the prepared statement.
*/

public void open()
throws IOException {
try {
connectionProvider.getOrEstablishConnection();
}
catch (Exception e) {
throw new IOException("unable to open JDBC writer", e);
}
jdbcStatementExecutor = createAndOpenStatementExecutor(statementExecutorFactory);

if (jdbcConnectorOptions.getBatchIntervalMs() != 0 && jdbcConnectorOptions.getBatchSize() != 1) {
this.scheduler =
Executors.newScheduledThreadPool(
1, runnable -> {
AtomicInteger cnt = new AtomicInteger(0);
Thread thread = new Thread(runnable);
thread.setDaemon(true);
thread.setName("jdbc-upsert-output-format" + "-" + cnt.incrementAndGet());
return thread;
});
this.scheduledFuture =
this.scheduler.scheduleWithFixedDelay(
() -> {
synchronized (JdbcOutputFormat.this) {
if (!closed) {
try {
flush();
}
catch (Exception e) {
flushException = e;
}
}
}
},
jdbcConnectorOptions.getBatchIntervalMs(),
jdbcConnectorOptions.getBatchIntervalMs(),
TimeUnit.MILLISECONDS);
}
}

private E createAndOpenStatementExecutor(
StatementExecutorFactory<E> statementExecutorFactory)
throws IOException {
E exec = statementExecutorFactory.get();
try {
exec.prepareStatements(connectionProvider.getConnection());
}
catch (SQLException e) {
throw new IOException("unable to open JDBC writer", e);
}
return exec;
}

private void checkFlushException() {
if (flushException != null) {
throw new RuntimeException("Writing records to JDBC failed.", flushException);
}
}

public final synchronized void writeRecord(I record)
throws IOException {
checkFlushException();
try {
addToBatch(record);
batchCount++;
if (jdbcConnectorOptions.getBatchSize() > 0
&& batchCount >= jdbcConnectorOptions.getBatchSize()) {
flush();
}
}
catch (Exception e) {
throw new IOException("Writing records to JDBC failed.", e);
}
}

protected void addToBatch(I record)
throws SQLException {
jdbcStatementExecutor.addToBatch(record);
}

public synchronized void flush()
throws IOException {
checkFlushException();
final int sleepMs = 1000;
for (int i = 0; i <= jdbcConnectorOptions.getMaxRetries(); i++) {
try {
attemptFlush();
batchCount = 0;
break;
}
catch (SQLException e) {
LOG.error("JDBC executeBatch error, retry times = {}", i, e);
if (i >= jdbcConnectorOptions.getMaxRetries()) {
ExceptionUtils.rethrowIOException(e);
}
try {
if (!connectionProvider.isConnectionValid()) {
updateExecutor(true);
}
}
catch (Exception exception) {
LOG.error(
"JDBC connection is not valid, and reestablish connection failed.",
exception);
throw new IOException("Reestablish JDBC connection failed", exception);
}
try {
Thread.sleep(sleepMs * i);
}
catch (InterruptedException ex) {
Thread.currentThread().interrupt();
throw new IOException(
"unable to flush; interrupted while doing another attempt", e);
}
}
}
}

protected void attemptFlush()
throws SQLException {
jdbcStatementExecutor.executeBatch();
}

/**
* Executes prepared statement and closes all resources of this instance.
*/
public synchronized void close() {
if (!closed) {
closed = true;

if (this.scheduledFuture != null) {
scheduledFuture.cancel(false);
this.scheduler.shutdown();
}

if (batchCount > 0) {
try {
flush();
}
catch (Exception e) {
LOG.warn("Writing records to JDBC failed.", e);
throw new RuntimeException("Writing records to JDBC failed.", e);
}
}

try {
if (jdbcStatementExecutor != null) {
jdbcStatementExecutor.closeStatements();
}
}
catch (SQLException e) {
LOG.warn("Close JDBC writer failed.", e);
}
}
connectionProvider.closeConnection();
checkFlushException();
}

public void updateExecutor(boolean reconnect)
throws SQLException, ClassNotFoundException {
jdbcStatementExecutor.closeStatements();
jdbcStatementExecutor.prepareStatements(
reconnect ? connectionProvider.reestablishConnection() : connectionProvider.getConnection());
}

@VisibleForTesting
public Connection getConnection() {
return connectionProvider.getConnection();
}

/**
* A factory for creating {@link JdbcBatchStatementExecutor} instance.
*
* @param <T> The type of instance.
*/
public interface StatementExecutorFactory<T extends JdbcBatchStatementExecutor<?>>
extends Supplier<T>, Serializable {}

;
}
Loading