Skip to content

Commit 2fa36be

Browse files
committed
Merge branch '1.5_v3.6.1_orderby' into '1.5_v3.6.1'
[支持对orderby字段的解析][对orderby解析缺失][18642] See merge request !67
2 parents 2c13933 + 880d472 commit 2fa36be

File tree

3 files changed

+39
-0
lines changed

3 files changed

+39
-0
lines changed

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

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import org.apache.calcite.sql.SqlJoin;
2727
import org.apache.calcite.sql.SqlKind;
2828
import org.apache.calcite.sql.SqlNode;
29+
import org.apache.calcite.sql.SqlOrderBy;
2930
import org.apache.calcite.sql.SqlSelect;
3031
import org.apache.calcite.sql.parser.SqlParseException;
3132
import org.apache.calcite.sql.parser.SqlParser;
@@ -132,6 +133,10 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){
132133
parseNode(unionRight, sqlParseResult);
133134
}
134135
break;
136+
case ORDER_BY:
137+
SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode;
138+
parseNode(sqlOrderBy.query, sqlParseResult);
139+
break;
135140
default:
136141
//do nothing
137142
break;

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

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import org.apache.calcite.sql.SqlKind;
3232
import org.apache.calcite.sql.SqlNode;
3333
import org.apache.calcite.sql.SqlOperator;
34+
import org.apache.calcite.sql.SqlOrderBy;
3435
import org.apache.calcite.sql.SqlSelect;
3536
import org.apache.calcite.sql.parser.SqlParseException;
3637
import org.apache.calcite.sql.parser.SqlParser;
@@ -120,6 +121,10 @@ private Object parseSql(SqlNode sqlNode, Set<String> sideTableSet, Queue<Object>
120121
parseSql(unionRight, sideTableSet, queueInfo);
121122

122123
break;
124+
125+
case ORDER_BY:
126+
SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode;
127+
parseSql(sqlOrderBy.query, sideTableSet, queueInfo);
123128
}
124129

125130
return "";

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

Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@
3535
import org.apache.calcite.sql.SqlLiteral;
3636
import org.apache.calcite.sql.SqlNode;
3737
import org.apache.calcite.sql.SqlNodeList;
38+
import org.apache.calcite.sql.SqlOrderBy;
3839
import org.apache.calcite.sql.SqlSelect;
3940
import org.apache.calcite.sql.fun.SqlCase;
4041
import org.apache.calcite.sql.parser.SqlParseException;
@@ -244,11 +245,39 @@ private void replaceFieldName(SqlNode sqlNode, HashBasedTable<String, String, St
244245
replaceFieldName(unionRight, mappingTable, targetTableName, tableAlias);
245246

246247
break;
248+
249+
case ORDER_BY:
250+
SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode;
251+
replaceFieldName(sqlOrderBy.query, mappingTable, targetTableName, tableAlias);
252+
SqlNodeList orderFiledList = sqlOrderBy.orderList;
253+
for (int i=0 ;i<orderFiledList.size();i++) {
254+
SqlNode replaceNode = replaceOrderByTableName(orderFiledList.get(i), tableAlias);
255+
orderFiledList.set(i, replaceNode);
256+
}
247257
default:
248258
break;
249259
}
250260
}
251261

262+
private SqlNode replaceOrderByTableName(SqlNode orderNode, String tableAlias) {
263+
if(orderNode.getKind() == IDENTIFIER){
264+
SqlIdentifier sqlIdentifier = (SqlIdentifier) orderNode;
265+
if (sqlIdentifier.names.size() == 1) {
266+
return orderNode;
267+
}
268+
return sqlIdentifier.setName(0, tableAlias);
269+
} else if (orderNode instanceof SqlBasicCall) {
270+
SqlBasicCall sqlBasicCall = (SqlBasicCall) orderNode;
271+
for(int i=0; i<sqlBasicCall.getOperandList().size(); i++){
272+
SqlNode sqlNode = sqlBasicCall.getOperandList().get(i);
273+
sqlBasicCall.getOperands()[i] = replaceOrderByTableName(sqlNode , tableAlias);
274+
}
275+
return sqlBasicCall;
276+
} else {
277+
return orderNode;
278+
}
279+
}
280+
252281
private SqlNode replaceNodeInfo(SqlNode groupNode, HashBasedTable<String, String, String> mappingTable, String tableAlias){
253282
if(groupNode.getKind() == IDENTIFIER){
254283
SqlIdentifier sqlIdentifier = (SqlIdentifier) groupNode;

0 commit comments

Comments
 (0)