From c49d18897d8448c7a65d3fc7ca21d50905772f1f Mon Sep 17 00:00:00 2001 From: vinnerzhang Date: Fri, 5 Jul 2024 18:10:23 +0800 Subject: [PATCH] [#369][Bugfix] Change the starrocks connector UNKNOWN data type handle method --- .../flink/row/sink/StarRocksTableRowTransformer.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/starrocks/connector/flink/row/sink/StarRocksTableRowTransformer.java b/src/main/java/com/starrocks/connector/flink/row/sink/StarRocksTableRowTransformer.java index aef0c02b..9fa5f2e7 100644 --- a/src/main/java/com/starrocks/connector/flink/row/sink/StarRocksTableRowTransformer.java +++ b/src/main/java/com/starrocks/connector/flink/row/sink/StarRocksTableRowTransformer.java @@ -130,8 +130,10 @@ private Object typeConvertion(LogicalType type, RowData record, int pos) { } StarRocksDataType starRocksDataType = columns.getOrDefault(columnNames[pos], StarRocksDataType.UNKNOWN); - if ((starRocksDataType == StarRocksDataType.JSON || - starRocksDataType == StarRocksDataType.UNKNOWN) + if (starRocksDataType == StarRocksDataType.UNKNOWN) { + return sValue; + } + if ((starRocksDataType == StarRocksDataType.JSON) && (sValue.charAt(0) == '{' || sValue.charAt(0) == '[')) { // The json string need to be converted to a json object, and to the json string // again via JSON.toJSONString in StarRocksJsonSerializer#serialize. Otherwise,