|
26 | 26 | import com.google.common.base.Preconditions; |
27 | 27 | import com.google.common.base.Strings; |
28 | 28 | import com.google.common.collect.HashBasedTable; |
29 | | -import com.google.common.collect.HashBiMap; |
30 | 29 | import com.google.common.collect.Lists; |
31 | 30 | import org.apache.calcite.sql.SqlAsOperator; |
32 | 31 | import org.apache.calcite.sql.SqlBasicCall; |
|
54 | 53 | import java.util.regex.Matcher; |
55 | 54 | import java.util.regex.Pattern; |
56 | 55 |
|
57 | | -import static org.apache.calcite.sql.SqlKind.*; |
| 56 | +import static org.apache.calcite.sql.SqlKind.AGGREGATE; |
| 57 | +import static org.apache.calcite.sql.SqlKind.AND; |
| 58 | +import static org.apache.calcite.sql.SqlKind.AS; |
| 59 | +import static org.apache.calcite.sql.SqlKind.AVG_AGG_FUNCTIONS; |
| 60 | +import static org.apache.calcite.sql.SqlKind.BETWEEN; |
58 | 61 | import static org.apache.calcite.sql.SqlKind.CASE; |
| 62 | +import static org.apache.calcite.sql.SqlKind.CAST; |
| 63 | +import static org.apache.calcite.sql.SqlKind.COALESCE; |
| 64 | +import static org.apache.calcite.sql.SqlKind.COMPARISON; |
| 65 | +import static org.apache.calcite.sql.SqlKind.CONTAINS; |
| 66 | +import static org.apache.calcite.sql.SqlKind.DIVIDE; |
| 67 | +import static org.apache.calcite.sql.SqlKind.EQUALS; |
| 68 | +import static org.apache.calcite.sql.SqlKind.HOP; |
| 69 | +import static org.apache.calcite.sql.SqlKind.HOP_END; |
| 70 | +import static org.apache.calcite.sql.SqlKind.HOP_START; |
| 71 | +import static org.apache.calcite.sql.SqlKind.IDENTIFIER; |
| 72 | +import static org.apache.calcite.sql.SqlKind.IS_NOT_NULL; |
| 73 | +import static org.apache.calcite.sql.SqlKind.IS_NULL; |
| 74 | +import static org.apache.calcite.sql.SqlKind.LIKE; |
| 75 | +import static org.apache.calcite.sql.SqlKind.LITERAL; |
| 76 | +import static org.apache.calcite.sql.SqlKind.LITERAL_CHAIN; |
| 77 | +import static org.apache.calcite.sql.SqlKind.MINUS; |
| 78 | +import static org.apache.calcite.sql.SqlKind.NOT_IN; |
| 79 | +import static org.apache.calcite.sql.SqlKind.OR; |
59 | 80 | import static org.apache.calcite.sql.SqlKind.OTHER; |
| 81 | +import static org.apache.calcite.sql.SqlKind.OTHER_FUNCTION; |
| 82 | +import static org.apache.calcite.sql.SqlKind.PLUS; |
| 83 | +import static org.apache.calcite.sql.SqlKind.SELECT; |
| 84 | +import static org.apache.calcite.sql.SqlKind.SESSION; |
| 85 | +import static org.apache.calcite.sql.SqlKind.SESSION_END; |
| 86 | +import static org.apache.calcite.sql.SqlKind.SESSION_START; |
| 87 | +import static org.apache.calcite.sql.SqlKind.TIMES; |
| 88 | +import static org.apache.calcite.sql.SqlKind.TIMESTAMP_ADD; |
| 89 | +import static org.apache.calcite.sql.SqlKind.TIMESTAMP_DIFF; |
| 90 | +import static org.apache.calcite.sql.SqlKind.TRIM; |
| 91 | +import static org.apache.calcite.sql.SqlKind.TUMBLE; |
| 92 | +import static org.apache.calcite.sql.SqlKind.TUMBLE_END; |
| 93 | +import static org.apache.calcite.sql.SqlKind.TUMBLE_START; |
| 94 | +import static org.apache.calcite.sql.SqlKind.UNION; |
60 | 95 |
|
61 | 96 | /** |
62 | 97 | * 表的解析相关 |
@@ -798,4 +833,89 @@ public static boolean checkIsTimeGroupByFunction(String functionName ){ |
798 | 833 | || functionName.equalsIgnoreCase("hop"); |
799 | 834 | } |
800 | 835 |
|
| 836 | + /** |
| 837 | + * 判断group by中是否包含维表,包含则需要撤回,不管嵌套多少层子查询只要有一层包含都需要撤回 |
| 838 | + * |
| 839 | + * @param sqlNode sql语句 |
| 840 | + * @param newRegisterTableList 维表集合 |
| 841 | + * @return true:需要撤回,false:和原生保持一样 |
| 842 | + */ |
| 843 | + public static boolean checkIsDimTableGroupBy(SqlNode sqlNode, Collection<String> newRegisterTableList) { |
| 844 | + // 维表集合为空 |
| 845 | + if (newRegisterTableList == null || newRegisterTableList.size() == 0) { |
| 846 | + return false; |
| 847 | + } |
| 848 | + SqlKind sqlKind = sqlNode.getKind(); |
| 849 | + switch (sqlKind) { |
| 850 | + case SELECT: |
| 851 | + SqlSelect selectNode = (SqlSelect) sqlNode; |
| 852 | + SqlNodeList groupNodeList = selectNode.getGroup(); |
| 853 | + SqlNode fromNode = selectNode.getFrom(); |
| 854 | + |
| 855 | + // 1.(sub query) group by |
| 856 | + // 2.(sub query) as alias group by |
| 857 | + // 3.tableName group by |
| 858 | + // 4.tableName as alias group by |
| 859 | + |
| 860 | + // (子查询) group by:1.(sub query) group by |
| 861 | + if (fromNode.getKind() == SELECT) { |
| 862 | + return checkIsDimTableGroupBy(fromNode, newRegisterTableList); |
| 863 | + } |
| 864 | + |
| 865 | + // 表名 as 别名 group by、(子查询) as 别名 group by、表名 group by |
| 866 | + if (fromNode.getKind() == AS || fromNode.getKind() == IDENTIFIER) { |
| 867 | + SqlNode operand; |
| 868 | + // 表名 as 别名 group by:4.tableName as alias group by |
| 869 | + if (fromNode.getKind() == AS) { |
| 870 | + operand = ((SqlBasicCall) fromNode).getOperands()[0]; |
| 871 | + } else { |
| 872 | + // 表名 group by:3.tableName group by |
| 873 | + operand = fromNode; |
| 874 | + } |
| 875 | + // 最里层是表名 group by,且group by字段不为空,且表名包含在维表中 |
| 876 | + if (operand.getKind() == IDENTIFIER |
| 877 | + && groupNodeList != null |
| 878 | + && groupNodeList.size() != 0 |
| 879 | + && newRegisterTableList.contains(operand.toString())) { |
| 880 | + boolean isRetract = false; |
| 881 | + // 判断完所有的group by字段 |
| 882 | + for (SqlNode node : groupNodeList.getList()) { |
| 883 | + // 判断是否有函数 |
| 884 | + if (node.getKind() == OTHER_FUNCTION) { |
| 885 | + String functionName = ((SqlBasicCall) node).getOperator().toString().toLowerCase(); |
| 886 | + boolean isTimeGroupByFunction = checkIsTimeGroupByFunction(functionName); |
| 887 | + // 只要有窗口就不需要撤回,直接返回 |
| 888 | + if (isTimeGroupByFunction) { |
| 889 | + return false; |
| 890 | + } |
| 891 | + // 非窗口需要撤回,继续迭代后面的字段 |
| 892 | + isRetract = true; |
| 893 | + } else { |
| 894 | + // 其他情况需要撤回,继续迭代后面的字段 |
| 895 | + isRetract = true; |
| 896 | + } |
| 897 | + } |
| 898 | + return isRetract; |
| 899 | + } else { |
| 900 | + // (子查询) as 别名 group by:2.(sub query) as alias group by |
| 901 | + // 没有group by语句也会走进来,但是最后会返回不需要撤回 |
| 902 | + return checkIsDimTableGroupBy(fromNode, newRegisterTableList); |
| 903 | + } |
| 904 | + } |
| 905 | + |
| 906 | + return false; |
| 907 | + case INSERT: |
| 908 | + return checkIsDimTableGroupBy(((SqlInsert) sqlNode).getSource(), newRegisterTableList); |
| 909 | + case UNION: |
| 910 | + SqlNode unionLeft = ((SqlBasicCall) sqlNode).getOperands()[0]; |
| 911 | + SqlNode unionRight = ((SqlBasicCall) sqlNode).getOperands()[1]; |
| 912 | + return checkIsDimTableGroupBy(unionLeft, newRegisterTableList) |
| 913 | + || checkIsDimTableGroupBy(unionRight, newRegisterTableList); |
| 914 | + case AS: |
| 915 | + SqlNode info = ((SqlBasicCall) sqlNode).getOperands()[0]; |
| 916 | + return checkIsDimTableGroupBy(info, newRegisterTableList); |
| 917 | + default: |
| 918 | + return false; |
| 919 | + } |
| 920 | + } |
801 | 921 | } |
0 commit comments