Skip to content

Commit d6e7274

Browse files
author
dapeng
committed
Merge branch '1.8_release_3.10.x' into 1.8_release-github
# Conflicts: # launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java # launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java
2 parents a9f9299 + 31f1ea1 commit d6e7274

File tree

76 files changed

+2174
-185
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

76 files changed

+2174
-185
lines changed

ci/sonar_notify.sh

+1-1
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,7 @@
11
#!/bin/bash
22
#参考钉钉文档 https://open-doc.dingtalk.com/microapp/serverapi2/qf2nxq
33
sonarreport=$(curl -s http://172.16.100.198:8082/?projectname=dt-insight-engine/flinkStreamSQL)
4-
curl -s "https://oapi.dingtalk.com/robot/send?access_token=71555061297a53d3ac922a6f4d94285d8e23bccdca0c00b4dc6df0a2d49da724" \
4+
curl -s "https://oapi.dingtalk.com/robot/send?access_token=58fd731d8bed3b17708d3aa27e49a7e2c41c7e6545f6c4be3170963a7bba7e2a" \
55
-H "Content-Type: application/json" \
66
-d "{
77
\"msgtype\": \"markdown\",

clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java

-2
Original file line numberDiff line numberDiff line change
@@ -67,8 +67,6 @@ public void open(Configuration parameters) throws Exception {
6767
vo.setFileResolverCachingEnabled(false);
6868
Vertx vertx = Vertx.vertx(vo);
6969
setRdbSqlClient(JDBCClient.createNonShared(vertx, clickhouseClientConfig));
70-
setExecutor(new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS,
71-
new LinkedBlockingQueue<>(10), new DTThreadFactory("clickhouseAsyncExec")));
7270
}
7371

7472
}

core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java

+27-11
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import org.apache.flink.api.common.JobExecutionResult;
2323
import org.apache.flink.api.java.ExecutionEnvironment;
2424
import org.apache.flink.configuration.Configuration;
25+
import org.apache.flink.configuration.RestOptions;
2526
import org.apache.flink.configuration.TaskManagerOptions;
2627
import org.apache.flink.runtime.jobgraph.JobGraph;
2728
import org.apache.flink.runtime.minicluster.MiniCluster;
@@ -93,33 +94,48 @@ public JobExecutionResult execute(String jobName) throws Exception {
9394
// transform the streaming program into a JobGraph
9495
StreamGraph streamGraph = getStreamGraph();
9596
streamGraph.setJobName(jobName);
97+
return execute(streamGraph);
98+
}
99+
100+
public JobExecutionResult execute(StreamGraph streamGraph) throws Exception {
96101

97102
JobGraph jobGraph = streamGraph.getJobGraph();
98103
jobGraph.setClasspaths(classpaths);
104+
jobGraph.setAllowQueuedScheduling(true);
99105

100106
Configuration configuration = new Configuration();
101107
configuration.addAll(jobGraph.getJobConfiguration());
102-
103-
configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "512M");
104-
configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, jobGraph.getMaximumParallelism());
108+
configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "0");
105109

106110
// add (and override) the settings with what the user defined
107111
configuration.addAll(this.conf);
108112

109-
MiniClusterConfiguration.Builder configBuilder = new MiniClusterConfiguration.Builder();
110-
configBuilder.setConfiguration(configuration);
113+
if (!configuration.contains(RestOptions.BIND_PORT)) {
114+
configuration.setString(RestOptions.BIND_PORT, "0");
115+
}
116+
117+
int numSlotsPerTaskManager = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS, jobGraph.getMaximumParallelism());
118+
119+
MiniClusterConfiguration cfg = new MiniClusterConfiguration.Builder()
120+
.setConfiguration(configuration)
121+
.setNumSlotsPerTaskManager(numSlotsPerTaskManager)
122+
.build();
111123

112124
if (LOG.isInfoEnabled()) {
113125
LOG.info("Running job on local embedded Flink mini cluster");
114126
}
115127

