Skip to content

Commit 04028d5

Browse files
committed
deal oracle char type
1 parent 0572368 commit 04028d5

File tree

3 files changed

+83
-18
lines changed

3 files changed

+83
-18
lines changed

oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleDialect.java

Lines changed: 71 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,14 @@
3535
*/
3636
public class OracleDialect implements JDBCDialect {
3737

38+
private final String SQL_DEFAULT_PLACEHOLDER = " ? ";
39+
private final String DEAL_CHAR_KEY = "char";
40+
private String RPAD_FORMAT = " rpad(?, %d, ' ') ";
41+
42+
private List<String> fieldList;
43+
private List<String> fieldTypeList;
44+
private List<TableInfo.FieldExtraInfo> fieldExtraInfoList;
45+
3846
@Override
3947
public boolean canHandle(String url) {
4048
return url.startsWith("jdbc:oracle:");
@@ -48,39 +56,50 @@ public Optional<String> defaultDriverName() {
4856
@Override
4957
public Optional<String> getUpsertStatement(String schema, String tableName, String[] fieldNames, String[] uniqueKeyFields, boolean allReplace) {
5058
tableName = DtStringUtil.getTableFullPath(schema, tableName);
51-
StringBuilder sb = new StringBuilder();
52-
sb.append("MERGE INTO " + tableName + " T1 USING "
53-
+ "(" + buildDualQueryStatement(fieldNames) + ") T2 ON ("
54-
+ buildConnectionConditions(uniqueKeyFields) + ") ");
59+
StringBuilder mergeIntoSql = new StringBuilder();
60+
mergeIntoSql.append("MERGE INTO " + tableName + " T1 USING (")
61+
.append(buildDualQueryStatement(fieldNames))
62+
.append(") T2 ON (")
63+
.append(buildConnectionConditions(uniqueKeyFields) + ") ");
5564

5665
String updateSql = buildUpdateConnection(fieldNames, uniqueKeyFields, allReplace);
5766

5867
if (StringUtils.isNotEmpty(updateSql)) {
59-
sb.append(" WHEN MATCHED THEN UPDATE SET ");
60-
sb.append(updateSql);
68+
mergeIntoSql.append(" WHEN MATCHED THEN UPDATE SET ");
69+
mergeIntoSql.append(updateSql);
6170
}
6271

63-
sb.append(" WHEN NOT MATCHED THEN "
64-
+ "INSERT (" + Arrays.stream(fieldNames).map(this::quoteIdentifier).collect(Collectors.joining(",")) + ") VALUES ("
65-
+ Arrays.stream(fieldNames).map(col -> "T2." + quoteIdentifier(col)).collect(Collectors.joining(",")) + ")");
72+
mergeIntoSql.append(" WHEN NOT MATCHED THEN ")
73+
.append("INSERT (")
74+
.append(Arrays.stream(fieldNames).map(this::quoteIdentifier).collect(Collectors.joining(",")))
75+
.append(") VALUES (")
76+
.append(Arrays.stream(fieldNames).map(col -> "T2." + quoteIdentifier(col)).collect(Collectors.joining(",")))
77+
.append(")");
6678

67-
return Optional.of(sb.toString());
79+
return Optional.of(mergeIntoSql.toString());
6880
}
6981

7082
/**
71-
* build T1."A"=T2."A" or T1."A"=nvl(T2."A",T1."A")
83+
* build T1."A"=T2."A" or T1."A"=nvl(T2."A",T1."A")
7284
* @param fieldNames
7385
* @param uniqueKeyFields
7486
* @param allReplace
7587
* @return
7688
*/
7789
private String buildUpdateConnection(String[] fieldNames, String[] uniqueKeyFields, boolean allReplace) {
7890
List<String> uniqueKeyList = Arrays.asList(uniqueKeyFields);
79-
return Arrays.stream(fieldNames).filter(col -> !uniqueKeyList.contains(col)).map(col -> {
80-
return allReplace ? quoteIdentifier("T1") + "." + quoteIdentifier(col) + " = " + quoteIdentifier("T2") + "." + quoteIdentifier(col) :
81-
quoteIdentifier("T1") + "." + quoteIdentifier(col) + " =nvl(" + quoteIdentifier("T2") + "." + quoteIdentifier(col) + ","
82-
+ quoteIdentifier("T1") + "." + quoteIdentifier(col) + ")";
83-
}).collect(Collectors.joining(","));
91+
String updateConnectionSql = Arrays.stream(fieldNames).
92+
filter(col -> !uniqueKeyList.contains(col))
93+
.map(col -> buildConnectionByAllReplace(allReplace, col))
94+
.collect(Collectors.joining(","));
95+
return updateConnectionSql;
96+
}
97+
98+
private String buildConnectionByAllReplace(boolean allReplace, String col) {
99+
String conncetionSql = allReplace ? quoteIdentifier("T1") + "." + quoteIdentifier(col) + " = " + quoteIdentifier("T2") + "." + quoteIdentifier(col) :
100+
quoteIdentifier("T1") + "." + quoteIdentifier(col) + " =nvl(" + quoteIdentifier("T2") + "." + quoteIdentifier(col) + ","
101+
+ quoteIdentifier("T1") + "." + quoteIdentifier(col) + ")";
102+
return conncetionSql;
84103
}
85104

86105

@@ -96,8 +115,43 @@ private String buildConnectionConditions(String[] uniqueKeyFields) {
96115
*/
97116
public String buildDualQueryStatement(String[] column) {
98117
StringBuilder sb = new StringBuilder("SELECT ");
99-
String collect = Arrays.stream(column).map(col -> " ? " + quoteIdentifier(col)).collect(Collectors.joining(", "));
118+
String collect = Arrays.stream(column)
119+
.map(col -> wrapperPlaceholder(col) + quoteIdentifier(col))
120+
.collect(Collectors.joining(", "));
100121
sb.append(collect).append(" FROM DUAL");
101122
return sb.toString();
102123
}
124+
125+
126+
/**
127+
* char type is wrapped with rpad
128+
* @param fieldName
129+
* @return
130+
*/
131+
public String wrapperPlaceholder(String fieldName) {
132+
int pos = fieldList.indexOf(fieldName);
133+
String type = fieldTypeList.get(pos);
134+
135+
if (StringUtils.contains(type.toLowerCase(), DEAL_CHAR_KEY)) {
136+
TableInfo.FieldExtraInfo fieldExtraInfo = fieldExtraInfoList.get(pos);
137+
int charLength = fieldExtraInfo == null ? 0 : fieldExtraInfo.getLength();
138+
if (charLength > 0) {
139+
return String.format(RPAD_FORMAT, charLength);
140+
}
141+
}
142+
return SQL_DEFAULT_PLACEHOLDER;
143+
}
144+
145+
146+
public void setFieldList(List<String> fieldList) {
147+
this.fieldList = fieldList;
148+
}
149+
150+
public void setFieldTypeList(List<String> fieldTypeList) {
151+
this.fieldTypeList = fieldTypeList;
152+
}
153+
154+
public void setFieldExtraInfoList(List<TableInfo.FieldExtraInfo> fieldExtraInfoList) {
155+
this.fieldExtraInfoList = fieldExtraInfoList;
156+
}
103157
}

oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,10 @@ public OracleSink() {
3737

3838
@Override
3939
public JDBCUpsertOutputFormat getOutputFormat() {
40+
((OracleDialect) jdbcDialect).setFieldList(fieldList);
41+
((OracleDialect) jdbcDialect).setFieldTypeList(fieldTypeList);
42+
((OracleDialect) jdbcDialect).setFieldExtraInfoList(fieldExtraInfoList);
43+
4044
JDBCOptions jdbcOptions = JDBCOptions.builder()
4145
.setDBUrl(dbURL)
4246
.setDialect(jdbcDialect)

rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
import com.dtstack.flink.sql.sink.IStreamSinkGener;
2121
import com.dtstack.flink.sql.sink.rdb.format.JDBCUpsertOutputFormat;
2222
import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo;
23+
import com.dtstack.flink.sql.table.TableInfo;
2324
import com.dtstack.flink.sql.table.TargetTableInfo;
2425
import org.apache.flink.api.common.typeinfo.TypeInformation;
2526
import org.apache.flink.api.java.tuple.Tuple2;
@@ -83,6 +84,10 @@ public abstract class RdbSink implements RetractStreamTableSink<Row>, Serializab
8384

8485
protected String updateMode;
8586

87+
public List<String> fieldList;
88+
public List<String> fieldTypeList;
89+
public List<TableInfo.FieldExtraInfo> fieldExtraInfoList;
90+
8691
public RdbSink(JDBCDialect jdbcDialect) {
8792
this.jdbcDialect = jdbcDialect;
8893
}
@@ -106,7 +111,9 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) {
106111
this.sqlTypes = JDBCTypeConvertUtils.buildSqlTypes(fieldTypeArray);
107112
this.allReplace = rdbTableInfo.isAllReplace();
108113
this.updateMode = rdbTableInfo.getUpdateMode();
109-
rdbTableInfo.getFieldList();
114+
this.fieldList = rdbTableInfo.getFieldList();
115+
this.fieldTypeList = rdbTableInfo.getFieldTypeList();
116+
this.fieldExtraInfoList = rdbTableInfo.getFieldExtraInfoList();
110117
return this;
111118
}
112119

0 commit comments

Comments
 (0)