Skip to content

Commit 636e10c

Browse files
author
dapeng
committed
Merge branch 'hotfix_1.8_3.10.x_26972' into feat_1.8_hbaseRefactor
2 parents 6aee245 + 31f1ea1 commit 636e10c

File tree

60 files changed

+1917
-119
lines changed

Some content is hidden

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

60 files changed

+1917
-119
lines changed

ci/sonar_notify.sh

Lines changed: 1 addition & 1 deletion
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

Lines changed: 0 additions & 2 deletions
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(50, 50, 0, TimeUnit.MILLISECONDS,
71-
new LinkedBlockingQueue<>(10000), new DTThreadFactory("clickhouseAsyncExec"), new ThreadPoolExecutor.CallerRunsPolicy()));
7270
}
7371

7472
}

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

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -292,9 +292,7 @@ public static Set<URL> registerTable(SqlTree sqlTree, StreamExecutionEnvironment
292292

293293
RowTypeInfo typeInfo = new RowTypeInfo(adaptTable.getSchema().getFieldTypes(), adaptTable.getSchema().getFieldNames());
294294
DataStream adaptStream = tableEnv.toRetractStream(adaptTable, typeInfo)
295-
.map((Tuple2<Boolean, Row> f0) -> {
296-
return f0.f1;
297-
})
295+
.map((Tuple2<Boolean, Row> f0) -> f0.f1)
298296
.returns(typeInfo);
299297

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

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

Lines changed: 3 additions & 2 deletions
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

Lines changed: 13 additions & 2 deletions
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

Lines changed: 55 additions & 9 deletions
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

Lines changed: 1 addition & 1 deletion
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

Lines changed: 5 additions & 2 deletions
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/SideSqlExec.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -115,7 +115,11 @@ public void exec(String sql,
115115
SideSQLParser sideSQLParser = new SideSQLParser();
116116
sideSQLParser.setLocalTableCache(localTableCache);
117117
Queue<Object> exeQueue = sideSQLParser.getExeQueue(sql, sideTableMap.keySet(), scope);
118-
Object pollObj = null;
118+
Object pollObj;
119+
120+
//need clean
121+
boolean preIsSideJoin = false;
122+
List<FieldReplaceInfo> replaceInfoList = Lists.newArrayList();
119123

120124
while((pollObj = exeQueue.poll()) != null){
121125

core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java

Lines changed: 51 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -106,6 +106,57 @@ public static String replaceIgnoreQuota(String str, String oriStr, String replac
106106
return str.replaceAll(splitPatternStr, replaceStr);
107107
}
108108

109+
/**
110+
* 处理 sql 中 "--" 注释,而不删除引号内的内容
111+
*
112+
* @param sql 解析出来的 sql
113+
* @return 返回无注释内容的 sql
114+
*/
115+
public static String dealSqlComment(String sql) {
116+
boolean inQuotes = false;
117+
boolean inSingleQuotes = false;
118+
int bracketLeftNum = 0;
119+
StringBuilder b = new StringBuilder(sql.length());
120+
char[] chars = sql.toCharArray();
121+
for (int index = 0; index < chars.length; index ++) {
122+
if (index == chars.length) {
123+
return b.toString();
124+
}
125+
StringBuilder tempSb = new StringBuilder(2);
126+
if (index > 1) {
127+
tempSb.append(chars[index - 1]);
128+
tempSb.append(chars[index]);
129+
}
130+
131+
if (tempSb.toString().equals("--")) {
132+
if (inQuotes) {
133+
b.append(chars[index]);
134+
} else if (inSingleQuotes) {
135+
b.append(chars[index]);
136+
} else if (bracketLeftNum > 0) {
137+
b.append(chars[index]);
138+
} else {
139+
b.deleteCharAt(b.length() - 1);
140+
while (chars[index] != '\n') {
141+
// 判断注释内容是不是行尾或者 sql 的最后一行
142+
if (index == chars.length - 1) {
143+
break;
144+
}
145+
index++;
146+
}
147+
}
148+
} else if (chars[index] == '\"' && '\\' != chars[index] && !inSingleQuotes) {
149+
inQuotes = !inQuotes;
150+
b.append(chars[index]);
151+
} else if (chars[index] == '\'' && '\\' != chars[index] && !inQuotes) {
152+
inSingleQuotes = !inSingleQuotes;
153+
b.append(chars[index]);
154+
} else {
155+
b.append(chars[index]);
156+
}
157+
}
158+
return b.toString();
159+
}
109160

110161
public static String col2string(Object column, String type) {
111162
String rowData = column.toString();

0 commit comments

Comments
 (0)