From 0859b556971d9677d20df9e8e2c1d380832b2b49 Mon Sep 17 00:00:00 2001 From: maqi Date: Mon, 13 Jan 2020 15:53:21 +0800 Subject: [PATCH 01/13] multi join --- .../com/dtstack/flink/sql/side/JoinInfo.java | 17 - .../flink/sql/side/ParserJoinField.java | 2 +- .../dtstack/flink/sql/side/SideSQLParser.java | 326 ++++++++++-------- .../dtstack/flink/sql/side/SideSqlExec.java | 108 ++++-- .../dtstack/flink/sql/util/ParseUtils.java | 214 ++++-------- 5 files changed, 323 insertions(+), 344 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java index 6fde02493..851653a48 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -41,8 +41,6 @@ public class JoinInfo implements Serializable { //左表是否是维表 private boolean leftIsSideTable; - //左表是 转换后的中间表 - private boolean leftIsMidTable; //右表是否是维表 private boolean rightIsSideTable; @@ -92,21 +90,6 @@ public String getNewTableName(){ return leftStr + "_" + rightTableName; } - public boolean isLeftIsMidTable() { - return leftIsMidTable; - } - - public void setLeftIsMidTable(boolean leftIsMidTable) { - this.leftIsMidTable = leftIsMidTable; - } - - public Map getLeftTabMapping() { - return leftTabMapping; - } - - public void setLeftTabMapping(Map leftTabMapping) { - this.leftTabMapping = leftTabMapping; - } public String getNewTableAlias(){ return leftTableAlias + "_" + rightTableAlias; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java index 74d303c24..37a4aac19 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java @@ -107,7 +107,7 @@ public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, b return fieldInfoList; } - + //TODO 丢弃多余的PROCTIME private static List getAllField(JoinScope scope){ Iterator prefixId = scope.getChildren().iterator(); List fieldInfoList = Lists.newArrayList(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 6a547964b..994a81756 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.config.CalciteConfig; import com.dtstack.flink.sql.util.ParseUtils; +import com.google.common.collect.HashBasedTable; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; @@ -39,15 +40,18 @@ import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Queues; +import org.apache.flink.table.api.Table; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Queue; @@ -65,10 +69,16 @@ public class SideSQLParser { private static final Logger LOG = LoggerFactory.getLogger(SideSQLParser.class); + private Map localTableCache = Maps.newHashMap(); private final char SPLIT = '_'; - private String tempSQL = "SELECT * FROM TMP"; + /** 处理连续join时,中间表存储子表字段映射 */ + private Map> midTableFileNameMapping = Maps.newHashMap(); + /** 处理连续join时,原始表与中间表的映射 */ + private Map>> midTableNameMapping = Maps.newHashMap(); + + public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { System.out.println("---exeSql---"); System.out.println(exeSql); @@ -78,11 +88,62 @@ public Queue getExeQueue(String exeSql, Set sideTableSet) throws Queue queueInfo = Queues.newLinkedBlockingQueue(); SqlParser sqlParser = SqlParser.create(exeSql, CalciteConfig.MYSQL_LEX_CONFIG); SqlNode sqlNode = sqlParser.parseStmt(); + SqlNode original = sqlNode; + + try { + checkAndReplaceMultiJoin(sqlNode, sideTableSet); + } catch (Exception e) { + sqlNode = original; + LOG.error("checkAndReplaceMultiJoin method error ", e); + } + parseSql(sqlNode, sideTableSet, queueInfo); queueInfo.offer(sqlNode); return queueInfo; } + private void checkAndReplaceMultiJoin(SqlNode sqlNode, Set sideTableSet) { + SqlKind sqlKind = sqlNode.getKind(); + switch (sqlKind) { + case WITH: { + SqlWith sqlWith = (SqlWith) sqlNode; + SqlNodeList sqlNodeList = sqlWith.withList; + for (SqlNode withAsTable : sqlNodeList) { + SqlWithItem sqlWithItem = (SqlWithItem) withAsTable; + checkAndReplaceMultiJoin(sqlWithItem.query, sideTableSet); + } + checkAndReplaceMultiJoin(sqlWith.body, sideTableSet); + break; + } + case INSERT: + SqlNode sqlSource = ((SqlInsert) sqlNode).getSource(); + checkAndReplaceMultiJoin(sqlSource, sideTableSet); + break; + case SELECT: + SqlNode sqlFrom = ((SqlSelect) sqlNode).getFrom(); + if (sqlFrom.getKind() != IDENTIFIER) { + checkAndReplaceMultiJoin(sqlFrom, sideTableSet); + } + break; + case JOIN: + convertMultiJoinToNestQuery((SqlJoin) sqlNode, sideTableSet); + break; + case AS: + SqlNode info = ((SqlBasicCall) sqlNode).getOperands()[0]; + if (info.getKind() != IDENTIFIER) { + checkAndReplaceMultiJoin(info, sideTableSet); + } + break; + case UNION: + SqlNode unionLeft = ((SqlBasicCall) sqlNode).getOperands()[0]; + SqlNode unionRight = ((SqlBasicCall) sqlNode).getOperands()[1]; + checkAndReplaceMultiJoin(unionLeft, sideTableSet); + checkAndReplaceMultiJoin(unionRight, sideTableSet); + break; + } + } + + private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue queueInfo){ SqlKind sqlKind = sqlNode.getKind(); switch (sqlKind){ @@ -105,7 +166,7 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue if(sqlFrom.getKind() != IDENTIFIER){ Object result = parseSql(sqlFrom, sideTableSet, queueInfo); if(result instanceof JoinInfo){ - dealSelectResultWithJoinInfo((JoinInfo)result, (SqlSelect) sqlNode, queueInfo); + return dealSelectResultWithJoinInfo((JoinInfo) result, (SqlSelect) sqlNode, queueInfo); }else if(result instanceof AliasInfo){ String tableName = ((AliasInfo) result).getName(); if(sideTableSet.contains(tableName)){ @@ -124,11 +185,11 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue case AS: SqlNode info = ((SqlBasicCall)sqlNode).getOperands()[0]; SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; - String infoStr; + String infoStr = ""; if(info.getKind() == IDENTIFIER){ infoStr = info.toString(); - }else{ + } else { infoStr = parseSql(info, sideTableSet, queueInfo).toString(); } @@ -143,19 +204,86 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue SqlNode unionRight = ((SqlBasicCall)sqlNode).getOperands()[1]; parseSql(unionLeft, sideTableSet, queueInfo); - parseSql(unionRight, sideTableSet, queueInfo); - break; - case ORDER_BY: SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode; parseSql(sqlOrderBy.query, sideTableSet, queueInfo); } - return ""; } + private boolean isMultiJoinSqlNode(SqlNode sqlNode) { + return sqlNode.getKind() == JOIN && ((SqlJoin) sqlNode).getLeft().getKind() == JOIN; + } + + private AliasInfo getSqlNodeAliasInfo(SqlNode sqlNode) { + SqlNode info = ((SqlBasicCall) sqlNode).getOperands()[0]; + SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; + String infoStr = info.getKind() == IDENTIFIER ? info.toString() : null; + + AliasInfo aliasInfo = new AliasInfo(); + aliasInfo.setName(infoStr); + aliasInfo.setAlias(alias.toString()); + return aliasInfo; + } + + private void convertMultiJoinToNestQuery(SqlNode sqlNode, Set sideTableSet) { + SqlKind sqlKind = sqlNode.getKind(); + switch (sqlKind) { + case JOIN: + checkAndReplaceMultiJoin(((SqlJoin) sqlNode).getRight(), sideTableSet); + checkAndReplaceMultiJoin(((SqlJoin) sqlNode).getLeft(), sideTableSet); + if (isMultiJoinSqlNode(sqlNode)) { + AliasInfo rightTableAliasInfo = getSqlNodeAliasInfo(((SqlJoin) sqlNode).getRight()); + String rightTableName = StringUtils.isEmpty(rightTableAliasInfo.getName()) ? rightTableAliasInfo.getAlias() : rightTableAliasInfo.getName(); + if (sideTableSet.contains(rightTableName)) { + List> joinIncludeSourceTable = Lists.newArrayList(); + ParseUtils.parseLeftNodeTableName(((SqlJoin) sqlNode).getLeft(), joinIncludeSourceTable, sideTableSet); + + // last source table alias name + side or child query table alias name + _0 + String leftFirstTableAlias = joinIncludeSourceTable.get(0).f0; + String internalTableName = buildInternalTableName(leftFirstTableAlias, SPLIT, rightTableName) + "_0"; + midTableNameMapping.put(internalTableName, joinIncludeSourceTable); + + // select * from xxx + SqlNode newSource = buildSelectByLeftNode(((SqlJoin) sqlNode).getLeft()); + // ( select * from xxx) as xxx_0 + SqlBasicCall newAsNode = buildAsSqlNode(internalTableName, newSource); + ((SqlJoin) sqlNode).setLeft(newAsNode); + + String asNodeAlias = buildInternalTableName(internalTableName, SPLIT, rightTableName); + buildAsSqlNode(asNodeAlias, sqlNode); + + HashBasedTable mappingTable = HashBasedTable.create(); + Set sourceTableName = localTableCache.keySet(); + joinIncludeSourceTable.stream().filter((Tuple2 tabName) -> { + return null != tabName.f1 && sourceTableName.contains(tabName.f1); + }).forEach((Tuple2 tabName ) -> { + String realTableName = tabName.f1; + String tableAlias = tabName.f0; + Table table = localTableCache.get(realTableName); + ParseUtils.fillFieldNameMapping(mappingTable, table.getSchema().getFieldNames(), tableAlias); + + }); +// ParseUtils.replaceJoinConditionTabName(((SqlJoin) sqlNode).getCondition(), mappingTable, internalTableName); + System.out.println(""); + } + } + break; + } + } + + private SqlBasicCall buildAsSqlNode(String internalTableName, SqlNode newSource) { + SqlOperator operator = new SqlAsOperator(); + SqlParserPos sqlParserPos = new SqlParserPos(0, 0); + SqlIdentifier sqlIdentifierAlias = new SqlIdentifier(internalTableName, null, sqlParserPos); + SqlNode[] sqlNodes = new SqlNode[2]; + sqlNodes[0] = newSource; + sqlNodes[1] = sqlIdentifierAlias; + return new SqlBasicCall(operator, sqlNodes, sqlParserPos); + } + private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue queueInfo) { SqlNode leftNode = joinNode.getLeft(); SqlNode rightNode = joinNode.getRight(); @@ -164,44 +292,17 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< String leftTbAlias = ""; String rightTableName = ""; String rightTableAlias = ""; - Map midTableMapping = null ; - boolean leftIsMidTable = false; - // 右节点已经被解析 - boolean rightIsParse = false; Tuple2 rightTableNameAndAlias = null; - - if(leftNode.getKind() == IDENTIFIER){ leftTbName = leftNode.toString(); - }else if(leftNode.getKind() == JOIN){ - JoinInfo leftNodeJoinInfo = (JoinInfo)parseSql(leftNode, sideTableSet, queueInfo);//解析多JOIN - - rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo); - rightIsParse = true; - if (checkIsSideTable(rightTableNameAndAlias.f0, sideTableSet)) { - // select * from xxx - SqlNode sqlNode = buildSelectByLeftNode(leftNode); - // ( select * from xxx) as xxx_0 - SqlBasicCall newAsNode = buildAsNodeByJoinInfo(leftNodeJoinInfo, sqlNode); - leftNode = newAsNode; - joinNode.setLeft(leftNode); - - leftIsMidTable = true; - midTableMapping = saveTabMapping(leftNodeJoinInfo); - - AliasInfo aliasInfo = (AliasInfo) parseSql(newAsNode, sideTableSet, queueInfo); - leftTbName = aliasInfo.getName(); - leftTbAlias = aliasInfo.getAlias(); - } else { - leftTbName = leftNodeJoinInfo.getRightTableName(); - leftTbAlias = leftNodeJoinInfo.getRightTableAlias(); - } - - }else if(leftNode.getKind() == AS){ + } else if (leftNode.getKind() == JOIN) { + System.out.println(leftNode.toString()); + } else if (leftNode.getKind() == AS) { AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo); leftTbName = aliasInfo.getName(); leftTbAlias = aliasInfo.getAlias(); - }else{ + + } else { throw new RuntimeException("---not deal---"); } @@ -210,9 +311,7 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< throw new RuntimeException("side-table must be at the right of join operator"); } - if (!rightIsParse) { - rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo); - } + rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo); rightTableName = rightTableNameAndAlias.f0; rightTableAlias = rightTableNameAndAlias.f1; @@ -221,21 +320,28 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< throw new RuntimeException("side join not support join type of right[current support inner join and left join]"); } - if (leftIsMidTable) { - // 替换右边 on语句 中的字段别名 - SqlNode afterReplaceNameCondition = ParseUtils.replaceJoinConditionTabName(joinNode.getCondition(), midTableMapping); - joinNode.setOperand(5, afterReplaceNameCondition); + Iterator iterator = ((HashMap) midTableNameMapping).values().iterator(); + while (iterator.hasNext()) { + List> next = (List) iterator.next(); + String finalRightTableAlias = rightTableAlias; + String finalRightTableName = rightTableName; + next.forEach(tp2 -> { + if (tp2.f1 == null && tp2.f0 == finalRightTableAlias) { + tp2.f1 = finalRightTableName; + } + }); } + JoinInfo tableInfo = new JoinInfo(); tableInfo.setLeftTableName(leftTbName); tableInfo.setRightTableName(rightTableName); - if (leftTbAlias.equals("")){ + if (StringUtils.isEmpty(leftTbAlias)){ tableInfo.setLeftTableAlias(leftTbName); } else { tableInfo.setLeftTableAlias(leftTbAlias); } - if (rightTableAlias.equals("")){ + if (StringUtils.isEmpty(rightTableAlias)){ tableInfo.setRightTableAlias(rightTableName); } else { tableInfo.setRightTableAlias(rightTableAlias); @@ -246,10 +352,6 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< tableInfo.setRightNode(rightNode); tableInfo.setJoinType(joinType); tableInfo.setCondition(joinNode.getCondition()); - - tableInfo.setLeftIsMidTable(leftIsMidTable); - tableInfo.setLeftTabMapping(midTableMapping); - return tableInfo; } @@ -265,30 +367,6 @@ private Tuple2 parseRightNode(SqlNode sqlNode, Set sideT return tabName; } - private Map saveTabMapping(JoinInfo leftNodeJoinInfo) { - Map midTableMapping = Maps.newHashMap();; - - String midTab = buidTableName(leftNodeJoinInfo.getLeftTableAlias(), SPLIT, leftNodeJoinInfo.getRightTableAlias()); - String finalMidTab = midTab + "_0"; - - if(leftNodeJoinInfo.isLeftIsMidTable()) { - midTableMapping.putAll(leftNodeJoinInfo.getLeftTabMapping()); - } - fillLeftAllTable(leftNodeJoinInfo, midTableMapping, finalMidTab); - return midTableMapping; - } - - private void fillLeftAllTable(JoinInfo leftNodeJoinInfo, Map midTableMapping, String finalMidTab) { - List tablesName = Lists.newArrayList(); - ParseUtils.parseLeftNodeTableName(leftNodeJoinInfo.getLeftNode(), tablesName); - - tablesName.forEach(tab ->{ - midTableMapping.put(tab, finalMidTab); - }); - midTableMapping.put(leftNodeJoinInfo.getRightTableAlias(), finalMidTab); - } - - private SqlNode buildSelectByLeftNode(SqlNode leftNode) { SqlParser sqlParser = SqlParser.create(tempSQL, CalciteConfig.MYSQL_LEX_CONFIG); SqlNode sqlNode = null; @@ -302,94 +380,54 @@ private SqlNode buildSelectByLeftNode(SqlNode leftNode) { return sqlNode; } - - private void dealSelectResultWithJoinInfo(JoinInfo joinInfo, SqlSelect sqlNode, Queue queueInfo){ - // 中间虚拟表进行表名称替换 - if (joinInfo.isLeftIsMidTable()){ - SqlNode whereNode = sqlNode.getWhere(); - SqlNodeList sqlGroup = sqlNode.getGroup(); - SqlNodeList sqlSelectList = sqlNode.getSelectList(); - List newSelectNodeList = Lists.newArrayList(); - - for( int i=0; i queueInfo) { //SideJoinInfo rename - if(joinInfo.checkIsSide()){ + if (joinInfo.checkIsSide()) { joinInfo.setSelectFields(sqlNode.getSelectList()); joinInfo.setSelectNode(sqlNode); - if(joinInfo.isRightIsSideTable()){ + if (joinInfo.isRightIsSideTable()) { //Analyzing left is not a simple table - if(joinInfo.getLeftNode().toString().contains("SELECT")){ + if (joinInfo.getLeftNode().getKind() == SELECT) { queueInfo.offer(joinInfo.getLeftNode()); } queueInfo.offer(joinInfo); - }else{ + } else { //Determining right is not a simple table - if(joinInfo.getRightNode().getKind() == SELECT){ + if (joinInfo.getRightNode().getKind() == SELECT) { queueInfo.offer(joinInfo.getLeftNode()); } queueInfo.offer(joinInfo); } replaceFromNodeForJoin(joinInfo, sqlNode); - + return joinInfo.getNewTableName(); } + return ""; } private void replaceFromNodeForJoin(JoinInfo joinInfo, SqlSelect sqlNode) { //Update from node - SqlBasicCall sqlBasicCall = buildAsNodeByJoinInfo(joinInfo, null); + SqlBasicCall sqlBasicCall = buildAsNodeByJoinInfo(joinInfo, null, null); sqlNode.setFrom(sqlBasicCall); } - private SqlBasicCall buildAsNodeByJoinInfo(JoinInfo joinInfo, SqlNode sqlNode0) { + private SqlBasicCall buildAsNodeByJoinInfo(JoinInfo joinInfo, SqlNode sqlNode0, String tableAlias) { SqlOperator operator = new SqlAsOperator(); SqlParserPos sqlParserPos = new SqlParserPos(0, 0); String joinLeftTableName = joinInfo.getLeftTableName(); String joinLeftTableAlias = joinInfo.getLeftTableAlias(); joinLeftTableName = Strings.isNullOrEmpty(joinLeftTableName) ? joinLeftTableAlias : joinLeftTableName; - String newTableName = buidTableName(joinLeftTableName, SPLIT, joinInfo.getRightTableName()); - String newTableAlias = buidTableName(joinInfo.getLeftTableAlias(), SPLIT, joinInfo.getRightTableAlias()); - - // mid table alias a_b_0 - if (null != sqlNode0) { - newTableAlias += "_0"; - } + String newTableName = buildInternalTableName(joinLeftTableName, SPLIT, joinInfo.getRightTableName()); + String newTableAlias = !StringUtils.isEmpty(tableAlias) ? tableAlias : buildInternalTableName(joinInfo.getLeftTableAlias(), SPLIT, joinInfo.getRightTableAlias()); if (null == sqlNode0) { sqlNode0 = new SqlIdentifier(newTableName, null, sqlParserPos); @@ -402,7 +440,7 @@ private SqlBasicCall buildAsNodeByJoinInfo(JoinInfo joinInfo, SqlNode sqlNode0) return new SqlBasicCall(operator, sqlNodes, sqlParserPos); } - private String buidTableName(String left, char split, String right) { + private String buildInternalTableName(String left, char split, String right) { StringBuilder sb = new StringBuilder(); return sb.append(left).append(split).append(right).toString(); } @@ -413,4 +451,16 @@ private boolean checkIsSideTable(String tableName, Set sideTableList){ } return false; } + + public Map> getMidTableFileNameMapping() { + return midTableFileNameMapping; + } + + public Map>> getMidTableNameMapping() { + return midTableNameMapping; + } + + public void setLocalTableCache(Map localTableCache) { + this.localTableCache = localTableCache; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 0fddcbc37..97daa5d97 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -45,6 +45,7 @@ import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -60,11 +61,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.Timestamp; +import java.util.Arrays; import java.util.Collection; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; +import java.util.Set; import static org.apache.calcite.sql.SqlKind.*; @@ -87,6 +90,7 @@ public class SideSqlExec { private SidePredicatesParser sidePredicatesParser = new SidePredicatesParser(); private Map localTableCache = Maps.newHashMap(); + private StreamTableEnvironment tableEnv ; public void exec(String sql, Map sideTableMap, StreamTableEnvironment tableEnv, Map tableCache, StreamQueryConfig queryConfig) throws Exception { @@ -101,6 +105,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl LOG.error("fill predicates for sideTable fail ", e); } + sideSQLParser.setLocalTableCache(localTableCache); Queue exeQueue = sideSQLParser.getExeQueue(sql, sideTableMap.keySet()); Object pollObj = null; @@ -158,15 +163,11 @@ private void addAliasForFieldNode(SqlNode pollSqlNode, List fieldList, H SqlNode source = ((SqlInsert) pollSqlNode).getSource(); addAliasForFieldNode(source, fieldList, mappingTable); break; - case AS: addAliasForFieldNode(((SqlBasicCall) pollSqlNode).getOperands()[0], fieldList, mappingTable); break; - case SELECT: - SqlNodeList selectList = ((SqlSelect) pollSqlNode).getSelectList(); - selectList.getList().forEach(node -> { if (node.getKind() == IDENTIFIER) { SqlIdentifier sqlIdentifier = (SqlIdentifier) node; @@ -181,7 +182,6 @@ private void addAliasForFieldNode(SqlNode pollSqlNode, List fieldList, H } }); - for (int i = 0; i < selectList.getList().size(); i++) { SqlNode node = selectList.get(i); if (node.getKind() == IDENTIFIER) { @@ -189,7 +189,6 @@ private void addAliasForFieldNode(SqlNode pollSqlNode, List fieldList, H if (sqlIdentifier.names.size() == 1) { return; } - String name = sqlIdentifier.names.get(1); // avoid real field pv0 convert pv if (name.endsWith("0") && !fieldList.contains(name) && !fieldList.contains(name.substring(0, name.length() - 1))) { @@ -204,7 +203,6 @@ private void addAliasForFieldNode(SqlNode pollSqlNode, List fieldList, H selectList.set(i, sqlBasicCall); } - } } break; @@ -228,27 +226,24 @@ public AliasInfo parseASNode(SqlNode sqlNode) throws SqlParseException { return aliasInfo; } - public RowTypeInfo buildOutRowTypeInfo(List sideJoinFieldInfo, HashBasedTable mappingTable){ + public RowTypeInfo buildOutRowTypeInfo(List sideJoinFieldInfo, HashBasedTable mappingTable) { TypeInformation[] sideOutTypes = new TypeInformation[sideJoinFieldInfo.size()]; String[] sideOutNames = new String[sideJoinFieldInfo.size()]; - for(int i=0; i internelMappingTable = sideSQLParser.getMidTableFileNameMapping().values().iterator().next(); + mappingFieldName = internelMappingTable.get(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); + } catch (Exception e) { + LOG.error("internelMappingTable table get mappingFieldName error ",e); + } + } + + if (mappingFieldName == null) { + throw new RuntimeException("can't find mapping fieldName:" + selectNode.toString() ); } sqlIdentifier = sqlIdentifier.setName(0, tableAlias); @@ -697,13 +696,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, if(pollSqlNode.getKind() == INSERT){ tableEnv.sqlUpdate(pollSqlNode.toString()); }else if(pollSqlNode.getKind() == AS){ - AliasInfo aliasInfo = parseASNode(pollSqlNode); - Table table = tableEnv.sqlQuery(aliasInfo.getName()); - tableEnv.registerTable(aliasInfo.getAlias(), table); - if(LOG.isInfoEnabled()){ - LOG.info("Register Table {} by {}", aliasInfo.getAlias(), aliasInfo.getName()); - } - localTableCache.put(aliasInfo.getAlias(), table); + dealAsSourceTable(tableEnv, pollSqlNode); } else if (pollSqlNode.getKind() == SELECT){ Table table = tableEnv.sqlQuery(pollObj.toString()); if (result.getFieldsInfoStr() == null){ @@ -727,6 +720,19 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, } } + protected void dealAsSourceTable(StreamTableEnvironment tableEnv, SqlNode pollSqlNode) throws SqlParseException { + AliasInfo aliasInfo = parseASNode(pollSqlNode); + if (localTableCache.containsKey(aliasInfo.getName())) { + return; + } + Table table = tableEnv.sqlQuery(aliasInfo.getName()); + tableEnv.registerTable(aliasInfo.getAlias(), table); + if (LOG.isInfoEnabled()) { + LOG.info("Register Table {} by {}", aliasInfo.getAlias(), aliasInfo.getName()); + } + localTableCache.put(aliasInfo.getAlias(), table); + } + private void joinFun(Object pollObj, Map localTableCache, Map sideTableMap, StreamTableEnvironment tableEnv, List replaceInfoList) throws Exception{ @@ -737,6 +743,7 @@ private void joinFun(Object pollObj, Map localTableCache, leftScopeChild.setAlias(joinInfo.getLeftTableAlias()); leftScopeChild.setTableName(joinInfo.getLeftTableName()); + dealAsSourceTable(tableEnv,joinInfo.getLeftNode()); Table leftTable = getTableFromCache(localTableCache, joinInfo.getLeftTableAlias(), joinInfo.getLeftTableName()); RowTypeInfo leftTypeInfo = new RowTypeInfo(leftTable.getSchema().getTypes(), leftTable.getSchema().getColumnNames()); leftScopeChild.setRowTypeInfo(leftTypeInfo); @@ -793,9 +800,16 @@ private void joinFun(Object pollObj, Map localTableCache, }else{ dsOut = SideAsyncOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); } - + // TODO 将嵌套表中的字段传递过去, 去除冗余的ROWtime HashBasedTable mappingTable = HashBasedTable.create(); RowTypeInfo sideOutTypeInfo = buildOutRowTypeInfo(sideJoinFieldInfo, mappingTable); + + if (sideSQLParser.getMidTableNameMapping().containsKey(joinInfo.getLeftTableAlias())) { + dealMultiJoinName(localTableCache, sideTableMap, joinInfo); + // TODO 替换连接条件的属性名 +// joinInfo.getCondition() + } + dsOut.getTransformation().setOutputType(sideOutTypeInfo); String targetTableName = joinInfo.getNewTableName(); String targetTableAlias = joinInfo.getNewTableAlias(); @@ -804,7 +818,6 @@ private void joinFun(Object pollObj, Map localTableCache, replaceInfo.setMappingTable(mappingTable); replaceInfo.setTargetTableName(targetTableName); replaceInfo.setTargetTableAlias(targetTableAlias); - replaceInfoList.add(replaceInfo); if (!tableEnv.isRegistered(joinInfo.getNewTableName())){ @@ -812,6 +825,37 @@ private void joinFun(Object pollObj, Map localTableCache, } } + private void dealMultiJoinName(Map localTableCache, Map sideTableMap, JoinInfo joinInfo) { + HashBasedTable midTableMapping = sideSQLParser.getMidTableFileNameMapping().computeIfAbsent(joinInfo.getLeftTableAlias(), key -> { + return HashBasedTable.create(); + }); + + if (midTableMapping.isEmpty()) { + List> childTable = sideSQLParser.getMidTableNameMapping().get(joinInfo.getLeftTableAlias()); + childTable.forEach(tabName -> { + /** mid convert table */ + if (tabName.f1 == null && sideSQLParser.getMidTableFileNameMapping().containsKey(tabName.f0)) { + midTableMapping.putAll(sideSQLParser.getMidTableFileNameMapping().get(tabName.f0)); + sideSQLParser.getMidTableFileNameMapping().remove(tabName.f0); + } else { + /** source or child query table */ + String sourceTable = StringUtils.isEmpty(tabName.f1) ? tabName.f0 : tabName.f1; + String sourceTableAlias = StringUtils.isEmpty(tabName.f1) ? tabName.f1 : tabName.f0; + Table table = localTableCache.get(sourceTable); + Table sourceTable1 = tableEnv.scan("sourceTable"); + if (null != table) { + ParseUtils.fillFieldNameMapping(midTableMapping, table.getSchema().getFieldNames(), sourceTableAlias); + } + } + }); + } + + String[] fieldNames = sideTableMap.get(joinInfo.getRightTableName()).getRowTypeInfo().getFieldNames(); + ParseUtils.fillFieldNameMapping(midTableMapping, fieldNames, joinInfo.getRightTableAlias()); + sideSQLParser.getMidTableFileNameMapping().put(joinInfo.getLeftTableAlias(), midTableMapping); + } + + private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Table table) { diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index 978d93b09..2229e0f38 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -36,14 +36,15 @@ package com.dtstack.flink.sql.util; -import com.google.common.collect.Lists; +import com.google.common.collect.HashBasedTable; import org.apache.calcite.sql.*; -import org.apache.calcite.sql.fun.SqlBetweenOperator; -import org.apache.calcite.sql.fun.SqlCase; -import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Set; import static org.apache.calcite.sql.SqlKind.*; @@ -53,11 +54,11 @@ * @Description: */ public class ParseUtils { - public static void parseAnd(SqlNode conditionNode, List sqlNodeList){ - if(conditionNode.getKind() == SqlKind.AND && ((SqlBasicCall)conditionNode).getOperandList().size()==2){ - parseAnd(((SqlBasicCall)conditionNode).getOperands()[0], sqlNodeList); - sqlNodeList.add(((SqlBasicCall)conditionNode).getOperands()[1]); - }else{ + public static void parseAnd(SqlNode conditionNode, List sqlNodeList) { + if (conditionNode.getKind() == SqlKind.AND && ((SqlBasicCall) conditionNode).getOperandList().size() == 2) { + parseAnd(((SqlBasicCall) conditionNode).getOperands()[0], sqlNodeList); + sqlNodeList.add(((SqlBasicCall) conditionNode).getOperands()[1]); + } else { sqlNodeList.add(conditionNode); } } @@ -76,180 +77,81 @@ public static void parseJoinCompareOperate(SqlNode condition, List sqlJo } public static String transformNotEqualsOperator(SqlKind sqlKind) { - if (StringUtils.equalsIgnoreCase(sqlKind.toString(), "NOT_EQUALS")){ + if (StringUtils.equalsIgnoreCase(sqlKind.toString(), "NOT_EQUALS")) { return "!="; } return sqlKind.sql; } - public static SqlNode replaceJoinConditionTabName(SqlNode conditionNode, Map mappingTable) { - SqlNode[] operands = ((SqlBasicCall) conditionNode).getOperands(); - - for (int i = 0; i < operands.length; i++) { - SqlNode sqlNode = operands[i]; - SqlNode replaceNode = replaceNodeInfo(sqlNode, mappingTable); - operands[i] = replaceNode; - } - return conditionNode; - } - /** - * m.id covert m_x_0.id - * @param selectNode - * @param mapTab - * @return + * parse multi join table name, child query RealTabName is null + * @param leftJoin + * @param aliasAndRealTabName */ - public static SqlNode replaceSelectFieldTabName(SqlNode selectNode, Map mapTab) { - if (selectNode.getKind() == AS) { - SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; - SqlNode replaceNode = replaceSelectFieldTabName(leftNode, mapTab); - if (replaceNode != null) { - ((SqlBasicCall) selectNode).getOperands()[0] = replaceNode; + public static void parseLeftNodeTableName(SqlNode leftJoin, List> aliasAndRealTabName, Set sideTableSet) { + if (leftJoin.getKind() == IDENTIFIER) { + SqlIdentifier sqlIdentifier = (SqlIdentifier) leftJoin; + if (sqlIdentifier.names.size() == 1 && !sideTableSet.contains(sqlIdentifier.names.get(0))) { + aliasAndRealTabName.add(new Tuple2<>(sqlIdentifier.names.get(0), sqlIdentifier.names.get(0))); } - - return selectNode; - }else if(selectNode.getKind() == IDENTIFIER){ - SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; - - if(sqlIdentifier.names.size() == 1){ - return selectNode; + } else if (leftJoin.getKind() == AS) { + SqlNode sqlNode = ((SqlBasicCall) leftJoin).getOperands()[0]; + if (sideTableSet.contains(sqlNode.toString())) { + return; } - - String newTableName = ParseUtils.getRootName(mapTab, sqlIdentifier.getComponent(0).getSimple()); - - if(newTableName == null){ - return selectNode; + if (sqlNode.getKind() == IDENTIFIER) { + aliasAndRealTabName.add(new Tuple2<>(((SqlBasicCall) leftJoin).getOperands()[1].toString(), sqlNode.toString())); + } else { + // child query + aliasAndRealTabName.add(new Tuple2<>(((SqlBasicCall) leftJoin).getOperands()[1].toString(), null)); } - sqlIdentifier = sqlIdentifier.setName(0, newTableName); - return sqlIdentifier; - - }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 - return selectNode; - }else if( AGGREGATE.contains(selectNode.getKind()) - || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) - || COMPARISON.contains(selectNode.getKind()) - || selectNode.getKind() == OTHER_FUNCTION - || selectNode.getKind() == DIVIDE - || selectNode.getKind() == CAST - || selectNode.getKind() == TRIM - || selectNode.getKind() == TIMES - || selectNode.getKind() == PLUS - || selectNode.getKind() == NOT_IN - || selectNode.getKind() == OR - || selectNode.getKind() == AND - || selectNode.getKind() == MINUS - || selectNode.getKind() == TUMBLE - || selectNode.getKind() == TUMBLE_START - || selectNode.getKind() == TUMBLE_END - || selectNode.getKind() == SESSION - || selectNode.getKind() == SESSION_START - || selectNode.getKind() == SESSION_END - || selectNode.getKind() == HOP - || selectNode.getKind() == HOP_START - || selectNode.getKind() == HOP_END - || selectNode.getKind() == BETWEEN - || selectNode.getKind() == IS_NULL - || selectNode.getKind() == IS_NOT_NULL - || selectNode.getKind() == CONTAINS - || selectNode.getKind() == TIMESTAMP_ADD - || selectNode.getKind() == TIMESTAMP_DIFF - - ){ - SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; - for(int i=0; i mapTab, SqlNodeList thenOperands) { - for(int i=0; i mappingTable, String tabAlias) { + if (conditionNode.getKind() == SqlKind.AND && ((SqlBasicCall) conditionNode).getOperandList().size() == 2) { + SqlNode[] operands = ((SqlBasicCall) conditionNode).getOperands(); + Arrays.stream(operands).forEach(op -> replaceJoinConditionTabName(op, mappingTable, tabAlias)); + } else { + SqlNode[] operands = ((SqlBasicCall) conditionNode).getOperands(); + for (int i = 0; i < operands.length; i++) { + SqlNode sqlNode = operands[i]; + SqlNode replaceNode = replaceNodeInfo(sqlNode, mappingTable, tabAlias); + operands[i] = replaceNode; } } + return conditionNode; } - public static SqlNode replaceNodeInfo(SqlNode parseNode, Map mapTab) { + public static SqlNode replaceNodeInfo(SqlNode parseNode, HashBasedTable mappingTable, String tabAlias) { if (parseNode.getKind() == IDENTIFIER) { SqlIdentifier sqlIdentifier = (SqlIdentifier) parseNode; - String newTableName = ParseUtils.getRootName(mapTab, sqlIdentifier.getComponent(0).getSimple());; - if (newTableName == null || sqlIdentifier.names.size() == 1) { - return sqlIdentifier; - } - sqlIdentifier = sqlIdentifier.setName(0, newTableName); return sqlIdentifier; - } else if (parseNode instanceof SqlBasicCall) { - SqlBasicCall sqlBasicCall = (SqlBasicCall) parseNode; - for (int i = 0; i < sqlBasicCall.getOperandList().size(); i++) { - SqlNode sqlNode = sqlBasicCall.getOperandList().get(i); - SqlNode replaceNode = replaceSelectFieldTabName(sqlNode, mapTab); - sqlBasicCall.getOperands()[i] = replaceNode; - } - - return sqlBasicCall; - } else { - return parseNode; } + return parseNode; } - - public static String getRootName(Map maps, String key) { - String res = null; - while (maps.get(key) !=null) { - res = maps.get(key); - key = res; - } - return res; + public static void fillFieldNameMapping(HashBasedTable midTableMapping, String[] fieldNames, String tableAlias) { + Arrays.asList(fieldNames).forEach(fieldName -> { + String mappingTableName = dealDuplicateFieldName(midTableMapping, fieldName); + midTableMapping.put(tableAlias, fieldName, mappingTableName); + }); } - public static void parseLeftNodeTableName(SqlNode leftJoin, List tablesName) { - if (leftJoin.getKind() == IDENTIFIER) { - SqlIdentifier sqlIdentifier = (SqlIdentifier) leftJoin; - tablesName.add(sqlIdentifier.names.get(0)); - } else if (leftJoin.getKind() == AS) { - SqlNode sqlNode = ((SqlBasicCall) leftJoin).getOperands()[1]; - tablesName.add(sqlNode.toString()); - } else if (leftJoin.getKind() == JOIN) { - parseLeftNodeTableName(((SqlJoin) leftJoin).getLeft(), tablesName); - parseLeftNodeTableName(((SqlJoin) leftJoin).getRight(), tablesName); + public static String dealDuplicateFieldName(HashBasedTable mappingTable, String fieldName) { + String mappingFieldName = fieldName; + int index = 0; + String originalVal = fieldName; + while (!mappingTable.column(originalVal).isEmpty() && (mappingTable.column(originalVal).containsValue(fieldName) || !mappingTable.column(fieldName).isEmpty())) { + mappingFieldName = originalVal + index; + fieldName = fieldName + index; + index++; } + return mappingFieldName; } } From 7f19daa7efaa2e06640923157439cb77766ba3c4 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 6 Feb 2020 21:21:09 +0800 Subject: [PATCH 02/13] =?UTF-8?q?=E5=88=9D=E6=AD=A5=E5=AE=8C=E6=88=90flink?= =?UTF-8?q?=20=E5=A4=9A=E7=BB=B4=E8=A1=A8join=20=E6=94=B9=E9=80=A0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/FieldReplaceInfo.java | 37 +++- .../com/dtstack/flink/sql/side/JoinInfo.java | 12 +- .../flink/sql/side/ParserJoinField.java | 9 +- .../dtstack/flink/sql/side/SideSQLParser.java | 163 +++++++----------- .../dtstack/flink/sql/side/SideSqlExec.java | 138 ++++++--------- 5 files changed, 168 insertions(+), 191 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java index 37b23d046..17f906048 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java @@ -21,9 +21,10 @@ package com.dtstack.flink.sql.side; import com.google.common.collect.HashBasedTable; +import org.apache.commons.lang3.StringUtils; /** - * Reason: + * 用于记录转换之后的表和原来表直接字段的关联关系 * Date: 2018/8/30 * Company: www.dtstack.com * @author xuchao @@ -37,6 +38,8 @@ public class FieldReplaceInfo { private String targetTableAlias = null; + private FieldReplaceInfo preNode = null; + public void setMappingTable(HashBasedTable mappingTable) { this.mappingTable = mappingTable; } @@ -57,7 +60,39 @@ public String getTargetTableAlias() { return targetTableAlias; } + public FieldReplaceInfo getPreNode() { + return preNode; + } + + public void setPreNode(FieldReplaceInfo preNode) { + this.preNode = preNode; + } + public void setTargetTableAlias(String targetTableAlias) { this.targetTableAlias = targetTableAlias; } + + /** + * 根据原始的tableName + fieldName 获取转换之后的fieldName + * @param tableName + * @param fieldName + * @return + */ + public String getTargetFieldName(String tableName, String fieldName){ + String targetFieldName = mappingTable.get(tableName, fieldName); + if(StringUtils.isNotBlank(targetFieldName)){ + return targetFieldName; + } + + if(preNode == null){ + return null; + } + + String preNodeTargetFieldName = preNode.getTargetFieldName(tableName, fieldName); + if(StringUtils.isBlank(preNodeTargetFieldName)){ + return null; + } + + return mappingTable.get(preNode.getTargetTableName(), preNodeTargetFieldName); + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java index 851653a48..469e919b3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -42,6 +42,8 @@ public class JoinInfo implements Serializable { //左表是否是维表 private boolean leftIsSideTable; + private boolean leftIsTmpTable = false; + //右表是否是维表 private boolean rightIsSideTable; @@ -64,8 +66,6 @@ public class JoinInfo implements Serializable { private SqlNode selectNode; private JoinType joinType; - // 左边是中间转换表,做表映射关系,给替换属性名称使用 - private Map leftTabMapping; public String getSideTableName(){ if(leftIsSideTable){ @@ -194,4 +194,12 @@ public JoinType getJoinType() { public void setJoinType(JoinType joinType) { this.joinType = joinType; } + + public boolean isLeftIsTmpTable() { + return leftIsTmpTable; + } + + public void setLeftIsTmpTable(boolean leftIsTmpTable) { + this.leftIsTmpTable = leftIsTmpTable; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java index 37a4aac19..dfda49465 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java @@ -51,15 +51,15 @@ public class ParserJoinField { */ public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, boolean getAll){ - if(sqlNode.getKind() != SqlKind.SELECT){ - throw new RuntimeException("------not select node--------\n" + sqlNode.toString()); - } - List fieldInfoList = Lists.newArrayList(); if(getAll){ return getAllField(scope); } + if(sqlNode.getKind() != SqlKind.SELECT){ + throw new RuntimeException("------not select node--------\n" + sqlNode.toString()); + } + SqlSelect sqlSelect = (SqlSelect)sqlNode; SqlNodeList sqlNodeList = sqlSelect.getSelectList(); for(SqlNode fieldNode : sqlNodeList.getList()){ @@ -107,6 +107,7 @@ public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, b return fieldInfoList; } + //TODO 丢弃多余的PROCTIME private static List getAllField(JoinScope scope){ Iterator prefixId = scope.getChildren().iterator(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 994a81756..170a8f88f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -21,8 +21,9 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.config.CalciteConfig; -import com.dtstack.flink.sql.util.ParseUtils; -import com.google.common.collect.HashBasedTable; +import com.google.common.base.Strings; +import com.google.common.collect.Maps; +import com.google.common.collect.Queues; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; @@ -42,17 +43,10 @@ import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; -import com.google.common.base.Strings; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Queues; import org.apache.flink.table.api.Table; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Queue; import java.util.Set; @@ -71,13 +65,6 @@ public class SideSQLParser { private Map localTableCache = Maps.newHashMap(); private final char SPLIT = '_'; - private String tempSQL = "SELECT * FROM TMP"; - - /** 处理连续join时,中间表存储子表字段映射 */ - private Map> midTableFileNameMapping = Maps.newHashMap(); - /** 处理连续join时,原始表与中间表的映射 */ - private Map>> midTableNameMapping = Maps.newHashMap(); - public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { System.out.println("---exeSql---"); @@ -88,14 +75,6 @@ public Queue getExeQueue(String exeSql, Set sideTableSet) throws Queue queueInfo = Queues.newLinkedBlockingQueue(); SqlParser sqlParser = SqlParser.create(exeSql, CalciteConfig.MYSQL_LEX_CONFIG); SqlNode sqlNode = sqlParser.parseStmt(); - SqlNode original = sqlNode; - - try { - checkAndReplaceMultiJoin(sqlNode, sideTableSet); - } catch (Exception e) { - sqlNode = original; - LOG.error("checkAndReplaceMultiJoin method error ", e); - } parseSql(sqlNode, sideTableSet, queueInfo); queueInfo.offer(sqlNode); @@ -126,7 +105,7 @@ private void checkAndReplaceMultiJoin(SqlNode sqlNode, Set sideTableSet) } break; case JOIN: - convertMultiJoinToNestQuery((SqlJoin) sqlNode, sideTableSet); + convertSideJoinToNewQuery((SqlJoin) sqlNode, sideTableSet); break; case AS: SqlNode info = ((SqlBasicCall) sqlNode).getOperands()[0]; @@ -213,10 +192,6 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue return ""; } - private boolean isMultiJoinSqlNode(SqlNode sqlNode) { - return sqlNode.getKind() == JOIN && ((SqlJoin) sqlNode).getLeft().getKind() == JOIN; - } - private AliasInfo getSqlNodeAliasInfo(SqlNode sqlNode) { SqlNode info = ((SqlBasicCall) sqlNode).getOperands()[0]; SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; @@ -228,49 +203,18 @@ private AliasInfo getSqlNodeAliasInfo(SqlNode sqlNode) { return aliasInfo; } - private void convertMultiJoinToNestQuery(SqlNode sqlNode, Set sideTableSet) { - SqlKind sqlKind = sqlNode.getKind(); - switch (sqlKind) { - case JOIN: - checkAndReplaceMultiJoin(((SqlJoin) sqlNode).getRight(), sideTableSet); - checkAndReplaceMultiJoin(((SqlJoin) sqlNode).getLeft(), sideTableSet); - if (isMultiJoinSqlNode(sqlNode)) { - AliasInfo rightTableAliasInfo = getSqlNodeAliasInfo(((SqlJoin) sqlNode).getRight()); - String rightTableName = StringUtils.isEmpty(rightTableAliasInfo.getName()) ? rightTableAliasInfo.getAlias() : rightTableAliasInfo.getName(); - if (sideTableSet.contains(rightTableName)) { - List> joinIncludeSourceTable = Lists.newArrayList(); - ParseUtils.parseLeftNodeTableName(((SqlJoin) sqlNode).getLeft(), joinIncludeSourceTable, sideTableSet); - - // last source table alias name + side or child query table alias name + _0 - String leftFirstTableAlias = joinIncludeSourceTable.get(0).f0; - String internalTableName = buildInternalTableName(leftFirstTableAlias, SPLIT, rightTableName) + "_0"; - midTableNameMapping.put(internalTableName, joinIncludeSourceTable); - - // select * from xxx - SqlNode newSource = buildSelectByLeftNode(((SqlJoin) sqlNode).getLeft()); - // ( select * from xxx) as xxx_0 - SqlBasicCall newAsNode = buildAsSqlNode(internalTableName, newSource); - ((SqlJoin) sqlNode).setLeft(newAsNode); - - String asNodeAlias = buildInternalTableName(internalTableName, SPLIT, rightTableName); - buildAsSqlNode(asNodeAlias, sqlNode); - - HashBasedTable mappingTable = HashBasedTable.create(); - Set sourceTableName = localTableCache.keySet(); - joinIncludeSourceTable.stream().filter((Tuple2 tabName) -> { - return null != tabName.f1 && sourceTableName.contains(tabName.f1); - }).forEach((Tuple2 tabName ) -> { - String realTableName = tabName.f1; - String tableAlias = tabName.f0; - Table table = localTableCache.get(realTableName); - ParseUtils.fillFieldNameMapping(mappingTable, table.getSchema().getFieldNames(), tableAlias); - - }); -// ParseUtils.replaceJoinConditionTabName(((SqlJoin) sqlNode).getCondition(), mappingTable, internalTableName); - System.out.println(""); - } - } - break; + /** + * 将和维表关联的join 替换为一个新的查询 + * @param sqlNode + * @param sideTableSet + */ + private void convertSideJoinToNewQuery(SqlJoin sqlNode, Set sideTableSet) { + checkAndReplaceMultiJoin(sqlNode.getLeft(), sideTableSet); + checkAndReplaceMultiJoin(sqlNode.getRight(), sideTableSet); + + AliasInfo rightTableAliasInfo = getSqlNodeAliasInfo(sqlNode.getRight()); + if(sideTableSet.contains(rightTableAliasInfo.getName())){ + //构建新的查询 } } @@ -288,15 +232,21 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< SqlNode leftNode = joinNode.getLeft(); SqlNode rightNode = joinNode.getRight(); JoinType joinType = joinNode.getJoinType(); + String leftTbName = ""; String leftTbAlias = ""; String rightTableName = ""; String rightTableAlias = ""; + boolean leftTbisTmp = false; + Tuple2 rightTableNameAndAlias = null; if(leftNode.getKind() == IDENTIFIER){ leftTbName = leftNode.toString(); } else if (leftNode.getKind() == JOIN) { - System.out.println(leftNode.toString()); + //处理连续join + SqlBasicCall sqlBasicCall = dealNestJoin((SqlJoin) leftNode, sideTableSet, queueInfo); + leftTbName = sqlBasicCall.getOperands()[0].toString(); + leftTbisTmp = true; } else if (leftNode.getKind() == AS) { AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo); leftTbName = aliasInfo.getName(); @@ -320,19 +270,6 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< throw new RuntimeException("side join not support join type of right[current support inner join and left join]"); } - Iterator iterator = ((HashMap) midTableNameMapping).values().iterator(); - while (iterator.hasNext()) { - List> next = (List) iterator.next(); - String finalRightTableAlias = rightTableAlias; - String finalRightTableName = rightTableName; - next.forEach(tp2 -> { - if (tp2.f1 == null && tp2.f0 == finalRightTableAlias) { - tp2.f1 = finalRightTableName; - } - }); - } - - JoinInfo tableInfo = new JoinInfo(); tableInfo.setLeftTableName(leftTbName); tableInfo.setRightTableName(rightTableName); @@ -341,11 +278,15 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< } else { tableInfo.setLeftTableAlias(leftTbAlias); } + if (StringUtils.isEmpty(rightTableAlias)){ tableInfo.setRightTableAlias(rightTableName); } else { tableInfo.setRightTableAlias(rightTableAlias); } + + tableInfo.setLeftIsTmpTable(leftTbisTmp); + tableInfo.setLeftIsSideTable(leftIsSide); tableInfo.setRightIsSideTable(rightIsSide); tableInfo.setLeftNode(leftNode); @@ -355,6 +296,24 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< return tableInfo; } + //构建新的查询 + private SqlBasicCall dealNestJoin(SqlJoin joinNode, Set sideTableSet, Queue queueInfo){ + SqlNode rightNode = joinNode.getRight(); + + Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo); + String rightTableName = rightTableNameAndAlias.f0; + boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); + + if(!rightIsSide){ + return null; + } + + JoinInfo joinInfo = dealJoinNode(joinNode, sideTableSet, queueInfo); + queueInfo.offer(joinInfo); + return buildAsNodeByJoinInfo(joinInfo, null, null); + + } + private Tuple2 parseRightNode(SqlNode sqlNode, Set sideTableSet, Queue queueInfo) { Tuple2 tabName = new Tuple2<>("", ""); if(sqlNode.getKind() == IDENTIFIER){ @@ -367,19 +326,25 @@ private Tuple2 parseRightNode(SqlNode sqlNode, Set sideT return tabName; } - private SqlNode buildSelectByLeftNode(SqlNode leftNode) { - SqlParser sqlParser = SqlParser.create(tempSQL, CalciteConfig.MYSQL_LEX_CONFIG); - SqlNode sqlNode = null; - try { - sqlNode = sqlParser.parseStmt(); - }catch (Exception e) { - LOG.error("tmp sql parse error..", e); + private Tuple2 parseLeftNode(SqlNode sqlNode){ + Tuple2 tabName = new Tuple2<>("", ""); + if(sqlNode.getKind() == IDENTIFIER){ + tabName.f0 = sqlNode.toString(); + tabName.f1 = sqlNode.toString(); + }else if (sqlNode.getKind() == AS){ + SqlNode info = ((SqlBasicCall)sqlNode).getOperands()[0]; + SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; + + tabName.f0 = info.toString(); + tabName.f1 = alias.toString(); + }else { + throw new RuntimeException(""); } - ((SqlSelect) sqlNode).setFrom(leftNode); - return sqlNode; + return tabName; } + /** * * @param joinInfo @@ -452,14 +417,6 @@ private boolean checkIsSideTable(String tableName, Set sideTableList){ return false; } - public Map> getMidTableFileNameMapping() { - return midTableFileNameMapping; - } - - public Map>> getMidTableNameMapping() { - return midTableNameMapping; - } - public void setLocalTableCache(Map localTableCache) { this.localTableCache = localTableCache; } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 97daa5d97..bb52a7baf 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -56,6 +56,7 @@ import org.apache.flink.table.api.StreamQueryConfig; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.plan.schema.DataStreamTable; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -123,7 +124,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl List fieldNames = null; for(FieldReplaceInfo replaceInfo : replaceInfoList){ fieldNames = Lists.newArrayList(); - replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); + replaceFieldName(pollSqlNode, replaceInfo); addAliasForFieldNode(pollSqlNode, fieldNames, replaceInfo.getMappingTable()); } } @@ -156,6 +157,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl } + //TODO private void addAliasForFieldNode(SqlNode pollSqlNode, List fieldList, HashBasedTable mappingTable) { SqlKind sqlKind = pollSqlNode.getKind(); switch (sqlKind) { @@ -267,19 +269,19 @@ private TypeInformation convertTimeAttributeType(TypeInformation typeInformation } //需要考虑更多的情况 - private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String targetTableName, String tableAlias) { + private void replaceFieldName(SqlNode sqlNode, FieldReplaceInfo replaceInfo) { SqlKind sqlKind = sqlNode.getKind(); switch (sqlKind) { case INSERT: SqlNode sqlSource = ((SqlInsert) sqlNode).getSource(); - replaceFieldName(sqlSource, mappingTable, targetTableName, tableAlias); + replaceFieldName(sqlSource, replaceInfo); break; case AS: SqlNode asNode = ((SqlBasicCall) sqlNode).getOperands()[0]; - replaceFieldName(asNode, mappingTable, targetTableName, tableAlias); + replaceFieldName(asNode, replaceInfo); break; case SELECT: - SqlSelect sqlSelect = (SqlSelect) filterNodeWithTargetName(sqlNode, targetTableName); + SqlSelect sqlSelect = (SqlSelect) filterNodeWithTargetName(sqlNode, replaceInfo.getTargetTableName()); if(sqlSelect == null){ return; } @@ -287,7 +289,7 @@ private void replaceFieldName(SqlNode sqlNode, HashBasedTable replaceNodeList = replaceSelectStarFieldName(selectNode, mappingTable, tableAlias); + List replaceNodeList = replaceSelectStarFieldName(selectNode, replaceInfo); newSelectNodeList.addAll(replaceNodeList); continue; } - SqlNode replaceNode = replaceSelectFieldName(selectNode, mappingTable, tableAlias); + SqlNode replaceNode = replaceSelectFieldName(selectNode, replaceInfo); if(replaceNode == null){ continue; } @@ -322,14 +324,14 @@ private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String tableAlias){ + private SqlNode replaceNodeInfo(SqlNode groupNode, FieldReplaceInfo replaceInfo){ if(groupNode.getKind() == IDENTIFIER){ SqlIdentifier sqlIdentifier = (SqlIdentifier) groupNode; if(sqlIdentifier.names.size() == 1){ return sqlIdentifier; } - String mappingFieldName = mappingTable.get(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); + String mappingFieldName = replaceInfo.getTargetFieldName(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); if(mappingFieldName == null){ throw new RuntimeException("can't find mapping fieldName:" + sqlIdentifier.toString() ); } - sqlIdentifier = sqlIdentifier.setName(0, tableAlias); + + sqlIdentifier = sqlIdentifier.setName(0, replaceInfo.getTargetTableAlias()); return sqlIdentifier.setName(1, mappingFieldName); }else if(groupNode instanceof SqlBasicCall){ SqlBasicCall sqlBasicCall = (SqlBasicCall) groupNode; for(int i=0; i localTableCache, String table return table; } - private List replaceSelectStarFieldName(SqlNode selectNode, HashBasedTable mappingTable, String tableAlias){ + private List replaceSelectStarFieldName(SqlNode selectNode, FieldReplaceInfo replaceInfo){ SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; List sqlNodes = Lists.newArrayList(); if(sqlIdentifier.isStar()){//处理 [* or table.*] int identifierSize = sqlIdentifier.names.size(); Collection columns = null; if(identifierSize == 1){ - columns = mappingTable.values(); + columns = replaceInfo.getMappingTable().values(); }else{ - columns = mappingTable.row(sqlIdentifier.names.get(0)).values(); + columns = replaceInfo.getMappingTable().row(sqlIdentifier.names.get(0)).values(); } for(String colAlias : columns){ SqlParserPos sqlParserPos = new SqlParserPos(0, 0); List columnInfo = Lists.newArrayList(); - columnInfo.add(tableAlias); + columnInfo.add(replaceInfo.getTargetTableAlias()); columnInfo.add(colAlias); SqlIdentifier sqlIdentifierAlias = new SqlIdentifier(columnInfo, sqlParserPos); sqlNodes.add(sqlIdentifierAlias); @@ -490,10 +493,10 @@ private List replaceSelectStarFieldName(SqlNode selectNode, HashBasedTa } } - private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable mappingTable, String tableAlias) { + private SqlNode replaceSelectFieldName(SqlNode selectNode, FieldReplaceInfo replaceInfo) { if (selectNode.getKind() == AS) { SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; - SqlNode replaceNode = replaceSelectFieldName(leftNode, mappingTable, tableAlias); + SqlNode replaceNode = replaceSelectFieldName(leftNode, replaceInfo); if (replaceNode != null) { ((SqlBasicCall) selectNode).getOperands()[0] = replaceNode; } @@ -505,23 +508,14 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable internelMappingTable = sideSQLParser.getMidTableFileNameMapping().values().iterator().next(); - mappingFieldName = internelMappingTable.get(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); - } catch (Exception e) { - LOG.error("internelMappingTable table get mappingFieldName error ",e); - } - } + //Same level mappingTable + String mappingFieldName = replaceInfo.getTargetFieldName(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); if (mappingFieldName == null) { throw new RuntimeException("can't find mapping fieldName:" + selectNode.toString() ); } - sqlIdentifier = sqlIdentifier.setName(0, tableAlias); + sqlIdentifier = sqlIdentifier.setName(0, replaceInfo.getTargetTableAlias()); sqlIdentifier = sqlIdentifier.setName(1, mappingFieldName); return sqlIdentifier; }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 @@ -568,7 +562,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable fieldNames = null; for (FieldReplaceInfo replaceInfo : replaceInfoList) { fieldNames = Lists.newArrayList(); - replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); + replaceFieldName(pollSqlNode, replaceInfo); addAliasForFieldNode(pollSqlNode, fieldNames, replaceInfo.getMappingTable()); } } @@ -725,11 +719,13 @@ protected void dealAsSourceTable(StreamTableEnvironment tableEnv, SqlNode pollSq if (localTableCache.containsKey(aliasInfo.getName())) { return; } + Table table = tableEnv.sqlQuery(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); if (LOG.isInfoEnabled()) { LOG.info("Register Table {} by {}", aliasInfo.getAlias(), aliasInfo.getName()); } + localTableCache.put(aliasInfo.getAlias(), table); } @@ -743,7 +739,15 @@ private void joinFun(Object pollObj, Map localTableCache, leftScopeChild.setAlias(joinInfo.getLeftTableAlias()); leftScopeChild.setTableName(joinInfo.getLeftTableName()); - dealAsSourceTable(tableEnv,joinInfo.getLeftNode()); + SqlKind sqlKind = joinInfo.getLeftNode().getKind(); + if(sqlKind == AS){ + dealAsSourceTable(tableEnv, joinInfo.getLeftNode()); + } else if(joinInfo.isLeftIsTmpTable()){ + //do nothing + } else { + throw new RuntimeException("unsupport left table for join"); + } + Table leftTable = getTableFromCache(localTableCache, joinInfo.getLeftTableAlias(), joinInfo.getLeftTableName()); RowTypeInfo leftTypeInfo = new RowTypeInfo(leftTable.getSchema().getTypes(), leftTable.getSchema().getColumnNames()); leftScopeChild.setRowTypeInfo(leftTypeInfo); @@ -804,12 +808,6 @@ private void joinFun(Object pollObj, Map localTableCache, HashBasedTable mappingTable = HashBasedTable.create(); RowTypeInfo sideOutTypeInfo = buildOutRowTypeInfo(sideJoinFieldInfo, mappingTable); - if (sideSQLParser.getMidTableNameMapping().containsKey(joinInfo.getLeftTableAlias())) { - dealMultiJoinName(localTableCache, sideTableMap, joinInfo); - // TODO 替换连接条件的属性名 -// joinInfo.getCondition() - } - dsOut.getTransformation().setOutputType(sideOutTypeInfo); String targetTableName = joinInfo.getNewTableName(); String targetTableAlias = joinInfo.getNewTableAlias(); @@ -818,46 +816,24 @@ private void joinFun(Object pollObj, Map localTableCache, replaceInfo.setMappingTable(mappingTable); replaceInfo.setTargetTableName(targetTableName); replaceInfo.setTargetTableAlias(targetTableAlias); - replaceInfoList.add(replaceInfo); - if (!tableEnv.isRegistered(joinInfo.getNewTableName())){ - tableEnv.registerDataStream(joinInfo.getNewTableName(), dsOut, String.join(",", sideOutTypeInfo.getFieldNames())); + //判断之前是不是被替换过,被替换过则设置之前的替换信息作为上一个节点 + for(FieldReplaceInfo tmp : replaceInfoList){ + if(tmp.getTargetTableName().equalsIgnoreCase(joinInfo.getLeftTableName())){ + replaceInfo.setPreNode(tmp); + break; + } } - } - private void dealMultiJoinName(Map localTableCache, Map sideTableMap, JoinInfo joinInfo) { - HashBasedTable midTableMapping = sideSQLParser.getMidTableFileNameMapping().computeIfAbsent(joinInfo.getLeftTableAlias(), key -> { - return HashBasedTable.create(); - }); + replaceInfoList.add(replaceInfo); - if (midTableMapping.isEmpty()) { - List> childTable = sideSQLParser.getMidTableNameMapping().get(joinInfo.getLeftTableAlias()); - childTable.forEach(tabName -> { - /** mid convert table */ - if (tabName.f1 == null && sideSQLParser.getMidTableFileNameMapping().containsKey(tabName.f0)) { - midTableMapping.putAll(sideSQLParser.getMidTableFileNameMapping().get(tabName.f0)); - sideSQLParser.getMidTableFileNameMapping().remove(tabName.f0); - } else { - /** source or child query table */ - String sourceTable = StringUtils.isEmpty(tabName.f1) ? tabName.f0 : tabName.f1; - String sourceTableAlias = StringUtils.isEmpty(tabName.f1) ? tabName.f1 : tabName.f0; - Table table = localTableCache.get(sourceTable); - Table sourceTable1 = tableEnv.scan("sourceTable"); - if (null != table) { - ParseUtils.fillFieldNameMapping(midTableMapping, table.getSchema().getFieldNames(), sourceTableAlias); - } - } - }); + if (!tableEnv.isRegistered(joinInfo.getNewTableName())){ + Table joinTable = tableEnv.fromDataStream(dsOut, String.join(",", sideOutTypeInfo.getFieldNames())); + tableEnv.registerTable(joinInfo.getNewTableName(), joinTable); + localTableCache.put(joinInfo.getNewTableName(), joinTable); } - - String[] fieldNames = sideTableMap.get(joinInfo.getRightTableName()).getRowTypeInfo().getFieldNames(); - ParseUtils.fillFieldNameMapping(midTableMapping, fieldNames, joinInfo.getRightTableAlias()); - sideSQLParser.getMidTableFileNameMapping().put(joinInfo.getLeftTableAlias(), midTableMapping); } - - - private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Table table) { List fieldNames = new LinkedList<>(); String fieldsInfo = result.getFieldsInfoStr(); From 73b50861beea6ee12a09f7c728153c98570b912a Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 11 Feb 2020 19:35:34 +0800 Subject: [PATCH 03/13] =?UTF-8?q?=E5=88=AB=E5=90=8D=E4=BC=98=E5=8C=96?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/side/FieldReplaceInfo.java | 2 +- .../main/java/com/dtstack/flink/sql/side/SideSQLParser.java | 1 + .../java/com/dtstack/flink/sql/table/AbsTableParser.java | 5 +++++ 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java index 17f906048..703721ef2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java @@ -93,6 +93,6 @@ public String getTargetFieldName(String tableName, String fieldName){ return null; } - return mappingTable.get(preNode.getTargetTableName(), preNodeTargetFieldName); + return mappingTable.get(preNode.getTargetTableAlias(), preNodeTargetFieldName); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 170a8f88f..e84317374 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -246,6 +246,7 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< //处理连续join SqlBasicCall sqlBasicCall = dealNestJoin((SqlJoin) leftNode, sideTableSet, queueInfo); leftTbName = sqlBasicCall.getOperands()[0].toString(); + leftTbAlias = sqlBasicCall.getOperands()[1].toString(); leftTbisTmp = true; } else if (leftNode.getKind() == AS) { AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 5af06d087..3f4d1217d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -24,6 +24,7 @@ import com.dtstack.flink.sql.util.DtStringUtil; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import org.apache.commons.lang3.StringUtils; import java.util.List; import java.util.Map; @@ -85,6 +86,10 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ for(String fieldRow : fieldRows){ fieldRow = fieldRow.trim(); + if(StringUtils.isBlank(fieldRow)){ + throw new RuntimeException(String.format("table [%s],exists field empty.", tableInfo.getName())); + } + String[] filedInfoArr = fieldRow.split("\\s+"); if(filedInfoArr.length < 2 ){ throw new RuntimeException(String.format("table [%s] field [%s] format error.", tableInfo.getName(), fieldRow)); From 91a6768071bb11c5615c30b2f18c61c29b848d54 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 11 Feb 2020 20:18:50 +0800 Subject: [PATCH 04/13] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=E9=87=8D=E5=A4=8D?= =?UTF-8?q?=E5=90=8D=E7=A7=B0=E7=9A=84=E6=96=B0=E5=90=8D=E7=A7=B0=E7=94=9F?= =?UTF-8?q?=E6=88=90=E8=A7=84=E5=88=99?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/util/ParseUtils.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index 2229e0f38..d399b533c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -146,10 +146,8 @@ public static void fillFieldNameMapping(HashBasedTable m public static String dealDuplicateFieldName(HashBasedTable mappingTable, String fieldName) { String mappingFieldName = fieldName; int index = 0; - String originalVal = fieldName; - while (!mappingTable.column(originalVal).isEmpty() && (mappingTable.column(originalVal).containsValue(fieldName) || !mappingTable.column(fieldName).isEmpty())) { - mappingFieldName = originalVal + index; - fieldName = fieldName + index; + while (!mappingTable.column(mappingFieldName).isEmpty()) { + mappingFieldName = mappingFieldName + index; index++; } return mappingFieldName; From fbdd7f59c05238aacab467b6155a36c2385840e5 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 17 Feb 2020 14:43:15 +0800 Subject: [PATCH 05/13] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E6=94=AF=E6=8C=81=20?= =?UTF-8?q?=E6=B5=81=20join=20=E6=B5=81=20join=20=E7=BB=B4=E8=A1=A8?= =?UTF-8?q?=E7=B1=BB=E5=9E=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/side/SideSQLParser.java | 357 ++++++++++++++++-- .../dtstack/flink/sql/side/SideSqlExec.java | 60 ++- .../dtstack/flink/sql/util/TableUtils.java | 88 +++++ 3 files changed, 467 insertions(+), 38 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index e84317374..eadeb8264 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -22,15 +22,20 @@ import com.dtstack.flink.sql.config.CalciteConfig; import com.google.common.base.Strings; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Queues; +import com.google.common.collect.Sets; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlBinaryOperator; +import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlJoin; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperator; @@ -38,15 +43,19 @@ import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.SqlWith; import org.apache.calcite.sql.SqlWithItem; +import org.apache.calcite.sql.fun.SqlCase; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.api.Table; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.List; import java.util.Map; import java.util.Queue; import java.util.Set; @@ -66,6 +75,9 @@ public class SideSQLParser { private Map localTableCache = Maps.newHashMap(); private final char SPLIT = '_'; + //regular joins(不带时间窗口) 不允许查询出rowtime或者proctime + private final String SELECT_TEMP_SQL = "select %s from %s %s"; + public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { System.out.println("---exeSql---"); System.out.println(exeSql); @@ -76,7 +88,7 @@ public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParser sqlParser = SqlParser.create(exeSql, CalciteConfig.MYSQL_LEX_CONFIG); SqlNode sqlNode = sqlParser.parseStmt(); - parseSql(sqlNode, sideTableSet, queueInfo); + parseSql(sqlNode, sideTableSet, queueInfo, null, null); queueInfo.offer(sqlNode); return queueInfo; } @@ -123,7 +135,7 @@ private void checkAndReplaceMultiJoin(SqlNode sqlNode, Set sideTableSet) } - private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue queueInfo){ + private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue queueInfo, SqlNode parentWhere, SqlNodeList parentSelectList){ SqlKind sqlKind = sqlNode.getKind(); switch (sqlKind){ case WITH: { @@ -131,19 +143,22 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue SqlNodeList sqlNodeList = sqlWith.withList; for (SqlNode withAsTable : sqlNodeList) { SqlWithItem sqlWithItem = (SqlWithItem) withAsTable; - parseSql(sqlWithItem.query, sideTableSet, queueInfo); + parseSql(sqlWithItem.query, sideTableSet, queueInfo, parentWhere, parentSelectList); queueInfo.add(sqlWithItem); } - parseSql(sqlWith.body, sideTableSet, queueInfo); + parseSql(sqlWith.body, sideTableSet, queueInfo, parentWhere, parentSelectList); break; } case INSERT: SqlNode sqlSource = ((SqlInsert)sqlNode).getSource(); - return parseSql(sqlSource, sideTableSet, queueInfo); + return parseSql(sqlSource, sideTableSet, queueInfo, parentWhere, parentSelectList); case SELECT: SqlNode sqlFrom = ((SqlSelect)sqlNode).getFrom(); + SqlNode sqlWhere = ((SqlSelect)sqlNode).getWhere(); + SqlNodeList selectList = ((SqlSelect)sqlNode).getSelectList(); + if(sqlFrom.getKind() != IDENTIFIER){ - Object result = parseSql(sqlFrom, sideTableSet, queueInfo); + Object result = parseSql(sqlFrom, sideTableSet, queueInfo, sqlWhere, selectList); if(result instanceof JoinInfo){ return dealSelectResultWithJoinInfo((JoinInfo) result, (SqlSelect) sqlNode, queueInfo); }else if(result instanceof AliasInfo){ @@ -160,7 +175,7 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue } break; case JOIN: - return dealJoinNode((SqlJoin) sqlNode, sideTableSet, queueInfo); + return dealJoinNode((SqlJoin) sqlNode, sideTableSet, queueInfo, parentWhere, parentSelectList); case AS: SqlNode info = ((SqlBasicCall)sqlNode).getOperands()[0]; SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; @@ -169,7 +184,7 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue if(info.getKind() == IDENTIFIER){ infoStr = info.toString(); } else { - infoStr = parseSql(info, sideTableSet, queueInfo).toString(); + infoStr = parseSql(info, sideTableSet, queueInfo, parentWhere, parentSelectList).toString(); } AliasInfo aliasInfo = new AliasInfo(); @@ -182,12 +197,12 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue SqlNode unionLeft = ((SqlBasicCall)sqlNode).getOperands()[0]; SqlNode unionRight = ((SqlBasicCall)sqlNode).getOperands()[1]; - parseSql(unionLeft, sideTableSet, queueInfo); - parseSql(unionRight, sideTableSet, queueInfo); + parseSql(unionLeft, sideTableSet, queueInfo, parentWhere, parentSelectList); + parseSql(unionRight, sideTableSet, queueInfo, parentWhere, parentSelectList); break; case ORDER_BY: SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode; - parseSql(sqlOrderBy.query, sideTableSet, queueInfo); + parseSql(sqlOrderBy.query, sideTableSet, queueInfo, parentWhere, parentSelectList); } return ""; } @@ -228,7 +243,8 @@ private SqlBasicCall buildAsSqlNode(String internalTableName, SqlNode newSource) return new SqlBasicCall(operator, sqlNodes, sqlParserPos); } - private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue queueInfo) { + private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue queueInfo, + SqlNode parentWhere, SqlNodeList parentSelectList) { SqlNode leftNode = joinNode.getLeft(); SqlNode rightNode = joinNode.getRight(); JoinType joinType = joinNode.getJoinType(); @@ -244,12 +260,12 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< leftTbName = leftNode.toString(); } else if (leftNode.getKind() == JOIN) { //处理连续join - SqlBasicCall sqlBasicCall = dealNestJoin((SqlJoin) leftNode, sideTableSet, queueInfo); + SqlBasicCall sqlBasicCall = dealNestJoin((SqlJoin) leftNode, sideTableSet, queueInfo, parentWhere, parentSelectList); leftTbName = sqlBasicCall.getOperands()[0].toString(); leftTbAlias = sqlBasicCall.getOperands()[1].toString(); leftTbisTmp = true; } else if (leftNode.getKind() == AS) { - AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo); + AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo, parentWhere, parentSelectList); leftTbName = aliasInfo.getName(); leftTbAlias = aliasInfo.getAlias(); @@ -262,7 +278,7 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< throw new RuntimeException("side-table must be at the right of join operator"); } - rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo); + rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, parentSelectList); rightTableName = rightTableNameAndAlias.f0; rightTableAlias = rightTableNameAndAlias.f1; @@ -294,33 +310,207 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< tableInfo.setRightNode(rightNode); tableInfo.setJoinType(joinType); tableInfo.setCondition(joinNode.getCondition()); + + //TODO 抽取 + if(tableInfo.getLeftNode().getKind() != AS){ + //build 临时中间查询 + try{ + //父一级的where 条件中如果只和临时查询相关的条件都截取进来 + Set fromTableNameSet = Sets.newHashSet(); + List extractCondition = Lists.newArrayList(); + + getFromTableInfo(tableInfo.getLeftNode(), fromTableNameSet); + checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere, extractCondition); + + //TODO 查询的字段需要根据最上层的字段中获取,而不是直接设置为*,当然如果上一层就是*另说 + + List extractSelectField = extractSelectList(parentSelectList, fromTableNameSet); + String extractSelectFieldStr = buildSelectNode(extractSelectField); + String extractConditionStr = buildCondition(extractCondition); + + String tmpSelectSql = String.format(SELECT_TEMP_SQL, + extractSelectFieldStr, + tableInfo.getLeftNode().toString(), + extractConditionStr); + + SqlParser sqlParser = SqlParser.create(tmpSelectSql, CalciteConfig.MYSQL_LEX_CONFIG); + SqlNode sqlNode = sqlParser.parseStmt(); + SqlBasicCall sqlBasicCall = buildAsSqlNode(tableInfo.getLeftTableAlias(), sqlNode); + queueInfo.offer(sqlBasicCall); + + //TODO 打印合适的提示 + System.out.println(tmpSelectSql); + }catch (Exception e){ + e.printStackTrace(); + throw new RuntimeException(e); + } + + }else { + SqlKind asFirstKind = ((SqlBasicCall)tableInfo.getLeftNode()).operands[0].getKind(); + if(asFirstKind == SELECT){ + queueInfo.offer(tableInfo.getLeftNode()); + tableInfo.setLeftNode(((SqlBasicCall)tableInfo.getLeftNode()).operands[1]); + } + } return tableInfo; } //构建新的查询 - private SqlBasicCall dealNestJoin(SqlJoin joinNode, Set sideTableSet, Queue queueInfo){ + private SqlBasicCall dealNestJoin(SqlJoin joinNode, Set sideTableSet, Queue queueInfo, SqlNode parentWhere, SqlNodeList selectList){ SqlNode rightNode = joinNode.getRight(); - Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo); + Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, selectList); + + JoinInfo joinInfo = dealJoinNode(joinNode, sideTableSet, queueInfo, parentWhere, selectList); + String rightTableName = rightTableNameAndAlias.f0; boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); - if(!rightIsSide){ - return null; + //右表不是维表的情况 + }else{ + //右边表是维表需要重新构建左表的临时查询 + queueInfo.offer(joinInfo); } - JoinInfo joinInfo = dealJoinNode(joinNode, sideTableSet, queueInfo); - queueInfo.offer(joinInfo); return buildAsNodeByJoinInfo(joinInfo, null, null); + } + + public boolean checkAndRemoveCondition(Set fromTableNameSet, SqlBasicCall parentWhere, List extractContition){ + SqlKind kind = parentWhere.getKind(); + if(kind == AND){ + boolean removeLeft = checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[0], extractContition); + boolean removeRight = checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[1], extractContition); + //DO remove + if(removeLeft){ + extractContition.add(removeWhereConditionNode(parentWhere, 0)); + } + + if(removeRight){ + extractContition.add(removeWhereConditionNode(parentWhere, 1)); + } + + return false; + }else{ + Set conditionRefTableNameSet = Sets.newHashSet(); + getConditionRefTable(parentWhere, conditionRefTableNameSet); + + if(fromTableNameSet.containsAll(conditionRefTableNameSet)){ + return true; + } + return false; + } } - private Tuple2 parseRightNode(SqlNode sqlNode, Set sideTableSet, Queue queueInfo) { + /** + * 抽取上层需用使用到的字段 + * 由于where字段已经抽取到上一层了所以不用查询出来 + * @param parentSelectList + * @param fromTableNameSet + * @return + */ + private List extractSelectList(SqlNodeList parentSelectList, Set fromTableNameSet){ + List extractFieldList = Lists.newArrayList(); + for(SqlNode selectNode : parentSelectList.getList()){ + extractSelectField(selectNode, extractFieldList, fromTableNameSet); + } + + return extractFieldList; + } + + private void extractSelectField(SqlNode selectNode, List extractFieldList, Set fromTableNameSet){ + if (selectNode.getKind() == AS) { + SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; + extractSelectField(leftNode, extractFieldList, fromTableNameSet); + + }else if(selectNode.getKind() == IDENTIFIER) { + SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; + + if(sqlIdentifier.names.size() == 1){ + return; + } + + String tableName = sqlIdentifier.names.get(0); + if(fromTableNameSet.contains(tableName)){ + extractFieldList.add(sqlIdentifier.toString()); + } + + }else if( AGGREGATE.contains(selectNode.getKind()) + || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) + || COMPARISON.contains(selectNode.getKind()) + || selectNode.getKind() == OTHER_FUNCTION + || selectNode.getKind() == DIVIDE + || selectNode.getKind() == CAST + || selectNode.getKind() == TRIM + || selectNode.getKind() == TIMES + || selectNode.getKind() == PLUS + || selectNode.getKind() == NOT_IN + || selectNode.getKind() == OR + || selectNode.getKind() == AND + || selectNode.getKind() == MINUS + || selectNode.getKind() == TUMBLE + || selectNode.getKind() == TUMBLE_START + || selectNode.getKind() == TUMBLE_END + || selectNode.getKind() == SESSION + || selectNode.getKind() == SESSION_START + || selectNode.getKind() == SESSION_END + || selectNode.getKind() == HOP + || selectNode.getKind() == HOP_START + || selectNode.getKind() == HOP_END + || selectNode.getKind() == BETWEEN + || selectNode.getKind() == IS_NULL + || selectNode.getKind() == IS_NOT_NULL + || selectNode.getKind() == CONTAINS + || selectNode.getKind() == TIMESTAMP_ADD + || selectNode.getKind() == TIMESTAMP_DIFF + || selectNode.getKind() == LIKE + + ){ + SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; + for(int i=0; i parseRightNode(SqlNode sqlNode, Set sideTableSet, Queue queueInfo, + SqlNode parentWhere, SqlNodeList selectList) { Tuple2 tabName = new Tuple2<>("", ""); if(sqlNode.getKind() == IDENTIFIER){ tabName.f0 = sqlNode.toString(); }else{ - AliasInfo aliasInfo = (AliasInfo)parseSql(sqlNode, sideTableSet, queueInfo); + AliasInfo aliasInfo = (AliasInfo)parseSql(sqlNode, sideTableSet, queueInfo, parentWhere, selectList); tabName.f0 = aliasInfo.getName(); tabName.f1 = aliasInfo.getAlias(); } @@ -345,6 +535,38 @@ private Tuple2 parseLeftNode(SqlNode sqlNode){ return tabName; } + public SqlBasicCall removeWhereConditionNode(SqlBasicCall parentWhere, int index){ + //构造1=1 条件 + SqlBasicCall oldCondition = (SqlBasicCall) parentWhere.getOperands()[index]; + parentWhere.setOperand(index, buildDefaultCondition()); + return oldCondition; + } + + public String buildCondition(List conditionList){ + if(CollectionUtils.isEmpty(conditionList)){ + return ""; + } + + return " where " + StringUtils.join(conditionList, " AND "); + } + + public String buildSelectNode(List extractSelectField){ + if(CollectionUtils.isEmpty(extractSelectField)){ + throw new RuntimeException("no field is used"); + } + + return StringUtils.join(extractSelectField, ","); + } + + public SqlBasicCall buildDefaultCondition(){ + SqlBinaryOperator equalsOperators = SqlStdOperatorTable.EQUALS; + SqlNode[] operands = new SqlNode[2]; + operands[0] = SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO); + operands[1] = SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO); + + return new SqlBasicCall(equalsOperators, operands, SqlParserPos.ZERO); + } + /** * @@ -421,4 +643,93 @@ private boolean checkIsSideTable(String tableName, Set sideTableList){ public void setLocalTableCache(Map localTableCache) { this.localTableCache = localTableCache; } + + //TODO 之后抽取 + private void getConditionRefTable(SqlNode selectNode, Set tableNameSet) { + if(selectNode.getKind() == IDENTIFIER){ + SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; + + if(sqlIdentifier.names.size() == 1){ + return; + } + + String tableName = sqlIdentifier.names.asList().get(0); + tableNameSet.add(tableName); + return; + }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 + return; + }else if( AGGREGATE.contains(selectNode.getKind()) + || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) + || COMPARISON.contains(selectNode.getKind()) + || selectNode.getKind() == OTHER_FUNCTION + || selectNode.getKind() == DIVIDE + || selectNode.getKind() == CAST + || selectNode.getKind() == TRIM + || selectNode.getKind() == TIMES + || selectNode.getKind() == PLUS + || selectNode.getKind() == NOT_IN + || selectNode.getKind() == OR + || selectNode.getKind() == AND + || selectNode.getKind() == MINUS + || selectNode.getKind() == TUMBLE + || selectNode.getKind() == TUMBLE_START + || selectNode.getKind() == TUMBLE_END + || selectNode.getKind() == SESSION + || selectNode.getKind() == SESSION_START + || selectNode.getKind() == SESSION_END + || selectNode.getKind() == HOP + || selectNode.getKind() == HOP_START + || selectNode.getKind() == HOP_END + || selectNode.getKind() == BETWEEN + || selectNode.getKind() == IS_NULL + || selectNode.getKind() == IS_NOT_NULL + || selectNode.getKind() == CONTAINS + || selectNode.getKind() == TIMESTAMP_ADD + || selectNode.getKind() == TIMESTAMP_DIFF + || selectNode.getKind() == LIKE + + ){ + SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; + for(int i=0; i tableNameSet){ + System.out.println(fromTable); + SqlKind sqlKind = fromTable.getKind(); + switch (sqlKind){ + case AS: + SqlNode alias = ((SqlBasicCall) fromTable).getOperands()[1]; + tableNameSet.add(alias.toString()); + return; + case JOIN: + getFromTableInfo(((SqlJoin)fromTable).getLeft(), tableNameSet); + getFromTableInfo(((SqlJoin)fromTable).getRight(), tableNameSet); + return; + case IDENTIFIER: + tableNameSet.add(((SqlIdentifier)fromTable).getSimple()); + return; + default: + throw new RuntimeException("not support sqlKind:" + sqlKind); + } + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index bb52a7baf..04ac98835 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -27,6 +27,10 @@ import com.dtstack.flink.sql.side.operator.SideWithAllCacheOperator; import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.ParseUtils; +import com.dtstack.flink.sql.util.TableUtils; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlDataTypeSpec; @@ -45,30 +49,24 @@ import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import com.google.common.collect.HashBasedTable; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.StreamQueryConfig; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; -import org.apache.flink.table.plan.schema.DataStreamTable; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.sql.Timestamp; -import java.util.Arrays; import java.util.Collection; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; -import java.util.Set; import static org.apache.calcite.sql.SqlKind.*; @@ -141,6 +139,12 @@ public void exec(String sql, Map sideTableMap, StreamTabl Table table = tableEnv.sqlQuery(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); localTableCache.put(aliasInfo.getAlias(), table); + + //TODO 解析出as查询的表和字段的关系 + FieldReplaceInfo fieldReplaceInfo = parseAsQuery((SqlBasicCall) pollSqlNode, tableCache); + if(fieldReplaceInfo != null){ + replaceInfoList.add(fieldReplaceInfo); + } } else if (pollSqlNode.getKind() == WITH_ITEM) { SqlWithItem sqlWithItem = (SqlWithItem) pollSqlNode; String TableAlias = sqlWithItem.name.toString(); @@ -156,6 +160,35 @@ public void exec(String sql, Map sideTableMap, StreamTabl } + //TODO + //FIXME 如果和create view 的名称命名相同 + private FieldReplaceInfo parseAsQuery(SqlBasicCall asSqlNode, Map tableCache){ + SqlNode info = asSqlNode.getOperands()[0]; + SqlNode alias = asSqlNode.getOperands()[1]; + + SqlKind infoKind = info.getKind(); + if(infoKind != SELECT){ + return null; + } + + List extractFieldList = TableUtils.parserSelectField((SqlSelect) info, tableCache); + System.out.println(extractFieldList); + + HashBasedTable mappingTable = HashBasedTable.create(); + for (FieldInfo fieldInfo : extractFieldList) { + String tableName = fieldInfo.getTable(); + String fieldName = fieldInfo.getFieldName(); + String mappingFieldName = ParseUtils.dealDuplicateFieldName(mappingTable, fieldName); + mappingTable.put(tableName, fieldName, mappingFieldName); + } + + FieldReplaceInfo replaceInfo = new FieldReplaceInfo(); + replaceInfo.setMappingTable(mappingTable); + replaceInfo.setTargetTableName(alias.toString()); + replaceInfo.setTargetTableAlias(alias.toString()); + return replaceInfo; + } + //TODO private void addAliasForFieldNode(SqlNode pollSqlNode, List fieldList, HashBasedTable mappingTable) { @@ -335,7 +368,6 @@ private void replaceFieldName(SqlNode sqlNode, FieldReplaceInfo replaceInfo) { sqlGroup.set(i, replaceNode); } } - System.out.println("-----------------"); } }else{ //TODO @@ -742,10 +774,6 @@ private void joinFun(Object pollObj, Map localTableCache, SqlKind sqlKind = joinInfo.getLeftNode().getKind(); if(sqlKind == AS){ dealAsSourceTable(tableEnv, joinInfo.getLeftNode()); - } else if(joinInfo.isLeftIsTmpTable()){ - //do nothing - } else { - throw new RuntimeException("unsupport left table for join"); } Table leftTable = getTableFromCache(localTableCache, joinInfo.getLeftTableAlias(), joinInfo.getLeftTableName()); @@ -785,7 +813,7 @@ private void joinFun(Object pollObj, Map localTableCache, RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class) - .map((Tuple2 f0) -> { return f0.f1; }) + .map((Tuple2 f0) -> f0.f1) .returns(Row.class); @@ -798,12 +826,13 @@ private void joinFun(Object pollObj, Map localTableCache, adaptStream = adaptStream.keyBy(leftJoinColArr); } - DataStream dsOut = null; + DataStream dsOut; if(ECacheType.ALL.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ dsOut = SideWithAllCacheOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); }else{ dsOut = SideAsyncOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); } + // TODO 将嵌套表中的字段传递过去, 去除冗余的ROWtime HashBasedTable mappingTable = HashBasedTable.create(); RowTypeInfo sideOutTypeInfo = buildOutRowTypeInfo(sideJoinFieldInfo, mappingTable); @@ -819,7 +848,8 @@ private void joinFun(Object pollObj, Map localTableCache, //判断之前是不是被替换过,被替换过则设置之前的替换信息作为上一个节点 for(FieldReplaceInfo tmp : replaceInfoList){ - if(tmp.getTargetTableName().equalsIgnoreCase(joinInfo.getLeftTableName())){ + if(tmp.getTargetTableName().equalsIgnoreCase(joinInfo.getLeftTableName()) + ||tmp.getTargetTableName().equalsIgnoreCase(joinInfo.getLeftTableAlias())){ replaceInfo.setPreNode(tmp); break; } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java new file mode 100644 index 000000000..9ca76627c --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flink.sql.util; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.google.common.collect.Lists; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlSelect; +import org.apache.flink.table.api.Table; + +import java.util.List; +import java.util.Map; + +/** + * + * Date: 2020/2/17 + * Company: www.dtstack.com + * @author xuchao + */ + +public class TableUtils { + + /** + * 获取select 的字段 + * @param sqlSelect + */ + public static List parserSelectField(SqlSelect sqlSelect, Map localTableCache){ + SqlNodeList sqlNodeList = sqlSelect.getSelectList(); + List fieldInfoList = Lists.newArrayList(); + + for(SqlNode fieldNode : sqlNodeList.getList()){ + SqlIdentifier identifier = (SqlIdentifier)fieldNode; + if(!identifier.isStar()) { + System.out.println(identifier); + String tableName = identifier.getComponent(0).getSimple(); + String fieldName = identifier.getComponent(1).getSimple(); + FieldInfo fieldInfo = new FieldInfo(); + fieldInfo.setTable(tableName); + fieldInfo.setFieldName(fieldName); + fieldInfoList.add(fieldInfo); + } else { + //处理 + int identifierSize = identifier.names.size(); + + switch(identifierSize) { + case 1: + throw new RuntimeException("not support to parse * without scope of table"); + default: + SqlIdentifier tableIdentify = identifier.skipLast(1); + Table registerTable = localTableCache.get(tableIdentify.getSimple()); + if(registerTable == null){ + throw new RuntimeException("can't find table alias " + tableIdentify.getSimple()); + } + + String[] fieldNames = registerTable.getSchema().getFieldNames(); + for(String fieldName : fieldNames){ + FieldInfo fieldInfo = new FieldInfo(); + fieldInfo.setTable(tableIdentify.getSimple()); + fieldInfo.setFieldName(fieldName); + fieldInfoList.add(fieldInfo); + } + } + } + } + + return fieldInfoList; + } + +} From 3a7659d12d6067a304ba03d6c10ccb246c1607f3 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 17 Feb 2020 17:37:57 +0800 Subject: [PATCH 06/13] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E6=94=AF=E6=8C=81=20?= =?UTF-8?q?=E6=B5=81=20join=20=E6=B5=81=20join=20=E7=BB=B4=E8=A1=A8?= =?UTF-8?q?=E7=B1=BB=E5=9E=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/Main.java | 4 + .../flink/sql/side/ParserJoinField.java | 2 - .../dtstack/flink/sql/side/SideSQLParser.java | 200 +++++++----------- .../dtstack/flink/sql/side/SideSqlExec.java | 17 +- .../dtstack/flink/sql/util/TableUtils.java | 83 +++++++- .../sql/side/rdb/all/RdbAllSideInfo.java | 2 +- .../sql/side/rdb/async/RdbAsyncSideInfo.java | 2 +- 7 files changed, 171 insertions(+), 139 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 435117a46..24d616384 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -181,8 +181,12 @@ private static void sqlTranslation(String localSqlPluginPath, StreamTableEnviron //sql-dimensional table contains the dimension table of execution sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig); }else{ + System.out.println("----------exec sql without dimension join-----------" ); + System.out.println("----------real sql exec is--------------------------"); + System.out.println(result.getExecSql()); FlinkSQLExec.sqlUpdate(tableEnv, result.getExecSql(), queryConfig); if(LOG.isInfoEnabled()){ + System.out.println(); LOG.info("exec sql: " + result.getExecSql()); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java index dfda49465..738e0f84e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java @@ -65,7 +65,6 @@ public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, b for(SqlNode fieldNode : sqlNodeList.getList()){ SqlIdentifier identifier = (SqlIdentifier)fieldNode; if(!identifier.isStar()) { - System.out.println(identifier); String tableName = identifier.getComponent(0).getSimple(); String fieldName = identifier.getComponent(1).getSimple(); TypeInformation type = scope.getFieldType(tableName, fieldName); @@ -76,7 +75,6 @@ public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, b fieldInfoList.add(fieldInfo); } else { //处理 - System.out.println("----------"); int identifierSize = identifier.names.size(); switch(identifierSize) { diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index eadeb8264..c0d305125 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -21,7 +21,8 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.config.CalciteConfig; -import com.google.common.base.Strings; +import com.dtstack.flink.sql.util.TableUtils; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Queues; @@ -73,15 +74,14 @@ public class SideSQLParser { private static final Logger LOG = LoggerFactory.getLogger(SideSQLParser.class); private Map localTableCache = Maps.newHashMap(); - private final char SPLIT = '_'; - //regular joins(不带时间窗口) 不允许查询出rowtime或者proctime - private final String SELECT_TEMP_SQL = "select %s from %s %s"; + //用来构建临时的中间查询 + private static final String SELECT_TEMP_SQL = "select %s from %s %s"; public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { - System.out.println("---exeSql---"); + System.out.println("----------exec original Sql----------"); System.out.println(exeSql); - LOG.info("---exeSql---"); + LOG.info("----------exec original Sql----------"); LOG.info(exeSql); Queue queueInfo = Queues.newLinkedBlockingQueue(); @@ -160,7 +160,7 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue if(sqlFrom.getKind() != IDENTIFIER){ Object result = parseSql(sqlFrom, sideTableSet, queueInfo, sqlWhere, selectList); if(result instanceof JoinInfo){ - return dealSelectResultWithJoinInfo((JoinInfo) result, (SqlSelect) sqlNode, queueInfo); + return TableUtils.dealSelectResultWithJoinInfo((JoinInfo) result, (SqlSelect) sqlNode, queueInfo); }else if(result instanceof AliasInfo){ String tableName = ((AliasInfo) result).getName(); if(sideTableSet.contains(tableName)){ @@ -226,11 +226,6 @@ private AliasInfo getSqlNodeAliasInfo(SqlNode sqlNode) { private void convertSideJoinToNewQuery(SqlJoin sqlNode, Set sideTableSet) { checkAndReplaceMultiJoin(sqlNode.getLeft(), sideTableSet); checkAndReplaceMultiJoin(sqlNode.getRight(), sideTableSet); - - AliasInfo rightTableAliasInfo = getSqlNodeAliasInfo(sqlNode.getRight()); - if(sideTableSet.contains(rightTableAliasInfo.getName())){ - //构建新的查询 - } } private SqlBasicCall buildAsSqlNode(String internalTableName, SqlNode newSource) { @@ -243,6 +238,15 @@ private SqlBasicCall buildAsSqlNode(String internalTableName, SqlNode newSource) return new SqlBasicCall(operator, sqlNodes, sqlParserPos); } + /** + * 解析 join 操作 + * @param joinNode + * @param sideTableSet + * @param queueInfo + * @param parentWhere + * @param parentSelectList + * @return + */ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue queueInfo, SqlNode parentWhere, SqlNodeList parentSelectList) { SqlNode leftNode = joinNode.getLeft(); @@ -255,14 +259,17 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< String rightTableAlias = ""; boolean leftTbisTmp = false; - Tuple2 rightTableNameAndAlias = null; + //如果是连续join 判断是否已经处理过添加到执行队列 + Boolean alreadyOffer = false; + if(leftNode.getKind() == IDENTIFIER){ leftTbName = leftNode.toString(); } else if (leftNode.getKind() == JOIN) { //处理连续join - SqlBasicCall sqlBasicCall = dealNestJoin((SqlJoin) leftNode, sideTableSet, queueInfo, parentWhere, parentSelectList); - leftTbName = sqlBasicCall.getOperands()[0].toString(); - leftTbAlias = sqlBasicCall.getOperands()[1].toString(); + Tuple2 nestJoinResult = dealNestJoin((SqlJoin) leftNode, sideTableSet, queueInfo, parentWhere, parentSelectList); + alreadyOffer = nestJoinResult.f0; + leftTbName = nestJoinResult.f1.getOperands()[0].toString(); + leftTbAlias = nestJoinResult.f1.getOperands()[1].toString(); leftTbisTmp = true; } else if (leftNode.getKind() == AS) { AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo, parentWhere, parentSelectList); @@ -270,20 +277,18 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< leftTbAlias = aliasInfo.getAlias(); } else { - throw new RuntimeException("---not deal---"); + throw new RuntimeException(String.format("---not deal node with type %s", leftNode.getKind().toString())); } boolean leftIsSide = checkIsSideTable(leftTbName, sideTableSet); - if(leftIsSide){ - throw new RuntimeException("side-table must be at the right of join operator"); - } + Preconditions.checkState(!leftIsSide, "side-table must be at the right of join operator"); - rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, parentSelectList); + Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, parentSelectList); rightTableName = rightTableNameAndAlias.f0; rightTableAlias = rightTableNameAndAlias.f1; boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); - if(joinType == JoinType.RIGHT){ + if(rightIsSide && joinType == JoinType.RIGHT){ throw new RuntimeException("side join not support join type of right[current support inner join and left join]"); } @@ -303,7 +308,6 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< } tableInfo.setLeftIsTmpTable(leftTbisTmp); - tableInfo.setLeftIsSideTable(leftIsSide); tableInfo.setRightIsSideTable(rightIsSide); tableInfo.setLeftNode(leftNode); @@ -311,43 +315,15 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< tableInfo.setJoinType(joinType); tableInfo.setCondition(joinNode.getCondition()); - //TODO 抽取 - if(tableInfo.getLeftNode().getKind() != AS){ - //build 临时中间查询 - try{ - //父一级的where 条件中如果只和临时查询相关的条件都截取进来 - Set fromTableNameSet = Sets.newHashSet(); - List extractCondition = Lists.newArrayList(); - - getFromTableInfo(tableInfo.getLeftNode(), fromTableNameSet); - checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere, extractCondition); - - //TODO 查询的字段需要根据最上层的字段中获取,而不是直接设置为*,当然如果上一层就是*另说 - - List extractSelectField = extractSelectList(parentSelectList, fromTableNameSet); - String extractSelectFieldStr = buildSelectNode(extractSelectField); - String extractConditionStr = buildCondition(extractCondition); - - String tmpSelectSql = String.format(SELECT_TEMP_SQL, - extractSelectFieldStr, - tableInfo.getLeftNode().toString(), - extractConditionStr); - - SqlParser sqlParser = SqlParser.create(tmpSelectSql, CalciteConfig.MYSQL_LEX_CONFIG); - SqlNode sqlNode = sqlParser.parseStmt(); - SqlBasicCall sqlBasicCall = buildAsSqlNode(tableInfo.getLeftTableAlias(), sqlNode); - queueInfo.offer(sqlBasicCall); - - //TODO 打印合适的提示 - System.out.println(tmpSelectSql); - }catch (Exception e){ - e.printStackTrace(); - throw new RuntimeException(e); - } + if(!rightIsSide || alreadyOffer){ + return tableInfo; + } + if(tableInfo.getLeftNode().getKind() != AS){ + extractTemporaryQuery(tableInfo.getLeftNode(), tableInfo.getLeftTableAlias(), (SqlBasicCall) parentWhere, parentSelectList, queueInfo); }else { - SqlKind asFirstKind = ((SqlBasicCall)tableInfo.getLeftNode()).operands[0].getKind(); - if(asFirstKind == SELECT){ + SqlKind asNodeFirstKind = ((SqlBasicCall)tableInfo.getLeftNode()).operands[0].getKind(); + if(asNodeFirstKind == SELECT){ queueInfo.offer(tableInfo.getLeftNode()); tableInfo.setLeftNode(((SqlBasicCall)tableInfo.getLeftNode()).operands[1]); } @@ -356,23 +332,24 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< } //构建新的查询 - private SqlBasicCall dealNestJoin(SqlJoin joinNode, Set sideTableSet, Queue queueInfo, SqlNode parentWhere, SqlNodeList selectList){ + private Tuple2 dealNestJoin(SqlJoin joinNode, Set sideTableSet, Queue queueInfo, SqlNode parentWhere, SqlNodeList selectList){ SqlNode rightNode = joinNode.getRight(); - Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, selectList); - JoinInfo joinInfo = dealJoinNode(joinNode, sideTableSet, queueInfo, parentWhere, selectList); String rightTableName = rightTableNameAndAlias.f0; boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); + boolean alreadyOffer = false; + if(!rightIsSide){ //右表不是维表的情况 }else{ //右边表是维表需要重新构建左表的临时查询 queueInfo.offer(joinInfo); + alreadyOffer = true; } - return buildAsNodeByJoinInfo(joinInfo, null, null); + return Tuple2.of(alreadyOffer, TableUtils.buildAsNodeByJoinInfo(joinInfo, null, null)); } public boolean checkAndRemoveCondition(Set fromTableNameSet, SqlBasicCall parentWhere, List extractContition){ @@ -402,6 +379,40 @@ public boolean checkAndRemoveCondition(Set fromTableNameSet, SqlBasicCal } } + private void extractTemporaryQuery(SqlNode node, String tableAlias, SqlBasicCall parentWhere, + SqlNodeList parentSelectList, Queue queueInfo){ + try{ + //父一级的where 条件中如果只和临时查询相关的条件都截取进来 + Set fromTableNameSet = Sets.newHashSet(); + List extractCondition = Lists.newArrayList(); + + getFromTableInfo(node, fromTableNameSet); + checkAndRemoveCondition(fromTableNameSet, parentWhere, extractCondition); + + List extractSelectField = extractSelectList(parentSelectList, fromTableNameSet); + String extractSelectFieldStr = buildSelectNode(extractSelectField); + String extractConditionStr = buildCondition(extractCondition); + + String tmpSelectSql = String.format(SELECT_TEMP_SQL, + extractSelectFieldStr, + node.toString(), + extractConditionStr); + + SqlParser sqlParser = SqlParser.create(tmpSelectSql, CalciteConfig.MYSQL_LEX_CONFIG); + SqlNode sqlNode = sqlParser.parseStmt(); + SqlBasicCall sqlBasicCall = buildAsSqlNode(tableAlias, sqlNode); + queueInfo.offer(sqlBasicCall); + + //TODO 打印合适的提示 + System.out.println("-------build temporary query-----------"); + System.out.println(tmpSelectSql); + System.out.println("---------------------------------------"); + }catch (Exception e){ + e.printStackTrace(); + throw new RuntimeException(e); + } + } + /** * 抽取上层需用使用到的字段 * 由于where字段已经抽取到上一层了所以不用查询出来 @@ -568,71 +579,6 @@ public SqlBasicCall buildDefaultCondition(){ } - /** - * - * @param joinInfo - * @param sqlNode - * @param queueInfo - * @return 两个边关联后的新表表名 - */ - private String dealSelectResultWithJoinInfo(JoinInfo joinInfo, SqlSelect sqlNode, Queue queueInfo) { - //SideJoinInfo rename - if (joinInfo.checkIsSide()) { - joinInfo.setSelectFields(sqlNode.getSelectList()); - joinInfo.setSelectNode(sqlNode); - if (joinInfo.isRightIsSideTable()) { - //Analyzing left is not a simple table - if (joinInfo.getLeftNode().getKind() == SELECT) { - queueInfo.offer(joinInfo.getLeftNode()); - } - - queueInfo.offer(joinInfo); - } else { - //Determining right is not a simple table - if (joinInfo.getRightNode().getKind() == SELECT) { - queueInfo.offer(joinInfo.getLeftNode()); - } - - queueInfo.offer(joinInfo); - } - replaceFromNodeForJoin(joinInfo, sqlNode); - return joinInfo.getNewTableName(); - } - return ""; - } - - private void replaceFromNodeForJoin(JoinInfo joinInfo, SqlSelect sqlNode) { - //Update from node - SqlBasicCall sqlBasicCall = buildAsNodeByJoinInfo(joinInfo, null, null); - sqlNode.setFrom(sqlBasicCall); - } - - private SqlBasicCall buildAsNodeByJoinInfo(JoinInfo joinInfo, SqlNode sqlNode0, String tableAlias) { - SqlOperator operator = new SqlAsOperator(); - - SqlParserPos sqlParserPos = new SqlParserPos(0, 0); - String joinLeftTableName = joinInfo.getLeftTableName(); - String joinLeftTableAlias = joinInfo.getLeftTableAlias(); - joinLeftTableName = Strings.isNullOrEmpty(joinLeftTableName) ? joinLeftTableAlias : joinLeftTableName; - String newTableName = buildInternalTableName(joinLeftTableName, SPLIT, joinInfo.getRightTableName()); - String newTableAlias = !StringUtils.isEmpty(tableAlias) ? tableAlias : buildInternalTableName(joinInfo.getLeftTableAlias(), SPLIT, joinInfo.getRightTableAlias()); - - if (null == sqlNode0) { - sqlNode0 = new SqlIdentifier(newTableName, null, sqlParserPos); - } - - SqlIdentifier sqlIdentifierAlias = new SqlIdentifier(newTableAlias, null, sqlParserPos); - SqlNode[] sqlNodes = new SqlNode[2]; - sqlNodes[0] = sqlNode0; - sqlNodes[1] = sqlIdentifierAlias; - return new SqlBasicCall(operator, sqlNodes, sqlParserPos); - } - - private String buildInternalTableName(String left, char split, String right) { - StringBuilder sb = new StringBuilder(); - return sb.append(left).append(split).append(right).toString(); - } - private boolean checkIsSideTable(String tableName, Set sideTableList){ if(sideTableList.contains(tableName)){ return true; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 04ac98835..65e5bd13c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -140,7 +140,6 @@ public void exec(String sql, Map sideTableMap, StreamTabl tableEnv.registerTable(aliasInfo.getAlias(), table); localTableCache.put(aliasInfo.getAlias(), table); - //TODO 解析出as查询的表和字段的关系 FieldReplaceInfo fieldReplaceInfo = parseAsQuery((SqlBasicCall) pollSqlNode, tableCache); if(fieldReplaceInfo != null){ replaceInfoList.add(fieldReplaceInfo); @@ -160,8 +159,12 @@ public void exec(String sql, Map sideTableMap, StreamTabl } - //TODO - //FIXME 如果和create view 的名称命名相同 + /** + * 解析出as查询的表和字段的关系 + * @param asSqlNode + * @param tableCache + * @return + */ private FieldReplaceInfo parseAsQuery(SqlBasicCall asSqlNode, Map tableCache){ SqlNode info = asSqlNode.getOperands()[0]; SqlNode alias = asSqlNode.getOperands()[1]; @@ -172,7 +175,6 @@ private FieldReplaceInfo parseAsQuery(SqlBasicCall asSqlNode, Map } List extractFieldList = TableUtils.parserSelectField((SqlSelect) info, tableCache); - System.out.println(extractFieldList); HashBasedTable mappingTable = HashBasedTable.create(); for (FieldInfo fieldInfo : extractFieldList) { @@ -190,7 +192,12 @@ private FieldReplaceInfo parseAsQuery(SqlBasicCall asSqlNode, Map } - //TODO + /** + * 添加字段别名 + * @param pollSqlNode + * @param fieldList + * @param mappingTable + */ private void addAliasForFieldNode(SqlNode pollSqlNode, List fieldList, HashBasedTable mappingTable) { SqlKind sqlKind = pollSqlNode.getKind(); switch (sqlKind) { diff --git a/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java index 9ca76627c..911b6fba6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java @@ -20,15 +20,25 @@ package com.dtstack.flink.sql.util; import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.google.common.base.Strings; import com.google.common.collect.Lists; +import org.apache.calcite.sql.SqlAsOperator; +import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.table.api.Table; import java.util.List; import java.util.Map; +import java.util.Queue; + +import static org.apache.calcite.sql.SqlKind.SELECT; /** * @@ -39,6 +49,8 @@ public class TableUtils { + public static final char SPLIT = '_'; + /** * 获取select 的字段 * @param sqlSelect @@ -46,13 +58,13 @@ public class TableUtils { public static List parserSelectField(SqlSelect sqlSelect, Map localTableCache){ SqlNodeList sqlNodeList = sqlSelect.getSelectList(); List fieldInfoList = Lists.newArrayList(); + String fromNode = sqlSelect.getFrom().toString(); for(SqlNode fieldNode : sqlNodeList.getList()){ SqlIdentifier identifier = (SqlIdentifier)fieldNode; if(!identifier.isStar()) { - System.out.println(identifier); - String tableName = identifier.getComponent(0).getSimple(); - String fieldName = identifier.getComponent(1).getSimple(); + String tableName = identifier.names.size() == 1 ? fromNode : identifier.getComponent(0).getSimple(); + String fieldName = identifier.names.size() == 1 ? identifier.getComponent(0).getSimple() : identifier.getComponent(1).getSimple(); FieldInfo fieldInfo = new FieldInfo(); fieldInfo.setTable(tableName); fieldInfo.setFieldName(fieldName); @@ -85,4 +97,69 @@ public static List parserSelectField(SqlSelect sqlSelect, Map queueInfo) { + //SideJoinInfo rename + if (joinInfo.checkIsSide()) { + joinInfo.setSelectFields(sqlNode.getSelectList()); + joinInfo.setSelectNode(sqlNode); + if (joinInfo.isRightIsSideTable()) { + //Analyzing left is not a simple table + if (joinInfo.getLeftNode().getKind() == SELECT) { + queueInfo.offer(joinInfo.getLeftNode()); + } + + queueInfo.offer(joinInfo); + } else { + //Determining right is not a simple table + if (joinInfo.getRightNode().getKind() == SELECT) { + queueInfo.offer(joinInfo.getLeftNode()); + } + + queueInfo.offer(joinInfo); + } + replaceFromNodeForJoin(joinInfo, sqlNode); + return joinInfo.getNewTableName(); + } + return ""; + } + + public static void replaceFromNodeForJoin(JoinInfo joinInfo, SqlSelect sqlNode) { + //Update from node + SqlBasicCall sqlBasicCall = buildAsNodeByJoinInfo(joinInfo, null, null); + sqlNode.setFrom(sqlBasicCall); + } + } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java index 48afa3179..619d08529 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java @@ -56,7 +56,7 @@ public RdbAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List Date: Thu, 20 Feb 2020 11:54:38 +0800 Subject: [PATCH 07/13] =?UTF-8?q?fix=20=E4=B8=8D=E5=B8=A6where=20=E6=9D=A1?= =?UTF-8?q?=E4=BB=B6=E4=B8=8B=E7=9A=84=E7=A9=BA=E6=8C=87=E9=92=88=E9=97=AE?= =?UTF-8?q?=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/side/SideSQLParser.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index c0d305125..6ac0f7173 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -353,6 +353,11 @@ private Tuple2 dealNestJoin(SqlJoin joinNode, Set } public boolean checkAndRemoveCondition(Set fromTableNameSet, SqlBasicCall parentWhere, List extractContition){ + + if(parentWhere == null){ + return false; + } + SqlKind kind = parentWhere.getKind(); if(kind == AND){ boolean removeLeft = checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[0], extractContition); From 669e4b183640149931d3e3d3a89c0288dbfe2ae1 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Feb 2020 20:02:19 +0800 Subject: [PATCH 08/13] =?UTF-8?q?=E4=BB=8Ejoin=20on=20=E5=85=B3=E8=81=94?= =?UTF-8?q?=E7=9A=84=E6=9D=A1=E4=BB=B6=E4=B8=AD=E8=8E=B7=E5=8F=96=E5=AD=97?= =?UTF-8?q?=E6=AE=B5=E4=BF=A1=E6=81=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/side/SideSQLParser.java | 92 ++++++++++++++----- 1 file changed, 67 insertions(+), 25 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 6ac0f7173..d4f9388e8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -175,7 +175,8 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue } break; case JOIN: - return dealJoinNode((SqlJoin) sqlNode, sideTableSet, queueInfo, parentWhere, parentSelectList); + Set> joinFieldSet = Sets.newHashSet(); + return dealJoinNode((SqlJoin) sqlNode, sideTableSet, queueInfo, parentWhere, parentSelectList, joinFieldSet); case AS: SqlNode info = ((SqlBasicCall)sqlNode).getOperands()[0]; SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; @@ -248,7 +249,7 @@ private SqlBasicCall buildAsSqlNode(String internalTableName, SqlNode newSource) * @return */ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue queueInfo, - SqlNode parentWhere, SqlNodeList parentSelectList) { + SqlNode parentWhere, SqlNodeList parentSelectList, Set> joinFieldSet) { SqlNode leftNode = joinNode.getLeft(); SqlNode rightNode = joinNode.getRight(); JoinType joinType = joinNode.getJoinType(); @@ -261,12 +262,14 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< //如果是连续join 判断是否已经处理过添加到执行队列 Boolean alreadyOffer = false; + extractJoinField(joinNode.getCondition(), joinFieldSet); if(leftNode.getKind() == IDENTIFIER){ leftTbName = leftNode.toString(); } else if (leftNode.getKind() == JOIN) { //处理连续join - Tuple2 nestJoinResult = dealNestJoin((SqlJoin) leftNode, sideTableSet, queueInfo, parentWhere, parentSelectList); + Tuple2 nestJoinResult = dealNestJoin((SqlJoin) leftNode, sideTableSet, + queueInfo, parentWhere, parentSelectList, joinFieldSet); alreadyOffer = nestJoinResult.f0; leftTbName = nestJoinResult.f1.getOperands()[0].toString(); leftTbAlias = nestJoinResult.f1.getOperands()[1].toString(); @@ -320,7 +323,8 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< } if(tableInfo.getLeftNode().getKind() != AS){ - extractTemporaryQuery(tableInfo.getLeftNode(), tableInfo.getLeftTableAlias(), (SqlBasicCall) parentWhere, parentSelectList, queueInfo); + extractTemporaryQuery(tableInfo.getLeftNode(), tableInfo.getLeftTableAlias(), (SqlBasicCall) parentWhere, + parentSelectList, queueInfo, joinFieldSet); }else { SqlKind asNodeFirstKind = ((SqlBasicCall)tableInfo.getLeftNode()).operands[0].getKind(); if(asNodeFirstKind == SELECT){ @@ -331,11 +335,14 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< return tableInfo; } + //构建新的查询 - private Tuple2 dealNestJoin(SqlJoin joinNode, Set sideTableSet, Queue queueInfo, SqlNode parentWhere, SqlNodeList selectList){ + private Tuple2 dealNestJoin(SqlJoin joinNode, Set sideTableSet, + Queue queueInfo, SqlNode parentWhere, + SqlNodeList selectList, Set> joinFieldSet){ SqlNode rightNode = joinNode.getRight(); Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, selectList); - JoinInfo joinInfo = dealJoinNode(joinNode, sideTableSet, queueInfo, parentWhere, selectList); + JoinInfo joinInfo = dealJoinNode(joinNode, sideTableSet, queueInfo, parentWhere, selectList, joinFieldSet); String rightTableName = rightTableNameAndAlias.f0; boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); @@ -352,7 +359,7 @@ private Tuple2 dealNestJoin(SqlJoin joinNode, Set return Tuple2.of(alreadyOffer, TableUtils.buildAsNodeByJoinInfo(joinInfo, null, null)); } - public boolean checkAndRemoveCondition(Set fromTableNameSet, SqlBasicCall parentWhere, List extractContition){ + public boolean checkAndRemoveCondition(Set fromTableNameSet, SqlBasicCall parentWhere, List extractCondition){ if(parentWhere == null){ return false; @@ -360,15 +367,15 @@ public boolean checkAndRemoveCondition(Set fromTableNameSet, SqlBasicCal SqlKind kind = parentWhere.getKind(); if(kind == AND){ - boolean removeLeft = checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[0], extractContition); - boolean removeRight = checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[1], extractContition); + boolean removeLeft = checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[0], extractCondition); + boolean removeRight = checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[1], extractCondition); //DO remove if(removeLeft){ - extractContition.add(removeWhereConditionNode(parentWhere, 0)); + extractCondition.add(removeWhereConditionNode(parentWhere, 0)); } if(removeRight){ - extractContition.add(removeWhereConditionNode(parentWhere, 1)); + extractCondition.add(removeWhereConditionNode(parentWhere, 1)); } return false; @@ -385,7 +392,8 @@ public boolean checkAndRemoveCondition(Set fromTableNameSet, SqlBasicCal } private void extractTemporaryQuery(SqlNode node, String tableAlias, SqlBasicCall parentWhere, - SqlNodeList parentSelectList, Queue queueInfo){ + SqlNodeList parentSelectList, Queue queueInfo, + Set> joinFieldSet){ try{ //父一级的where 条件中如果只和临时查询相关的条件都截取进来 Set fromTableNameSet = Sets.newHashSet(); @@ -394,8 +402,9 @@ private void extractTemporaryQuery(SqlNode node, String tableAlias, SqlBasicCall getFromTableInfo(node, fromTableNameSet); checkAndRemoveCondition(fromTableNameSet, parentWhere, extractCondition); - List extractSelectField = extractSelectList(parentSelectList, fromTableNameSet); - String extractSelectFieldStr = buildSelectNode(extractSelectField); + Set extractSelectField = extractSelectFields(parentSelectList, fromTableNameSet); + Set fieldFromJoinCondition = extractSelectFieldFromJoinCondition(joinFieldSet, fromTableNameSet); + String extractSelectFieldStr = buildSelectNode(extractSelectField, fieldFromJoinCondition); String extractConditionStr = buildCondition(extractCondition); String tmpSelectSql = String.format(SELECT_TEMP_SQL, @@ -425,8 +434,8 @@ private void extractTemporaryQuery(SqlNode node, String tableAlias, SqlBasicCall * @param fromTableNameSet * @return */ - private List extractSelectList(SqlNodeList parentSelectList, Set fromTableNameSet){ - List extractFieldList = Lists.newArrayList(); + private Set extractSelectFields(SqlNodeList parentSelectList, Set fromTableNameSet){ + Set extractFieldList = Sets.newHashSet(); for(SqlNode selectNode : parentSelectList.getList()){ extractSelectField(selectNode, extractFieldList, fromTableNameSet); } @@ -434,10 +443,41 @@ private List extractSelectList(SqlNodeList parentSelectList, Set return extractFieldList; } - private void extractSelectField(SqlNode selectNode, List extractFieldList, Set fromTableNameSet){ + private Set extractSelectFieldFromJoinCondition(Set> joinFieldSet, Set fromTableNameSet){ + Set extractFieldList = Sets.newHashSet(); + for(Tuple2 field : joinFieldSet){ + if(fromTableNameSet.contains(field.f0)){ + extractFieldList.add(field.f0 + "." + field.f1); + } + } + + return extractFieldList; + } + + /** + * 从join的条件中获取字段信息 + * @param condition + * @param joinFieldSet + */ + private void extractJoinField(SqlNode condition, Set> joinFieldSet){ + SqlKind joinKind = condition.getKind(); + if( joinKind == AND ){ + extractJoinField(((SqlBasicCall)condition).operands[0], joinFieldSet); + extractJoinField(((SqlBasicCall)condition).operands[1], joinFieldSet); + }else if( joinKind == EQUALS ){ + extractJoinField(((SqlBasicCall)condition).operands[0], joinFieldSet); + extractJoinField(((SqlBasicCall)condition).operands[1], joinFieldSet); + }else{ + Preconditions.checkState(((SqlIdentifier)condition).names.size() == 2, "join condition must be format table.field"); + Tuple2 tuple2 = Tuple2.of(((SqlIdentifier)condition).names.get(0), ((SqlIdentifier)condition).names.get(1)); + joinFieldSet.add(tuple2); + } + } + + private void extractSelectField(SqlNode selectNode, Set extractFieldSet, Set fromTableNameSet){ if (selectNode.getKind() == AS) { SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; - extractSelectField(leftNode, extractFieldList, fromTableNameSet); + extractSelectField(leftNode, extractFieldSet, fromTableNameSet); }else if(selectNode.getKind() == IDENTIFIER) { SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; @@ -448,7 +488,7 @@ private void extractSelectField(SqlNode selectNode, List extractFieldLis String tableName = sqlIdentifier.names.get(0); if(fromTableNameSet.contains(tableName)){ - extractFieldList.add(sqlIdentifier.toString()); + extractFieldSet.add(sqlIdentifier.toString()); } }else if( AGGREGATE.contains(selectNode.getKind()) @@ -493,7 +533,7 @@ private void extractSelectField(SqlNode selectNode, List extractFieldLis continue; } - extractSelectField(sqlNode, extractFieldList, fromTableNameSet); + extractSelectField(sqlNode, extractFieldSet, fromTableNameSet); } }else if(selectNode.getKind() == CASE){ @@ -505,15 +545,15 @@ private void extractSelectField(SqlNode selectNode, List extractFieldLis for(int i=0; i conditionList){ return " where " + StringUtils.join(conditionList, " AND "); } - public String buildSelectNode(List extractSelectField){ + public String buildSelectNode(Set extractSelectField, Set joinFieldSet){ if(CollectionUtils.isEmpty(extractSelectField)){ throw new RuntimeException("no field is used"); } - return StringUtils.join(extractSelectField, ","); + Sets.SetView view = Sets.union(extractSelectField, joinFieldSet); + + return StringUtils.join(view, ","); } public SqlBasicCall buildDefaultCondition(){ From 6533361e24b2da99ee6ecfa96db91cc56d4a5cb7 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 21 Feb 2020 10:00:13 +0800 Subject: [PATCH 09/13] =?UTF-8?q?=E5=90=88=E5=B9=B6=E9=87=8D=E5=A4=8D?= =?UTF-8?q?=E7=9A=84=E6=9D=A1=E4=BB=B6=E4=BB=A3=E7=A0=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/side/SideSQLParser.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index d4f9388e8..604a34a82 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -461,10 +461,7 @@ private Set extractSelectFieldFromJoinCondition(Set> joinFieldSet){ SqlKind joinKind = condition.getKind(); - if( joinKind == AND ){ - extractJoinField(((SqlBasicCall)condition).operands[0], joinFieldSet); - extractJoinField(((SqlBasicCall)condition).operands[1], joinFieldSet); - }else if( joinKind == EQUALS ){ + if( joinKind == AND || joinKind == EQUALS ){ extractJoinField(((SqlBasicCall)condition).operands[0], joinFieldSet); extractJoinField(((SqlBasicCall)condition).operands[1], joinFieldSet); }else{ From d9f48d3ba7b47e7fc39a28ce70db4fb2a8bebde9 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 21 Feb 2020 10:04:48 +0800 Subject: [PATCH 10/13] =?UTF-8?q?=E5=8E=BB=E9=99=A4=E6=B3=A8=E9=87=8A?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 604a34a82..c12bc981e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -417,7 +417,6 @@ private void extractTemporaryQuery(SqlNode node, String tableAlias, SqlBasicCall SqlBasicCall sqlBasicCall = buildAsSqlNode(tableAlias, sqlNode); queueInfo.offer(sqlBasicCall); - //TODO 打印合适的提示 System.out.println("-------build temporary query-----------"); System.out.println(tmpSelectSql); System.out.println("---------------------------------------"); From 8ab5fb06f17a874d2c4c4a88cd3e13d890d28e16 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 21 Feb 2020 15:06:46 +0800 Subject: [PATCH 11/13] =?UTF-8?q?=E5=8E=BB=E6=8E=89=E9=87=8D=E5=A4=8D?= =?UTF-8?q?=E7=9A=84=E5=AD=97=E6=AE=B5=E6=B3=A8=E5=86=8C?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 65e5bd13c..566fc7dc4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -865,7 +865,7 @@ private void joinFun(Object pollObj, Map localTableCache, replaceInfoList.add(replaceInfo); if (!tableEnv.isRegistered(joinInfo.getNewTableName())){ - Table joinTable = tableEnv.fromDataStream(dsOut, String.join(",", sideOutTypeInfo.getFieldNames())); + Table joinTable = tableEnv.fromDataStream(dsOut); tableEnv.registerTable(joinInfo.getNewTableName(), joinTable); localTableCache.put(joinInfo.getNewTableName(), joinTable); } From 6a9cc95e7c20cff46ef072d65315919630595129 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 27 Feb 2020 21:53:57 +0800 Subject: [PATCH 12/13] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E5=A4=9A=E7=BB=B4?= =?UTF-8?q?=E8=A1=A8join=20=E9=83=A8=E5=88=86=E8=A1=A8=E6=98=8E=E6=9C=AA?= =?UTF-8?q?=E8=A2=AB=E6=9B=BF=E6=8D=A2=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/side/JoinInfo.java | 19 + .../flink/sql/side/JoinNodeDealer.java | 583 ++++++++++++++++++ .../dtstack/flink/sql/side/SideSQLParser.java | 496 +-------------- .../dtstack/flink/sql/side/SideSqlExec.java | 20 +- .../dtstack/flink/sql/util/TableUtils.java | 325 +++++++++- 5 files changed, 948 insertions(+), 495 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/JoinNodeDealer.java diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java index 469e919b3..210fb5b6a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -202,4 +202,23 @@ public boolean isLeftIsTmpTable() { public void setLeftIsTmpTable(boolean leftIsTmpTable) { this.leftIsTmpTable = leftIsTmpTable; } + + @Override + public String toString() { + return "JoinInfo{" + + "leftIsSideTable=" + leftIsSideTable + + ", leftIsTmpTable=" + leftIsTmpTable + + ", rightIsSideTable=" + rightIsSideTable + + ", leftTableName='" + leftTableName + '\'' + + ", leftTableAlias='" + leftTableAlias + '\'' + + ", rightTableName='" + rightTableName + '\'' + + ", rightTableAlias='" + rightTableAlias + '\'' + + ", leftNode=" + leftNode + + ", rightNode=" + rightNode + + ", condition=" + condition + + ", selectFields=" + selectFields + + ", selectNode=" + selectNode + + ", joinType=" + joinType + + '}'; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinNodeDealer.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinNodeDealer.java new file mode 100644 index 000000000..a1c3cecb1 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinNodeDealer.java @@ -0,0 +1,583 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flink.sql.side; + +import com.dtstack.flink.sql.config.CalciteConfig; +import com.dtstack.flink.sql.util.TableUtils; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.calcite.sql.JoinType; +import org.apache.calcite.sql.SqlAsOperator; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlBinaryOperator; +import org.apache.calcite.sql.SqlDataTypeSpec; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlLiteral; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.fun.SqlCase; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.parser.SqlParser; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.java.tuple.Tuple2; + +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; + +import static org.apache.calcite.sql.SqlKind.*; + +/** + * 处理join 节点 + * 1:如果包含维表节点替换为临时查询 + * Date: 2020/2/27 + * Company: www.dtstack.com + * @author xuchao + */ + +public class JoinNodeDealer { + + //用来构建临时的中间查询 + private static final String SELECT_TEMP_SQL = "select %s from %s %s"; + + private SideSQLParser sideSQLParser; + + public JoinNodeDealer(SideSQLParser sideSQLParser){ + this.sideSQLParser = sideSQLParser; + } + + /** + * 解析 join 操作 + * @param joinNode + * @param sideTableSet 标明哪些表名是维表 + * @param queueInfo + * @param parentWhere join 关联的最上层的where 节点 + * @param parentSelectList join 关联的最上层的select 节点 + * @param joinFieldSet + * @param tableRef 存储构建临时表查询后源表和新表之间的关联关系 + * @return + */ + public JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, + Queue queueInfo, SqlNode parentWhere, + SqlNodeList parentSelectList, Set> joinFieldSet, + Map tableRef) { + SqlNode leftNode = joinNode.getLeft(); + SqlNode rightNode = joinNode.getRight(); + JoinType joinType = joinNode.getJoinType(); + + String leftTbName = ""; + String leftTbAlias = ""; + String rightTableName = ""; + String rightTableAlias = ""; + boolean leftTbisTmp = false; + + //如果是连续join 判断是否已经处理过添加到执行队列 + Boolean needBuildTemp = false; + extractJoinField(joinNode.getCondition(), joinFieldSet); + + if(leftNode.getKind() == IDENTIFIER){ + leftTbName = leftNode.toString(); + } else if (leftNode.getKind() == JOIN) { + //处理连续join + Tuple2 nestJoinResult = dealNestJoin((SqlJoin) leftNode, sideTableSet, + queueInfo, parentWhere, parentSelectList, joinFieldSet, tableRef); + needBuildTemp = nestJoinResult.f0; + SqlBasicCall buildAs = TableUtils.buildAsNodeByJoinInfo(nestJoinResult.f1, null, null); + + if(needBuildTemp){ + //记录表之间的关联关系 + String newLeftTableName = buildAs.getOperands()[1].toString(); + Set fromTableNameSet = Sets.newHashSet(); + TableUtils.getFromTableInfo(joinNode.getLeft(), fromTableNameSet); + for(String tbTmp : fromTableNameSet){ + tableRef.put(tbTmp, newLeftTableName); + } + + //替换leftNode 为新的查询 + joinNode.setLeft(buildAs); + leftNode = buildAs; + + //替换select field 中的对应字段 + for(SqlNode sqlNode : parentSelectList.getList()){ + for(String tbTmp : fromTableNameSet) { + TableUtils.replaceSelectFieldTable(sqlNode, tbTmp, newLeftTableName); + } + } + + //替换where 中的条件相关 + for(String tbTmp : fromTableNameSet){ + TableUtils.replaceWhereCondition(parentWhere, tbTmp, newLeftTableName); + } + + leftTbisTmp = true; + + } + + leftTbName = buildAs.getOperands()[0].toString(); + leftTbAlias = buildAs.getOperands()[1].toString(); + + } else if (leftNode.getKind() == AS) { + AliasInfo aliasInfo = (AliasInfo) sideSQLParser.parseSql(leftNode, sideTableSet, queueInfo, parentWhere, parentSelectList); + leftTbName = aliasInfo.getName(); + leftTbAlias = aliasInfo.getAlias(); + + } else { + throw new RuntimeException(String.format("---not deal node with type %s", leftNode.getKind().toString())); + } + + boolean leftIsSide = checkIsSideTable(leftTbName, sideTableSet); + Preconditions.checkState(!leftIsSide, "side-table must be at the right of join operator"); + + Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, parentSelectList); + rightTableName = rightTableNameAndAlias.f0; + rightTableAlias = rightTableNameAndAlias.f1; + + boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); + if(rightIsSide && joinType == JoinType.RIGHT){ + throw new RuntimeException("side join not support join type of right[current support inner join and left join]"); + } + + if(leftNode.getKind() == JOIN && rightIsSide){ + needBuildTemp = true; + } + + JoinInfo tableInfo = new JoinInfo(); + tableInfo.setLeftTableName(leftTbName); + tableInfo.setRightTableName(rightTableName); + if (StringUtils.isEmpty(leftTbAlias)){ + tableInfo.setLeftTableAlias(leftTbName); + } else { + tableInfo.setLeftTableAlias(leftTbAlias); + } + + if (StringUtils.isEmpty(rightTableAlias)){ + tableInfo.setRightTableAlias(rightTableName); + } else { + tableInfo.setRightTableAlias(rightTableAlias); + } + + TableUtils.replaceJoinFieldRefTableName(joinNode.getCondition(), tableRef); + + tableInfo.setLeftIsTmpTable(leftTbisTmp); + tableInfo.setLeftIsSideTable(leftIsSide); + tableInfo.setRightIsSideTable(rightIsSide); + tableInfo.setLeftNode(leftNode); + tableInfo.setRightNode(rightNode); + tableInfo.setJoinType(joinType); + tableInfo.setCondition(joinNode.getCondition()); + + if(!needBuildTemp){ + return tableInfo; + } + + if(tableInfo.getLeftNode().getKind() != AS){ + extractTemporaryQuery(tableInfo.getLeftNode(), tableInfo.getLeftTableAlias(), (SqlBasicCall) parentWhere, + parentSelectList, queueInfo, joinFieldSet, tableRef); + }else { + SqlKind asNodeFirstKind = ((SqlBasicCall)tableInfo.getLeftNode()).operands[0].getKind(); + if(asNodeFirstKind == SELECT){ + queueInfo.offer(tableInfo.getLeftNode()); + tableInfo.setLeftNode(((SqlBasicCall)tableInfo.getLeftNode()).operands[1]); + } + } + return tableInfo; + } + + + //处理多层join + private Tuple2 dealNestJoin(SqlJoin joinNode, Set sideTableSet, + Queue queueInfo, SqlNode parentWhere, + SqlNodeList selectList, Set> joinFieldSet, + Map tableRef){ + SqlNode rightNode = joinNode.getRight(); + Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, selectList); + JoinInfo joinInfo = dealJoinNode(joinNode, sideTableSet, queueInfo, parentWhere, selectList, joinFieldSet, tableRef); + + String rightTableName = rightTableNameAndAlias.f0; + boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); + boolean needBuildTemp = false; + + if(!rightIsSide){ + //右表不是维表的情况 + }else{ + //右边表是维表需要重新构建左表的临时查询 + queueInfo.offer(joinInfo); + needBuildTemp = true; + } + + //return Tuple2.of(needBuildTemp, TableUtils.buildAsNodeByJoinInfo(joinInfo, null, null)); + return Tuple2.of(needBuildTemp, joinInfo); + } + + private void extractTemporaryQuery(SqlNode node, String tableAlias, SqlBasicCall parentWhere, + SqlNodeList parentSelectList, Queue queueInfo, + Set> joinFieldSet, + Map tableRef){ + try{ + //父一级的where 条件中如果只和临时查询相关的条件都截取进来 + Set fromTableNameSet = Sets.newHashSet(); + List extractCondition = Lists.newArrayList(); + + TableUtils.getFromTableInfo(node, fromTableNameSet); + checkAndRemoveWhereCondition(fromTableNameSet, parentWhere, extractCondition); + + if(node.getKind() == JOIN){ + checkAndReplaceJoinCondition(((SqlJoin)node).getCondition(), tableRef); + } + + Set extractSelectField = extractSelectFields(parentSelectList, fromTableNameSet, tableRef); + Set fieldFromJoinCondition = extractSelectFieldFromJoinCondition(joinFieldSet, fromTableNameSet); + String extractSelectFieldStr = buildSelectNode(extractSelectField, fieldFromJoinCondition); + String extractConditionStr = buildCondition(extractCondition); + + String tmpSelectSql = String.format(SELECT_TEMP_SQL, + extractSelectFieldStr, + node.toString(), + extractConditionStr); + + SqlParser sqlParser = SqlParser.create(tmpSelectSql, CalciteConfig.MYSQL_LEX_CONFIG); + SqlNode sqlNode = sqlParser.parseStmt(); + SqlBasicCall sqlBasicCall = buildAsSqlNode(tableAlias, sqlNode); + queueInfo.offer(sqlBasicCall); + + System.out.println("-------build temporary query-----------"); + System.out.println(tmpSelectSql); + System.out.println("---------------------------------------"); + }catch (Exception e){ + e.printStackTrace(); + throw new RuntimeException(e); + } + } + + /** + * 抽取上层需用使用到的字段 + * 由于where字段已经抽取到上一层了所以不用查询出来 + * @param parentSelectList + * @param fromTableNameSet + * @return + */ + private Set extractSelectFields(SqlNodeList parentSelectList, + Set fromTableNameSet, + Map tableRef){ + Set extractFieldList = Sets.newHashSet(); + for(SqlNode selectNode : parentSelectList.getList()){ + extractSelectField(selectNode, extractFieldList, fromTableNameSet, tableRef); + } + + return extractFieldList; + } + + private Set extractSelectFieldFromJoinCondition(Set> joinFieldSet, Set fromTableNameSet){ + Set extractFieldList = Sets.newHashSet(); + for(Tuple2 field : joinFieldSet){ + if(fromTableNameSet.contains(field.f0)){ + extractFieldList.add(field.f0 + "." + field.f1); + } + } + + return extractFieldList; + } + + /** + * 从join的条件中获取字段信息 + * @param condition + * @param joinFieldSet + */ + private void extractJoinField(SqlNode condition, Set> joinFieldSet){ + SqlKind joinKind = condition.getKind(); + if( joinKind == AND || joinKind == EQUALS ){ + extractJoinField(((SqlBasicCall)condition).operands[0], joinFieldSet); + extractJoinField(((SqlBasicCall)condition).operands[1], joinFieldSet); + }else{ + Preconditions.checkState(((SqlIdentifier)condition).names.size() == 2, "join condition must be format table.field"); + Tuple2 tuple2 = Tuple2.of(((SqlIdentifier)condition).names.get(0), ((SqlIdentifier)condition).names.get(1)); + joinFieldSet.add(tuple2); + } + } + + + private void extractSelectField(SqlNode selectNode, + Set extractFieldSet, + Set fromTableNameSet, + Map tableRef){ + if (selectNode.getKind() == AS) { + SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; + extractSelectField(leftNode, extractFieldSet, fromTableNameSet, tableRef); + + }else if(selectNode.getKind() == IDENTIFIER) { + SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; + + if(sqlIdentifier.names.size() == 1){ + return; + } + + String tableName = sqlIdentifier.names.get(0); + //TODO + if(fromTableNameSet.contains(tableName)){ + extractFieldSet.add(sqlIdentifier.toString()); + } else if(fromTableNameSet.contains(tableRef.get(tableName))){ + //TODO extractFieldSet.add(sqlIdentifier.setName(0, tableRef.get(tableName)).toString()); + } + + }else if( AGGREGATE.contains(selectNode.getKind()) + || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) + || COMPARISON.contains(selectNode.getKind()) + || selectNode.getKind() == OTHER_FUNCTION + || selectNode.getKind() == DIVIDE + || selectNode.getKind() == CAST + || selectNode.getKind() == TRIM + || selectNode.getKind() == TIMES + || selectNode.getKind() == PLUS + || selectNode.getKind() == NOT_IN + || selectNode.getKind() == OR + || selectNode.getKind() == AND + || selectNode.getKind() == MINUS + || selectNode.getKind() == TUMBLE + || selectNode.getKind() == TUMBLE_START + || selectNode.getKind() == TUMBLE_END + || selectNode.getKind() == SESSION + || selectNode.getKind() == SESSION_START + || selectNode.getKind() == SESSION_END + || selectNode.getKind() == HOP + || selectNode.getKind() == HOP_START + || selectNode.getKind() == HOP_END + || selectNode.getKind() == BETWEEN + || selectNode.getKind() == IS_NULL + || selectNode.getKind() == IS_NOT_NULL + || selectNode.getKind() == CONTAINS + || selectNode.getKind() == TIMESTAMP_ADD + || selectNode.getKind() == TIMESTAMP_DIFF + || selectNode.getKind() == LIKE + + ){ + SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; + for(int i=0; i parseRightNode(SqlNode sqlNode, Set sideTableSet, Queue queueInfo, + SqlNode parentWhere, SqlNodeList selectList) { + Tuple2 tabName = new Tuple2<>("", ""); + if(sqlNode.getKind() == IDENTIFIER){ + tabName.f0 = sqlNode.toString(); + }else{ + AliasInfo aliasInfo = (AliasInfo)sideSQLParser.parseSql(sqlNode, sideTableSet, queueInfo, parentWhere, selectList); + tabName.f0 = aliasInfo.getName(); + tabName.f1 = aliasInfo.getAlias(); + } + return tabName; + } + + private Tuple2 parseLeftNode(SqlNode sqlNode){ + Tuple2 tabName = new Tuple2<>("", ""); + if(sqlNode.getKind() == IDENTIFIER){ + tabName.f0 = sqlNode.toString(); + tabName.f1 = sqlNode.toString(); + }else if (sqlNode.getKind() == AS){ + SqlNode info = ((SqlBasicCall)sqlNode).getOperands()[0]; + SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; + + tabName.f0 = info.toString(); + tabName.f1 = alias.toString(); + }else { + throw new RuntimeException(""); + } + + return tabName; + } + + public String buildCondition(List conditionList){ + if(CollectionUtils.isEmpty(conditionList)){ + return ""; + } + + return " where " + StringUtils.join(conditionList, " AND "); + } + + public String buildSelectNode(Set extractSelectField, Set joinFieldSet){ + if(CollectionUtils.isEmpty(extractSelectField)){ + throw new RuntimeException("no field is used"); + } + + Sets.SetView view = Sets.union(extractSelectField, joinFieldSet); + + return StringUtils.join(view, ","); + } + + private boolean checkIsSideTable(String tableName, Set sideTableList){ + if(sideTableList.contains(tableName)){ + return true; + } + return false; + } + + private SqlBasicCall buildAsSqlNode(String internalTableName, SqlNode newSource) { + SqlOperator operator = new SqlAsOperator(); + SqlParserPos sqlParserPos = new SqlParserPos(0, 0); + SqlIdentifier sqlIdentifierAlias = new SqlIdentifier(internalTableName, null, sqlParserPos); + SqlNode[] sqlNodes = new SqlNode[2]; + sqlNodes[0] = newSource; + sqlNodes[1] = sqlIdentifierAlias; + return new SqlBasicCall(operator, sqlNodes, sqlParserPos); + } + + + /** + * 检查关联的where 条件中的判断是否可以下移到新构建的子查询 + * @param fromTableNameSet + * @param parentWhere + * @param extractCondition + * @return + */ + private boolean checkAndRemoveWhereCondition(Set fromTableNameSet, + SqlBasicCall parentWhere, + List extractCondition){ + if(parentWhere == null){ + return false; + } + + SqlKind kind = parentWhere.getKind(); + if(kind == AND){ + boolean removeLeft = checkAndRemoveWhereCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[0], extractCondition); + boolean removeRight = checkAndRemoveWhereCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[1], extractCondition); + //DO remove + if(removeLeft){ + extractCondition.add(removeWhereConditionNode(parentWhere, 0)); + } + + if(removeRight){ + extractCondition.add(removeWhereConditionNode(parentWhere, 1)); + } + + return false; + } else { + Set conditionRefTableNameSet = Sets.newHashSet(); + TableUtils.getConditionRefTable(parentWhere, conditionRefTableNameSet); + + if(fromTableNameSet.containsAll(conditionRefTableNameSet)){ + return true; + } + + return false; + } + } + + /** + * 抽取where 条件中指定的条件 + * @param parentWhere + * @param index + * @return + */ + public SqlBasicCall removeWhereConditionNode(SqlBasicCall parentWhere, int index){ + SqlBasicCall oldCondition = (SqlBasicCall) parentWhere.getOperands()[index]; + parentWhere.setOperand(index, buildEmptyCondition()); + return oldCondition; + } + + /** + * 构建 1=1的 where 条件 + * @return + */ + public SqlBasicCall buildEmptyCondition(){ + SqlBinaryOperator equalsOperators = SqlStdOperatorTable.EQUALS; + SqlNode[] operands = new SqlNode[2]; + operands[0] = SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO); + operands[1] = SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO); + + return new SqlBasicCall(equalsOperators, operands, SqlParserPos.ZERO); + } + + /** + * 替换join 条件中的表名称 + * @param node + * @param tableMap 表名的关联关系 + */ + private SqlIdentifier checkAndReplaceJoinCondition(SqlNode node, Map tableMap){ + + SqlKind joinKind = node.getKind(); + if( joinKind == AND || joinKind == EQUALS ){ + SqlIdentifier leftNode = checkAndReplaceJoinCondition(((SqlBasicCall)node).operands[0], tableMap); + SqlIdentifier rightNode = checkAndReplaceJoinCondition(((SqlBasicCall)node).operands[1], tableMap); + + if(leftNode != null){ + ((SqlBasicCall)node).setOperand(0, leftNode); + } + + if(rightNode != null){ + ((SqlBasicCall)node).setOperand(1, leftNode); + } + + return null; + } else { + //replace table + Preconditions.checkState(((SqlIdentifier)node).names.size() == 2, "join condition must be format table.field"); + String tbName = ((SqlIdentifier) node).names.get(0); + if(tableMap.containsKey(tbName)){ + tbName = tableMap.get(tbName); + return ((SqlIdentifier) node).setName(0, tbName); + } + + return null; + } + } + + + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index c12bc981e..061fe52a2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -75,9 +75,6 @@ public class SideSQLParser { private Map localTableCache = Maps.newHashMap(); - //用来构建临时的中间查询 - private static final String SELECT_TEMP_SQL = "select %s from %s %s"; - public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { System.out.println("----------exec original Sql----------"); System.out.println(exeSql); @@ -135,7 +132,7 @@ private void checkAndReplaceMultiJoin(SqlNode sqlNode, Set sideTableSet) } - private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue queueInfo, SqlNode parentWhere, SqlNodeList parentSelectList){ + public Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue queueInfo, SqlNode parentWhere, SqlNodeList parentSelectList){ SqlKind sqlKind = sqlNode.getKind(); switch (sqlKind){ case WITH: { @@ -175,8 +172,10 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue } break; case JOIN: + JoinNodeDealer joinNodeDealer = new JoinNodeDealer(this); Set> joinFieldSet = Sets.newHashSet(); - return dealJoinNode((SqlJoin) sqlNode, sideTableSet, queueInfo, parentWhere, parentSelectList, joinFieldSet); + Map tableRef = Maps.newHashMap(); + return joinNodeDealer.dealJoinNode((SqlJoin) sqlNode, sideTableSet, queueInfo, parentWhere, parentSelectList, joinFieldSet, tableRef); case AS: SqlNode info = ((SqlBasicCall)sqlNode).getOperands()[0]; SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; @@ -229,496 +228,9 @@ private void convertSideJoinToNewQuery(SqlJoin sqlNode, Set sideTableSet checkAndReplaceMultiJoin(sqlNode.getRight(), sideTableSet); } - private SqlBasicCall buildAsSqlNode(String internalTableName, SqlNode newSource) { - SqlOperator operator = new SqlAsOperator(); - SqlParserPos sqlParserPos = new SqlParserPos(0, 0); - SqlIdentifier sqlIdentifierAlias = new SqlIdentifier(internalTableName, null, sqlParserPos); - SqlNode[] sqlNodes = new SqlNode[2]; - sqlNodes[0] = newSource; - sqlNodes[1] = sqlIdentifierAlias; - return new SqlBasicCall(operator, sqlNodes, sqlParserPos); - } - - /** - * 解析 join 操作 - * @param joinNode - * @param sideTableSet - * @param queueInfo - * @param parentWhere - * @param parentSelectList - * @return - */ - private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue queueInfo, - SqlNode parentWhere, SqlNodeList parentSelectList, Set> joinFieldSet) { - SqlNode leftNode = joinNode.getLeft(); - SqlNode rightNode = joinNode.getRight(); - JoinType joinType = joinNode.getJoinType(); - - String leftTbName = ""; - String leftTbAlias = ""; - String rightTableName = ""; - String rightTableAlias = ""; - boolean leftTbisTmp = false; - - //如果是连续join 判断是否已经处理过添加到执行队列 - Boolean alreadyOffer = false; - extractJoinField(joinNode.getCondition(), joinFieldSet); - - if(leftNode.getKind() == IDENTIFIER){ - leftTbName = leftNode.toString(); - } else if (leftNode.getKind() == JOIN) { - //处理连续join - Tuple2 nestJoinResult = dealNestJoin((SqlJoin) leftNode, sideTableSet, - queueInfo, parentWhere, parentSelectList, joinFieldSet); - alreadyOffer = nestJoinResult.f0; - leftTbName = nestJoinResult.f1.getOperands()[0].toString(); - leftTbAlias = nestJoinResult.f1.getOperands()[1].toString(); - leftTbisTmp = true; - } else if (leftNode.getKind() == AS) { - AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo, parentWhere, parentSelectList); - leftTbName = aliasInfo.getName(); - leftTbAlias = aliasInfo.getAlias(); - - } else { - throw new RuntimeException(String.format("---not deal node with type %s", leftNode.getKind().toString())); - } - - boolean leftIsSide = checkIsSideTable(leftTbName, sideTableSet); - Preconditions.checkState(!leftIsSide, "side-table must be at the right of join operator"); - - Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, parentSelectList); - rightTableName = rightTableNameAndAlias.f0; - rightTableAlias = rightTableNameAndAlias.f1; - - boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); - if(rightIsSide && joinType == JoinType.RIGHT){ - throw new RuntimeException("side join not support join type of right[current support inner join and left join]"); - } - - JoinInfo tableInfo = new JoinInfo(); - tableInfo.setLeftTableName(leftTbName); - tableInfo.setRightTableName(rightTableName); - if (StringUtils.isEmpty(leftTbAlias)){ - tableInfo.setLeftTableAlias(leftTbName); - } else { - tableInfo.setLeftTableAlias(leftTbAlias); - } - - if (StringUtils.isEmpty(rightTableAlias)){ - tableInfo.setRightTableAlias(rightTableName); - } else { - tableInfo.setRightTableAlias(rightTableAlias); - } - - tableInfo.setLeftIsTmpTable(leftTbisTmp); - tableInfo.setLeftIsSideTable(leftIsSide); - tableInfo.setRightIsSideTable(rightIsSide); - tableInfo.setLeftNode(leftNode); - tableInfo.setRightNode(rightNode); - tableInfo.setJoinType(joinType); - tableInfo.setCondition(joinNode.getCondition()); - - if(!rightIsSide || alreadyOffer){ - return tableInfo; - } - - if(tableInfo.getLeftNode().getKind() != AS){ - extractTemporaryQuery(tableInfo.getLeftNode(), tableInfo.getLeftTableAlias(), (SqlBasicCall) parentWhere, - parentSelectList, queueInfo, joinFieldSet); - }else { - SqlKind asNodeFirstKind = ((SqlBasicCall)tableInfo.getLeftNode()).operands[0].getKind(); - if(asNodeFirstKind == SELECT){ - queueInfo.offer(tableInfo.getLeftNode()); - tableInfo.setLeftNode(((SqlBasicCall)tableInfo.getLeftNode()).operands[1]); - } - } - return tableInfo; - } - - - //构建新的查询 - private Tuple2 dealNestJoin(SqlJoin joinNode, Set sideTableSet, - Queue queueInfo, SqlNode parentWhere, - SqlNodeList selectList, Set> joinFieldSet){ - SqlNode rightNode = joinNode.getRight(); - Tuple2 rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo, parentWhere, selectList); - JoinInfo joinInfo = dealJoinNode(joinNode, sideTableSet, queueInfo, parentWhere, selectList, joinFieldSet); - - String rightTableName = rightTableNameAndAlias.f0; - boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); - boolean alreadyOffer = false; - - if(!rightIsSide){ - //右表不是维表的情况 - }else{ - //右边表是维表需要重新构建左表的临时查询 - queueInfo.offer(joinInfo); - alreadyOffer = true; - } - - return Tuple2.of(alreadyOffer, TableUtils.buildAsNodeByJoinInfo(joinInfo, null, null)); - } - - public boolean checkAndRemoveCondition(Set fromTableNameSet, SqlBasicCall parentWhere, List extractCondition){ - - if(parentWhere == null){ - return false; - } - - SqlKind kind = parentWhere.getKind(); - if(kind == AND){ - boolean removeLeft = checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[0], extractCondition); - boolean removeRight = checkAndRemoveCondition(fromTableNameSet, (SqlBasicCall) parentWhere.getOperands()[1], extractCondition); - //DO remove - if(removeLeft){ - extractCondition.add(removeWhereConditionNode(parentWhere, 0)); - } - - if(removeRight){ - extractCondition.add(removeWhereConditionNode(parentWhere, 1)); - } - - return false; - }else{ - Set conditionRefTableNameSet = Sets.newHashSet(); - getConditionRefTable(parentWhere, conditionRefTableNameSet); - - if(fromTableNameSet.containsAll(conditionRefTableNameSet)){ - return true; - } - - return false; - } - } - - private void extractTemporaryQuery(SqlNode node, String tableAlias, SqlBasicCall parentWhere, - SqlNodeList parentSelectList, Queue queueInfo, - Set> joinFieldSet){ - try{ - //父一级的where 条件中如果只和临时查询相关的条件都截取进来 - Set fromTableNameSet = Sets.newHashSet(); - List extractCondition = Lists.newArrayList(); - - getFromTableInfo(node, fromTableNameSet); - checkAndRemoveCondition(fromTableNameSet, parentWhere, extractCondition); - - Set extractSelectField = extractSelectFields(parentSelectList, fromTableNameSet); - Set fieldFromJoinCondition = extractSelectFieldFromJoinCondition(joinFieldSet, fromTableNameSet); - String extractSelectFieldStr = buildSelectNode(extractSelectField, fieldFromJoinCondition); - String extractConditionStr = buildCondition(extractCondition); - - String tmpSelectSql = String.format(SELECT_TEMP_SQL, - extractSelectFieldStr, - node.toString(), - extractConditionStr); - - SqlParser sqlParser = SqlParser.create(tmpSelectSql, CalciteConfig.MYSQL_LEX_CONFIG); - SqlNode sqlNode = sqlParser.parseStmt(); - SqlBasicCall sqlBasicCall = buildAsSqlNode(tableAlias, sqlNode); - queueInfo.offer(sqlBasicCall); - - System.out.println("-------build temporary query-----------"); - System.out.println(tmpSelectSql); - System.out.println("---------------------------------------"); - }catch (Exception e){ - e.printStackTrace(); - throw new RuntimeException(e); - } - } - - /** - * 抽取上层需用使用到的字段 - * 由于where字段已经抽取到上一层了所以不用查询出来 - * @param parentSelectList - * @param fromTableNameSet - * @return - */ - private Set extractSelectFields(SqlNodeList parentSelectList, Set fromTableNameSet){ - Set extractFieldList = Sets.newHashSet(); - for(SqlNode selectNode : parentSelectList.getList()){ - extractSelectField(selectNode, extractFieldList, fromTableNameSet); - } - - return extractFieldList; - } - - private Set extractSelectFieldFromJoinCondition(Set> joinFieldSet, Set fromTableNameSet){ - Set extractFieldList = Sets.newHashSet(); - for(Tuple2 field : joinFieldSet){ - if(fromTableNameSet.contains(field.f0)){ - extractFieldList.add(field.f0 + "." + field.f1); - } - } - - return extractFieldList; - } - - /** - * 从join的条件中获取字段信息 - * @param condition - * @param joinFieldSet - */ - private void extractJoinField(SqlNode condition, Set> joinFieldSet){ - SqlKind joinKind = condition.getKind(); - if( joinKind == AND || joinKind == EQUALS ){ - extractJoinField(((SqlBasicCall)condition).operands[0], joinFieldSet); - extractJoinField(((SqlBasicCall)condition).operands[1], joinFieldSet); - }else{ - Preconditions.checkState(((SqlIdentifier)condition).names.size() == 2, "join condition must be format table.field"); - Tuple2 tuple2 = Tuple2.of(((SqlIdentifier)condition).names.get(0), ((SqlIdentifier)condition).names.get(1)); - joinFieldSet.add(tuple2); - } - } - - private void extractSelectField(SqlNode selectNode, Set extractFieldSet, Set fromTableNameSet){ - if (selectNode.getKind() == AS) { - SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; - extractSelectField(leftNode, extractFieldSet, fromTableNameSet); - - }else if(selectNode.getKind() == IDENTIFIER) { - SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; - - if(sqlIdentifier.names.size() == 1){ - return; - } - - String tableName = sqlIdentifier.names.get(0); - if(fromTableNameSet.contains(tableName)){ - extractFieldSet.add(sqlIdentifier.toString()); - } - - }else if( AGGREGATE.contains(selectNode.getKind()) - || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) - || COMPARISON.contains(selectNode.getKind()) - || selectNode.getKind() == OTHER_FUNCTION - || selectNode.getKind() == DIVIDE - || selectNode.getKind() == CAST - || selectNode.getKind() == TRIM - || selectNode.getKind() == TIMES - || selectNode.getKind() == PLUS - || selectNode.getKind() == NOT_IN - || selectNode.getKind() == OR - || selectNode.getKind() == AND - || selectNode.getKind() == MINUS - || selectNode.getKind() == TUMBLE - || selectNode.getKind() == TUMBLE_START - || selectNode.getKind() == TUMBLE_END - || selectNode.getKind() == SESSION - || selectNode.getKind() == SESSION_START - || selectNode.getKind() == SESSION_END - || selectNode.getKind() == HOP - || selectNode.getKind() == HOP_START - || selectNode.getKind() == HOP_END - || selectNode.getKind() == BETWEEN - || selectNode.getKind() == IS_NULL - || selectNode.getKind() == IS_NOT_NULL - || selectNode.getKind() == CONTAINS - || selectNode.getKind() == TIMESTAMP_ADD - || selectNode.getKind() == TIMESTAMP_DIFF - || selectNode.getKind() == LIKE - - ){ - SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; - for(int i=0; i parseRightNode(SqlNode sqlNode, Set sideTableSet, Queue queueInfo, - SqlNode parentWhere, SqlNodeList selectList) { - Tuple2 tabName = new Tuple2<>("", ""); - if(sqlNode.getKind() == IDENTIFIER){ - tabName.f0 = sqlNode.toString(); - }else{ - AliasInfo aliasInfo = (AliasInfo)parseSql(sqlNode, sideTableSet, queueInfo, parentWhere, selectList); - tabName.f0 = aliasInfo.getName(); - tabName.f1 = aliasInfo.getAlias(); - } - return tabName; - } - - private Tuple2 parseLeftNode(SqlNode sqlNode){ - Tuple2 tabName = new Tuple2<>("", ""); - if(sqlNode.getKind() == IDENTIFIER){ - tabName.f0 = sqlNode.toString(); - tabName.f1 = sqlNode.toString(); - }else if (sqlNode.getKind() == AS){ - SqlNode info = ((SqlBasicCall)sqlNode).getOperands()[0]; - SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; - - tabName.f0 = info.toString(); - tabName.f1 = alias.toString(); - }else { - throw new RuntimeException(""); - } - - return tabName; - } - - public SqlBasicCall removeWhereConditionNode(SqlBasicCall parentWhere, int index){ - //构造1=1 条件 - SqlBasicCall oldCondition = (SqlBasicCall) parentWhere.getOperands()[index]; - parentWhere.setOperand(index, buildDefaultCondition()); - return oldCondition; - } - - public String buildCondition(List conditionList){ - if(CollectionUtils.isEmpty(conditionList)){ - return ""; - } - - return " where " + StringUtils.join(conditionList, " AND "); - } - - public String buildSelectNode(Set extractSelectField, Set joinFieldSet){ - if(CollectionUtils.isEmpty(extractSelectField)){ - throw new RuntimeException("no field is used"); - } - - Sets.SetView view = Sets.union(extractSelectField, joinFieldSet); - - return StringUtils.join(view, ","); - } - - public SqlBasicCall buildDefaultCondition(){ - SqlBinaryOperator equalsOperators = SqlStdOperatorTable.EQUALS; - SqlNode[] operands = new SqlNode[2]; - operands[0] = SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO); - operands[1] = SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO); - - return new SqlBasicCall(equalsOperators, operands, SqlParserPos.ZERO); - } - - - private boolean checkIsSideTable(String tableName, Set sideTableList){ - if(sideTableList.contains(tableName)){ - return true; - } - return false; - } public void setLocalTableCache(Map localTableCache) { this.localTableCache = localTableCache; } - //TODO 之后抽取 - private void getConditionRefTable(SqlNode selectNode, Set tableNameSet) { - if(selectNode.getKind() == IDENTIFIER){ - SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; - - if(sqlIdentifier.names.size() == 1){ - return; - } - - String tableName = sqlIdentifier.names.asList().get(0); - tableNameSet.add(tableName); - return; - }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 - return; - }else if( AGGREGATE.contains(selectNode.getKind()) - || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) - || COMPARISON.contains(selectNode.getKind()) - || selectNode.getKind() == OTHER_FUNCTION - || selectNode.getKind() == DIVIDE - || selectNode.getKind() == CAST - || selectNode.getKind() == TRIM - || selectNode.getKind() == TIMES - || selectNode.getKind() == PLUS - || selectNode.getKind() == NOT_IN - || selectNode.getKind() == OR - || selectNode.getKind() == AND - || selectNode.getKind() == MINUS - || selectNode.getKind() == TUMBLE - || selectNode.getKind() == TUMBLE_START - || selectNode.getKind() == TUMBLE_END - || selectNode.getKind() == SESSION - || selectNode.getKind() == SESSION_START - || selectNode.getKind() == SESSION_END - || selectNode.getKind() == HOP - || selectNode.getKind() == HOP_START - || selectNode.getKind() == HOP_END - || selectNode.getKind() == BETWEEN - || selectNode.getKind() == IS_NULL - || selectNode.getKind() == IS_NOT_NULL - || selectNode.getKind() == CONTAINS - || selectNode.getKind() == TIMESTAMP_ADD - || selectNode.getKind() == TIMESTAMP_DIFF - || selectNode.getKind() == LIKE - - ){ - SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; - for(int i=0; i tableNameSet){ - System.out.println(fromTable); - SqlKind sqlKind = fromTable.getKind(); - switch (sqlKind){ - case AS: - SqlNode alias = ((SqlBasicCall) fromTable).getOperands()[1]; - tableNameSet.add(alias.toString()); - return; - case JOIN: - getFromTableInfo(((SqlJoin)fromTable).getLeft(), tableNameSet); - getFromTableInfo(((SqlJoin)fromTable).getRight(), tableNameSet); - return; - case IDENTIFIER: - tableNameSet.add(((SqlIdentifier)fromTable).getSimple()); - return; - default: - throw new RuntimeException("not support sqlKind:" + sqlKind); - } - } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 566fc7dc4..a6c9491d3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -31,6 +31,7 @@ import com.google.common.collect.HashBasedTable; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlDataTypeSpec; @@ -67,6 +68,7 @@ import java.util.List; import java.util.Map; import java.util.Queue; +import java.util.Set; import static org.apache.calcite.sql.SqlKind.*; @@ -86,10 +88,10 @@ public class SideSqlExec { private String tmpFields = null; private SideSQLParser sideSQLParser = new SideSQLParser(); + private SidePredicatesParser sidePredicatesParser = new SidePredicatesParser(); private Map localTableCache = Maps.newHashMap(); - private StreamTableEnvironment tableEnv ; public void exec(String sql, Map sideTableMap, StreamTableEnvironment tableEnv, Map tableCache, StreamQueryConfig queryConfig) throws Exception { @@ -142,6 +144,17 @@ public void exec(String sql, Map sideTableMap, StreamTabl FieldReplaceInfo fieldReplaceInfo = parseAsQuery((SqlBasicCall) pollSqlNode, tableCache); if(fieldReplaceInfo != null){ + //as 的源表 + Set fromTableNameSet = Sets.newHashSet(); + SqlNode fromNode = ((SqlBasicCall)pollSqlNode).getOperands()[0]; + TableUtils.getFromTableInfo(fromNode, fromTableNameSet); + for(FieldReplaceInfo tmp : replaceInfoList){ + if(fromTableNameSet.contains(tmp.getTargetTableName()) + || fromTableNameSet.contains(tmp.getTargetTableAlias())){ + fieldReplaceInfo.setPreNode(tmp); + break; + } + } replaceInfoList.add(fieldReplaceInfo); } } else if (pollSqlNode.getKind() == WITH_ITEM) { @@ -152,6 +165,8 @@ public void exec(String sql, Map sideTableMap, StreamTabl } }else if (pollObj instanceof JoinInfo){ + System.out.println("----------exec join info----------"); + System.out.println(pollObj.toString()); preIsSideJoin = true; joinFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); } @@ -694,6 +709,7 @@ public List getConditionFields(SqlNode conditionNode, String specifyTabl return conditionFields; } + //TODO 合并临时表处理逻辑 public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, Map sideTableMap, StreamTableEnvironment tableEnv, Map tableCache) @@ -748,6 +764,8 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, }else if (pollObj instanceof JoinInfo){ preIsSideJoin = true; + System.out.println("----------exec join info----------"); + System.out.println(pollObj.toString()); joinFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java index 911b6fba6..3c8740213 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/TableUtils.java @@ -20,28 +20,40 @@ package com.dtstack.flink.sql.util; import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.FieldReplaceInfo; import com.dtstack.flink.sql.side.JoinInfo; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.fun.SqlCase; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.api.Table; import java.util.List; import java.util.Map; import java.util.Queue; +import java.util.Set; -import static org.apache.calcite.sql.SqlKind.SELECT; +import static org.apache.calcite.sql.SqlKind.*; +import static org.apache.calcite.sql.SqlKind.CASE; +import static org.apache.calcite.sql.SqlKind.OTHER; /** - * + * 表的解析相关 * Date: 2020/2/17 * Company: www.dtstack.com * @author xuchao @@ -162,4 +174,313 @@ public static void replaceFromNodeForJoin(JoinInfo joinInfo, SqlSelect sqlNode) sqlNode.setFrom(sqlBasicCall); } + + /** + * 获取节点关联的查询表 + * @param fromTable + * @param tableNameSet + */ + public static void getFromTableInfo(SqlNode fromTable, Set tableNameSet){ + SqlKind sqlKind = fromTable.getKind(); + switch (sqlKind){ + case AS: + SqlNode alias = ((SqlBasicCall) fromTable).getOperands()[1]; + tableNameSet.add(alias.toString()); + return; + case JOIN: + getFromTableInfo(((SqlJoin)fromTable).getLeft(), tableNameSet); + getFromTableInfo(((SqlJoin)fromTable).getRight(), tableNameSet); + return; + case IDENTIFIER: + tableNameSet.add(((SqlIdentifier)fromTable).getSimple()); + return; + case SELECT: + getFromTableInfo(((SqlSelect) fromTable).getFrom(), tableNameSet); + return; + default: + throw new RuntimeException("not support sqlKind:" + sqlKind); + } + } + + public static void replaceSelectFieldTable(SqlNode selectNode, String oldTbName, String newTbName) { + if (selectNode.getKind() == AS) { + SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; + replaceSelectFieldTable(leftNode, oldTbName, newTbName); + + }else if(selectNode.getKind() == IDENTIFIER){ + SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; + + if(sqlIdentifier.names.size() == 1){ + return ; + } + + if(oldTbName.equalsIgnoreCase(((SqlIdentifier)selectNode).names.get(0))){ + SqlIdentifier newField = ((SqlIdentifier)selectNode).setName(0, newTbName); + ((SqlIdentifier)selectNode).assignNamesFrom(newField); + } + + }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 + return; + }else if( AGGREGATE.contains(selectNode.getKind()) + || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) + || COMPARISON.contains(selectNode.getKind()) + || selectNode.getKind() == OTHER_FUNCTION + || selectNode.getKind() == DIVIDE + || selectNode.getKind() == CAST + || selectNode.getKind() == TRIM + || selectNode.getKind() == TIMES + || selectNode.getKind() == PLUS + || selectNode.getKind() == NOT_IN + || selectNode.getKind() == OR + || selectNode.getKind() == AND + || selectNode.getKind() == MINUS + || selectNode.getKind() == TUMBLE + || selectNode.getKind() == TUMBLE_START + || selectNode.getKind() == TUMBLE_END + || selectNode.getKind() == SESSION + || selectNode.getKind() == SESSION_START + || selectNode.getKind() == SESSION_END + || selectNode.getKind() == HOP + || selectNode.getKind() == HOP_START + || selectNode.getKind() == HOP_END + || selectNode.getKind() == BETWEEN + || selectNode.getKind() == IS_NULL + || selectNode.getKind() == IS_NOT_NULL + || selectNode.getKind() == CONTAINS + || selectNode.getKind() == TIMESTAMP_ADD + || selectNode.getKind() == TIMESTAMP_DIFF + || selectNode.getKind() == LIKE + + ){ + SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; + for(int i=0; i tableRef){ + SqlKind joinKind = condition.getKind(); + if( joinKind == AND || joinKind == EQUALS ){ + replaceJoinFieldRefTableName(((SqlBasicCall)condition).operands[0], tableRef); + replaceJoinFieldRefTableName(((SqlBasicCall)condition).operands[1], tableRef); + }else{ + Preconditions.checkState(((SqlIdentifier)condition).names.size() == 2, "join condition must be format table.field"); + String fieldRefTable = ((SqlIdentifier)condition).names.get(0); + + String targetTableName = TableUtils.getTargetRefTable(tableRef, fieldRefTable); + if(StringUtils.isNotBlank(targetTableName) && !fieldRefTable.equalsIgnoreCase(targetTableName)){ + SqlIdentifier newField = ((SqlIdentifier)condition).setName(0, targetTableName); + ((SqlIdentifier)condition).assignNamesFrom(newField); + } + } + } + + public static String getTargetRefTable(Map refTableMap, String tableName){ + String targetTableName = null; + String preTableName; + + do { + preTableName = targetTableName == null ? tableName : targetTableName; + targetTableName = refTableMap.get(preTableName); + } while (targetTableName != null); + + return preTableName; + } + + public static void replaceWhereCondition(SqlNode parentWhere, String oldTbName, String newTbName){ + + if(parentWhere == null){ + return; + } + + SqlKind kind = parentWhere.getKind(); + if(kind == AND){ + replaceWhereCondition(((SqlBasicCall) parentWhere).getOperands()[0], oldTbName, newTbName); + replaceWhereCondition(((SqlBasicCall) parentWhere).getOperands()[1], oldTbName, newTbName); + + } else { + replaceConditionNode(parentWhere, oldTbName, newTbName); + } + } + + private static void replaceConditionNode(SqlNode selectNode, String oldTbName, String newTbName) { + if(selectNode.getKind() == IDENTIFIER){ + SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; + + if(sqlIdentifier.names.size() == 1){ + return; + } + + String tableName = sqlIdentifier.names.asList().get(0); + if(tableName.equalsIgnoreCase(oldTbName)){ + SqlIdentifier newField = ((SqlIdentifier)selectNode).setName(0, newTbName); + ((SqlIdentifier)selectNode).assignNamesFrom(newField); + } + return; + }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 + return; + }else if( AGGREGATE.contains(selectNode.getKind()) + || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) + || COMPARISON.contains(selectNode.getKind()) + || selectNode.getKind() == OTHER_FUNCTION + || selectNode.getKind() == DIVIDE + || selectNode.getKind() == CAST + || selectNode.getKind() == TRIM + || selectNode.getKind() == TIMES + || selectNode.getKind() == PLUS + || selectNode.getKind() == NOT_IN + || selectNode.getKind() == OR + || selectNode.getKind() == AND + || selectNode.getKind() == MINUS + || selectNode.getKind() == TUMBLE + || selectNode.getKind() == TUMBLE_START + || selectNode.getKind() == TUMBLE_END + || selectNode.getKind() == SESSION + || selectNode.getKind() == SESSION_START + || selectNode.getKind() == SESSION_END + || selectNode.getKind() == HOP + || selectNode.getKind() == HOP_START + || selectNode.getKind() == HOP_END + || selectNode.getKind() == BETWEEN + || selectNode.getKind() == IS_NULL + || selectNode.getKind() == IS_NOT_NULL + || selectNode.getKind() == CONTAINS + || selectNode.getKind() == TIMESTAMP_ADD + || selectNode.getKind() == TIMESTAMP_DIFF + || selectNode.getKind() == LIKE + + ){ + SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; + for(int i=0; i tableNameSet) { + if(selectNode.getKind() == IDENTIFIER){ + SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; + + if(sqlIdentifier.names.size() == 1){ + return; + } + + String tableName = sqlIdentifier.names.asList().get(0); + tableNameSet.add(tableName); + return; + }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 + return; + }else if( AGGREGATE.contains(selectNode.getKind()) + || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) + || COMPARISON.contains(selectNode.getKind()) + || selectNode.getKind() == OTHER_FUNCTION + || selectNode.getKind() == DIVIDE + || selectNode.getKind() == CAST + || selectNode.getKind() == TRIM + || selectNode.getKind() == TIMES + || selectNode.getKind() == PLUS + || selectNode.getKind() == NOT_IN + || selectNode.getKind() == OR + || selectNode.getKind() == AND + || selectNode.getKind() == MINUS + || selectNode.getKind() == TUMBLE + || selectNode.getKind() == TUMBLE_START + || selectNode.getKind() == TUMBLE_END + || selectNode.getKind() == SESSION + || selectNode.getKind() == SESSION_START + || selectNode.getKind() == SESSION_END + || selectNode.getKind() == HOP + || selectNode.getKind() == HOP_START + || selectNode.getKind() == HOP_END + || selectNode.getKind() == BETWEEN + || selectNode.getKind() == IS_NULL + || selectNode.getKind() == IS_NOT_NULL + || selectNode.getKind() == CONTAINS + || selectNode.getKind() == TIMESTAMP_ADD + || selectNode.getKind() == TIMESTAMP_DIFF + || selectNode.getKind() == LIKE + + ){ + SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; + for(int i=0; i Date: Fri, 28 Feb 2020 13:01:38 +0800 Subject: [PATCH 13/13] =?UTF-8?q?=E5=90=88=E5=B9=B6=E7=BB=B4=E8=A1=A8?= =?UTF-8?q?=E5=85=B3=E8=81=94=E9=83=A8=E5=88=86=E4=B8=B4=E6=97=B6=E8=A1=A8?= =?UTF-8?q?=E6=B3=A8=E5=86=8C=E5=92=8Cinsert=20=E9=83=A8=E5=88=86?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/Main.java | 6 +- .../com/dtstack/flink/sql/side/JoinInfo.java | 2 - .../dtstack/flink/sql/side/SideSqlExec.java | 143 +++++++----------- 3 files changed, 57 insertions(+), 94 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 24d616384..299754238 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -153,7 +153,7 @@ private static void sqlTranslation(String localSqlPluginPath, StreamTableEnviron SideSqlExec sideSqlExec = new SideSqlExec(); sideSqlExec.setLocalSqlPluginPath(localSqlPluginPath); for (CreateTmpTableParser.SqlParserResult result : sqlTree.getTmpSqlList()) { - sideSqlExec.registerTmpTable(result, sideTableMap, tableEnv, registerTableCache); + sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, result); } for (InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()) { @@ -169,7 +169,7 @@ private static void sqlTranslation(String localSqlPluginPath, StreamTableEnviron SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql, CalciteConfig.MYSQL_LEX_CONFIG).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); - sideSqlExec.registerTmpTable(tmp, sideTableMap, tableEnv, registerTableCache); + sideSqlExec.exec(tmp.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, tmp); } else { for (String sourceTable : result.getSourceTableList()) { if (sideTableMap.containsKey(sourceTable)) { @@ -179,7 +179,7 @@ private static void sqlTranslation(String localSqlPluginPath, StreamTableEnviron } if (isSide) { //sql-dimensional table contains the dimension table of execution - sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig); + sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache, queryConfig, null); }else{ System.out.println("----------exec sql without dimension join-----------" ); System.out.println("----------real sql exec is--------------------------"); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java index 210fb5b6a..8a8fe21f6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -213,8 +213,6 @@ public String toString() { ", leftTableAlias='" + leftTableAlias + '\'' + ", rightTableName='" + rightTableName + '\'' + ", rightTableAlias='" + rightTableAlias + '\'' + - ", leftNode=" + leftNode + - ", rightNode=" + rightNode + ", condition=" + condition + ", selectFields=" + selectFields + ", selectNode=" + selectNode + diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index a6c9491d3..91fff5e71 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -28,6 +28,7 @@ import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.ParseUtils; import com.dtstack.flink.sql.util.TableUtils; +import com.google.common.base.Preconditions; import com.google.common.collect.HashBasedTable; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -87,14 +88,12 @@ public class SideSqlExec { private String tmpFields = null; - private SideSQLParser sideSQLParser = new SideSQLParser(); - private SidePredicatesParser sidePredicatesParser = new SidePredicatesParser(); private Map localTableCache = Maps.newHashMap(); public void exec(String sql, Map sideTableMap, StreamTableEnvironment tableEnv, - Map tableCache, StreamQueryConfig queryConfig) throws Exception { + Map tableCache, StreamQueryConfig queryConfig, CreateTmpTableParser.SqlParserResult createView) throws Exception { if(localSqlPluginPath == null){ throw new RuntimeException("need to set localSqlPluginPath"); } @@ -106,6 +105,13 @@ public void exec(String sql, Map sideTableMap, StreamTabl LOG.error("fill predicates for sideTable fail ", e); } + if(createView != null){ + LOG.warn("create view info\n"); + LOG.warn(createView.getExecSql()); + LOG.warn("-----------------"); + } + + SideSQLParser sideSQLParser = new SideSQLParser(); sideSQLParser.setLocalTableCache(localTableCache); Queue exeQueue = sideSQLParser.getExeQueue(sql, sideTableMap.keySet()); Object pollObj = null; @@ -136,32 +142,32 @@ public void exec(String sql, Map sideTableMap, StreamTabl if(LOG.isInfoEnabled()){ LOG.info("exec sql: " + pollSqlNode.toString()); } + }else if(pollSqlNode.getKind() == AS){ - AliasInfo aliasInfo = parseASNode(pollSqlNode); - Table table = tableEnv.sqlQuery(aliasInfo.getName()); - tableEnv.registerTable(aliasInfo.getAlias(), table); - localTableCache.put(aliasInfo.getAlias(), table); - - FieldReplaceInfo fieldReplaceInfo = parseAsQuery((SqlBasicCall) pollSqlNode, tableCache); - if(fieldReplaceInfo != null){ - //as 的源表 - Set fromTableNameSet = Sets.newHashSet(); - SqlNode fromNode = ((SqlBasicCall)pollSqlNode).getOperands()[0]; - TableUtils.getFromTableInfo(fromNode, fromTableNameSet); - for(FieldReplaceInfo tmp : replaceInfoList){ - if(fromTableNameSet.contains(tmp.getTargetTableName()) - || fromTableNameSet.contains(tmp.getTargetTableAlias())){ - fieldReplaceInfo.setPreNode(tmp); - break; - } - } - replaceInfoList.add(fieldReplaceInfo); - } + dealAsSourceTable(tableEnv, pollSqlNode, tableCache, replaceInfoList); + } else if (pollSqlNode.getKind() == WITH_ITEM) { SqlWithItem sqlWithItem = (SqlWithItem) pollSqlNode; String TableAlias = sqlWithItem.name.toString(); Table table = tableEnv.sqlQuery(sqlWithItem.query.toString()); tableEnv.registerTable(TableAlias, table); + + } else if (pollSqlNode.getKind() == SELECT){ + Preconditions.checkState(createView != null, "select sql must included by create view"); + Table table = tableEnv.sqlQuery(pollObj.toString()); + + if (createView.getFieldsInfoStr() == null){ + tableEnv.registerTable(createView.getTableName(), table); + } else { + if (checkFieldsInfo(createView, table)){ + table = table.as(tmpFields); + tableEnv.registerTable(createView.getTableName(), table); + } else { + throw new RuntimeException("Fields mismatch"); + } + } + + localTableCache.put(createView.getTableName(), table); } }else if (pollObj instanceof JoinInfo){ @@ -174,6 +180,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl } + /** * 解析出as查询的表和字段的关系 * @param asSqlNode @@ -709,69 +716,11 @@ public List getConditionFields(SqlNode conditionNode, String specifyTabl return conditionFields; } - //TODO 合并临时表处理逻辑 - public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, - Map sideTableMap, StreamTableEnvironment tableEnv, - Map tableCache) - throws Exception { - - if(localSqlPluginPath == null){ - throw new RuntimeException("need to set localSqlPluginPath"); - } - - localTableCache.putAll(tableCache); - Queue exeQueue = sideSQLParser.getExeQueue(result.getExecSql(), sideTableMap.keySet()); - Object pollObj = null; + protected void dealAsSourceTable(StreamTableEnvironment tableEnv, + SqlNode pollSqlNode, + Map tableCache, + List replaceInfoList) throws SqlParseException { - //need clean - boolean preIsSideJoin = false; - List replaceInfoList = Lists.newArrayList(); - - while((pollObj = exeQueue.poll()) != null){ - - if(pollObj instanceof SqlNode){ - SqlNode pollSqlNode = (SqlNode) pollObj; - - if(preIsSideJoin){ - preIsSideJoin = false; - List fieldNames = null; - for (FieldReplaceInfo replaceInfo : replaceInfoList) { - fieldNames = Lists.newArrayList(); - replaceFieldName(pollSqlNode, replaceInfo); - addAliasForFieldNode(pollSqlNode, fieldNames, replaceInfo.getMappingTable()); - } - } - - if(pollSqlNode.getKind() == INSERT){ - tableEnv.sqlUpdate(pollSqlNode.toString()); - }else if(pollSqlNode.getKind() == AS){ - dealAsSourceTable(tableEnv, pollSqlNode); - } else if (pollSqlNode.getKind() == SELECT){ - Table table = tableEnv.sqlQuery(pollObj.toString()); - if (result.getFieldsInfoStr() == null){ - tableEnv.registerTable(result.getTableName(), table); - } else { - if (checkFieldsInfo(result, table)){ - table = table.as(tmpFields); - tableEnv.registerTable(result.getTableName(), table); - } else { - throw new RuntimeException("Fields mismatch"); - } - } - localTableCache.put(result.getTableName(), table); - - } - - }else if (pollObj instanceof JoinInfo){ - preIsSideJoin = true; - System.out.println("----------exec join info----------"); - System.out.println(pollObj.toString()); - joinFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); - } - } - } - - protected void dealAsSourceTable(StreamTableEnvironment tableEnv, SqlNode pollSqlNode) throws SqlParseException { AliasInfo aliasInfo = parseASNode(pollSqlNode); if (localTableCache.containsKey(aliasInfo.getName())) { return; @@ -779,11 +728,27 @@ protected void dealAsSourceTable(StreamTableEnvironment tableEnv, SqlNode pollSq Table table = tableEnv.sqlQuery(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); - if (LOG.isInfoEnabled()) { - LOG.info("Register Table {} by {}", aliasInfo.getAlias(), aliasInfo.getName()); + localTableCache.put(aliasInfo.getAlias(), table); + + LOG.info("Register Table {} by {}", aliasInfo.getAlias(), aliasInfo.getName()); + + FieldReplaceInfo fieldReplaceInfo = parseAsQuery((SqlBasicCall) pollSqlNode, tableCache); + if(fieldReplaceInfo == null){ + return; } - localTableCache.put(aliasInfo.getAlias(), table); + //as 的源表 + Set fromTableNameSet = Sets.newHashSet(); + SqlNode fromNode = ((SqlBasicCall)pollSqlNode).getOperands()[0]; + TableUtils.getFromTableInfo(fromNode, fromTableNameSet); + for(FieldReplaceInfo tmp : replaceInfoList){ + if(fromTableNameSet.contains(tmp.getTargetTableName()) + || fromTableNameSet.contains(tmp.getTargetTableAlias())){ + fieldReplaceInfo.setPreNode(tmp); + break; + } + } + replaceInfoList.add(fieldReplaceInfo); } private void joinFun(Object pollObj, Map localTableCache, @@ -798,7 +763,7 @@ private void joinFun(Object pollObj, Map localTableCache, SqlKind sqlKind = joinInfo.getLeftNode().getKind(); if(sqlKind == AS){ - dealAsSourceTable(tableEnv, joinInfo.getLeftNode()); + dealAsSourceTable(tableEnv, joinInfo.getLeftNode(), localTableCache, replaceInfoList); } Table leftTable = getTableFromCache(localTableCache, joinInfo.getLeftTableAlias(), joinInfo.getLeftTableName());