Skip to content

Commit

Permalink
Move testInsertIntoNotNullColumn to base connector test
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed Dec 7, 2021
1 parent 21b8219 commit 5b785cf
Show file tree
Hide file tree
Showing 17 changed files with 103 additions and 122 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_CREATE_VIEW:
return true;

case SUPPORTS_INSERT_NOT_NULL_COLUMN:
return false;

default:
return super.hasBehavior(connectorBehavior);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import java.util.Properties;

import static io.trino.plugin.jdbc.H2QueryRunner.createH2QueryRunner;
import static java.lang.String.format;
import static java.util.Locale.ENGLISH;

// Single-threaded because H2 DDL operations can sometimes take a global lock, leading to apparent deadlocks
// like in https://github.com/trinodb/trino/issues/7209.
Expand Down Expand Up @@ -117,6 +119,12 @@ protected Optional<DataMappingTestSetup> filterDataMappingSmokeTestData(DataMapp
return Optional.of(dataMappingTestSetup);
}

@Override
protected String errorMessageForInsertIntoNotNullColumn(String columnName)
{
return format("NULL not allowed for column \"%s\"(?s).*", columnName.toUpperCase(ENGLISH));
}

@Override
protected JdbcSqlExecutor onRemoteDatabase()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_TOPN_PUSHDOWN:
return false;

case SUPPORTS_INSERT_NOT_NULL_COLUMN:
return false;

case SUPPORTS_DELETE:
return true;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import static io.trino.testing.MaterializedResult.resultBuilder;
import static io.trino.testing.assertions.Assert.assertEquals;
import static io.trino.testing.sql.TestTable.randomTableSuffix;
import static java.lang.String.format;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.testng.Assert.assertFalse;
Expand Down Expand Up @@ -433,6 +434,25 @@ protected TestTable createTableWithDoubleAndRealColumns(String name, List<String
return new TestTable(onRemoteDatabase(), name, "(t_double Nullable(Float64), u_double Nullable(Float64), v_real Nullable(Float32), w_real Nullable(Float32)) Engine=Log", rows);
}

@Test
@Override
public void testInsertIntoNotNullColumn()
{
try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_not_null_", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) {
assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1);
assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)");
// ClickHouse inserts default values (e.g. 0 for integer column) even if we don't specify default clause in CREATE TABLE statement
assertUpdate(format("INSERT INTO %s (nullable_col) VALUES (1)", table.getName()), 1);
assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2), (1, 0)");
}

try (TestTable table = new TestTable(getQueryRunner()::execute, "test_commuted_not_null_table", "(nullable_col BIGINT, not_null_col BIGINT NOT NULL)")) {
assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1);
assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)");
assertQueryFails(format("INSERT INTO %s (not_null_col, nullable_col) VALUES (NULL, 3)", table.getName()), "NULL value not allowed for NOT NULL column: not_null_col");
}
}

@Override
protected SqlExecutor onRemoteDatabase()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -216,6 +216,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_CREATE_VIEW:
return true;

case SUPPORTS_INSERT_NOT_NULL_COLUMN:
return false;

case SUPPORTS_DELETE:
return true;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1021,20 +1021,10 @@ private long getLatestSnapshotId(String tableName)
.getOnlyValue();
}

