Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
58 changes: 54 additions & 4 deletions fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,13 @@
import org.apache.doris.catalog.constraint.Constraint;
import org.apache.doris.catalog.constraint.ForeignKeyConstraint;
import org.apache.doris.catalog.constraint.PrimaryKeyConstraint;
import org.apache.doris.catalog.constraint.TableIdentifier;
import org.apache.doris.catalog.constraint.UniqueConstraint;
import org.apache.doris.cluster.ClusterNamespace;
import org.apache.doris.common.DdlException;
import org.apache.doris.common.MetaNotFoundException;
import org.apache.doris.common.Pair;
import org.apache.doris.common.util.MetaLockUtils;
import org.apache.doris.datasource.systable.SysTable;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction;
Expand All @@ -47,6 +49,7 @@
import java.io.DataOutput;
import java.io.IOException;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
Expand Down Expand Up @@ -334,6 +337,56 @@ default void replayDropConstraint(String name) {
dropConstraint(name, true);
}

// when table has foreign key constraint referencing to primary key of other table,
// need to remove this table identifier from primary table's foreign table set when drop this
// when table has primary key constraint, when drop table(this), need to remove the foreign key referenced this
default void removeTableIdentifierFromPrimaryTable() {
Map<String, Constraint> constraintMap = getConstraintsMapUnsafe();
for (Constraint constraint : constraintMap.values()) {
dropConstraintRefWithLock(constraint);
}
}

default void dropConstraintRefWithLock(Constraint constraint) {
List<TableIf> tables = getConstraintRelatedTables(constraint);
tables.sort((Comparator.comparing(TableIf::getId)));
MetaLockUtils.writeLockTables(tables);
try {
dropConstraintRef(constraint);
} finally {
MetaLockUtils.writeUnlockTables(tables);
}
}

default void dropConstraintRef(Constraint constraint) {
if (constraint instanceof PrimaryKeyConstraint) {
((PrimaryKeyConstraint) constraint).getForeignTables()
.forEach(t -> t.dropFKReferringPK(this, (PrimaryKeyConstraint) constraint));
} else if (constraint instanceof ForeignKeyConstraint) {
ForeignKeyConstraint foreignKeyConstraint = (ForeignKeyConstraint) constraint;
Optional<TableIf> primaryTableIf = foreignKeyConstraint.getReferencedTableOrNull();
if (primaryTableIf.isPresent()) {
Map<String, Constraint> refTableConstraintMap = primaryTableIf.get().getConstraintsMapUnsafe();
for (Constraint refTableConstraint : refTableConstraintMap.values()) {
if (refTableConstraint instanceof PrimaryKeyConstraint) {
PrimaryKeyConstraint primaryKeyConstraint = (PrimaryKeyConstraint) refTableConstraint;
primaryKeyConstraint.removeForeignTable(new TableIdentifier(this));
}
}
}
}
}

default List<TableIf> getConstraintRelatedTables(Constraint constraint) {
List<TableIf> tables = Lists.newArrayList();
if (constraint instanceof PrimaryKeyConstraint) {
tables.addAll(((PrimaryKeyConstraint) constraint).getForeignTables());
} else if (constraint instanceof ForeignKeyConstraint) {
tables.add(((ForeignKeyConstraint) constraint).getReferencedTable());
}
return tables;
}

