Skip to content

Commit

Permalink
Add current user security mode for views
Browse files Browse the repository at this point in the history
  • Loading branch information
electrum committed Jan 25, 2019
1 parent cf17d08 commit c408a78
Show file tree
Hide file tree
Showing 14 changed files with 206 additions and 32 deletions.
21 changes: 20 additions & 1 deletion presto-docs/src/main/sphinx/sql/create-view.rst
Expand Up @@ -7,7 +7,9 @@ Synopsis

.. code-block:: none
CREATE [ OR REPLACE ] VIEW view_name AS query
CREATE [ OR REPLACE ] VIEW view_name
[ SECURITY { DEFINER | INVOKER } ]
AS query
Description
-----------
Expand All @@ -20,6 +22,23 @@ referenced by another query.
The optional ``OR REPLACE`` clause causes the view to be replaced if it
already exists rather than raising an error.

Security
--------

In the default ``DEFINER`` security mode, tables referenced in the view
are accessed using the permissions of the view owner (the *creator* or
*definer* of the view) rather than the user executing the query. This
allows providing restricted access to the underlying tables, for which
the user may not be allowed to access directly.

In the ``INVOKER`` security mode, tables referenced in the view are accessed
using the permissions of the user executing the query (the *invoker* of the view).
A view created in this mode is simply a stored query.

Regardless of the security mode, the ``current_user`` function will
always return the user executing the query and thus may be used
within views to filter out rows or otherwise restrict access.

Examples
--------

Expand Down
Expand Up @@ -40,6 +40,7 @@
import static io.prestosql.metadata.MetadataUtil.createQualifiedObjectName;
import static io.prestosql.metadata.ViewDefinition.ViewColumn;
import static io.prestosql.sql.SqlFormatterUtil.getFormattedSql;
import static io.prestosql.sql.tree.CreateView.Security.INVOKER;
import static java.util.Objects.requireNonNull;

