[jira] [Comment Edited] (CALCITE-2844) Babel parser should parse PostgreSQL table functions
[ https://issues.apache.org/jira/browse/CALCITE-2844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16807878#comment-16807878 ] Hongze Zhang edited comment on CALCITE-2844 at 4/2/19 3:50 PM: --- I've run the two tests and reproduced the failures. The first one {{testOuterApplyFunctionFails}} fails because the table function "ramp(deptno)" is now get successfully parsed, as it was previously not a legal syntax. Besides, the SQL used in the case is wired since it ends with two right brackets "{{))}}": {code:sql} select * from dept outer apply ramp(deptno^)^) {code} In general it should be {code:sql} select * from dept outer apply ramp(deptno^)^ {code} The second bracket looks like a typo. I think you can just remove the bracket and error anchors (the "^"). Then let the test case pass. The case {{testTableExtend}} fails because of bad lookahead. I see your LOOKAHEAD hint is set as {code} LOOKAHEAD({ (getToken(1).kind == IDENTIFIER && getToken(2).kind == LPAREN) || (getToken(1).kind == SPECIFIC && getToken(2).kind == IDENTIFIER && getToken(3).kind == LPAREN) }) {code} The hint is conflict with "extend clause" (see CALCITE-493). There should be multiple solutions to solve the conflict: using syntactic lookahead, or changing the expansion match order, etc. Regarding the parsing performance I would prefer to change the match order. was (Author: zhztheplayer): I've run the two tests and reproduced the failures. The first one {{testOuterApplyFunctionFails}} fails because the table function "ramp(deptno)" is now get successfully parsed, as it was previously not a legal syntax. Besides, the SQL used in the case is wired since it ends with two right brackets "{{))}}": {code:sql} select * from dept outer apply ramp(deptno^)^) {code} In general it should be {code:sql} select * from dept outer apply ramp(deptno^)^ {code} The second bracket looks like a typo. I think you can just remove the bracket and error anchors (the "^"). Then let the test case pass. The case {{testTableExtend}} fails because of bad lookahead. I see your LOOKAHEAD hint is set as {code} LOOKAHEAD({ (getToken(1).kind == IDENTIFIER && getToken(2).kind == LPAREN) || (getToken(1).kind == SPECIFIC && getToken(2).kind == IDENTIFIER && getToken(3).kind == LPAREN) }) {code} The hint is conflict with "extend clause" (see CALCITE-493). There should be multiple solutions to solve the conflict (using syntactic lookahead, or changing the expansion match order, etc.) > Babel parser should parse PostgreSQL table functions > > > Key: CALCITE-2844 > URL: https://issues.apache.org/jira/browse/CALCITE-2844 > Project: Calcite > Issue Type: Bug > Components: babel >Affects Versions: 1.18.0 >Reporter: Muhammad Gelbana >Priority: Major > > *Code to reproduce the problem* > {code:java} > public static void main(String[] args) throws Exception { > Config parserConfig = > configBuilder().setConformance(SqlConformanceEnum.BABEL).setParserFactory(SqlBabelParserImpl.FACTORY).build(); > FrameworkConfig frameworkConfig = > Frameworks.newConfigBuilder().parserConfig(parserConfig).build(); > Planner planner = Frameworks.getPlanner(frameworkConfig); > String pg = "select * from generate_series(1, 5)"; > planner.parse(pg); > } > {code} > > *Thrown exception* > {noformat} > Exception in thread "main" org.apache.calcite.sql.parser.SqlParseException: > Encountered "( 1" at line 1, column 30. > Was expecting one of: > > "ORDER" ... > "LIMIT" ... > "OFFSET" ... > "FETCH" ... > "NATURAL" ... > "JOIN" ... > "INNER" ... > "LEFT" ... > "RIGHT" ... > "FULL" ... > "CROSS" ... > "," ... > "OUTER" ... > "(" ... > "(" ... > "(" ... > "(" ... > "(" ... > "MATCH_RECOGNIZE" ... > "." ... > "(" ... > ... > ... > ... > ... > ... > "TABLESAMPLE" ... > "WHERE" ... > "GROUP" ... > "HAVING" ... > "WINDOW" ... > "UNION" ... > "INTERSECT" ... > "EXCEPT" ... > "MINUS" ... > > at > org.apache.calcite.sql.parser.babel.SqlBabelParserImpl.convertException(SqlBabelParserImpl.java:354) > at > org.apache.calcite.sql.parser.babel.SqlBabelParserImpl.normalizeException(SqlBabelParserImpl.java:142) > at org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:156) > at org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:181) > at org.apache.calcite.prepare.PlannerImpl.parse(PlannerImpl.java:174) > at org.apache.calcite.tools.Planner.parse(Planner.java:50) > at com.lab.calcite.App2.main(App2.java:24) > Caused by: org.apache.calcite.sql.parser.babel.ParseException: Encountered "( > 1" at line 1
[jira] [Comment Edited] (CALCITE-2844) Babel parser should parse PostgreSQL table functions
[ https://issues.apache.org/jira/browse/CALCITE-2844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16807878#comment-16807878 ] Hongze Zhang edited comment on CALCITE-2844 at 4/2/19 3:48 PM: --- I've run the two tests and reproduced the failures. The first one {{testOuterApplyFunctionFails}} fails because the table function "ramp(deptno)" is now get successfully parsed, as it was previously not a legal syntax. Besides, the SQL used in the case is wired since it ends with two right brackets "{{))}}": {code:sql} select * from dept outer apply ramp(deptno^)^) {code} In general it should be {code:sql} select * from dept outer apply ramp(deptno^)^ {code} The second bracket looks like a typo. I think you can just remove the bracket and error anchors (the "^"). Then let the test case pass. The case {{testTableExtend}} fails because of bad lookahead. I see your LOOKAHEAD hint is set as {code} LOOKAHEAD({ (getToken(1).kind == IDENTIFIER && getToken(2).kind == LPAREN) || (getToken(1).kind == SPECIFIC && getToken(2).kind == IDENTIFIER && getToken(3).kind == LPAREN) }) {code} The hint is conflict with "extend clause" (see CALCITE-493). There should be multiple solutions to solve the conflict (using syntactic lookahead, or changing the expansion match order, etc.) was (Author: zhztheplayer): I've run the two tests and reproduced the failures. The first one {{testOuterApplyFunctionFails}} fails because the table function "ramp(deptno)" is now get successfully parsed, as it was previously not a legal syntax. Besides, the SQL used in the case is wired since it ends with two right brackets "{{))}}": {code:sql} select * from dept outer apply ramp(deptno^)^) {code} In general it should be {code:sql} select * from dept outer apply ramp(deptno^)^ {code} The second bracket looks like a typo. I think you can just remove the bracket and error anchors (the "^"). Then let the test case pass. The case {{testTableExtend}} fails because of bad lookahead. I see your LOOKAHEAD hint is set as {code} LOOKAHEAD({ (getToken(1).kind == IDENTIFIER && getToken(2).kind == LPAREN) || (getToken(1).kind == SPECIFIC && getToken(2).kind == IDENTIFIER && getToken(3).kind == LPAREN) }) {code} The hint is conflict with "extend clause" (see CALCITE-493). There should be multiple solutions to solve the conflict (using syntactic lookahead, or changing the expansion match order, etc. Maybe the later is better?) > Babel parser should parse PostgreSQL table functions > > > Key: CALCITE-2844 > URL: https://issues.apache.org/jira/browse/CALCITE-2844 > Project: Calcite > Issue Type: Bug > Components: babel >Affects Versions: 1.18.0 >Reporter: Muhammad Gelbana >Priority: Major > > *Code to reproduce the problem* > {code:java} > public static void main(String[] args) throws Exception { > Config parserConfig = > configBuilder().setConformance(SqlConformanceEnum.BABEL).setParserFactory(SqlBabelParserImpl.FACTORY).build(); > FrameworkConfig frameworkConfig = > Frameworks.newConfigBuilder().parserConfig(parserConfig).build(); > Planner planner = Frameworks.getPlanner(frameworkConfig); > String pg = "select * from generate_series(1, 5)"; > planner.parse(pg); > } > {code} > > *Thrown exception* > {noformat} > Exception in thread "main" org.apache.calcite.sql.parser.SqlParseException: > Encountered "( 1" at line 1, column 30. > Was expecting one of: > > "ORDER" ... > "LIMIT" ... > "OFFSET" ... > "FETCH" ... > "NATURAL" ... > "JOIN" ... > "INNER" ... > "LEFT" ... > "RIGHT" ... > "FULL" ... > "CROSS" ... > "," ... > "OUTER" ... > "(" ... > "(" ... > "(" ... > "(" ... > "(" ... > "MATCH_RECOGNIZE" ... > "." ... > "(" ... > ... > ... > ... > ... > ... > "TABLESAMPLE" ... > "WHERE" ... > "GROUP" ... > "HAVING" ... > "WINDOW" ... > "UNION" ... > "INTERSECT" ... > "EXCEPT" ... > "MINUS" ... > > at > org.apache.calcite.sql.parser.babel.SqlBabelParserImpl.convertException(SqlBabelParserImpl.java:354) > at > org.apache.calcite.sql.parser.babel.SqlBabelParserImpl.normalizeException(SqlBabelParserImpl.java:142) > at org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:156) > at org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:181) > at org.apache.calcite.prepare.PlannerImpl.parse(PlannerImpl.java:174) > at org.apache.calcite.tools.Planner.parse(Planner.java:50) > at com.lab.calcite.App2.main(App2.java:24) > Caused by: org.apache.calcite.sql.parser.babel.ParseException: Encountered "( > 1" at line 1, column 30. > Was expecting one of: > >