Skip to content

Commit

Permalink
Implement EXECUTE prepared statements
Browse files Browse the repository at this point in the history
Syntax:
EXECUTE <name>

Executes the prepared statement named name.
Does not yet support parameters.
  • Loading branch information
rschlussel-zz authored and electrum committed May 27, 2016
1 parent 9417bb9 commit 49de083
Show file tree
Hide file tree
Showing 13 changed files with 238 additions and 5 deletions.
11 changes: 11 additions & 0 deletions presto-main/src/main/java/com/facebook/presto/Session.java
Expand Up @@ -19,6 +19,7 @@
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.security.Identity;
import com.facebook.presto.spi.type.TimeZoneKey;
import com.facebook.presto.sql.tree.Execute;
import com.facebook.presto.transaction.TransactionId;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
Expand All @@ -34,6 +35,8 @@
import java.util.Optional;
import java.util.TimeZone;

import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND;
import static com.facebook.presto.util.Failures.checkCondition;
import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
Expand Down Expand Up @@ -197,6 +200,14 @@ public Map<String, String> getPreparedStatements()
return preparedStatements;
}

public String getPreparedStatementFromExecute(Execute execute)
{
String name = execute.getName();
String sql = preparedStatements.get(name);
checkCondition(sql != null, NOT_FOUND, "Prepared statement not found: " + name);
return sql;
}

