dumpling: fix default collation with upstream when dump database and table (#30292)

This commit is contained in:
WizardXiao
2021-12-04 23:19:54 +08:00
committed by GitHub
parent 28446605c2
commit e3e2e8d946
9 changed files with 239 additions and 25 deletions

View File

@ -8,7 +8,6 @@ import (
"database/sql"
"encoding/hex"
"fmt"
"github.com/go-sql-driver/mysql"
"math/big"
"sort"
"strconv"
@ -17,7 +16,7 @@ import (
"time"
// import mysql driver
_ "github.com/go-sql-driver/mysql"
"github.com/go-sql-driver/mysql"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
pclog "github.com/pingcap/log"
@ -31,6 +30,9 @@ import (
"github.com/pingcap/tidb/dumpling/cli"
tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/pingcap/tidb/dumpling/log"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/format"
"github.com/pingcap/tidb/store/helper"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/codec"
@ -49,9 +51,10 @@ type Dumper struct {
extStore storage.ExternalStorage
dbHandle *sql.DB
tidbPDClientForGC pd.Client
selectTiDBTableRegionFunc func(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta) (pkFields []string, pkVals [][]string, err error)
totalTables int64
tidbPDClientForGC pd.Client
selectTiDBTableRegionFunc func(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta) (pkFields []string, pkVals [][]string, err error)
totalTables int64
charsetAndDefaultCollationMap map[string]string
}
// NewDumper returns a new Dumper
@ -151,6 +154,12 @@ func (d *Dumper) Dump() (dumpErr error) {
tctx.L().Info("get global metadata failed", zap.Error(err))
}
//init charset and default collation map
d.charsetAndDefaultCollationMap, err = GetCharsetAndDefaultCollation(tctx.Context, metaConn)
if err != nil {
return err
}
// for other consistencies, we should get table list after consistency is set up and GlobalMetaData is cached
if conf.Consistency != consistencyTypeLock {
if err = prepareTableListToDump(tctx, conf, metaConn); err != nil {
@ -320,7 +329,7 @@ func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *sql.Conn, taskC
for _, policy := range policyNames {
createPolicySQL, err := ShowCreatePlacementPolicy(metaConn, policy)
if err != nil {
return err
return errors.Trace(err)
}
wrappedCreatePolicySQL := fmt.Sprintf("/*T![placement] %s */", createPolicySQL)
task := NewTaskPolicyMeta(policy, wrappedCreatePolicySQL)
@ -331,11 +340,17 @@ func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *sql.Conn, taskC
}
}
parser1 := parser.New()
for dbName, tables := range allTables {
if !conf.NoSchemas {
createDatabaseSQL, err := ShowCreateDatabase(metaConn, dbName)
if err != nil {
return err
return errors.Trace(err)
}
// adjust db collation
createDatabaseSQL, err = adjustDatabaseCollation(tctx, parser1, createDatabaseSQL, d.charsetAndDefaultCollationMap)
if err != nil {
return errors.Trace(err)
}
task := NewTaskDatabaseMeta(dbName, createDatabaseSQL)
ctxDone := d.sendTaskToChan(tctx, task, taskChan)
@ -349,7 +364,7 @@ func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *sql.Conn, taskC
zap.String("table", table.Name))
meta, err := dumpTableMeta(conf, metaConn, dbName, table)
if err != nil {
return err
return errors.Trace(err)
}
if !conf.NoSchemas {
@ -360,6 +375,12 @@ func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *sql.Conn, taskC
return tctx.Err()
}
} else {
// adjust table collation
newCreateSQL, err := adjustTableCollation(tctx, parser1, meta.ShowCreateTable(), d.charsetAndDefaultCollationMap)
if err != nil {
return errors.Trace(err)
}
meta.(*tableMeta).showCreateTable = newCreateSQL
task := NewTaskTableMeta(dbName, table.Name, meta.ShowCreateTable())
ctxDone := d.sendTaskToChan(tctx, task, taskChan)
if ctxDone {
@ -370,7 +391,7 @@ func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *sql.Conn, taskC
if table.Type == TableTypeBase {
err = d.dumpTableData(tctx, metaConn, meta, taskChan)
if err != nil {
return err
return errors.Trace(err)
}
}
}
@ -379,6 +400,92 @@ func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *sql.Conn, taskC
return nil
}
// adjustDatabaseCollation adjusts db collation and return new create sql and collation
func adjustDatabaseCollation(tctx *tcontext.Context, parser *parser.Parser, originSQL string, charsetAndDefaultCollationMap map[string]string) (string, error) {
stmt, err := parser.ParseOneStmt(originSQL, "", "")
if err != nil {
tctx.L().Warn("parse create database error, maybe tidb parser doesn't support it", zap.String("originSQL", originSQL), log.ShortError(err))
return originSQL, nil
}
createStmt, ok := stmt.(*ast.CreateDatabaseStmt)
if !ok {
return originSQL, nil
}
var charset string
for _, createOption := range createStmt.Options {
// already have 'Collation'
if createOption.Tp == ast.DatabaseOptionCollate {
return originSQL, nil
}
if createOption.Tp == ast.DatabaseOptionCharset {
charset = createOption.Value
}
}
// get db collation
collation, ok := charsetAndDefaultCollationMap[strings.ToLower(charset)]
if !ok {
tctx.L().Warn("not found database charset default collation.", zap.String("originSQL", originSQL), zap.String("charset", strings.ToLower(charset)))
return originSQL, nil
}
// add collation
createStmt.Options = append(createStmt.Options, &ast.DatabaseOption{Tp: ast.DatabaseOptionCollate, Value: collation})
// rewrite sql
var b []byte
bf := bytes.NewBuffer(b)
err = createStmt.Restore(&format.RestoreCtx{
Flags: format.DefaultRestoreFlags | format.RestoreTiDBSpecialComment,
In: bf,
})
if err != nil {
return "", errors.Trace(err)
}
return bf.String(), nil
}
// adjustTableCollation adjusts table collation
func adjustTableCollation(tctx *tcontext.Context, parser *parser.Parser, originSQL string, charsetAndDefaultCollationMap map[string]string) (string, error) {
stmt, err := parser.ParseOneStmt(originSQL, "", "")
if err != nil {
tctx.L().Warn("parse create table error, maybe tidb parser doesn't support it", zap.String("originSQL", originSQL), log.ShortError(err))
return originSQL, nil
}
createStmt, ok := stmt.(*ast.CreateTableStmt)
if !ok {
return originSQL, nil
}
var charset string
for _, createOption := range createStmt.Options {
// already have 'Collation'
if createOption.Tp == ast.TableOptionCollate {
return originSQL, nil
}
if createOption.Tp == ast.TableOptionCharset {
charset = createOption.StrValue
}
}
// get db collation
collation, ok := charsetAndDefaultCollationMap[strings.ToLower(charset)]
if !ok {
tctx.L().Warn("not found table charset default collation.", zap.String("originSQL", originSQL), zap.String("charset", strings.ToLower(charset)))
return originSQL, nil
}
// add collation
createStmt.Options = append(createStmt.Options, &ast.TableOption{Tp: ast.TableOptionCollate, StrValue: collation})
// rewrite sql
var b []byte
bf := bytes.NewBuffer(b)
err = createStmt.Restore(&format.RestoreCtx{
Flags: format.DefaultRestoreFlags | format.RestoreTiDBSpecialComment,
In: bf,
})
if err != nil {
return "", errors.Trace(err)
}
return bf.String(), nil
}
func (d *Dumper) dumpTableData(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta, taskChan chan<- Task) error {
conf := d.conf
if conf.NoData {

View File

@ -15,6 +15,7 @@ import (
"github.com/pingcap/tidb/br/pkg/version"
tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/pingcap/tidb/parser"
)
func TestDumpBlock(t *testing.T) {
@ -29,6 +30,9 @@ func TestDumpBlock(t *testing.T) {
mock.ExpectQuery(fmt.Sprintf("SHOW CREATE DATABASE `%s`", escapeString(database))).
WillReturnRows(sqlmock.NewRows([]string{"Database", "Create Database"}).
AddRow("test", "CREATE DATABASE `test` /*!40100 DEFAULT CHARACTER SET utf8mb4 */"))
mock.ExpectQuery(fmt.Sprintf("SELECT DEFAULT_COLLATION_NAME FROM INFORMATION_SCHEMA.SCHEMATA WHERE SCHEMA_NAME = '%s'", escapeString(database))).
WillReturnRows(sqlmock.NewRows([]string{"DEFAULT_COLLATION_NAME"}).
AddRow("utf8mb4_bin"))
tctx, cancel := tcontext.Background().WithLogger(appLogger).WithCancel()
defer cancel()
@ -128,3 +132,53 @@ func TestGetListTableTypeByConf(t *testing.T) {
require.Equalf(t, x.expected, getListTableTypeByConf(conf), "server info: %s, consistency: %s", x.serverInfo, x.consistency)
}
}
func TestAdjustDatabaseCollation(t *testing.T) {
t.Parallel()
tctx, cancel := tcontext.Background().WithLogger(appLogger).WithCancel()
defer cancel()
parser1 := parser.New()
originSQLs := []string{
"create database `test` CHARACTER SET=utf8mb4 COLLATE=utf8mb4_general_ci",
"create database `test` CHARACTER SET=utf8mb4",
}
expectedSQLs := []string{
"create database `test` CHARACTER SET=utf8mb4 COLLATE=utf8mb4_general_ci",
"CREATE DATABASE `test` CHARACTER SET = utf8mb4 COLLATE = utf8mb4_general_ci",
}
charsetAndDefaultCollationMap := map[string]string{"utf8mb4": "utf8mb4_general_ci"}
for i, originSQL := range originSQLs {
newSQL, err := adjustDatabaseCollation(tctx, parser1, originSQL, charsetAndDefaultCollationMap)
require.NoError(t, err)
require.Equal(t, expectedSQLs[i], newSQL)
}
}
func TestAdjustTableCollation(t *testing.T) {
t.Parallel()
tctx, cancel := tcontext.Background().WithLogger(appLogger).WithCancel()
defer cancel()
parser1 := parser.New()
originSQLs := []string{
"create table `test`.`t1` (id int) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci",
"create table `test`.`t1` (id int) CHARSET=utf8mb4",
}
expectedSQLs := []string{
"create table `test`.`t1` (id int) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci",
"CREATE TABLE `test`.`t1` (`id` INT) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI",
}
charsetAndDefaultCollationMap := map[string]string{"utf8mb4": "utf8mb4_general_ci"}
for i, originSQL := range originSQLs {
newSQL, err := adjustTableCollation(tctx, parser1, originSQL, charsetAndDefaultCollationMap)
require.NoError(t, err)
require.Equal(t, expectedSQLs[i], newSQL)
}
}

View File

@ -1387,3 +1387,43 @@ func GetRegionInfos(db *sql.Conn) (*helper.RegionsInfo, error) {
})
return regionsInfo, err
}
// GetCharsetAndDefaultCollation gets charset and default collation map.
func GetCharsetAndDefaultCollation(ctx context.Context, db *sql.Conn) (map[string]string, error) {
charsetAndDefaultCollation := make(map[string]string)
query := "SHOW CHARACTER SET"
// Show an example.
/*
mysql> SHOW CHARACTER SET;
+----------+---------------------------------+---------------------+--------+
| Charset | Description | Default collation | Maxlen |
+----------+---------------------------------+---------------------+--------+
| armscii8 | ARMSCII-8 Armenian | armscii8_general_ci | 1 |
| ascii | US ASCII | ascii_general_ci | 1 |
| big5 | Big5 Traditional Chinese | big5_chinese_ci | 2 |
| binary | Binary pseudo charset | binary | 1 |
| cp1250 | Windows Central European | cp1250_general_ci | 1 |
| cp1251 | Windows Cyrillic | cp1251_general_ci | 1 |
+----------+---------------------------------+---------------------+--------+
*/
rows, err := db.QueryContext(ctx, query)
if err != nil {
return nil, errors.Annotatef(err, "sql: %s", query)
}
defer rows.Close()
for rows.Next() {
var charset, description, collation string
var maxlen int
if scanErr := rows.Scan(&charset, &description, &collation, &maxlen); scanErr != nil {
return nil, errors.Annotatef(err, "sql: %s", query)
}
charsetAndDefaultCollation[strings.ToLower(charset)] = collation
}
if err = rows.Close(); err != nil {
return nil, errors.Annotatef(err, "sql: %s", query)
}
return charsetAndDefaultCollation, err
}

View File

@ -9,7 +9,6 @@ import (
"encoding/csv"
"encoding/json"
"fmt"
"github.com/go-sql-driver/mysql"
"io"
"os"
"path"
@ -18,6 +17,8 @@ import (
"strings"
"testing"
"github.com/go-sql-driver/mysql"
"github.com/DATA-DOG/go-sqlmock"
"github.com/coreos/go-semver/semver"
"github.com/pingcap/errors"
@ -1746,6 +1747,28 @@ func TestPickupPossibleField(t *testing.T) {
}
}
func TestGetCharsetAndDefaultCollation(t *testing.T) {
t.Parallel()
db, mock, err := sqlmock.New()
require.NoError(t, err)
defer func() {
require.NoError(t, db.Close())
}()
ctx := context.Background()
conn, err := db.Conn(ctx)
require.NoError(t, err)
mock.ExpectQuery("SHOW CHARACTER SET").
WillReturnRows(sqlmock.NewRows([]string{"Charset", "Description", "Default collation", "Maxlen"}).
AddRow("utf8mb4", "UTF-8 Unicode", "utf8mb4_0900_ai_ci", 4).
AddRow("latin1", "cp1252 West European", "latin1_swedish_ci", 1))
charsetAndDefaultCollation, err := GetCharsetAndDefaultCollation(ctx, conn)
require.NoError(t, err)
require.Equal(t, "utf8mb4_0900_ai_ci", charsetAndDefaultCollation["utf8mb4"])
require.Equal(t, "latin1_swedish_ci", charsetAndDefaultCollation["latin1"])
}
func makeVersion(major, minor, patch int64, preRelease string) *semver.Version {
return &semver.Version{
Major: major,

View File

@ -25,6 +25,6 @@ file_should_exist "$DUMPLING_OUTPUT_DIR/x1-placement-policy-create.sql"
diff "$DUMPLING_BASE_NAME/result/x-placement-policy-create.sql" "$DUMPLING_OUTPUT_DIR/x-placement-policy-create.sql"
diff "$DUMPLING_BASE_NAME/result/x1-placement-policy-create.sql" "$DUMPLING_OUTPUT_DIR/x1-placement-policy-create.sql"
run_sql "drop database if exists policy"
run_sql "drop placement policy if exists x"
run_sql "drop placement policy if exists x1"
run_sql "drop database if exists policy"

View File

@ -1,3 +1,3 @@
/*!40101 SET NAMES binary*/;
/*T![placement] SET PLACEMENT_CHECKS = 0*/;
CREATE DATABASE `quo``te/database` /*!40100 DEFAULT CHARACTER SET latin1 */;
CREATE DATABASE `quo``te/database` CHARACTER SET = latin1 COLLATE = latin1_swedish_ci;

View File

@ -1,3 +1,3 @@
/*!40101 SET NAMES binary*/;
/*T![placement] SET PLACEMENT_CHECKS = 0*/;
CREATE DATABASE `quo``te/database` /*!40100 DEFAULT CHARACTER SET latin1 */ /*!80016 DEFAULT ENCRYPTION='N' */;
CREATE DATABASE `quo``te/database` CHARACTER SET = latin1 ENCRYPTION = 'N' COLLATE = latin1_swedish_ci;

View File

@ -1,8 +1,3 @@
/*!40101 SET NAMES binary*/;
/*T![placement] SET PLACEMENT_CHECKS = 0*/;
CREATE TABLE `quo``te/table` (
`quo``te/col` int(11) NOT NULL,
`a` int(11) DEFAULT NULL,
`gen``id` int(11) GENERATED ALWAYS AS (`quo``te/col`) VIRTUAL,
PRIMARY KEY (`quo``te/col`)
) ENGINE=InnoDB DEFAULT CHARSET=latin1;
CREATE TABLE `quo``te/table` (`quo``te/col` INT(11) NOT NULL,`a` INT(11) DEFAULT NULL,`gen``id` INT(11) GENERATED ALWAYS AS(`quo``te/col`) VIRTUAL,PRIMARY KEY(`quo``te/col`)) ENGINE = InnoDB DEFAULT CHARACTER SET = LATIN1 DEFAULT COLLATE = LATIN1_SWEDISH_CI;

View File

@ -1,8 +1,3 @@
/*!40101 SET NAMES binary*/;
/*T![placement] SET PLACEMENT_CHECKS = 0*/;
CREATE TABLE `quo``te/table` (
`quo``te/col` int NOT NULL,
`a` int DEFAULT NULL,
`gen``id` int GENERATED ALWAYS AS (`quo``te/col`) VIRTUAL,
PRIMARY KEY (`quo``te/col`)
) ENGINE=InnoDB DEFAULT CHARSET=latin1;
CREATE TABLE `quo``te/table` (`quo``te/col` INT NOT NULL,`a` INT DEFAULT NULL,`gen``id` INT GENERATED ALWAYS AS(`quo``te/col`) VIRTUAL,PRIMARY KEY(`quo``te/col`)) ENGINE = InnoDB DEFAULT CHARACTER SET = LATIN1 DEFAULT COLLATE = LATIN1_SWEDISH_CI;