br: add more options for br sql client (#53039)
close pingcap/tidb#53040
This commit is contained in:
@ -415,12 +415,12 @@ func parseCipherType(t string) (encryptionpb.EncryptionMethod, error) {
|
||||
func checkCipherKey(cipherKey, cipherKeyFile string) error {
|
||||
if (len(cipherKey) == 0) == (len(cipherKeyFile) == 0) {
|
||||
return errors.Annotate(berrors.ErrInvalidArgument,
|
||||
"exactly one of --crypter.key or --crypter.key-file should be provided")
|
||||
"exactly one of cipher key or keyfile path should be provided")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func getCipherKeyContent(cipherKey, cipherKeyFile string) ([]byte, error) {
|
||||
func GetCipherKeyContent(cipherKey, cipherKeyFile string) ([]byte, error) {
|
||||
if err := checkCipherKey(cipherKey, cipherKeyFile); err != nil {
|
||||
return nil, errors.Trace(err)
|
||||
}
|
||||
@ -480,7 +480,7 @@ func (cfg *Config) parseCipherInfo(flags *pflag.FlagSet) error {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
cfg.CipherInfo.CipherKey, err = getCipherKeyContent(key, keyFilePath)
|
||||
cfg.CipherInfo.CipherKey, err = GetCipherKeyContent(key, keyFilePath)
|
||||
if err != nil {
|
||||
return errors.Trace(err)
|
||||
}
|
||||
|
||||
@ -460,7 +460,9 @@ go_test(
|
||||
"@com_github_pingcap_errors//:errors",
|
||||
"@com_github_pingcap_failpoint//:failpoint",
|
||||
"@com_github_pingcap_fn//:fn",
|
||||
"@com_github_pingcap_kvproto//pkg/brpb",
|
||||
"@com_github_pingcap_kvproto//pkg/diagnosticspb",
|
||||
"@com_github_pingcap_kvproto//pkg/encryptionpb",
|
||||
"@com_github_pingcap_kvproto//pkg/kvrpcpb",
|
||||
"@com_github_pingcap_kvproto//pkg/metapb",
|
||||
"@com_github_pingcap_log//:log",
|
||||
|
||||
@ -312,6 +312,9 @@ func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema)
|
||||
default:
|
||||
}
|
||||
|
||||
failpoint.Inject("modifyStore", func(v failpoint.Value) {
|
||||
tidbCfg.Store = v.(string)
|
||||
})
|
||||
if tidbCfg.Store != "tikv" {
|
||||
b.err = errors.Errorf("%s requires tikv store, not %s", s.Kind, tidbCfg.Store)
|
||||
return nil
|
||||
@ -330,6 +333,28 @@ func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema)
|
||||
cfg.Checksum = opt.UintValue != 0
|
||||
case ast.BRIEOptionSendCreds:
|
||||
cfg.SendCreds = opt.UintValue != 0
|
||||
case ast.BRIEOptionChecksumConcurrency:
|
||||
cfg.ChecksumConcurrency = uint(opt.UintValue)
|
||||
case ast.BRIEOptionEncryptionKeyFile:
|
||||
cfg.CipherInfo.CipherKey, err = task.GetCipherKeyContent("", opt.StrValue)
|
||||
if err != nil {
|
||||
b.err = err
|
||||
return nil
|
||||
}
|
||||
case ast.BRIEOptionEncryptionMethod:
|
||||
switch opt.StrValue {
|
||||
case "aes128-ctr":
|
||||
cfg.CipherInfo.CipherType = encryptionpb.EncryptionMethod_AES128_CTR
|
||||
case "aes192-ctr":
|
||||
cfg.CipherInfo.CipherType = encryptionpb.EncryptionMethod_AES192_CTR
|
||||
case "aes256-ctr":
|
||||
cfg.CipherInfo.CipherType = encryptionpb.EncryptionMethod_AES256_CTR
|
||||
case "plaintext":
|
||||
cfg.CipherInfo.CipherType = encryptionpb.EncryptionMethod_PLAINTEXT
|
||||
default:
|
||||
b.err = errors.Errorf("unsupported encryption method: %s", opt.StrValue)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -383,6 +408,22 @@ func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema)
|
||||
return nil
|
||||
}
|
||||
e.backupCfg.BackupTS = tso
|
||||
case ast.BRIEOptionCompression:
|
||||
switch opt.StrValue {
|
||||
case "zstd":
|
||||
e.backupCfg.CompressionConfig.CompressionType = backuppb.CompressionType_ZSTD
|
||||
case "snappy":
|
||||
e.backupCfg.CompressionConfig.CompressionType = backuppb.CompressionType_SNAPPY
|
||||
case "lz4":
|
||||
e.backupCfg.CompressionConfig.CompressionType = backuppb.CompressionType_LZ4
|
||||
default:
|
||||
b.err = errors.Errorf("unsupported compression type: %s", opt.StrValue)
|
||||
return nil
|
||||
}
|
||||
case ast.BRIEOptionCompressionLevel:
|
||||
e.backupCfg.CompressionConfig.CompressionLevel = int32(opt.UintValue)
|
||||
case ast.BRIEOptionIgnoreStats:
|
||||
e.backupCfg.IgnoreStats = opt.UintValue != 0
|
||||
}
|
||||
}
|
||||
|
||||
@ -391,8 +432,15 @@ func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema)
|
||||
rcfg.Config = cfg
|
||||
e.restoreCfg = &rcfg
|
||||
for _, opt := range s.Options {
|
||||
if opt.Tp == ast.BRIEOptionOnline {
|
||||
switch opt.Tp {
|
||||
case ast.BRIEOptionOnline:
|
||||
e.restoreCfg.Online = opt.UintValue != 0
|
||||
case ast.BRIEOptionWaitTiflashReady:
|
||||
e.restoreCfg.WaitTiflashReady = opt.UintValue != 0
|
||||
case ast.BRIEOptionWithSysTable:
|
||||
e.restoreCfg.WithSysTable = opt.UintValue != 0
|
||||
case ast.BRIEOptionLoadStats:
|
||||
e.restoreCfg.LoadStats = opt.UintValue != 0
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -17,11 +17,15 @@ package executor
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/failpoint"
|
||||
backuppb "github.com/pingcap/kvproto/pkg/brpb"
|
||||
"github.com/pingcap/kvproto/pkg/encryptionpb"
|
||||
"github.com/pingcap/tidb/pkg/executor/internal/exec"
|
||||
"github.com/pingcap/tidb/pkg/infoschema"
|
||||
"github.com/pingcap/tidb/pkg/parser"
|
||||
@ -140,3 +144,82 @@ func TestFetchShowBRIE(t *testing.T) {
|
||||
globalBRIEQueue.clearTask(e.Ctx().GetSessionVars().StmtCtx)
|
||||
require.Equal(t, info2Res, fetchShowBRIEResult(t, e, brieColTypes))
|
||||
}
|
||||
|
||||
func TestBRIEBuilderOPtions(t *testing.T) {
|
||||
sctx := mock.NewContext()
|
||||
sctx.GetSessionVars().User = &auth.UserIdentity{Username: "test"}
|
||||
is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()})
|
||||
ResetGlobalBRIEQueueForTest()
|
||||
builder := NewMockExecutorBuilderForTest(sctx, is)
|
||||
ctx := context.Background()
|
||||
p := parser.New()
|
||||
p.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true})
|
||||
failpoint.Enable("github.com/pingcap/tidb/pkg/executor/modifyStore", `return("tikv")`)
|
||||
defer failpoint.Disable("github.com/pingcap/tidb/pkg/executor/modifyStore")
|
||||
err := os.WriteFile("/tmp/keyfile", []byte(strings.Repeat("A", 128)), 0644)
|
||||
|
||||
require.NoError(t, err)
|
||||
stmt, err := p.ParseOneStmt("BACKUP TABLE `a` TO 'noop://' CHECKSUM_CONCURRENCY = 4 IGNORE_STATS = 1 COMPRESSION_LEVEL = 4 COMPRESSION_TYPE = 'lz4' ENCRYPTION_METHOD = 'aes256-ctr' ENCRYPTION_KEYFILE = '/tmp/keyfile'", "", "")
|
||||
require.NoError(t, err)
|
||||
plan, err := core.BuildLogicalPlanForTest(ctx, sctx, stmt, infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable(), core.MockView()}))
|
||||
require.NoError(t, err)
|
||||
s, ok := stmt.(*ast.BRIEStmt)
|
||||
require.True(t, ok)
|
||||
require.True(t, s.Kind == ast.BRIEKindBackup)
|
||||
for _, opt := range s.Options {
|
||||
switch opt.Tp {
|
||||
case ast.BRIEOptionChecksumConcurrency:
|
||||
require.Equal(t, uint64(4), opt.UintValue)
|
||||
case ast.BRIEOptionCompressionLevel:
|
||||
require.Equal(t, uint64(4), opt.UintValue)
|
||||
case ast.BRIEOptionIgnoreStats:
|
||||
require.Equal(t, uint64(1), opt.UintValue)
|
||||
case ast.BRIEOptionCompression:
|
||||
require.Equal(t, "lz4", opt.StrValue)
|
||||
case ast.BRIEOptionEncryptionMethod:
|
||||
require.Equal(t, "aes256-ctr", opt.StrValue)
|
||||
case ast.BRIEOptionEncryptionKeyFile:
|
||||
require.Equal(t, "/tmp/keyfile", opt.StrValue)
|
||||
}
|
||||
}
|
||||
schema := plan.Schema()
|
||||
exec := builder.buildBRIE(s, schema)
|
||||
require.NoError(t, builder.err)
|
||||
e, ok := exec.(*BRIEExec)
|
||||
require.True(t, ok)
|
||||
require.Equal(t, uint(4), e.backupCfg.ChecksumConcurrency)
|
||||
require.Equal(t, int32(4), e.backupCfg.CompressionLevel)
|
||||
require.Equal(t, true, e.backupCfg.IgnoreStats)
|
||||
require.Equal(t, backuppb.CompressionType_LZ4, e.backupCfg.CompressionConfig.CompressionType)
|
||||
require.Equal(t, encryptionpb.EncryptionMethod_AES256_CTR, e.backupCfg.CipherInfo.CipherType)
|
||||
require.Greater(t, len(e.backupCfg.CipherInfo.CipherKey), 0)
|
||||
|
||||
stmt, err = p.ParseOneStmt("RESTORE TABLE `a` FROM 'noop://' CHECKSUM_CONCURRENCY = 4 WAIT_TIFLASH_READY = 1 WITH_SYS_TABLE = 1 LOAD_STATS = 1", "", "")
|
||||
require.NoError(t, err)
|
||||
plan, err = core.BuildLogicalPlanForTest(ctx, sctx, stmt, infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable(), core.MockView()}))
|
||||
require.NoError(t, err)
|
||||
s, ok = stmt.(*ast.BRIEStmt)
|
||||
require.True(t, ok)
|
||||
require.True(t, s.Kind == ast.BRIEKindRestore)
|
||||
for _, opt := range s.Options {
|
||||
switch opt.Tp {
|
||||
case ast.BRIEOptionChecksumConcurrency:
|
||||
require.Equal(t, uint64(4), opt.UintValue)
|
||||
case ast.BRIEOptionWaitTiflashReady:
|
||||
require.Equal(t, uint64(1), opt.UintValue)
|
||||
case ast.BRIEOptionWithSysTable:
|
||||
require.Equal(t, uint64(1), opt.UintValue)
|
||||
case ast.BRIEOptionLoadStats:
|
||||
require.Equal(t, uint64(1), opt.UintValue)
|
||||
}
|
||||
}
|
||||
schema = plan.Schema()
|
||||
exec = builder.buildBRIE(s, schema)
|
||||
require.NoError(t, builder.err)
|
||||
e, ok = exec.(*BRIEExec)
|
||||
require.True(t, ok)
|
||||
require.Equal(t, uint(4), e.restoreCfg.ChecksumConcurrency)
|
||||
require.True(t, e.restoreCfg.WaitTiflashReady)
|
||||
require.True(t, e.restoreCfg.WithSysTable)
|
||||
require.True(t, e.restoreCfg.LoadStats)
|
||||
}
|
||||
|
||||
@ -3278,6 +3278,9 @@ const (
|
||||
BRIEOptionCheckpoint
|
||||
BRIEOptionStartTS
|
||||
BRIEOptionUntilTS
|
||||
BRIEOptionChecksumConcurrency
|
||||
BRIEOptionEncryptionMethod
|
||||
BRIEOptionEncryptionKeyFile
|
||||
// backup options
|
||||
BRIEOptionBackupTimeAgo
|
||||
BRIEOptionBackupTS
|
||||
@ -3285,10 +3288,16 @@ const (
|
||||
BRIEOptionLastBackupTS
|
||||
BRIEOptionLastBackupTSO
|
||||
BRIEOptionGCTTL
|
||||
BRIEOptionCompressionLevel
|
||||
BRIEOptionCompression
|
||||
BRIEOptionIgnoreStats
|
||||
BRIEOptionLoadStats
|
||||
// restore options
|
||||
BRIEOptionOnline
|
||||
BRIEOptionFullBackupStorage
|
||||
BRIEOptionRestoredTS
|
||||
BRIEOptionWaitTiflashReady
|
||||
BRIEOptionWithSysTable
|
||||
// import options
|
||||
BRIEOptionAnalyze
|
||||
BRIEOptionBackend
|
||||
@ -3408,6 +3417,24 @@ func (kind BRIEOptionType) String() string {
|
||||
return "UNTIL_TS"
|
||||
case BRIEOptionGCTTL:
|
||||
return "GC_TTL"
|
||||
case BRIEOptionWaitTiflashReady:
|
||||
return "WAIT_TIFLASH_READY"
|
||||
case BRIEOptionWithSysTable:
|
||||
return "WITH_SYS_TABLE"
|
||||
case BRIEOptionIgnoreStats:
|
||||
return "IGNORE_STATS"
|
||||
case BRIEOptionLoadStats:
|
||||
return "LOAD_STATS"
|
||||
case BRIEOptionChecksumConcurrency:
|
||||
return "CHECKSUM_CONCURRENCY"
|
||||
case BRIEOptionCompressionLevel:
|
||||
return "COMPRESSION_LEVEL"
|
||||
case BRIEOptionCompression:
|
||||
return "COMPRESSION_TYPE"
|
||||
case BRIEOptionEncryptionMethod:
|
||||
return "ENCRYPTION_METHOD"
|
||||
case BRIEOptionEncryptionKeyFile:
|
||||
return "ENCRYPTION_KEY_FILE"
|
||||
default:
|
||||
return ""
|
||||
}
|
||||
@ -3436,7 +3463,7 @@ func (opt *BRIEOption) Restore(ctx *format.RestoreCtx) error {
|
||||
ctx.WriteKeyWord(opt.Tp.String())
|
||||
ctx.WritePlain(" = ")
|
||||
switch opt.Tp {
|
||||
case BRIEOptionBackupTS, BRIEOptionLastBackupTS, BRIEOptionBackend, BRIEOptionOnDuplicate, BRIEOptionTiKVImporter, BRIEOptionCSVDelimiter, BRIEOptionCSVNull, BRIEOptionCSVSeparator, BRIEOptionFullBackupStorage, BRIEOptionRestoredTS, BRIEOptionStartTS, BRIEOptionUntilTS, BRIEOptionGCTTL:
|
||||
case BRIEOptionBackupTS, BRIEOptionLastBackupTS, BRIEOptionBackend, BRIEOptionOnDuplicate, BRIEOptionTiKVImporter, BRIEOptionCSVDelimiter, BRIEOptionCSVNull, BRIEOptionCSVSeparator, BRIEOptionFullBackupStorage, BRIEOptionRestoredTS, BRIEOptionStartTS, BRIEOptionUntilTS, BRIEOptionGCTTL, BRIEOptionCompression, BRIEOptionEncryptionMethod, BRIEOptionEncryptionKeyFile:
|
||||
ctx.WriteString(opt.StrValue)
|
||||
case BRIEOptionBackupTimeAgo:
|
||||
ctx.WritePlainf("%d ", opt.UintValue/1000)
|
||||
|
||||
@ -300,6 +300,7 @@ var Keywords = []KeywordsType{
|
||||
{"CHARSET", false, "unreserved"},
|
||||
{"CHECKPOINT", false, "unreserved"},
|
||||
{"CHECKSUM", false, "unreserved"},
|
||||
{"CHECKSUM_CONCURRENCY", false, "unreserved"},
|
||||
{"CIPHER", false, "unreserved"},
|
||||
{"CLEANUP", false, "unreserved"},
|
||||
{"CLIENT", false, "unreserved"},
|
||||
@ -317,6 +318,8 @@ var Keywords = []KeywordsType{
|
||||
{"COMPACT", false, "unreserved"},
|
||||
{"COMPRESSED", false, "unreserved"},
|
||||
{"COMPRESSION", false, "unreserved"},
|
||||
{"COMPRESSION_LEVEL", false, "unreserved"},
|
||||
{"COMPRESSION_TYPE", false, "unreserved"},
|
||||
{"CONCURRENCY", false, "unreserved"},
|
||||
{"CONFIG", false, "unreserved"},
|
||||
{"CONNECTION", false, "unreserved"},
|
||||
@ -353,6 +356,8 @@ var Keywords = []KeywordsType{
|
||||
{"ENABLE", false, "unreserved"},
|
||||
{"ENABLED", false, "unreserved"},
|
||||
{"ENCRYPTION", false, "unreserved"},
|
||||
{"ENCRYPTION_KEYFILE", false, "unreserved"},
|
||||
{"ENCRYPTION_METHOD", false, "unreserved"},
|
||||
{"END", false, "unreserved"},
|
||||
{"ENFORCED", false, "unreserved"},
|
||||
{"ENGINE", false, "unreserved"},
|
||||
@ -394,6 +399,7 @@ var Keywords = []KeywordsType{
|
||||
{"HOUR", false, "unreserved"},
|
||||
{"HYPO", false, "unreserved"},
|
||||
{"IDENTIFIED", false, "unreserved"},
|
||||
{"IGNORE_STATS", false, "unreserved"},
|
||||
{"IMPORT", false, "unreserved"},
|
||||
{"IMPORTS", false, "unreserved"},
|
||||
{"INCREMENT", false, "unreserved"},
|
||||
@ -417,6 +423,7 @@ var Keywords = []KeywordsType{
|
||||
{"LESS", false, "unreserved"},
|
||||
{"LEVEL", false, "unreserved"},
|
||||
{"LIST", false, "unreserved"},
|
||||
{"LOAD_STATS", false, "unreserved"},
|
||||
{"LOCAL", false, "unreserved"},
|
||||
{"LOCATION", false, "unreserved"},
|
||||
{"LOCKED", false, "unreserved"},
|
||||
@ -619,10 +626,12 @@ var Keywords = []KeywordsType{
|
||||
{"VIEW", false, "unreserved"},
|
||||
{"VISIBLE", false, "unreserved"},
|
||||
{"WAIT", false, "unreserved"},
|
||||
{"WAIT_TIFLASH_READY", false, "unreserved"},
|
||||
{"WARNINGS", false, "unreserved"},
|
||||
{"WEEK", false, "unreserved"},
|
||||
{"WEIGHT_STRING", false, "unreserved"},
|
||||
{"WITHOUT", false, "unreserved"},
|
||||
{"WITH_SYS_TABLE", false, "unreserved"},
|
||||
{"WORKLOAD", false, "unreserved"},
|
||||
{"X509", false, "unreserved"},
|
||||
{"YEAR", false, "unreserved"},
|
||||
|
||||
@ -36,7 +36,7 @@ func TestKeywords(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestKeywordsLength(t *testing.T) {
|
||||
require.Equal(t, 644, len(parser.Keywords))
|
||||
require.Equal(t, 653, len(parser.Keywords))
|
||||
|
||||
reservedNr := 0
|
||||
for _, kw := range parser.Keywords {
|
||||
|
||||
@ -286,6 +286,15 @@ var tokenMap = map[string]int{
|
||||
"CSV_NULL": csvNull,
|
||||
"CSV_SEPARATOR": csvSeparator,
|
||||
"CSV_TRIM_LAST_SEPARATORS": csvTrimLastSeparators,
|
||||
"WAIT_TIFLASH_READY": waitTiflashReady,
|
||||
"WITH_SYS_TABLE": withSysTable,
|
||||
"IGNORE_STATS": ignoreStats,
|
||||
"LOAD_STATS": loadStats,
|
||||
"CHECKSUM_CONCURRENCY": checksumConcurrency,
|
||||
"COMPRESSION_LEVEL": compressionLevel,
|
||||
"COMPRESSION_TYPE": compressionType,
|
||||
"ENCRYPTION_METHOD": encryptionMethod,
|
||||
"ENCRYPTION_KEYFILE": encryptionKeyFile,
|
||||
"CURDATE": curDate,
|
||||
"CURRENT_DATE": currentDate,
|
||||
"CURRENT_ROLE": currentRole,
|
||||
|
||||
22488
pkg/parser/parser.go
22488
pkg/parser/parser.go
File diff suppressed because it is too large
Load Diff
@ -682,6 +682,15 @@ import (
|
||||
workload "WORKLOAD"
|
||||
x509 "X509"
|
||||
yearType "YEAR"
|
||||
withSysTable "WITH_SYS_TABLE"
|
||||
waitTiflashReady "WAIT_TIFLASH_READY"
|
||||
ignoreStats "IGNORE_STATS"
|
||||
loadStats "LOAD_STATS"
|
||||
checksumConcurrency "CHECKSUM_CONCURRENCY"
|
||||
compressionLevel "COMPRESSION_LEVEL"
|
||||
compressionType "COMPRESSION_TYPE"
|
||||
encryptionMethod "ENCRYPTION_METHOD"
|
||||
encryptionKeyFile "ENCRYPTION_KEYFILE"
|
||||
|
||||
/* The following tokens belong to NotKeywordToken. Notice: make sure these tokens are contained in NotKeywordToken. */
|
||||
addDate "ADDDATE"
|
||||
@ -5589,6 +5598,14 @@ BRIEIntegerOptionName:
|
||||
{
|
||||
$$ = ast.BRIEOptionResume
|
||||
}
|
||||
| "CHECKSUM_CONCURRENCY"
|
||||
{
|
||||
$$ = ast.BRIEOptionChecksumConcurrency
|
||||
}
|
||||
| "COMPRESSION_LEVEL"
|
||||
{
|
||||
$$ = ast.BRIEOptionCompressionLevel
|
||||
}
|
||||
|
||||
BRIEBooleanOptionName:
|
||||
"SEND_CREDENTIALS_TO_TIKV"
|
||||
@ -5623,6 +5640,22 @@ BRIEBooleanOptionName:
|
||||
{
|
||||
$$ = ast.BRIEOptionCSVTrimLastSeparators
|
||||
}
|
||||
| "WAIT_TIFLASH_READY"
|
||||
{
|
||||
$$ = ast.BRIEOptionWaitTiflashReady
|
||||
}
|
||||
| "WITH_SYS_TABLE"
|
||||
{
|
||||
$$ = ast.BRIEOptionWithSysTable
|
||||
}
|
||||
| "IGNORE_STATS"
|
||||
{
|
||||
$$ = ast.BRIEOptionIgnoreStats
|
||||
}
|
||||
| "LOAD_STATS"
|
||||
{
|
||||
$$ = ast.BRIEOptionLoadStats
|
||||
}
|
||||
|
||||
BRIEStringOptionName:
|
||||
"TIKV_IMPORTER"
|
||||
@ -5641,6 +5674,18 @@ BRIEStringOptionName:
|
||||
{
|
||||
$$ = ast.BRIEOptionCSVNull
|
||||
}
|
||||
| "COMPRESSION_TYPE"
|
||||
{
|
||||
$$ = ast.BRIEOptionCompression
|
||||
}
|
||||
| "ENCRYPTION_METHOD"
|
||||
{
|
||||
$$ = ast.BRIEOptionEncryptionMethod
|
||||
}
|
||||
| "ENCRYPTION_KEYFILE"
|
||||
{
|
||||
$$ = ast.BRIEOptionEncryptionKeyFile
|
||||
}
|
||||
|
||||
BRIEKeywordOptionName:
|
||||
"BACKEND"
|
||||
@ -6917,6 +6962,15 @@ UnReservedKeyword:
|
||||
| "OLTP_READ_ONLY"
|
||||
| "OLTP_WRITE_ONLY"
|
||||
| "TPCH_10"
|
||||
| "WITH_SYS_TABLE"
|
||||
| "WAIT_TIFLASH_READY"
|
||||
| "IGNORE_STATS"
|
||||
| "LOAD_STATS"
|
||||
| "CHECKSUM_CONCURRENCY"
|
||||
| "COMPRESSION_LEVEL"
|
||||
| "COMPRESSION_TYPE"
|
||||
| "ENCRYPTION_METHOD"
|
||||
| "ENCRYPTION_KEYFILE"
|
||||
|
||||
TiDBKeyword:
|
||||
"ADMIN"
|
||||
|
||||
@ -6867,7 +6867,8 @@ func TestBRIE(t *testing.T) {
|
||||
{"BACKUP DATABASE *, a TO 'noop://'", false, ""},
|
||||
{"BACKUP DATABASE a, * TO 'noop://'", false, ""},
|
||||
{"BACKUP DATABASE TO 'noop://'", false, ""},
|
||||
{"BACKUP TABLE a TO 'noop://'", true, "BACKUP TABLE `a` TO 'noop://'"},
|
||||
{"BACKUP TABLE a TO 'noop://' checksum_concurrency 4 compression_level 4 ignore_stats 1 compression_type 'lz4'", true, "BACKUP TABLE `a` TO 'noop://' CHECKSUM_CONCURRENCY = 4 COMPRESSION_LEVEL = 4 IGNORE_STATS = 1 COMPRESSION_TYPE = 'lz4'"},
|
||||
{"RESTORE TABLE a FROM 'noop://' checksum_concurrency 4 wait_tiflash_ready 1 with_sys_table 1", true, "RESTORE TABLE `a` FROM 'noop://' CHECKSUM_CONCURRENCY = 4 WAIT_TIFLASH_READY = 1 WITH_SYS_TABLE = 1"},
|
||||
{"BACKUP TABLE a.b TO 'noop://'", true, "BACKUP TABLE `a`.`b` TO 'noop://'"},
|
||||
{"BACKUP TABLE a.b,c.d,e TO 'noop://'", true, "BACKUP TABLE `a`.`b`, `c`.`d`, `e` TO 'noop://'"},
|
||||
{"BACKUP TABLE a.* TO 'noop://'", false, ""},
|
||||
|
||||
Reference in New Issue
Block a user