diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/SyntaxParseException.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/SyntaxParseException.java new file mode 100644 index 00000000000000..cced35792492ca --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/exceptions/SyntaxParseException.java @@ -0,0 +1,32 @@ +// 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 org.apache.doris.nereids.exceptions; + +import org.apache.doris.nereids.parser.Origin; + +import java.util.Optional; + +/** + * syntax parse exception + */ +public class SyntaxParseException extends ParseException { + + public SyntaxParseException(String message, Origin start, Optional command) { + super(message, start, command); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/ParseErrorListener.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/ParseErrorListener.java index 749b37eae0c214..56e56c87099627 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/ParseErrorListener.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/ParseErrorListener.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.parser; import org.apache.doris.nereids.exceptions.ParseException; +import org.apache.doris.nereids.exceptions.SyntaxParseException; import org.antlr.v4.runtime.BaseErrorListener; import org.antlr.v4.runtime.CommonToken; @@ -40,6 +41,6 @@ public void syntaxError(Recognizer recognizer, Object offendingSymbol, int } else { start = new Origin(line, charPositionInLine); } - throw new ParseException(msg, start, Optional.empty()); + throw new SyntaxParseException(msg, start, Optional.empty()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java index 3ac1d0315ab178..ad6c3bee6d0dae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java @@ -367,9 +367,12 @@ private static void logAuditLogImpl(ConnectContext ctx, String origStmt, Stateme statistics == null ? 0 : statistics.getScanBytesFromRemoteStorage()); } - boolean isAnalysisErr = ctx.getState().getStateType() == MysqlStateType.ERR - && ctx.getState().getErrType() == QueryState.ErrType.ANALYSIS_ERR; - String encryptSql = isAnalysisErr ? ctx.getState().getErrorMessage() : origStmt; + boolean isSyntaxErr = ctx.getState().getStateType() == MysqlStateType.ERR + && ctx.getState().getErrType() == QueryState.ErrType.SYNTAX_PARSE_ERR; + String encryptSql = isSyntaxErr ? "Syntax Error" : origStmt; + if (isSyntaxErr) { + auditEventBuilder.setErrorMessage("Syntax Error"); + } // We put origin query stmt at the end of audit log, for parsing the log more convenient. if (parsedStmt instanceof LogicalPlanAdapter) { LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 5b8ea1dd1e265c..cb6d36dfe83e9c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -49,6 +49,7 @@ import org.apache.doris.nereids.SqlCacheContext.CacheKeyType; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.exceptions.NotSupportedException; +import org.apache.doris.nereids.exceptions.SyntaxParseException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.minidump.MinidumpUtils; import org.apache.doris.nereids.parser.NereidsParser; @@ -428,7 +429,7 @@ protected List parseWithFallback(String originStmt, String conver SessionVariable sessionVariable) throws ConnectionException { try { return new NereidsParser().parseSQL(convertedStmt, sessionVariable); - } catch (NotSupportedException e) { + } catch (NotSupportedException | SyntaxParseException e) { List stmts = tryRetryOriginalSql(originStmt, convertedStmt, sessionVariable); if (stmts == null) { handleQueryException(e, convertedStmt, null, null); @@ -483,31 +484,34 @@ protected void handleQueryException(Throwable throwable, String origStmt, if (ctx.getMinidump() != null) { MinidumpUtils.saveMinidumpString(ctx.getMinidump(), DebugUtil.printId(ctx.queryId())); } - if (throwable instanceof ConnectionException) { + if (throwable instanceof SyntaxParseException) { + // Syntax parse exception. + Throwable e = new AnalysisException(throwable.getMessage(), throwable); + ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, e.getMessage()); + ctx.getState().setErrType(QueryState.ErrType.SYNTAX_PARSE_ERR); + } else if (throwable instanceof ConnectionException) { // Throw this exception to close the connection outside. - LOG.warn("Process one query failed because ConnectionException: ", throwable); throw (ConnectionException) throwable; } else if (throwable instanceof IOException) { // Client failed. - LOG.warn("Process one query failed because IOException: ", throwable); ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, "Doris process failed: " + throwable.getMessage()); } else if (throwable instanceof UserException) { - LOG.warn("Process one query failed because.", throwable); ctx.getState().setError(((UserException) throwable).getMysqlErrorCode(), throwable.getMessage()); // set it as ANALYSIS_ERR so that it won't be treated as a query failure. ctx.getState().setErrType(QueryState.ErrType.ANALYSIS_ERR); } else if (throwable instanceof NotSupportedException) { - LOG.warn("Process one query failed because.", throwable); ctx.getState().setError(ErrorCode.ERR_NOT_SUPPORTED_YET, throwable.getMessage()); // set it as ANALYSIS_ERR so that it won't be treated as a query failure. ctx.getState().setErrType(QueryState.ErrType.ANALYSIS_ERR); } else { // Catch all throwable. // If reach here, maybe palo bug. - LOG.warn("Process one query failed because unknown reason: ", throwable); ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, throwable.getClass().getSimpleName() + ", msg: " + throwable.getMessage()); } + if (LOG.isDebugEnabled()) { + LOG.debug("Process one query failed because: {}", throwable.getMessage()); + } auditAfterExec(origStmt, parsedStmt, statistics, true); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/QueryState.java b/fe/fe-core/src/main/java/org/apache/doris/qe/QueryState.java index 89c44babad911b..e4592b2b983bfb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/QueryState.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/QueryState.java @@ -36,6 +36,7 @@ public enum MysqlStateType { public enum ErrType { ANALYSIS_ERR, + SYNTAX_PARSE_ERR, OTHER_ERR } @@ -59,6 +60,7 @@ public QueryState() { public void reset() { stateType = MysqlStateType.OK; errorCode = null; + errType = ErrType.OTHER_ERR; infoMessage = null; errorMessage = ""; serverStatus = 0; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/EncryptSQLTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/EncryptSQLTest.java index b4e19fa55b5317..e41245b7cf43ce 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/EncryptSQLTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/EncryptSQLTest.java @@ -17,14 +17,18 @@ package org.apache.doris.nereids.parser; -import org.apache.doris.analysis.StatementBase; +import org.apache.doris.analysis.AccessTestUtil; import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.plugin.AuditEvent; -import org.apache.doris.qe.AuditLogHelper; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.MysqlConnectProcessor; +import org.apache.doris.qe.StmtExecutor; import org.apache.doris.resource.workloadschedpolicy.WorkloadRuntimeStatusMgr; +import mockit.Mock; +import mockit.MockUp; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -32,14 +36,23 @@ public class EncryptSQLTest extends ParserTestBase { - NereidsParser parser = new NereidsParser(); ConnectContext ctx = ConnectContext.get(); WorkloadRuntimeStatusMgr mgr = Env.getCurrentEnv().getWorkloadRuntimeStatusMgr(); List auditEvents = Deencapsulation.getField(mgr, "queryAuditEventList"); + MysqlConnectProcessor processor = new MysqlConnectProcessor(ctx); + Env env = AccessTestUtil.fetchAdminCatalog(); @Test - public void testEncryption() { + public void testEncryption() throws Exception { ctx.setDatabase("test"); + new MockUp() { + @Mock + public boolean isForwardToMaster() { + return false; + } + }; + ctx.setEnv(env); + Config.enable_nereids_load = true; String sql = "EXPORT TABLE export_table TO \"s3://abc/aaa\" " + "PROPERTIES(" @@ -182,71 +195,71 @@ public void testEncryption() { + ")"; res = "CREATE EXTERNAL TABLE broker_tbl(" - + " k1 tinyint," - + " k2 smallint," - + " k3 int," - + " k4 bigint) " - + "ENGINE=broker " - + "PROPERTIES(" - + " \"broker_name\" = \"hdfs\"," - + " \"path\" = \"hdfs://abc/qe/a.txt\"" - + ") " - + "BROKER PROPERTIES(" - + " \"username\" = \"root\"," - + " \"password\" = \"*XXX\"" - + ")"; + + " k1 tinyint," + + " k2 smallint," + + " k3 int," + + " k4 bigint) " + + "ENGINE=broker " + + "PROPERTIES(" + + " \"broker_name\" = \"hdfs\"," + + " \"path\" = \"hdfs://abc/qe/a.txt\"" + + ") " + + "BROKER PROPERTIES(" + + " \"username\" = \"root\"," + + " \"password\" = \"*XXX\"" + + ")"; parseAndCheck(sql, res); sql = "INSERT INTO test_s3load " - + "SELECT * FROM s3_tbl(" - + " \"uri\" = \"s3://your_bucket_name/s3load_example.csv\"," - + " \"format\" = \"csv\"," - + " \"provider\" = \"OSS\"," - + " \"s3.endpoint\" = \"oss-cn-hangzhou.aliyuncs.com\"," - + " \"s3.region\" = \"oss-cn-hangzhou\"," - + " \"s3.access_key\" = \"abc\"," - + " \"s3.secret_key\" = \"abc\"," - + " \"column_separator\" = \",\"," - + " \"csv_schema\" = \"user_id:int;name:string;age:int\"" - + ")"; + + "SELECT * FROM s3_tbl(" + + " \"uri\" = \"s3://your_bucket_name/s3load_example.csv\"," + + " \"format\" = \"csv\"," + + " \"provider\" = \"OSS\"," + + " \"s3.endpoint\" = \"oss-cn-hangzhou.aliyuncs.com\"," + + " \"s3.region\" = \"oss-cn-hangzhou\"," + + " \"s3.access_key\" = \"abc\"," + + " \"s3.secret_key\" = \"abc\"," + + " \"column_separator\" = \",\"," + + " \"csv_schema\" = \"user_id:int;name:string;age:int\"" + + ")"; res = "INSERT INTO test_s3load " - + "SELECT * FROM s3_tbl(" - + " \"uri\" = \"s3://your_bucket_name/s3load_example.csv\"," - + " \"format\" = \"csv\"," - + " \"provider\" = \"OSS\"," - + " \"s3.endpoint\" = \"oss-cn-hangzhou.aliyuncs.com\"," - + " \"s3.region\" = \"oss-cn-hangzhou\"," - + " \"s3.access_key\" = \"abc\"," - + " \"s3.secret_key\" = \"*XXX\"," - + " \"column_separator\" = \",\"," - + " \"csv_schema\" = \"user_id:int;name:string;age:int\"" - + ")"; + + "SELECT * FROM s3_tbl(" + + " \"uri\" = \"s3://your_bucket_name/s3load_example.csv\"," + + " \"format\" = \"csv\"," + + " \"provider\" = \"OSS\"," + + " \"s3.endpoint\" = \"oss-cn-hangzhou.aliyuncs.com\"," + + " \"s3.region\" = \"oss-cn-hangzhou\"," + + " \"s3.access_key\" = \"abc\"," + + " \"s3.secret_key\" = \"*XXX\"," + + " \"column_separator\" = \",\"," + + " \"csv_schema\" = \"user_id:int;name:string;age:int\"" + + ")"; parseAndCheck(sql, res); sql = "SELECT * FROM s3_tbl(" - + " \"uri\" = \"s3://your_bucket_name/s3load_example.csv\"," - + " \"format\" = \"csv\"," - + " \"provider\" = \"OSS\"," - + " \"s3.endpoint\" = \"oss-cn-hangzhou.aliyuncs.com\"," - + " \"s3.region\" = \"oss-cn-hangzhou\"," - + " \"s3.access_key\" = \"abc\"," - + " \"s3.secret_key\" = \"abc\"," - + " \"column_separator\" = \",\"," - + " \"csv_schema\" = \"user_id:int;name:string;age:int\"" - + ")"; + + " \"uri\" = \"s3://your_bucket_name/s3load_example.csv\"," + + " \"format\" = \"csv\"," + + " \"provider\" = \"OSS\"," + + " \"s3.endpoint\" = \"oss-cn-hangzhou.aliyuncs.com\"," + + " \"s3.region\" = \"oss-cn-hangzhou\"," + + " \"s3.access_key\" = \"abc\"," + + " \"s3.secret_key\" = \"abc\"," + + " \"column_separator\" = \",\"," + + " \"csv_schema\" = \"user_id:int;name:string;age:int\"" + + ")"; res = "SELECT * FROM s3_tbl(" - + " \"uri\" = \"s3://your_bucket_name/s3load_example.csv\"," - + " \"format\" = \"csv\"," - + " \"provider\" = \"OSS\"," - + " \"s3.endpoint\" = \"oss-cn-hangzhou.aliyuncs.com\"," - + " \"s3.region\" = \"oss-cn-hangzhou\"," - + " \"s3.access_key\" = \"abc\"," - + " \"s3.secret_key\" = \"*XXX\"," - + " \"column_separator\" = \",\"," - + " \"csv_schema\" = \"user_id:int;name:string;age:int\"" - + ")"; + + " \"uri\" = \"s3://your_bucket_name/s3load_example.csv\"," + + " \"format\" = \"csv\"," + + " \"provider\" = \"OSS\"," + + " \"s3.endpoint\" = \"oss-cn-hangzhou.aliyuncs.com\"," + + " \"s3.region\" = \"oss-cn-hangzhou\"," + + " \"s3.access_key\" = \"abc\"," + + " \"s3.secret_key\" = \"*XXX\"," + + " \"column_separator\" = \",\"," + + " \"csv_schema\" = \"user_id:int;name:string;age:int\"" + + ")"; parseAndCheck(sql, res); sql = "SET LDAP_ADMIN_PASSWORD = PASSWORD('123456')"; @@ -316,11 +329,23 @@ public void testEncryption() { + " \"s3.secret_key\" = \"*XXX\"" + " );"; parseAndCheck(sql, res); + + sql = "selected * from tbl"; + res = "Syntax Error"; + parseAndCheck(sql, res); + + sql = "select * from tbl"; + res = "select * from tbl"; + processor.executeQuery(sql); + AuditEvent event = auditEvents.get(auditEvents.size() - 1); + Assertions.assertEquals(res, event.stmt); + + String errorMsg = "errCode = 2, detailMessage = Database [test] does not exist."; + Assertions.assertTrue(event.errorMessage.contains(errorMsg)); } - private void parseAndCheck(String sql, String expected) { - StatementBase parsedStmt = parser.parseSQL(sql).get(0); - AuditLogHelper.logAuditLog(ctx, sql, parsedStmt, null, false); + private void parseAndCheck(String sql, String expected) throws Exception { + processor.executeQuery(sql); AuditEvent event = auditEvents.get(auditEvents.size() - 1); Assertions.assertEquals(expected, event.stmt); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java index 0d173aed5ce467..0de0b043f36dfd 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.exceptions.ParseException; +import org.apache.doris.nereids.exceptions.SyntaxParseException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; @@ -91,7 +92,7 @@ public void testParseMultiple() { public void testParseMultipleError() { NereidsParser nereidsParser = new NereidsParser(); String sql = "SELECT b FROM test SELECT a FROM test;"; - Assertions.assertThrowsExactly(ParseException.class, () -> nereidsParser.parseMultiple(sql)); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> nereidsParser.parseMultiple(sql)); } @Test @@ -111,7 +112,7 @@ public void testSingle() { @Test public void testErrorListener() { parsePlan("select * from t1 where a = 1 illegal_symbol") - .assertThrowsExactly(ParseException.class) + .assertThrowsExactly(SyntaxParseException.class) .assertMessageEquals("\nextraneous input 'illegal_symbol' expecting {, ';'}(line 1, pos 29)\n"); } @@ -160,7 +161,7 @@ public void testParseWindowFunctions() { Assertions.assertEquals(((LogicalAggregate) logicalPlan).getOutputExpressions().size(), 3); String windowSql3 = "select rank() over from t1"; - parsePlan(windowSql3).assertThrowsExactly(ParseException.class) + parsePlan(windowSql3).assertThrowsExactly(SyntaxParseException.class) .assertMessageContains("mismatched input 'from' expecting '('"); } @@ -395,7 +396,7 @@ public void testJoinHint() { // invalid hint position parsePlan("select * from [shuffle] t1 join t2 on t1.keyy=t2.keyy") - .assertThrowsExactly(ParseException.class); + .assertThrowsExactly(SyntaxParseException.class); // invalid hint content parsePlan("select * from t1 join [bucket] t2 on t1.keyy=t2.keyy") @@ -409,7 +410,7 @@ public void testJoinHint() { // invalid multiple hints parsePlan("select * from t1 join [shuffle,broadcast] t2 on t1.keyy=t2.keyy") - .assertThrowsExactly(ParseException.class); + .assertThrowsExactly(SyntaxParseException.class); } @Test @@ -991,10 +992,10 @@ public void testTrim() { Assertions.assertEquals(1, unboundFunction.arity()); Assertions.assertEquals("1", ((StringLikeLiteral) unboundFunction.child(0)).getStringValue()); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("trim(invalid '2' from '1')")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("trim(invalid '2' '1')")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("trim(from '1')")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("trim(both '1')")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("trim(invalid '2' from '1')")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("trim(invalid '2' '1')")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("trim(from '1')")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("trim(both '1')")); } @Test @@ -1042,11 +1043,11 @@ public void testSubstring() { Assertions.assertEquals("Quadratically", ((StringLikeLiteral) unboundFunction.child(0)).getStringValue()); Assertions.assertEquals(5, ((IntegerLikeLiteral) unboundFunction.child(1)).getIntValue()); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substring('Sakila' for 2)")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substring('Sakila' from for)")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substring('Sakila' from)")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substring(from 1)")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substring(for 1)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substring('Sakila' for 2)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substring('Sakila' from for)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substring('Sakila' from)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substring(from 1)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substring(for 1)")); } @Test @@ -1094,11 +1095,11 @@ public void testSubstr() { Assertions.assertEquals("Quadratically", ((StringLikeLiteral) unboundFunction.child(0)).getStringValue()); Assertions.assertEquals(5, ((IntegerLikeLiteral) unboundFunction.child(1)).getIntValue()); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substr('Sakila' for 2)")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substr('Sakila' from for)")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substr('Sakila' from)")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substr(from 1)")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("substr(for 1)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substr('Sakila' for 2)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substr('Sakila' from for)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substr('Sakila' from)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substr(from 1)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("substr(for 1)")); } @Test @@ -1117,9 +1118,9 @@ public void testPositon() { Assertions.assertEquals("bar", ((StringLikeLiteral) unboundFunction.child(0)).getStringValue()); Assertions.assertEquals("foobarbar", ((StringLikeLiteral) unboundFunction.child(1)).getStringValue()); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("position('bar' in)")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("position(in 'foobarbar')")); - Assertions.assertThrowsExactly(ParseException.class, () -> parser.parseExpression("position(in)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("position('bar' in)")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("position(in 'foobarbar')")); + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> parser.parseExpression("position(in)")); } @Test @@ -1157,7 +1158,7 @@ private void testNoBackSlashEscapes(String sql, String onResult, String offResul try (MockedStatic helperMockedStatic = Mockito.mockStatic(SqlModeHelper.class)) { helperMockedStatic.when(SqlModeHelper::hasNoBackSlashEscapes).thenReturn(true); if (onResult == null) { - Assertions.assertThrowsExactly(ParseException.class, () -> nereidsParser.parseExpression(sql), + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> nereidsParser.parseExpression(sql), "should failed when NO_BACKSLASH_ESCAPES = 1: " + sql); } else { Assertions.assertEquals(onResult, @@ -1169,7 +1170,7 @@ private void testNoBackSlashEscapes(String sql, String onResult, String offResul try (MockedStatic helperMockedStatic = Mockito.mockStatic(SqlModeHelper.class)) { helperMockedStatic.when(SqlModeHelper::hasNoBackSlashEscapes).thenReturn(false); if (offResult == null) { - Assertions.assertThrowsExactly(ParseException.class, () -> nereidsParser.parseExpression(sql), + Assertions.assertThrowsExactly(SyntaxParseException.class, () -> nereidsParser.parseExpression(sql), "should failed when NO_BACKSLASH_ESCAPES = 0: " + sql); } else { Assertions.assertEquals(offResult, @@ -1209,35 +1210,35 @@ public void testComment() { @Test public void testLambdaSelect() { parsePlan("SELECT x -> x + 1") - .assertThrowsExactly(ParseException.class) + .assertThrowsExactly(SyntaxParseException.class) .assertMessageContains("mismatched input '->' expecting {, ';'}"); } @Test public void testLambdaGroupBy() { parsePlan("SELECT 1 from ( select 2 ) t group by x -> x + 1") - .assertThrowsExactly(ParseException.class) + .assertThrowsExactly(SyntaxParseException.class) .assertMessageContains("mismatched input '->' expecting {, ';'}"); } @Test public void testLambdaSort() { parsePlan("SELECT 1 from ( select 2 ) t order by x -> x + 1") - .assertThrowsExactly(ParseException.class) + .assertThrowsExactly(SyntaxParseException.class) .assertMessageContains("mismatched input '->' expecting {, ';'}"); } @Test public void testLambdaHaving() { parsePlan("SELECT 1 from ( select 2 ) t having x -> x + 1") - .assertThrowsExactly(ParseException.class) + .assertThrowsExactly(SyntaxParseException.class) .assertMessageContains("mismatched input '->' expecting {, ';'}"); } @Test public void testLambdaJoin() { parsePlan("SELECT 1 from ( select 2 as a1 ) t1 join ( select 2 as a2 ) as t2 on x -> x + 1 = t1.a1") - .assertThrowsExactly(ParseException.class) + .assertThrowsExactly(SyntaxParseException.class) .assertMessageContains("mismatched input '->' expecting {, ';'}"); } @@ -1297,7 +1298,7 @@ public void testAdminRotateTdeRootKey() { nereidsParser.parseSingle(sql); parsePlan("admin rotate tde root key properties()") - .assertThrowsExactly(ParseException.class) + .assertThrowsExactly(SyntaxParseException.class) .assertMessageContains("mismatched input ')' expecting"); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java index f33bd493824845..41017520046151 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ExpressionParserTest.java @@ -18,7 +18,7 @@ package org.apache.doris.nereids.trees.expressions; import org.apache.doris.nereids.analyzer.UnboundSlot; -import org.apache.doris.nereids.exceptions.ParseException; +import org.apache.doris.nereids.exceptions.SyntaxParseException; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.parser.ParserTestBase; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; @@ -149,7 +149,7 @@ public void testExprArithmetic() { assertExpr(subtract); parseExpression("3 += 2") - .assertThrowsExactly(ParseException.class) + .assertThrowsExactly(SyntaxParseException.class) .assertMessageContains("extraneous input '=' expecting {'("); }