public class CreateViewTask
Expand Down Expand Up @@ -88,7 +89,13 @@ public ListenableFuture<?> execute(CreateView statement, TransactionManager tran
.map(field -> new ViewColumn(field.getName().get(), field.getType()))
.collect(toImmutableList());

String data = codec.toJson(new ViewDefinition(sql, session.getCatalog(), session.getSchema(), columns, Optional.of(session.getUser())));
// use DEFINER security by default
Optional<String> owner = Optional.of(session.getUser());
if (statement.getSecurity().orElse(null) == INVOKER) {
owner = Optional.empty();
}

String data = codec.toJson(new ViewDefinition(sql, session.getCatalog(), session.getSchema(), columns, owner, !owner.isPresent()));

metadata.createView(session, name, data, statement.isReplace());

Expand Down
Expand Up @@ -836,7 +836,7 @@ public Optional<ViewDefinition> getView(Session session, QualifiedObjectName vie
ConnectorViewDefinition view = views.get(viewName.asSchemaTableName());
if (view != null) {
ViewDefinition definition = deserializeView(view.getViewData());
if (view.getOwner().isPresent()) {
if (view.getOwner().isPresent() && !definition.isRunAsInvoker()) {
definition = definition.withOwner(view.getOwner().get());
}
return Optional.of(definition);
Expand Down Expand Up @@ -1007,7 +1007,8 @@ private ConnectorMetadata getMetadataForWrite(Session session, ConnectorId conne
return getCatalogMetadataForWrite(session, connectorId).getMetadata();
}

private static JsonCodec<ViewDefinition> createTestingViewCodec()
@VisibleForTesting
static JsonCodec<ViewDefinition> createTestingViewCodec()
{
ObjectMapperProvider provider = new ObjectMapperProvider();
provider.setJsonDeserializers(ImmutableMap.of(Type.class, new TypeDeserializer(new TypeRegistry())));
Expand Down
Expand Up @@ -22,6 +22,7 @@
import java.util.Optional;

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

public final class ViewDefinition
Expand All @@ -31,20 +32,24 @@ public final class ViewDefinition
private final Optional<String> schema;
private final List<ViewColumn> columns;
private final Optional<String> owner;
private final boolean runAsInvoker;

@JsonCreator
public ViewDefinition(
@JsonProperty("originalSql") String originalSql,
@JsonProperty("catalog") Optional<String> catalog,
@JsonProperty("schema") Optional<String> schema,
@JsonProperty("columns") List<ViewColumn> columns,
@JsonProperty("owner") Optional<String> owner)
@JsonProperty("owner") Optional<String> owner,
@JsonProperty("runAsInvoker") boolean runAsInvoker)
{
this.originalSql = requireNonNull(originalSql, "originalSql is null");
this.catalog = requireNonNull(catalog, "catalog is null");
this.schema = requireNonNull(schema, "schema is null");
this.columns = ImmutableList.copyOf(requireNonNull(columns, "columns is null"));
this.owner = requireNonNull(owner, "owner is null");
this.runAsInvoker = runAsInvoker;
checkArgument(!runAsInvoker || !owner.isPresent(), "owner cannot be present with runAsInvoker");
}

@JsonProperty
Expand Down Expand Up @@ -77,6 +82,12 @@ public Optional<String> getOwner()
return owner;
}

@JsonProperty
public boolean isRunAsInvoker()
{
return runAsInvoker;
}

@Override
public String toString()
{
Expand All @@ -86,13 +97,14 @@ public String toString()
.add("schema", schema.orElse(null))
.add("columns", columns)
.add("owner", owner.orElse(null))
.add("runAsInvoker", runAsInvoker)
.omitNullValues()
.toString();
}

public ViewDefinition withOwner(String owner)
{
return new ViewDefinition(originalSql, catalog, schema, columns, Optional.of(owner));
return new ViewDefinition(originalSql, catalog, schema, columns, Optional.of(owner), runAsInvoker);
}

public static final class ViewColumn
Expand Down
Expand Up @@ -373,7 +373,7 @@ protected Node visitShowCreate(ShowCreate node, Void context)
}

Query query = parseView(viewDefinition.get().getOriginalSql(), objectName, node);
String sql = formatSql(new CreateView(createQualifiedName(objectName), query, false), Optional.of(parameters)).trim();
String sql = formatSql(new CreateView(createQualifiedName(objectName), query, false, Optional.empty()), Optional.of(parameters)).trim();
return singleValueQuery("Create View", sql);
}

Expand Down
Expand Up @@ -74,7 +74,7 @@ public TestInformationSchemaMetadata()
new SchemaTableName("test_schema", "test_view"),
new SchemaTableName("test_schema", "another_table")))
.withGetViews((connectorSession, prefix) -> {
String viewJson = VIEW_DEFINITION_JSON_CODEC.toJson(new ViewDefinition("select 1", Optional.of("test_catalog"), Optional.of("test_schema"), ImmutableList.of(), Optional.empty()));
String viewJson = VIEW_DEFINITION_JSON_CODEC.toJson(new ViewDefinition("select 1", Optional.of("test_catalog"), Optional.of("test_schema"), ImmutableList.of(), Optional.empty(), false));
SchemaTableName viewName = new SchemaTableName("test_schema", "test_view");
return ImmutableMap.of(viewName, new ConnectorViewDefinition(viewName, Optional.empty(), viewJson));
}).build();
Expand Down
@@ -0,0 +1,81 @@
/*
* 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 io.prestosql.metadata;

import io.airlift.json.JsonCodec;
import io.prestosql.metadata.ViewDefinition.ViewColumn;
import io.prestosql.spi.type.IntegerType;
import org.testng.annotations.Test;

import java.util.Optional;

import static com.google.common.collect.Iterables.getOnlyElement;
import static io.prestosql.metadata.MetadataManager.createTestingViewCodec;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;

public class TestViewDefinition
{
private static final JsonCodec<ViewDefinition> CODEC = createTestingViewCodec();
private static final String BASE_JSON = "" +
"\"originalSql\": \"SELECT 42 x\", " +
"\"columns\": [{\"name\": \"x\", \"type\": \"integer\"}]";

@Test
public void testLegacyViewWithoutOwner()
{
// very old view before owner was added
ViewDefinition view = CODEC.fromJson("{" + BASE_JSON + "}");
assertBaseView(view);
assertFalse(view.getOwner().isPresent());
}

@Test
public void testViewWithOwner()
{
// old view before invoker security was added
ViewDefinition view = CODEC.fromJson("{" + BASE_JSON + ", \"owner\": \"abc\"}");
assertBaseView(view);
assertEquals(view.getOwner(), Optional.of("abc"));
assertFalse(view.isRunAsInvoker());
}

@Test
public void testViewSecurityDefiner()
{
ViewDefinition view = CODEC.fromJson("{" + BASE_JSON + ", \"owner\": \"abc\", \"runAsInvoker\": false}");
assertBaseView(view);
assertEquals(view.getOwner(), Optional.of("abc"));
assertFalse(view.isRunAsInvoker());
}

@Test
public void testViewSecurityInvoker()
{
ViewDefinition view = CODEC.fromJson("{" + BASE_JSON + ", \"runAsInvoker\": true}");
assertBaseView(view);
assertFalse(view.getOwner().isPresent());
assertTrue(view.isRunAsInvoker());
}

private static void assertBaseView(ViewDefinition view)
{
assertEquals(view.getOriginalSql(), "SELECT 42 x");
assertEquals(view.getColumns().size(), 1);
ViewColumn column = getOnlyElement(view.getColumns());
assertEquals(column.getName(), "x");
assertEquals(column.getType(), IntegerType.INTEGER);
}
}
Expand Up @@ -1574,7 +1574,8 @@ public void setup()
Optional.of(TPCH_CATALOG),
Optional.of("s1"),
ImmutableList.of(new ViewColumn("a", BIGINT)),
Optional.of("user")));
Optional.of("user"),
false));
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName(TPCH_CATALOG, "s1", "v1"), viewData1, false));

// stale view (different column type)
Expand All @@ -1584,7 +1585,8 @@ public void setup()
Optional.of(TPCH_CATALOG),
Optional.of("s1"),
ImmutableList.of(new ViewColumn("a", VARCHAR)),
Optional.of("user")));
Optional.of("user"),
false));
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName(TPCH_CATALOG, "s1", "v2"), viewData2, false));

// view referencing table in different schema from itself and session
Expand All @@ -1594,7 +1596,8 @@ public void setup()
Optional.of(SECOND_CATALOG),
Optional.of("s2"),
ImmutableList.of(new ViewColumn("a", BIGINT)),
Optional.of("owner")));
Optional.of("owner"),
false));
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName(THIRD_CATALOG, "s3", "v3"), viewData3, false));

// valid view with uppercase column name
Expand All @@ -1604,7 +1607,8 @@ public void setup()
Optional.of("tpch"),
Optional.of("s1"),
ImmutableList.of(new ViewColumn("a", BIGINT)),
Optional.of("user")));
Optional.of("user"),
false));
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName("tpch", "s1", "v4"), viewData4, false));

// recursive view referencing to itself
Expand All @@ -1614,7 +1618,8 @@ public void setup()
Optional.of(TPCH_CATALOG),
Optional.of("s1"),
ImmutableList.of(new ViewColumn("a", BIGINT)),
Optional.of("user")));
Optional.of("user"),
false));
inSetupTransaction(session -> metadata.createView(session, new QualifiedObjectName(TPCH_CATALOG, "s1", "v5"), viewData5, false));
}

Expand Down
12 changes: 8 additions & 4 deletions presto-parser/src/main/antlr4/io/prestosql/sql/parser/SqlBase.g4
Expand Up @@ -56,7 +56,8 @@ statement
DROP COLUMN column=qualifiedName #dropColumn
| ALTER TABLE tableName=qualifiedName
ADD COLUMN column=columnDefinition #addColumn
| CREATE (OR REPLACE)? VIEW qualifiedName AS query #createView
| CREATE (OR REPLACE)? VIEW qualifiedName
(SECURITY (DEFINER | INVOKER))? AS query #createView
| DROP VIEW (IF EXISTS)? qualifiedName #dropView
| CALL qualifiedName '(' (callArgument (',' callArgument)*)? ')' #call
| GRANT
Expand Down Expand Up @@ -455,20 +456,20 @@ nonReserved
: ADD | ALL | ANALYZE | ANY | ARRAY | ASC | AT
| BERNOULLI
| CALL | CASCADE | CATALOGS | COLUMN | COLUMNS | COMMENT | COMMIT | COMMITTED | CURRENT
| DATA | DATE | DAY | DESC | DISTRIBUTED
| DATA | DATE | DAY | DEFINER | DESC | DISTRIBUTED
| EXCLUDING | EXPLAIN
| FILTER | FIRST | FOLLOWING | FORMAT | FUNCTIONS
| GRANT | GRANTS | GRAPHVIZ
| HOUR
| IF | INCLUDING | INPUT | INTERVAL | IO | ISOLATION
| IF | INCLUDING | INPUT | INTERVAL | INVOKER | IO | ISOLATION
| JSON
| LAST | LATERAL | LEVEL | LIMIT | LOGICAL
| MAP | MINUTE | MONTH
| NFC | NFD | NFKC | NFKD | NO | NULLIF | NULLS
| ONLY | OPTION | ORDINALITY | OUTPUT | OVER
| PARTITION | PARTITIONS | PATH | POSITION | PRECEDING | PRIVILEGES | PROPERTIES
| RANGE | READ | RENAME | REPEATABLE | REPLACE | RESET | RESTRICT | REVOKE | ROLLBACK | ROW | ROWS
| SCHEMA | SCHEMAS | SECOND | SERIALIZABLE | SESSION | SET | SETS
| SCHEMA | SCHEMAS | SECOND | SECURITY | SERIALIZABLE | SESSION | SET | SETS
| SHOW | SOME | START | STATS | SUBSTRING | SYSTEM
| TABLES | TABLESAMPLE | TEXT | TIME | TIMESTAMP | TO | TRANSACTION | TRY_CAST | TYPE
| UNBOUNDED | UNCOMMITTED | USE
Expand Down Expand Up @@ -515,6 +516,7 @@ DATA: 'DATA';
DATE: 'DATE';
DAY: 'DAY';
DEALLOCATE: 'DEALLOCATE';
DEFINER: 'DEFINER';
DELETE: 'DELETE';
DESC: 'DESC';
DESCRIBE: 'DESCRIBE';
Expand Down Expand Up @@ -555,6 +557,7 @@ INSERT: 'INSERT';
INTERSECT: 'INTERSECT';
INTERVAL: 'INTERVAL';
INTO: 'INTO';
INVOKER: 'INVOKER';
IO: 'IO';
IS: 'IS';
ISOLATION: 'ISOLATION';
Expand Down Expand Up @@ -617,6 +620,7 @@ ROWS: 'ROWS';
SCHEMA: 'SCHEMA';
SCHEMAS: 'SCHEMAS';
SECOND: 'SECOND';
SECURITY: 'SECURITY';
SELECT: 'SELECT';
SERIALIZABLE: 'SERIALIZABLE';
SESSION: 'SESSION';
Expand Down
Expand Up @@ -531,8 +531,13 @@ protected Void visitCreateView(CreateView node, Integer indent)
builder.append("OR REPLACE ");
}
builder.append("VIEW ")
.append(formatName(node.getName()))
.append(" AS\n");
.append(formatName(node.getName()));

node.getSecurity().ifPresent(security ->
builder.append(" SECURITY ")
.append(security.toString()));

builder.append(" AS\n");

process(node.getQuery(), indent);

Expand Down
Expand Up @@ -369,11 +369,20 @@ public Node visitDropColumn(SqlBaseParser.DropColumnContext context)
@Override
public Node visitCreateView(SqlBaseParser.CreateViewContext context)
{
Optional<CreateView.Security> security = Optional.empty();
if (context.DEFINER() != null) {
security = Optional.of(CreateView.Security.DEFINER);
}
else if (context.INVOKER() != null) {
security = Optional.of(CreateView.Security.INVOKER);
}

return new CreateView(
getLocation(context),
getQualifiedName(context.qualifiedName()),
(Query) visit(context.query()),
context.REPLACE() != null);
context.REPLACE() != null,
security);
}

@Override
Expand Down

0 comments on commit c408a78

Please sign in to comment.