Skip to content

Commit

Permalink
Merge branch 'master' into add-fk-to-info-schema
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo authored Mar 24, 2022
2 parents 2f0e90c + 5c35121 commit 88708af
Show file tree
Hide file tree
Showing 25 changed files with 8,058 additions and 7,835 deletions.
4 changes: 3 additions & 1 deletion br/pkg/task/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,7 @@ func (cfg *RestoreConfig) ParseFromFlags(flags *pflag.FlagSet) error {
}

// adjustRestoreConfig is use for BR(binary) and BR in TiDB.
// When new config was add and not included in parser.
// When new config was added and not included in parser.
// we should set proper value in this function.
// so that both binary and TiDB will use same default value.
func (cfg *RestoreConfig) adjustRestoreConfig() {
Expand Down Expand Up @@ -532,6 +532,8 @@ func dropToBlackhole(
return outCh
}

// filterRestoreFiles filters tables that can't be processed after applying cfg.TableFilter.MatchTable.
// if the db has no table that can be processed, the db will be filtered too.
func filterRestoreFiles(
client *restore.Client,
cfg *RestoreConfig,
Expand Down
4 changes: 2 additions & 2 deletions docs/tidb_http_api.md
Original file line number Diff line number Diff line change
Expand Up @@ -303,10 +303,10 @@ timezone.*
}
```

*Hint: On a partitioned table, use the `table(partition)` pattern as the table name, `test(p1)` for example:*
*Hint: On a partitioned table, use the `table(partition)` pattern as the table name, `t1(p1)` for example:*

```shell
$curl http://127.0.0.1:10080/mvcc/index/test(p1)/t1/idx/1\?a\=A
$curl http://127.0.0.1:10080/mvcc/index/test/t1(p1)/idx/1\?a\=A
```

If the handle is clustered, also specify the primary key column values in the query string
Expand Down
8 changes: 8 additions & 0 deletions dumpling/export/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ const (
flagConsistency = "consistency"
flagSnapshot = "snapshot"
flagNoViews = "no-views"
flagNoSequences = "no-sequences"
flagSortByPk = "order-by-primary-key"
flagStatusAddr = "status-addr"
flagRows = "rows"
Expand Down Expand Up @@ -84,6 +85,7 @@ type Config struct {
AllowCleartextPasswords bool
SortByPk bool
NoViews bool
NoSequences bool
NoHeader bool
NoSchemas bool
NoData bool
Expand Down Expand Up @@ -166,6 +168,7 @@ func DefaultConfig() *Config {
Snapshot: "",
Consistency: consistencyTypeAuto,
NoViews: true,
NoSequences: true,
Rows: UnspecifiedSize,
Where: "",
FileType: "",
Expand Down Expand Up @@ -231,6 +234,7 @@ func (conf *Config) DefineFlags(flags *pflag.FlagSet) {
flags.String(flagConsistency, consistencyTypeAuto, "Consistency level during dumping: {auto|none|flush|lock|snapshot}")
flags.String(flagSnapshot, "", "Snapshot position (uint64 or MySQL style string timestamp). Valid only when consistency=snapshot")
flags.BoolP(flagNoViews, "W", true, "Do not dump views")
flags.Bool(flagNoSequences, true, "Do not dump sequences")
flags.Bool(flagSortByPk, true, "Sort dump results by primary key through order by sql")
flags.String(flagStatusAddr, ":8281", "dumpling API server and pprof addr")
flags.Uint64P(flagRows, "r", UnspecifiedSize, "If specified, dumpling will split table into chunks and concurrently dump them to different files to improve efficiency. For TiDB v3.0+, specify this will make dumpling split table with each file one TiDB region(no matter how many rows is).\n"+
Expand Down Expand Up @@ -328,6 +332,10 @@ func (conf *Config) ParseFromFlags(flags *pflag.FlagSet) error {
if err != nil {
return errors.Trace(err)
}
conf.NoSequences, err = flags.GetBool(flagNoSequences)
if err != nil {
return errors.Trace(err)
}
conf.SortByPk, err = flags.GetBool(flagSortByPk)
if err != nil {
return errors.Trace(err)
Expand Down
38 changes: 29 additions & 9 deletions dumpling/export/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -386,14 +386,20 @@ func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *BaseConn, taskC
}

if !conf.NoSchemas {
if table.Type == TableTypeView {
switch table.Type {
case TableTypeView:
task := NewTaskViewMeta(dbName, table.Name, meta.ShowCreateTable(), meta.ShowCreateView())
ctxDone := d.sendTaskToChan(tctx, task, taskChan)
if ctxDone {
return tctx.Err()
}
} else {

case TableTypeSequence:
task := NewTaskSequenceMeta(dbName, table.Name, meta.ShowCreateTable())
ctxDone := d.sendTaskToChan(tctx, task, taskChan)
if ctxDone {
return tctx.Err()
}
default:
// adjust table collation
newCreateSQL, err := adjustTableCollation(tctx, d.conf.CollationCompatible, parser1, meta.ShowCreateTable(), d.charsetAndDefaultCollationMap)
if err != nil {
Expand Down Expand Up @@ -1089,14 +1095,16 @@ func prepareTableListToDump(tctx *tcontext.Context, conf *Config, db *sql.Conn)
if !conf.NoViews {
tableTypes = append(tableTypes, TableTypeView)
}
if !conf.NoSequences {
tableTypes = append(tableTypes, TableTypeSequence)
}

ifSeqExists, err := CheckIfSeqExists(db)
if err != nil {
return err
}
var listType listTableType
if ifSeqExists {
tctx.L().Warn("dumpling tableType `sequence` is unsupported for now")
listType = listTableByShowFullTables
} else {
listType = getListTableTypeByConf(conf)
Expand Down Expand Up @@ -1129,10 +1137,12 @@ func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *BaseConn, db stri
}

// If all columns are generated
if selectField == "" {
colTypes, err = GetColumnTypes(tctx, conn, "*", db, tbl)
} else {
colTypes, err = GetColumnTypes(tctx, conn, selectField, db, tbl)
if table.Type == TableTypeBase {
if selectField == "" {
colTypes, err = GetColumnTypes(tctx, conn, "*", db, tbl)
} else {
colTypes, err = GetColumnTypes(tctx, conn, selectField, db, tbl)
}
}
if err != nil {
return nil, err
Expand All @@ -1154,7 +1164,8 @@ func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *BaseConn, db stri
if conf.NoSchemas {
return meta, nil
}
if table.Type == TableTypeView {
switch table.Type {
case TableTypeView:
viewName := table.Name
createTableSQL, createViewSQL, err1 := ShowCreateView(tctx, conn, db, viewName)
if err1 != nil {
Expand All @@ -1163,7 +1174,16 @@ func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *BaseConn, db stri
meta.showCreateTable = createTableSQL
meta.showCreateView = createViewSQL
return meta, nil
case TableTypeSequence:
sequenceName := table.Name
createSequenceSQL, err2 := ShowCreateSequence(tctx, conn, db, sequenceName, conf)
if err2 != nil {
return meta, err2
}
meta.showCreateTable = createSequenceSQL
return meta, nil
}

createTableSQL, err := ShowCreateTable(tctx, conn, db, tbl)
if err != nil {
return nil, err
Expand Down
11 changes: 6 additions & 5 deletions dumpling/export/prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,12 @@ import (
)

const (
outputFileTemplateSchema = "schema"
outputFileTemplateTable = "table"
outputFileTemplateView = "view"
outputFileTemplateData = "data"
outputFileTemplatePolicy = "placement-policy"
outputFileTemplateSchema = "schema"
outputFileTemplateTable = "table"
outputFileTemplateView = "view"
outputFileTemplateSequence = "sequence"
outputFileTemplateData = "data"
outputFileTemplatePolicy = "placement-policy"

defaultOutputFileTemplateBase = `
{{- define "objectName" -}}
Expand Down
55 changes: 55 additions & 0 deletions dumpling/export/sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"go.uber.org/multierr"
"go.uber.org/zap"

"github.com/pingcap/tidb/br/pkg/version"
dbconfig "github.com/pingcap/tidb/config"
tcontext "github.com/pingcap/tidb/dumpling/context"
"github.com/pingcap/tidb/dumpling/log"
Expand Down Expand Up @@ -174,6 +175,60 @@ func ShowCreateView(tctx *tcontext.Context, db *BaseConn, database, view string)
return createTableSQL.String(), createViewSQL.String(), nil
}

