Skip to content

Commit

Permalink
Add owner to view definition
Browse files Browse the repository at this point in the history
  • Loading branch information
dain committed Sep 16, 2015
1 parent 71e16fe commit b438df6
Show file tree
Hide file tree
Showing 12 changed files with 112 additions and 34 deletions.
Expand Up @@ -25,6 +25,7 @@
import com.facebook.presto.spi.ConnectorTableLayoutHandle; import com.facebook.presto.spi.ConnectorTableLayoutHandle;
import com.facebook.presto.spi.ConnectorTableLayoutResult; import com.facebook.presto.spi.ConnectorTableLayoutResult;
import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorTableMetadata;
import com.facebook.presto.spi.ConnectorViewDefinition;
import com.facebook.presto.spi.Constraint; import com.facebook.presto.spi.Constraint;
import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.SchemaNotFoundException; import com.facebook.presto.spi.SchemaNotFoundException;
Expand Down Expand Up @@ -693,7 +694,7 @@ public void createView(ConnectorSession session, SchemaTableName viewName, Strin
@Override @Override
public void dropView(ConnectorSession session, SchemaTableName viewName) public void dropView(ConnectorSession session, SchemaTableName viewName)
{ {
String view = getViews(session, viewName.toSchemaTablePrefix()).get(viewName); ConnectorViewDefinition view = getViews(session, viewName.toSchemaTablePrefix()).get(viewName);
if (view == null) { if (view == null) {
throw new ViewNotFoundException(viewName); throw new ViewNotFoundException(viewName);
} }
Expand All @@ -719,9 +720,9 @@ public List<SchemaTableName> listViews(ConnectorSession session, String schemaNa
} }


@Override @Override
public Map<SchemaTableName, String> getViews(ConnectorSession session, SchemaTablePrefix prefix) public Map<SchemaTableName, ConnectorViewDefinition> getViews(ConnectorSession session, SchemaTablePrefix prefix)
{ {
ImmutableMap.Builder<SchemaTableName, String> views = ImmutableMap.builder(); ImmutableMap.Builder<SchemaTableName, ConnectorViewDefinition> views = ImmutableMap.builder();
List<SchemaTableName> tableNames; List<SchemaTableName> tableNames;
if (prefix.getTableName() != null) { if (prefix.getTableName() != null) {
tableNames = ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName())); tableNames = ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName()));
Expand All @@ -733,7 +734,10 @@ public Map<SchemaTableName, String> getViews(ConnectorSession session, SchemaTab
for (SchemaTableName schemaTableName : tableNames) { for (SchemaTableName schemaTableName : tableNames) {
Optional<Table> table = metastore.getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()); Optional<Table> table = metastore.getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName());
if (table.isPresent() && HiveUtil.isPrestoView(table.get())) { if (table.isPresent() && HiveUtil.isPrestoView(table.get())) {
views.put(schemaTableName, decodeViewData(table.get().getViewOriginalText())); views.put(schemaTableName, new ConnectorViewDefinition(
schemaTableName,
Optional.ofNullable(table.get().getOwner()),
decodeViewData(table.get().getViewOriginalText())));
} }
} }


Expand Down
Expand Up @@ -36,6 +36,7 @@
import com.facebook.presto.spi.ConnectorTableLayoutHandle; import com.facebook.presto.spi.ConnectorTableLayoutHandle;
import com.facebook.presto.spi.ConnectorTableLayoutResult; import com.facebook.presto.spi.ConnectorTableLayoutResult;
import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorTableMetadata;
import com.facebook.presto.spi.ConnectorViewDefinition;
import com.facebook.presto.spi.Constraint; import com.facebook.presto.spi.Constraint;
import com.facebook.presto.spi.Domain; import com.facebook.presto.spi.Domain;
import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.PrestoException;
Expand Down Expand Up @@ -1296,9 +1297,9 @@ private void doCreateView(SchemaTableName viewName, boolean replace)


metadata.createView(SESSION, viewName, viewData, replace); metadata.createView(SESSION, viewName, viewData, replace);


Map<SchemaTableName, String> views = metadata.getViews(SESSION, viewName.toSchemaTablePrefix()); Map<SchemaTableName, ConnectorViewDefinition> views = metadata.getViews(SESSION, viewName.toSchemaTablePrefix());
assertEquals(views.size(), 1); assertEquals(views.size(), 1);
assertEquals(views.get(viewName), viewData); assertEquals(views.get(viewName).getViewData(), viewData);


assertTrue(metadata.listViews(SESSION, viewName.getSchemaName()).contains(viewName)); assertTrue(metadata.listViews(SESSION, viewName.getSchemaName()).contains(viewName));
} }
Expand Down
Expand Up @@ -92,7 +92,7 @@ public void execute(CreateView statement, Session session, Metadata metadata, Ac
.map(field -> new ViewColumn(field.getName().get(), field.getType())) .map(field -> new ViewColumn(field.getName().get(), field.getType()))
.collect(toImmutableList()); .collect(toImmutableList());


String data = codec.toJson(new ViewDefinition(sql, session.getCatalog(), session.getSchema(), columns)); String data = codec.toJson(new ViewDefinition(sql, session.getCatalog(), session.getSchema(), columns, Optional.of(session.getUser())));


metadata.createView(session, name, data, statement.isReplace()); metadata.createView(session, name, data, statement.isReplace());
} }
Expand Down
Expand Up @@ -29,6 +29,7 @@
import com.facebook.presto.spi.ConnectorTableLayout; import com.facebook.presto.spi.ConnectorTableLayout;
import com.facebook.presto.spi.ConnectorTableLayoutResult; import com.facebook.presto.spi.ConnectorTableLayoutResult;
import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorTableMetadata;
import com.facebook.presto.spi.ConnectorViewDefinition;
import com.facebook.presto.spi.Constraint; import com.facebook.presto.spi.Constraint;
import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTableName;
Expand Down Expand Up @@ -455,14 +456,14 @@ public Map<QualifiedTableName, List<ColumnMetadata>> listTableColumns(Session se
} }