@Test
public void testInsertIntoNotNullColumn()
{
assertUpdate("CREATE TABLE test_not_null_table (c1 INTEGER, c2 INTEGER NOT NULL)");
assertUpdate("INSERT INTO test_not_null_table (c2) VALUES (2)", 1);
assertQuery("SELECT * FROM test_not_null_table", "VALUES (NULL, 2)");
assertQueryFails("INSERT INTO test_not_null_table (c1) VALUES (1)", "NULL value not allowed for NOT NULL column: c2");
dropTable("test_not_null_table");

assertUpdate("CREATE TABLE test_commuted_not_null_table (a BIGINT, b BIGINT NOT NULL)");
assertUpdate("INSERT INTO test_commuted_not_null_table (b) VALUES (2)", 1);
assertQuery("SELECT * FROM test_commuted_not_null_table", "VALUES (NULL, 2)");
assertQueryFails("INSERT INTO test_commuted_not_null_table (b, a) VALUES (NULL, 3)", "NULL value not allowed for NOT NULL column: b");
dropTable("test_commuted_not_null_table");
@Override
protected String errorMessageForInsertIntoNotNullColumn(String columnName)
{
return "NULL value not allowed for NOT NULL column: " + columnName;
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_CREATE_VIEW:
return true;

case SUPPORTS_INSERT_NOT_NULL_COLUMN:
return false;

default:
return super.hasBehavior(connectorBehavior);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import io.trino.testing.TestingConnectorBehavior;
import io.trino.testing.sql.SqlExecutor;
import io.trino.testing.sql.TestTable;
import org.intellij.lang.annotations.Language;
import org.testng.SkipException;
import org.testng.annotations.AfterClass;
import org.testng.annotations.Test;
Expand Down Expand Up @@ -237,42 +236,10 @@ public void testCharTrailingSpace()
assertUpdate("DROP TABLE char_trailing_space");
}

@Test
public void testInsertIntoNotNullColumn()
@Override
protected String errorMessageForInsertIntoNotNullColumn(String columnName)
{
@Language("SQL") String createTableSql = format("" +
"CREATE TABLE %s.tpch.test_insert_not_null (\n" +
" column_a date,\n" +
" column_b date NOT NULL\n" +
")",
getSession().getCatalog().get());
assertUpdate(createTableSql);
assertEquals(computeScalar("SHOW CREATE TABLE test_insert_not_null"), createTableSql);

assertQueryFails("INSERT INTO test_insert_not_null (column_a) VALUES (date '2012-12-31')", ".* Field 'column_b' doesn't have a default value");
assertQueryFails("INSERT INTO test_insert_not_null (column_a, column_b) VALUES (date '2012-12-31', null)", "NULL value not allowed for NOT NULL column: column_b");

assertUpdate("ALTER TABLE test_insert_not_null ADD COLUMN column_c BIGINT NOT NULL");

createTableSql = format("" +
"CREATE TABLE %s.tpch.test_insert_not_null (\n" +
" column_a date,\n" +
" column_b date NOT NULL,\n" +
" column_c bigint NOT NULL\n" +
")",
getSession().getCatalog().get());
assertEquals(computeScalar("SHOW CREATE TABLE test_insert_not_null"), createTableSql);

assertQueryFails("INSERT INTO test_insert_not_null (column_b) VALUES (date '2012-12-31')", ".* Field 'column_c' doesn't have a default value");
assertQueryFails("INSERT INTO test_insert_not_null (column_b, column_c) VALUES (date '2012-12-31', null)", "NULL value not allowed for NOT NULL column: column_c");

assertUpdate("INSERT INTO test_insert_not_null (column_b, column_c) VALUES (date '2012-12-31', 1)", 1);
assertUpdate("INSERT INTO test_insert_not_null (column_a, column_b, column_c) VALUES (date '2013-01-01', date '2013-01-02', 2)", 1);
assertQuery(
"SELECT * FROM test_insert_not_null",
"VALUES (NULL, CAST('2012-12-31' AS DATE), 1), (CAST('2013-01-01' AS DATE), CAST('2013-01-02' AS DATE), 2)");

assertUpdate("DROP TABLE test_insert_not_null");
return format(".* Field '%s' doesn't have a default value", columnName);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_RENAME_TABLE:
return false;

case SUPPORTS_INSERT_NOT_NULL_COLUMN:
return false;

default:
return super.hasBehavior(connectorBehavior);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import io.trino.testing.MaterializedRow;
import io.trino.testing.TestingConnectorBehavior;
import io.trino.testing.sql.TestTable;
import org.intellij.lang.annotations.Language;
import org.testng.annotations.Test;

import java.util.Optional;
Expand Down Expand Up @@ -254,42 +253,10 @@ public void testCharTrailingSpace()
assertUpdate("DROP TABLE char_trailing_space");
}

@Test
public void testInsertIntoNotNullColumn()
@Override
protected String errorMessageForInsertIntoNotNullColumn(String columnName)
{
@Language("SQL") String createTableSql = format("" +
"CREATE TABLE %s.tpch.test_insert_not_null (\n" +
" column_a date,\n" +
" column_b date NOT NULL\n" +
")",
getSession().getCatalog().get());
assertUpdate(createTableSql);
assertEquals(computeScalar("SHOW CREATE TABLE test_insert_not_null"), createTableSql);

assertQueryFails("INSERT INTO test_insert_not_null (column_a) VALUES (date '2012-12-31')", "Failed to insert data: Field 'column_b' doesn't have a default value");
assertQueryFails("INSERT INTO test_insert_not_null (column_a, column_b) VALUES (date '2012-12-31', null)", "NULL value not allowed for NOT NULL column: column_b");

assertUpdate("ALTER TABLE test_insert_not_null ADD COLUMN column_c BIGINT NOT NULL");

createTableSql = format("" +
"CREATE TABLE %s.tpch.test_insert_not_null (\n" +
" column_a date,\n" +
" column_b date NOT NULL,\n" +
" column_c bigint NOT NULL\n" +
")",
getSession().getCatalog().get());
assertEquals(computeScalar("SHOW CREATE TABLE test_insert_not_null"), createTableSql);

assertQueryFails("INSERT INTO test_insert_not_null (column_b) VALUES (date '2012-12-31')", "Failed to insert data: Field 'column_c' doesn't have a default value");
assertQueryFails("INSERT INTO test_insert_not_null (column_b, column_c) VALUES (date '2012-12-31', null)", "NULL value not allowed for NOT NULL column: column_c");

assertUpdate("INSERT INTO test_insert_not_null (column_b, column_c) VALUES (date '2012-12-31', 1)", 1);
assertUpdate("INSERT INTO test_insert_not_null (column_a, column_b, column_c) VALUES (date '2013-01-01', date '2013-01-02', 2)", 1);
assertQuery(
"SELECT * FROM test_insert_not_null",
"VALUES (NULL, CAST('2012-12-31' AS DATE), 1), (CAST('2013-01-01' AS DATE), CAST('2013-01-02' AS DATE), 2)");

assertUpdate("DROP TABLE test_insert_not_null");
return format("Failed to insert data: Field '%s' doesn't have a default value", columnName);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import static io.trino.testing.assertions.Assert.assertEquals;
import static io.trino.testing.sql.TestTable.randomTableSuffix;
import static java.lang.String.format;
import static java.util.Locale.ENGLISH;
import static org.assertj.core.api.Assertions.assertThat;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;
Expand Down Expand Up @@ -394,6 +395,12 @@ public void testTooLargeDomainCompactionThreshold()
"SELECT * from nation", "Domain compaction threshold \\(10000\\) cannot exceed 1000");
}

