Skip to content

Commit

Permalink
Add new error code and handling: QUERY_EXCEEDED_COMPILER_LIMIT
Browse files Browse the repository at this point in the history
  • Loading branch information
mduggan-starburst committed Oct 3, 2024
1 parent 2ce6829 commit ab2456c
Show file tree
Hide file tree
Showing 5 changed files with 18 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@
import static io.trino.cache.SafeCaches.buildNonEvictableCache;
import static io.trino.operator.project.PageFieldsToInputParametersRewriter.rewritePageFieldsToInputParameters;
import static io.trino.spi.StandardErrorCode.COMPILER_ERROR;
import static io.trino.spi.StandardErrorCode.QUERY_EXCEEDED_COMPILER_LIMIT;
import static io.trino.sql.gen.BytecodeUtils.generateWrite;
import static io.trino.sql.gen.BytecodeUtils.invoke;
import static io.trino.sql.gen.LambdaExpressionExtractor.extractLambdaExpressions;
Expand Down Expand Up @@ -206,7 +207,7 @@ private Supplier<PageProjection> compileProjectionInternal(RowExpression project
}
catch (Exception e) {
if (Throwables.getRootCause(e) instanceof MethodTooLargeException) {
throw new TrinoException(COMPILER_ERROR,
throw new TrinoException(QUERY_EXCEEDED_COMPILER_LIMIT,
"Query exceeded maximum columns. Please reduce the number of columns referenced and re-run the query.", e);
}
throw new TrinoException(COMPILER_ERROR, e);
Expand Down Expand Up @@ -400,7 +401,7 @@ private Supplier<PageFilter> compileFilterInternal(RowExpression filter, Optiona
}
catch (Exception e) {
if (Throwables.getRootCause(e) instanceof MethodTooLargeException) {
throw new TrinoException(COMPILER_ERROR,
throw new TrinoException(QUERY_EXCEEDED_COMPILER_LIMIT,
"Query exceeded maximum filters. Please reduce the number of filters referenced and re-run the query.", e);
}
throw new TrinoException(COMPILER_ERROR, filter.toString(), e.getCause());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
import static io.trino.cache.SafeCaches.buildNonEvictableCache;
import static io.trino.operator.project.PageFieldsToInputParametersRewriter.rewritePageFieldsToInputParameters;
import static io.trino.spi.StandardErrorCode.COMPILER_ERROR;
import static io.trino.spi.StandardErrorCode.QUERY_EXCEEDED_COMPILER_LIMIT;
import static io.trino.sql.gen.BytecodeUtils.invoke;
import static io.trino.sql.gen.columnar.FilterEvaluator.isNotExpression;
import static io.trino.sql.gen.columnar.IsNotNullColumnarFilter.createIsNotNullColumnarFilter;
Expand Down Expand Up @@ -179,7 +180,7 @@ static Supplier<ColumnarFilter> createClassInstance(CallSiteBinder binder, Class
}
catch (Exception e) {
if (Throwables.getRootCause(e) instanceof MethodTooLargeException) {
throw new TrinoException(COMPILER_ERROR,
throw new TrinoException(QUERY_EXCEEDED_COMPILER_LIMIT,
"Query exceeded maximum filters. Please reduce the number of filters referenced and re-run the query.", e);
}
throw new TrinoException(COMPILER_ERROR, e.getCause());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package io.trino.sql.planner;

import com.google.common.base.Throwables;
import com.google.common.base.VerifyException;
import com.google.common.cache.CacheBuilder;
import com.google.common.collect.ContiguousSet;
Expand Down Expand Up @@ -270,6 +271,7 @@
import io.trino.sql.relational.SqlToRowExpressionTranslator;
import io.trino.type.BlockTypeOperators;
import io.trino.type.FunctionType;
import org.objectweb.asm.MethodTooLargeException;

import java.util.AbstractMap.SimpleEntry;
import java.util.ArrayList;
Expand Down Expand Up @@ -350,6 +352,7 @@
import static io.trino.operator.window.pattern.PhysicalValuePointer.CLASSIFIER;
import static io.trino.operator.window.pattern.PhysicalValuePointer.MATCH_NUMBER;
import static io.trino.spi.StandardErrorCode.COMPILER_ERROR;
import static io.trino.spi.StandardErrorCode.QUERY_EXCEEDED_COMPILER_LIMIT;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.TypeUtils.readNativeValue;
import static io.trino.spi.type.TypeUtils.writeNativeValue;
Expand Down Expand Up @@ -2067,11 +2070,12 @@ else if (sourceNode instanceof SampleNode sampleNode) {
throw e;
}
catch (RuntimeException e) {
throw new TrinoException(
COMPILER_ERROR,
"Compiler failed. Possible reasons include: the query may have too many or too complex expressions, " +
"or the underlying tables may have too many columns",
e);
if (Throwables.getRootCause(e) instanceof MethodTooLargeException) {
throw new TrinoException(QUERY_EXCEEDED_COMPILER_LIMIT,
"Compiler failed. Possible reasons include: the query may have too many or too complex expressions, " +
"or the underlying tables may have too many columns", e);
}
throw new TrinoException(COMPILER_ERROR, e);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@

import static io.airlift.testing.Closeables.closeAllRuntimeException;
import static io.trino.SessionTestUtils.TEST_SESSION;
import static io.trino.spi.StandardErrorCode.COMPILER_ERROR;
import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT;
import static io.trino.spi.StandardErrorCode.QUERY_EXCEEDED_COMPILER_LIMIT;
import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy;
import static java.util.Collections.nCopies;
import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;
Expand Down Expand Up @@ -61,7 +61,7 @@ public void testProjectionCompilerFailure()
String outer = "x + x + " + Joiner.on(" + ").join(nCopies(100, inner));

assertTrinoExceptionThrownBy(() -> runner.execute("SELECT " + outer + " FROM (VALUES rand()) t(x)"))
.hasErrorCode(COMPILER_ERROR)
.hasErrorCode(QUERY_EXCEEDED_COMPILER_LIMIT)
.hasMessage("Query exceeded maximum columns. Please reduce the number of columns referenced and re-run the query.");
}

Expand All @@ -75,7 +75,7 @@ public void testFilterCompilerFailure()
+ " OR " + Joiner.on(" AND ").join(nCopies(1000, " c3 = rand()"));

assertTrinoExceptionThrownBy(() -> runner.execute("SELECT * " + filterQueryInner + filterQueryWhere))
.hasErrorCode(COMPILER_ERROR)
.hasErrorCode(QUERY_EXCEEDED_COMPILER_LIMIT)
.hasMessage("Query exceeded maximum filters. Please reduce the number of filters referenced and re-run the query.");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,7 @@ public enum StandardErrorCode
INVALID_PLAN(129, USER_ERROR),
INVALID_VIEW_PROPERTY(130, USER_ERROR),
INVALID_ENTITY_KIND(131, USER_ERROR),
QUERY_EXCEEDED_COMPILER_LIMIT(132, USER_ERROR),

GENERIC_INTERNAL_ERROR(65536, INTERNAL_ERROR),
TOO_MANY_REQUESTS_FAILED(65537, INTERNAL_ERROR),
Expand Down

0 comments on commit ab2456c

Please sign in to comment.