// if table and view names overlap, the view wins // if table and view names overlap, the view wins
for (Entry<SchemaTableName, String> entry : metadata.getViews(connectorSession, tablePrefix).entrySet()) { for (Entry<SchemaTableName, ConnectorViewDefinition> entry : metadata.getViews(connectorSession, tablePrefix).entrySet()) {
QualifiedTableName tableName = new QualifiedTableName( QualifiedTableName tableName = new QualifiedTableName(
prefix.getCatalogName(), prefix.getCatalogName(),
entry.getKey().getSchemaName(), entry.getKey().getSchemaName(),
entry.getKey().getTableName()); entry.getKey().getTableName());


ImmutableList.Builder<ColumnMetadata> columns = ImmutableList.builder(); ImmutableList.Builder<ColumnMetadata> columns = ImmutableList.builder();
for (ViewColumn column : deserializeView(entry.getValue()).getColumns()) { for (ViewColumn column : deserializeView(entry.getValue().getViewData()).getColumns()) {
columns.add(new ColumnMetadata(column.getName(), column.getType(), false)); columns.add(new ColumnMetadata(column.getName(), column.getType(), false));
} }


Expand Down Expand Up @@ -621,12 +622,12 @@ public Map<QualifiedTableName, ViewDefinition> getViews(Session session, Qualifi
Map<QualifiedTableName, ViewDefinition> views = new LinkedHashMap<>(); Map<QualifiedTableName, ViewDefinition> views = new LinkedHashMap<>();
for (ConnectorMetadataEntry metadata : allConnectorsFor(prefix.getCatalogName())) { for (ConnectorMetadataEntry metadata : allConnectorsFor(prefix.getCatalogName())) {
ConnectorSession connectorSession = session.toConnectorSession(metadata.getCatalog()); ConnectorSession connectorSession = session.toConnectorSession(metadata.getCatalog());
for (Entry<SchemaTableName, String> entry : metadata.getMetadata().getViews(connectorSession, tablePrefix).entrySet()) { for (Entry<SchemaTableName, ConnectorViewDefinition> entry : metadata.getMetadata().getViews(connectorSession, tablePrefix).entrySet()) {
QualifiedTableName viewName = new QualifiedTableName( QualifiedTableName viewName = new QualifiedTableName(
prefix.getCatalogName(), prefix.getCatalogName(),
entry.getKey().getSchemaName(), entry.getKey().getSchemaName(),
entry.getKey().getTableName()); entry.getKey().getTableName());
views.put(viewName, deserializeView(entry.getValue())); views.put(viewName, deserializeView(entry.getValue().getViewData()));
} }
} }
return ImmutableMap.copyOf(views); return ImmutableMap.copyOf(views);
Expand All @@ -637,11 +638,13 @@ public Optional<ViewDefinition> getView(Session session, QualifiedTableName view
{ {
ConnectorMetadataEntry entry = getConnectorFor(viewName); ConnectorMetadataEntry entry = getConnectorFor(viewName);
if (entry != null) { if (entry != null) {
SchemaTablePrefix prefix = viewName.asSchemaTableName().toSchemaTablePrefix(); ConnectorMetadata metadata = entry.getMetadata();
Map<SchemaTableName, String> views = entry.getMetadata().getViews(session.toConnectorSession(entry.getCatalog()), prefix); Map<SchemaTableName, ConnectorViewDefinition> views = metadata.getViews(
String view = views.get(viewName.asSchemaTableName()); session.toConnectorSession(entry.getCatalog()),
viewName.asSchemaTableName().toSchemaTablePrefix());
ConnectorViewDefinition view = views.get(viewName.asSchemaTableName());
if (view != null) { if (view != null) {
return Optional.of(deserializeView(view)); return Optional.of(deserializeView(view.getViewData()));
} }
} }
return Optional.empty(); return Optional.empty();
Expand Down
Expand Up @@ -19,6 +19,7 @@
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;


import java.util.List; import java.util.List;
import java.util.Optional;


import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.MoreObjects.toStringHelper;
import static java.util.Objects.requireNonNull; import static java.util.Objects.requireNonNull;
Expand All @@ -29,18 +30,21 @@ public final class ViewDefinition
private final String catalog; private final String catalog;
private final String schema; private final String schema;
private final List<ViewColumn> columns; private final List<ViewColumn> columns;
private final Optional<String> owner;


@JsonCreator @JsonCreator
public ViewDefinition( public ViewDefinition(
@JsonProperty("originalSql") String originalSql, @JsonProperty("originalSql") String originalSql,
@JsonProperty("catalog") String catalog, @JsonProperty("catalog") String catalog,
@JsonProperty("schema") String schema, @JsonProperty("schema") String schema,
@JsonProperty("columns") List<ViewColumn> columns) @JsonProperty("columns") List<ViewColumn> columns,
@JsonProperty("owner") Optional<String> owner)
{ {
this.originalSql = requireNonNull(originalSql, "originalSql is null"); this.originalSql = requireNonNull(originalSql, "originalSql is null");
this.catalog = requireNonNull(catalog, "catalog is null"); this.catalog = requireNonNull(catalog, "catalog is null");
this.schema = requireNonNull(schema, "schema is null"); this.schema = requireNonNull(schema, "schema is null");
this.columns = ImmutableList.copyOf(requireNonNull(columns, "columns is null")); this.columns = ImmutableList.copyOf(requireNonNull(columns, "columns is null"));
this.owner = requireNonNull(owner, "owner is null");
} }


@JsonProperty @JsonProperty
Expand All @@ -67,6 +71,12 @@ public List<ViewColumn> getColumns()
return columns; return columns;
} }


@JsonProperty
public Optional<String> getOwner()
{
return owner;
}

@Override @Override
public String toString() public String toString()
{ {
Expand All @@ -75,6 +85,7 @@ public String toString()
.add("catalog", catalog) .add("catalog", catalog)
.add("schema", schema) .add("schema", schema)
.add("columns", columns) .add("columns", columns)
.add("owner", owner)
.toString(); .toString();
} }


Expand Down
Expand Up @@ -19,6 +19,7 @@
import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.ConnectorTableHandle; import com.facebook.presto.spi.ConnectorTableHandle;
import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorTableMetadata;
import com.facebook.presto.spi.ConnectorViewDefinition;
import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.SchemaTablePrefix; import com.facebook.presto.spi.SchemaTablePrefix;
Expand All @@ -30,6 +31,7 @@
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
Expand Down Expand Up @@ -181,12 +183,12 @@ public List<SchemaTableName> listViews(ConnectorSession session, String schemaNa
} }


@Override @Override
public Map<SchemaTableName, String> getViews(ConnectorSession session, SchemaTablePrefix prefix) public Map<SchemaTableName, ConnectorViewDefinition> getViews(ConnectorSession session, SchemaTablePrefix prefix)
{ {
ImmutableMap.Builder<SchemaTableName, String> map = ImmutableMap.builder(); ImmutableMap.Builder<SchemaTableName, ConnectorViewDefinition> map = ImmutableMap.builder();
for (Map.Entry<SchemaTableName, String> entry : views.entrySet()) { for (Map.Entry<SchemaTableName, String> entry : views.entrySet()) {
if (prefix.matches(entry.getKey())) { if (prefix.matches(entry.getKey())) {
map.put(entry); map.put(entry.getKey(), new ConnectorViewDefinition(entry.getKey(), Optional.empty(), entry.getValue()));
} }
} }
return map.build(); return map.build();
Expand Down
Expand Up @@ -840,20 +840,17 @@ public void setup()


// valid view referencing table in same schema // valid view referencing table in same schema
String viewData1 = JsonCodec.jsonCodec(ViewDefinition.class).toJson( String viewData1 = JsonCodec.jsonCodec(ViewDefinition.class).toJson(
new ViewDefinition("select a from t1", "tpch", "default", ImmutableList.of( new ViewDefinition("select a from t1", "tpch", "default", ImmutableList.of(new ViewColumn("a", BIGINT)), Optional.of("user")));
new ViewColumn("a", BIGINT))));
metadata.createView(SESSION, new QualifiedTableName("tpch", "default", "v1"), viewData1, false); metadata.createView(SESSION, new QualifiedTableName("tpch", "default", "v1"), viewData1, false);


// stale view (different column type) // stale view (different column type)
String viewData2 = JsonCodec.jsonCodec(ViewDefinition.class).toJson( String viewData2 = JsonCodec.jsonCodec(ViewDefinition.class).toJson(
new ViewDefinition("select a from t1", "tpch", "default", ImmutableList.of( new ViewDefinition("select a from t1", "tpch", "default", ImmutableList.of(new ViewColumn("a", VARCHAR)), Optional.of("user")));
new ViewColumn("a", VARCHAR))));
metadata.createView(SESSION, new QualifiedTableName("tpch", "default", "v2"), viewData2, false); metadata.createView(SESSION, new QualifiedTableName("tpch", "default", "v2"), viewData2, false);


// view referencing table in different schema from itself and session // view referencing table in different schema from itself and session
String viewData3 = JsonCodec.jsonCodec(ViewDefinition.class).toJson( String viewData3 = JsonCodec.jsonCodec(ViewDefinition.class).toJson(
new ViewDefinition("select a from t4", "c2", "s2", ImmutableList.of( new ViewDefinition("select a from t4", "c2", "s2", ImmutableList.of(new ViewColumn("a", BIGINT)), Optional.of("user")));
new ViewColumn("a", BIGINT))));
metadata.createView(SESSION, new QualifiedTableName("c3", "s3", "v3"), viewData3, false); metadata.createView(SESSION, new QualifiedTableName("c3", "s3", "v3"), viewData3, false);


analyzer = new Analyzer( analyzer = new Analyzer(
Expand Down
Expand Up @@ -31,6 +31,7 @@
import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.ConnectorTableHandle; import com.facebook.presto.spi.ConnectorTableHandle;
import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorTableMetadata;
import com.facebook.presto.spi.ConnectorViewDefinition;
import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.SchemaTablePrefix; import com.facebook.presto.spi.SchemaTablePrefix;
Expand Down Expand Up @@ -521,11 +522,11 @@ public List<SchemaTableName> listViews(ConnectorSession session, String schemaNa
} }


@Override @Override
public Map<SchemaTableName, String> getViews(ConnectorSession session, SchemaTablePrefix prefix) public Map<SchemaTableName, ConnectorViewDefinition> getViews(ConnectorSession session, SchemaTablePrefix prefix)
{ {
ImmutableMap.Builder<SchemaTableName, String> map = ImmutableMap.builder(); ImmutableMap.Builder<SchemaTableName, ConnectorViewDefinition> map = ImmutableMap.builder();
for (ViewResult view : dao.getViews(prefix.getSchemaName(), prefix.getTableName())) { for (ViewResult view : dao.getViews(prefix.getSchemaName(), prefix.getTableName())) {
map.put(view.getName(), view.getData()); map.put(view.getName(), new ConnectorViewDefinition(view.getName(), Optional.empty(), view.getData()));
} }
return map.build(); return map.build();
} }
Expand Down
Expand Up @@ -23,6 +23,7 @@
import com.facebook.presto.spi.ConnectorMetadata; import com.facebook.presto.spi.ConnectorMetadata;
import com.facebook.presto.spi.ConnectorTableHandle; import com.facebook.presto.spi.ConnectorTableHandle;
import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorTableMetadata;
import com.facebook.presto.spi.ConnectorViewDefinition;
import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.SchemaTablePrefix; import com.facebook.presto.spi.SchemaTablePrefix;
Expand Down Expand Up @@ -305,10 +306,10 @@ public void testViews()
assertEqualsIgnoreOrder(list, ImmutableList.of(test1, test2)); assertEqualsIgnoreOrder(list, ImmutableList.of(test1, test2));


// verify getting data // verify getting data
Map<SchemaTableName, String> views = metadata.getViews(SESSION, new SchemaTablePrefix("test")); Map<SchemaTableName, ConnectorViewDefinition> views = metadata.getViews(SESSION, new SchemaTablePrefix("test"));
assertEquals(views.keySet(), ImmutableSet.of(test1, test2)); assertEquals(views.keySet(), ImmutableSet.of(test1, test2));
assertEquals(views.get(test1), "test1"); assertEquals(views.get(test1).getViewData(), "test1");
assertEquals(views.get(test2), "test2"); assertEquals(views.get(test2).getViewData(), "test2");


// drop first view // drop first view
metadata.dropView(SESSION, test1); metadata.dropView(SESSION, test1);
Expand Down Expand Up @@ -349,7 +350,7 @@ public void testCreateViewWithReplace()
metadata.createView(SESSION, test, "aaa", true); metadata.createView(SESSION, test, "aaa", true);
metadata.createView(SESSION, test, "bbb", true); metadata.createView(SESSION, test, "bbb", true);


assertEquals(metadata.getViews(SESSION, test.toSchemaTablePrefix()).get(test), "bbb"); assertEquals(metadata.getViews(SESSION, test.toSchemaTablePrefix()).get(test).getViewData(), "bbb");
} }


private static ConnectorTableMetadata getOrdersTable() private static ConnectorTableMetadata getOrdersTable()
Expand Down
Expand Up @@ -242,7 +242,7 @@ default List<SchemaTableName> listViews(ConnectorSession session, String schemaN
/** /**
* Gets the view data for views that match the specified table prefix. * Gets the view data for views that match the specified table prefix.
*/ */
default Map<SchemaTableName, String> getViews(ConnectorSession session, SchemaTablePrefix prefix) default Map<SchemaTableName, ConnectorViewDefinition> getViews(ConnectorSession session, SchemaTablePrefix prefix)
{ {
return emptyMap(); return emptyMap();
} }
Expand Down
@@ -0,0 +1,57 @@
/*
* 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.spi;

import java.util.Optional;

import static java.util.Objects.requireNonNull;

public class ConnectorViewDefinition
{
private final SchemaTableName name;
private final Optional<String> owner;
private final String viewData;

public ConnectorViewDefinition(SchemaTableName name, Optional<String> owner, String viewData)
{
this.name = requireNonNull(name, "name is null");
this.owner = requireNonNull(owner, "owner is null");
this.viewData = requireNonNull(viewData, "viewData is null");
}

public SchemaTableName getName()
{
return name;
}

public Optional<String> getOwner()
{
return owner;
}

public String getViewData()
{
return viewData;
}

@Override
public String toString()
{
StringBuilder sb = new StringBuilder("ConnectorViewDefinition{");
sb.append("name=").append(name);
sb.append(", owner=").append(owner);
sb.append('}');
return sb.toString();
}
}
Expand Up @@ -24,6 +24,7 @@
import com.facebook.presto.spi.ConnectorTableLayoutHandle; import com.facebook.presto.spi.ConnectorTableLayoutHandle;
import com.facebook.presto.spi.ConnectorTableLayoutResult; import com.facebook.presto.spi.ConnectorTableLayoutResult;
import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorTableMetadata;
import com.facebook.presto.spi.ConnectorViewDefinition;
import com.facebook.presto.spi.Constraint; import com.facebook.presto.spi.Constraint;
import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.SchemaTablePrefix; import com.facebook.presto.spi.SchemaTablePrefix;
Expand Down Expand Up @@ -222,7 +223,7 @@ public List<SchemaTableName> listViews(ConnectorSession session, String schemaNa
} }


@Override @Override
public Map<SchemaTableName, String> getViews(ConnectorSession session, SchemaTablePrefix prefix) public Map<SchemaTableName, ConnectorViewDefinition> getViews(ConnectorSession session, SchemaTablePrefix prefix)
{ {
try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) {
return delegate.getViews(session, prefix); return delegate.getViews(session, prefix);
Expand Down

0 comments on commit b438df6

Please sign in to comment.