@Override
protected String errorMessageForInsertIntoNotNullColumn(String columnName)
{
return format("ORA-01400: cannot insert NULL into \\(.*\"%s\"\\)\n", columnName.toUpperCase(ENGLISH));
}

private void predicatePushdownTest(String oracleType, String oracleLiteral, String operator, String filterLiteral)
{
String tableName = ("test_pdown_" + oracleType.replaceAll("[^a-zA-Z0-9]", ""))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_TRUNCATE:
return false;

case SUPPORTS_INSERT_NOT_NULL_COLUMN:
return false;

default:
return super.hasBehavior(connectorBehavior);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_TRUNCATE:
return false;

case SUPPORTS_INSERT_NOT_NULL_COLUMN:
return false;

default:
return super.hasBehavior(connectorBehavior);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import io.trino.testing.sql.SqlExecutor;
import io.trino.testing.sql.TestTable;
import io.trino.testing.sql.TestView;
import org.intellij.lang.annotations.Language;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

Expand Down Expand Up @@ -499,42 +498,10 @@ public void testCharTrailingSpace()
assertUpdate("DROP TABLE char_trailing_space");
}

@Test
public void testInsertIntoNotNullColumn()
@Override
protected String errorMessageForInsertIntoNotNullColumn(String columnName)
{
@Language("SQL") String createTableSql = format("" +
"CREATE TABLE %s.tpch.test_insert_not_null (\n" +
" column_a date,\n" +
" column_b date NOT NULL\n" +
")",
getSession().getCatalog().get());
assertUpdate(createTableSql);
assertEquals(computeScalar("SHOW CREATE TABLE test_insert_not_null"), createTableSql);

assertQueryFails("INSERT INTO test_insert_not_null (column_a) VALUES (date '2012-12-31')", "(?s).*null value in column \"column_b\" violates not-null constraint.*");
assertQueryFails("INSERT INTO test_insert_not_null (column_a, column_b) VALUES (date '2012-12-31', null)", "NULL value not allowed for NOT NULL column: column_b");

assertUpdate("ALTER TABLE test_insert_not_null ADD COLUMN column_c BIGINT NOT NULL");

createTableSql = format("" +
"CREATE TABLE %s.tpch.test_insert_not_null (\n" +
" column_a date,\n" +
" column_b date NOT NULL,\n" +
" column_c bigint NOT NULL\n" +
")",
getSession().getCatalog().get());
assertEquals(computeScalar("SHOW CREATE TABLE test_insert_not_null"), createTableSql);

assertQueryFails("INSERT INTO test_insert_not_null (column_b) VALUES (date '2012-12-31')", "(?s).*null value in column \"column_c\" violates not-null constraint.*");
assertQueryFails("INSERT INTO test_insert_not_null (column_b, column_c) VALUES (date '2012-12-31', null)", "NULL value not allowed for NOT NULL column: column_c");

assertUpdate("INSERT INTO test_insert_not_null (column_b, column_c) VALUES (date '2012-12-31', 1)", 1);
assertUpdate("INSERT INTO test_insert_not_null (column_a, column_b, column_c) VALUES (date '2013-01-01', date '2013-01-02', 2)", 1);
assertQuery(
"SELECT * FROM test_insert_not_null",
"VALUES (NULL, CAST('2012-12-31' AS DATE), 1), (CAST('2013-01-01' AS DATE), CAST('2013-01-02' AS DATE), 2)");

assertUpdate("DROP TABLE test_insert_not_null");
return format("(?s).*null value in column \"%s\" violates not-null constraint.*", columnName);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -453,6 +453,12 @@ public void testShowCreateForUniqueConstraintCompressedTable()
assertUpdate("DROP TABLE test_show_unique_constraint_table");
}

