Skip to content

Commit

Permalink
Add ParsingWarnings to the WarningCollector
Browse files Browse the repository at this point in the history
This is motivated by the need to produce warnings during parsing
  • Loading branch information
Elon Azoulay committed Apr 5, 2019
1 parent 31e1409 commit 0d782dc
Show file tree
Hide file tree
Showing 15 changed files with 179 additions and 30 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package com.facebook.presto.execution;

import com.facebook.presto.Session;
import com.facebook.presto.execution.warnings.WarningCollector;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.resourceGroups.QueryType;
import com.facebook.presto.sql.analyzer.SemanticException;
Expand Down Expand Up @@ -49,17 +50,17 @@ public QueryPreparer(SqlParser sqlParser)
this.sqlParser = requireNonNull(sqlParser, "sqlParser is null");
}

public PreparedQuery prepareQuery(Session session, String query)
public PreparedQuery prepareQuery(Session session, String query, WarningCollector warningCollector)
throws ParsingException, PrestoException, SemanticException
{
Statement wrappedStatement = sqlParser.createStatement(query, createParsingOptions(session));
return prepareQuery(session, wrappedStatement);
Statement wrappedStatement = sqlParser.createStatement(query, createParsingOptions(session, warningCollector));
return prepareQuery(session, wrappedStatement, warningCollector);
}