116-
try (MiniCluster exec = new MiniCluster(configBuilder.build());) {
117-
exec.start();
118-
JobExecutionResult jobExecutionResult = exec.executeJobBlocking(jobGraph);
128+
MiniCluster miniCluster = new MiniCluster(cfg);
129+
130+
try {
131+
miniCluster.start();
132+
configuration.setInteger(RestOptions.PORT, miniCluster.getRestAddress().get().getPort());
133+
134+
return miniCluster.executeJobBlocking(jobGraph);
135+
}
136+
finally {
119137
transformations.clear();
120-
return jobExecutionResult;
121-
} catch (Exception e) {
122-
throw new RuntimeException(e);
138+
miniCluster.close();
123139
}
124140
}
125141
}

core/src/main/java/com/dtstack/flink/sql/exec/ExecuteProcessHelper.java

+10-6
Original file line numberDiff line numberDiff line change
@@ -200,14 +200,18 @@ private static void sqlTranslation(String localSqlPluginPath,
200200

201201
SideSqlExec sideSqlExec = new SideSqlExec();
202202
sideSqlExec.setLocalSqlPluginPath(localSqlPluginPath);
203+
204+
int scope = 0;
203205
for (CreateTmpTableParser.SqlParserResult result : sqlTree.getTmpSqlList()) {
204-
sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, result);
206+
sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, result, scope + "");
207+
scope++;
205208
}
206209

