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

[feat][jdbc]improvement jdbc read/write performance #1488

Merged
merged 1 commit into from
Jan 28, 2023
Merged
Show file tree
Hide file tree
Changes from all 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 @@ -33,7 +33,8 @@ public class GreenplumDialect extends PostgresqlDialect {

private static final String DIALECT_NAME = "Greenplum";
private static final String DRIVER = "com.pivotal.jdbc.GreenplumDriver";
private static final String URL_START = "jdbc:pivotal:greenplum:";
public static final String URL_START = "jdbc:pivotal:greenplum:";
public static final String DATABASE_NAME = ";DatabaseName=";

@Override
public String dialectName() {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,246 @@
/*
* 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.dtstack.chunjun.connector.greenplum.sink;

import com.dtstack.chunjun.connector.jdbc.converter.JdbcColumnConverter;
import com.dtstack.chunjun.connector.jdbc.sink.JdbcOutputFormat;
import com.dtstack.chunjun.connector.postgresql.converter.PostgresqlColumnConverter;
import com.dtstack.chunjun.connector.postgresql.dialect.PostgresqlDialect;
import com.dtstack.chunjun.constants.ConstantValue;
import com.dtstack.chunjun.element.ColumnRowData;
import com.dtstack.chunjun.enums.EWriteMode;
import com.dtstack.chunjun.throwable.NoRestartException;
import com.dtstack.chunjun.throwable.WriteRecordException;

import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.table.data.RowData;

import org.apache.commons.lang3.math.NumberUtils;
import org.postgresql.copy.CopyManager;
import org.postgresql.core.BaseConnection;

import java.io.ByteArrayInputStream;
import java.math.BigDecimal;
import java.nio.charset.StandardCharsets;
import java.sql.Connection;
import java.sql.SQLException;

/**
* @program: flinkx
* @author: jier
*/
public class GreenplumOutputFormat extends JdbcOutputFormat {

// pg 字符串里含有\u0000 会报错 ERROR: invalid byte sequence for encoding "UTF8": 0x00
public static final String SPACE = "\u0000";

private static final String LINE_DELIMITER = "\n";
private CopyManager copyManager;
private boolean disableCopyMode = false;
private String copySql = "";
public static final String INSERT_SQL_MODE_TYPE = "copy";
private static final String DEFAULT_FIELD_DELIMITER = "\001";

private static final String DEFAULT_NULL_VALUE = "\002";

/** 数据源类型信息 * */
private final String dbType = DbType.POSTGRESQL.name();

@Override
protected void openInternal(int taskNumber, int numTasks) {
super.openInternal(taskNumber, numTasks);
try {
// check is use copy mode for insert
disableCopyMode =
jdbcConf.getInsertSqlMode() != null
&& !INSERT_SQL_MODE_TYPE.equalsIgnoreCase(jdbcConf.getInsertSqlMode());
if (EWriteMode.INSERT.name().equalsIgnoreCase(jdbcConf.getMode()) && !disableCopyMode) {
LOG.info("will use copy mode");
copyManager = new CopyManager((BaseConnection) dbConn);

PostgresqlDialect pgDialect = (PostgresqlDialect) jdbcDialect;
copySql =
pgDialect.getCopyStatement(
jdbcConf.getSchema(),
jdbcConf.getTable(),
columnNameList.toArray(new String[0]),
DEFAULT_FIELD_DELIMITER,
DEFAULT_NULL_VALUE);

LOG.info("write sql:{}", copySql);
}
checkUpsert();
if (rowConverter instanceof PostgresqlColumnConverter
&& dbConn instanceof BaseConnection) {
((PostgresqlColumnConverter) rowConverter).setConnection((BaseConnection) dbConn);
}
} catch (SQLException sqe) {
throw new IllegalArgumentException("checkUpsert() failed.", sqe);
}
}

@Override
protected void writeSingleRecordInternal(RowData row) throws WriteRecordException {
if (disableCopyMode) {
super.writeSingleRecordInternal(row);
} else {
if (rowConverter instanceof JdbcColumnConverter) {
ColumnRowData colRowData = (ColumnRowData) row;
// write with copy
int index = 0;
try {
StringBuilder rowStr = new StringBuilder();
int lastIndex = row.getArity() - 1;
for (; index < row.getArity(); index++) {
appendColumn(colRowData, index, rowStr, index == lastIndex);
}
String rowVal = copyModeReplace(rowStr.toString());
try (ByteArrayInputStream bi =
new ByteArrayInputStream(rowVal.getBytes(StandardCharsets.UTF_8))) {
copyManager.copyIn(copySql, bi);
}
} catch (Exception e) {
processWriteException(e, index, row);
}
} else {
throw new NoRestartException("copy mode only support data sync with out table");
}
}
}

@Override
protected void writeMultipleRecordsInternal() throws Exception {
if (disableCopyMode) {
super.writeMultipleRecordsInternal();
} else {
if (rowConverter instanceof JdbcColumnConverter) {
StringBuilder rowsStrBuilder = new StringBuilder(128);
for (RowData row : rows) {
ColumnRowData colRowData = (ColumnRowData) row;
int lastIndex = row.getArity() - 1;
StringBuilder rowStr = new StringBuilder(128);
for (int index = 0; index < row.getArity(); index++) {
appendColumn(colRowData, index, rowStr, index == lastIndex);
}
String tempData = rowStr.toString();
rowsStrBuilder.append(copyModeReplace(tempData)).append(LINE_DELIMITER);
}
String rowVal = rowsStrBuilder.toString();
try (ByteArrayInputStream bi =
new ByteArrayInputStream(rowVal.getBytes(StandardCharsets.UTF_8))) {
copyManager.copyIn(copySql, bi);
if (checkpointEnabled && CheckpointingMode.EXACTLY_ONCE == checkpointMode) {
rowsOfCurrentTransaction += rows.size();
}
}
} else {
throw new NoRestartException("copy mode only support data sync with out table");
}
}
}

private void appendColumn(
ColumnRowData colRowData, int pos, StringBuilder rowStr, boolean isLast) {
Object col = colRowData.getField(pos);
if (col == null) {
rowStr.append(DEFAULT_NULL_VALUE);
} else {
rowStr.append(col);
}
if (!isLast) {
rowStr.append(DEFAULT_FIELD_DELIMITER);
}
}

/**
* \r \n \ 等特殊字符串需要转义
*
* @return
*/
private String copyModeReplace(String rowStr) {
if (rowStr.contains("\\")) {
rowStr = rowStr.replaceAll("\\\\", "\\\\\\\\");
}
if (rowStr.contains("\r")) {
rowStr = rowStr.replaceAll("\r", "\\\\r");
}

if (rowStr.contains("\n")) {
rowStr = rowStr.replaceAll("\n", "\\\\n");
}

// pg 字符串里含有\u0000 会报错 ERROR: invalid byte sequence for encoding "UTF8": 0x00
if (rowStr.contains(SPACE)) {
rowStr = rowStr.replaceAll(SPACE, "");
}
return rowStr;
}

/** 数据源类型 * */
public enum DbType {
POSTGRESQL,
ADB
}

/**
* 当mode为update时进行校验
*
* @return
* @throws SQLException
*/
public void checkUpsert() throws SQLException {
if (EWriteMode.UPDATE.name().equalsIgnoreCase(jdbcConf.getMode())) {
try (Connection connection = getConnection()) {

// 效验版本
String databaseProductVersion =
connection.getMetaData().getDatabaseProductVersion();
LOG.info("source version is {}", databaseProductVersion);
String[] split = databaseProductVersion.split("\\.");
// 10.1.12
if (split.length > 2) {
databaseProductVersion = split[0] + ConstantValue.POINT_SYMBOL + split[1];
}

if (NumberUtils.isNumber(databaseProductVersion)) {
BigDecimal sourceVersion = new BigDecimal(databaseProductVersion);
if (dbType.equalsIgnoreCase(DbType.POSTGRESQL.name())) {
// pg大于等于9.5
if (sourceVersion.compareTo(new BigDecimal("9.5")) < 0) {
throw new RuntimeException(
"the postgreSql version is ["
+ databaseProductVersion
+ "] and must greater than or equal to 9.5 when you use update mode and source is "
+ DbType.POSTGRESQL.name());
}
} else if (dbType.equalsIgnoreCase(DbType.ADB.name())) {
// adb大于等于9.4
if (sourceVersion.compareTo(new BigDecimal("9.4")) < 0) {
throw new RuntimeException(
"the postgreSql version is ["
+ databaseProductVersion
+ "] and must greater than or equal to 9.4 when you use update mode and source is "
+ DbType.ADB.name());
}
}
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,16 +20,44 @@

import com.dtstack.chunjun.conf.SyncConf;
import com.dtstack.chunjun.connector.greenplum.dialect.GreenplumDialect;
import com.dtstack.chunjun.connector.postgresql.sink.PostgresqlSinkFactory;
import com.dtstack.chunjun.connector.jdbc.sink.JdbcOutputFormatBuilder;
import com.dtstack.chunjun.connector.jdbc.sink.JdbcSinkFactory;
import com.dtstack.chunjun.connector.postgresql.dialect.PostgresqlDialect;

import org.apache.commons.lang.StringUtils;

import static com.dtstack.chunjun.connector.greenplum.sink.GreenplumOutputFormat.INSERT_SQL_MODE_TYPE;

/**
* company www.dtstack.com
*
* @author jier
*/
public class GreenplumSinkFactory extends PostgresqlSinkFactory {
public class GreenplumSinkFactory extends JdbcSinkFactory {

public GreenplumSinkFactory(SyncConf syncConf) {
super(syncConf, new GreenplumDialect());
super(syncConf, null);
if (syncConf.getWriter().getParameter().get("insertSqlMode") != null
&& INSERT_SQL_MODE_TYPE.equalsIgnoreCase(
syncConf.getWriter().getParameter().get("insertSqlMode").toString())) {
this.jdbcDialect = new PostgresqlDialect();
String pgUrl = changeToPostgresqlUrl(this.jdbcConf.getJdbcUrl());
this.jdbcConf.setJdbcUrl(pgUrl);
} else {
this.jdbcDialect = new GreenplumDialect();
}
}

@Override
protected JdbcOutputFormatBuilder getBuilder() {
return new JdbcOutputFormatBuilder(new GreenplumOutputFormat());
}

private String changeToPostgresqlUrl(String gpUrl) {
String pgUrl =
StringUtils.replaceOnce(
gpUrl, GreenplumDialect.URL_START, PostgresqlDialect.URL_START);
pgUrl = StringUtils.replaceOnce(pgUrl, GreenplumDialect.DATABASE_NAME, "/");
return pgUrl;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,10 @@ public class JdbcConf extends ChunJunCommonConf implements Serializable {
/** upsert 写数据库时,是否null覆盖原来的值 */
protected boolean allReplace = false;

protected boolean isAutoCommit = false;

private boolean defineColumnTypeForStatement = false;

public Boolean getInitReporter() {
return initReporter;
}
Expand Down Expand Up @@ -419,6 +423,14 @@ public void setAllReplace(boolean allReplace) {
this.allReplace = allReplace;
}

public boolean isAutoCommit() {
return isAutoCommit;
}

public boolean isDefineColumnTypeForStatement() {
return defineColumnTypeForStatement;
}

public String getSplitStrategy() {
return splitStrategy;
}
Expand Down Expand Up @@ -485,9 +497,13 @@ public String toString() {
+ increment
+ ", polling="
+ polling
+ ", pollingFromMax="
+ pollingFromMax
+ ", increColumn='"
+ increColumn
+ '\''
+ ", isOrderBy="
+ isOrderBy
+ ", increColumnIndex="
+ increColumnIndex
+ ", increColumnType='"
Expand All @@ -508,6 +524,8 @@ public String toString() {
+ restoreColumnIndex
+ ", useMaxFunc="
+ useMaxFunc
+ ", initReporter="
+ initReporter
+ ", mode='"
+ mode
+ '\''
Expand All @@ -521,6 +539,10 @@ public String toString() {
+ updateKey
+ ", allReplace="
+ allReplace
+ ", isAutoCommit="
+ isAutoCommit
+ ", defineColumnTypeForStatement="
+ defineColumnTypeForStatement
+ '}';
}
}
Loading