|
24 | 24 | import org.apache.commons.codec.binary.Hex;
|
25 | 25 | import org.apache.flink.table.api.ResultKind;
|
26 | 26 | import org.apache.flink.table.catalog.Column;
|
27 |
| -import org.apache.flink.table.data.ArrayData; |
28 |
| -import org.apache.flink.table.data.MapData; |
29 |
| -import org.apache.flink.table.data.RowData; |
| 27 | +import org.apache.flink.table.data.*; |
30 | 28 | import org.apache.flink.table.gateway.api.results.ResultSet;
|
31 | 29 | import org.apache.flink.table.types.logical.*;
|
32 | 30 |
|
@@ -128,8 +126,9 @@ private static Object getDataFromRow(Object rowData, LogicalType logicalType, in
|
128 | 126 | return dataClass.getDeclaredMethod("getDouble", int.class).invoke(rowData, index);
|
129 | 127 | case DECIMAL:
|
130 | 128 | DecimalType decimalType = (DecimalType) logicalType;
|
131 |
| - return dataClass.getDeclaredMethod("getDecimal", int.class, int.class, int.class) |
| 129 | + DecimalData decimalData = (DecimalData) dataClass.getDeclaredMethod("getDecimal", int.class, int.class, int.class) |
132 | 130 | .invoke(rowData, index, decimalType.getPrecision(), decimalType.getScale());
|
| 131 | + return decimalData.toBigDecimal().doubleValue(); |
133 | 132 | case BIGINT:
|
134 | 133 | case INTERVAL_DAY_TIME:
|
135 | 134 | return dataClass.getDeclaredMethod("getLong", int.class).invoke(rowData, index);
|
@@ -178,7 +177,8 @@ private static Object getDataFromRow(Object rowData, LogicalType logicalType, in
|
178 | 177 | return list;
|
179 | 178 | case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
180 | 179 | case TIMESTAMP_WITHOUT_TIME_ZONE:
|
181 |
| - return dataClass.getDeclaredMethod("getTimestamp", int.class, int.class).invoke(rowData, index, ((TimestampType) logicalType).getPrecision()); |
| 180 | + TimestampData timestampData = (TimestampData) dataClass.getDeclaredMethod("getTimestamp", int.class, int.class).invoke(rowData, index, ((TimestampType) logicalType).getPrecision()); |
| 181 | + return timestampData.toTimestamp(); |
182 | 182 | case DISTINCT_TYPE:
|
183 | 183 | DistinctType distinctType = (DistinctType) logicalType;
|
184 | 184 | LogicalType sourceType = distinctType.getSourceType();
|
|
0 commit comments