public PreparedQuery prepareQuery(Session session, Statement wrappedStatement)
public PreparedQuery prepareQuery(Session session, Statement wrappedStatement, WarningCollector warningCollector)
throws ParsingException, PrestoException, SemanticException
{
Statement statement = unwrapExecuteStatement(wrappedStatement, sqlParser, session);
Statement statement = unwrapExecuteStatement(wrappedStatement, sqlParser, session, warningCollector);
if (statement instanceof Explain && ((Explain) statement).isAnalyze()) {
Statement innerStatement = ((Explain) statement).getStatement();
Optional<QueryType> innerQueryType = StatementUtils.getQueryType(innerStatement.getClass());
Expand All @@ -75,14 +76,14 @@ public PreparedQuery prepareQuery(Session session, Statement wrappedStatement)
return new PreparedQuery(statement, parameters);
}

private static Statement unwrapExecuteStatement(Statement statement, SqlParser sqlParser, Session session)
private static Statement unwrapExecuteStatement(Statement statement, SqlParser sqlParser, Session session, WarningCollector warningCollector)
{
if (!(statement instanceof Execute)) {
return statement;
}

String sql = session.getPreparedStatementFromExecute((Execute) statement);
return sqlParser.createStatement(sql, createParsingOptions(session));
return sqlParser.createStatement(sql, createParsingOptions(session, warningCollector));
}

private static void validateParameters(Statement node, List<Expression> parameterValues)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.facebook.presto.execution.StateMachine.StateChangeListener;
import com.facebook.presto.execution.resourceGroups.ResourceGroupManager;
import com.facebook.presto.execution.scheduler.NodeSchedulerConfig;
import com.facebook.presto.execution.warnings.WarningCollector;
import com.facebook.presto.execution.warnings.WarningCollectorFactory;
import com.facebook.presto.memory.ClusterMemoryManager;
import com.facebook.presto.metadata.SessionPropertyManager;
Expand Down Expand Up @@ -336,8 +337,10 @@ private <C> void createQueryInternal(QueryId queryId, SessionContext sessionCont
// decode session
session = sessionSupplier.createSession(queryId, sessionContext);

WarningCollector warningCollector = warningCollectorFactory.create();

// prepare query
preparedQuery = queryPreparer.prepareQuery(session, query);
preparedQuery = queryPreparer.prepareQuery(session, query, warningCollector);

// select resource group
queryType = getQueryType(preparedQuery.getStatement().getClass());
Expand Down Expand Up @@ -365,7 +368,7 @@ private <C> void createQueryInternal(QueryId queryId, SessionContext sessionCont
session,
preparedQuery,
selectionContext.getResourceGroupId(),
warningCollectorFactory.create(),
warningCollector,
queryType);
}
catch (RuntimeException e) {
Expand Down
15 changes: 13 additions & 2 deletions presto-main/src/main/java/com/facebook/presto/sql/ParsingUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -14,18 +14,29 @@
package com.facebook.presto.sql;

import com.facebook.presto.Session;
import com.facebook.presto.execution.warnings.WarningCollector;
import com.facebook.presto.spi.PrestoWarning;
import com.facebook.presto.sql.parser.ParsingOptions;

import static com.facebook.presto.SystemSessionProperties.isParseDecimalLiteralsAsDouble;
import static com.facebook.presto.spi.StandardWarningCode.PARSER_WARNING;
import static com.facebook.presto.sql.parser.ParsingOptions.DecimalLiteralTreatment.AS_DECIMAL;
import static com.facebook.presto.sql.parser.ParsingOptions.DecimalLiteralTreatment.AS_DOUBLE;

public class ParsingUtil
{
private ParsingUtil() {}

public static ParsingOptions createParsingOptions(Session session)
{
return new ParsingOptions(isParseDecimalLiteralsAsDouble(session) ? AS_DOUBLE : AS_DECIMAL);
return createParsingOptions(session, WarningCollector.NOOP);
}

private ParsingUtil() {}
public static ParsingOptions createParsingOptions(Session session, WarningCollector warningCollector)
{
return ParsingOptions.builder()
.setDecimalLiteralTreatment(isParseDecimalLiteralsAsDouble(session) ? AS_DOUBLE : AS_DECIMAL)
.setWarningConsumer(warning -> warningCollector.add(new PrestoWarning(PARSER_WARNING, warning.getMessage())))
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1962,7 +1962,7 @@ private RelationType analyzeView(Query query, QualifiedObjectName name, Optional
private Query parseView(String view, QualifiedObjectName name, Node node)
{
try {
return (Query) sqlParser.createStatement(view, createParsingOptions(session));
return (Query) sqlParser.createStatement(view, createParsingOptions(session, warningCollector));
}
catch (ParsingException e) {
throw new SemanticException(VIEW_PARSE_ERROR, node, "Failed parsing stored view '%s': %s", name, e.getMessage());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ protected Node visitDescribeInput(DescribeInput node, Void context)
throws SemanticException
{
String sqlString = session.getPreparedStatement(node.getName().getValue());
Statement statement = parser.createStatement(sqlString, createParsingOptions(session));
Statement statement = parser.createStatement(sqlString, createParsingOptions(session, warningCollector));

// create analysis for the query we are describing.
Analyzer analyzer = new Analyzer(session, metadata, parser, accessControl, queryExplainer, parameters, warningCollector);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,7 @@ public Visitor(
protected Node visitDescribeOutput(DescribeOutput node, Void context)
{
String sqlString = session.getPreparedStatement(node.getName().getValue());
Statement statement = parser.createStatement(sqlString, createParsingOptions(session));
Statement statement = parser.createStatement(sqlString, createParsingOptions(session, warningCollector));

Analyzer analyzer = new Analyzer(session, metadata, parser, accessControl, queryExplainer, parameters, warningCollector);
Analysis analysis = analyzer.analyze(statement, true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ protected Node visitExplain(Explain node, Void context)
private Node getQueryPlan(Explain node, ExplainType.Type planType, ExplainFormat.Type planFormat)
throws IllegalArgumentException
{
PreparedQuery preparedQuery = queryPreparer.prepareQuery(session, node.getStatement());
PreparedQuery preparedQuery = queryPreparer.prepareQuery(session, node.getStatement(), warningCollector);

if (planType == VALIDATE) {
queryExplainer.get().analyze(session, preparedQuery.getStatement(), preparedQuery.getParameters(), warningCollector);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -614,7 +614,7 @@ protected Node visitShowSession(ShowSession node, Void context)
private Query parseView(String view, QualifiedObjectName name, Node node)
{
try {
Statement statement = sqlParser.createStatement(view, createParsingOptions(session));
Statement statement = sqlParser.createStatement(view, createParsingOptions(session, warningCollector));
return (Query) statement;
}
catch (ParsingException e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,9 @@
import com.facebook.presto.execution.scheduler.LegacyNetworkTopology;
import com.facebook.presto.execution.scheduler.NodeScheduler;
import com.facebook.presto.execution.scheduler.NodeSchedulerConfig;
import com.facebook.presto.execution.warnings.DefaultWarningCollector;
import com.facebook.presto.execution.warnings.WarningCollector;
import com.facebook.presto.execution.warnings.WarningCollectorConfig;
import com.facebook.presto.index.IndexManager;
import com.facebook.presto.memory.MemoryManagerConfig;
import com.facebook.presto.metadata.AnalyzePropertyManager;
Expand Down Expand Up @@ -593,13 +595,13 @@ public MaterializedResult execute(@Language("SQL") String sql)
@Override
public MaterializedResult execute(Session session, @Language("SQL") String sql)
{
return executeWithPlan(session, sql, WarningCollector.NOOP).getMaterializedResult();
return executeWithPlan(session, sql, new DefaultWarningCollector(new WarningCollectorConfig())).getMaterializedResult();
}

@Override
public MaterializedResultWithPlan executeWithPlan(Session session, String sql, WarningCollector warningCollector)
{
return inTransaction(session, transactionSession -> executeInternal(transactionSession, sql));
return inTransaction(session, transactionSession -> executeInternal(transactionSession, sql, warningCollector));
}

public <T> T inTransaction(Function<Session, T> transactionSessionConsumer)
Expand All @@ -614,7 +616,7 @@ public <T> T inTransaction(Session session, Function<Session, T> transactionSess
.execute(session, transactionSessionConsumer);
}

private MaterializedResultWithPlan executeInternal(Session session, @Language("SQL") String sql)
private MaterializedResultWithPlan executeInternal(Session session, @Language("SQL") String sql, WarningCollector warningCollector)
{
lock.readLock().lock();
try (Closer closer = Closer.create()) {
Expand All @@ -629,7 +631,7 @@ private MaterializedResultWithPlan executeInternal(Session session, @Language("S
.setQueryMaxSpillSize(nodeSpillConfig.getQueryMaxSpillPerNode())
.build();

Plan plan = createPlan(session, sql, WarningCollector.NOOP);
Plan plan = createPlan(session, sql, warningCollector);
List<Driver> drivers = createDrivers(session, plan, outputFactory, taskContext);
drivers.forEach(closer::register);

Expand Down Expand Up @@ -807,7 +809,7 @@ public Plan createPlan(Session session, @Language("SQL") String sql, LogicalPlan

public Plan createPlan(Session session, @Language("SQL") String sql, LogicalPlanner.Stage stage, boolean forceSingleNode, WarningCollector warningCollector)
{
PreparedQuery preparedQuery = new QueryPreparer(sqlParser).prepareQuery(session, sql);
PreparedQuery preparedQuery = new QueryPreparer(sqlParser).prepareQuery(session, sql, warningCollector);

assertFormattedSql(sqlParser, createParsingOptions(session), preparedQuery.getStatement());

Expand Down Expand Up @@ -841,8 +843,8 @@ public Plan createPlan(Session session, @Language("SQL") String sql, List<PlanOp

public Plan createPlan(Session session, @Language("SQL") String sql, List<PlanOptimizer> optimizers, LogicalPlanner.Stage stage, WarningCollector warningCollector)
{
Statement wrappedStatement = sqlParser.createStatement(sql, createParsingOptions(session));
PreparedQuery preparedQuery = new QueryPreparer(sqlParser).prepareQuery(session, wrappedStatement);
Statement wrappedStatement = sqlParser.createStatement(sql, createParsingOptions(session, warningCollector));
PreparedQuery preparedQuery = new QueryPreparer(sqlParser).prepareQuery(session, wrappedStatement, warningCollector);

assertFormattedSql(sqlParser, createParsingOptions(session), preparedQuery.getStatement());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import com.facebook.presto.Session;
import com.facebook.presto.execution.QueryPreparer.PreparedQuery;
import com.facebook.presto.execution.warnings.WarningCollector;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.sql.analyzer.SemanticException;
import com.facebook.presto.sql.parser.SqlParser;
Expand All @@ -40,7 +41,7 @@ public class TestQueryPreparer
@Test
public void testSelectStatement()
{
PreparedQuery preparedQuery = QUERY_PREPARER.prepareQuery(TEST_SESSION, "SELECT * FROM foo");
PreparedQuery preparedQuery = QUERY_PREPARER.prepareQuery(TEST_SESSION, "SELECT * FROM foo", WarningCollector.NOOP);
assertEquals(preparedQuery.getStatement(),
simpleQuery(selectList(new AllColumns()), table(QualifiedName.of("foo"))));
}
Expand All @@ -51,7 +52,7 @@ public void testExecuteStatement()
Session session = testSessionBuilder()
.addPreparedStatement("my_query", "SELECT * FROM foo")
.build();
PreparedQuery preparedQuery = QUERY_PREPARER.prepareQuery(session, "EXECUTE my_query");
PreparedQuery preparedQuery = QUERY_PREPARER.prepareQuery(session, "EXECUTE my_query", WarningCollector.NOOP);
assertEquals(preparedQuery.getStatement(),
simpleQuery(selectList(new AllColumns()), table(QualifiedName.of("foo"))));
}
Expand All @@ -60,7 +61,7 @@ public void testExecuteStatement()
public void testExecuteStatementDoesNotExist()
{
try {
QUERY_PREPARER.prepareQuery(TEST_SESSION, "execute my_query");
QUERY_PREPARER.prepareQuery(TEST_SESSION, "execute my_query", WarningCollector.NOOP);
fail("expected exception");
}
catch (PrestoException e) {
Expand All @@ -75,7 +76,7 @@ public void testTooManyParameters()
Session session = testSessionBuilder()
.addPreparedStatement("my_query", "SELECT * FROM foo where col1 = ?")
.build();
QUERY_PREPARER.prepareQuery(session, "EXECUTE my_query USING 1,2");
QUERY_PREPARER.prepareQuery(session, "EXECUTE my_query USING 1,2", WarningCollector.NOOP);
fail("expected exception");
}
catch (SemanticException e) {
Expand All @@ -90,7 +91,7 @@ public void testTooFewParameters()
Session session = testSessionBuilder()
.addPreparedStatement("my_query", "SELECT ? FROM foo where col1 = ?")
.build();
QUERY_PREPARER.prepareQuery(session, "EXECUTE my_query USING 1");
QUERY_PREPARER.prepareQuery(session, "EXECUTE my_query USING 1", WarningCollector.NOOP);
fail("expected exception");
}
catch (SemanticException e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,10 +13,14 @@
*/
package com.facebook.presto.sql.parser;

import java.util.function.Consumer;

import static java.util.Objects.requireNonNull;

public class ParsingOptions
{
private static final Consumer<ParsingWarning> NOOP_WARNING_CONSUMER = ignored -> {};

public enum DecimalLiteralTreatment
{
AS_DOUBLE,
Expand All @@ -25,19 +29,69 @@ public enum DecimalLiteralTreatment
}

private final DecimalLiteralTreatment decimalLiteralTreatment;
private final Consumer<ParsingWarning> warningConsumer;

/**
* @deprecated Use (@link #builder())
*/
@Deprecated
public ParsingOptions()
{
this(DecimalLiteralTreatment.REJECT);
}

/**
* @deprecated Use (@link #builder())
*/
@Deprecated
public ParsingOptions(DecimalLiteralTreatment decimalLiteralTreatment)
{
this(decimalLiteralTreatment, NOOP_WARNING_CONSUMER);
}

private ParsingOptions(DecimalLiteralTreatment decimalLiteralTreatment, Consumer<ParsingWarning> warningConsumer)
{
this.decimalLiteralTreatment = requireNonNull(decimalLiteralTreatment, "decimalLiteralTreatment is null");
this.warningConsumer = requireNonNull(warningConsumer, "warningConsumer is null");
}

public DecimalLiteralTreatment getDecimalLiteralTreatment()
{
return decimalLiteralTreatment;
}

public Consumer<ParsingWarning> getWarningConsumer()
{
return warningConsumer;
}

public static Builder builder()
{
return new Builder();
}

public static class Builder
{
private DecimalLiteralTreatment decimalLiteralTreatment = DecimalLiteralTreatment.REJECT;
private Consumer<ParsingWarning> warningConsumer = NOOP_WARNING_CONSUMER;

private Builder() {}

public Builder setDecimalLiteralTreatment(DecimalLiteralTreatment decimalLiteralTreatment)
{
this.decimalLiteralTreatment = requireNonNull(decimalLiteralTreatment, "decimalLiteralTreatment is null");
return this;
}

public Builder setWarningConsumer(Consumer<ParsingWarning> warningConsumer)
{
this.warningConsumer = requireNonNull(warningConsumer, "warningConsumer is null");
return this;
}

public ParsingOptions build()
{
return new ParsingOptions(decimalLiteralTreatment, warningConsumer);
}
}
}
Loading

0 comments on commit 0d782dc

Please sign in to comment.