default void dropConstraint(String name, boolean replay) {
Map<String, Constraint> constraintMap = getConstraintsMapUnsafe();
if (!constraintMap.containsKey(name)) {
Expand All @@ -342,10 +395,7 @@ default void dropConstraint(String name, boolean replay) {
}
Constraint constraint = constraintMap.get(name);
constraintMap.remove(name);
if (constraint instanceof PrimaryKeyConstraint) {
((PrimaryKeyConstraint) constraint).getForeignTables()
.forEach(t -> t.dropFKReferringPK(this, (PrimaryKeyConstraint) constraint));
}
dropConstraintRefWithLock(constraint);
if (!replay) {
Env.getCurrentEnv().getEditLog().logDropConstraint(new AlterConstraintLog(constraint, this));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;

public class ForeignKeyConstraint extends Constraint {
Expand Down Expand Up @@ -90,6 +91,16 @@ public TableIf getReferencedTable() {
return referencedTable.toTableIf();
}

public Optional<TableIf> getReferencedTableOrNull() {
TableIf res = null;
try {
res = referencedTable.toTableIf();
} catch (Exception ignored) {
// do nothing
}
return Optional.ofNullable(res);
}

public Boolean isReferringPK(TableIf table, PrimaryKeyConstraint constraint) {
return constraint.getPrimaryKeyNames().equals(getPrimaryKeyNames())
&& getReferencedTable().equals(table);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,10 @@ public List<TableIf> getForeignTables() {
.collect(ImmutableList.toImmutableList());
}

public void removeForeignTable(TableIdentifier tableIdentifier) {
foreignTables.remove(tableIdentifier);
}

@Override
public String toString() {
return "PRIMARY KEY (" + String.join(", ", columns) + ")";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public TableIf toTableIf() {
}
TableIf tableIf = databaseIf.getTableNullable(tableId);
if (tableIf == null) {
throw new MetaNotFoundException(String.format("Can not find table %s in constraint", databaseId));
throw new MetaNotFoundException(String.format("Can not find table %s in constraint", tableId));
}
return tableIf;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1061,6 +1061,7 @@ public boolean unprotectDropTable(Database db, Table table, boolean isForceDrop,
}

Env.getCurrentEnv().getAnalysisManager().removeTableStats(table.getId());
table.removeTableIdentifierFromPrimaryTable();
db.unregisterTable(table.getId());
StopWatch watch = StopWatch.createStarted();
Env.getCurrentRecycleBin().recycleTable(db.getId(), table, isReplay, isForceDrop, recycleTime);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@

import org.apache.doris.catalog.TableIf;
import org.apache.doris.catalog.constraint.Constraint;
import org.apache.doris.catalog.constraint.ForeignKeyConstraint;
import org.apache.doris.catalog.constraint.PrimaryKeyConstraint;
import org.apache.doris.common.util.MetaLockUtils;
import org.apache.doris.nereids.NereidsPlanner;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.properties.PhysicalProperties;
Expand All @@ -37,7 +37,6 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;

import java.util.Comparator;
import java.util.List;
import java.util.Set;

Expand All @@ -59,29 +58,34 @@ public DropConstraintCommand(String name, LogicalPlan plan) {
this.plan = plan;
}

private static List<TableIf> getConstraintRelatedTables(Constraint constraint) {
List<TableIf> tables = Lists.newArrayList();
if (constraint instanceof PrimaryKeyConstraint) {
tables.addAll(((PrimaryKeyConstraint) constraint).getForeignTables());
} else if (constraint instanceof ForeignKeyConstraint) {
tables.add(((ForeignKeyConstraint) constraint).getReferencedTable());
}
return tables;
}

@Override
public void run(ConnectContext ctx, StmtExecutor executor) throws Exception {
TableIf table = extractTable(ctx, plan);
List<TableIf> tables = Lists.newArrayList(table);
table.readLock();
try {
Constraint constraint = table.getConstraintsMapUnsafe().get(name);
if (constraint == null) {
throw new AnalysisException(
String.format("Unknown constraint %s on table %s.", name, table.getName()));
}
if (constraint instanceof PrimaryKeyConstraint) {
tables.addAll(((PrimaryKeyConstraint) constraint).getForeignTables());
}
} finally {
table.readUnlock();
}
tables.sort((Comparator.comparing(TableIf::getId)));
MetaLockUtils.writeLockTables(tables);
table.writeLock();
try {
table.dropConstraint(name, false);
} finally {
MetaLockUtils.writeUnlockTables(tables);
table.writeUnlock();
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you 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.

suite("test_pk_fk_drop_table") {
// case 1
multi_sql """drop table if exists store_sales_test;
drop table if exists customer_test;
create table customer_test(c_customer_sk int,c_customer_id int);
CREATE TABLE `store_sales_test` (
`ss_customer_sk` int NULL,
`d_date` date NULL
) ENGINE=OLAP
DUPLICATE KEY(`ss_customer_sk`, `d_date`)
DISTRIBUTED BY RANDOM BUCKETS AUTO
PROPERTIES (
"replication_allocation" = "tag.location.default: 1"
);
alter table customer_test add constraint c_pk primary key(c_customer_sk);
alter table store_sales_test add constraint ss_c_fk foreign key(ss_customer_sk) references customer_test(c_customer_sk);
drop table store_sales_test;"""
// expect: successful drop
sql "alter table customer_test drop constraint c_pk;"

multi_sql """
drop table if exists store_sales_test;
drop table if exists customer_test;
create table customer_test(c_customer_sk int,c_customer_id int) PROPERTIES (
"replication_allocation" = "tag.location.default: 1"
);
CREATE TABLE `store_sales_test` (
`ss_customer_sk` int NULL,
`d_date` date NULL
) ENGINE=OLAP
DUPLICATE KEY(`ss_customer_sk`, `d_date`)
DISTRIBUTED BY RANDOM BUCKETS AUTO
PROPERTIES (
"replication_allocation" = "tag.location.default: 1"
);
alter table customer_test add constraint c_pk primary key(c_customer_sk);
alter table store_sales_test add constraint ss_c_fk foreign key(ss_customer_sk) references customer_test(c_customer_sk);
drop table customer_test;
"""
// expect: not throw
sql "show constraints from store_sales_test;"
}
Loading