// ShowCreateSequence constructs the create sequence SQL for a specified sequence
// returns (createSequenceSQL, error)
func ShowCreateSequence(tctx *tcontext.Context, db *BaseConn, database, sequence string, conf *Config) (string, error) {
var oneRow [2]string
handleOneRow := func(rows *sql.Rows) error {
return rows.Scan(&oneRow[0], &oneRow[1])
}
var (
createSequenceSQL strings.Builder
nextNotCachedValue int64
)
query := fmt.Sprintf("SHOW CREATE SEQUENCE `%s`.`%s`", escapeString(database), escapeString(sequence))
err := db.QuerySQL(tctx, handleOneRow, func() {
oneRow[0], oneRow[1] = "", ""
}, query)
if err != nil {
return "", err
}
createSequenceSQL.WriteString(oneRow[1])
createSequenceSQL.WriteString(";\n")

switch conf.ServerInfo.ServerType {
case version.ServerTypeTiDB:
// Get next not allocated auto increment id of the whole cluster
query := fmt.Sprintf("SHOW TABLE `%s`.`%s` NEXT_ROW_ID", escapeString(database), escapeString(sequence))
results, err := db.QuerySQLWithColumns(tctx, []string{"NEXT_GLOBAL_ROW_ID", "ID_TYPE"}, query)
if err != nil {
return "", err
}
for _, oneRow := range results {
nextGlobalRowId, idType := oneRow[0], oneRow[1]
if idType == "SEQUENCE" {
nextNotCachedValue, _ = strconv.ParseInt(nextGlobalRowId, 10, 64)
}
}
fmt.Fprintf(&createSequenceSQL, "SELECT SETVAL(`%s`,%d);\n", escapeString(sequence), nextNotCachedValue)
case version.ServerTypeMariaDB:
var oneRow1 string
handleOneRow1 := func(rows *sql.Rows) error {
return rows.Scan(&oneRow1)
}
query := fmt.Sprintf("SELECT NEXT_NOT_CACHED_VALUE FROM `%s`.`%s`", escapeString(database), escapeString(sequence))
err := db.QuerySQL(tctx, handleOneRow1, func() {
oneRow1 = ""
}, query)
if err != nil {
return "", err
}
nextNotCachedValue, _ = strconv.ParseInt(oneRow1, 10, 64)
fmt.Fprintf(&createSequenceSQL, "SELECT SETVAL(`%s`,%d);\n", escapeString(sequence), nextNotCachedValue)
}
return createSequenceSQL.String(), nil
}

