|
26 | 26 | import io.vertx.core.json.JsonArray; |
27 | 27 | import io.vertx.ext.sql.SQLClient; |
28 | 28 | import io.vertx.ext.sql.SQLConnection; |
29 | | -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; |
30 | 29 | import org.apache.flink.api.java.typeutils.RowTypeInfo; |
31 | 30 | import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; |
32 | | -import org.apache.flink.configuration.Configuration; |
33 | 31 | import org.apache.flink.streaming.api.functions.async.ResultFuture; |
34 | 32 | import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; |
35 | 33 | import org.apache.flink.types.Row; |
36 | 34 | import org.slf4j.Logger; |
37 | 35 | import org.slf4j.LoggerFactory; |
38 | 36 |
|
39 | | -import java.math.BigInteger; |
40 | 37 | import java.sql.Timestamp; |
41 | | -import java.util.Collections; |
42 | 38 | import java.util.List; |
43 | 39 | import java.util.Map; |
44 | 40 |
|
@@ -122,18 +118,21 @@ public void asyncInvoke(Row input, ResultFuture<Row> resultFuture) throws Except |
122 | 118 |
|
123 | 119 | int resultSize = rs.result().getResults().size(); |
124 | 120 | if (resultSize > 0) { |
125 | | - for (JsonArray line : rs.result().getResults()) { |
| 121 | + List<Row> rowList = Lists.newArrayList(); |
126 | 122 |
|
| 123 | + for (JsonArray line : rs.result().getResults()) { |
127 | 124 | Row row = fillData(input, line); |
128 | 125 | if (openCache()) { |
129 | 126 | cacheContent.add(line); |
130 | 127 | } |
131 | | - resultFuture.complete(Collections.singleton(row)); |
| 128 | + rowList.add(row); |
132 | 129 | } |
133 | 130 |
|
134 | 131 | if (openCache()) { |
135 | 132 | putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); |
136 | 133 | } |
| 134 | + |
| 135 | + resultFuture.complete(rowList); |
137 | 136 | } else { |
138 | 137 | dealMissKey(input, resultFuture); |
139 | 138 | if (openCache()) { |
|
0 commit comments