207210
for (InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()) {
208211
if (LOG.isInfoEnabled()) {
209212
LOG.info("exe-sql:\n" + result.getExecSql());
210213
}
214+
211215
boolean isSide = false;
212216
for (String tableName : result.getTargetTableList()) {
213217
if (sqlTree.getTmpTableMap().containsKey(tableName)) {
@@ -218,7 +222,7 @@ private static void sqlTranslation(String localSqlPluginPath,
218222
SqlNode sqlNode = flinkPlanner.parse(realSql);
219223
String tmpSql = ((SqlInsert) sqlNode).getSource().toString();
220224
tmp.setExecSql(tmpSql);
221-
sideSqlExec.exec(tmp.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, tmp);
225+
sideSqlExec.exec(tmp.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, tmp, scope + "");
222226
} else {
223227
for (String sourceTable : result.getSourceTableList()) {
224228
if (sideTableMap.containsKey(sourceTable)) {
@@ -228,7 +232,7 @@ private static void sqlTranslation(String localSqlPluginPath,
228232
}
229233
if (isSide) {
230234
//sql-dimensional table contains the dimension table of execution
231-
sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, null);
235+
sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, null, null);
232236
} else {
233237
LOG.info("----------exec sql without dimension join-----------");
234238
LOG.info("----------real sql exec is--------------------------\n{}", result.getExecSql());
@@ -238,6 +242,8 @@ private static void sqlTranslation(String localSqlPluginPath,
238242
}
239243
}
240244
}
245+
246+
scope++;
241247
}
242248
}
243249
}
@@ -289,9 +295,7 @@ public static Set<URL> registerTable(SqlTree sqlTree, StreamExecutionEnvironment
289295

290296
RowTypeInfo typeInfo = new RowTypeInfo(adaptTable.getSchema().getFieldTypes(), adaptTable.getSchema().getFieldNames());
291297
DataStream adaptStream = tableEnv.toRetractStream(adaptTable, typeInfo)
292-
.map((Tuple2<Boolean, Row> f0) -> {
293-
return f0.f1;
294-
})
298+
.map((Tuple2<Boolean, Row> f0) -> f0.f1)
295299
.returns(typeInfo);
296300

297301
String fields = String.join(",", typeInfo.getFieldNames());

core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import org.apache.commons.lang.StringUtils;
2727
import java.lang.reflect.InvocationTargetException;
2828
import java.lang.reflect.Field;
29+
import java.nio.charset.StandardCharsets;
2930
import java.util.List;
3031
import java.util.Map;
3132
import java.io.File;
@@ -102,8 +103,8 @@ public List<String> getProgramExeArgList() throws Exception {
102103
continue;
103104
} else if (OPTION_SQL.equalsIgnoreCase(key)) {
104105
File file = new File(value.toString());
105-
String content = FileUtils.readFile(file, "UTF-8");
106-
value = URLEncoder.encode(content, Charsets.UTF_8.name());
106+
String content = FileUtils.readFile(file, StandardCharsets.UTF_8.name());
107+
value = URLEncoder.encode(content, StandardCharsets.UTF_8.name());
107108
}
108109
args.add("-" + key);
109110
args.add(value.toString());

core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java

+13-2
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,8 @@ public class CreateTableParser implements IParser {
4141

4242
private static final Pattern PATTERN = Pattern.compile(PATTERN_STR);
4343

44+
private static final Pattern PROP_PATTERN = Pattern.compile("^'\\s*(.+)\\s*'$");
45+
4446
public static CreateTableParser newInstance(){
4547
return new CreateTableParser();
4648
}
@@ -69,18 +71,27 @@ public void parseSql(String sql, SqlTree sqlTree) {
6971
}
7072

7173
private Map parseProp(String propsStr){
72-
String[] strs = propsStr.trim().split("'\\s*,");
74+
propsStr = propsStr.replaceAll("'\\s*,", "'|");
75+
String[] strs = propsStr.trim().split("\\|");
7376
Map<String, Object> propMap = Maps.newHashMap();
7477
for(int i=0; i<strs.length; i++){
7578
List<String> ss = DtStringUtil.splitIgnoreQuota(strs[i], '=');
7679
String key = ss.get(0).trim();
77-
String value = ss.get(1).trim().replaceAll("'", "").trim();
80+
String value = extractValue(ss.get(1).trim());
7881
propMap.put(key, value);
7982
}
8083

8184
return propMap;
8285
}
8386

87+
private String extractValue(String value) {
88+
Matcher matcher = PROP_PATTERN.matcher(value);
89+
if (matcher.find()) {
90+
return matcher.group(1);
91+
}
92+
throw new RuntimeException("[" + value + "] format is invalid");
93+
}
94+
8495
public static class SqlParserResult{
8596

8697
private String tableName;

core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java

+55-9
Original file line numberDiff line numberDiff line change
@@ -20,17 +20,18 @@
2020

2121
package com.dtstack.flink.sql.parser;
2222

23-
import org.apache.calcite.config.Lex;
24-
import org.apache.calcite.sql.SqlBasicCall;
2523
import org.apache.calcite.sql.SqlInsert;
2624
import org.apache.calcite.sql.SqlJoin;
2725
import org.apache.calcite.sql.SqlKind;
28-
import org.apache.calcite.sql.SqlMatchRecognize;
2926
import org.apache.calcite.sql.SqlNode;
30-
import org.apache.calcite.sql.SqlOrderBy;
3127
import org.apache.calcite.sql.SqlSelect;
32-
import org.apache.calcite.sql.parser.SqlParseException;
33-
import org.apache.calcite.sql.parser.SqlParser;
28+
import org.apache.calcite.sql.SqlNodeList;
29+
import org.apache.calcite.sql.SqlBasicCall;
30+
import org.apache.calcite.sql.SqlMatchRecognize;
31+
import org.apache.calcite.sql.SqlOrderBy;
32+
import org.apache.calcite.sql.SqlIdentifier;
33+
import org.apache.calcite.sql.SqlAsOperator;
34+
import org.apache.calcite.sql.parser.SqlParserPos;
3435
import org.apache.commons.lang3.StringUtils;
3536
import com.google.common.collect.Lists;
3637
import org.apache.flink.table.calcite.FlinkPlannerImpl;
@@ -49,6 +50,9 @@
4950

5051
public class InsertSqlParser implements IParser {
5152

53+
// 用来标识当前解析节点的上一层节点是否为 insert 节点
54+
private static Boolean parentIsInsert = false;
55+
5256
@Override
5357
public boolean verify(String sql) {
5458
return StringUtils.isNotBlank(sql) && sql.trim().toLowerCase().startsWith("insert");
@@ -78,13 +82,19 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){
7882
SqlNode sqlTarget = ((SqlInsert)sqlNode).getTargetTable();
7983
SqlNode sqlSource = ((SqlInsert)sqlNode).getSource();
8084
sqlParseResult.addTargetTable(sqlTarget.toString());
85+
parentIsInsert = true;
8186
parseNode(sqlSource, sqlParseResult);
8287
break;
8388
case SELECT:
84-
SqlNode sqlFrom = ((SqlSelect)sqlNode).getFrom();
85-
if(sqlFrom.getKind() == IDENTIFIER){
89+
SqlSelect sqlSelect = (SqlSelect) sqlNode;
90+
if (parentIsInsert) {
91+
rebuildSelectNode(sqlSelect.getSelectList(), sqlSelect);
92+
}
93+
SqlNode sqlFrom = ((SqlSelect) sqlNode).getFrom();
94+
if (sqlFrom.getKind() == IDENTIFIER) {
8695
sqlParseResult.addSourceTable(sqlFrom.toString());
87-
}else{
96+
} else {
97+
parentIsInsert = false;
8898
parseNode(sqlFrom, sqlParseResult);
8999
}
90100
break;
@@ -141,6 +151,42 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){
141151
}
142152
}
143153

154+
/**
155+
* 将第一层 select 中的 sqlNode 转化为 AsNode,解决字段名冲突问题
156+
* @param selectList select Node 的 select 字段
157+
* @param sqlSelect 第一层解析出来的 selectNode
158+
*/
159+
private static void rebuildSelectNode(SqlNodeList selectList, SqlSelect sqlSelect) {
160+
SqlNodeList sqlNodes = new SqlNodeList(selectList.getParserPosition());
161+
162+
for (int index = 0; index < selectList.size(); index++) {
163+
if (selectList.get(index).getKind().equals(SqlKind.AS)) {
164+
sqlNodes.add(selectList.get(index));
165+
continue;
166+
}
167+
sqlNodes.add(transformToAsNode(selectList.get(index)));
168+
}
169+
sqlSelect.setSelectList(sqlNodes);
170+
}
171+
172+
/**
173+
* 将 sqlNode 转化为 AsNode
174+
* @param sqlNode 需要转化的 sqlNode
175+
* @return 重新构造的 AsNode
176+
*/
177+
public static SqlBasicCall transformToAsNode(SqlNode sqlNode) {
178+
String asName = "";
179+
SqlParserPos pos = new SqlParserPos(sqlNode.getParserPosition().getLineNum(),
180+
sqlNode.getParserPosition().getEndColumnNum());
181+
if (sqlNode.getKind().equals(SqlKind.IDENTIFIER)) {
182+
asName = ((SqlIdentifier) sqlNode).names.get(1);
183+
}
184+
SqlNode[] operands = new SqlNode[2];
185+
operands[0] = sqlNode;
186+
operands[1] = new SqlIdentifier(asName, null, pos);
187+
return new SqlBasicCall(new SqlAsOperator(), operands, pos);
188+
}
189+
144190
public static class SqlParseResult {
145191

146192
private List<String> sourceTableList = Lists.newArrayList();

core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -68,7 +68,7 @@ public static SqlTree parseSql(String sql) throws Exception {
6868
throw new RuntimeException("need to set local sql plugin root");
6969
}
7070

71-
sql = sql.replaceAll("--.*", "")
71+
sql = DtStringUtil.dealSqlComment(sql)
7272
.replaceAll("\r\n", " ")
7373
.replaceAll("\n", " ")
7474
.replace("\t", " ").trim();

core/src/main/java/com/dtstack/flink/sql/side/BaseAsyncReqRow.java

+5-2
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222

2323
import com.dtstack.flink.sql.enums.ECacheContentType;
2424
import com.dtstack.flink.sql.enums.ECacheType;
25+
import com.dtstack.flink.sql.factory.DTThreadFactory;
2526
import com.dtstack.flink.sql.metric.MetricConstant;
2627
import com.dtstack.flink.sql.side.cache.AbstractSideCache;
2728
import com.dtstack.flink.sql.side.cache.CacheObj;
@@ -68,6 +69,7 @@ public abstract class BaseAsyncReqRow extends RichAsyncFunction<CRow, CRow> impl
6869
private int timeOutNum = 0;
6970
protected BaseSideInfo sideInfo;
7071
protected transient Counter parseErrorRecords;
72+
private transient ThreadPoolExecutor cancelExecutor;
7173

7274
public BaseAsyncReqRow(BaseSideInfo sideInfo){
7375
this.sideInfo = sideInfo;
@@ -82,6 +84,8 @@ public void open(Configuration parameters) throws Exception {
8284
super.open(parameters);
8385
initCache();
8486
initMetric();
87+
cancelExecutor = new ThreadPoolExecutor(1, 1, 0, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(100000),
88+
new DTThreadFactory("cancel-timer-executor"));
8589
LOG.info("async dim table config info: {} ", sideInfo.getSideTableInfo().toString());
8690
}
8791

@@ -248,12 +252,11 @@ public void onProcessingTime(long timestamp) throws Exception {
248252
}
249253

250254
protected void cancelTimerWhenComplete(ResultFuture<CRow> resultFuture, ScheduledFuture<?> timerFuture){
251-
ThreadPoolExecutor executors = new ThreadPoolExecutor(1, 1,0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>());
252255
if(resultFuture instanceof StreamRecordQueueEntry){
253256
StreamRecordQueueEntry streamRecordBufferEntry = (StreamRecordQueueEntry) resultFuture;
254257
streamRecordBufferEntry.onComplete((Object value) -> {
255258
timerFuture.cancel(true);
256-
},executors);
259+
}, cancelExecutor);
257260
}
258261
}
259262

core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java

+15-2
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020

2121
package com.dtstack.flink.sql.side;
2222

23+
import com.dtstack.flink.sql.util.TableUtils;
2324
import com.google.common.collect.HashBasedTable;
2425
import com.google.common.collect.Maps;
2526
import org.apache.calcite.sql.JoinType;
@@ -66,6 +67,8 @@ public class JoinInfo implements Serializable {
6667

6768
private JoinType joinType;
6869

70+
private String scope = "";
71+
6972
/**
7073
* 左表需要查询的字段信息和output的时候对应的列名称
7174
*/
@@ -96,12 +99,14 @@ public String getNewTableName(){
9699
//兼容左边表是as 的情况
97100
String leftStr = leftTableName;
98101
leftStr = Strings.isNullOrEmpty(leftStr) ? leftTableAlias : leftStr;
99-
return leftStr + "_" + rightTableName;
102+
String newName = leftStr + "_" + rightTableName;
103+
return TableUtils.buildTableNameWithScope(newName, scope);
100104
}
101105

102106

103107
public String getNewTableAlias(){
104-
return leftTableAlias + "_" + rightTableAlias;
108+
String newName = leftTableAlias + "_" + rightTableAlias;
109+
return TableUtils.buildTableNameWithScope(newName, scope);
105110
}
106111

107112
public boolean isLeftIsSideTable() {
@@ -233,6 +238,14 @@ public HashBasedTable<String, String, String> getTableFieldRef(){
233238
return mappingTable;
234239
}
235240

241+
public String getScope() {
242+
return scope;
243+
}
244+
245+
public void setScope(String scope) {
246+
this.scope = scope;
247+
}
248+
236249
@Override
237250
public String toString() {
238251
return "JoinInfo{" +

0 commit comments

Comments
 (0)