Skip to content

Commit

Permalink
fix create view on exist table throw view exist exception
Browse files Browse the repository at this point in the history
  • Loading branch information
kyo-tom authored and sopel39 committed Dec 22, 2021
1 parent b56c4e3 commit 7053fb1
Show file tree
Hide file tree
Showing 5 changed files with 181 additions and 7 deletions.
Expand Up @@ -36,8 +36,10 @@
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
import static io.trino.metadata.MetadataUtil.createQualifiedObjectName;
import static io.trino.spi.StandardErrorCode.TABLE_ALREADY_EXISTS;
import static io.trino.sql.ParameterUtils.parameterExtractor;
import static io.trino.sql.SqlFormatterUtil.getFormattedSql;
import static io.trino.sql.analyzer.SemanticExceptions.semanticException;
import static io.trino.sql.tree.CreateView.Security.INVOKER;
import static java.util.Objects.requireNonNull;

Expand Down Expand Up @@ -76,6 +78,18 @@ public ListenableFuture<Void> execute(

accessControl.checkCanCreateView(session.toSecurityContext(), name);

if (metadata.isMaterializedView(session, name)) {
throw semanticException(TABLE_ALREADY_EXISTS, statement, "Materialized view already exists: '%s'", name);
}
if (metadata.isView(session, name)) {
if (!statement.isReplace()) {
throw semanticException(TABLE_ALREADY_EXISTS, statement, "View already exists: '%s'", name);
}
}
else if (metadata.getTableHandle(session, name).isPresent()) {
throw semanticException(TABLE_ALREADY_EXISTS, statement, "Table already exists: '%s'", name);
}

String sql = getFormattedSql(statement.getQuery(), sqlParser);

Analysis analysis = analyzerFactory.createAnalyzer(session, parameters, parameterExtractor(statement, parameters), stateMachine.getWarningCollector())
Expand Down
@@ -0,0 +1,160 @@
/*
* 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.trino.execution;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.util.concurrent.ListenableFuture;
import io.trino.execution.warnings.WarningCollector;
import io.trino.metadata.AnalyzePropertyManager;
import io.trino.metadata.MetadataManager;
import io.trino.metadata.QualifiedObjectName;
import io.trino.metadata.TablePropertyManager;
import io.trino.security.AccessControl;
import io.trino.security.AllowAllAccessControl;
import io.trino.spi.resourcegroups.ResourceGroupId;
import io.trino.sql.analyzer.AnalyzerFactory;
import io.trino.sql.parser.SqlParser;
import io.trino.sql.rewrite.StatementRewrite;
import io.trino.sql.tree.AllColumns;
import io.trino.sql.tree.CreateView;
import io.trino.sql.tree.QualifiedName;
import io.trino.sql.tree.Query;
import io.trino.transaction.TransactionManager;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

import java.net.URI;
import java.util.Optional;

import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static io.airlift.concurrent.MoreFutures.getFutureValue;
import static io.trino.spi.StandardErrorCode.TABLE_ALREADY_EXISTS;
import static io.trino.sql.QueryUtil.selectList;
import static io.trino.sql.QueryUtil.simpleQuery;
import static io.trino.sql.QueryUtil.table;
import static io.trino.sql.analyzer.StatementAnalyzerFactory.createTestingStatementAnalyzerFactory;
import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy;
import static org.assertj.core.api.Assertions.assertThat;

@Test(singleThreaded = true)
public class TestCreateViewTask
extends BaseDataDefinitionTaskTest
{
private static final String CATALOG_NAME = "catalog";
private SqlParser parser;
private AnalyzerFactory analyzerFactory;

@Override
@BeforeMethod
public void setUp()
{
super.setUp();
parser = new SqlParser();
analyzerFactory = new AnalyzerFactory(createTestingStatementAnalyzerFactory(plannerContext, new AllowAllAccessControl(), new TablePropertyManager(), new AnalyzePropertyManager()), new StatementRewrite(ImmutableSet.of()));
QualifiedObjectName tableName = qualifiedObjectName("mock_table");
metadata.createTable(testSession, CATALOG_NAME, someTable(tableName), false);
}

@Test
public void testCreateViewOnViewIfNotExists()
{
QualifiedObjectName viewName = qualifiedObjectName("new_view");
getFutureValue(executeCreateView(asQualifiedName(viewName), false));
assertThat(metadata.isView(testSession, viewName)).isTrue();
}

@Test
public void testCreateViewOnViewIfExists()
{
QualifiedObjectName viewName = qualifiedObjectName("existing_view");
metadata.createView(testSession, viewName, someView(), false);

assertTrinoExceptionThrownBy(() -> getFutureValue(executeCreateView(asQualifiedName(viewName), false)))
.hasErrorCode(TABLE_ALREADY_EXISTS)
.hasMessage("View already exists: '%s'", viewName);
}

@Test
public void testReplaceViewOnViewIfExists()
{
QualifiedObjectName viewName = qualifiedObjectName("existing_view");
metadata.createView(testSession, viewName, someView(), false);

getFutureValue(executeCreateView(asQualifiedName(viewName), true));
assertThat(metadata.isView(testSession, viewName)).isTrue();
}

@Test
public void testCreateViewOnTableIfExists()
{
QualifiedObjectName tableName = qualifiedObjectName("existing_table");
metadata.createTable(testSession, CATALOG_NAME, someTable(tableName), false);

assertTrinoExceptionThrownBy(() -> getFutureValue(executeCreateView(asQualifiedName(tableName), false)))
.hasErrorCode(TABLE_ALREADY_EXISTS)
.hasMessage("Table already exists: '%s'", tableName, tableName);
}

@Test
public void testReplaceViewOnTableIfExists()
{
QualifiedObjectName tableName = qualifiedObjectName("existing_table");
metadata.createTable(testSession, CATALOG_NAME, someTable(tableName), false);

assertTrinoExceptionThrownBy(() -> getFutureValue(executeCreateView(asQualifiedName(tableName), true)))
.hasErrorCode(TABLE_ALREADY_EXISTS)
.hasMessage("Table already exists: '%s'", tableName, tableName);
}

@Test
public void testCreateViewOnMaterializedView()
{
QualifiedObjectName viewName = qualifiedObjectName("existing_materialized_view");
metadata.createMaterializedView(testSession, viewName, someMaterializedView(), false, false);

assertTrinoExceptionThrownBy(() -> getFutureValue(executeCreateView(asQualifiedName(viewName), false)))
.hasErrorCode(TABLE_ALREADY_EXISTS)
.hasMessage("Materialized view already exists: '%s'", viewName);
}

private QueryStateMachine stateMachine(TransactionManager transactionManager, MetadataManager metadata, AccessControl accessControl)
{
return QueryStateMachine.begin(
"test",
Optional.empty(),
testSession,
URI.create("fake://uri"),
new ResourceGroupId("test"),
false,
transactionManager,
accessControl,
directExecutor(),
metadata,
WarningCollector.NOOP,
Optional.empty());
}

private ListenableFuture<Void> executeCreateView(QualifiedName viewName, boolean replace)
{
Query query = simpleQuery(selectList(new AllColumns()), table(QualifiedName.of("mock_table")));
CreateView statement = new CreateView(
viewName,
query,
replace,
Optional.empty(),
Optional.empty());
return new CreateViewTask(metadata, new AllowAllAccessControl(), parser, analyzerFactory).execute(statement, queryStateMachine, ImmutableList.of(), WarningCollector.NOOP);
}
}
Expand Up @@ -556,7 +556,7 @@ public void testViews()
assertUpdate("CREATE OR REPLACE VIEW test_view AS " + query);

assertQueryFails("CREATE TABLE test_view (x date)", "View \\[default.test_view] already exists");
assertQueryFails("CREATE VIEW test_view AS SELECT 123 x", "View already exists: default.test_view");
assertQueryFails("CREATE VIEW test_view AS SELECT 123 x", ".*View already exists: 'memory.default.test_view'");

assertQuery("SELECT * FROM test_view", query);

Expand Down
Expand Up @@ -100,10 +100,10 @@ public void testCreateTableViewAlreadyExists()
public void testCreateViewTableAlreadyExists()
{
assertUpdate("CREATE TABLE table_already_exists (id integer)");
assertQueryFails("CREATE VIEW table_already_exists AS SELECT 1 a", "Table already exists: tpch.table_already_exists");
assertQueryFails("CREATE VIEW Table_Already_Exists AS SELECT 1 a", "Table already exists: tpch.table_already_exists");
assertQueryFails("CREATE OR REPLACE VIEW table_already_exists AS SELECT 1 a", "Table already exists: tpch.table_already_exists");
assertQueryFails("CREATE OR REPLACE VIEW Table_Already_Exists AS SELECT 1 a", "Table already exists: tpch.table_already_exists");
assertQueryFails("CREATE VIEW table_already_exists AS SELECT 1 a", ".*Table already exists: 'raptor.tpch.table_already_exists'");
assertQueryFails("CREATE VIEW Table_Already_Exists AS SELECT 1 a", ".*Table already exists: 'raptor.tpch.table_already_exists'");
assertQueryFails("CREATE OR REPLACE VIEW table_already_exists AS SELECT 1 a", ".*Table already exists: 'raptor.tpch.table_already_exists'");
assertQueryFails("CREATE OR REPLACE VIEW Table_Already_Exists AS SELECT 1 a", ".*Table already exists: 'raptor.tpch.table_already_exists'");
assertUpdate("DROP TABLE table_already_exists");
}

Expand Down
Expand Up @@ -413,13 +413,13 @@ public void testReferencedTablesWithMaterializedViews()
public void testReferencedTablesInCreateView()
throws Exception
{
runQueryAndWaitForEvents("CREATE VIEW mock.default.test_view AS SELECT * FROM nation", 2);
runQueryAndWaitForEvents("CREATE VIEW mock.default.create_another_test_view AS SELECT * FROM nation", 2);

QueryCompletedEvent event = getOnlyElement(generatedEvents.getQueryCompletedEvents());

assertThat(event.getIoMetadata().getOutput().get().getCatalogName()).isEqualTo("mock");
assertThat(event.getIoMetadata().getOutput().get().getSchema()).isEqualTo("default");
assertThat(event.getIoMetadata().getOutput().get().getTable()).isEqualTo("test_view");
assertThat(event.getIoMetadata().getOutput().get().getTable()).isEqualTo("create_another_test_view");
assertThat(event.getIoMetadata().getOutput().get().getColumns().get())
.containsExactly(
new OutputColumnMetadata("nationkey", BIGINT_TYPE, ImmutableSet.of(new ColumnDetail("tpch", "tiny", "nation", "nationkey"))),
Expand Down

0 comments on commit 7053fb1

Please sign in to comment.