53
53
import org .apache .flink .table .api .Table ;
54
54
import org .apache .flink .table .api .java .StreamTableEnvironment ;
55
55
import org .apache .flink .table .dataformat .BaseRow ;
56
+ import org .apache .flink .table .planner .plan .QueryOperationConverter ;
56
57
import org .apache .flink .table .runtime .typeutils .BaseRowTypeInfo ;
57
58
import org .apache .flink .table .runtime .typeutils .BigDecimalTypeInfo ;
58
59
import org .apache .flink .table .runtime .typeutils .LegacyLocalDateTimeTypeInfo ;
68
69
import java .time .LocalDateTime ;
69
70
import java .util .ArrayList ;
70
71
import java .util .Arrays ;
72
+ import java .util .Collection ;
71
73
import java .util .LinkedList ;
72
74
import java .util .List ;
73
75
import java .util .Map ;
@@ -101,6 +103,9 @@ public class SideSqlExec {
101
103
102
104
private Map <String , Table > localTableCache = Maps .newHashMap ();
103
105
106
+ //维表重新注册之后的名字缓存
107
+ private static Map <String , Table > dimTableNewTable = Maps .newHashMap ();
108
+
104
109
public void exec (String sql ,
105
110
Map <String , AbstractSideTableInfo > sideTableMap ,
106
111
StreamTableEnvironment tableEnv ,
@@ -128,6 +133,8 @@ public void exec(String sql,
128
133
sideSQLParser .setLocalTableCache (localTableCache );
129
134
Queue <Object > exeQueue = sideSQLParser .getExeQueue (sql , sideTableMap .keySet (), scope );
130
135
Object pollObj = null ;
136
+ // create view中是否包含维表
137
+ boolean includeDimTable = false ;
131
138
132
139
while ((pollObj = exeQueue .poll ()) != null ) {
133
140
@@ -136,41 +143,49 @@ public void exec(String sql,
136
143
137
144
138
145
if (pollSqlNode .getKind () == INSERT ) {
139
- FlinkSQLExec .sqlUpdate (tableEnv , pollSqlNode .toString ());
146
+ Collection <String > newRegisterTableList = dimTableNewTable .keySet ();
147
+ FlinkSQLExec .sqlInsert (tableEnv , pollSqlNode , newRegisterTableList );
140
148
if (LOG .isInfoEnabled ()) {
141
149
LOG .info ("----------real exec sql-----------\n {}" , pollSqlNode .toString ());
142
150
}
143
151
144
152
} else if (pollSqlNode .getKind () == AS ) {
145
- dealAsSourceTable (tableEnv , pollSqlNode , tableCache );
153
+ Collection <String > newRegisterTableList = dimTableNewTable .keySet ();
154
+ dealAsSourceTable (tableEnv , pollSqlNode , tableCache , newRegisterTableList );
146
155
147
156
} else if (pollSqlNode .getKind () == WITH_ITEM ) {
148
157
SqlWithItem sqlWithItem = (SqlWithItem ) pollSqlNode ;
149
- String TableAlias = sqlWithItem .name .toString ();
150
- Table table = tableEnv .sqlQuery (sqlWithItem .query .toString ());
151
- tableEnv .registerTable (TableAlias , table );
158
+ String tableAlias = sqlWithItem .name .toString ();
159
+ Collection <String > newRegisterTableList = dimTableNewTable .keySet ();
160
+ Table table = FlinkSQLExec .sqlQuery (tableEnv , sqlWithItem .query , newRegisterTableList );
161
+ tableEnv .createTemporaryView (tableAlias , table );
152
162
153
163
} else if (pollSqlNode .getKind () == SELECT ) {
154
164
Preconditions .checkState (createView != null , "select sql must included by create view" );
155
- Table table = tableEnv .sqlQuery (pollObj .toString ());
165
+ Collection <String > newRegisterTableList = dimTableNewTable .keySet ();
166
+ Table table = FlinkSQLExec .sqlQuery (tableEnv , pollSqlNode , newRegisterTableList );
156
167
157
168
if (createView .getFieldsInfoStr () == null ) {
158
169
tableEnv .registerTable (createView .getTableName (), table );
159
170
} else {
160
171
if (checkFieldsInfo (createView , table )) {
161
172
table = table .as (tmpFields );
162
- tableEnv .registerTable (createView .getTableName (), table );
173
+ tableEnv .createTemporaryView (createView .getTableName (), table );
163
174
} else {
164
175
throw new RuntimeException ("Fields mismatch" );
165
176
}
166
177
}
167
178
168
179
localTableCache .put (createView .getTableName (), table );
180
+ if (includeDimTable ){
181
+ dimTableNewTable .put (createView .getTableName (), table );
182
+ }
169
183
}
170
-
184
+ includeDimTable = false ;
171
185
} else if (pollObj instanceof JoinInfo ) {
186
+ includeDimTable = true ;
172
187
LOG .info ("----------exec join info----------\n {}" , pollObj .toString ());
173
- joinFun (pollObj , localTableCache , sideTableMap , tableEnv );
188
+ joinFun (pollObj , localTableCache , dimTableNewTable , sideTableMap , tableEnv );
174
189
}
175
190
}
176
191
@@ -414,14 +429,21 @@ public List<String> getConditionFields(SqlNode conditionNode, String specifyTabl
414
429
415
430
protected void dealAsSourceTable (StreamTableEnvironment tableEnv ,
416
431
SqlNode pollSqlNode ,
417
- Map <String , Table > tableCache ) throws SqlParseException {
432
+ Map <String , Table > tableCache ,
433
+ Collection <String > newRegisterTableList ) throws SqlParseException {
418
434
419
435
AliasInfo aliasInfo = parseASNode (pollSqlNode );
420
436
if (localTableCache .containsKey (aliasInfo .getName ())) {
421
437
return ;
422
438
}
423
439
440
+ boolean isGroupByTimeWindow = TableUtils .checkIsDimTableGroupBy (pollSqlNode , newRegisterTableList );
441
+ if (isGroupByTimeWindow ){
442
+ QueryOperationConverter .setProducesUpdates (true );
443
+ }
424
444
Table table = tableEnv .sqlQuery (aliasInfo .getName ());
445
+ QueryOperationConverter .setProducesUpdates (false );
446
+
425
447
tableEnv .registerTable (aliasInfo .getAlias (), table );
426
448
localTableCache .put (aliasInfo .getAlias (), table );
427
449
@@ -441,6 +463,7 @@ protected void dealAsSourceTable(StreamTableEnvironment tableEnv,
441
463
442
464
private void joinFun (Object pollObj ,
443
465
Map <String , Table > localTableCache ,
466
+ Map <String , Table > dimTableNewTable ,
444
467
Map <String , AbstractSideTableInfo > sideTableMap ,
445
468
StreamTableEnvironment tableEnv ) throws Exception {
446
469
JoinInfo joinInfo = (JoinInfo ) pollObj ;
@@ -525,7 +548,6 @@ private void joinFun(Object pollObj,
525
548
RowTypeInfo typeInfo = new RowTypeInfo (fieldDataTypes , targetTable .getSchema ().getFieldNames ());
526
549
527
550
DataStream <BaseRow > adaptStream = tableEnv .toRetractStream (targetTable , typeInfo )
528
- .filter (f -> f .f0 )
529
551
.map (f -> RowDataConvert .convertToBaseRow (f ));
530
552
531
553
//join side table before keyby ===> Reducing the size of each dimension table cache of async
@@ -565,6 +587,7 @@ private void joinFun(Object pollObj,
565
587
Table joinTable = tableEnv .fromDataStream (dsOut );
566
588
tableEnv .createTemporaryView (targetTableName , joinTable );
567
589
localTableCache .put (joinInfo .getNewTableName (), joinTable );
590
+ dimTableNewTable .put (joinInfo .getNewTableName (), joinTable );
568
591
}
569
592
}
570
593
@@ -593,4 +616,8 @@ private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Tab
593
616
return true ;
594
617
}
595
618
619
+ public static Map <String , Table > getDimTableNewTable (){
620
+ return dimTableNewTable ;
621
+ }
622
+
596
623
}
0 commit comments