|
16 | 16 | * limitations under the License. |
17 | 17 | */ |
18 | 18 |
|
19 | | - |
| 19 | + |
20 | 20 |
|
21 | 21 | package com.dtstack.flink.sql.side; |
22 | 22 |
|
|
41 | 41 | import org.apache.calcite.sql.parser.SqlParserPos; |
42 | 42 | import org.apache.commons.collections.CollectionUtils; |
43 | 43 | import org.apache.flink.api.common.typeinfo.TypeInformation; |
| 44 | +import org.apache.flink.api.java.tuple.Tuple2; |
44 | 45 | import org.apache.flink.api.java.typeutils.RowTypeInfo; |
45 | 46 | import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; |
46 | 47 | import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; |
47 | 48 | import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; |
48 | 49 | import org.apache.flink.streaming.api.datastream.DataStream; |
49 | 50 | import org.apache.flink.table.api.Table; |
50 | 51 | import org.apache.flink.table.api.java.StreamTableEnvironment; |
| 52 | +import org.apache.flink.types.Row; |
51 | 53 |
|
52 | 54 | import java.util.*; |
53 | 55 |
|
@@ -593,9 +595,9 @@ private void joinFun(Object pollObj, Map<String, Table> localTableCache, |
593 | 595 | } |
594 | 596 |
|
595 | 597 | RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); |
596 | | - //DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class); |
597 | | - DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); |
598 | | - |
| 598 | + DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class) |
| 599 | + .map((Tuple2<Boolean, Row> f0) -> { return f0.f1; }) |
| 600 | + .returns(typeInfo); |
599 | 601 | //join side table before keyby ===> Reducing the size of each dimension table cache of async |
600 | 602 | if(sideTableInfo.isPartitionedJoin()){ |
601 | 603 | List<String> leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias()); |
|
0 commit comments