This is an automated email from the ASF dual-hosted git repository. jooger pushed a commit to branch ignite-21673 in repository https://gitbox.apache.org/repos/asf/ignite-3.git
commit a097d7f33ee71e0eae9cccaa5d463ac87ce1e608 Author: Yury Gerzhedovich <[email protected]> AuthorDate: Wed Mar 27 16:26:43 2024 +0200 IGNITE-21673 Do not allow duplicate column names in unique column list of a primary key constraint. --- .../catalog/commands/CreateTableCommand.java | 22 ++++-- .../internal/catalog/commands/TablePrimaryKey.java | 26 +++++-- .../catalog/commands/TableSortedPrimaryKey.java | 6 +- .../commands/CreateTableCommandValidationTest.java | 82 ++++++++++++++++++++++ .../catalog/commands/TablePrimaryKeyTest.java | 22 +++--- .../ignite/internal/sql/api/ItSqlApiBaseTest.java | 40 +++++------ .../internal/sql/engine/ItCreateTableDdlTest.java | 17 +++-- .../prepare/ddl/DdlSqlToCommandConverter.java | 56 ++++----------- 8 files changed, 176 insertions(+), 95 deletions(-) diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateTableCommand.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateTableCommand.java index 0cef7d0548..24c8e32209 100644 --- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateTableCommand.java +++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateTableCommand.java @@ -36,6 +36,7 @@ import org.apache.ignite.internal.catalog.Catalog; import org.apache.ignite.internal.catalog.CatalogCommand; import org.apache.ignite.internal.catalog.CatalogService; import org.apache.ignite.internal.catalog.CatalogValidationException; +import org.apache.ignite.internal.catalog.commands.DefaultValue.Type; import org.apache.ignite.internal.catalog.descriptors.CatalogColumnCollation; import org.apache.ignite.internal.catalog.descriptors.CatalogHashIndexDescriptor; import org.apache.ignite.internal.catalog.descriptors.CatalogIndexColumnDescriptor; @@ -138,32 +139,39 @@ public class CreateTableCommand extends AbstractTableCommand { private void validate() { if (nullOrEmpty(columns)) { - throw new CatalogValidationException("Table should have at least one column"); + throw new CatalogValidationException("Table should have at least one column."); } Set<String> columnNames = new HashSet<>(); - for (ColumnParams column : columns) { if (!columnNames.add(column.name())) { - throw new CatalogValidationException(format("Column with name '{}' specified more than once", column.name())); + throw new CatalogValidationException(format("Column with name '{}' specified more than once.", column.name())); } } if (primaryKey == null || nullOrEmpty(primaryKey.columns())) { - throw new CatalogValidationException("Table should have primary key"); + throw new CatalogValidationException("Table should have primary key."); } - primaryKey.validate(columnNames); + primaryKey.validate(columns); + + for (ColumnParams column : columns) { + boolean partOfPk = primaryKey.columns().contains(column.name()); + if (!partOfPk && column.defaultValueDefinition().type == Type.FUNCTION_CALL) { + throw new CatalogValidationException( + format("Functional defaults are not supported for non-primary key columns [col={}].", column.name())); + } + } if (nullOrEmpty(colocationColumns)) { - throw new CatalogValidationException("Colocation columns could not be empty"); + throw new CatalogValidationException("Colocation columns could not be empty."); } Set<String> colocationColumnsSet = new HashSet<>(); for (String name : colocationColumns) { if (!primaryKey.columns().contains(name)) { - throw new CatalogValidationException(format("Colocation column '{}' is not part of PK", name)); + throw new CatalogValidationException(format("Colocation column '{}' is not part of PK.", name)); } if (!colocationColumnsSet.add(name)) { diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/TablePrimaryKey.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/TablePrimaryKey.java index a8a9ef3ba8..f00d471c0e 100644 --- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/TablePrimaryKey.java +++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/TablePrimaryKey.java @@ -22,6 +22,7 @@ import static org.apache.ignite.internal.lang.IgniteStringFormatter.format; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.stream.Collectors; import org.apache.ignite.internal.catalog.CatalogValidationException; /** Base class for a primary key. */ @@ -44,16 +45,29 @@ public abstract class TablePrimaryKey { } /** Performs additional validation of this primary key. */ - void validate(Set<String> allColumns) { - Set<String> columnSet = new HashSet<>(); + void validate(List<ColumnParams> allColumns) { + Set<String> allColumnNames = new HashSet<>(allColumns.size()); + for (ColumnParams column : allColumns) { + allColumnNames.add(column.name()); - for (String name : columns) { - if (!allColumns.contains(name)) { - throw new CatalogValidationException(format("PK column '{}' is not part of table", name)); + boolean partOfPk = columns.contains(column.name()); + if (partOfPk) { + if (column.nullable()) { + throw new CatalogValidationException(format("Primary key cannot contain nullable column [col={}].", column.name())); + } } + } + List<String> columnsNotInTable = columns.stream().filter(n -> !allColumnNames.contains(n)).collect(Collectors.toList()); + if (!columnsNotInTable.isEmpty()) { + throw new CatalogValidationException( + format("Primary key constraint contains undefined columns: [cols={}].", columnsNotInTable)); + } + + Set<String> columnSet = new HashSet<>(); + for (String name : columns) { if (!columnSet.add(name)) { - throw new CatalogValidationException(format("PK column '{}' specified more that once", name)); + throw new CatalogValidationException(format("PK column '{}' specified more that once.", name)); } } } diff --git a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/TableSortedPrimaryKey.java b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/TableSortedPrimaryKey.java index 201777d481..e3b8f44ec4 100644 --- a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/TableSortedPrimaryKey.java +++ b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/TableSortedPrimaryKey.java @@ -46,11 +46,11 @@ public class TableSortedPrimaryKey extends TablePrimaryKey { /** {@inheritDoc} */ @Override - void validate(Set<String> columns) { - super.validate(columns); + void validate(List<ColumnParams> allColumns) { + super.validate(allColumns); if (columns().size() != collations.size()) { - throw new CatalogValidationException("Number of collations does not match"); + throw new CatalogValidationException("Number of collations does not match."); } } diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateTableCommandValidationTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateTableCommandValidationTest.java index 8b4e87b196..8e4582a612 100644 --- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateTableCommandValidationTest.java +++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateTableCommandValidationTest.java @@ -84,6 +84,40 @@ public class CreateTableCommandValidationTest extends AbstractCommandValidationT ); } + @Test + void functionalDefaultsNotSupportsForNonPkColumns() { + CreateTableCommandBuilder builder = CreateTableCommand.builder(); + + builder = fillProperties(builder); + + builder.columns(List.of( + ColumnParams.builder().name("ID").type(INT32).defaultValue(DefaultValue.functionCall("function")).build(), + ColumnParams.builder().name("C").type(INT32).defaultValue(DefaultValue.constant(1)).build() + + )); + + assertThrowsWithCause( + builder::build, + CatalogValidationException.class, + "Functional defaults are not supported for non-primary key columns [col=ID]." + ); + } + + @Test + void functionalDefaultsSupportsForPkColumns() { + CreateTableCommandBuilder builder = CreateTableCommand.builder(); + + builder = fillProperties(builder); + + builder.columns(List.of( + ColumnParams.builder().name("C").type(INT32).defaultValue(DefaultValue.functionCall("function")).build(), + ColumnParams.builder().name("D").type(INT32).defaultValue(DefaultValue.constant(1)).build() + + )).primaryKey(primaryKey("C","D")); + + builder.build(); + } + @Test void columnShouldNotHaveDuplicates() { CreateTableCommandBuilder builder = CreateTableCommand.builder(); @@ -102,6 +136,54 @@ public class CreateTableCommandValidationTest extends AbstractCommandValidationT ); } + @Test + void primaryKeyColumnsShouldNotHaveDuplicates() { + CreateTableCommandBuilder builder = CreateTableCommand.builder(); + + builder = fillProperties(builder) + .primaryKey(primaryKey("C", "C")); + + assertThrowsWithCause( + builder::build, + CatalogValidationException.class, + "PK column 'C' specified more that once." + ); + } + + @Test + void primaryKeyColumnsShouldNotContainsNullable() { + CreateTableCommandBuilder builder = CreateTableCommand.builder(); + + builder = fillProperties(builder) + .columns(List.of( + ColumnParams.builder().name("C").type(INT32).nullable(true).build(), + ColumnParams.builder().name("D").type(INT32).build())) + .primaryKey(primaryKey("D","C")); + + assertThrowsWithCause( + builder::build, + CatalogValidationException.class, + "Primary key cannot contain nullable column [col=C]." + ); + } + + @Test + void primaryKeyColumnsCanContainsOnlyTableColumns() { + CreateTableCommandBuilder builder = CreateTableCommand.builder(); + + builder = fillProperties(builder) + .columns(List.of( + ColumnParams.builder().name("C").type(INT32).build(), + ColumnParams.builder().name("D").type(INT32).build())) + .primaryKey(primaryKey("Z","D","E")); + + assertThrowsWithCause( + builder::build, + CatalogValidationException.class, + "Primary key constraint contains undefined columns: [cols=[Z, E]]." + ); + } + @ParameterizedTest(name = "[{index}] {arguments}") @MethodSource("nullAndEmptyPrimaryKeys") void tableShouldHaveAtLeastOnePrimaryKeyColumn(TablePrimaryKey pk) { diff --git a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/TablePrimaryKeyTest.java b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/TablePrimaryKeyTest.java index 0ecd80e5ff..5b4be5c1d6 100644 --- a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/TablePrimaryKeyTest.java +++ b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/TablePrimaryKeyTest.java @@ -23,12 +23,13 @@ import static org.junit.jupiter.api.Named.named; import java.util.Arrays; import java.util.List; -import java.util.Set; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.ignite.internal.catalog.CatalogValidationException; import org.apache.ignite.internal.catalog.commands.TablePrimaryKey.TablePrimaryKeyBuilder; import org.apache.ignite.internal.catalog.descriptors.CatalogColumnCollation; import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest; +import org.apache.ignite.sql.ColumnType; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -44,9 +45,9 @@ public class TablePrimaryKeyTest extends BaseIgniteAbstractTest { TablePrimaryKey pk = builder.columns(List.of("C1", "C2", "C1")).build(); assertThrowsWithCause( - () -> pk.validate(Set.of("C1", "C2", "C3")), + () -> pk.validate(createSimpleColumnParams("C1", "C2", "C3")), CatalogValidationException.class, - "PK column 'C1' specified more that once" + "PK column 'C1' specified more that once." ); } @@ -56,9 +57,9 @@ public class TablePrimaryKeyTest extends BaseIgniteAbstractTest { TablePrimaryKey pk = builder.columns(List.of("C1", "foo")).build(); assertThrowsWithCause( - () -> pk.validate(Set.of("C1", "C2", "C3")), + () -> pk.validate(createSimpleColumnParams("C1", "C2", "C3")), CatalogValidationException.class, - "PK column 'foo' is not part of table" + "Primary key constraint contains undefined columns: [cols=[foo]]." ); } @@ -126,7 +127,7 @@ public class TablePrimaryKeyTest extends BaseIgniteAbstractTest { .collations(List.of(collation, otherCollation)) .build(); - pk.validate(Set.of("C1", "C2", "C3")); + pk.validate(createSimpleColumnParams("C1", "C2", "C3")); assertEquals(List.of("C1", "C2"), pk.columns()); assertEquals(List.of(collation, otherCollation), pk.collations(), "collations"); @@ -140,9 +141,14 @@ public class TablePrimaryKeyTest extends BaseIgniteAbstractTest { .build(); assertThrowsWithCause( - () -> pk.validate(Set.of("C1", "C2", "C3")), + () -> pk.validate(createSimpleColumnParams("C1", "C2", "C3")), CatalogValidationException.class, - "Number of collations does not match" + "Number of collations does not match." ); } + + private List<ColumnParams> createSimpleColumnParams(String... columns) { + return Arrays.stream(columns).map(col -> ColumnParams.builder().name(col).type(ColumnType.INT8).build()) + .collect(Collectors.toList()); + } } diff --git a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java index dbe5a400ff..dc35463961 100644 --- a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java +++ b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlApiBaseTest.java @@ -23,10 +23,10 @@ import static org.apache.ignite.internal.sql.engine.util.QueryChecker.containsTa import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.asStream; import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.assertThrowsSqlException; import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCode; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Arrays; @@ -39,7 +39,6 @@ import org.apache.ignite.internal.app.IgniteImpl; import org.apache.ignite.internal.catalog.commands.CatalogUtils; import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; import org.apache.ignite.internal.sql.api.ColumnMetadataImpl.ColumnOriginImpl; -import org.apache.ignite.internal.testframework.IgniteTestUtils; import org.apache.ignite.internal.tx.TxManager; import org.apache.ignite.lang.CursorClosedException; import org.apache.ignite.lang.ErrorGroups.Common; @@ -506,7 +505,7 @@ public abstract class ItSqlApiBaseTest extends BaseSqlIntegrationTest { Thread.sleep(300); // ResultSetImpl fetches next page in background, wait to it to complete to avoid flakiness. rs.close(); - IgniteTestUtils.assertThrowsWithCode( + assertThrowsWithCode( CursorClosedException.class, Common.CURSOR_ALREADY_CLOSED_ERR, () -> rs.forEachRemaining(Object::hashCode), @@ -646,16 +645,11 @@ public abstract class ItSqlApiBaseTest extends BaseSqlIntegrationTest { "Division by zero", () -> execute(tx, sql, dmlQuery, 1).affectedRows()); - IgniteException err = assertThrows(IgniteException.class, () -> { - ResultSet<SqlRow> rs = executeForRead(sql, tx, query, 2); - if (rs.hasRowSet()) { - assertTrue(rs.hasNext()); - } else { - assertTrue(rs.wasApplied()); - } - }); - - assertEquals(Transactions.TX_ALREADY_FINISHED_ERR, err.code(), err.toString()); + assertThrowsWithCode( + IgniteException.class, + Transactions.TX_ALREADY_FINISHED_ERR, + () -> executeForRead(sql, tx, query, 2), + "Transaction is already finished"); } @ParameterizedTest @@ -677,16 +671,11 @@ public abstract class ItSqlApiBaseTest extends BaseSqlIntegrationTest { "Division by zero", () -> execute(tx, sql, "SELECT val/? FROM tst WHERE id=?", 0, 1)); - IgniteException err = assertThrows(IgniteException.class, () -> { - ResultSet<SqlRow> rs = executeForRead(sql, tx, query, 2); - if (rs.hasRowSet()) { - assertTrue(rs.hasNext()); - } else { - assertTrue(rs.wasApplied()); - } - }); - - assertEquals(Transactions.TX_ALREADY_FINISHED_ERR, err.code(), err.toString()); + assertThrowsWithCode( + IgniteException.class, + Transactions.TX_ALREADY_FINISHED_ERR, + () -> executeForRead(sql, tx, query, 2), + "Transaction is already finished"); } @Test @@ -698,7 +687,10 @@ public abstract class ItSqlApiBaseTest extends BaseSqlIntegrationTest { { Transaction tx = igniteTx().begin(); - assertThrows(RuntimeException.class, () -> execute(tx, sql, "SELECT 1/0")); + assertThrowsSqlException( + Sql.RUNTIME_ERR, + "Division by zero", + () -> execute(tx, sql, "SELECT 1/0")); tx.rollback(); diff --git a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java index a9426e7ca1..7e49d7aea7 100644 --- a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java +++ b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java @@ -64,22 +64,31 @@ public class ItCreateTableDdlTest extends BaseSqlIntegrationTest { public void pkWithNullableColumns() { assertThrowsSqlException( STMT_VALIDATION_ERR, - "Primary key cannot contain nullable column [col=ID0]", + "Primary key cannot contain nullable column [col=ID0].", () -> sql("CREATE TABLE T0(ID0 INT NULL, ID1 INT NOT NULL, VAL INT, PRIMARY KEY (ID1, ID0))") ); assertThrowsSqlException( STMT_VALIDATION_ERR, - "Primary key cannot contain nullable column [col=ID]", + "Primary key cannot contain nullable column [col=ID].", () -> sql("CREATE TABLE T0(ID INT NULL PRIMARY KEY, VAL INT)") ); } + @Test + public void pkWithDuplicatesColumn() { + assertThrowsSqlException( + STMT_VALIDATION_ERR, + "PK column 'ID1' specified more that once.", + () -> sql("CREATE TABLE T0(ID0 INT, ID1 INT, VAL INT, PRIMARY KEY (ID1, ID0, ID1))") + ); + } + @Test public void pkWithInvalidColumns() { assertThrowsSqlException( STMT_VALIDATION_ERR, - "Primary key constraint contains undefined columns: [cols=[ID2]]", + "Primary key constraint contains undefined columns: [cols=[ID2]].", () -> sql("CREATE TABLE T0(ID0 INT, ID1 INT, VAL INT, PRIMARY KEY (ID2, ID0))") ); } @@ -128,7 +137,7 @@ public class ItCreateTableDdlTest extends BaseSqlIntegrationTest { public void undefinedColumnsInPrimaryKey() { assertThrowsSqlException( STMT_VALIDATION_ERR, - "Primary key constraint contains undefined columns: [cols=[ID0, ID2, ID1]]", + "Primary key constraint contains undefined columns: [cols=[ID1, ID0, ID2]].", () -> sql("CREATE TABLE T0(ID INT, VAL INT, PRIMARY KEY (ID1, ID0, ID2))") ); } diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java index b6b81ca6a1..9e4ddb5e41 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java @@ -46,7 +46,6 @@ import java.time.LocalTime; import java.time.Period; import java.util.AbstractMap.SimpleEntry; import java.util.ArrayList; -import java.util.Collections; import java.util.EnumMap; import java.util.EnumSet; import java.util.HashSet; @@ -322,31 +321,22 @@ public class DdlSqlToCommandConverter { + "querySql=\"" + ctx.query() + "\"]"); } - // TODO: https://issues.apache.org/jira/browse/IGNITE-21673 Remove dedupSetPk after this issue is fixed. - Set<String> dedupSetPk; List<String> pkColumns; - if (!pkConstraints.isEmpty()) { - dedupSetPk = new HashSet<>(); - IgniteSqlPrimaryKeyConstraint pkConstraint = pkConstraints.get(0); - SqlNodeList columnNodes = pkConstraint.getColumnList(); + IgniteSqlPrimaryKeyConstraint pkConstraint = pkConstraints.get(0); + SqlNodeList columnNodes = pkConstraint.getColumnList(); - pkColumns = new ArrayList<>(columnNodes.size()); - List<Collation> pkCollations = new ArrayList<>(columnNodes.size()); + pkColumns = new ArrayList<>(columnNodes.size()); + List<Collation> pkCollations = new ArrayList<>(columnNodes.size()); - PrimaryKeyIndexType pkIndexType = convertPrimaryIndexType(pkConstraint.getIndexType()); - boolean supportCollation = pkIndexType == PrimaryKeyIndexType.SORTED; + PrimaryKeyIndexType pkIndexType = convertPrimaryIndexType(pkConstraint.getIndexType()); + boolean supportCollation = pkIndexType == PrimaryKeyIndexType.SORTED; - parseColumnList(pkConstraint.getColumnList(), pkColumns, pkCollations, supportCollation, dedupSetPk); + parseColumnList(pkConstraint.getColumnList(), pkColumns, pkCollations, supportCollation); - createTblCmd.primaryIndexType(pkIndexType); - createTblCmd.primaryKeyColumns(pkColumns); - createTblCmd.primaryKeyCollations(pkCollations); - - } else { - dedupSetPk = Collections.emptySet(); - pkColumns = Collections.emptyList(); - } + createTblCmd.primaryIndexType(pkIndexType); + createTblCmd.primaryKeyColumns(pkColumns); + createTblCmd.primaryKeyCollations(pkCollations); List<String> colocationCols = createTblNode.colocationColumns() == null ? null @@ -375,27 +365,12 @@ public class DdlSqlToCommandConverter { String name = col.name.getSimple(); - if (col.dataType.getNullable() != null && col.dataType.getNullable() && dedupSetPk.contains(name)) { - throw new SqlException(STMT_VALIDATION_ERR, "Primary key cannot contain nullable column [col=" + name + "]"); - } - - RelDataType relType = planner.convert(col.dataType, !dedupSetPk.contains(name)); - - dedupSetPk.remove(name); + RelDataType relType = planner.convert(col.dataType, !pkColumns.contains(name)); DefaultValueDefinition dflt = convertDefault(col.expression, relType, name); - if (dflt.type() == DefaultValueDefinition.Type.FUNCTION_CALL && !pkColumns.contains(name)) { - throw new SqlException(STMT_VALIDATION_ERR, - "Functional defaults are not supported for non-primary key columns [col=" + name + "]"); - } - cols.add(new ColumnDefinition(name, relType, dflt)); } - if (!dedupSetPk.isEmpty()) { - throw new SqlException(STMT_VALIDATION_ERR, "Primary key constraint contains undefined columns: [cols=" + dedupSetPk + "]"); - } - createTblCmd.columns(cols); return createTblCmd; @@ -544,7 +519,7 @@ public class DdlSqlToCommandConverter { List<Collation> collations = new ArrayList<>(columnList.size()); boolean supportCollation = createIdxCmd.type() == Type.SORTED; - parseColumnList(columnList, columns, collations, supportCollation, null); + parseColumnList(columnList, columns, collations, supportCollation); createIdxCmd.columns(columns); createIdxCmd.collations(collations); @@ -558,8 +533,7 @@ public class DdlSqlToCommandConverter { SqlNodeList columnList, List<String> columns, List<Collation> collations, - boolean supportCollation, - @Nullable Set<String> dedup + boolean supportCollation ) { for (SqlNode col : columnList.getList()) { boolean desc = false; @@ -572,10 +546,6 @@ public class DdlSqlToCommandConverter { String columnName = ((SqlIdentifier) col).getSimple(); columns.add(columnName); - // TODO: https://issues.apache.org/jira/browse/IGNITE-21673 Remove dedupSetPk after this issue is fixed. - if (dedup != null && !dedup.add(columnName)) { - continue; - } if (supportCollation) { collations.add(desc ? Collation.DESC_NULLS_FIRST : Collation.ASC_NULLS_LAST); }