public Session withTransactionId(TransactionId transactionId)
{
requireNonNull(transactionId, "transactionId is null");
Expand Down
Expand Up @@ -22,6 +22,7 @@
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.sql.parser.ParsingException;
import com.facebook.presto.sql.parser.SqlParser;
import com.facebook.presto.sql.tree.Execute;
import com.facebook.presto.sql.tree.Explain;
import com.facebook.presto.sql.tree.Statement;
import com.facebook.presto.transaction.TransactionManager;
Expand Down Expand Up @@ -278,7 +279,7 @@ public QueryInfo createQuery(Session session, String query)
QueryExecution queryExecution;
Statement statement;
try {
statement = sqlParser.createStatement(query);
statement = unwrapExecuteStatement(sqlParser.createStatement(query), sqlParser, session);
QueryExecutionFactory<?> queryExecutionFactory = executionFactories.get(statement.getClass());
if (queryExecutionFactory == null) {
throw new PrestoException(NOT_SUPPORTED, "Unsupported statement type: " + statement.getClass().getSimpleName());
Expand Down Expand Up @@ -343,6 +344,16 @@ public QueryInfo createQuery(Session session, String query)
return queryInfo;
}

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

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

@Override
public void cancelQuery(QueryId queryId)
{
Expand Down
Expand Up @@ -139,6 +139,7 @@
import static com.facebook.presto.connector.informationSchema.InformationSchemaMetadata.TABLE_INTERNAL_PARTITIONS;
import static com.facebook.presto.connector.informationSchema.InformationSchemaMetadata.TABLE_SCHEMATA;
import static com.facebook.presto.connector.informationSchema.InformationSchemaMetadata.TABLE_TABLES;
import static com.facebook.presto.execution.SqlQueryManager.unwrapExecuteStatement;
import static com.facebook.presto.metadata.FunctionKind.AGGREGATE;
import static com.facebook.presto.metadata.FunctionKind.APPROXIMATE_AGGREGATE;
import static com.facebook.presto.metadata.FunctionKind.WINDOW;
Expand Down Expand Up @@ -902,11 +903,12 @@ protected RelationType visitExplain(Explain node, AnalysisContext context)
private String getQueryPlan(Explain node, ExplainType.Type planType, ExplainFormat.Type planFormat)
throws IllegalArgumentException
{
Statement statement = unwrapExecuteStatement(node.getStatement(), sqlParser, session);
switch (planFormat) {
case GRAPHVIZ:
return queryExplainer.get().getGraphvizPlan(session, node.getStatement(), planType);
return queryExplainer.get().getGraphvizPlan(session, statement, planType);
case TEXT:
return queryExplainer.get().getPlan(session, node.getStatement(), planType);
return queryExplainer.get().getPlan(session, statement, planType);
}
throw new IllegalArgumentException("Invalid Explain Format: " + planFormat.toString());
}
Expand Down
Expand Up @@ -150,6 +150,7 @@
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Function;

import static com.facebook.presto.execution.SqlQueryManager.unwrapExecuteStatement;
import static com.facebook.presto.spi.type.BigintType.BIGINT;
import static com.facebook.presto.sql.testing.TreeAssertions.assertFormattedSql;
import static com.facebook.presto.testing.TestingTaskContext.createTaskContext;
Expand Down Expand Up @@ -584,7 +585,7 @@ public List<Driver> createDrivers(Session session, @Language("SQL") String sql,

public Plan createPlan(Session session, @Language("SQL") String sql)
{
Statement statement = sqlParser.createStatement(sql);
Statement statement = unwrapExecuteStatement(sqlParser.createStatement(sql), sqlParser, session);

assertFormattedSql(sqlParser, statement);

Expand Down
@@ -0,0 +1,70 @@
/*
* Licensed 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 com.facebook.presto.execution;

import com.facebook.presto.Session;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.sql.parser.SqlParser;
import com.facebook.presto.sql.tree.AllColumns;
import com.facebook.presto.sql.tree.QualifiedName;
import com.facebook.presto.sql.tree.Statement;
import org.testng.annotations.Test;

import static com.facebook.presto.SessionTestUtils.TEST_SESSION;
import static com.facebook.presto.execution.SqlQueryManager.unwrapExecuteStatement;
import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND;
import static com.facebook.presto.sql.QueryUtil.selectList;
import static com.facebook.presto.sql.QueryUtil.simpleQuery;
import static com.facebook.presto.sql.QueryUtil.table;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.fail;

public class TestUnwrapExecute
{
private static final SqlParser SQL_PARSER = new SqlParser();

@Test
public void testSelectStatement()
throws Exception
{
Statement statement = SQL_PARSER.createStatement("SELECT * FROM foo");
assertEquals(unwrapExecuteStatement(statement, SQL_PARSER, TEST_SESSION),
simpleQuery(selectList(new AllColumns()), table(QualifiedName.of("foo"))));
}

@Test
public void testExecuteStatement()
throws Exception
{
Session session = TEST_SESSION.withPreparedStatement("my_query", "SELECT * FROM foo");
Statement statement = SQL_PARSER.createStatement("EXECUTE my_query");
assertEquals(unwrapExecuteStatement(statement, SQL_PARSER, session),
simpleQuery(selectList(new AllColumns()), table(QualifiedName.of("foo"))));
}

@Test
public void testExecuteStatementDoesNotExist()
throws Exception
{
try {
Statement statement = SQL_PARSER.createStatement("execute my_query");
unwrapExecuteStatement(statement, SQL_PARSER, TEST_SESSION);
fail("expected exception");
}
catch (PrestoException e) {
assertEquals(e.getErrorCode(), NOT_FOUND.toErrorCode());
}
}
}
Expand Up @@ -78,6 +78,7 @@ statement
(LIMIT limit=(INTEGER_VALUE | ALL))? #showPartitions
| PREPARE identifier FROM statement #prepare
| DEALLOCATE PREPARE identifier #deallocate
| EXECUTE identifier #execute
;

query
Expand Down Expand Up @@ -592,6 +593,7 @@ ONLY: 'ONLY';
CALL: 'CALL';
PREPARE: 'PREPARE';
DEALLOCATE: 'DEALLOCATE';
EXECUTE: 'EXECUTE';

NORMALIZE: 'NORMALIZE';
NFD : 'NFD';
Expand Down
Expand Up @@ -28,6 +28,7 @@
import com.facebook.presto.sql.tree.DropTable;
import com.facebook.presto.sql.tree.DropView;
import com.facebook.presto.sql.tree.Except;
import com.facebook.presto.sql.tree.Execute;
import com.facebook.presto.sql.tree.Explain;
import com.facebook.presto.sql.tree.ExplainFormat;
import com.facebook.presto.sql.tree.ExplainOption;
Expand Down Expand Up @@ -168,6 +169,14 @@ protected Void visitDeallocate(Deallocate node, Integer indent)
return null;
}

@Override
protected Void visitExecute(Execute node, Integer indent)
{
append(indent, "EXECUTE ");
builder.append(node.getName());
return null;
}

@Override
protected Void visitQuery(Query node, Integer indent)
{
Expand Down
Expand Up @@ -45,6 +45,7 @@
import com.facebook.presto.sql.tree.DropTable;
import com.facebook.presto.sql.tree.DropView;
import com.facebook.presto.sql.tree.Except;
import com.facebook.presto.sql.tree.Execute;
import com.facebook.presto.sql.tree.ExistsPredicate;
import com.facebook.presto.sql.tree.Explain;
import com.facebook.presto.sql.tree.ExplainFormat;
Expand Down Expand Up @@ -341,6 +342,13 @@ public Node visitDeallocate(SqlBaseParser.DeallocateContext context)
return new Deallocate(getLocation(context), name);
}

@Override
public Node visitExecute(SqlBaseParser.ExecuteContext context)
{
String name = context.identifier().getText();
return new Execute(getLocation(context), name);
}

// ********************** query expressions ********************

@Override
Expand Down
Expand Up @@ -92,6 +92,11 @@ protected R visitDeallocate(Deallocate node, C context)
return visitStatement(node, context);
}

protected R visitExecute(Execute node, C context)
{
return visitStatement(node, context);
}

protected R visitQuery(Query node, C context)
{
return visitStatement(node, context);
Expand Down
@@ -0,0 +1,80 @@
/*
* Licensed 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 com.facebook.presto.sql.tree;

import java.util.Objects;
import java.util.Optional;

import static com.google.common.base.MoreObjects.toStringHelper;
import static java.util.Objects.requireNonNull;

public class Execute
extends Statement
{
private final String name;

public Execute(NodeLocation location, String name)
{
this(Optional.of(location), name);
}

public Execute(String name)
{
this(Optional.empty(), name);
}

private Execute(Optional<NodeLocation> location, String name)
{
super(location);
this.name = requireNonNull(name, "name is null");
}

public String getName()
{
return name;
}

@Override
public <R, C> R accept(AstVisitor<R, C> visitor, C context)
{
return visitor.visitExecute(this, context);
}

@Override
public int hashCode()
{
return Objects.hash(name);
}

@Override
public boolean equals(Object obj)
{
if (this == obj) {
return true;
}
if ((obj == null) || (getClass() != obj.getClass())) {
return false;
}
Execute o = (Execute) obj;
return Objects.equals(name, o.name);
}

@Override
public String toString()
{
return toStringHelper(this)
.add("name", name)
.toString();
}
}
Expand Up @@ -39,6 +39,7 @@
import com.facebook.presto.sql.tree.DoubleLiteral;
import com.facebook.presto.sql.tree.DropTable;
import com.facebook.presto.sql.tree.DropView;
import com.facebook.presto.sql.tree.Execute;
import com.facebook.presto.sql.tree.Explain;
import com.facebook.presto.sql.tree.ExplainFormat;
import com.facebook.presto.sql.tree.ExplainType;
Expand Down Expand Up @@ -1462,6 +1463,12 @@ public void testDeallocatePrepare()
assertStatement("DEALLOCATE PREPARE myquery", new Deallocate("myquery"));
}

@Test
public void testExecute()
{
assertStatement("EXECUTE myquery", new Execute("myquery"));
}

private static void assertCast(String type)
{
assertCast(type, type);
Expand Down
Expand Up @@ -13,6 +13,7 @@
*/
package com.facebook.presto.tests;

import com.facebook.presto.Session;
import com.facebook.presto.metadata.FunctionListBuilder;
import com.facebook.presto.metadata.SqlFunction;
import com.facebook.presto.spi.session.PropertyMetadata;
Expand Down Expand Up @@ -4427,6 +4428,14 @@ private void assertExplainDdl(String query, String expected)
assertEquals(getOnlyElement(result.getOnlyColumnAsSet()), expected);
}

@Test
public void testExplainExecute()
{
Session session = getSession().withPreparedStatement("my_query", "SELECT * FROM orders");
MaterializedResult result = computeActual(session, "EXPLAIN (TYPE LOGICAL) EXECUTE my_query");
assertEquals(getOnlyElement(result.getOnlyColumnAsSet()), getExplainPlan("SELECT * FROM orders", LOGICAL));
}

@Test
public void testShowCatalogs()
throws Exception
Expand Down Expand Up @@ -6469,4 +6478,17 @@ public void testAccessControl()
assertAccessDenied("DELETE FROM orders", "Cannot delete from table .*.orders.*", privilege("orders", DELETE_TABLE));
assertAccessDenied("CREATE TABLE foo AS SELECT * FROM orders", "Cannot create table .*.foo.*", privilege("foo", CREATE_TABLE));
}

@Test
public void testExecute() throws Exception
{
Session session = getSession().withPreparedStatement("my_query", "SELECT 123, 'abc'");
assertQuery(session, "EXECUTE my_query", "SELECT 123, 'abc'");
}

@Test
public void testExecuteNoSuchQuery()
{
assertQueryFails("EXECUTE my_query", "Prepared statement not found: my_query");
}
}

0 comments on commit 49de083

Please sign in to comment.