|
20 | 20 | package com.dtstack.flink.sql.util; |
21 | 21 |
|
22 | 22 | import com.dtstack.flink.sql.side.FieldInfo; |
23 | | -import com.dtstack.flink.sql.side.FieldReplaceInfo; |
24 | 23 | import com.dtstack.flink.sql.side.JoinInfo; |
25 | 24 | import com.google.common.base.Preconditions; |
26 | 25 | import com.google.common.base.Strings; |
27 | 26 | import com.google.common.collect.Lists; |
28 | | -import com.google.common.collect.Sets; |
29 | 27 | import org.apache.calcite.sql.SqlAsOperator; |
30 | 28 | import org.apache.calcite.sql.SqlBasicCall; |
31 | 29 | import org.apache.calcite.sql.SqlDataTypeSpec; |
|
40 | 38 | import org.apache.calcite.sql.fun.SqlCase; |
41 | 39 | import org.apache.calcite.sql.parser.SqlParserPos; |
42 | 40 | import org.apache.commons.lang3.StringUtils; |
43 | | -import org.apache.flink.api.java.tuple.Tuple2; |
44 | 41 | import org.apache.flink.table.api.Table; |
45 | 42 |
|
46 | 43 | import java.util.List; |
@@ -72,41 +69,119 @@ public static List<FieldInfo> parserSelectField(SqlSelect sqlSelect, Map<String, |
72 | 69 | List<FieldInfo> fieldInfoList = Lists.newArrayList(); |
73 | 70 | String fromNode = sqlSelect.getFrom().toString(); |
74 | 71 |
|
75 | | - for(SqlNode fieldNode : sqlNodeList.getList()){ |
76 | | - SqlIdentifier identifier = (SqlIdentifier)fieldNode; |
77 | | - if(!identifier.isStar()) { |
| 72 | + for (SqlNode fieldNode : sqlNodeList.getList()) { |
| 73 | + extractSelectFieldToFieldInfo(fieldNode,fromNode,fieldInfoList,localTableCache); |
| 74 | + } |
| 75 | + |
| 76 | + return fieldInfoList; |
| 77 | + } |
| 78 | + |
| 79 | + /** |
| 80 | + * 解析select Node 提取FieldInfo |
| 81 | + * @param fieldNode |
| 82 | + * @param fromNode |
| 83 | + * @param fieldInfoList |
| 84 | + * @param localTableCache |
| 85 | + */ |
| 86 | + public static void extractSelectFieldToFieldInfo(SqlNode fieldNode, String fromNode, List<FieldInfo> fieldInfoList, Map<String, Table> localTableCache) { |
| 87 | + if (fieldNode.getKind() == IDENTIFIER) { |
| 88 | + SqlIdentifier identifier = (SqlIdentifier) fieldNode; |
| 89 | + if (!identifier.isStar()) { |
78 | 90 | String tableName = identifier.names.size() == 1 ? fromNode : identifier.getComponent(0).getSimple(); |
79 | 91 | String fieldName = identifier.names.size() == 1 ? identifier.getComponent(0).getSimple() : identifier.getComponent(1).getSimple(); |
80 | 92 | FieldInfo fieldInfo = new FieldInfo(); |
81 | 93 | fieldInfo.setTable(tableName); |
82 | 94 | fieldInfo.setFieldName(fieldName); |
83 | | - fieldInfoList.add(fieldInfo); |
| 95 | + |
| 96 | + if (!fieldInfoList.contains(fieldInfo)) { |
| 97 | + fieldInfoList.add(fieldInfo); |
| 98 | + } |
84 | 99 | } else { |
85 | 100 | //处理 |
86 | 101 | int identifierSize = identifier.names.size(); |
87 | | - |
88 | | - switch(identifierSize) { |
| 102 | + switch (identifierSize) { |
89 | 103 | case 1: |
90 | 104 | throw new RuntimeException("not support to parse * without scope of table"); |
91 | 105 | default: |
92 | 106 | SqlIdentifier tableIdentify = identifier.skipLast(1); |
93 | 107 | Table registerTable = localTableCache.get(tableIdentify.getSimple()); |
94 | | - if(registerTable == null){ |
| 108 | + if (registerTable == null) { |
95 | 109 | throw new RuntimeException("can't find table alias " + tableIdentify.getSimple()); |
96 | 110 | } |
97 | 111 |
|
98 | 112 | String[] fieldNames = registerTable.getSchema().getFieldNames(); |
99 | | - for(String fieldName : fieldNames){ |
| 113 | + for (String fieldName : fieldNames) { |
100 | 114 | FieldInfo fieldInfo = new FieldInfo(); |
101 | 115 | fieldInfo.setTable(tableIdentify.getSimple()); |
102 | 116 | fieldInfo.setFieldName(fieldName); |
103 | 117 | fieldInfoList.add(fieldInfo); |
104 | 118 | } |
105 | 119 | } |
106 | 120 | } |
107 | | - } |
| 121 | + } else if (AGGREGATE.contains(fieldNode.getKind()) |
| 122 | + || AVG_AGG_FUNCTIONS.contains(fieldNode.getKind()) |
| 123 | + || COMPARISON.contains(fieldNode.getKind()) |
| 124 | + || fieldNode.getKind() == OTHER_FUNCTION |
| 125 | + || fieldNode.getKind() == DIVIDE |
| 126 | + || fieldNode.getKind() == CAST |
| 127 | + || fieldNode.getKind() == TRIM |
| 128 | + || fieldNode.getKind() == TIMES |
| 129 | + || fieldNode.getKind() == PLUS |
| 130 | + || fieldNode.getKind() == NOT_IN |
| 131 | + || fieldNode.getKind() == OR |
| 132 | + || fieldNode.getKind() == AND |
| 133 | + || fieldNode.getKind() == MINUS |
| 134 | + || fieldNode.getKind() == TUMBLE |
| 135 | + || fieldNode.getKind() == TUMBLE_START |
| 136 | + || fieldNode.getKind() == TUMBLE_END |
| 137 | + || fieldNode.getKind() == SESSION |
| 138 | + || fieldNode.getKind() == SESSION_START |
| 139 | + || fieldNode.getKind() == SESSION_END |
| 140 | + || fieldNode.getKind() == HOP |
| 141 | + || fieldNode.getKind() == HOP_START |
| 142 | + || fieldNode.getKind() == HOP_END |
| 143 | + || fieldNode.getKind() == BETWEEN |
| 144 | + || fieldNode.getKind() == IS_NULL |
| 145 | + || fieldNode.getKind() == IS_NOT_NULL |
| 146 | + || fieldNode.getKind() == CONTAINS |
| 147 | + || fieldNode.getKind() == TIMESTAMP_ADD |
| 148 | + || fieldNode.getKind() == TIMESTAMP_DIFF |
| 149 | + || fieldNode.getKind() == LIKE |
| 150 | + ) { |
| 151 | + SqlBasicCall sqlBasicCall = (SqlBasicCall) fieldNode; |
| 152 | + for (int i = 0; i < sqlBasicCall.getOperands().length; i++) { |
| 153 | + SqlNode sqlNode = sqlBasicCall.getOperands()[i]; |
| 154 | + if (sqlNode instanceof SqlLiteral) { |
| 155 | + continue; |
| 156 | + } |
108 | 157 |
|
109 | | - return fieldInfoList; |
| 158 | + if (sqlNode instanceof SqlDataTypeSpec) { |
| 159 | + continue; |
| 160 | + } |
| 161 | + extractSelectFieldToFieldInfo(sqlNode, fromNode, fieldInfoList, localTableCache); |
| 162 | + } |
| 163 | + } else if (fieldNode.getKind() == AS) { |
| 164 | + SqlNode leftNode = ((SqlBasicCall) fieldNode).getOperands()[0]; |
| 165 | + extractSelectFieldToFieldInfo(leftNode, fromNode,fieldInfoList, localTableCache); |
| 166 | + } else if (fieldNode.getKind() == CASE) { |
| 167 | + SqlCase sqlCase = (SqlCase) fieldNode; |
| 168 | + SqlNodeList whenOperands = sqlCase.getWhenOperands(); |
| 169 | + SqlNodeList thenOperands = sqlCase.getThenOperands(); |
| 170 | + SqlNode elseNode = sqlCase.getElseOperand(); |
| 171 | + |
| 172 | + for (int i = 0; i < whenOperands.size(); i++) { |
| 173 | + SqlNode oneOperand = whenOperands.get(i); |
| 174 | + extractSelectFieldToFieldInfo(oneOperand, fromNode, fieldInfoList, localTableCache); |
| 175 | + } |
| 176 | + |
| 177 | + for (int i = 0; i < thenOperands.size(); i++) { |
| 178 | + SqlNode oneOperand = thenOperands.get(i); |
| 179 | + extractSelectFieldToFieldInfo(oneOperand, fromNode, fieldInfoList, localTableCache); |
| 180 | + |
| 181 | + } |
| 182 | + |
| 183 | + extractSelectFieldToFieldInfo(elseNode, fromNode, fieldInfoList, localTableCache); |
| 184 | + } |
110 | 185 | } |
111 | 186 |
|
112 | 187 | public static String buildInternalTableName(String left, char split, String right) { |
|
0 commit comments