@Override
protected String errorMessageForInsertIntoNotNullColumn(String columnName)
{
return format("Cannot insert the value NULL into column '%s'.*", columnName);
}

private String getLongInClause(int start, int length)
{
String longValues = range(start, start + length)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DELETE;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DROP_COLUMN;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT_NOT_NULL_COLUMN;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_MULTI_STATEMENT_WRITES;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_COLUMN;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_MATERIALIZED_VIEW;
Expand Down Expand Up @@ -1171,6 +1172,32 @@ public void testRenameColumn()
super.testRenameColumn();
}

@Test
public void testInsertIntoNotNullColumn()
{
skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE) && hasBehavior(SUPPORTS_INSERT_NOT_NULL_COLUMN));

try (TestTable table = new TestTable(getQueryRunner()::execute, "insert_not_null", "(nullable_col INTEGER, not_null_col INTEGER NOT NULL)")) {
assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1);
assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)");
// The error message comes from remote databases when ConnectorMetadata.supportsMissingColumnsOnInsert is true
assertQueryFails(format("INSERT INTO %s (nullable_col) VALUES (1)", table.getName()), errorMessageForInsertIntoNotNullColumn("not_null_col"));
}

try (TestTable table = new TestTable(getQueryRunner()::execute, "commuted_not_null", "(nullable_col BIGINT, not_null_col BIGINT NOT NULL)")) {
assertUpdate(format("INSERT INTO %s (not_null_col) VALUES (2)", table.getName()), 1);
assertQuery("SELECT * FROM " + table.getName(), "VALUES (NULL, 2)");
// This is enforced by the engine and not the connector
assertQueryFails(format("INSERT INTO %s (not_null_col, nullable_col) VALUES (NULL, 3)", table.getName()), "NULL value not allowed for NOT NULL column: not_null_col");
}
}

@Language("RegExp")
protected String errorMessageForInsertIntoNotNullColumn(String columnName)
{
throw new UnsupportedOperationException("This method should be overridden");
}

@Test
public void verifySupportsDeleteDeclaration()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ public enum TestingConnectorBehavior
SUPPORTS_RENAME_MATERIALIZED_VIEW_ACROSS_SCHEMAS(SUPPORTS_RENAME_MATERIALIZED_VIEW),

SUPPORTS_INSERT,
SUPPORTS_INSERT_NOT_NULL_COLUMN(SUPPORTS_INSERT),

SUPPORTS_DELETE(false),
SUPPORTS_ROW_LEVEL_DELETE(SUPPORTS_DELETE),
Expand Down

0 comments on commit 5b785cf

Please sign in to comment.