// SetCharset builds the set charset SQLs
func SetCharset(w *strings.Builder, characterSet, collationConnection string) {
w.WriteString("SET @PREV_CHARACTER_SET_CLIENT=@@CHARACTER_SET_CLIENT;\n")
Expand Down
40 changes: 40 additions & 0 deletions dumpling/export/sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -368,6 +368,46 @@ func TestShowCreateView(t *testing.T) {
require.NoError(t, mock.ExpectationsWereMet())
}

func TestShowCreateSequence(t *testing.T) {
conf := defaultConfigForTest(t)
db, mock, err := sqlmock.New()
require.NoError(t, err)
defer func() {
require.NoError(t, db.Close())
}()

conn, err := db.Conn(context.Background())
require.NoError(t, err)
tctx := tcontext.Background().WithLogger(appLogger)
baseConn := newBaseConn(conn, true, nil)

conf.ServerInfo.ServerType = version.ServerTypeTiDB
mock.ExpectQuery("SHOW CREATE SEQUENCE `test`.`s`").
WillReturnRows(sqlmock.NewRows([]string{"Sequence", "Create Sequence"}).
AddRow("s", "CREATE SEQUENCE `s` start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 1 cache 1000 nocycle ENGINE=InnoDB"))
mock.ExpectQuery("SHOW TABLE `test`.`s` NEXT_ROW_ID").
WillReturnRows(sqlmock.NewRows([]string{"DB_NAME", "TABLE_NAME", "COLUMN_NAME", "NEXT_GLOBAL_ROW_ID", "ID_TYPE"}).
AddRow("test", "s", nil, 1001, "SEQUENCE"))

createSequenceSQL, err := ShowCreateSequence(tctx, baseConn, "test", "s", conf)
require.NoError(t, err)
require.Equal(t, "CREATE SEQUENCE `s` start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 1 cache 1000 nocycle ENGINE=InnoDB;\nSELECT SETVAL(`s`,1001);\n", createSequenceSQL)
require.NoError(t, mock.ExpectationsWereMet())

conf.ServerInfo.ServerType = version.ServerTypeMariaDB
mock.ExpectQuery("SHOW CREATE SEQUENCE `test`.`s`").
WillReturnRows(sqlmock.NewRows([]string{"Table", "Create Table"}).
AddRow("s", "CREATE SEQUENCE `s` start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 1 cache 1000 nocycle ENGINE=InnoDB"))
mock.ExpectQuery("SELECT NEXT_NOT_CACHED_VALUE FROM `test`.`s`").
WillReturnRows(sqlmock.NewRows([]string{"next_not_cached_value"}).
AddRow(1001))

createSequenceSQL, err = ShowCreateSequence(tctx, baseConn, "test", "s", conf)
require.NoError(t, err)
require.Equal(t, "CREATE SEQUENCE `s` start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 1 cache 1000 nocycle ENGINE=InnoDB;\nSELECT SETVAL(`s`,1001);\n", createSequenceSQL)
require.NoError(t, mock.ExpectationsWereMet())
}

func TestShowCreatePolicy(t *testing.T) {
db, mock, err := sqlmock.New()
require.NoError(t, err)
Expand Down
22 changes: 22 additions & 0 deletions dumpling/export/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,14 @@ type TaskViewMeta struct {
CreateViewSQL string
}

// TaskSequenceMeta is a dumping sequence metadata task
type TaskSequenceMeta struct {
Task
DatabaseName string
SequenceName string
CreateSequenceSQL string
}

// TaskPolicyMeta is a dumping view metadata task
type TaskPolicyMeta struct {
Task
Expand Down Expand Up @@ -77,6 +85,15 @@ func NewTaskViewMeta(dbName, tblName, createTableSQL, createViewSQL string) *Tas
}
}

// NewTaskSequenceMeta returns a new dumping sequence metadata task
func NewTaskSequenceMeta(dbName, tblName, createSequenceSQL string) *TaskSequenceMeta {
return &TaskSequenceMeta{
DatabaseName: dbName,
SequenceName: tblName,
CreateSequenceSQL: createSequenceSQL,
}
}

// NewTaskPolicyMeta returns a new dumping placement policy metadata task
func NewTaskPolicyMeta(policyName, createPolicySQL string) *TaskPolicyMeta {
return &TaskPolicyMeta{
Expand Down Expand Up @@ -110,6 +127,11 @@ func (t *TaskViewMeta) Brief() string {
return fmt.Sprintf("meta of view '%s'.'%s'", t.DatabaseName, t.ViewName)
}

// Brief implements task.Brief
func (t *TaskSequenceMeta) Brief() string {
return fmt.Sprintf("meta of sequence '%s'.'%s'", t.DatabaseName, t.SequenceName)
}

// Brief implements task.Brief
func (t *TaskPolicyMeta) Brief() string {
return fmt.Sprintf("meta of placement policy '%s'", t.PolicyName)
Expand Down
12 changes: 12 additions & 0 deletions dumpling/export/writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,8 @@ func (w *Writer) handleTask(task Task) error {
return w.WriteTableMeta(t.DatabaseName, t.TableName, t.CreateTableSQL)
case *TaskViewMeta:
return w.WriteViewMeta(t.DatabaseName, t.ViewName, t.CreateTableSQL, t.CreateViewSQL)
case *TaskSequenceMeta:
return w.WriteSequenceMeta(t.DatabaseName, t.SequenceName, t.CreateSequenceSQL)
case *TaskPolicyMeta:
return w.WritePolicyMeta(t.PolicyName, t.CreatePolicySQL)
case *TaskTableData:
Expand Down Expand Up @@ -164,6 +166,16 @@ func (w *Writer) WriteViewMeta(db, view, createTableSQL, createViewSQL string) e
return writeMetaToFile(tctx, db, createViewSQL, w.extStorage, fileNameView+".sql", conf.CompressType)
}

// WriteSequenceMeta writes sequence meta to a file
func (w *Writer) WriteSequenceMeta(db, sequence, createSQL string) error {
tctx, conf := w.tctx, w.conf
fileName, err := (&outputFileNamer{DB: db, Table: sequence}).render(conf.OutputFileTemplate, outputFileTemplateSequence)
if err != nil {
return err
}
return writeMetaToFile(tctx, db, createSQL, w.extStorage, fileName+".sql", conf.CompressType)
}

// WriteTableData writes table data to a file with retry
func (w *Writer) WriteTableData(meta TableMeta, ir TableDataIR, currentChunk int) error {
tctx, conf, conn := w.tctx, w.conf, w.conn
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
/*!40101 SET NAMES binary*/;
CREATE SEQUENCE `s` start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 1 cache 1000 nocycle ENGINE=InnoDB;
SELECT SETVAL(`s`,1001);
30 changes: 30 additions & 0 deletions dumpling/tests/sequences/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
#!/bin/sh
#
# Copyright 2020 PingCAP, Inc. Licensed under Apache-2.0.

set -eu

export DUMPLING_TEST_PORT=4000

run_sql "drop database if exists sequences;"
run_sql "create database sequences DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;"
export DUMPLING_TEST_DATABASE="sequences"

run_sql "create sequence s start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 1 cache 1000 nocycle ENGINE=InnoDB;"
run_sql "select nextval(s);"

# test --no-sequences
run_dumpling --no-sequences
file_not_exist "$DUMPLING_OUTPUT_DIR/sequences.s-schema-sequence.sql"

rm -rf $DUMPLING_OUTPUT_DIR
run_dumpling --no-sequences=false
diff "$DUMPLING_BASE_NAME/data/sequences.s-schema-sequence-expect.sql" "$DUMPLING_OUTPUT_DIR/sequences.s-schema-sequence.sql"
file_not_exist "$DUMPLING_OUTPUT_DIR/sequences.s.000000000.sql"

# test --no-schemas
rm -rf $DUMPLING_OUTPUT_DIR
run_dumpling --no-schemas
file_not_exist "$DUMPLING_OUTPUT_DIR/sequences-schema-create.sql"
file_not_exist "$DUMPLING_OUTPUT_DIR/sequences.s-schema-sequence.sql"
file_not_exist "$DUMPLING_OUTPUT_DIR/sequences.s.000000000.sql"
Loading

0 comments on commit 88708af

Please sign in to comment.