Skip to content
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 @@ -21,6 +21,7 @@
import com.dtstack.chunjun.conf.ChunJunCommonConf;
import com.dtstack.chunjun.conf.FieldConf;
import com.dtstack.chunjun.connector.jdbc.statement.FieldNamedPreparedStatement;
import com.dtstack.chunjun.constants.ConstantValue;
import com.dtstack.chunjun.converter.AbstractRowConverter;
import com.dtstack.chunjun.converter.IDeserializationConverter;
import com.dtstack.chunjun.converter.ISerializationConverter;
Expand Down Expand Up @@ -88,12 +89,25 @@ public JdbcColumnConverter(RowType rowType, ChunJunCommonConf commonConf) {
@SuppressWarnings("unchecked")
public RowData toInternal(ResultSet resultSet) throws Exception {
List<FieldConf> fieldConfList = commonConf.getColumn();
ColumnRowData result = new ColumnRowData(fieldConfList.size());
ColumnRowData result;
if (fieldConfList.size() == 1
&& ConstantValue.STAR_SYMBOL.equals(fieldConfList.get(0).getName())) {
result = new ColumnRowData(fieldTypes.length);
for (int index = 0; index < fieldTypes.length; index++) {
Object field = resultSet.getObject(index + 1);
AbstractBaseColumn baseColumn =
(AbstractBaseColumn) toInternalConverters.get(index).deserialize(field);
result.addField(baseColumn);
}
return result;
}
int converterIndex = 0;
result = new ColumnRowData(fieldConfList.size());
for (FieldConf fieldConf : fieldConfList) {
AbstractBaseColumn baseColumn = null;
if (StringUtils.isBlank(fieldConf.getValue())) {
Object field = resultSet.getObject(converterIndex + 1);

baseColumn =
(AbstractBaseColumn)
toInternalConverters.get(converterIndex).deserialize(field);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import java.io.InputStream;
import java.io.Reader;
import java.math.BigDecimal;
import java.sql.Array;
import java.sql.Connection;
import java.sql.Date;
import java.sql.ResultSet;
Expand Down Expand Up @@ -354,6 +355,11 @@ public void setClob(int fieldIndex, Reader reader) throws SQLException {
currentFieldNamedPstmt.setClob(fieldIndex, reader);
}

@Override
public void setArray(int fieldIndex, Array array) throws SQLException {
currentFieldNamedPstmt.setArray(fieldIndex, array);
}

@Override
public void close() throws SQLException {
currentFieldNamedPstmt.close();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.io.InputStream;
import java.io.Reader;
import java.math.BigDecimal;
import java.sql.Array;
import java.sql.Connection;
import java.sql.Date;
import java.sql.PreparedStatement;
Expand Down Expand Up @@ -250,6 +251,9 @@ static FieldNamedPreparedStatement prepareStatement(
void setBlob(int fieldIndex, InputStream is) throws SQLException;

void setClob(int fieldIndex, Reader reader) throws SQLException;

void setArray(int fieldIndex, Array array) throws SQLException;

/**
* Releases this <code>Statement</code> object's database and JDBC resources immediately instead
* of waiting for this to happen when it is automatically closed. It is generally good practice
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.io.InputStream;
import java.io.Reader;
import java.math.BigDecimal;
import java.sql.Array;
import java.sql.Connection;
import java.sql.Date;
import java.sql.PreparedStatement;
Expand Down Expand Up @@ -258,6 +259,13 @@ public void setClob(int fieldIndex, Reader reader) throws SQLException {
}
}

@Override
public void setArray(int fieldIndex, Array array) throws SQLException {
for (int index : indexMapping[fieldIndex]) {
statement.setArray(index, array);
}
}

@Override
public void close() throws SQLException {
statement.close();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,227 @@
/*
*
* *
* * * 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.postgresql.converter;

import com.dtstack.chunjun.conf.ChunJunCommonConf;
import com.dtstack.chunjun.connector.jdbc.converter.JdbcColumnConverter;
import com.dtstack.chunjun.connector.jdbc.statement.FieldNamedPreparedStatement;
import com.dtstack.chunjun.converter.IDeserializationConverter;
import com.dtstack.chunjun.converter.ISerializationConverter;
import com.dtstack.chunjun.element.AbstractBaseColumn;
import com.dtstack.chunjun.element.ColumnRowData;
import com.dtstack.chunjun.element.column.ArrayColumn;
import com.dtstack.chunjun.element.column.BigDecimalColumn;
import com.dtstack.chunjun.element.column.BooleanColumn;
import com.dtstack.chunjun.element.column.BytesColumn;
import com.dtstack.chunjun.element.column.SqlDateColumn;
import com.dtstack.chunjun.element.column.StringColumn;
import com.dtstack.chunjun.element.column.TimeColumn;
import com.dtstack.chunjun.element.column.TimestampColumn;

import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.types.logical.TimestampType;
import org.apache.flink.table.types.logical.YearMonthIntervalType;

import org.postgresql.core.BaseConnection;
import org.postgresql.core.Oid;
import org.postgresql.jdbc.PgArray;

import java.math.BigDecimal;
import java.sql.Array;
import java.sql.Date;
import java.sql.Time;
import java.sql.Timestamp;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

/**
* Company:www.dtstack.com.
*
* @author shitou
* @date 2022/4/14
*/
public class PostgresqlColumnConverter extends JdbcColumnConverter {

private List<String> fieldTypeList;
private transient BaseConnection connection;
private static final Map<String, Integer> arrayType = new HashMap<>();

public PostgresqlColumnConverter(RowType rowType, ChunJunCommonConf commonConf) {
super(rowType, commonConf);
}

static {
arrayType.put("_int4", Oid.INT4_ARRAY);
arrayType.put("_int8", Oid.INT8_ARRAY);
arrayType.put("_float4", Oid.FLOAT4_ARRAY);
arrayType.put("_text", Oid.TEXT_ARRAY);
}

@Override
public FieldNamedPreparedStatement toExternal(
RowData rowData, FieldNamedPreparedStatement statement) throws Exception {
for (int index = 0; index < rowData.getArity(); index++) {
if (arrayType.containsKey(fieldTypeList.get(index))) {
// eg: {1000,1000,10001}、{{1000,1000,10001},{1,2,3}}
String field = ((ColumnRowData) rowData).getField(index).asString();
Array array =
new PgArray(connection, arrayType.get(fieldTypeList.get(index)), field);
AbstractBaseColumn arrayColumn = new ArrayColumn(array);
((ColumnRowData) rowData).setField(index, arrayColumn);
}
toExternalConverters.get(index).serialize(rowData, index, statement);
}
return statement;
}

@Override
protected IDeserializationConverter createInternalConverter(LogicalType type) {
switch (type.getTypeRoot()) {
case BOOLEAN:
return val -> new BooleanColumn(Boolean.parseBoolean(val.toString()));
case TINYINT:
return val -> new BigDecimalColumn(((Integer) val).byteValue());
case SMALLINT:
case INTEGER:
return val -> new BigDecimalColumn((Integer) val);
case INTERVAL_YEAR_MONTH:
return (IDeserializationConverter<Object, AbstractBaseColumn>)
val -> {
YearMonthIntervalType yearMonthIntervalType =
(YearMonthIntervalType) type;
switch (yearMonthIntervalType.getResolution()) {
case YEAR:
return new BigDecimalColumn(
Integer.parseInt(String.valueOf(val).substring(0, 4)));
case MONTH:
case YEAR_TO_MONTH:
default:
throw new UnsupportedOperationException(
"jdbc converter only support YEAR");
}
};
case FLOAT:
return val -> new BigDecimalColumn((Float) val);
case DOUBLE:
return val -> new BigDecimalColumn((Double) val);
case BIGINT:
return val -> new BigDecimalColumn((Long) val);
case DECIMAL:
return val -> new BigDecimalColumn((BigDecimal) val);
case CHAR:
case VARCHAR:
return val -> new StringColumn((String) val);
case DATE:
return val -> new SqlDateColumn((Date) val);
case TIME_WITHOUT_TIME_ZONE:
return val -> new TimeColumn((Time) val);
case TIMESTAMP_WITH_TIME_ZONE:
case TIMESTAMP_WITHOUT_TIME_ZONE:
return (IDeserializationConverter<Object, AbstractBaseColumn>)
val ->
new TimestampColumn(
(Timestamp) val, ((TimestampType) (type)).getPrecision());

case BINARY:
case VARBINARY:
return val -> new BytesColumn((byte[]) val);
case ARRAY:
// integer[] -> {1000,1000,10001}
// integer[][]-> {{1000,1000,10001},{1,2,3}}
return val -> new StringColumn(((Array) val).toString());
default:
throw new UnsupportedOperationException("Unsupported type:" + type);
}
}

@Override
protected ISerializationConverter<FieldNamedPreparedStatement> createExternalConverter(
LogicalType type) {
switch (type.getTypeRoot()) {
case BOOLEAN:
return (val, index, statement) ->
statement.setBoolean(
index, ((ColumnRowData) val).getField(index).asBoolean());
case TINYINT:
return (val, index, statement) -> statement.setByte(index, val.getByte(index));
case SMALLINT:
case INTEGER:
case INTERVAL_YEAR_MONTH:
return (val, index, statement) ->
statement.setInt(index, ((ColumnRowData) val).getField(index).asYearInt());
case FLOAT:
return (val, index, statement) ->
statement.setFloat(index, ((ColumnRowData) val).getField(index).asFloat());
case DOUBLE:
return (val, index, statement) ->
statement.setDouble(
index, ((ColumnRowData) val).getField(index).asDouble());

case BIGINT:
return (val, index, statement) ->
statement.setLong(index, ((ColumnRowData) val).getField(index).asLong());
case DECIMAL:
return (val, index, statement) ->
statement.setBigDecimal(
index, ((ColumnRowData) val).getField(index).asBigDecimal());
case CHAR:
case VARCHAR:
return (val, index, statement) ->
statement.setString(
index, ((ColumnRowData) val).getField(index).asString());
case DATE:
return (val, index, statement) ->
statement.setDate(index, ((ColumnRowData) val).getField(index).asSqlDate());
case TIME_WITHOUT_TIME_ZONE:
return (val, index, statement) ->
statement.setTime(index, ((ColumnRowData) val).getField(index).asTime());
case TIMESTAMP_WITH_TIME_ZONE:
case TIMESTAMP_WITHOUT_TIME_ZONE:
return (val, index, statement) ->
statement.setTimestamp(
index, ((ColumnRowData) val).getField(index).asTimestamp());

case BINARY:
case VARBINARY:
return (val, index, statement) ->
statement.setBytes(index, ((ColumnRowData) val).getField(index).asBytes());
case ARRAY:
return (val, index, statement) ->
statement.setArray(
index, (Array) ((ColumnRowData) val).getField(index).getData());
default:
throw new UnsupportedOperationException("Unsupported type:" + type);
}
}

public void setFieldTypeList(List<String> fieldTypeList) {
this.fieldTypeList = fieldTypeList;
}

public void setConnection(BaseConnection connection) {
this.connection = connection;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,11 @@
import com.dtstack.chunjun.throwable.UnsupportedTypeException;

import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.types.AtomicDataType;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.FloatType;
import org.apache.flink.table.types.logical.IntType;
import org.apache.flink.table.types.logical.VarCharType;

import java.util.Locale;

Expand Down Expand Up @@ -95,6 +99,13 @@ public static DataType apply(String type) {
case "BOOLEAN":
case "BOOL":
return DataTypes.BOOLEAN();
case "_INT4":
case "_INT8":
return DataTypes.ARRAY(new AtomicDataType(new IntType()));
case "_TEXT":
return DataTypes.ARRAY(new AtomicDataType(new VarCharType()));
case "_FLOAT4":
return DataTypes.ARRAY(new AtomicDataType(new FloatType()));

// 以下类型无法支持
// Enumerated Types
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,22 @@

package com.dtstack.chunjun.connector.postgresql.dialect;

import com.dtstack.chunjun.conf.ChunJunCommonConf;
import com.dtstack.chunjun.connector.jdbc.dialect.JdbcDialect;
import com.dtstack.chunjun.connector.jdbc.statement.FieldNamedPreparedStatement;
import com.dtstack.chunjun.connector.jdbc.util.JdbcUtil;
import com.dtstack.chunjun.connector.postgresql.converter.PostgresqlColumnConverter;
import com.dtstack.chunjun.connector.postgresql.converter.PostgresqlRawTypeConverter;
import com.dtstack.chunjun.converter.AbstractRowConverter;
import com.dtstack.chunjun.converter.RawTypeConverter;

import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;

import io.vertx.core.json.JsonArray;
import org.apache.commons.lang3.StringUtils;

import java.sql.ResultSet;
import java.util.Arrays;
import java.util.Optional;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -58,6 +67,12 @@ public RawTypeConverter getRawTypeConverter() {
return PostgresqlRawTypeConverter::apply;
}

@Override
public AbstractRowConverter<ResultSet, JsonArray, FieldNamedPreparedStatement, LogicalType>
getColumnConverter(RowType rowType, ChunJunCommonConf commonConf) {
return new PostgresqlColumnConverter(rowType, commonConf);
}

@Override
public Optional<String> defaultDriverName() {
return Optional.of(DRIVER);
Expand Down
Loading