Skip to content

Commit 99ed97d

Browse files
libailinlihongwei
authored and
lihongwei
committed
[Feature-DTStack#1933][s3] Support more data type conversions
1 parent 1d77727 commit 99ed97d

File tree

1 file changed

+58
-2
lines changed
  • chunjun-connectors/chunjun-connector-s3/src/main/java/com/dtstack/chunjun/connector/s3/converter

1 file changed

+58
-2
lines changed

chunjun-connectors/chunjun-connector-s3/src/main/java/com/dtstack/chunjun/connector/s3/converter/S3SqlConverter.java

+58-2
Original file line numberDiff line numberDiff line change
@@ -23,16 +23,27 @@
2323
import com.dtstack.chunjun.converter.IDeserializationConverter;
2424
import com.dtstack.chunjun.converter.ISerializationConverter;
2525
import com.dtstack.chunjun.throwable.UnsupportedTypeException;
26+
import com.dtstack.chunjun.util.DateUtil;
2627

28+
import org.apache.flink.table.data.DecimalData;
2729
import org.apache.flink.table.data.GenericRowData;
2830
import org.apache.flink.table.data.RowData;
2931
import org.apache.flink.table.data.StringData;
32+
import org.apache.flink.table.data.TimestampData;
33+
import org.apache.flink.table.types.logical.DecimalType;
3034
import org.apache.flink.table.types.logical.LogicalType;
3135
import org.apache.flink.table.types.logical.RowType;
36+
import org.apache.flink.table.types.logical.TimestampType;
3237

38+
import org.apache.commons.lang3.StringUtils;
39+
40+
import java.math.BigDecimal;
41+
import java.math.BigInteger;
3342
import java.sql.Date;
3443
import java.sql.Time;
44+
import java.sql.Timestamp;
3545
import java.time.LocalDate;
46+
import java.time.LocalDateTime;
3647
import java.time.LocalTime;
3748

3849
public class S3SqlConverter extends AbstractRowConverter<String[], RowData, String[], LogicalType> {
@@ -82,12 +93,41 @@ protected IDeserializationConverter createInternalConverter(LogicalType type) {
8293
return val -> Float.valueOf((String) val);
8394
case DOUBLE:
8495
return val -> Double.valueOf((String) val);
96+
case DECIMAL:
97+
final int precision = ((DecimalType) type).getPrecision();
98+
final int scale = ((DecimalType) type).getScale();
99+
return val ->
100+
val instanceof BigInteger
101+
? DecimalData.fromBigDecimal(
102+
new BigDecimal((BigInteger) val, 0), precision, scale)
103+
: DecimalData.fromBigDecimal(
104+
StringUtils.isNotEmpty(String.valueOf(val))
105+
? new BigDecimal(String.valueOf(val))
106+
: BigDecimal.ZERO,
107+
precision,
108+
scale);
109+
case TIMESTAMP_WITH_TIME_ZONE:
110+
case TIMESTAMP_WITHOUT_TIME_ZONE:
111+
return val -> {
112+
if (val instanceof String) {
113+
return TimestampData.fromTimestamp(Timestamp.valueOf((String) val));
114+
} else if (val instanceof LocalDateTime) {
115+
return TimestampData.fromTimestamp(Timestamp.valueOf((LocalDateTime) val));
116+
} else {
117+
return TimestampData.fromTimestamp(((Timestamp) val));
118+
}
119+
};
85120
case CHAR:
86121
case VARCHAR:
87122
return val -> StringData.fromString((String) val);
88123
case DATE:
89-
return val ->
90-
(int) ((Date.valueOf(String.valueOf(val))).toLocalDate().toEpochDay());
124+
return val -> {
125+
if (StringUtils.isEmpty(String.valueOf(val))) {
126+
return null;
127+
}
128+
Date date = new Date(DateUtil.stringToDate(String.valueOf(val)).getTime());
129+
return (int) date.toLocalDate().toEpochDay();
130+
};
91131
case TIME_WITHOUT_TIME_ZONE:
92132
return val ->
93133
(int)
@@ -124,6 +164,22 @@ protected ISerializationConverter<String[]> createExternalConverter(LogicalType
124164
output[index] =
125165
Time.valueOf(LocalTime.ofNanoOfDay(val.getInt(index) * 1_000_000L))
126166
.toString();
167+
case DECIMAL:
168+
return (rowData, index, data) ->
169+
data[index] =
170+
String.valueOf(
171+
rowData.getDecimal(
172+
index,
173+
((DecimalType) type).getPrecision(),
174+
((DecimalType) type).getScale()));
175+
case TIMESTAMP_WITHOUT_TIME_ZONE:
176+
return (rowData, index, data) ->
177+
data[index] =
178+
String.valueOf(
179+
rowData.getTimestamp(
180+
index,
181+
((TimestampType) type).getPrecision())
182+
.toTimestamp());
127183
default:
128184
throw new UnsupportedTypeException(type.toString());
129185
}

0 commit comments

Comments
 (0)