Skip to content

Commit

Permalink
lightning: disable foreign key checks (#40032)
Browse files Browse the repository at this point in the history
close #40027
  • Loading branch information
sleepymole authored Dec 26, 2022
1 parent c30a9ff commit 1ace970
Show file tree
Hide file tree
Showing 6 changed files with 51 additions and 133 deletions.
48 changes: 6 additions & 42 deletions br/pkg/lightning/restore/tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,8 +95,10 @@ func DBFromConfig(ctx context.Context, dsn config.DBStore) (*sql.DB, error) {
"tidb_opt_write_row_id": "1",
// always set auto-commit to ON
"autocommit": "1",
// alway set transaction mode to optimistic
// always set transaction mode to optimistic
"tidb_txn_mode": "optimistic",
// disable foreign key checks
"foreign_key_checks": "0",
}

if dsn.Vars != nil {
Expand Down Expand Up @@ -143,47 +145,6 @@ func (timgr *TiDBManager) Close() {
timgr.db.Close()
}

func InitSchema(ctx context.Context, g glue.Glue, database string, tablesSchema map[string]string) error {
logger := log.FromContext(ctx).With(zap.String("db", database))
sqlExecutor := g.GetSQLExecutor()

var createDatabase strings.Builder
createDatabase.WriteString("CREATE DATABASE IF NOT EXISTS ")
common.WriteMySQLIdentifier(&createDatabase, database)
err := sqlExecutor.ExecuteWithLog(ctx, createDatabase.String(), "create database", logger)
if err != nil {
return errors.Trace(err)
}

task := logger.Begin(zap.InfoLevel, "create tables")
var sqlCreateStmts []string
loopCreate:
for tbl, sqlCreateTable := range tablesSchema {
task.Debug("create table", zap.String("schema", sqlCreateTable))

sqlCreateStmts, err = createIfNotExistsStmt(g.GetParser(), sqlCreateTable, database, tbl)
if err != nil {
break
}

// TODO: maybe we should put these createStems into a transaction
for _, s := range sqlCreateStmts {
err = sqlExecutor.ExecuteWithLog(
ctx,
s,
"create table",
logger.With(zap.String("table", common.UniqueTable(database, tbl))),
)
if err != nil {
break loopCreate
}
}
}
task.End(zap.ErrorLevel, err)

return errors.Trace(err)
}

func createIfNotExistsStmt(p *parser.Parser, createTable, dbName, tblName string) ([]string, error) {
stmts, _, err := p.ParseSQL(createTable)
if err != nil {
Expand All @@ -199,6 +160,9 @@ func createIfNotExistsStmt(p *parser.Parser, createTable, dbName, tblName string
case *ast.CreateDatabaseStmt:
node.Name = model.NewCIStr(dbName)
node.IfNotExists = true
case *ast.DropDatabaseStmt:
node.Name = model.NewCIStr(dbName)
node.IfExists = true
case *ast.CreateTableStmt:
node.Table.Schema = model.NewCIStr(dbName)
node.Table.Name = model.NewCIStr(tblName)
Expand Down
91 changes: 0 additions & 91 deletions br/pkg/lightning/restore/tidb_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,97 +165,6 @@ func TestCreateTableIfNotExistsStmt(t *testing.T) {
`, "m"))
}

func TestInitSchema(t *testing.T) {
s := newTiDBSuite(t)
ctx := context.Background()

s.mockDB.
ExpectExec("CREATE DATABASE IF NOT EXISTS `db`").
WillReturnResult(sqlmock.NewResult(1, 1))
s.mockDB.
ExpectExec("\\QCREATE TABLE IF NOT EXISTS `db`.`t1` (`a` INT PRIMARY KEY,`b` VARCHAR(200));\\E").
WillReturnResult(sqlmock.NewResult(2, 1))
s.mockDB.
ExpectExec("\\QSET @@SESSION.`FOREIGN_KEY_CHECKS`=0;\\E").
WillReturnResult(sqlmock.NewResult(0, 0))
s.mockDB.
ExpectExec("\\QCREATE TABLE IF NOT EXISTS `db`.`t2` (`xx` TEXT) AUTO_INCREMENT = 11203;\\E").
WillReturnResult(sqlmock.NewResult(2, 1))
s.mockDB.
ExpectClose()

s.mockDB.MatchExpectationsInOrder(false) // maps are unordered.
err := InitSchema(ctx, s.tiGlue, "db", map[string]string{
"t1": "create table t1 (a int primary key, b varchar(200));",
"t2": "/*!40014 SET FOREIGN_KEY_CHECKS=0*/;CREATE TABLE `db`.`t2` (xx TEXT) AUTO_INCREMENT=11203;",
})
s.mockDB.MatchExpectationsInOrder(true)
require.NoError(t, err)
}

func TestInitSchemaSyntaxError(t *testing.T) {
s := newTiDBSuite(t)
ctx := context.Background()

s.mockDB.
ExpectExec("CREATE DATABASE IF NOT EXISTS `db`").
WillReturnResult(sqlmock.NewResult(1, 1))
s.mockDB.
ExpectClose()

err := InitSchema(ctx, s.tiGlue, "db", map[string]string{
"t1": "create table `t1` with invalid syntax;",
})
require.Error(t, err)
}

func TestInitSchemaErrorLost(t *testing.T) {
s := newTiDBSuite(t)
ctx := context.Background()

s.mockDB.
ExpectExec("CREATE DATABASE IF NOT EXISTS `db`").
WillReturnResult(sqlmock.NewResult(1, 1))

s.mockDB.
ExpectExec("CREATE TABLE IF NOT EXISTS.*").
WillReturnError(&mysql.MySQLError{
Number: tmysql.ErrTooBigFieldlength,
Message: "Column length too big",
})

s.mockDB.
ExpectClose()

err := InitSchema(ctx, s.tiGlue, "db", map[string]string{
"t1": "create table `t1` (a int);",
"t2": "create table t2 (a int primary key, b varchar(200));",
})
require.Regexp(t, ".*Column length too big.*", err.Error())
}

func TestInitSchemaUnsupportedSchemaError(t *testing.T) {
s := newTiDBSuite(t)
ctx := context.Background()

s.mockDB.
ExpectExec("CREATE DATABASE IF NOT EXISTS `db`").
WillReturnResult(sqlmock.NewResult(1, 1))
s.mockDB.
ExpectExec("CREATE TABLE IF NOT EXISTS `db`.`t1`.*").
WillReturnError(&mysql.MySQLError{
Number: tmysql.ErrTooBigFieldlength,
Message: "Column length too big",
})
s.mockDB.
ExpectClose()

err := InitSchema(ctx, s.tiGlue, "db", map[string]string{
"t1": "create table `t1` (a VARCHAR(999999999));",
})
require.Regexp(t, ".*Column length too big.*", err.Error())
}

func TestDropTable(t *testing.T) {
s := newTiDBSuite(t)
ctx := context.Background()
Expand Down
3 changes: 3 additions & 0 deletions br/tests/lightning_foreign_key/config.toml
Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
[tikv-importer]
# Set on-duplicate=error to force using insert statement to write data.
on-duplicate = "error"
8 changes: 8 additions & 0 deletions br/tests/lightning_foreign_key/data/fk.t-schema.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,8 @@
CREATE TABLE `t`
(
`a` bigint(20) NOT NULL,
`b` bigint(20) DEFAULT NULL,
PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */,
KEY `fk_1` (`b`),
CONSTRAINT `fk_1` FOREIGN KEY (`b`) REFERENCES `test`.`t2` (`a`)
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;
6 changes: 6 additions & 0 deletions br/tests/lightning_foreign_key/data/fk.t.csv
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
a,b
1,1
2,2
3,3
4,4
5,5
28 changes: 28 additions & 0 deletions br/tests/lightning_foreign_key/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
#!/bin/bash
#
# Copyright 2022 PingCAP, Inc.
#
# 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.

set -eu

# Create existing tables that import data will reference.
run_sql 'CREATE DATABASE IF NOT EXISTS fk;'
run_sql 'CREATE TABLE fk.t2 (a BIGINT PRIMARY KEY);'

for BACKEND in tidb local; do
run_sql 'DROP TABLE IF EXISTS fk.t;'
run_lightning --backend $BACKEND
run_sql 'SELECT GROUP_CONCAT(a) FROM fk.t ORDER BY a;'
check_contains '1,2,3,4,5'
done

0 comments on commit 1ace970

Please sign in to comment.