diff --git a/.gitignore b/.gitignore index b1c7ad4934527..c71bd34b114bd 100644 --- a/.gitignore +++ b/.gitignore @@ -24,3 +24,5 @@ coverage.txt var fix.sql export-20*/ +*-coverage.xml +*-junit-report.xml diff --git a/Makefile b/Makefile index 0e5e9879816fd..36291f5d6ca40 100644 --- a/Makefile +++ b/Makefile @@ -109,13 +109,6 @@ explaintest: server_check ddltest: @cd cmd/ddltest && $(GO) test -o ../../bin/ddltest -c -upload-coverage: -ifneq ($(CODECOV_TOKEN), "") - curl -LO ${FILE_SERVER_URL}/download/cicd/ci-tools/codecov - chmod +x codecov - ./codecov -t ${CODECOV_TOKEN} -endif - devgotest: failpoint-enable # grep regex: Filter out all tidb logs starting with: # - '[20' (like [2021/09/15 ...] [INFO]..) @@ -132,9 +125,29 @@ gotest: failpoint-enable $(GOTEST) -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -cover $(PACKAGES_TIDB_TESTS) -coverprofile=coverage.txt -check.p true > gotest.log || { $(FAILPOINT_DISABLE); cat 'gotest.log'; exit 1; } @$(FAILPOINT_DISABLE) +gotest_in_verify_ci_part_1: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml + @echo "Running gotest_in_verify_ci_part_1." + @mkdir -p $(TEST_COVERAGE_DIR) + @export log_level=info; export TZ='Asia/Shanghai'; \ + CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) --race \ + -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" \ + $(PACKAGES_TIDB_TESTS_EXPENSIVE) -check.p true || { $(FAILPOINT_DISABLE); exit 1; } + tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/tidb-coverage.xml" + @$(FAILPOINT_DISABLE) + +gotest_in_verify_ci_part_2: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml + @echo "Running gotest_in_verify_ci_part_2." + @mkdir -p $(TEST_COVERAGE_DIR) + @export log_level=info; export TZ='Asia/Shanghai'; \ + CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) --race \ + -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" \ + $(PACKAGES_TIDB_TESTS_OTHERS) -check.p true || { $(FAILPOINT_DISABLE); exit 1; } + tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/tidb-coverage.xml" + @$(FAILPOINT_DISABLE) + race: failpoint-enable @export log_level=debug; \ - $(GOTEST) -timeout 20m -race $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; } + $(GOTEST) -timeout 25m -race $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) leak: failpoint-enable @@ -310,7 +323,16 @@ br_unit_test: export ARGS=$$($(BR_PACKAGES)) br_unit_test: @make failpoint-enable @export TZ='Asia/Shanghai'; - $(GOTEST) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -tags leak $(ARGS) || ( make failpoint-disable && exit 1 ) + $(GOTEST) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -tags leak $(ARGS) -coverprofile=coverage.txt || ( make failpoint-disable && exit 1 ) + @make failpoint-disable +br_unit_test_in_verify_ci: export ARGS=$$($(BR_PACKAGES)) +br_unit_test_in_verify_ci: tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml + @make failpoint-enable + @export TZ='Asia/Shanghai'; + @mkdir -p $(TEST_COVERAGE_DIR) + CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/br-junit-report.xml" -- $(RACE_FLAG) -ldflags '$(LDFLAGS)' \ + -tags leak $(ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/br_cov.unit_test.out" || ( make failpoint-disable && exit 1 ) + tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/br_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/br-coverage.xml" @make failpoint-disable br_integration_test: br_bins build_br build_for_br_integration_test @@ -369,6 +391,13 @@ dumpling_unit_test: export DUMPLING_ARGS=$$($(DUMPLING_PACKAGES)) dumpling_unit_test: failpoint-enable $(DUMPLING_GOTEST) $(RACE_FLAG) -coverprofile=coverage.txt -covermode=atomic -tags leak $(DUMPLING_ARGS) || ( make failpoint-disable && exit 1 ) @make failpoint-disable +dumpling_unit_test_in_verify_ci: export DUMPLING_ARGS=$$($(DUMPLING_PACKAGES)) +dumpling_unit_test_in_verify_ci: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml + @mkdir -p $(TEST_COVERAGE_DIR) + CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/dumpling-junit-report.xml" -- -tags leak $(DUMPLING_ARGS) \ + $(RACE_FLAG) -coverprofile="$(TEST_COVERAGE_DIR)/dumpling_cov.unit_test.out" || ( make failpoint-disable && exit 1 ) + tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/dumpling_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/dumpling-coverage.xml" + @make failpoint-disable dumpling_integration_test: dumpling_bins failpoint-enable build_dumpling @make failpoint-disable @@ -388,3 +417,12 @@ dumpling_bins: @which bin/minio @which bin/tidb-lightning @which bin/sync_diff_inspector + +tools/bin/gotestsum: tools/check/go.mod + cd tools/check && $(GO) build -o ../bin/gotestsum gotest.tools/gotestsum + +tools/bin/gocov: tools/check/go.mod + cd tools/check && $(GO) build -o ../bin/gocov github.com/axw/gocov/gocov + +tools/bin/gocov-xml: tools/check/go.mod + cd tools/check && $(GO) build -o ../bin/gocov-xml github.com/AlekSi/gocov-xml diff --git a/Makefile.common b/Makefile.common index 0052c490f95b6..2a8ea369521b1 100644 --- a/Makefile.common +++ b/Makefile.common @@ -47,8 +47,11 @@ MAC := "Darwin" PACKAGE_LIST := go list ./... PACKAGE_LIST_TIDB_TESTS := go list ./... | grep -vE "github.com\/pingcap\/tidb\/br|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/dumpling" +PACKAGE_LIST_TEST_OTHERS := go list ./... | grep -vE "github.com\/pingcap\/tidb\/br|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/dumpling|github.com\/pingcap\/tidb\/executor|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/ddl" PACKAGES ?= $$($(PACKAGE_LIST)) PACKAGES_TIDB_TESTS ?= $$($(PACKAGE_LIST_TIDB_TESTS)) +PACKAGES_TIDB_TESTS_EXPENSIVE ?= "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/ddl" +PACKAGES_TIDB_TESTS_OTHERS ?= $$($(PACKAGE_LIST_TEST_OTHERS)) PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||' PACKAGE_DIRECTORIES_TIDB_TESTS := $(PACKAGE_LIST_TIDB_TESTS) | sed 's|github.com/pingcap/$(PROJECT)/||' FILES := $$(find $$($(PACKAGE_DIRECTORIES)) -name "*.go") @@ -111,3 +114,5 @@ DUMPLING_LDFLAGS += -X "github.com/pingcap/tidb/dumpling/cli.GoVersion=$(shell g DUMPLING_GOBUILD := CGO_ENABLED=0 GO111MODULE=on go build -trimpath -ldflags '$(DUMPLING_LDFLAGS)' DUMPLING_GOTEST := CGO_ENABLED=1 GO111MODULE=on go test -ldflags '$(DUMPLING_LDFLAGS)' + +TEST_COVERAGE_DIR := "test_coverage" diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index 027404a48bb1a..d080d1bad16cf 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -552,27 +552,37 @@ type Security struct { KeyPath string `toml:"key-path" json:"key-path"` // RedactInfoLog indicates that whether enabling redact log RedactInfoLog bool `toml:"redact-info-log" json:"redact-info-log"` + + // TLSConfigName is used to set tls config for lightning in DM, so we don't expose this field to user + // DM may running many lightning instances at same time, so we need to set different tls config name for each lightning + TLSConfigName string `toml:"-" json:"-"` } -// RegistersMySQL registers (or deregisters) the TLS config with name "cluster" +// RegisterMySQL registers the TLS config with name "cluster" or security.TLSConfigName // for use in `sql.Open()`. This method is goroutine-safe. func (sec *Security) RegisterMySQL() error { if sec == nil { return nil } tlsConfig, err := common.ToTLSConfig(sec.CAPath, sec.CertPath, sec.KeyPath) - switch { - case err != nil: + if err != nil { return errors.Trace(err) - case tlsConfig != nil: + } + if tlsConfig != nil { // error happens only when the key coincides with the built-in names. - _ = gomysql.RegisterTLSConfig("cluster", tlsConfig) - default: - gomysql.DeregisterTLSConfig("cluster") + _ = gomysql.RegisterTLSConfig(sec.TLSConfigName, tlsConfig) } return nil } +// DeregisterMySQL deregisters the TLS config with security.TLSConfigName +func (sec *Security) DeregisterMySQL() { + if sec == nil || len(sec.CAPath) == 0 { + return + } + gomysql.DeregisterTLSConfig(sec.TLSConfigName) +} + // A duration which can be deserialized from a TOML string. // Implemented as https://github.com/BurntSushi/toml#using-the-encodingtextunmarshaler-interface type Duration struct { @@ -1124,7 +1134,10 @@ func (cfg *Config) CheckAndAdjustSecurity() error { switch cfg.TiDB.TLS { case "": if len(cfg.TiDB.Security.CAPath) > 0 { - cfg.TiDB.TLS = "cluster" + if cfg.TiDB.Security.TLSConfigName == "" { + cfg.TiDB.Security.TLSConfigName = "cluster" // adjust this the default value + } + cfg.TiDB.TLS = cfg.TiDB.Security.TLSConfigName } else { cfg.TiDB.TLS = "false" } diff --git a/br/pkg/lightning/config/config_test.go b/br/pkg/lightning/config/config_test.go index 581d4e67f8bfa..07dd7a922da4e 100644 --- a/br/pkg/lightning/config/config_test.go +++ b/br/pkg/lightning/config/config_test.go @@ -323,6 +323,13 @@ func (s *configTestSuite) TestAdjustSecuritySection(c *C) { c.Assert(cfg.TiDB.Security.CAPath, Equals, tc.expectedCA, comment) c.Assert(cfg.TiDB.TLS, Equals, tc.expectedTLS, comment) } + // test different tls config name + cfg := config.NewConfig() + assignMinimalLegalValue(cfg) + cfg.Security.CAPath = "/path/to/ca.pem" + cfg.Security.TLSConfigName = "tidb-tls" + c.Assert(cfg.Adjust(context.Background()), IsNil) + c.Assert(cfg.TiDB.Security.TLSConfigName, Equals, cfg.TiDB.TLS) } func (s *configTestSuite) TestInvalidCSV(c *C) { diff --git a/br/pkg/lightning/lightning.go b/br/pkg/lightning/lightning.go index a8b2c0cc563d9..575b661be6cac 100644 --- a/br/pkg/lightning/lightning.go +++ b/br/pkg/lightning/lightning.go @@ -274,10 +274,7 @@ func (l *Lightning) run(taskCtx context.Context, taskCfg *config.Config, g glue. if taskCfg.TiDB.Security == nil { return } - taskCfg.TiDB.Security.CAPath = "" - if err := taskCfg.TiDB.Security.RegisterMySQL(); err != nil { - log.L().Warn("failed to deregister TLS config", log.ShortError(err)) - } + taskCfg.TiDB.Security.DeregisterMySQL() }() // initiation of default glue should be after RegisterMySQL, which is ready to be called after taskCfg.Adjust diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index bf417602bf4d2..777350a855d7b 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -787,6 +787,7 @@ func (rc *Controller) restoreSchema(ctx context.Context) error { os.Exit(0) }) + rc.checkpointsWg.Add(1) // checkpointsWg will be done in `rc.listenCheckpointUpdates` go rc.listenCheckpointUpdates() sysVars := ObtainImportantVariables(ctx, rc.tidbGlue.GetSQLExecutor(), !rc.isTiDBBackend()) @@ -993,7 +994,7 @@ func (rc *Controller) saveStatusCheckpoint(ctx context.Context, tableName string // listenCheckpointUpdates will combine several checkpoints together to reduce database load. func (rc *Controller) listenCheckpointUpdates() { - rc.checkpointsWg.Add(1) + defer rc.checkpointsWg.Done() var lock sync.Mutex coalesed := make(map[string]*checkpoints.TableCheckpointDiff) @@ -1082,7 +1083,6 @@ func (rc *Controller) listenCheckpointUpdates() { } }) } - rc.checkpointsWg.Done() } // buildRunPeriodicActionAndCancelFunc build the runPeriodicAction func and a cancel func diff --git a/br/pkg/lightning/restore/restore_test.go b/br/pkg/lightning/restore/restore_test.go index 62fc9d0175fd2..b710a6627a668 100644 --- a/br/pkg/lightning/restore/restore_test.go +++ b/br/pkg/lightning/restore/restore_test.go @@ -1118,6 +1118,7 @@ func (s *tableRestoreSuite) TestSaveStatusCheckpoint(c *C) { saveCpCh: saveCpCh, checkpointsDB: checkpoints.NewNullCheckpointsDB(), } + rc.checkpointsWg.Add(1) go rc.listenCheckpointUpdates() start := time.Now() diff --git a/br/pkg/restore/split_client.go b/br/pkg/restore/split_client.go index e5105a56dc603..10a9913d8e683 100755 --- a/br/pkg/restore/split_client.go +++ b/br/pkg/restore/split_client.go @@ -416,6 +416,11 @@ func (c *pdClient) getMaxReplica(ctx context.Context) (int, error) { if err != nil { return 0, errors.Trace(err) } + defer func() { + if err = res.Body.Close(); err != nil { + log.Error("Response fail to close", zap.Error(err)) + } + }() var conf config.Config if err := json.NewDecoder(res.Body).Decode(&conf); err != nil { return 0, errors.Trace(err) @@ -482,11 +487,15 @@ func (c *pdClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) if err != nil { return rule, errors.Trace(err) } + defer func() { + if err = res.Body.Close(); err != nil { + log.Error("Response fail to close", zap.Error(err)) + } + }() b, err := io.ReadAll(res.Body) if err != nil { return rule, errors.Trace(err) } - res.Body.Close() err = json.Unmarshal(b, &rule) if err != nil { return rule, errors.Trace(err) diff --git a/br/pkg/storage/gcs.go b/br/pkg/storage/gcs.go index 07ce5c8a862b9..e4835e0eb6111 100644 --- a/br/pkg/storage/gcs.go +++ b/br/pkg/storage/gcs.go @@ -180,11 +180,6 @@ func (s *gcsStorage) WalkDir(ctx context.Context, opt *WalkOption, fn func(strin opt = &WalkOption{} } - maxKeys := int64(1000) - if opt.ListCount > 0 { - maxKeys = opt.ListCount - } - prefix := path.Join(s.gcs.Prefix, opt.SubDir) if len(prefix) > 0 && !strings.HasSuffix(prefix, "/") { prefix += "/" @@ -194,7 +189,7 @@ func (s *gcsStorage) WalkDir(ctx context.Context, opt *WalkOption, fn func(strin // only need each object's name and size query.SetAttrSelection([]string{"Name", "Size"}) iter := s.bucket.Objects(ctx, query) - for i := int64(0); i != maxKeys; i++ { + for { attrs, err := iter.Next() if err == iterator.Done { break diff --git a/br/pkg/storage/gcs_test.go b/br/pkg/storage/gcs_test.go index c3e63d6d410a2..ccf3927497bea 100644 --- a/br/pkg/storage/gcs_test.go +++ b/br/pkg/storage/gcs_test.go @@ -4,6 +4,7 @@ package storage import ( "context" + "fmt" "io" "os" @@ -95,6 +96,31 @@ func (r *testStorageSuite) TestGCS(c *C) { c.Assert(list, Equals, "keykey1key2") c.Assert(totalSize, Equals, int64(42)) + // test 1003 files + totalSize = 0 + for i := 0; i < 1000; i += 1 { + err = stg.WriteFile(ctx, fmt.Sprintf("f%d", i), []byte("data")) + c.Assert(err, IsNil) + } + filesSet := make(map[string]struct{}, 1003) + err = stg.WalkDir(ctx, nil, func(name string, size int64) error { + filesSet[name] = struct{}{} + totalSize += size + return nil + }) + c.Assert(err, IsNil) + c.Assert(totalSize, Equals, int64(42+4000)) + _, ok := filesSet["key"] + c.Assert(ok, IsTrue) + _, ok = filesSet["key1"] + c.Assert(ok, IsTrue) + _, ok = filesSet["key2"] + c.Assert(ok, IsTrue) + for i := 0; i < 1000; i += 1 { + _, ok = filesSet[fmt.Sprintf("f%d", i)] + c.Assert(ok, IsTrue) + } + efr, err := stg.Open(ctx, "key2") c.Assert(err, IsNil) diff --git a/br/pkg/task/common.go b/br/pkg/task/common.go index 36de8583ea92e..4ae54f03cde5a 100644 --- a/br/pkg/task/common.go +++ b/br/pkg/task/common.go @@ -355,7 +355,7 @@ func (cfg *Config) parseCipherInfo(flags *pflag.FlagSet) error { } if !checkCipherKeyMatch(&cfg.CipherInfo) { - return errors.Annotate(err, "Cipher type and key not match") + return errors.Annotate(berrors.ErrInvalidArgument, "crypter method and key length not match") } return nil diff --git a/br/pkg/task/common_test.go b/br/pkg/task/common_test.go index 49bd3cabd0811..6d3baea1bfbbd 100644 --- a/br/pkg/task/common_test.go +++ b/br/pkg/task/common_test.go @@ -3,9 +3,12 @@ package task import ( + "encoding/hex" "fmt" . "github.com/pingcap/check" + backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/tidb/config" "github.com/spf13/pflag" ) @@ -62,3 +65,99 @@ func (s *testCommonSuite) TestStripingPDURL(c *C) { c.Assert(err, IsNil) c.Assert(noChange, Equals, "127.0.0.1:2379") } + +func (s *testCommonSuite) TestCheckCipherKeyMatch(c *C) { + testCases := []struct { + CipherType encryptionpb.EncryptionMethod + CipherKey string + ok bool + }{ + { + CipherType: encryptionpb.EncryptionMethod_PLAINTEXT, + ok: true, + }, + { + CipherType: encryptionpb.EncryptionMethod_UNKNOWN, + ok: false, + }, + { + CipherType: encryptionpb.EncryptionMethod_AES128_CTR, + CipherKey: "0123456789abcdef0123456789abcdef", + ok: true, + }, + { + CipherType: encryptionpb.EncryptionMethod_AES128_CTR, + CipherKey: "0123456789abcdef0123456789abcd", + ok: false, + }, + { + CipherType: encryptionpb.EncryptionMethod_AES192_CTR, + CipherKey: "0123456789abcdef0123456789abcdef0123456789abcdef", + ok: true, + }, + { + CipherType: encryptionpb.EncryptionMethod_AES192_CTR, + CipherKey: "0123456789abcdef0123456789abcdef0123456789abcdefff", + ok: false, + }, + { + CipherType: encryptionpb.EncryptionMethod_AES256_CTR, + CipherKey: "0123456789abcdef0123456789abcdef0123456789abcdef0123456789abcdef", + ok: true, + }, + { + CipherType: encryptionpb.EncryptionMethod_AES256_CTR, + CipherKey: "", + ok: false, + }, + } + + for _, t := range testCases { + cipherKey, err := hex.DecodeString(t.CipherKey) + c.Assert(err, IsNil) + + r := checkCipherKeyMatch(&backuppb.CipherInfo{ + CipherType: t.CipherType, + CipherKey: cipherKey, + }) + c.Assert(r, Equals, t.ok) + } +} + +func (s *testCommonSuite) TestCheckCipherKey(c *C) { + cases := []struct { + cipherKey string + keyFile string + ok bool + }{ + { + cipherKey: "0123456789abcdef0123456789abcdef", + keyFile: "", + ok: true, + }, + { + cipherKey: "0123456789abcdef0123456789abcdef", + keyFile: "/tmp/abc", + ok: false, + }, + { + cipherKey: "", + keyFile: "/tmp/abc", + ok: true, + }, + { + cipherKey: "", + keyFile: "", + ok: false, + }, + } + + for _, t := range cases { + err := checkCipherKey(t.cipherKey, t.keyFile) + if t.ok { + c.Assert(err, IsNil) + } else { + c.Assert(err, NotNil) + } + } +} diff --git a/br/tests/br_log_restore/run.sh b/br/tests/br_log_restore/run.sh deleted file mode 100755 index 3753e58267816..0000000000000 --- a/br/tests/br_log_restore/run.sh +++ /dev/null @@ -1,181 +0,0 @@ -#!/bin/bash -# -# Copyright 2020 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 -eux -DB="$TEST_NAME" -TABLE="usertable" -DB_COUNT=3 -BUCKET="cdcs3" -CDC_COUNT=3 - -# start the s3 server -export MINIO_ACCESS_KEY=brs3accesskey -export MINIO_SECRET_KEY=brs3secretkey -export MINIO_BROWSER=off -export AWS_ACCESS_KEY_ID=$MINIO_ACCESS_KEY -export AWS_SECRET_ACCESS_KEY=$MINIO_SECRET_KEY -export S3_ENDPOINT=127.0.0.1:24928 -rm -rf "$TEST_DIR/$DB" -mkdir -p "$TEST_DIR/$DB" -bin/minio server --address $S3_ENDPOINT "$TEST_DIR/$DB" & -i=0 -while ! curl -o /dev/null -s "http://$S3_ENDPOINT/"; do - i=$(($i+1)) - if [ $i -gt 30 ]; then - echo 'Failed to start minio' - exit 1 - fi - sleep 2 -done - -bin/mc config --config-dir "$TEST_DIR/$TEST_NAME" \ - host add minio http://$S3_ENDPOINT $MINIO_ACCESS_KEY $MINIO_SECRET_KEY -bin/mc mb --config-dir "$TEST_DIR/$TEST_NAME" minio/$BUCKET - -# Start cdc servers -run_cdc server --pd=https://$PD_ADDR --log-file=ticdc.log --addr=0.0.0.0:18301 --advertise-addr=127.0.0.1:18301 & -trap 'cat ticdc.log' ERR - -# TODO: remove this after TiCDC supports TiDB clustered index -run_sql "set @@global.tidb_enable_clustered_index=0" -# TiDB global variables cache 2 seconds -sleep 2 - -# create change feed for s3 log -run_cdc cli changefeed create --pd=https://$PD_ADDR --sink-uri="s3://$BUCKET/$DB?endpoint=http://$S3_ENDPOINT" --changefeed-id="simple-replication-task" - -start_ts=$(run_sql "show master status;" | grep Position | awk -F ':' '{print $2}' | xargs) - -# Fill in the database -for i in $(seq $DB_COUNT); do - run_sql "CREATE DATABASE $DB${i};" - go-ycsb load mysql -P tests/$TEST_NAME/workload -p mysql.host=$TIDB_IP -p mysql.port=$TIDB_PORT -p mysql.user=root -p mysql.db=$DB${i} -done - -for i in $(seq $DB_COUNT); do - row_count_ori[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') -done - -# test drop & create schema/table, finally only db2 has one row -run_sql "create schema ${DB}_DDL1;" -run_sql "create table ${DB}_DDL1.t1 (a int primary key, b varchar(10));" -run_sql "insert into ${DB}_DDL1.t1 values (1, 'x');" - -run_sql "drop schema ${DB}_DDL1;" -run_sql "create schema ${DB}_DDL1;" -run_sql "create schema ${DB}_DDL2;" - -run_sql "create table ${DB}_DDL2.t2 (a int primary key, b varchar(10));" -run_sql "insert into ${DB}_DDl2.t2 values (2, 'x');" - -run_sql "drop table ${DB}_DDL2.t2;" -run_sql "create table ${DB}_DDL2.t2 (a int primary key, b varchar(10));" -run_sql "insert into ${DB}_DDL2.t2 values (3, 'x');" -run_sql "delete from ${DB}_DDL2.t2 where a = 3;" -run_sql "insert into ${DB}_DDL2.t2 values (4, 'x');" - -end_ts=$(run_sql "show master status;" | grep Position | awk -F ':' '{print $2}' | xargs) - - -# if we restore with ts range [start_ts, end_ts], then the below record won't be restored. -run_sql "insert into ${DB}_DDL2.t2 values (5, 'x');" - -wait_time=0 -checkpoint_ts=$(run_cdc cli changefeed query -c simple-replication-task --pd=https://$PD_ADDR | jq '.status."checkpoint-ts"') -while [ "$checkpoint_ts" -lt "$end_ts" ]; do - echo "waiting for cdclog syncing... (checkpoint_ts = $checkpoint_ts; end_ts = $end_ts)" - if [ "$wait_time" -gt 300 ]; then - echo "cdc failed to sync after 300s, please check the CDC log." - exit 1 - fi - sleep 5 - wait_time=$(( wait_time + 5 )) - checkpoint_ts=$(run_cdc cli changefeed query -c simple-replication-task --pd=https://$PD_ADDR | jq '.status."checkpoint-ts"') -done - -# remove the change feed, because we don't want to record the drop ddl. -echo "Y" | run_cdc cli unsafe reset --pd=https://$PD_ADDR - -for i in $(seq $DB_COUNT); do - run_sql "DROP DATABASE $DB${i};" -done -run_sql "DROP DATABASE ${DB}_DDL1" -run_sql "DROP DATABASE ${DB}_DDL2" - -# restore full -export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=return("notleader")' -echo "restore start..." -run_br restore cdclog -s "s3://$BUCKET/$DB" --pd $PD_ADDR --s3.endpoint="http://$S3_ENDPOINT" \ - --log-file "restore.log" --log-level "info" --start-ts $start_ts --end-ts $end_ts - -for i in $(seq $DB_COUNT); do - row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') -done - -fail=false -row_count=$(run_sql "SELECT COUNT(*) FROM ${DB}_DDL2.t2 WHERE a=4;" | awk '/COUNT/{print $2}') -if [ "$row_count" -ne "1" ]; then - fail=true - echo "TEST: [$TEST_NAME] fail on dml&ddl drop test." -fi - - -# record a=5 shouldn't be restore, because we set -end-ts without this record. -row_count=$(run_sql "SELECT COUNT(*) FROM ${DB}_DDL2.t2 WHERE a=5;" | awk '/COUNT/{print $2}') -if [ "$row_count" -ne "0" ]; then - fail=true - echo "TEST: [$TEST_NAME] fail on ts range test." -fi - -export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=return("epochnotmatch")' -echo "restore again to restore a=5 record..." -run_br restore cdclog -s "s3://$BUCKET/$DB" --pd $PD_ADDR --s3.endpoint="http://$S3_ENDPOINT" \ - --log-file "restore.log" --log-level "info" --start-ts $end_ts - -# record a=5 should be restore, because we set -end-ts without this record. -row_count=$(run_sql "SELECT COUNT(*) FROM ${DB}_DDL2.t2 WHERE a=5;" | awk '/COUNT/{print $2}') -if [ "$row_count" -ne "1" ]; then - fail=true - echo "TEST: [$TEST_NAME] fail on recover ts range test." -fi - -# record a=3 should be deleted -row_count=$(run_sql "SELECT COUNT(*) FROM ${DB}_DDL2.t2 WHERE a=3;" | awk '/COUNT/{print $2}') -if [ "$row_count" -ne "0" ]; then - fail=true - echo "TEST: [$TEST_NAME] fail on key not deleted." -fi - - -for i in $(seq $DB_COUNT); do - if [ "${row_count_ori[i]}" != "${row_count_new[i]}" ];then - fail=true - echo "TEST: [$TEST_NAME] fail on database $DB${i}" - fi - echo "database $DB${i} [original] row count: ${row_count_ori[i]}, [after br] row count: ${row_count_new[i]}" -done - -if $fail; then - echo "TEST: [$TEST_NAME] failed!" - exit 1 -fi - -for i in $(seq $DB_COUNT); do - run_sql "DROP DATABASE $DB${i};" -done - -run_sql "DROP DATABASE ${DB}_DDL1" -run_sql "DROP DATABASE ${DB}_DDL2" diff --git a/br/tests/br_log_restore/workload b/br/tests/br_log_restore/workload deleted file mode 100644 index 664fe7ee88228..0000000000000 --- a/br/tests/br_log_restore/workload +++ /dev/null @@ -1,12 +0,0 @@ -recordcount=1000 -operationcount=0 -workload=core - -readallfields=true - -readproportion=0 -updateproportion=0 -scanproportion=0 -insertproportion=0 - -requestdistribution=uniform diff --git a/br/tests/br_other/run.sh b/br/tests/br_other/run.sh index 313f2c5e273c0..79ffb9d2732e8 100644 --- a/br/tests/br_other/run.sh +++ b/br/tests/br_other/run.sh @@ -95,6 +95,8 @@ run_curl https://$PD_ADDR/pd/api/v1/config/schedule | jq '."max-merge-region-siz run_curl https://$PD_ADDR/pd/api/v1/config/schedule | jq '."max-merge-region-keys"' | grep -E "^0$" backup_fail=0 +# generate 1.sst to make another backup failed. +touch "$TEST_DIR/$DB/lock/1.sst" echo "another backup start expect to fail due to last backup add a lockfile" run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB/lock" --concurrency 4 || backup_fail=1 if [ "$backup_fail" -ne "1" ];then diff --git a/br/tests/lightning_error_summary/run.sh b/br/tests/lightning_error_summary/run.sh index 3cbbaa0f9b7d7..dcb06d6bf8c2f 100755 --- a/br/tests/lightning_error_summary/run.sh +++ b/br/tests/lightning_error_summary/run.sh @@ -16,8 +16,6 @@ set -eux -# skip for temporary due to checksum for table a,c succeed, but expect to fail. -exit 0 # Check that error summary are written at the bottom of import. run_sql 'DROP DATABASE IF EXISTS tidb_lightning_checkpoint_error_summary;' diff --git a/cmd/benchfilesort/README.md b/cmd/benchfilesort/README.md deleted file mode 100644 index 79dd784a5709f..0000000000000 --- a/cmd/benchfilesort/README.md +++ /dev/null @@ -1,121 +0,0 @@ -## BenchFileSort - -BenchFileSort is a command line tool to test the performance of util/filesort. - -### Quick Start (Examples) - -Step 1 - Generate the synthetic data - -``` -./benchfilesort gen -keySize 8 -valSize 16 -scale 1000 -``` - -Expected output: - -``` -Generating... -Done! -Data placed in: /path/to/data.out -Time used: xxxx ms -================================= -``` - -Step 2 - Load the data and run the benchmark - -``` -./benchfilesort run -bufSize 50 -nWorkers 1 -inputRatio 100 -outputRatio 50 -``` - -Expected output: - -``` -Loading... - number of rows = 1000, key size = 8, value size = 16 - load 1000 rows -Done! -Loaded 1000 rows -Time used: xxxx ms -================================= -Inputing... -Done! -Input 1000 rows -Time used: xxxx s -================================= -Outputing... -Done! -Output 500 rows -Time used: xxxx ms -================================= -Closing... -Done! -Time used: xxxx ms -================================= -``` - -For performance tuning purpose, `Input` time and `Output` time are two KPIs you should focus on. -`Close` time reflects the GC performance, which might be noteworthy sometimes. - -### Commands and Arguments - -#### `gen` command - -The `gen` command generate the synthetic data for the benchmark. - -You can specify how many rows you want to generate, the key size -and value size for each row. - -The generated data is located in `$dir/data.out` (`$dir` is specified -by the `dir` argument). - -The `gen` command supports the following arguments: - -* `dir` (default: current working directory) - Specify the home directory of generated data - -* `keySize` (default: 8) - Specify the key size for generated rows - -* `valSize` (default: 8) - Specify the value size for generated rows - -* `scale` (default: 100) - Specify how many rows to generate - -* `cpuprofile` (default: "") - Turn on the CPU profile - -#### `run` command - -The `run` command load the synthetic data and run the benchmark. - -You can specify the home directory of the synthetic data. - -The benchmark will use predefined amount of memory, which is controlled -by the `bufSize` argument, to run the test. - -You can control how many rows to input into and output from, which are -defined by the `inputRatio` and `outputRatio` arguments. - -The `run` command supports the following arguments: - -* `dir` (default: current working directory) - Specify the home directory of synthetic data - -* `bufSize` (default: 500000) - Specify the amount of memory used by the benchmark - -* `nWorkers` (default: 1) - Specify the number of workers used in async sorting - -* `inputRatio` (default: 100) - Specify the percentage of rows to input: - - `# of rows to input = # of total rows * inputRatio / 100` - -* `outputRatio` (default: 100) - Specify the percentage of rows to output: - - `# of rows to output = # of rows to input * outputRatio / 100` - -* `cpuprofile` (default: "") - Turn on the CPU profile diff --git a/cmd/benchfilesort/main.go b/cmd/benchfilesort/main.go deleted file mode 100644 index 9a518cf0d5d4b..0000000000000 --- a/cmd/benchfilesort/main.go +++ /dev/null @@ -1,439 +0,0 @@ -// Copyright 2017 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. - -package main - -import ( - "encoding/binary" - "flag" - "fmt" - "math/rand" - "os" - "path/filepath" - "runtime/pprof" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/filesort" - "github.com/pingcap/tidb/util/logutil" -) - -type comparableRow struct { - key []types.Datum - val []types.Datum - handle int64 -} - -var ( - genCmd = flag.NewFlagSet("gen", flag.ExitOnError) - runCmd = flag.NewFlagSet("run", flag.ExitOnError) - - logLevel = "warn" - cpuprofile string - tmpDir string - keySize int - valSize int - bufSize int - scale int - nWorkers int - inputRatio int - outputRatio int -) - -// #nosec G404 -func nextRow(r *rand.Rand, keySize int, valSize int) *comparableRow { - key := make([]types.Datum, keySize) - for i := range key { - key[i] = types.NewDatum(r.Int()) - } - - val := make([]types.Datum, valSize) - for j := range val { - val[j] = types.NewDatum(r.Int()) - } - - handle := r.Int63() - return &comparableRow{key: key, val: val, handle: handle} -} - -func encodeRow(b []byte, row *comparableRow) ([]byte, error) { - var ( - err error - head = make([]byte, 8) - body []byte - ) - sc := &stmtctx.StatementContext{TimeZone: time.Local} - body, err = codec.EncodeKey(sc, body, row.key...) - if err != nil { - return b, errors.Trace(err) - } - body, err = codec.EncodeKey(sc, body, row.val...) - if err != nil { - return b, errors.Trace(err) - } - body, err = codec.EncodeKey(sc, body, types.NewIntDatum(row.handle)) - if err != nil { - return b, errors.Trace(err) - } - - binary.BigEndian.PutUint64(head, uint64(len(body))) - - b = append(b, head...) - b = append(b, body...) - - return b, nil -} - -func decodeRow(fd *os.File) (*comparableRow, error) { - var ( - err error - n int - head = make([]byte, 8) - dcod = make([]types.Datum, 0, keySize+valSize+1) - ) - - n, err = fd.Read(head) - if n != 8 { - return nil, errors.New("incorrect header") - } - if err != nil { - return nil, errors.Trace(err) - } - - rowSize := int(binary.BigEndian.Uint64(head)) - rowBytes := make([]byte, rowSize) - - n, err = fd.Read(rowBytes) - if n != rowSize { - return nil, errors.New("incorrect row") - } - if err != nil { - return nil, errors.Trace(err) - } - - dcod, err = codec.Decode(rowBytes, keySize+valSize+1) - if err != nil { - return nil, errors.Trace(err) - } - - return &comparableRow{ - key: dcod[:keySize], - val: dcod[keySize : keySize+valSize], - handle: dcod[keySize+valSize:][0].GetInt64(), - }, nil -} - -func encodeMeta(b []byte, scale int, keySize int, valSize int) []byte { - meta := make([]byte, 8) - - binary.BigEndian.PutUint64(meta, uint64(scale)) - b = append(b, meta...) - binary.BigEndian.PutUint64(meta, uint64(keySize)) - b = append(b, meta...) - binary.BigEndian.PutUint64(meta, uint64(valSize)) - b = append(b, meta...) - - return b -} - -func decodeMeta(fd *os.File) error { - meta := make([]byte, 24) - if n, err := fd.Read(meta); err != nil || n != 24 { - if n != 24 { - return errors.New("incorrect meta data") - } - return errors.Trace(err) - } - - scale = int(binary.BigEndian.Uint64(meta[:8])) - if scale <= 0 { - return errors.New("number of rows must be positive") - } - - keySize = int(binary.BigEndian.Uint64(meta[8:16])) - if keySize <= 0 { - return errors.New("key size must be positive") - } - - valSize = int(binary.BigEndian.Uint64(meta[16:])) - if valSize <= 0 { - return errors.New("value size must be positive") - } - - return nil -} - -/* - * The synthetic data is exported as a binary format. - * The encoding format is: - * 1) Meta Data - * Three 64-bit integers represent scale size, key size and value size. - * 2) Row Data - * Each row is encoded as: - * One 64-bit integer represent the row size in bytes, followed by the - * the actual row bytes. - */ -// #nosec G404 -func export() error { - var outputBytes []byte - - fileName := filepath.Join(tmpDir, "data.out") - outputFile, err := os.OpenFile(fileName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) - if err != nil { - return errors.Trace(err) - } - defer terror.Call(outputFile.Close) - - outputBytes = encodeMeta(outputBytes, scale, keySize, valSize) - - seed := rand.NewSource(time.Now().UnixNano()) - r := rand.New(seed) - - for i := 1; i <= scale; i++ { - outputBytes, err = encodeRow(outputBytes, nextRow(r, keySize, valSize)) - if err != nil { - return errors.Trace(err) - } - _, err = outputFile.Write(outputBytes) - if err != nil { - return errors.Trace(err) - } - outputBytes = outputBytes[:0] - } - - return nil -} - -func load(ratio int) ([]*comparableRow, error) { - var ( - err error - fd *os.File - ) - - fileName := filepath.Join(tmpDir, "data.out") - fd, err = os.Open(fileName) - if os.IsNotExist(err) { - return nil, errors.New("data file (data.out) does not exist") - } - if err != nil { - return nil, errors.Trace(err) - } - defer terror.Call(fd.Close) - - err = decodeMeta(fd) - if err != nil { - return nil, errors.Trace(err) - } - - cLogf("\tnumber of rows = %d, key size = %d, value size = %d", scale, keySize, valSize) - - var ( - row *comparableRow - rows = make([]*comparableRow, 0, scale) - ) - - totalRows := int(float64(scale) * (float64(ratio) / 100.0)) - cLogf("\tload %d rows", totalRows) - for i := 1; i <= totalRows; i++ { - row, err = decodeRow(fd) - if err != nil { - return nil, errors.Trace(err) - } - rows = append(rows, row) - } - - return rows, nil -} - -func driveGenCmd() { - err := genCmd.Parse(os.Args[2:]) - terror.MustNil(err) - // Sanity checks - if keySize <= 0 { - log.Fatal("key size must be positive") - } - if valSize <= 0 { - log.Fatal("value size must be positive") - } - if scale <= 0 { - log.Fatal("scale must be positive") - } - if _, err = os.Stat(tmpDir); err != nil { - if os.IsNotExist(err) { - log.Fatal("tmpDir does not exist") - } - log.Fatal(err.Error()) - } - - cLog("Generating...") - start := time.Now() - err = export() - terror.MustNil(err) - cLog("Done!") - cLogf("Data placed in: %s", filepath.Join(tmpDir, "data.out")) - cLog("Time used: ", time.Since(start)) - cLog("=================================") -} - -func driveRunCmd() { - err := runCmd.Parse(os.Args[2:]) - terror.MustNil(err) - // Sanity checks - if bufSize <= 0 { - log.Fatal("buffer size must be positive") - } - if nWorkers <= 0 { - log.Fatal("the number of workers must be positive") - } - if inputRatio < 0 || inputRatio > 100 { - log.Fatal("input ratio must between 0 and 100 (inclusive)") - } - if outputRatio < 0 || outputRatio > 100 { - log.Fatal("output ratio must between 0 and 100 (inclusive)") - } - if _, err = os.Stat(tmpDir); err != nil { - if os.IsNotExist(err) { - log.Fatal("tmpDir does not exist") - } - terror.MustNil(err) - } - - var ( - dir string - profile *os.File - fs *filesort.FileSorter - ) - cLog("Loading...") - start := time.Now() - data, err := load(inputRatio) - terror.MustNil(err) - cLog("Done!") - cLogf("Loaded %d rows", len(data)) - cLog("Time used: ", time.Since(start)) - cLog("=================================") - - sc := new(stmtctx.StatementContext) - fsBuilder := new(filesort.Builder) - byDesc := make([]bool, keySize) - for i := 0; i < keySize; i++ { - byDesc[i] = false - } - dir, err = os.MkdirTemp(tmpDir, "benchfilesort_test") - terror.MustNil(err) - fs, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(nWorkers).SetDesc(byDesc).SetDir(dir).Build() - terror.MustNil(err) - - if cpuprofile != "" { - profile, err = os.Create(cpuprofile) - terror.MustNil(err) - } - - cLog("Inputing...") - start = time.Now() - for _, r := range data { - err = fs.Input(r.key, r.val, r.handle) - terror.MustNil(err) - } - cLog("Done!") - cLogf("Input %d rows", len(data)) - cLog("Time used: ", time.Since(start)) - cLog("=================================") - - cLog("Outputing...") - totalRows := int(float64(len(data)) * (float64(outputRatio) / 100.0)) - start = time.Now() - if cpuprofile != "" { - err = pprof.StartCPUProfile(profile) - terror.MustNil(err) - } - for i := 0; i < totalRows; i++ { - _, _, _, err = fs.Output() - terror.MustNil(err) - } - if cpuprofile != "" { - pprof.StopCPUProfile() - } - cLog("Done!") - cLogf("Output %d rows", totalRows) - cLog("Time used: ", time.Since(start)) - cLog("=================================") - - cLog("Closing...") - start = time.Now() - err = fs.Close() - terror.MustNil(err) - cLog("Done!") - cLog("Time used: ", time.Since(start)) - cLog("=================================") -} - -func init() { - err := logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false)) - terror.MustNil(err) - cwd, err1 := os.Getwd() - terror.MustNil(err1) - - genCmd.StringVar(&tmpDir, "dir", cwd, "where to store the generated rows") - genCmd.IntVar(&keySize, "keySize", 8, "the size of key") - genCmd.IntVar(&valSize, "valSize", 8, "the size of value") - genCmd.IntVar(&scale, "scale", 100, "how many rows to generate") - genCmd.StringVar(&cpuprofile, "cpuprofile", "", "write cpu profile to file") - - runCmd.StringVar(&tmpDir, "dir", cwd, "where to load the generated rows") - runCmd.IntVar(&bufSize, "bufSize", 500000, "how many rows held in memory at a time") - runCmd.IntVar(&nWorkers, "nWorkers", 1, "how many workers used in async sorting") - runCmd.IntVar(&inputRatio, "inputRatio", 100, "input percentage") - runCmd.IntVar(&outputRatio, "outputRatio", 100, "output percentage") - runCmd.StringVar(&cpuprofile, "cpuprofile", "", "write cpu profile to file") -} - -func main() { - flag.Parse() - - if len(os.Args) == 1 { - fmt.Printf("Usage:\n\n") - fmt.Printf("\tbenchfilesort command [arguments]\n\n") - fmt.Printf("The commands are:\n\n") - fmt.Println("\tgen\t", "generate rows") - fmt.Println("\trun\t", "run tests") - fmt.Println("") - fmt.Println("Checkout benchfilesort/README for more information.") - return - } - - switch os.Args[1] { - case "gen": - driveGenCmd() - case "run": - driveRunCmd() - default: - fmt.Printf("%q is not valid command.\n", os.Args[1]) - os.Exit(2) - } -} - -func cLogf(format string, args ...interface{}) { - str := fmt.Sprintf(format, args...) - fmt.Println("\033[0;32m" + str + "\033[0m") -} - -func cLog(args ...interface{}) { - str := fmt.Sprint(args...) - fmt.Println("\033[0;32m" + str + "\033[0m") -} diff --git a/cmd/ddltest/main_test.go b/cmd/ddltest/main_test.go index 3afc96f0eab62..890a52a7f4ab8 100644 --- a/cmd/ddltest/main_test.go +++ b/cmd/ddltest/main_test.go @@ -37,6 +37,8 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), goleak.IgnoreTopFunction("internal/poll.runtime_pollWait"), goleak.IgnoreTopFunction("net/http.(*persistConn).writeLoop"), + goleak.IgnoreTopFunction("github.com/go-sql-driver/mysql.(*mysqlConn).startWatcher.func1"), + goleak.IgnoreTopFunction("database/sql.(*DB).connectionOpener"), } goleak.VerifyTestMain(m, opts...) } diff --git a/ddl/db_test.go b/ddl/db_test.go index 31d50861bb0ab..deecc27b8974b 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -61,6 +61,7 @@ import ( "github.com/pingcap/tidb/util/domainutil" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" "github.com/tikv/client-go/v2/testutils" @@ -7290,6 +7291,51 @@ func (s *testSerialDBSuite) TestJsonUnmarshalErrWhenPanicInCancellingPath(c *C) c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '0' for key 'cc'") } +// Close issue #24172. +// See https://github.com/pingcap/tidb/issues/24172 +func (s *testSerialDBSuite) TestCancelJobWriteConflict(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk1 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int)") + + var cancelErr error + var rs []sqlexec.RecordSet + hook := &ddl.TestDDLCallback{} + d := s.dom.DDL() + originalHook := d.GetHook() + d.(ddl.DDLForTest).SetHook(hook) + defer d.(ddl.DDLForTest).SetHook(originalHook) + + // Test when cancelling cannot be retried and adding index succeeds. + hook.OnJobRunBeforeExported = func(job *model.Job) { + if job.Type == model.ActionAddIndex && job.State == model.JobStateRunning && job.SchemaState == model.StateWriteReorganization { + stmt := fmt.Sprintf("admin cancel ddl jobs %d", job.ID) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/kv/mockCommitErrorInNewTxn", `return("no_retry")`), IsNil) + defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/kv/mockCommitErrorInNewTxn"), IsNil) }() + rs, cancelErr = tk1.Se.Execute(context.Background(), stmt) + } + } + tk.MustExec("alter table t add index (id)") + c.Assert(cancelErr.Error(), Equals, "mock commit error") + + // Test when cancelling is retried only once and adding index is cancelled in the end. + var jobID int64 + hook.OnJobRunBeforeExported = func(job *model.Job) { + if job.Type == model.ActionAddIndex && job.State == model.JobStateRunning && job.SchemaState == model.StateWriteReorganization { + jobID = job.ID + stmt := fmt.Sprintf("admin cancel ddl jobs %d", job.ID) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/kv/mockCommitErrorInNewTxn", `return("retry_once")`), IsNil) + defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/kv/mockCommitErrorInNewTxn"), IsNil) }() + rs, cancelErr = tk1.Se.Execute(context.Background(), stmt) + } + } + tk.MustGetErrCode("alter table t add index (id)", errno.ErrCancelledDDLJob) + c.Assert(cancelErr, IsNil) + result := tk1.ResultSetToResultWithCtx(context.Background(), rs[0], Commentf("cancel ddl job fails")) + result.Check(testkit.Rows(fmt.Sprintf("%d successful", jobID))) +} + // For Close issue #24288 // see https://github.com/pingcap/tidb/issues/24288 func (s *testDBSuite8) TestDdlMaxLimitOfIdentifier(c *C) { diff --git a/ddl/ddl.go b/ddl/ddl.go index 1835eb9bfb28e..ebec94019105b 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -545,6 +545,10 @@ func (d *ddl) doDDLJob(ctx sessionctx.Context, job *model.Job) error { d.limitJobCh <- task // worker should restart to continue handling tasks in limitJobCh, and send back through task.err err := <-task.err + if err != nil { + // The transaction of enqueuing job is failed. + return errors.Trace(err) + } ctx.GetSessionVars().StmtCtx.IsDDLJobInQueue = true diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 7ff671fa4f6ea..30cadd4a3a2d7 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -774,6 +774,7 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o hasNullFlag = true case ast.ColumnOptionAutoIncrement: col.Flag |= mysql.AutoIncrementFlag + col.Flag |= mysql.NotNullFlag case ast.ColumnOptionPrimaryKey: // Check PriKeyFlag first to avoid extra duplicate constraints. if col.Flag&mysql.PriKeyFlag == 0 { diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 6cc3a77461c56..617c42c639d6c 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -301,6 +301,11 @@ func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) { return errors.Trace(err) } } + failpoint.Inject("mockAddBatchDDLJobsErr", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.Errorf("mockAddBatchDDLJobsErr")) + } + }) return nil }) var jobs string @@ -310,7 +315,11 @@ func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) { metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerAddDDLJob, task.job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) } - logutil.BgLogger().Info("[ddl] add DDL jobs", zap.Int("batch count", len(tasks)), zap.String("jobs", jobs)) + if err != nil { + logutil.BgLogger().Warn("[ddl] add DDL jobs failed", zap.String("jobs", jobs), zap.Error(err)) + } else { + logutil.BgLogger().Info("[ddl] add DDL jobs", zap.Int("batch count", len(tasks)), zap.String("jobs", jobs)) + } } // getHistoryDDLJob gets a DDL job with job's ID from history queue. diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index d42c6f888a689..5321d12671b40 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -494,6 +494,32 @@ func (s *testDDLSuite) TestColumnError(c *C) { doDDLJobErr(c, dbInfo.ID, tblInfo.ID, model.ActionDropColumns, []interface{}{[]model.CIStr{model.NewCIStr("c5"), model.NewCIStr("c6")}, make([]bool, 2)}, ctx, d) } +func (s *testDDLSerialSuite) TestAddBatchJobError(c *C) { + store := testCreateStore(c, "test_add_batch_job_error") + defer func() { + err := store.Close() + c.Assert(err, IsNil) + }() + d, err := testNewDDLAndStart( + context.Background(), + WithStore(store), + WithLease(testLease), + ) + c.Assert(err, IsNil) + defer func() { + err := d.Stop() + c.Assert(err, IsNil) + }() + ctx := testNewContext(d) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockAddBatchDDLJobsErr", `return(true)`), IsNil) + // Test the job runner should not hang forever. + job := &model.Job{SchemaID: 1, TableID: 1} + err = d.doDDLJob(ctx, job) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "mockAddBatchDDLJobsErr") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockAddBatchDDLJobsErr"), IsNil) +} + func testCheckOwner(c *C, d *ddl, expectedVal bool) { c.Assert(d.isOwner(), Equals, expectedVal) } diff --git a/ddl/partition.go b/ddl/partition.go index ef81b7da5b371..b9d0d72d91864 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -130,7 +130,7 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v return ver, errors.Trace(err) } - if err = infosync.PutRuleBundles(context.TODO(), bundles); err != nil { + if err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles); err != nil { job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } @@ -1040,7 +1040,7 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( if job.Type == model.ActionAddTablePartition { // It is rollbacked from adding table partition, just remove addingDefinitions from tableInfo. physicalTableIDs, pNames, rollbackBundles := rollbackAddingPartitionInfo(tblInfo) - err = infosync.PutRuleBundles(context.TODO(), rollbackBundles) + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), rollbackBundles) if err != nil { job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") @@ -1208,7 +1208,7 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e return ver, errors.Trace(err) } - err = infosync.PutRuleBundles(context.TODO(), bundles) + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles) if err != nil { job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") @@ -1412,7 +1412,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo return ver, errors.Trace(err) } - if err = infosync.PutRuleBundles(context.TODO(), bundles); err != nil { + if err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles); err != nil { job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } diff --git a/ddl/placement_policy.go b/ddl/placement_policy.go index 0fcd17270c1ab..65a445ca6263e 100644 --- a/ddl/placement_policy.go +++ b/ddl/placement_policy.go @@ -262,7 +262,7 @@ func onAlterPlacementPolicy(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, cp := bundle.Clone() bundles = append(bundles, cp.Reset(placement.RuleIndexPartition, []int64{id})) } - err = infosync.PutRuleBundles(context.TODO(), bundles) + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles) if err != nil { job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 26ca779dc3400..51dccd346b336 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -1652,6 +1652,7 @@ func (s *testIntegrationSuite7) TestInvisibleIndex(c *C) { // Implicit primary key cannot be invisible index // Create a implicit primary key tk.MustGetErrCode("create table t2(a int not null, unique (a) invisible)", errno.ErrPKIndexCantBeInvisible) + tk.MustGetErrCode("create table t2(a int auto_increment, unique key (a) invisible);", errno.ErrPKIndexCantBeInvisible) // Column `a` become implicit primary key after DDL statement on itself tk.MustExec("create table t2(a int not null)") tk.MustGetErrCode("alter table t2 add unique (a) invisible", errno.ErrPKIndexCantBeInvisible) diff --git a/ddl/table.go b/ddl/table.go index c2f217fc6eb21..906db66f3ac9d 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -101,7 +101,7 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) } // Send the placement bundle to PD. - err = infosync.PutRuleBundles(context.TODO(), bundles) + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles) if err != nil { job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") @@ -580,7 +580,7 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro return ver, errors.Trace(err) } - err = infosync.PutRuleBundles(context.TODO(), bundles) + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles) if err != nil { job.State = model.JobStateCancelled return 0, errors.Wrapf(err, "failed to notify PD the placement rules") @@ -1302,7 +1302,7 @@ func onAlterTablePartitionOptions(d *ddlCtx, t *meta.Meta, job *model.Job) (ver // Send the placement bundle to PD. if bundle != nil { - err = infosync.PutRuleBundles(context.TODO(), []*placement.Bundle{bundle}) + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), []*placement.Bundle{bundle}) } if err != nil { @@ -1353,7 +1353,7 @@ func onAlterTablePlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, // Send the placement bundle to PD. if bundle != nil { - err = infosync.PutRuleBundles(context.TODO(), []*placement.Bundle{bundle}) + err = infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), []*placement.Bundle{bundle}) } if err != nil { diff --git a/util/filesort/main_test.go b/domain/infosync/error.go similarity index 51% rename from util/filesort/main_test.go rename to domain/infosync/error.go index 6a3378a2187d7..53c1fedcd9d2b 100644 --- a/util/filesort/main_test.go +++ b/domain/infosync/error.go @@ -12,33 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -package filesort +package infosync import ( - "math/rand" - "testing" - - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testbridge" - "go.uber.org/goleak" + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/util/dbterror" ) -func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() - goleak.VerifyTestMain(m) -} - -func nextRow(r *rand.Rand, keySize int, valSize int) (key []types.Datum, val []types.Datum, handle int64) { - key = make([]types.Datum, keySize) - for i := range key { - key[i] = types.NewDatum(r.Int()) - } - - val = make([]types.Datum, valSize) - for j := range val { - val[j] = types.NewDatum(r.Int()) - } - - handle = r.Int63() - return -} +var ( + // ErrHTTPServiceError means we got a http response with a status code which is not '2xx' + ErrHTTPServiceError = dbterror.ClassDomain.NewStdErr( + errno.ErrHTTPServiceError, mysql.Message("HTTP request failed with status %s", nil), + ) +) diff --git a/domain/infosync/info.go b/domain/infosync/info.go index 29b71e8b9f276..fc58783ff3108 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -80,6 +80,10 @@ const ( TopologyPrometheus = "/topology/prometheus" // TablePrometheusCacheExpiry is the expiry time for prometheus address cache. TablePrometheusCacheExpiry = 10 * time.Second + // RequestRetryInterval is the sleep time before next retry for http request + RequestRetryInterval = 200 * time.Millisecond + // SyncBundlesMaxRetry is the max retry times for sync placement bundles + SyncBundlesMaxRetry = 3 ) // ErrPrometheusAddrIsNotSet is the error that Prometheus address is not set in PD and etcd @@ -349,10 +353,11 @@ func doRequest(ctx context.Context, addrs []string, route, method string, body i if err == nil { bodyBytes, err := io.ReadAll(res.Body) if err != nil { + terror.Log(res.Body.Close()) return nil, err } if res.StatusCode != http.StatusOK { - err = errors.Errorf("%s", bodyBytes) + err = ErrHTTPServiceError.FastGen("%s", bodyBytes) if res.StatusCode == http.StatusNotFound || res.StatusCode == http.StatusPreconditionFailed { err = nil bodyBytes = nil @@ -426,6 +431,16 @@ func GetRuleBundle(ctx context.Context, name string) (*placement.Bundle, error) // PutRuleBundles is used to post specific rule bundles to PD. func PutRuleBundles(ctx context.Context, bundles []*placement.Bundle) error { + failpoint.Inject("putRuleBundlesError", func(isServiceError failpoint.Value) { + var err error + if isServiceError.(bool) { + err = ErrHTTPServiceError.FastGen("mock service error") + } else { + err = errors.New("mock other error") + } + failpoint.Return(err) + }) + is, err := getGlobalInfoSyncer() if err != nil { return err @@ -434,6 +449,31 @@ func PutRuleBundles(ctx context.Context, bundles []*placement.Bundle) error { return is.placementManager.PutRuleBundles(ctx, bundles) } +// PutRuleBundlesWithRetry will retry for specified times when PutRuleBundles failed +func PutRuleBundlesWithRetry(ctx context.Context, bundles []*placement.Bundle, maxRetry int, interval time.Duration) (err error) { + if maxRetry < 0 { + maxRetry = 0 + } + + for i := 0; i <= maxRetry; i++ { + if err = PutRuleBundles(ctx, bundles); err == nil || ErrHTTPServiceError.Equal(err) { + return err + } + + if i != maxRetry { + logutil.BgLogger().Warn("Error occurs when PutRuleBundles, retry", zap.Error(err)) + time.Sleep(interval) + } + } + + return +} + +// PutRuleBundlesWithDefaultRetry will retry for default times +func PutRuleBundlesWithDefaultRetry(ctx context.Context, bundles []*placement.Bundle) (err error) { + return PutRuleBundlesWithRetry(ctx, bundles, SyncBundlesMaxRetry, RequestRetryInterval) +} + func (is *InfoSyncer) getAllServerInfo(ctx context.Context) (map[string]*ServerInfo, error) { allInfo := make(map[string]*ServerInfo) if is.etcdCli == nil { diff --git a/domain/infosync/info_test.go b/domain/infosync/info_test.go index e839b5daa8368..001a106632230 100644 --- a/domain/infosync/info_test.go +++ b/domain/infosync/info_test.go @@ -26,8 +26,10 @@ import ( "time" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/owner" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/testbridge" "github.com/stretchr/testify/require" "go.etcd.io/etcd/integration" @@ -145,3 +147,66 @@ func (is *InfoSyncer) ttlKeyExists(ctx context.Context) (bool, error) { } return len(resp.Kvs) == 1, nil } + +func TestPutBundlesRetry(t *testing.T) { + _, err := GlobalInfoSyncerInit(context.TODO(), "test", func() uint64 { return 1 }, nil, false) + require.NoError(t, err) + + bundle, err := placement.NewBundleFromOptions(&model.PlacementSettings{PrimaryRegion: "r1", Regions: "r1,r2"}) + require.NoError(t, err) + bundle = bundle.Reset(placement.RuleIndexTable, []int64{1024}) + + t.Run("serviceErrorShouldNotRetry", func(t *testing.T) { + require.NoError(t, PutRuleBundles(context.TODO(), []*placement.Bundle{{ID: bundle.ID}})) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/domain/infosync/putRuleBundlesError", "1*return(true)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/domain/infosync/putRuleBundlesError")) + }() + + err := PutRuleBundlesWithRetry(context.TODO(), []*placement.Bundle{bundle}, 3, time.Millisecond) + require.Error(t, err) + require.Equal(t, "[domain:8243]mock service error", err.Error()) + + got, err := GetRuleBundle(context.TODO(), bundle.ID) + require.NoError(t, err) + require.True(t, got.IsEmpty()) + }) + + t.Run("nonServiceErrorShouldRetry", func(t *testing.T) { + require.NoError(t, PutRuleBundles(context.TODO(), []*placement.Bundle{{ID: bundle.ID}})) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/domain/infosync/putRuleBundlesError", "3*return(false)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/domain/infosync/putRuleBundlesError")) + }() + + err := PutRuleBundlesWithRetry(context.TODO(), []*placement.Bundle{bundle}, 3, time.Millisecond) + require.NoError(t, err) + + got, err := GetRuleBundle(context.TODO(), bundle.ID) + require.NoError(t, err) + + gotJSON, err := json.Marshal(got) + require.NoError(t, err) + + expectJSON, err := json.Marshal(bundle) + require.NoError(t, err) + + require.Equal(t, expectJSON, gotJSON) + }) + + t.Run("nonServiceErrorRetryAndFail", func(t *testing.T) { + require.NoError(t, PutRuleBundles(context.TODO(), []*placement.Bundle{{ID: bundle.ID}})) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/domain/infosync/putRuleBundlesError", "4*return(false)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/domain/infosync/putRuleBundlesError")) + }() + + err := PutRuleBundlesWithRetry(context.TODO(), []*placement.Bundle{bundle}, 3, time.Millisecond) + require.Error(t, err) + require.Equal(t, "mock other error", err.Error()) + + got, err := GetRuleBundle(context.TODO(), bundle.ID) + require.NoError(t, err) + require.True(t, got.IsEmpty()) + }) +} diff --git a/dumpling/export/dump.go b/dumpling/export/dump.go index 8523460d648ad..95fd9bedc2596 100755 --- a/dumpling/export/dump.go +++ b/dumpling/export/dump.go @@ -454,25 +454,33 @@ func adjustTableCollation(tctx *tcontext.Context, parser *parser.Parser, originS return originSQL, nil } var charset string + var collation string for _, createOption := range createStmt.Options { // already have 'Collation' if createOption.Tp == ast.TableOptionCollate { - return originSQL, nil + collation = createOption.StrValue + break } 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 + if collation == "" && charset != "" { + // 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}) } - // add collation - createStmt.Options = append(createStmt.Options, &ast.TableOption{Tp: ast.TableOptionCollate, StrValue: collation}) + // adjust columns collation + adjustColumnsCollation(tctx, createStmt, charsetAndDefaultCollationMap) + // rewrite sql var b []byte bf := bytes.NewBuffer(b) @@ -486,6 +494,31 @@ func adjustTableCollation(tctx *tcontext.Context, parser *parser.Parser, originS return bf.String(), nil } +// adjustColumnsCollation adds column's collation. +func adjustColumnsCollation(tctx *tcontext.Context, createStmt *ast.CreateTableStmt, charsetAndDefaultCollationMap map[string]string) { + for _, col := range createStmt.Cols { + for _, options := range col.Options { + // already have 'Collation' + if options.Tp == ast.ColumnOptionCollate { + continue + } + } + fieldType := col.Tp + if fieldType.Collate != "" { + continue + } + if fieldType.Charset != "" { + // just have charset + collation, ok := charsetAndDefaultCollationMap[strings.ToLower(fieldType.Charset)] + if !ok { + tctx.L().Warn("not found charset default collation for column.", zap.String("table", createStmt.Table.Name.String()), zap.String("column", col.Name.String()), zap.String("charset", strings.ToLower(fieldType.Charset))) + continue + } + fieldType.Collate = collation + } + } +} + func (d *Dumper) dumpTableData(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta, taskChan chan<- Task) error { conf := d.conf if conf.NoData { @@ -1020,7 +1053,20 @@ func prepareTableListToDump(tctx *tcontext.Context, conf *Config, db *sql.Conn) if !conf.NoViews { tableTypes = append(tableTypes, TableTypeView) } - conf.Tables, err = ListAllDatabasesTables(tctx, db, databases, getListTableTypeByConf(conf), tableTypes...) + + 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) + } + + conf.Tables, err = ListAllDatabasesTables(tctx, db, databases, listType, tableTypes...) if err != nil { return err } diff --git a/dumpling/export/dump_test.go b/dumpling/export/dump_test.go index 82d1bf8f99705..505fd04d20a39 100644 --- a/dumpling/export/dump_test.go +++ b/dumpling/export/dump_test.go @@ -168,11 +168,27 @@ func TestAdjustTableCollation(t *testing.T) { originSQLs := []string{ "create table `test`.`t1` (id int) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci", "create table `test`.`t1` (id int) CHARSET=utf8mb4", + "create table `test`.`t1` (id int, name varchar(20) CHARACTER SET utf8mb4, work varchar(20)) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci ", + "create table `test`.`t1` (id int, name varchar(20), work varchar(20)) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci", + "create table `test`.`t1` (id int, name varchar(20) COLLATE utf8mb4_general_ci, work varchar(20)) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci", + "create table `test`.`t1` (id int, name varchar(20) COLLATE utf8mb4_general_ci, work varchar(20) CHARACTER SET utf8mb4) CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci", + "create table `test`.`t1` (id int, name varchar(20) CHARACTER SET utf8mb4, work varchar(20)) CHARSET=utf8mb4 ", + "create table `test`.`t1` (id int, name varchar(20), work varchar(20)) CHARSET=utf8mb4", + "create table `test`.`t1` (id int, name varchar(20) COLLATE utf8mb4_general_ci, work varchar(20)) CHARSET=utf8mb4", + "create table `test`.`t1` (id int, name varchar(20) COLLATE utf8mb4_general_ci, work varchar(20) CHARACTER SET utf8mb4) 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", + "CREATE TABLE `test`.`t1` (`id` INT) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI", + "CREATE TABLE `test`.`t1` (`id` INT,`name` VARCHAR(20) CHARACTER SET UTF8MB4 COLLATE utf8mb4_general_ci,`work` VARCHAR(20)) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI", + "CREATE TABLE `test`.`t1` (`id` INT,`name` VARCHAR(20),`work` VARCHAR(20)) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI", + "CREATE TABLE `test`.`t1` (`id` INT,`name` VARCHAR(20) COLLATE utf8mb4_general_ci,`work` VARCHAR(20)) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI", + "CREATE TABLE `test`.`t1` (`id` INT,`name` VARCHAR(20) COLLATE utf8mb4_general_ci,`work` VARCHAR(20) CHARACTER SET UTF8MB4 COLLATE utf8mb4_general_ci) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI", + "CREATE TABLE `test`.`t1` (`id` INT,`name` VARCHAR(20) CHARACTER SET UTF8MB4 COLLATE utf8mb4_general_ci,`work` VARCHAR(20)) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI", + "CREATE TABLE `test`.`t1` (`id` INT,`name` VARCHAR(20),`work` VARCHAR(20)) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI", + "CREATE TABLE `test`.`t1` (`id` INT,`name` VARCHAR(20) COLLATE utf8mb4_general_ci,`work` VARCHAR(20)) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI", + "CREATE TABLE `test`.`t1` (`id` INT,`name` VARCHAR(20) COLLATE utf8mb4_general_ci,`work` VARCHAR(20) CHARACTER SET UTF8MB4 COLLATE utf8mb4_general_ci) DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_GENERAL_CI", } charsetAndDefaultCollationMap := map[string]string{"utf8mb4": "utf8mb4_general_ci"} diff --git a/dumpling/export/prepare.go b/dumpling/export/prepare.go index fcc49cc6d7569..777b366c65a78 100644 --- a/dumpling/export/prepare.go +++ b/dumpling/export/prepare.go @@ -116,6 +116,9 @@ const ( TableTypeBase TableType = iota // TableTypeView represents the view table TableTypeView + // TableTypeSequence represents the view table + // TODO: need to be supported + TableTypeSequence ) const ( @@ -123,6 +126,8 @@ const ( TableTypeBaseStr = "BASE TABLE" // TableTypeViewStr represents the view table string TableTypeViewStr = "VIEW" + // TableTypeSequenceStr represents the view table string + TableTypeSequenceStr = "SEQUENCE" ) func (t TableType) String() string { @@ -131,6 +136,8 @@ func (t TableType) String() string { return TableTypeBaseStr case TableTypeView: return TableTypeViewStr + case TableTypeSequence: + return TableTypeSequenceStr default: return "UNKNOWN" } @@ -143,6 +150,8 @@ func ParseTableType(s string) (TableType, error) { return TableTypeBase, nil case TableTypeViewStr: return TableTypeView, nil + case TableTypeSequenceStr: + return TableTypeSequence, nil default: return TableTypeBase, errors.Errorf("unknown table type %s", s) } diff --git a/dumpling/export/sql.go b/dumpling/export/sql.go index aa05fa8d9c2b4..60e9d08589632 100644 --- a/dumpling/export/sql.go +++ b/dumpling/export/sql.go @@ -725,6 +725,19 @@ func CheckTiDBWithTiKV(db *sql.DB) (bool, error) { return count > 0, nil } +// CheckIfSeqExists use sql to check whether sequence exists +func CheckIfSeqExists(db *sql.Conn) (bool, error) { + var count int + const query = "SELECT COUNT(1) as c FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE='SEQUENCE'" + row := db.QueryRowContext(context.Background(), query) + err := row.Scan(&count) + if err != nil { + return false, errors.Annotatef(err, "sql: %s", query) + } + + return count > 0, nil +} + // CheckTiDBEnableTableLock use sql variable to check whether current TiDB has TiKV func CheckTiDBEnableTableLock(db *sql.Conn) (bool, error) { tidbConfig, err := getTiDBConfig(db) diff --git a/dumpling/export/sql_test.go b/dumpling/export/sql_test.go index 8606775e6b32c..44ba39ec01d8c 100644 --- a/dumpling/export/sql_test.go +++ b/dumpling/export/sql_test.go @@ -1747,6 +1747,35 @@ func TestPickupPossibleField(t *testing.T) { } } +func TestCheckIfSeqExists(t *testing.T) { + t.Parallel() + + 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) + + mock.ExpectQuery("SELECT COUNT"). + WillReturnRows(sqlmock.NewRows([]string{"c"}). + AddRow("1")) + + exists, err := CheckIfSeqExists(conn) + require.NoError(t, err) + require.Equal(t, true, exists) + + mock.ExpectQuery("SELECT COUNT"). + WillReturnRows(sqlmock.NewRows([]string{"c"}). + AddRow("0")) + + exists, err = CheckIfSeqExists(conn) + require.NoError(t, err) + require.Equal(t, false, exists) +} + func TestGetCharsetAndDefaultCollation(t *testing.T) { t.Parallel() db, mock, err := sqlmock.New() diff --git a/dumpling/tests/basic/run.sh b/dumpling/tests/basic/run.sh index d801970643b75..516fffeff2c30 100644 --- a/dumpling/tests/basic/run.sh +++ b/dumpling/tests/basic/run.sh @@ -88,6 +88,12 @@ actual=$(sed -n '2p' ${DUMPLING_OUTPUT_DIR}/result.000000000.csv) echo "expected 2, actual ${actual}" [ "$actual" = 2 ] +# Test for dump with sequence +run_dumpling | tee ${DUMPLING_OUTPUT_DIR}/dumpling.log +actual=$(grep -w "dump failed" ${DUMPLING_OUTPUT_DIR}/dumpling.log|wc -l) +echo "expected 0, actual ${actual}" +[ "$actual" = 0 ] + # Test for tidb_mem_quota_query configuration export GO_FAILPOINTS="github.com/pingcap/tidb/dumpling/export/PrintTiDBMemQuotaQuery=1*return" run_dumpling | tee ${DUMPLING_OUTPUT_DIR}/dumpling.log diff --git a/errno/errcode.go b/errno/errcode.go index e0c5f7cccb0a7..ea1fb86ea7e8c 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1055,6 +1055,7 @@ const ( ErrPlacementPolicyWithDirectOption = 8240 ErrPlacementPolicyInUse = 8241 ErrOptOnCacheTable = 8242 + ErrHTTPServiceError = 8243 // TiKV/PD/TiFlash errors. ErrPDServerTimeout = 9001 ErrTiKVServerTimeout = 9002 diff --git a/errors.toml b/errors.toml index 9d6c5886e8d97..996c701d95ede 100644 --- a/errors.toml +++ b/errors.toml @@ -741,6 +741,11 @@ error = ''' Information schema is changed during the execution of the statement(for example, table definition may be updated by other DDL ran in parallel). If you see this error often, try increasing `tidb_max_delta_schema_count`. [try again later] ''' +["domain:8243"] +error = ''' +HTTP request failed with status %s +''' + ["domain:9009"] error = ''' Prometheus address is not set in PD and etcd diff --git a/executor/adapter.go b/executor/adapter.go index b8610ffd10f60..87f87a9712516 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1022,13 +1022,12 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { if _, ok := a.StmtNode.(*ast.CommitStmt); ok { slowItems.PrevStmt = sessVars.PrevStmt.String() } + slowLog := sessVars.SlowLogFormat(slowItems) if trace.IsEnabled() { - trace.Log(a.GoCtx, "details", sessVars.SlowLogFormat(slowItems)) + trace.Log(a.GoCtx, "details", slowLog) } - if costTime < threshold { - logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(slowItems)) - } else { - logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(slowItems)) + logutil.SlowQueryLogger.Warn(slowLog) + if costTime >= threshold { if sessVars.InRestrictedSQL { totalQueryProcHistogramInternal.Observe(costTime.Seconds()) totalCopProcHistogramInternal.Observe(execDetail.TimeDetail.ProcessTime.Seconds()) diff --git a/executor/aggfuncs/func_group_concat.go b/executor/aggfuncs/func_group_concat.go index a7fb3b7359ca5..4cb3993be65ad 100644 --- a/executor/aggfuncs/func_group_concat.go +++ b/executor/aggfuncs/func_group_concat.go @@ -293,6 +293,7 @@ type topNRows struct { // ('---', 'ccc') should be poped from heap, so '-' should be appended to result. // eg: 'aaa---bbb---ccc' -> 'aaa---bbb-' isSepTruncated bool + collators []collate.Collator } func (h topNRows) Len() int { @@ -302,7 +303,7 @@ func (h topNRows) Len() int { func (h topNRows) Less(i, j int) bool { n := len(h.rows[i].byItems) for k := 0; k < n; k++ { - ret, err := h.rows[i].byItems[k].CompareDatum(h.sctx.GetSessionVars().StmtCtx, h.rows[j].byItems[k]) + ret, err := h.rows[i].byItems[k].Compare(h.sctx.GetSessionVars().StmtCtx, h.rows[j].byItems[k], h.collators[k]) if err != nil { h.err = err return false @@ -411,8 +412,10 @@ func (e *groupConcatOrder) AppendFinalResult2Chunk(sctx sessionctx.Context, pr P func (e *groupConcatOrder) AllocPartialResult() (pr PartialResult, memDelta int64) { desc := make([]bool, len(e.byItems)) + ctors := make([]collate.Collator, 0, len(e.byItems)) for i, byItem := range e.byItems { desc[i] = byItem.Desc + ctors = append(ctors, collate.GetCollator(byItem.Expr.GetType().Collate)) } p := &partialResult4GroupConcatOrder{ topN: &topNRows{ @@ -421,6 +424,7 @@ func (e *groupConcatOrder) AllocPartialResult() (pr PartialResult, memDelta int6 limitSize: e.maxLen, sepSize: uint64(len(e.sep)), isSepTruncated: false, + collators: ctors, }, } return PartialResult(p), DefPartialResult4GroupConcatOrderSize + DefTopNRowsSize @@ -513,8 +517,10 @@ func (e *groupConcatDistinctOrder) AppendFinalResult2Chunk(sctx sessionctx.Conte func (e *groupConcatDistinctOrder) AllocPartialResult() (pr PartialResult, memDelta int64) { desc := make([]bool, len(e.byItems)) + ctors := make([]collate.Collator, 0, len(e.byItems)) for i, byItem := range e.byItems { desc[i] = byItem.Desc + ctors = append(ctors, collate.GetCollator(byItem.Expr.GetType().Collate)) } valSet, setSize := set.NewStringSetWithMemoryUsage() p := &partialResult4GroupConcatOrderDistinct{ @@ -524,6 +530,7 @@ func (e *groupConcatDistinctOrder) AllocPartialResult() (pr PartialResult, memDe limitSize: e.maxLen, sepSize: uint64(len(e.sep)), isSepTruncated: false, + collators: ctors, }, valSet: valSet, } diff --git a/executor/builder.go b/executor/builder.go index 1912f17eb5458..f2a575d48560a 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/cteutil" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" @@ -282,15 +283,13 @@ func (b *executorBuilder) buildCancelDDLJobs(v *plannercore.CancelDDLJobs) Execu baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), jobIDs: v.JobIDs, } - txn, err := e.ctx.Txn(true) - if err != nil { - b.err = err - return nil - } - - e.errs, b.err = admin.CancelJobs(txn, e.jobIDs) - if b.err != nil { - return nil + // Run within a new transaction. If it runs within the session transaction, commit failure won't be reported to the user. + errInTxn := kv.RunInNewTxn(context.Background(), e.ctx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) (err error) { + e.errs, err = admin.CancelJobs(txn, e.jobIDs) + return + }) + if errInTxn != nil { + b.err = errInTxn } return e } @@ -1059,6 +1058,11 @@ func (b *executorBuilder) buildUnionScanFromReader(reader Executor, v *plannerco reader = sel.children[0] } + us.collators = make([]collate.Collator, 0, len(us.columns)) + for _, tp := range retTypes(us) { + us.collators = append(us.collators, collate.GetCollator(tp.Collate)) + } + switch x := reader.(type) { case *TableReaderExecutor: us.desc = x.desc @@ -1093,6 +1097,12 @@ func (b *executorBuilder) buildUnionScanFromReader(reader Executor, v *plannerco us.columns = x.columns us.table = x.table us.virtualColumnIndex = buildVirtualColumnIndex(us.Schema(), us.columns) + case *IndexMergeReaderExecutor: + // IndexMergeReader doesn't care order for now. So we will not set desc and useIndex. + us.conditions, us.conditionsWithVirCol = plannercore.SplitSelCondsWithVirtualColumn(v.Conditions) + us.columns = x.columns + us.table = x.table + us.virtualColumnIndex = buildVirtualColumnIndex(us.Schema(), us.columns) default: // The mem table will not be written by sql directly, so we can omit the union scan to avoid err reporting. return originReader @@ -2799,23 +2809,29 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) } innerKeyCols := make([]int, len(v.InnerJoinKeys)) innerKeyColIDs := make([]int64, len(v.InnerJoinKeys)) + keyCollators := make([]collate.Collator, 0, len(v.InnerJoinKeys)) for i := 0; i < len(v.InnerJoinKeys); i++ { innerKeyCols[i] = v.InnerJoinKeys[i].Index innerKeyColIDs[i] = v.InnerJoinKeys[i].ID + keyCollators = append(keyCollators, collate.GetCollator(v.InnerJoinKeys[i].RetType.Collate)) } e.outerCtx.keyCols = outerKeyCols e.innerCtx.keyCols = innerKeyCols e.innerCtx.keyColIDs = innerKeyColIDs + e.innerCtx.keyCollators = keyCollators outerHashCols, innerHashCols := make([]int, len(v.OuterHashKeys)), make([]int, len(v.InnerHashKeys)) + hashCollators := make([]collate.Collator, 0, len(v.InnerHashKeys)) for i := 0; i < len(v.OuterHashKeys); i++ { outerHashCols[i] = v.OuterHashKeys[i].Index } for i := 0; i < len(v.InnerHashKeys); i++ { innerHashCols[i] = v.InnerHashKeys[i].Index + hashCollators = append(hashCollators, collate.GetCollator(v.InnerHashKeys[i].RetType.Collate)) } e.outerCtx.hashCols = outerHashCols e.innerCtx.hashCols = innerHashCols + e.innerCtx.hashCollators = hashCollators e.joinResult = newFirstChunk(e) executorCounterIndexLookUpJoin.Inc() @@ -2867,8 +2883,10 @@ func (b *executorBuilder) buildIndexLookUpMergeJoin(v *plannercore.PhysicalIndex outerKeyCols[i] = v.OuterJoinKeys[i].Index } innerKeyCols := make([]int, len(v.InnerJoinKeys)) + keyCollators := make([]collate.Collator, 0, len(v.InnerJoinKeys)) for i := 0; i < len(v.InnerJoinKeys); i++ { innerKeyCols[i] = v.InnerJoinKeys[i].Index + keyCollators = append(keyCollators, collate.GetCollator(v.InnerJoinKeys[i].RetType.Collate)) } executorCounterIndexLookUpJoin.Inc() @@ -2887,6 +2905,7 @@ func (b *executorBuilder) buildIndexLookUpMergeJoin(v *plannercore.PhysicalIndex rowTypes: innerTypes, joinKeys: v.InnerJoinKeys, keyCols: innerKeyCols, + keyCollators: keyCollators, compareFuncs: v.CompareFuncs, colLens: v.IdxColLens, desc: v.Desc, diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 4753a33e62b15..eb3f344e0c3fb 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -1205,7 +1205,7 @@ func (s *testSuite6) TestMaxHandleAddIndex(c *C) { tk.MustExec("admin check table t1") } -func (s *testSuite6) TestSetDDLReorgWorkerCnt(c *C) { +func (s *testSerialSuite) TestSetDDLReorgWorkerCnt(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) @@ -1244,7 +1244,7 @@ func (s *testSuite6) TestSetDDLReorgWorkerCnt(c *C) { tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt").Check(testkit.Rows("256")) } -func (s *testSuite6) TestSetDDLReorgBatchSize(c *C) { +func (s *testSerialSuite) TestSetDDLReorgBatchSize(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) diff --git a/executor/executor_test.go b/executor/executor_test.go index c821901d5a18d..f503504d739b0 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -9487,3 +9487,17 @@ func (s *testSerialSuite) TestIssue28650(c *C) { }() } } + +func (s *testSerialSuite) TestIssue30289(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + fpName := "github.com/pingcap/tidb/executor/issue30289" + c.Assert(failpoint.Enable(fpName, `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable(fpName), IsNil) + }() + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + err := tk.QueryToErr("select /*+ hash_join(t1) */ * from t t1 join t t2 on t1.a=t2.a") + c.Assert(err.Error(), Matches, "issue30289 build return error") +} diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index a9eef1ccff25e..ff5d317e2bb5b 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" @@ -101,8 +102,10 @@ type innerCtx struct { rowTypes []*types.FieldType keyCols []int keyColIDs []int64 // the original ID in its table, used by dynamic partition pruning + keyCollators []collate.Collator hashTypes []*types.FieldType hashCols []int + hashCollators []collate.Collator colLens []int hasPrefixCol bool } @@ -629,7 +632,7 @@ func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, chkIdx, row } return nil, nil, err } - cmp, err := outerValue.CompareDatum(sc, &innerValue) + cmp, err := outerValue.Compare(sc, &innerValue, iw.hashCollators[i]) if err != nil { return nil, nil, err } @@ -651,7 +654,7 @@ func (iw *innerWorker) sortAndDedupLookUpContents(lookUpContents []*indexJoinLoo } sc := iw.ctx.GetSessionVars().StmtCtx sort.Slice(lookUpContents, func(i, j int) bool { - cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[j].keys) + cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[j].keys, iw.keyCollators) if cmp != 0 || iw.nextColCompareFilters == nil { return cmp < 0 } @@ -659,7 +662,7 @@ func (iw *innerWorker) sortAndDedupLookUpContents(lookUpContents []*indexJoinLoo }) deDupedLookupKeys := lookUpContents[:1] for i := 1; i < len(lookUpContents); i++ { - cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[i-1].keys) + cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[i-1].keys, iw.keyCollators) if cmp != 0 || (iw.nextColCompareFilters != nil && iw.nextColCompareFilters.CompareRow(lookUpContents[i].row, lookUpContents[i-1].row) != 0) { deDupedLookupKeys = append(deDupedLookupKeys, lookUpContents[i]) } @@ -667,9 +670,9 @@ func (iw *innerWorker) sortAndDedupLookUpContents(lookUpContents []*indexJoinLoo return deDupedLookupKeys } -func compareRow(sc *stmtctx.StatementContext, left, right []types.Datum) int { +func compareRow(sc *stmtctx.StatementContext, left, right []types.Datum, ctors []collate.Collator) int { for idx := 0; idx < len(left); idx++ { - cmp, err := left[idx].CompareDatum(sc, &right[idx]) + cmp, err := left[idx].Compare(sc, &right[idx], ctors[idx]) // We only compare rows with the same type, no error to return. terror.Log(err) if cmp > 0 { diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 746fc6a5733fc..7e4ac6a515ae9 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" @@ -88,6 +89,7 @@ type innerMergeCtx struct { rowTypes []*types.FieldType joinKeys []*expression.Column keyCols []int + keyCollators []collate.Collator compareFuncs []expression.CompareFunc colLens []int desc bool @@ -697,7 +699,7 @@ func (imw *innerMergeWorker) constructDatumLookupKey(task *lookUpMergeJoinTask, } return nil, err } - cmp, err := outerValue.CompareDatum(sc, &innerValue) + cmp, err := outerValue.Compare(sc, &innerValue, imw.keyCollators[i]) if err != nil { return nil, err } @@ -717,7 +719,7 @@ func (imw *innerMergeWorker) dedupDatumLookUpKeys(lookUpContents []*indexJoinLoo sc := imw.ctx.GetSessionVars().StmtCtx deDupedLookUpContents := lookUpContents[:1] for i := 1; i < len(lookUpContents); i++ { - cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[i-1].keys) + cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[i-1].keys, imw.keyCollators) if cmp != 0 || (imw.nextColCompareFilters != nil && imw.nextColCompareFilters.CompareRow(lookUpContents[i].row, lookUpContents[i-1].row) != 0) { deDupedLookUpContents = append(deDupedLookUpContents, lookUpContents[i]) } diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 60828bd514ac4..cd27a5049b8b5 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -133,6 +133,8 @@ func (e *IndexMergeReaderExecutor) Open(ctx context.Context) (err error) { } e.finished = make(chan struct{}) e.resultCh = make(chan *lookupTableTask, atomic.LoadInt32(&LookupTableTaskChannelSize)) + e.memTracker = memory.NewTracker(e.id, -1) + e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) return nil } @@ -520,7 +522,7 @@ func (e *IndexMergeReaderExecutor) startIndexMergeTableScanWorker(ctx context.Co finished: e.finished, indexMergeExec: e, tblPlans: e.tblPlans, - memTracker: memory.NewTracker(memory.LabelForSimpleTask, -1), + memTracker: e.memTracker, } ctx1, cancel := context.WithCancel(ctx) go func() { diff --git a/executor/index_merge_reader_test.go b/executor/index_merge_reader_test.go index 51620b861a94a..7fc2ac15e9473 100644 --- a/executor/index_merge_reader_test.go +++ b/executor/index_merge_reader_test.go @@ -17,6 +17,8 @@ package executor_test import ( "fmt" "math/rand" + "regexp" + "strconv" "strings" . "github.com/pingcap/check" @@ -170,3 +172,157 @@ func (s *testSuite1) TestPartitionTableRandomIndexMerge(c *C) { tk.MustQuery("select /*+ USE_INDEX_MERGE(tpk, a, b) */ * from tpk where " + cond).Sort().Check(result) } } + +func (s *testSuite1) TestIndexMergeInTransaction(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + + for i := 0; i < 2; i++ { + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int, c2 int, c3 int, pk int, key(c1), key(c2), key(c3), primary key(pk));") + if i == 1 { + tk.MustExec("set tx_isolation = 'READ-COMMITTED';") + } + tk.MustExec("begin;") + // Expect two IndexScan(c1, c2). + tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows( + "IndexMerge_9 1841.86 root ", + "├─IndexRangeScan_5(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo", + "├─IndexRangeScan_6(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", + "└─Selection_8(Probe) 1841.86 cop[tikv] lt(test.t1.c3, 10)", + " └─TableRowIDScan_7 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo")) + // Expect one IndexScan(c2) and one TableScan(pk). + tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows( + "IndexMerge_9 1106.67 root ", + "├─TableRangeScan_5(Build) 3333.33 cop[tikv] table:t1 range:[-inf,10), keep order:false, stats:pseudo", + "├─IndexRangeScan_6(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", + "└─Selection_8(Probe) 1106.67 cop[tikv] lt(test.t1.c3, 10)", + " └─TableRowIDScan_7 3330.01 cop[tikv] table:t1 keep order:false, stats:pseudo")) + + // Test with normal key. + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustExec("insert into t1 values(1, 1, 1, 1);") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows("1 1 1 1")) + tk.MustExec("update t1 set c3 = 100 where c3 = 1;") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustExec("delete from t1;") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) + + // Test with primary key, so the partialPlan is TableScan. + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustExec("insert into t1 values(1, 1, 1, 1);") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows("1 1 1 1")) + tk.MustExec("update t1 set c3 = 100 where c3 = 1;") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustExec("delete from t1;") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10;").Check(testkit.Rows()) + tk.MustExec("commit;") + if i == 1 { + tk.MustExec("set tx_isolation = 'REPEATABLE-READ';") + } + } + + // Same with above, but select ... for update. + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int, c2 int, c3 int, pk int, key(c1), key(c2), key(c3), primary key(pk));") + tk.MustExec("begin;") + tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows( + "SelectLock_6 1841.86 root for update 0", + "└─IndexMerge_11 1841.86 root ", + " ├─IndexRangeScan_7(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo", + " ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", + " └─Selection_10(Probe) 1841.86 cop[tikv] lt(test.t1.c3, 10)", + " └─TableRowIDScan_9 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo")) + tk.MustQuery("explain select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows( + "SelectLock_6 1106.67 root for update 0", + "└─IndexMerge_11 1106.67 root ", + " ├─TableRangeScan_7(Build) 3333.33 cop[tikv] table:t1 range:[-inf,10), keep order:false, stats:pseudo", + " ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", + " └─Selection_10(Probe) 1106.67 cop[tikv] lt(test.t1.c3, 10)", + " └─TableRowIDScan_9 3330.01 cop[tikv] table:t1 keep order:false, stats:pseudo")) + + // Test with normal key. + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows()) + tk.MustExec("insert into t1 values(1, 1, 1, 1);") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows("1 1 1 1")) + tk.MustExec("update t1 set c3 = 100 where c3 = 1;") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows()) + tk.MustExec("delete from t1;") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows()) + + // Test with primary key, so the partialPlan is TableScan. + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows()) + tk.MustExec("insert into t1 values(1, 1, 1, 1);") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows("1 1 1 1")) + tk.MustExec("update t1 set c3 = 100 where c3 = 1;") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows()) + tk.MustExec("delete from t1;") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 10 or c2 < 10) and c3 < 10 for update;").Check(testkit.Rows()) + tk.MustExec("commit;") + + // Test partition table. + tk.MustExec("drop table if exists t1;") + tk.MustExec(`create table t1(c1 int, c2 int, c3 int, pk int, part int, key(c1), key(c2), key(c3), primary key(pk, part)) + partition by range(part) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than (maxvalue))`) + tk.MustExec("begin;") + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 20 or c2 < 20) and c3 < 20;").Check(testkit.Rows()) + + tk.MustExec("insert into t1 values(1, 1, 1, 1, 1);") + tk.MustExec("insert into t1 values(11, 11, 11, 11, 11);") + tk.MustExec("insert into t1 values(21, 21, 21, 21, 21);") + tk.MustExec("insert into t1 values(31, 31, 31, 31, 31);") + res := tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 20 or c2 < 20) and c3 < 20;").Sort() + res.Check(testkit.Rows("1 1 1 1 1", "11 11 11 11 11")) + res = tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 20 or c2 < 20) and c3 < 20;").Sort() + res.Check(testkit.Rows("1 1 1 1 1", "11 11 11 11 11")) + + tk.MustExec("update t1 set c3 = 100 where c3 = 1;") + res = tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 20 or c2 < 20) and c3 < 20;") + res.Check(testkit.Rows("11 11 11 11 11")) + res = tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 20 or c2 < 20) and c3 < 20;") + res.Check(testkit.Rows("11 11 11 11 11")) + + tk.MustExec("delete from t1;") + res = tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (c1 < 20 or c2 < 20) and c3 < 20;") + res.Check(testkit.Rows()) + res = tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where (pk < 20 or c2 < 20) and c3 < 20;") + res.Check(testkit.Rows()) + tk.MustExec("commit;") +} + +func (test *testSerialSuite2) TestIndexMergeReaderMemTracker(c *C) { + tk := testkit.NewTestKit(c, test.store) + tk.MustExec("use test;") + tk.MustExec("create table t1(c1 int, c2 int, c3 int, key(c1), key(c2), key(c3));") + + insertStr := "insert into t1 values(0, 0, 0)" + rowNum := 1000 + for i := 0; i < rowNum; i++ { + insertStr += fmt.Sprintf(" ,(%d, %d, %d)", i, i, i) + } + insertStr += ";" + memTracker := tk.Se.GetSessionVars().StmtCtx.MemTracker + + tk.MustExec(insertStr) + + oriMaxUsage := memTracker.MaxConsumed() + + // We select all rows in t1, so the mem usage is more clear. + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where c1 > 1 or c2 > 1") + + newMaxUsage := memTracker.MaxConsumed() + c.Assert(newMaxUsage, Greater, oriMaxUsage) + + res := tk.MustQuery("explain analyze select /*+ use_index_merge(t1) */ * from t1 where c1 > 1 or c2 > 1") + c.Assert(len(res.Rows()), Equals, 4) + // Parse "xxx KB" and check it's greater than 0. + memStr := res.Rows()[0][7].(string) + re, err := regexp.Compile("[0-9]+ KB") + c.Assert(err, IsNil) + c.Assert(re.MatchString(memStr), IsTrue) + bytes, err := strconv.ParseFloat(memStr[:len(memStr)-3], 32) + c.Assert(err, IsNil) + c.Assert(bytes, Greater, 0.0) +} diff --git a/executor/join.go b/executor/join.go index 2b97c0b1f93cf..c28d1e107932e 100644 --- a/executor/join.go +++ b/executor/join.go @@ -214,9 +214,13 @@ func (e *HashJoinExec) fetchProbeSideChunks(ctx context.Context) { return } if !hasWaitedForBuild { + failpoint.Inject("issue30289", func(val failpoint.Value) { + if val.(bool) { + probeSideResult.Reset() + } + }) if probeSideResult.NumRows() == 0 && !e.useOuterToBuild { e.finished.Store(true) - return } emptyBuild, buildErr := e.wait4BuildSide() if buildErr != nil { @@ -258,6 +262,13 @@ func (e *HashJoinExec) wait4BuildSide() (emptyBuild bool, err error) { func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chunk.Chunk, doneCh <-chan struct{}) { defer close(chkCh) var err error + failpoint.Inject("issue30289", func(val failpoint.Value) { + if val.(bool) { + err = errors.Errorf("issue30289 build return error") + e.buildFinished <- errors.Trace(err) + return + } + }) for { if e.finished.Load().(bool) { return diff --git a/executor/mem_reader.go b/executor/mem_reader.go index 08509832c0ca2..d345d11670c6e 100644 --- a/executor/mem_reader.go +++ b/executor/mem_reader.go @@ -33,6 +33,18 @@ import ( "github.com/pingcap/tidb/util/rowcodec" ) +type memReader interface { + getMemRows() ([][]types.Datum, error) + getMemRowsHandle() ([]kv.Handle, error) +} + +var ( + _ memReader = &memIndexReader{} + _ memReader = &memTableReader{} + _ memReader = &memIndexLookUpReader{} + _ memReader = &memIndexMergeReader{} +) + type memIndexReader struct { ctx sessionctx.Context index *model.IndexInfo @@ -155,6 +167,8 @@ type memTableReader struct { buffer allocBuf pkColIDs []int64 cacheTable kv.MemBuffer + // Used when extracting handles from row in memTableReader.getMemRowsHandle. + handleCols plannercore.HandleCols } type allocBuf struct { @@ -313,6 +327,23 @@ func (m *memTableReader) getRowData(handle kv.Handle, value []byte) ([][]byte, e return values, nil } +// getMemRowsHandle is called when memIndexMergeReader.partialPlans[i] is TableScan. +func (m *memTableReader) getMemRowsHandle() ([]kv.Handle, error) { + rows, err := m.getMemRows() + if err != nil { + return nil, err + } + handles := make([]kv.Handle, 0, len(rows)) + for _, row := range rows { + handle, err := m.handleCols.BuildHandleByDatums(row) + if err != nil { + return nil, err + } + handles = append(handles, handle) + } + return handles, nil +} + func hasColVal(data [][]byte, colIDs map[int64]int, id int64) bool { offset, ok := colIDs[id] if ok && data[offset] != nil { @@ -486,26 +517,131 @@ func (m *memIndexLookUpReader) getMemRows() ([][]types.Datum, error) { return nil, nil } - colIDs := make(map[int64]int, len(m.columns)) - for i, col := range m.columns { - colIDs[col.ID] = i + colIDs, pkColIDs, rd := getColIDAndPkColIDs(m.table, m.columns) + memTblReader := &memTableReader{ + ctx: m.ctx, + table: m.table.Meta(), + columns: m.columns, + kvRanges: tblKVRanges, + conditions: m.conditions, + addedRows: make([][]types.Datum, 0, numHandles), + retFieldTypes: m.retFieldTypes, + colIDs: colIDs, + pkColIDs: pkColIDs, + buffer: allocBuf{ + handleBytes: make([]byte, 0, 16), + rd: rd, + }, + cacheTable: m.cacheTable, } - tblInfo := m.table.Meta() - colInfos := make([]rowcodec.ColInfo, 0, len(m.columns)) - for i := range m.columns { - col := m.columns[i] - colInfos = append(colInfos, rowcodec.ColInfo{ - ID: col.ID, - IsPKHandle: tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.Flag), - Ft: rowcodec.FieldTypeFromModelColumn(col), - }) + return memTblReader.getMemRows() +} + +func (m *memIndexLookUpReader) getMemRowsHandle() ([]kv.Handle, error) { + return nil, errors.New("getMemRowsHandle has not been implemented for memIndexLookUpReader") +} + +type memIndexMergeReader struct { + ctx sessionctx.Context + columns []*model.ColumnInfo + table table.Table + conditions []expression.Expression + retFieldTypes []*types.FieldType + indexMergeReader *IndexMergeReaderExecutor + memReaders []memReader + + // partition mode + partitionMode bool // if it is accessing a partition table + partitionTables []table.PhysicalTable // partition tables to access + partitionKVRanges [][][]kv.KeyRange // kv ranges for these partition tables +} + +func buildMemIndexMergeReader(us *UnionScanExec, indexMergeReader *IndexMergeReaderExecutor) *memIndexMergeReader { + indexCount := len(indexMergeReader.indexes) + memReaders := make([]memReader, 0, indexCount) + for i := 0; i < indexCount; i++ { + if indexMergeReader.indexes[i] == nil { + colIDs, pkColIDs, rd := getColIDAndPkColIDs(indexMergeReader.table, indexMergeReader.columns) + memReaders = append(memReaders, &memTableReader{ + ctx: us.ctx, + table: indexMergeReader.table.Meta(), + columns: indexMergeReader.columns, + kvRanges: nil, + conditions: us.conditions, + addedRows: make([][]types.Datum, 0), + retFieldTypes: retTypes(us), + colIDs: colIDs, + pkColIDs: pkColIDs, + buffer: allocBuf{ + handleBytes: make([]byte, 0, 16), + rd: rd, + }, + handleCols: indexMergeReader.handleCols, + }) + } else { + outputOffset := []int{len(indexMergeReader.indexes[i].Columns)} + memReaders = append(memReaders, &memIndexReader{ + ctx: us.ctx, + index: indexMergeReader.indexes[i], + table: indexMergeReader.table.Meta(), + kvRanges: nil, + desc: indexMergeReader.descs[i], + retFieldTypes: retTypes(us), + outputOffset: outputOffset, + belowHandleCols: us.belowHandleCols, + }) + } } - pkColIDs := tables.TryGetCommonPkColumnIds(tblInfo) - if len(pkColIDs) == 0 { - pkColIDs = []int64{-1} + + return &memIndexMergeReader{ + ctx: us.ctx, + table: indexMergeReader.table, + columns: indexMergeReader.columns, + conditions: us.conditions, + retFieldTypes: retTypes(us), + indexMergeReader: indexMergeReader, + memReaders: memReaders, + + partitionMode: indexMergeReader.partitionTableMode, + partitionTables: indexMergeReader.prunedPartitions, + partitionKVRanges: indexMergeReader.partitionKeyRanges, } - rd := rowcodec.NewByteDecoder(colInfos, pkColIDs, nil, nil) +} + +func (m *memIndexMergeReader) getMemRows() ([][]types.Datum, error) { + tbls := []table.Table{m.table} + // [partNum][indexNum][rangeNum] + var kvRanges [][][]kv.KeyRange + if m.partitionMode { + tbls = tbls[:0] + for _, p := range m.partitionTables { + tbls = append(tbls, p) + } + kvRanges = m.partitionKVRanges + } else { + kvRanges = append(kvRanges, m.indexMergeReader.keyRanges) + } + + tblKVRanges := make([]kv.KeyRange, 0, 16) + numHandles := 0 + for i, tbl := range tbls { + handles, err := m.unionHandles(kvRanges[i]) + if err != nil { + return nil, err + } + if len(handles) == 0 { + continue + } + numHandles += len(handles) + tblKVRanges = append(tblKVRanges, distsql.TableHandlesToKVRanges(getPhysicalTableID(tbl), handles)...) + } + + if numHandles == 0 { + return nil, nil + } + colIDs, pkColIDs, rd := getColIDAndPkColIDs(m.table, m.columns) + memTblReader := &memTableReader{ ctx: m.ctx, table: m.table.Meta(), @@ -520,8 +656,66 @@ func (m *memIndexLookUpReader) getMemRows() ([][]types.Datum, error) { handleBytes: make([]byte, 0, 16), rd: rd, }, - cacheTable: m.cacheTable, } return memTblReader.getMemRows() } + +// Union all handles of different Indexes. +func (m *memIndexMergeReader) unionHandles(kvRanges [][]kv.KeyRange) (finalHandles []kv.Handle, err error) { + if len(m.memReaders) != len(kvRanges) { + return nil, errors.Errorf("len(kvRanges) should be equal to len(memReaders)") + } + + hMap := kv.NewHandleMap() + var handles []kv.Handle + for i, reader := range m.memReaders { + switch r := reader.(type) { + case *memTableReader: + r.kvRanges = kvRanges[i] + case *memIndexReader: + r.kvRanges = kvRanges[i] + default: + return nil, errors.New("memReader have to be memTableReader or memIndexReader") + } + if handles, err = reader.getMemRowsHandle(); err != nil { + return nil, err + } + // Filter same row. + for _, h := range handles { + if _, ok := hMap.Get(h); !ok { + finalHandles = append(finalHandles, h) + hMap.Set(h, true) + } + } + } + return finalHandles, nil +} + +func (m *memIndexMergeReader) getMemRowsHandle() ([]kv.Handle, error) { + return nil, errors.New("getMemRowsHandle has not been implemented for memIndexMergeReader") +} + +func getColIDAndPkColIDs(table table.Table, columns []*model.ColumnInfo) (map[int64]int, []int64, *rowcodec.BytesDecoder) { + colIDs := make(map[int64]int, len(columns)) + for i, col := range columns { + colIDs[col.ID] = i + } + + tblInfo := table.Meta() + colInfos := make([]rowcodec.ColInfo, 0, len(columns)) + for i := range columns { + col := columns[i] + colInfos = append(colInfos, rowcodec.ColInfo{ + ID: col.ID, + IsPKHandle: tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.Flag), + Ft: rowcodec.FieldTypeFromModelColumn(col), + }) + } + pkColIDs := tables.TryGetCommonPkColumnIds(tblInfo) + if len(pkColIDs) == 0 { + pkColIDs = []int64{-1} + } + rd := rowcodec.NewByteDecoder(colInfos, pkColIDs, nil, nil) + return colIDs, pkColIDs, rd +} diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index e742e3a057e79..25c52e8c4954f 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -21,6 +21,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/variable" @@ -413,7 +414,7 @@ func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { // select a from t where a={x}; // the result is {x} x := rand.Intn(100) + 1 queryHash := fmt.Sprintf("select a from thash where a=%v", x) - queryRegular := fmt.Sprintf("select a from thash where a=%v", x) + queryRegular := fmt.Sprintf("select a from tregular where a=%v", x) c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used tk.MustQuery(queryHash).Check(tk.MustQuery(queryRegular).Rows()) } @@ -1726,6 +1727,36 @@ func (s *partitionTableSuite) TestAddDropPartitions(c *C) { tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) } +func (s *partitionTableSuite) TestMPPQueryExplainInfo(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database tiflash_partition_test") + tk.MustExec("use tiflash_partition_test") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table t(a int) partition by range(a) ( + partition p0 values less than (5), + partition p1 values less than (10), + partition p2 values less than (15))`) + tb := testGetTableByName(c, tk.Se, "tiflash_partition_test", "t") + for _, partition := range tb.Meta().GetPartitionInfo().Definitions { + err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.ID, true) + c.Assert(err, IsNil) + } + err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + tk.MustExec(`insert into t values (2), (7), (12)`) + tk.MustExec("set tidb_enforce_mpp=1") + tk.MustPartition(`select * from t where a < 3`, "p0").Sort().Check(testkit.Rows("2")) + tk.MustPartition(`select * from t where a < 8`, "p0,p1").Sort().Check(testkit.Rows("2", "7")) + tk.MustPartition(`select * from t where a < 20`, "all").Sort().Check(testkit.Rows("12", "2", "7")) + tk.MustPartition(`select * from t where a < 5 union all select * from t where a > 10`, "p0").Sort().Check(testkit.Rows("12", "2")) + tk.MustPartition(`select * from t where a < 5 union all select * from t where a > 10`, "p2").Sort().Check(testkit.Rows("12", "2")) +} + func (s *partitionTableSuite) PartitionPruningInTransaction(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") diff --git a/executor/pipelined_window.go b/executor/pipelined_window.go index 8e120376402be..7e8f6a0ed5c5e 100644 --- a/executor/pipelined_window.go +++ b/executor/pipelined_window.go @@ -179,7 +179,7 @@ func (e *PipelinedWindowExec) getRowsInPartition(ctx context.Context) (err error var drained, samePartition bool drained, err = e.fetchChild(ctx) if err != nil { - err = errors.Trace(err) + return errors.Trace(err) } // we return immediately to use a combination of true newPartition but 0 in e.rowToConsume to indicate the data source is drained, if drained { diff --git a/executor/show_test.go b/executor/show_test.go index 7373e1b67910e..34db4d54ae962 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1394,7 +1394,7 @@ func (s *testSuite5) TestShowBuiltin(c *C) { res := tk.MustQuery("show builtins;") c.Assert(res, NotNil) rows := res.Rows() - const builtinFuncNum = 273 + const builtinFuncNum = 274 c.Assert(builtinFuncNum, Equals, len(rows)) c.Assert("abs", Equals, rows[0][0].(string)) c.Assert("yearweek", Equals, rows[builtinFuncNum-1][0].(string)) diff --git a/executor/shuffle.go b/executor/shuffle.go index a71e388c02bac..9143afd032f10 100644 --- a/executor/shuffle.go +++ b/executor/shuffle.go @@ -142,17 +142,29 @@ func (e *ShuffleExec) Close() error { if !e.prepared { for _, w := range e.workers { for _, r := range w.receivers { - close(r.inputHolderCh) - close(r.inputCh) + if r.inputHolderCh != nil { + close(r.inputHolderCh) + } + if r.inputCh != nil { + close(r.inputCh) + } } - close(w.outputHolderCh) + if w.outputHolderCh != nil { + close(w.outputHolderCh) + } + } + if e.outputCh != nil { + close(e.outputCh) } - close(e.outputCh) } - close(e.finishCh) + if e.finishCh != nil { + close(e.finishCh) + } for _, w := range e.workers { for _, r := range w.receivers { - for range r.inputCh { + if r.inputCh != nil { + for range r.inputCh { + } } } // close child executor of each worker @@ -160,7 +172,9 @@ func (e *ShuffleExec) Close() error { firstErr = err } } - for range e.outputCh { // workers exit before `e.outputCh` is closed. + if e.outputCh != nil { + for range e.outputCh { // workers exit before `e.outputCh` is closed. + } } e.executed = false diff --git a/executor/union_scan.go b/executor/union_scan.go index c796d36bb6d31..86b696a8ee988 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" ) // UnionScanExec merges the rows from dirty table and the rows from distsql request. @@ -59,6 +60,7 @@ type UnionScanExec struct { // cacheTable not nil means it's reading from cached table. cacheTable kv.MemBuffer + collators []collate.Collator } // Open implements the Executor Open interface. @@ -101,6 +103,8 @@ func (us *UnionScanExec) open(ctx context.Context) error { us.addedRows, err = buildMemIndexReader(us, x).getMemRows() case *IndexLookUpExecutor: us.addedRows, err = buildMemIndexLookUpReader(us, x).getMemRows() + case *IndexMergeReaderExecutor: + us.addedRows, err = buildMemIndexMergeReader(us, x).getMemRows() default: err = fmt.Errorf("unexpected union scan children:%T", reader) } @@ -273,7 +277,7 @@ func (us *UnionScanExec) compare(a, b []types.Datum) (int, error) { for _, colOff := range us.usedIndex { aColumn := a[colOff] bColumn := b[colOff] - cmp, err := aColumn.CompareDatum(sc, &bColumn) + cmp, err := aColumn.Compare(sc, &bColumn, us.collators[colOff]) if err != nil { return 0, err } @@ -281,5 +285,5 @@ func (us *UnionScanExec) compare(a, b []types.Datum) (int, error) { return cmp, nil } } - return us.belowHandleCols.Compare(a, b) + return us.belowHandleCols.Compare(a, b, us.collators) } diff --git a/expression/aggregation/aggregation.go b/expression/aggregation/aggregation.go index 3a52e6719f087..84380552d7f71 100644 --- a/expression/aggregation/aggregation.go +++ b/expression/aggregation/aggregation.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tipb/go-tipb" ) @@ -68,9 +69,9 @@ func NewDistAggFunc(expr *tipb.Expr, fieldTps []*types.FieldType, sc *stmtctx.St case tipb.ExprType_GroupConcat: return &concatFunction{aggFunction: newAggFunc(ast.AggFuncGroupConcat, args, false)}, nil case tipb.ExprType_Max: - return &maxMinFunction{aggFunction: newAggFunc(ast.AggFuncMax, args, false), isMax: true}, nil + return &maxMinFunction{aggFunction: newAggFunc(ast.AggFuncMax, args, false), isMax: true, ctor: collate.GetCollator(args[0].GetType().Collate)}, nil case tipb.ExprType_Min: - return &maxMinFunction{aggFunction: newAggFunc(ast.AggFuncMin, args, false)}, nil + return &maxMinFunction{aggFunction: newAggFunc(ast.AggFuncMin, args, false), ctor: collate.GetCollator(args[0].GetType().Collate)}, nil case tipb.ExprType_First: return &firstRowFunction{aggFunction: newAggFunc(ast.AggFuncFirstRow, args, false)}, nil case tipb.ExprType_Agg_BitOr: diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index 1d5381f6c973d..30f020e7dfdf2 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" ) // AggFuncDesc describes an aggregation function signature, only used in planner. @@ -230,9 +231,9 @@ func (a *AggFuncDesc) GetAggFunc(ctx sessionctx.Context) Aggregation { } return &concatFunction{aggFunction: aggFunc, maxLen: maxLen} case ast.AggFuncMax: - return &maxMinFunction{aggFunction: aggFunc, isMax: true} + return &maxMinFunction{aggFunction: aggFunc, isMax: true, ctor: collate.GetCollator(a.Args[0].GetType().Collate)} case ast.AggFuncMin: - return &maxMinFunction{aggFunction: aggFunc, isMax: false} + return &maxMinFunction{aggFunction: aggFunc, isMax: false, ctor: collate.GetCollator(a.Args[0].GetType().Collate)} case ast.AggFuncFirstRow: return &firstRowFunction{aggFunction: aggFunc} case ast.AggFuncBitOr: diff --git a/expression/aggregation/max_min.go b/expression/aggregation/max_min.go index be25c7160a188..10f312d275023 100644 --- a/expression/aggregation/max_min.go +++ b/expression/aggregation/max_min.go @@ -18,11 +18,13 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" ) type maxMinFunction struct { aggFunction isMax bool + ctor collate.Collator } // GetResult implements Aggregation interface. @@ -49,7 +51,7 @@ func (mmf *maxMinFunction) Update(evalCtx *AggEvaluateContext, sc *stmtctx.State return nil } var c int - c, err = evalCtx.Value.CompareDatum(sc, &value) + c, err = evalCtx.Value.Compare(sc, &value, mmf.ctor) if err != nil { return err } diff --git a/expression/bench_test.go b/expression/bench_test.go index 70f8c8cf1b242..6dcc68b8486f4 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -1118,8 +1118,6 @@ func genVecExprBenchCase(ctx sessionctx.Context, funcName string, testCase vecEx // testVectorizedEvalOneVec is used to verify that the vectorized // expression is evaluated correctly during projection func testVectorizedEvalOneVec(t *testing.T, vecExprCases vecExprBenchCases) { - t.Parallel() - ctx := mock.NewContext() for funcName, testCases := range vecExprCases { for _, testCase := range testCases { @@ -1321,8 +1319,6 @@ func removeTestOptions(args []string) []string { // testVectorizedBuiltinFunc is used to verify that the vectorized // expression is evaluated correctly func testVectorizedBuiltinFunc(t *testing.T, vecExprCases vecExprBenchCases) { - t.Parallel() - testFunc := make(map[string]bool) argList := removeTestOptions(flag.Args()) testAll := len(argList) == 0 diff --git a/expression/builtin.go b/expression/builtin.go index 316e3a2ecb462..1b67239613498 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -772,6 +772,7 @@ var funcs = map[string]functionClass{ ast.IsIPv4Mapped: &isIPv4MappedFunctionClass{baseFunctionClass{ast.IsIPv4Mapped, 1, 1}}, ast.IsIPv6: &isIPv6FunctionClass{baseFunctionClass{ast.IsIPv6, 1, 1}}, ast.IsUsedLock: &isUsedLockFunctionClass{baseFunctionClass{ast.IsUsedLock, 1, 1}}, + ast.IsUUID: &isUUIDFunctionClass{baseFunctionClass{ast.IsUUID, 1, 1}}, ast.MasterPosWait: &masterPosWaitFunctionClass{baseFunctionClass{ast.MasterPosWait, 2, 4}}, ast.NameConst: &nameConstFunctionClass{baseFunctionClass{ast.NameConst, 2, 2}}, ast.ReleaseAllLocks: &releaseAllLocksFunctionClass{baseFunctionClass{ast.ReleaseAllLocks, 0, 0}}, diff --git a/expression/builtin_info.go b/expression/builtin_info.go index 66c166c0d785f..013ee74d66bac 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -190,7 +190,7 @@ func (b *builtinCurrentUserSig) evalString(row chunk.Row) (string, bool, error) if data == nil || data.User == nil { return "", true, errors.Errorf("Missing session variable when eval builtin") } - return data.User.AuthIdentityString(), false, nil + return data.User.String(), false, nil } type currentRoleFunctionClass struct { @@ -278,8 +278,7 @@ func (b *builtinUserSig) evalString(row chunk.Row) (string, bool, error) { if data == nil || data.User == nil { return "", true, errors.Errorf("Missing session variable when eval builtin") } - - return data.User.String(), false, nil + return data.User.LoginString(), false, nil } type connectionIDFunctionClass struct { diff --git a/expression/builtin_info_vec.go b/expression/builtin_info_vec.go index 20c6af4de9e4b..c4343f669e60b 100644 --- a/expression/builtin_info_vec.go +++ b/expression/builtin_info_vec.go @@ -114,7 +114,7 @@ func (b *builtinCurrentUserSig) vecEvalString(input *chunk.Chunk, result *chunk. return errors.Errorf("Missing session variable when eval builtin") } for i := 0; i < n; i++ { - result.AppendString(data.User.AuthIdentityString()) + result.AppendString(data.User.String()) } return nil } @@ -168,7 +168,7 @@ func (b *builtinUserSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) result.ReserveString(n) for i := 0; i < n; i++ { - result.AppendString(data.User.String()) + result.AppendString(data.User.LoginString()) } return nil } diff --git a/expression/builtin_miscellaneous.go b/expression/builtin_miscellaneous.go index ca9803ea97b9f..82b43d192c84c 100644 --- a/expression/builtin_miscellaneous.go +++ b/expression/builtin_miscellaneous.go @@ -57,6 +57,7 @@ var ( _ functionClass = &vitessHashFunctionClass{} _ functionClass = &uuidToBinFunctionClass{} _ functionClass = &binToUUIDFunctionClass{} + _ functionClass = &isUUIDFunctionClass{} ) var ( @@ -78,6 +79,7 @@ var ( _ builtinFunc = &builtinIsIPv4CompatSig{} _ builtinFunc = &builtinIsIPv4MappedSig{} _ builtinFunc = &builtinIsIPv6Sig{} + _ builtinFunc = &builtinIsUUIDSig{} _ builtinFunc = &builtinUUIDSig{} _ builtinFunc = &builtinVitessHashSig{} _ builtinFunc = &builtinUUIDToBinSig{} @@ -862,6 +864,47 @@ func (c *isUsedLockFunctionClass) getFunction(ctx sessionctx.Context, args []Exp return nil, errFunctionNotExists.GenWithStackByArgs("FUNCTION", "IS_USED_LOCK") } +type isUUIDFunctionClass struct { + baseFunctionClass +} + +func (c *isUUIDFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { + if err := c.verifyArgs(args); err != nil { + return nil, err + } + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } + bf.tp.Flen = 1 + sig := &builtinIsUUIDSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_IsUUID) + return sig, nil +} + +type builtinIsUUIDSig struct { + baseBuiltinFunc +} + +func (b *builtinIsUUIDSig) Clone() builtinFunc { + newSig := &builtinIsUUIDSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +// evalInt evals a builtinIsUUIDSig. +// See https://dev.mysql.com/doc/refman/8.0/en/miscellaneous-functions.html#function_is-uuid +func (b *builtinIsUUIDSig) evalInt(row chunk.Row) (int64, bool, error) { + val, isNull, err := b.args[0].EvalString(b.ctx, row) + if err != nil || isNull { + return 0, isNull, err + } + if _, err = uuid.Parse(val); err != nil { + return 0, false, nil + } + return 1, false, nil +} + type masterPosWaitFunctionClass struct { baseFunctionClass } diff --git a/expression/builtin_miscellaneous_test.go b/expression/builtin_miscellaneous_test.go index 9a1c7fd45e689..4de7136b29d14 100644 --- a/expression/builtin_miscellaneous_test.go +++ b/expression/builtin_miscellaneous_test.go @@ -97,6 +97,42 @@ func TestIsIPv4(t *testing.T) { trequire.DatumEqual(t, types.NewDatum(0), r) } +func TestIsUUID(t *testing.T) { + t.Parallel() + ctx := createContext(t) + tests := []struct { + uuid string + expect interface{} + }{ + {"6ccd780c-baba-1026-9564-5b8c656024db", 1}, + {"6CCD780C-BABA-1026-9564-5B8C656024DB", 1}, + {"6ccd780cbaba102695645b8c656024db", 1}, + {"{6ccd780c-baba-1026-9564-5b8c656024db}", 1}, + {"6ccd780c-baba-1026-9564-5b8c6560", 0}, + {"6CCD780C-BABA-1026-9564-5B8C656024DQ", 0}, + // This is a bug in google/uuid#60 + {"{99a9ad03-5298-11ec-8f5c-00ff90147ac3*", 1}, + // This is a format google/uuid support, while mysql doesn't + {"urn:uuid:99a9ad03-5298-11ec-8f5c-00ff90147ac3", 1}, + } + + fc := funcs[ast.IsUUID] + for _, test := range tests { + uuid := types.NewStringDatum(test.uuid) + f, err := fc.getFunction(ctx, datumsToConstants([]types.Datum{uuid})) + require.NoError(t, err) + result, err := evalBuiltinFunc(f, chunk.Row{}) + require.NoError(t, err) + trequire.DatumEqual(t, types.NewDatum(test.expect), result) + } + + var argNull types.Datum + f, _ := fc.getFunction(ctx, datumsToConstants([]types.Datum{argNull})) + r, err := evalBuiltinFunc(f, chunk.Row{}) + require.NoError(t, err) + require.True(t, r.IsNull()) +} + func TestUUID(t *testing.T) { t.Parallel() ctx := createContext(t) diff --git a/expression/builtin_miscellaneous_vec.go b/expression/builtin_miscellaneous_vec.go index 4ffad69ad8660..6e4b16c4cb7dd 100644 --- a/expression/builtin_miscellaneous_vec.go +++ b/expression/builtin_miscellaneous_vec.go @@ -154,6 +154,36 @@ func (b *builtinIsIPv6Sig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) return nil } +func (b *builtinIsUUIDSig) vectorized() bool { + return true +} + +func (b *builtinIsUUIDSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + buf, err := b.bufAllocator.get() + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { + return err + } + result.ResizeInt64(n, false) + i64s := result.Int64s() + result.MergeNulls(buf) + for i := 0; i < n; i++ { + if result.IsNull(i) { + continue + } + if _, err = uuid.Parse(buf.GetString(i)); err != nil { + i64s[i] = 0 + } else { + i64s[i] = 1 + } + } + return nil +} + func (b *builtinNameConstStringSig) vectorized() bool { return true } diff --git a/expression/builtin_miscellaneous_vec_test.go b/expression/builtin_miscellaneous_vec_test.go index 50c4508fe41be..386f7bc423878 100644 --- a/expression/builtin_miscellaneous_vec_test.go +++ b/expression/builtin_miscellaneous_vec_test.go @@ -108,6 +108,9 @@ var vecBuiltinMiscellaneousCases = map[string][]vecExprBenchCase{ {retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&uuidBinGener{newDefaultRandGen()}}}, {retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETString, types.ETInt}, geners: []dataGenerator{&uuidBinGener{newDefaultRandGen()}}}, }, + ast.IsUUID: { + {retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&uuidStrGener{newDefaultRandGen()}}}, + }, } func TestVectorizedBuiltinMiscellaneousEvalOneVec(t *testing.T) { diff --git a/expression/builtin_string.go b/expression/builtin_string.go index c15e4d4200920..9ac2eb370d380 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -289,7 +289,7 @@ func (c *concatFunctionClass) getFunction(ctx sessionctx.Context, args []Express if argType.Flen < 0 { bf.tp.Flen = mysql.MaxBlobWidth - logutil.BgLogger().Warn("unexpected `Flen` value(-1) in CONCAT's args", zap.Int("arg's index", i)) + logutil.BgLogger().Debug("unexpected `Flen` value(-1) in CONCAT's args", zap.Int("arg's index", i)) } bf.tp.Flen += argType.Flen } @@ -365,7 +365,7 @@ func (c *concatWSFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if i != 0 { if argType.Flen < 0 { bf.tp.Flen = mysql.MaxBlobWidth - logutil.BgLogger().Warn("unexpected `Flen` value(-1) in CONCAT_WS's args", zap.Int("arg's index", i)) + logutil.BgLogger().Debug("unexpected `Flen` value(-1) in CONCAT_WS's args", zap.Int("arg's index", i)) } bf.tp.Flen += argType.Flen } diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 6035e8b120fbf..54b98770b857c 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -439,7 +439,7 @@ func TestLeft(t *testing.T) { } } - _, err := funcs[ast.Left].getFunction(ctx, []Expression{varcharCon, int8Con}) + _, err := funcs[ast.Left].getFunction(ctx, []Expression{getVarcharCon(), getInt8Con()}) require.NoError(t, err) } @@ -490,7 +490,7 @@ func TestRight(t *testing.T) { } } - _, err := funcs[ast.Right].getFunction(ctx, []Expression{varcharCon, int8Con}) + _, err := funcs[ast.Right].getFunction(ctx, []Expression{getVarcharCon(), getInt8Con()}) require.NoError(t, err) } @@ -629,7 +629,7 @@ func TestLower(t *testing.T) { } } - _, err := funcs[ast.Lower].getFunction(ctx, []Expression{varcharCon}) + _, err := funcs[ast.Lower].getFunction(ctx, []Expression{getVarcharCon()}) require.NoError(t, err) // Test GBK String @@ -688,7 +688,7 @@ func TestUpper(t *testing.T) { } } - _, err := funcs[ast.Upper].getFunction(ctx, []Expression{varcharCon}) + _, err := funcs[ast.Upper].getFunction(ctx, []Expression{getVarcharCon()}) require.NoError(t, err) // Test GBK String @@ -1338,10 +1338,10 @@ func TestHexFunc(t *testing.T) { } } - _, err := funcs[ast.Hex].getFunction(ctx, []Expression{int8Con}) + _, err := funcs[ast.Hex].getFunction(ctx, []Expression{getInt8Con()}) require.NoError(t, err) - _, err = funcs[ast.Hex].getFunction(ctx, []Expression{varcharCon}) + _, err = funcs[ast.Hex].getFunction(ctx, []Expression{getVarcharCon()}) require.NoError(t, err) } diff --git a/expression/builtin_test.go b/expression/builtin_test.go index 57e39282100ab..79faddd2ffd3a 100644 --- a/expression/builtin_test.go +++ b/expression/builtin_test.go @@ -191,3 +191,11 @@ var ( // MySQL varchar. varcharCon = &Constant{RetType: &types.FieldType{Tp: mysql.TypeVarchar, Charset: charset.CharsetUTF8, Collate: charset.CollationUTF8}} ) + +func getInt8Con() Expression { + return int8Con.Clone() +} + +func getVarcharCon() Expression { + return varcharCon.Clone() +} diff --git a/expression/builtin_time.go b/expression/builtin_time.go index c6b3cfa669ef9..0c87fe8ec299c 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -4856,7 +4856,7 @@ func (b *builtinUnixTimestampIntSig) evalIntWithCtx(ctx sessionctx.Context, row } tz := ctx.GetSessionVars().Location() - t, err := val.GoTime(tz) + t, err := val.AdjustedGoTime(tz) if err != nil { return 0, false, nil } diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index 90fc1323117f9..13a0135d267b5 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -2390,7 +2390,7 @@ func (b *builtinUnixTimestampIntSig) vecEvalInt(input *chunk.Chunk, result *chun continue } - t, err := buf.GetTime(i).GoTime(getTimeZone(b.ctx)) + t, err := buf.GetTime(i).AdjustedGoTime(getTimeZone(b.ctx)) if err != nil { i64s[i] = 0 continue diff --git a/expression/constant_fold_test.go b/expression/constant_fold_test.go index cbdd966804bfa..d116da3916586 100644 --- a/expression/constant_fold_test.go +++ b/expression/constant_fold_test.go @@ -15,12 +15,18 @@ package expression_test import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/util/testkit" + "testing" + + "github.com/pingcap/tidb/testkit" ) -func (s *testIntegrationSuite) TestFoldIfNull(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestFoldIfNull(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test;`) tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a bigint, b bigint);`) diff --git a/expression/constant_propagation_test.go b/expression/constant_propagation_test.go index 56c6a89351943..ab88438623216 100644 --- a/expression/constant_propagation_test.go +++ b/expression/constant_propagation_test.go @@ -15,41 +15,20 @@ package expression_test import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testutil" -) + "testing" -var _ = Suite(&testSuite{}) + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" +) -type testSuite struct { - store kv.Storage - dom *domain.Domain - ctx sessionctx.Context - testData testutil.TestData -} +func TestOuterJoinPropConst(t *testing.T) { + t.Parallel() -func (s *testSuite) SetUpSuite(c *C) { - var err error - s.store, s.dom, err = newStoreWithBootstrap() - c.Assert(err, IsNil) - s.ctx = mock.NewContext() - s.testData, err = testutil.LoadTestSuiteData("testdata", "expression_suite") - c.Assert(err, IsNil) -} + store, clean := testkit.CreateMockStore(t) + defer clean() -func (s *testSuite) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) - s.dom.Close() - s.store.Close() -} - -func (s *testSuite) TestOuterJoinPropConst(c *C) { - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2;") tk.MustExec("create table t1(id bigint primary key, a int, b int);") @@ -60,11 +39,13 @@ func (s *testSuite) TestOuterJoinPropConst(c *C) { SQL string Result []string } - s.testData.GetTestCases(c, &input, &output) + + expressionSuiteData := expression.GetExpressionSuiteData() + expressionSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) } diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index 1763f71d2fb05..47dd46f87b39f 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tipb/go-tipb" ) @@ -40,7 +41,7 @@ func PbTypeToFieldType(tp *tipb.FieldType) *types.FieldType { Flen: int(tp.Flen), Decimal: int(tp.Decimal), Charset: tp.Charset, - Collate: protoToCollation(tp.Collate), + Collate: collate.ProtoToCollation(tp.Collate), Elems: tp.Elems, } } @@ -1216,7 +1217,7 @@ func convertUint(val []byte) (*Constant, error) { func convertString(val []byte, tp *tipb.FieldType) (*Constant, error) { var d types.Datum - d.SetBytesAsString(val, protoToCollation(tp.Collate), uint32(tp.Flen)) + d.SetBytesAsString(val, collate.ProtoToCollation(tp.Collate), uint32(tp.Flen)) return &Constant{Value: d, RetType: types.NewFieldType(mysql.TypeVarString)}, nil } diff --git a/expression/distsql_builtin_test.go b/expression/distsql_builtin_test.go index 06ced0d27211f..05337f806a4b8 100644 --- a/expression/distsql_builtin_test.go +++ b/expression/distsql_builtin_test.go @@ -868,7 +868,7 @@ func toPBFieldType(ft *types.FieldType) *tipb.FieldType { Flen: int32(ft.Flen), Decimal: int32(ft.Decimal), Charset: ft.Charset, - Collate: collationToProto(ft.Collate), + Collate: collate.CollationToProto(ft.Collate), Elems: ft.Elems, } } diff --git a/expression/expr_to_pb.go b/expression/expr_to_pb.go index 5cb01638eae0b..f6796b4c0517c 100644 --- a/expression/expr_to_pb.go +++ b/expression/expr_to_pb.go @@ -19,7 +19,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -157,7 +156,7 @@ func ToPBFieldType(ft *types.FieldType) *tipb.FieldType { Flen: int32(ft.Flen), Decimal: int32(ft.Decimal), Charset: ft.Charset, - Collate: collationToProto(ft.Collate), + Collate: collate.CollationToProto(ft.Collate), Elems: ft.Elems, } } @@ -170,39 +169,11 @@ func FieldTypeFromPB(ft *tipb.FieldType) *types.FieldType { Flen: int(ft.Flen), Decimal: int(ft.Decimal), Charset: ft.Charset, - Collate: protoToCollation(ft.Collate), + Collate: collate.ProtoToCollation(ft.Collate), Elems: ft.Elems, } } -func collationToProto(c string) int32 { - if coll, err := charset.GetCollationByName(c); err == nil { - return collate.RewriteNewCollationIDIfNeeded(int32(coll.ID)) - } - v := collate.RewriteNewCollationIDIfNeeded(int32(mysql.DefaultCollationID)) - logutil.BgLogger().Warn( - "Unable to get collation ID by name, use ID of the default collation instead", - zap.String("name", c), - zap.Int32("default collation ID", v), - zap.String("default collation", mysql.DefaultCollationName), - ) - return v -} - -func protoToCollation(c int32) string { - coll, err := charset.GetCollationByID(int(collate.RestoreCollationIDIfNeeded(c))) - if err == nil { - return coll.Name - } - logutil.BgLogger().Warn( - "Unable to get collation name from ID, use name of the default collation instead", - zap.Int32("id", c), - zap.Int("default collation ID", mysql.DefaultCollationID), - zap.String("default collation", mysql.DefaultCollationName), - ) - return mysql.DefaultCollationName -} - func (pc PbConverter) columnToPBExpr(column *Column) *tipb.Expr { if !pc.client.IsRequestTypeSupported(kv.ReqTypeSelect, int64(tipb.ExprType_ColumnRef)) { return nil diff --git a/expression/integration_serial_test.go b/expression/integration_serial_test.go new file mode 100644 index 0000000000000..6743637a6486e --- /dev/null +++ b/expression/integration_serial_test.go @@ -0,0 +1,1493 @@ +// Copyright 2021 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. + +package expression_test + +import ( + "fmt" + "math" + "strings" + "testing" + + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/kvcache" + "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" +) + +func TestIssue17727(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + var err error + session, err := session.CreateSession4TestWithOpt(store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + tk.SetSession(session) + require.NoError(t, err) + + tk.MustExec("use test;") + tk.MustExec("DROP TABLE IF EXISTS t1;") + tk.MustExec("CREATE TABLE t1 (id INT NOT NULL PRIMARY KEY auto_increment, a timestamp NOT NULL);") + tk.MustExec("INSERT INTO t1 VALUES (null, '2020-05-30 20:30:00');") + tk.MustExec("PREPARE mystmt FROM 'SELECT * FROM t1 WHERE UNIX_TIMESTAMP(a) >= ?';") + tk.MustExec("SET @a=1590868800;") + tk.MustQuery("EXECUTE mystmt USING @a;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + + tk.MustExec("SET @a=1590868801;") + tk.MustQuery("EXECUTE mystmt USING @a;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + + tk.MustExec("prepare stmt from 'select unix_timestamp(?)';") + tk.MustExec("set @a = '2020-05-30 20:30:00';") + tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1590841800")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + + tk.MustExec("set @a = '2020-06-12 13:47:58';") + tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1591940878")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) +} + +func TestIssue17891(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int, value set ('a','b','c') charset utf8mb4 collate utf8mb4_bin default 'a,b ');") + tk.MustExec("drop table t") + tk.MustExec("create table test(id int, value set ('a','b','c') charset utf8mb4 collate utf8mb4_general_ci default 'a,B ,C');") +} + +func TestIssue20268(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE `t` ( `a` enum('a','b') DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;") + tk.MustExec("insert into t values('a');") + tk.MustExec("select * from t where a = 'A';") +} + +func TestCollationBasic(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk.MustExec("use test") + tk.MustExec("create table t_ci(a varchar(10) collate utf8mb4_general_ci, unique key(a))") + tk.MustExec("insert into t_ci values ('a')") + tk.MustQuery("select * from t_ci").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci where a='a'").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci where a='A'").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci where a='a '").Check(testkit.Rows("a")) + tk.MustQuery("select * from t_ci where a='a '").Check(testkit.Rows("a")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(10) primary key,b int)") + tk.MustExec("insert into t values ('a', 1), ('b', 3), ('a', 2) on duplicate key update b = b + 1;") + tk.MustExec("set autocommit=0") + tk.MustExec("insert into t values ('a', 1), ('b', 3), ('a', 2) on duplicate key update b = b + 1;") + tk.MustQuery("select * from t").Check(testkit.Rows("a 4", "b 4")) + tk.MustExec("set autocommit=1") + tk.MustQuery("select * from t").Check(testkit.Rows("a 4", "b 4")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(10),b int, key tk (a))") + tk.MustExec("insert into t values ('', 1), ('', 3)") + tk.MustExec("set autocommit=0") + tk.MustExec("update t set b = b + 1") + tk.MustQuery("select * from t").Check(testkit.Rows(" 2", " 4")) + tk.MustExec("set autocommit=1") + tk.MustQuery("select * from t").Check(testkit.Rows(" 2", " 4")) + + tk.MustExec("drop table t_ci") + tk.MustExec("create table t_ci(id bigint primary key, a varchar(10) collate utf8mb4_general_ci, unique key(a, id))") + tk.MustExec("insert into t_ci values (1, 'a')") + tk.MustQuery("select a from t_ci").Check(testkit.Rows("a")) + tk.MustQuery("select a from t_ci").Check(testkit.Rows("a")) + tk.MustQuery("select a from t_ci where a='a'").Check(testkit.Rows("a")) + tk.MustQuery("select a from t_ci where a='A'").Check(testkit.Rows("a")) + tk.MustQuery("select a from t_ci where a='a '").Check(testkit.Rows("a")) + tk.MustQuery("select a from t_ci where a='a '").Check(testkit.Rows("a")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c set('A', 'B') collate utf8mb4_general_ci);") + tk.MustExec("insert into t values('a');") + tk.MustExec("insert into t values('B');") + tk.MustQuery("select c from t where c = 'a';").Check(testkit.Rows("A")) + tk.MustQuery("select c from t where c = 'A';").Check(testkit.Rows("A")) + tk.MustQuery("select c from t where c = 'b';").Check(testkit.Rows("B")) + tk.MustQuery("select c from t where c = 'B';").Check(testkit.Rows("B")) + + tk.MustExec("drop table if exists t1") + tk.MustExec("CREATE TABLE `t1` (" + + " `COL1` varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL," + + " PRIMARY KEY (`COL1`(5)) clustered" + + ")") + tk.MustExec("INSERT INTO `t1` VALUES ('Ȇ');") + tk.MustQuery("select * from t1 where col1 not in (0xc484, 0xe5a4bc, 0xc3b3);").Check(testkit.Rows("Ȇ")) + tk.MustQuery("select * from t1 where col1 >= 0xc484 and col1 <= 0xc3b3;").Check(testkit.Rows("Ȇ")) + + tk.MustQuery("select collation(IF('a' < 'B' collate utf8mb4_general_ci, 'smaller', 'greater' collate utf8mb4_unicode_ci));").Check(testkit.Rows("utf8mb4_unicode_ci")) +} + +func TestWeightString(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + type testCase struct { + input []string + result []string + resultAsChar1 []string + resultAsChar3 []string + resultAsBinary1 []string + resultAsBinary5 []string + resultExplicitCollateBin []string + } + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id int, a varchar(20) collate utf8mb4_general_ci)") + cases := testCase{ + input: []string{"aAÁàãăâ", "a", "a ", "中", "中 "}, + result: []string{"\x00A\x00A\x00A\x00A\x00A\x00A\x00A", "\x00A", "\x00A", "\x4E\x2D", "\x4E\x2D"}, + resultAsChar1: []string{"\x00A", "\x00A", "\x00A", "\x4E\x2D", "\x4E\x2D"}, + resultAsChar3: []string{"\x00A\x00A\x00A", "\x00A", "\x00A", "\x4E\x2D", "\x4E\x2D"}, + resultAsBinary1: []string{"a", "a", "a", "\xE4", "\xE4"}, + resultAsBinary5: []string{"aA\xc3\x81\xc3", "a\x00\x00\x00\x00", "a \x00\x00", "中\x00\x00", "中 \x00"}, + resultExplicitCollateBin: []string{"aAÁàãăâ", "a", "a", "中", "中"}, + } + values := make([]string, len(cases.input)) + for i, input := range cases.input { + values[i] = fmt.Sprintf("(%d, '%s')", i, input) + } + tk.MustExec("insert into t values " + strings.Join(values, ",")) + rows := tk.MustQuery("select weight_string(a) from t order by id").Rows() + for i, out := range cases.result { + require.Equal(t, out, rows[i][0].(string)) + } + rows = tk.MustQuery("select weight_string(a as char(1)) from t order by id").Rows() + for i, out := range cases.resultAsChar1 { + require.Equal(t, out, rows[i][0].(string)) + } + rows = tk.MustQuery("select weight_string(a as char(3)) from t order by id").Rows() + for i, out := range cases.resultAsChar3 { + require.Equal(t, out, rows[i][0].(string)) + } + rows = tk.MustQuery("select weight_string(a as binary(1)) from t order by id").Rows() + for i, out := range cases.resultAsBinary1 { + require.Equal(t, out, rows[i][0].(string)) + } + rows = tk.MustQuery("select weight_string(a as binary(5)) from t order by id").Rows() + for i, out := range cases.resultAsBinary5 { + require.Equal(t, out, rows[i][0].(string)) + } + require.Equal(t, "", tk.MustQuery("select weight_string(NULL);").Rows()[0][0]) + require.Equal(t, "", tk.MustQuery("select weight_string(7);").Rows()[0][0]) + require.Equal(t, "", tk.MustQuery("select weight_string(cast(7 as decimal(5)));").Rows()[0][0]) + require.Equal(t, "2019-08-21", tk.MustQuery("select weight_string(cast(20190821 as date));").Rows()[0][0]) + require.Equal(t, "2019-", tk.MustQuery("select weight_string(cast(20190821 as date) as binary(5));").Rows()[0][0]) + require.Equal(t, "", tk.MustQuery("select weight_string(7.0);").Rows()[0][0]) + require.Equal(t, "7\x00", tk.MustQuery("select weight_string(7 AS BINARY(2));").Rows()[0][0]) + // test explicit collation + require.Equal(t, "\x4E\x2D", tk.MustQuery("select weight_string('中 ' collate utf8mb4_general_ci);").Rows()[0][0]) + require.Equal(t, "中", tk.MustQuery("select weight_string('中 ' collate utf8mb4_bin);").Rows()[0][0]) + require.Equal(t, "\xFB\x40\xCE\x2D", tk.MustQuery("select weight_string('中 ' collate utf8mb4_unicode_ci);").Rows()[0][0]) + require.Equal(t, "utf8mb4_general_ci", tk.MustQuery("select collation(a collate utf8mb4_general_ci) from t order by id").Rows()[0][0]) + require.Equal(t, "utf8mb4_general_ci", tk.MustQuery("select collation('中 ' collate utf8mb4_general_ci);").Rows()[0][0]) + rows = tk.MustQuery("select weight_string(a collate utf8mb4_bin) from t order by id").Rows() + for i, out := range cases.resultExplicitCollateBin { + require.Equal(t, out, rows[i][0].(string)) + } + tk.MustGetErrMsg("select weight_string(a collate utf8_general_ci) from t order by id", "[ddl:1253]COLLATION 'utf8_general_ci' is not valid for CHARACTER SET 'utf8mb4'") + tk.MustGetErrMsg("select weight_string('中' collate utf8_bin)", "[ddl:1253]COLLATION 'utf8_bin' is not valid for CHARACTER SET 'utf8mb4'") +} + +func TestCollationCreateIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(10) collate utf8mb4_general_ci);") + tk.MustExec("insert into t values ('a');") + tk.MustExec("insert into t values ('A');") + tk.MustExec("insert into t values ('b');") + tk.MustExec("insert into t values ('B');") + tk.MustExec("insert into t values ('a');") + tk.MustExec("insert into t values ('A');") + tk.MustExec("insert into t values ('ß');") + tk.MustExec("insert into t values ('sa');") + tk.MustExec("create index idx on t(a);") + tk.MustQuery("select * from t order by a").Check(testkit.Rows("a", "A", "a", "A", "b", "B", "ß", "sa")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(10) collate utf8mb4_unicode_ci);") + tk.MustExec("insert into t values ('a');") + tk.MustExec("insert into t values ('A');") + tk.MustExec("insert into t values ('b');") + tk.MustExec("insert into t values ('B');") + tk.MustExec("insert into t values ('a');") + tk.MustExec("insert into t values ('A');") + tk.MustExec("insert into t values ('ß');") + tk.MustExec("insert into t values ('sa');") + tk.MustExec("create index idx on t(a);") + tk.MustQuery("select * from t order by a").Check(testkit.Rows("a", "A", "a", "A", "b", "B", "sa", "ß")) +} + +func TestCollateConstantPropagation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a char(10) collate utf8mb4_bin, b char(10) collate utf8mb4_general_ci);") + tk.MustExec("insert into t values ('a', 'A');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='a' collate utf8mb4_general_ci;").Check(nil) + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b>='a' collate utf8mb4_general_ci;").Check(nil) + tk.MustExec("drop table t;") + tk.MustExec("create table t (a char(10) collate utf8mb4_general_ci, b char(10) collate utf8mb4_general_ci);") + tk.MustExec("insert into t values ('A', 'a');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='a' collate utf8mb4_bin;").Check(testkit.Rows("A a A a")) + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b>='a' collate utf8mb4_bin;").Check(testkit.Rows("A a A a")) + tk.MustExec("drop table t;") + tk.MustExec("set names utf8mb4") + tk.MustExec("create table t (a char(10) collate utf8mb4_general_ci, b char(10) collate utf8_general_ci);") + tk.MustExec("insert into t values ('a', 'A');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='A'").Check(testkit.Rows("a A a A")) + tk.MustExec("drop table t;") + tk.MustExec("create table t(a char collate utf8_general_ci, b char collate utf8mb4_general_ci, c char collate utf8_bin);") + tk.MustExec("insert into t values ('b', 'B', 'B');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b=t2.c;").Check(testkit.Rows("b B B b B B")) + tk.MustExec("drop table t;") + tk.MustExec("create table t(a char collate utf8_bin, b char collate utf8_general_ci);") + tk.MustExec("insert into t values ('a', 'A');") + tk.MustQuery("select * from t t1, t t2 where t1.b=t2.b and t2.b=t1.a collate utf8_general_ci;").Check(testkit.Rows("a A a A")) + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_bin;") + tk.MustExec("insert into t1 values ('A', 'a');") + tk.MustExec("insert into t2 values ('a', 'a')") + tk.MustQuery("select * from t1 left join t2 on t1.a = t2.a where t1.a = 'a';").Check(testkit.Rows("A a ")) + tk.MustExec("drop table t;") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t(a char collate utf8mb4_bin, b char collate utf8mb4_general_ci);") + tk.MustExec("insert into t values ('a', 'a');") + tk.MustQuery("select * from t t1, t t2 where t2.b = 'A' and lower(concat(t1.a , '' )) = t2.b;").Check(testkit.Rows("a a a a")) + tk.MustExec("drop table t;") + tk.MustExec("create table t(a char collate utf8_unicode_ci, b char collate utf8mb4_unicode_ci, c char collate utf8_bin);") + tk.MustExec("insert into t values ('b', 'B', 'B');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b=t2.c;").Check(testkit.Rows("b B B b B B")) + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("set names utf8mb4 collate utf8mb4_unicode_ci;") + tk.MustExec("create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_unicode_ci;") + tk.MustExec("create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_bin;") + tk.MustExec("insert into t1 values ('A', 'a');") + tk.MustExec("insert into t2 values ('a', 'a')") + tk.MustQuery("select * from t1 left join t2 on t1.a = t2.a where t1.a = 'a';").Check(testkit.Rows("A a ")) + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_unicode_ci;") + tk.MustExec("insert into t1 values ('ß', 's');") + tk.MustExec("insert into t2 values ('s', 's')") + tk.MustQuery("select * from t1 left join t2 on t1.a = t2.a collate utf8mb4_unicode_ci where t1.a = 's';").Check(testkit.Rows("ß s ")) + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1(a char(10) collate utf8mb4_general_ci, index (a));") + tk.MustExec("create table t2(a char(10) collate utf8_bin, index (a));") + tk.MustExec("insert into t1 values ('a');") + tk.MustExec("insert into t2 values ('A');") + tk.MustExec("set names utf8 collate utf8_general_ci;") + tk.MustQuery("select * from t1, t2 where t1.a=t2.a and t1.a= 'a';").Check(testkit.Rows("a A")) + tk.MustQuery("select * from t1 where a='a' and a = 'A'").Check(testkit.Rows("a")) +} + +func TestMixCollation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk.MustGetErrMsg(`select 'a' collate utf8mb4_bin = 'a' collate utf8mb4_general_ci;`, "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation '='") + + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t ( + mb4general varchar(10) charset utf8mb4 collate utf8mb4_general_ci, + mb4unicode varchar(10) charset utf8mb4 collate utf8mb4_unicode_ci, + mb4bin varchar(10) charset utf8mb4 collate utf8mb4_bin, + general varchar(10) charset utf8 collate utf8_general_ci, + unicode varchar(10) charset utf8 collate utf8_unicode_ci, + utfbin varchar(10) charset utf8 collate utf8_bin, + bin varchar(10) charset binary collate binary, + latin1_bin varchar(10) charset latin1 collate latin1_bin, + ascii_bin varchar(10) charset ascii collate ascii_bin, + i int + );`) + tk.MustExec("insert into t values ('s', 's', 's', 's', 's', 's', 's', 's', 's', 1);") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + + tk.MustQuery("select * from t where mb4unicode = 's' collate utf8mb4_unicode_ci;").Check(testkit.Rows("s s s s s s s s s 1")) + tk.MustQuery(`select * from t t1, t t2 where t1.mb4unicode = t2.mb4general collate utf8mb4_general_ci;`).Check(testkit.Rows("s s s s s s s s s 1 s s s s s s s s s 1")) + tk.MustQuery(`select * from t t1, t t2 where t1.mb4general = t2.mb4unicode collate utf8mb4_general_ci;`).Check(testkit.Rows("s s s s s s s s s 1 s s s s s s s s s 1")) + tk.MustQuery(`select * from t t1, t t2 where t1.mb4general = t2.mb4unicode collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1 s s s s s s s s s 1")) + tk.MustQuery(`select * from t t1, t t2 where t1.mb4unicode = t2.mb4general collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1 s s s s s s s s s 1")) + tk.MustQuery(`select * from t where mb4general = mb4bin collate utf8mb4_general_ci;`).Check(testkit.Rows("s s s s s s s s s 1")) + tk.MustQuery(`select * from t where mb4unicode = mb4general collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1")) + tk.MustQuery(`select * from t where mb4general = mb4unicode collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1")) + tk.MustQuery(`select * from t where mb4unicode = 's' collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1")) + tk.MustQuery("select * from t where mb4unicode = mb4bin;").Check(testkit.Rows("s s s s s s s s s 1")) + tk.MustQuery("select * from t where general = mb4unicode;").Check(testkit.Rows("s s s s s s s s s 1")) + tk.MustQuery("select * from t where unicode = mb4unicode;").Check(testkit.Rows("s s s s s s s s s 1")) + tk.MustQuery("select * from t where mb4unicode = mb4unicode;").Check(testkit.Rows("s s s s s s s s s 1")) + + tk.MustQuery("select collation(concat(mb4unicode, mb4general collate utf8mb4_unicode_ci)) from t;").Check(testkit.Rows("utf8mb4_unicode_ci")) + tk.MustQuery("select collation(concat(mb4general, mb4unicode, mb4bin)) from t;").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select coercibility(concat(mb4general, mb4unicode, mb4bin)) from t;").Check(testkit.Rows("1")) + tk.MustQuery("select collation(concat(mb4unicode, mb4bin, concat(mb4general))) from t;").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select coercibility(concat(mb4unicode, mb4bin)) from t;").Check(testkit.Rows("2")) + tk.MustQuery("select collation(concat(mb4unicode, mb4bin)) from t;").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select coercibility(concat(mb4bin, concat(mb4general))) from t;").Check(testkit.Rows("2")) + tk.MustQuery("select collation(concaT(mb4bin, cOncAt(mb4general))) from t;").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select coercibility(concat(mb4unicode, mb4bin, concat(mb4general))) from t;").Check(testkit.Rows("2")) + tk.MustQuery("select collation(concat(mb4unicode, mb4bin, concat(mb4general))) from t;").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select coercibility(concat(mb4unicode, mb4general)) from t;").Check(testkit.Rows("1")) + tk.MustQuery("select collation(coalesce(mb4unicode, mb4general)) from t;").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select coercibility(coalesce(mb4unicode, mb4general)) from t;").Check(testkit.Rows("1")) + tk.MustQuery("select collation(CONCAT(concat(mb4unicode), concat(mb4general))) from t;").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select coercibility(cONcat(unicode, general)) from t;").Check(testkit.Rows("1")) + tk.MustQuery("select collation(concAt(unicode, general)) from t;").Check(testkit.Rows("utf8_bin")) + tk.MustQuery("select collation(concat(bin, mb4general)) from t;").Check(testkit.Rows("binary")) + tk.MustQuery("select coercibility(concat(bin, mb4general)) from t;").Check(testkit.Rows("2")) + tk.MustQuery("select collation(concat(mb4unicode, ascii_bin)) from t;").Check(testkit.Rows("utf8mb4_unicode_ci")) + tk.MustQuery("select coercibility(concat(mb4unicode, ascii_bin)) from t;").Check(testkit.Rows("2")) + tk.MustQuery("select collation(concat(mb4unicode, mb4unicode)) from t;").Check(testkit.Rows("utf8mb4_unicode_ci")) + tk.MustQuery("select coercibility(concat(mb4unicode, mb4unicode)) from t;").Check(testkit.Rows("2")) + tk.MustQuery("select collation(concat(bin, bin)) from t;").Check(testkit.Rows("binary")) + tk.MustQuery("select coercibility(concat(bin, bin)) from t;").Check(testkit.Rows("2")) + tk.MustQuery("select collation(concat(latin1_bin, ascii_bin)) from t;").Check(testkit.Rows("latin1_bin")) + tk.MustQuery("select coercibility(concat(latin1_bin, ascii_bin)) from t;").Check(testkit.Rows("2")) + tk.MustQuery("select collation(concat(mb4unicode, bin)) from t;").Check(testkit.Rows("binary")) + tk.MustQuery("select coercibility(concat(mb4unicode, bin)) from t;").Check(testkit.Rows("2")) + tk.MustQuery("select collation(mb4general collate utf8mb4_unicode_ci) from t;").Check(testkit.Rows("utf8mb4_unicode_ci")) + tk.MustQuery("select coercibility(mb4general collate utf8mb4_unicode_ci) from t;").Check(testkit.Rows("0")) + tk.MustQuery("select collation(concat(concat(mb4unicode, mb4general), concat(unicode, general))) from t;").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select coercibility(concat(concat(mb4unicode, mb4general), concat(unicode, general))) from t;").Check(testkit.Rows("1")) + tk.MustQuery("select collation(concat(i, 1)) from t;").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select coercibility(concat(i, 1)) from t;").Check(testkit.Rows("4")) + tk.MustQuery("select collation(concat(i, user())) from t;").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select coercibility(concat(i, user())) from t;").Check(testkit.Rows("3")) + tk.MustGetErrMsg("select * from t where mb4unicode = mb4general;", "[expression:1267]Illegal mix of collations (utf8mb4_unicode_ci,IMPLICIT) and (utf8mb4_general_ci,IMPLICIT) for operation '='") + tk.MustGetErrMsg("select * from t where unicode = general;", "[expression:1267]Illegal mix of collations (utf8_unicode_ci,IMPLICIT) and (utf8_general_ci,IMPLICIT) for operation '='") + tk.MustGetErrMsg("select concat(mb4general) = concat(mb4unicode) from t;", "[expression:1267]Illegal mix of collations (utf8mb4_general_ci,IMPLICIT) and (utf8mb4_unicode_ci,IMPLICIT) for operation '='") + tk.MustGetErrMsg("select * from t t1, t t2 where t1.mb4unicode = t2.mb4general;", "[expression:1267]Illegal mix of collations (utf8mb4_unicode_ci,IMPLICIT) and (utf8mb4_general_ci,IMPLICIT) for operation '='") + tk.MustGetErrMsg("select field('s', mb4general, mb4unicode, mb4bin) from t;", "[expression:1271]Illegal mix of collations for operation 'field'") + tk.MustGetErrMsg("select concat(mb4unicode, mb4general) = mb4unicode from t;", "[expression:1267]Illegal mix of collations (utf8mb4_bin,NONE) and (utf8mb4_unicode_ci,IMPLICIT) for operation '='") + + tk.MustExec("drop table t;") +} + +func prepare4Join(tk *testkit.TestKit) { + tk.MustExec("USE test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t_bin") + tk.MustExec("CREATE TABLE `t` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL)") + tk.MustExec("CREATE TABLE `t_bin` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET binary)") + tk.MustExec("insert into t values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") + tk.MustExec("insert into t_bin values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") +} + +func TestCollateHashJoin(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Join(tk) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b order by t1.a").Check( + testkit.Rows("1 a", "1 a", "1 a", "1 a", "2 À", "2 À", "2 À", "2 À", "3 á", "3 á", "3 á", "3 á", "4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b order by t1.a").Check( + testkit.Rows("1 a", "2 À", "3 á", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( + testkit.Rows("2 À", "3 á", "3 á", "4 à", "4 à", "4 à")) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( + testkit.Rows()) +} + +func TestCollateHashJoin2(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Join2(tk) + tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ * from t1, t2 where t1.v=t2.v order by t1.id").Check( + testkit.Rows("1 a a", "2 À À", "3 á á", "4 à à", "5 b b", "6 c c", "7 ")) +} + +func prepare4Join2(tk *testkit.TestKit) { + tk.MustExec("USE test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1 (id int, v varchar(5) character set binary, key(v))") + tk.MustExec("create table t2 (v varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, key(v))") + tk.MustExec("insert into t1 values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") + tk.MustExec("insert into t2 values ('a'), ('À'), ('á'), ('à'), ('b'), ('c'), (' ')") +} + +func TestCollateMergeJoin(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Join(tk) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b order by t1.a").Check( + testkit.Rows("1 a", "1 a", "1 a", "1 a", "2 À", "2 À", "2 À", "2 À", "3 á", "3 á", "3 á", "3 á", "4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b order by t1.a").Check( + testkit.Rows("1 a", "2 À", "3 á", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( + testkit.Rows("4 à", "5 b", "6 c", "7 ")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( + testkit.Rows("2 À", "3 á", "3 á", "4 à", "4 à", "4 à")) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( + testkit.Rows()) +} + +func TestCollateMergeJoin2(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Join2(tk) + tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ * from t1, t2 where t1.v=t2.v order by t1.id").Check( + testkit.Rows("1 a a", "2 À À", "3 á á", "4 à à", "5 b b", "6 c c", "7 ")) +} + +func TestCollateIndexMergeJoin(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, b varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, key(a), key(b))") + tk.MustExec("insert into t values ('a', 'x'), ('x', 'À'), ('á', 'x'), ('à', 'à'), ('à', 'x')") + + tk.MustExec("set tidb_enable_index_merge=1") + tk.MustQuery("select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 'a' or b = 'a'").Sort().Check( + testkit.Rows("a x", "x À", "à x", "à à", "á x")) +} + +func TestNewCollationCheckClusterIndexTable(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("create table t(name char(255) primary key, b int, c int, index idx(name), unique index uidx(name))") + tk.MustExec("insert into t values(\"aaaa\", 1, 1), (\"bbb\", 2, 2), (\"ccc\", 3, 3)") + tk.MustExec("admin check table t") +} + +func prepare4Collation(tk *testkit.TestKit, hasIndex bool) { + tk.MustExec("USE test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t_bin") + idxSQL := ", key(v)" + if !hasIndex { + idxSQL = "" + } + tk.MustExec(fmt.Sprintf("create table t (id int, v varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL %v)", idxSQL)) + tk.MustExec(fmt.Sprintf("create table t_bin (id int, v varchar(5) CHARACTER SET binary %v)", idxSQL)) + tk.MustExec("insert into t values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") + tk.MustExec("insert into t_bin values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") +} + +func TestCollateSelection(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Collation(tk, false) + tk.MustQuery("select v from t where v='a' order by id").Check(testkit.Rows("a", "À", "á", "à")) + tk.MustQuery("select v from t_bin where v='a' order by id").Check(testkit.Rows("a")) + tk.MustQuery("select v from t where v<'b' and id<=3").Check(testkit.Rows("a", "À", "á")) + tk.MustQuery("select v from t_bin where v<'b' and id<=3").Check(testkit.Rows("a")) +} + +func TestCollateSort(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Collation(tk, false) + tk.MustQuery("select id from t order by v, id").Check(testkit.Rows("7", "1", "2", "3", "4", "5", "6")) + tk.MustQuery("select id from t_bin order by v, id").Check(testkit.Rows("7", "1", "5", "6", "2", "4", "3")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10) collate utf8mb4_general_ci, key(a))") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustQuery("select * from t order by a collate utf8mb4_bin").Check(testkit.Rows("A", "A", "A", "a", "a", "a", "b", "b", "b")) + tk.MustQuery("select * from t order by a collate utf8mb4_general_ci").Check(testkit.Rows("a", "A", "a", "A", "a", "A", "b", "b", "b")) + tk.MustQuery("select * from t order by a collate utf8mb4_unicode_ci").Check(testkit.Rows("a", "A", "a", "A", "a", "A", "b", "b", "b")) +} + +func TestCollateHashAgg(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Collation(tk, false) + tk.HasPlan("select distinct(v) from t_bin", "HashAgg") + tk.MustQuery("select distinct(v) from t_bin").Sort().Check(testkit.Rows(" ", "a", "b", "c", "À", "à", "á")) + tk.HasPlan("select distinct(v) from t", "HashAgg") + tk.MustQuery("select distinct(v) from t").Sort().Check(testkit.Rows(" ", "a", "b", "c")) + tk.HasPlan("select v, count(*) from t_bin group by v", "HashAgg") + tk.MustQuery("select v, count(*) from t_bin group by v").Sort().Check(testkit.Rows(" 1", "a 1", "b 1", "c 1", "À 1", "à 1", "á 1")) + tk.HasPlan("select v, count(*) from t group by v", "HashAgg") + tk.MustQuery("select v, count(*) from t group by v").Sort().Check(testkit.Rows(" 1", "a 4", "b 1", "c 1")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10) collate utf8mb4_general_ci, key(a))") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustExec("insert into t values ('a'), ('A'), ('b')") + tk.MustExec("insert into t values ('s'), ('ss'), ('ß')") + tk.MustQuery("select count(1) from t group by a collate utf8mb4_bin order by a collate utf8mb4_bin").Check(testkit.Rows("3", "3", "3", "1", "1", "1")) + tk.MustQuery("select count(1) from t group by a collate utf8mb4_unicode_ci order by a collate utf8mb4_unicode_ci").Check(testkit.Rows("6", "3", "1", "2")) + tk.MustQuery("select count(1) from t group by a collate utf8mb4_general_ci order by a collate utf8mb4_general_ci").Check(testkit.Rows("6", "3", "2", "1")) +} + +func TestCollateStreamAgg(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Collation(tk, true) + tk.HasPlan("select distinct(v) from t_bin", "StreamAgg") + tk.MustQuery("select distinct(v) from t_bin").Sort().Check(testkit.Rows(" ", "a", "b", "c", "À", "à", "á")) + tk.HasPlan("select distinct(v) from t", "StreamAgg") + tk.MustQuery("select distinct(v) from t").Sort().Check(testkit.Rows(" ", "a", "b", "c")) + tk.HasPlan("select v, count(*) from t_bin group by v", "StreamAgg") + tk.MustQuery("select v, count(*) from t_bin group by v").Sort().Check(testkit.Rows(" 1", "a 1", "b 1", "c 1", "À 1", "à 1", "á 1")) + tk.HasPlan("select v, count(*) from t group by v", "StreamAgg") + tk.MustQuery("select v, count(*) from t group by v").Sort().Check(testkit.Rows(" 1", "a 4", "b 1", "c 1")) +} + +func TestCollateIndexReader(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Collation(tk, true) + tk.HasPlan("select v from t where v < 'b' order by v", "IndexReader") + tk.MustQuery("select v from t where v < 'b' order by v").Check(testkit.Rows(" ", "a", "À", "á", "à")) + tk.HasPlan("select v from t where v < 'b' and v > ' ' order by v", "IndexReader") + tk.MustQuery("select v from t where v < 'b' and v > ' ' order by v").Check(testkit.Rows("a", "À", "á", "à")) + tk.HasPlan("select v from t_bin where v < 'b' order by v", "IndexReader") + tk.MustQuery("select v from t_bin where v < 'b' order by v").Sort().Check(testkit.Rows(" ", "a")) + tk.HasPlan("select v from t_bin where v < 'b' and v > ' ' order by v", "IndexReader") + tk.MustQuery("select v from t_bin where v < 'b' and v > ' ' order by v").Sort().Check(testkit.Rows("a")) +} + +func TestCollateIndexLookup(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Collation(tk, true) + + tk.HasPlan("select id from t where v < 'b'", "IndexLookUp") + tk.MustQuery("select id from t where v < 'b'").Sort().Check(testkit.Rows("1", "2", "3", "4", "7")) + tk.HasPlan("select id from t where v < 'b' and v > ' '", "IndexLookUp") + tk.MustQuery("select id from t where v < 'b' and v > ' '").Sort().Check(testkit.Rows("1", "2", "3", "4")) + tk.HasPlan("select id from t_bin where v < 'b'", "IndexLookUp") + tk.MustQuery("select id from t_bin where v < 'b'").Sort().Check(testkit.Rows("1", "7")) + tk.HasPlan("select id from t_bin where v < 'b' and v > ' '", "IndexLookUp") + tk.MustQuery("select id from t_bin where v < 'b' and v > ' '").Sort().Check(testkit.Rows("1")) +} + +func TestIssue16668(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists tx") + tk.MustExec("CREATE TABLE `tx` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL)") + tk.MustExec("insert into tx values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") + tk.MustQuery("select count(distinct(b)) from tx").Check(testkit.Rows("4")) +} + +func TestIssue27091(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists tx") + tk.MustExec("CREATE TABLE `tx` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL, `c` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin DEFAULT NULL)") + tk.MustExec("insert into tx values (1, 'a', 'a'), (2, 'A ', 'a '), (3, 'A', 'A'), (4, 'a ', 'A ')") + tk.MustQuery("select count(distinct b) from tx").Check(testkit.Rows("1")) + tk.MustQuery("select count(distinct c) from tx").Check(testkit.Rows("2")) + tk.MustQuery("select count(distinct b, c) from tx where a < 3").Check(testkit.Rows("1")) + tk.MustQuery("select approx_count_distinct(b) from tx").Check(testkit.Rows("1")) + tk.MustQuery("select approx_count_distinct(c) from tx").Check(testkit.Rows("2")) + tk.MustQuery("select approx_count_distinct(b, c) from tx where a < 3").Check(testkit.Rows("1")) +} + +func TestCollateStringFunction(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + + tk.MustQuery("select field('a', 'b', 'a');").Check(testkit.Rows("2")) + tk.MustQuery("select field('a', 'b', 'A');").Check(testkit.Rows("0")) + tk.MustQuery("select field('a', 'b', 'A' collate utf8mb4_bin);").Check(testkit.Rows("0")) + tk.MustQuery("select field('a', 'b', 'a ' collate utf8mb4_bin);").Check(testkit.Rows("2")) + tk.MustQuery("select field('a', 'b', 'A' collate utf8mb4_unicode_ci);").Check(testkit.Rows("2")) + tk.MustQuery("select field('a', 'b', 'a ' collate utf8mb4_unicode_ci);").Check(testkit.Rows("2")) + tk.MustQuery("select field('a', 'b', 'A' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) + tk.MustQuery("select field('a', 'b', 'a ' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) + + tk.MustExec("USE test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10), b char (10)) collate utf8mb4_general_ci") + tk.MustExec("insert into t values ('a', 'A')") + tk.MustQuery("select field(a, b) from t").Check(testkit.Rows("1")) + + tk.MustQuery("select FIND_IN_SET('a','b,a,c,d');").Check(testkit.Rows("2")) + tk.MustQuery("select FIND_IN_SET('a','b,A,c,d');").Check(testkit.Rows("0")) + tk.MustQuery("select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_bin);").Check(testkit.Rows("0")) + tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_bin);").Check(testkit.Rows("2")) + tk.MustQuery("select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) + tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) + + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustQuery("select collation(cast('a' as char));").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select collation(cast('a' as binary));").Check(testkit.Rows("binary")) + tk.MustQuery("select collation(cast('a' collate utf8mb4_bin as char));").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select collation(cast('a' collate utf8mb4_bin as binary));").Check(testkit.Rows("binary")) + + tk.MustQuery("select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_unicode_ci);").Check(testkit.Rows("2")) + tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_unicode_ci);").Check(testkit.Rows("2")) + + tk.MustExec("select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_bin);") + tk.MustGetErrMsg("select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci);", "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation 'concat'") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char)") + tk.MustGetErrMsg("select * from t t1 join t t2 on t1.a collate utf8mb4_bin = t2.a collate utf8mb4_general_ci;", "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation '='") + + tk.MustExec("DROP TABLE IF EXISTS t1;") + tk.MustExec("CREATE TABLE t1 ( a int, p1 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin,p2 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_general_ci , p3 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin,p4 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci ,n1 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin,n2 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_general_ci , n3 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin,n4 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci );") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values(1,' 0aA1!测试テストמבחן ',' 0aA1!测试テストמבחן ',' 0aA1!测试テストמבחן ',' 0aA1!测试テストמבחן ',' 0Aa1!测试テストמבחן ',' 0Aa1!测试テストמבחן ',' 0Aa1!测试テストמבחן ',' 0Aa1!测试テストמבחן ');") + + tk.MustQuery("select INSTR(p1,n1) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p1,n2) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p1,n3) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p1,n4) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p2,n1) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p2,n2) from t1;").Check(testkit.Rows("1")) + tk.MustQuery("select INSTR(p2,n3) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p2,n4) from t1;").Check(testkit.Rows("1")) + tk.MustQuery("select INSTR(p3,n1) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p3,n2) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p3,n3) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p3,n4) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p4,n1) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p4,n2) from t1;").Check(testkit.Rows("1")) + tk.MustQuery("select INSTR(p4,n3) from t1;").Check(testkit.Rows("0")) + tk.MustQuery("select INSTR(p4,n4) from t1;").Check(testkit.Rows("1")) + + tk.MustExec("truncate table t1;") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (1,'0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן');") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (2,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן');") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (3,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ');") + + tk.MustQuery("select LOCATE(p1,n1) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p1,n2) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p1,n3) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p1,n4) from t1;").Check(testkit.Rows("0", "1", "1")) + tk.MustQuery("select LOCATE(p2,n1) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p2,n2) from t1;").Check(testkit.Rows("0", "1", "1")) + tk.MustQuery("select LOCATE(p2,n3) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p2,n4) from t1;").Check(testkit.Rows("0", "1", "1")) + tk.MustQuery("select LOCATE(p3,n1) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p3,n2) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p3,n3) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p3,n4) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p4,n1) from t1;").Check(testkit.Rows("0", "1", "1")) + tk.MustQuery("select LOCATE(p4,n2) from t1;").Check(testkit.Rows("0", "1", "1")) + tk.MustQuery("select LOCATE(p4,n3) from t1;").Check(testkit.Rows("0", "0", "0")) + tk.MustQuery("select LOCATE(p4,n4) from t1;").Check(testkit.Rows("0", "1", "1")) + + tk.MustQuery("select locate('S', 's' collate utf8mb4_general_ci);").Check(testkit.Rows("1")) + tk.MustQuery("select locate('S', 'a' collate utf8mb4_general_ci);").Check(testkit.Rows("0")) + // MySQL return 0 here, I believe it is a bug in MySQL since 'ß' == 's' under utf8mb4_general_ci collation. + tk.MustQuery("select locate('ß', 's' collate utf8mb4_general_ci);").Check(testkit.Rows("1")) + tk.MustQuery("select locate('S', 's' collate utf8mb4_unicode_ci);").Check(testkit.Rows("1")) + tk.MustQuery("select locate('S', 'a' collate utf8mb4_unicode_ci);").Check(testkit.Rows("0")) + tk.MustQuery("select locate('ß', 'ss' collate utf8mb4_unicode_ci);").Check(testkit.Rows("1")) + + tk.MustExec("truncate table t1;") + tk.MustExec("insert into t1 (a) values (1);") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (2,'0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן');") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (3,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן');") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (4,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ');") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (5,'0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0Aa1!测试','0Aa1!测试','0Aa1!测试','0Aa1!测试');") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (6,'0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试','0aA1!测试','0aA1!测试','0aA1!测试');") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (7,'0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן');") + tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (8,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ');") + + tk.MustQuery("select p1 REGEXP n1 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p1 REGEXP n2 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p1 REGEXP n3 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p1 REGEXP n4 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) + tk.MustQuery("select p2 REGEXP n1 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p2 REGEXP n2 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) + tk.MustQuery("select p2 REGEXP n3 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p2 REGEXP n4 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) + tk.MustQuery("select p3 REGEXP n1 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p3 REGEXP n2 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p3 REGEXP n3 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p3 REGEXP n4 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p4 REGEXP n1 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) + tk.MustQuery("select p4 REGEXP n2 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) + tk.MustQuery("select p4 REGEXP n3 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) + tk.MustQuery("select p4 REGEXP n4 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) + + tk.MustExec("drop table t1;") +} + +func TestCollateLike(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci") + tk.MustQuery("select 'a' like 'A'").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like 'A' collate utf8mb4_general_ci").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like 'À'").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like '%À'").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like '%À '").Check(testkit.Rows("0")) + tk.MustQuery("select 'a' like 'À%'").Check(testkit.Rows("1")) + tk.MustQuery("select 'a' like 'À_'").Check(testkit.Rows("0")) + tk.MustQuery("select 'a' like '%À%'").Check(testkit.Rows("1")) + tk.MustQuery("select 'aaa' like '%ÀAa%'").Check(testkit.Rows("1")) + tk.MustExec("set names utf8mb4 collate utf8mb4_bin") + + tk.MustExec("use test;") + tk.MustExec("drop table if exists t_like;") + tk.MustExec("create table t_like(id int, b varchar(20) collate utf8mb4_general_ci);") + tk.MustExec("insert into t_like values (1, 'aaa'), (2, 'abc'), (3, 'aac');") + tk.MustQuery("select b like 'AaÀ' from t_like order by id;").Check(testkit.Rows("1", "0", "0")) + tk.MustQuery("select b like 'Aa_' from t_like order by id;").Check(testkit.Rows("1", "0", "1")) + tk.MustQuery("select b like '_A_' from t_like order by id;").Check(testkit.Rows("1", "0", "1")) + tk.MustQuery("select b from t_like where b like 'Aa_' order by id;").Check(testkit.Rows("aaa", "aac")) + tk.MustQuery("select b from t_like where b like 'A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) + tk.MustQuery("select b from t_like where b like '%A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) + tk.MustExec("alter table t_like add index idx_b(b);") + tk.MustQuery("select b from t_like use index(idx_b) where b like 'Aa_' order by id;").Check(testkit.Rows("aaa", "aac")) + tk.MustQuery("select b from t_like use index(idx_b) where b like 'A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) + tk.MustQuery("select b from t_like use index(idx_b) where b like '%A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) +} + +func TestCollateSubQuery(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + prepare4Collation(tk, false) + tk.MustQuery("select id from t where v in (select v from t_bin) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) + tk.MustQuery("select id from t_bin where v in (select v from t) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) + tk.MustQuery("select id from t where v not in (select v from t_bin) order by id").Check(testkit.Rows()) + tk.MustQuery("select id from t_bin where v not in (select v from t) order by id").Check(testkit.Rows()) + tk.MustQuery("select id from t where exists (select 1 from t_bin where t_bin.v=t.v) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) + tk.MustQuery("select id from t_bin where exists (select 1 from t where t_bin.v=t.v) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) + tk.MustQuery("select id from t where not exists (select 1 from t_bin where t_bin.v=t.v) order by id").Check(testkit.Rows()) + tk.MustQuery("select id from t_bin where not exists (select 1 from t where t_bin.v=t.v) order by id").Check(testkit.Rows()) +} + +func TestCollateDDL(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database t;") + tk.MustExec("use t;") + tk.MustExec("drop database t;") +} + +func TestNewCollationWithClusterIndex(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("create table t(d double primary key, a int, name varchar(255), index idx(name(2)), index midx(a, name))") + tk.MustExec("insert into t values(2.11, 1, \"aa\"), (-1, 0, \"abcd\"), (9.99, 0, \"aaaa\")") + tk.MustQuery("select d from t use index(idx) where name=\"aa\"").Check(testkit.Rows("2.11")) +} + +func TestNewCollationBinaryFlag(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(2) binary, index (a));") + tk.MustExec("insert into t values ('a ');") + tk.MustQuery("select hex(a) from t;").Check(testkit.Rows("6120")) + tk.MustQuery("select hex(a) from t use index (a);").Check(testkit.Rows("6120")) + + showCreateTable := func(createSQL string) string { + tk.MustExec("drop table if exists t;") + tk.MustExec(createSQL) + s := tk.MustQuery("show create table t;").Rows()[0][1].(string) + return s + } + var sct string + // define case = tuple(table_charset, table_collation, column_charset, column_collation) + // case: (nil, nil, nil, nil) + sct = showCreateTable("create table t(a varchar(10) binary);") + require.Contains(t, sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") + // case: (nil, utf8_general_ci, nil, nil) + sct = showCreateTable("create table t(a varchar(10) binary) collate utf8_general_ci;") + require.Contains(t, sct, "varchar(10) COLLATE utf8_bin") + require.Contains(t, sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci") + // case: (nil, nil, nil, utf8_general_ci) + sct = showCreateTable("create table t(a varchar(10) binary collate utf8_general_ci);") + require.Contains(t, sct, "varchar(10) CHARACTER SET utf8 COLLATE utf8_bin") + // case: (nil, nil, utf8, utf8_general_ci) + sct = showCreateTable("create table t(a varchar(10) binary charset utf8 collate utf8_general_ci);") + require.Contains(t, sct, "varchar(10) CHARACTER SET utf8 COLLATE utf8_general_ci") + // case: (utf8, utf8_general_ci, utf8mb4, utf8mb4_unicode_ci) + sct = showCreateTable("create table t(a varchar(10) binary charset utf8mb4 collate utf8mb4_unicode_ci) charset utf8 collate utf8_general_ci;") + require.Contains(t, sct, "varchar(10) CHARACTER SET utf8mb4 COLLATE utf8mb4_unicode_ci") + require.Contains(t, sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci") + // case: (nil, nil, binary, nil) + sct = showCreateTable("create table t(a varchar(10) binary charset binary);") + require.Contains(t, sct, "varbinary(10) DEFAULT NULL") + require.Contains(t, sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") +} + +func TestIssue17176(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustGetErrMsg("create table t(a enum('a', 'a ')) charset utf8 collate utf8_bin;", "[types:1291]Column 'a' has duplicated value 'a' in ENUM") + tk.MustGetErrMsg("create table t(a enum('a', 'Á')) charset utf8 collate utf8_general_ci;", "[types:1291]Column 'a' has duplicated value 'Á' in ENUM") + tk.MustGetErrMsg("create table t(a enum('a', 'a ')) charset utf8mb4 collate utf8mb4_bin;", "[types:1291]Column 'a' has duplicated value 'a' in ENUM") + tk.MustExec("create table t(a enum('a', 'A')) charset utf8 collate utf8_bin;") + tk.MustExec("drop table t;") + tk.MustExec("create table t3(a enum('a', 'A')) charset utf8mb4 collate utf8mb4_bin;") +} + +func TestIssue18638(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a varchar(10) collate utf8mb4_bin, b varchar(10) collate utf8mb4_general_ci);") + tk.MustExec("insert into t (a, b) values ('a', 'A');") + tk.MustQuery("select * from t t1, t t2 where t1.a = t2.b collate utf8mb4_general_ci;").Check(testkit.Rows("a A a A")) + tk.MustQuery("select * from t t1 left join t t2 on t1.a = t2.b collate utf8mb4_general_ci;").Check(testkit.Rows("a A a A")) +} + +func TestCollationText(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a TINYTEXT collate UTF8MB4_GENERAL_CI, UNIQUE KEY `a`(`a`(10)));") + tk.MustExec("insert into t (a) values ('A');") + tk.MustQuery("select * from t t1 inner join t t2 on t1.a = t2.a where t1.a = 'A';").Check(testkit.Rows("A A")) + tk.MustExec("update t set a = 'B';") + tk.MustExec("admin check table t;") +} + +func TestIssue18662(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a varchar(10) collate utf8mb4_bin, b varchar(10) collate utf8mb4_general_ci);") + tk.MustExec("insert into t (a, b) values ('a', 'A');") + tk.MustQuery("select * from t where field('A', a collate utf8mb4_general_ci, b) > 1;").Check(testkit.Rows()) + tk.MustQuery("select * from t where field('A', a, b collate utf8mb4_general_ci) > 1;").Check(testkit.Rows()) + tk.MustQuery("select * from t where field('A' collate utf8mb4_general_ci, a, b) > 1;").Check(testkit.Rows()) + tk.MustQuery("select * from t where field('A', a, b) > 1;").Check(testkit.Rows("a A")) +} + +func TestIssue19116(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustQuery("select collation(concat(1 collate `binary`));").Check(testkit.Rows("binary")) + tk.MustQuery("select coercibility(concat(1 collate `binary`));").Check(testkit.Rows("0")) + tk.MustQuery("select collation(concat(NULL,NULL));").Check(testkit.Rows("binary")) + tk.MustQuery("select coercibility(concat(NULL,NULL));").Check(testkit.Rows("6")) + tk.MustQuery("select collation(concat(1,1));").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select coercibility(concat(1,1));").Check(testkit.Rows("4")) + tk.MustQuery("select collation(1);").Check(testkit.Rows("binary")) + tk.MustQuery("select coercibility(1);").Check(testkit.Rows("5")) + tk.MustQuery("select coercibility(1=1);").Check(testkit.Rows("5")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a datetime)") + tk.MustExec("insert into t values ('2020-02-02')") + tk.MustQuery("select collation(concat(unix_timestamp(a))) from t;").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select coercibility(concat(unix_timestamp(a))) from t;").Check(testkit.Rows("4")) +} + +func TestIssue17063(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test;`) + tk.MustExec(`drop table if exists t;`) + tk.MustExec("create table t(a char, b char) collate utf8mb4_general_ci;") + tk.MustExec(`insert into t values('a', 'b');`) + tk.MustExec(`insert into t values('a', 'B');`) + tk.MustQuery(`select * from t where if(a='x', a, b) = 'b';`).Check(testkit.Rows("a b", "a B")) + tk.MustQuery(`select collation(if(a='x', a, b)) from t;`).Check(testkit.Rows("utf8mb4_general_ci", "utf8mb4_general_ci")) + tk.MustQuery(`select coercibility(if(a='x', a, b)) from t;`).Check(testkit.Rows("2", "2")) + tk.MustQuery(`select collation(lag(b, 1, 'B') over w) from t window w as (order by b);`).Check(testkit.Rows("utf8mb4_general_ci", "utf8mb4_general_ci")) + tk.MustQuery(`select coercibility(lag(b, 1, 'B') over w) from t window w as (order by b);`).Check(testkit.Rows("2", "2")) +} + +func TestIssue11177(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustQuery("SELECT 'lvuleck' BETWEEN '2008-09-16 22:23:50' AND 0;").Check(testkit.Rows("0")) + tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: 'lvuleck'", "Warning 1292 Truncated incorrect DOUBLE value: '2008-09-16 22:23:50'")) + tk.MustQuery("SELECT 'aa' BETWEEN 'bb' AND 0;").Check(testkit.Rows("1")) + tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: 'aa'", "Warning 1292 Truncated incorrect DOUBLE value: 'bb'")) + tk.MustQuery("select 1 between 0 and b'110';").Check(testkit.Rows("1")) + tk.MustQuery("show warnings;").Check(testkit.Rows()) + tk.MustQuery("select 'b' between 'a' and b'110';").Check(testkit.Rows("0")) + tk.MustQuery("show warnings;").Check(testkit.Rows()) +} + +func TestIssue19804(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test;`) + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a set('a', 'b', 'c'));`) + tk.MustGetErrMsg("alter table t change a a set('a', 'b', 'c', 'c');", "[types:1291]Column 'a' has duplicated value 'c' in SET") + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a enum('a', 'b', 'c'));`) + tk.MustGetErrMsg("alter table t change a a enum('a', 'b', 'c', 'c');", "[types:1291]Column 'a' has duplicated value 'c' in ENUM") + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a set('a', 'b', 'c'));`) + tk.MustExec(`alter table t change a a set('a', 'b', 'c', 'd');`) + tk.MustExec(`insert into t values('d');`) + tk.MustGetErrMsg(`alter table t change a a set('a', 'b', 'c', 'e', 'f');`, "[types:1265]Data truncated for column 'a', value is 'd'") +} + +func TestIssue20209(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test;`) + tk.MustExec(`set @@character_set_client=utf8mb4;`) + tk.MustExec(`set @@collation_connection=utf8_bin;`) + tk.MustExec("CREATE VIEW tview_1 AS SELECT 'a' AS `id`;") +} + +func TestIssue18949(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test;`) + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a enum('a ', 'b\t', ' c '), b set('a ', 'b\t', ' c '));`) + result := tk.MustQuery("show create table t").Rows()[0][1] + require.Regexp(t, `(?s).*enum\('a','b ',' c'\).*set\('a','b ',' c'\).*`, result) + tk.MustExec(`alter table t change a aa enum('a ', 'b\t', ' c ');`) + result = tk.MustQuery("show create table t").Rows()[0][1] + require.Regexp(t, `(?s).*enum\('a','b ',' c'\).*set\('a','b ',' c'\).*`, result) +} + +func TestClusteredIndexAndNewCollationIndexEncodeDecodeV5(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("create table t(a int, b char(10) collate utf8mb4_bin, c char(10) collate utf8mb4_general_ci," + + "d varchar(10) collate utf8mb4_bin, e varchar(10) collate utf8mb4_general_ci, f char(10) collate utf8mb4_unicode_ci, g varchar(10) collate utf8mb4_unicode_ci, " + + "primary key(a, b, c, d, e, f, g), key a(a), unique key ua(a), key b(b), unique key ub(b), key c(c), unique key uc(c)," + + "key d(d), unique key ud(d),key e(e), unique key ue(e), key f(f), key g(g), unique key uf(f), unique key ug(g))") + + tk.MustExec("insert into t values (1, '啊 ', '啊 ', '啊 ', '啊 ', '啊 ', '啊 ')") + // Single Read. + tk.MustQuery("select * from t ").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + + tk.MustQuery("select * from t use index(a)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ua)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(b)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ub)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(c)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(uc)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(d)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ud)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(e)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ue)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(f)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(uf)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(g)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + tk.MustQuery("select * from t use index(ug)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) + + tk.MustExec("alter table t add column h varchar(10) collate utf8mb4_general_ci default '🐸'") + tk.MustExec("alter table t add column i varchar(10) collate utf8mb4_general_ci default '🐸'") + tk.MustExec("alter table t add index h(h)") + tk.MustExec("alter table t add unique index uh(h)") + + tk.MustQuery("select * from t use index(h)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(uh)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + + // Double read. + tk.MustQuery("select * from t use index(a)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(ua)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(b)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(ub)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(c)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(uc)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(d)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(ud)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(e)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustQuery("select * from t use index(ue)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) + tk.MustExec("admin check table t") + tk.MustExec("admin recover index t a") + tk.MustExec("alter table t add column n char(10) COLLATE utf8mb4_unicode_ci") + tk.MustExec("alter table t add index n(n)") + tk.MustExec("update t set n = '吧';") + tk.MustQuery("select * from t").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸,吧")) + tk.MustQuery("select * from t use index(n)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸,吧")) + tk.MustExec("admin check table t") + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a varchar(255) COLLATE utf8_general_ci primary key clustered, b int) partition by range columns(a) " + + "(partition p0 values less than ('0'), partition p1 values less than MAXVALUE);") + tk.MustExec("alter table t add index b(b);") + tk.MustExec("insert into t values ('0', 1);") + tk.MustQuery("select * from t use index(b);").Check(testkit.Rows("0 1")) + tk.MustQuery("select * from t use index();").Check(testkit.Rows("0 1")) + tk.MustExec("admin check table t") +} + +func TestClusteredIndexAndNewCollation(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("CREATE TABLE `t` (" + + "`a` char(10) COLLATE utf8mb4_unicode_ci NOT NULL," + + "`b` char(20) COLLATE utf8mb4_general_ci NOT NULL," + + "`c` int(11) NOT NULL," + + "PRIMARY KEY (`a`,`b`,`c`)," + + "KEY `idx` (`a`))") + + tk.MustExec("begin") + tk.MustExec("insert into t values ('a6', 'b6', 3)") + tk.MustQuery("select * from t").Check(testkit.Rows("a6 b6 3")) + tk.MustQuery("select * from t where a='a6'").Check(testkit.Rows("a6 b6 3")) + tk.MustExec("delete from t") + tk.MustQuery("select * from t").Check(testkit.Rows()) + tk.MustExec("commit") + tk.MustQuery("select * from t").Check(testkit.Rows()) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(`a` char(10) COLLATE utf8mb4_unicode_ci NOT NULL key)") + tk.MustExec("insert into t values ('&');") + tk.MustExec("replace into t values ('&');") + tk.MustQuery("select * from t").Check(testkit.Rows("&")) +} + +func TestIssue20608(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustQuery("select '䇇Հ' collate utf8mb4_bin like '___Հ';").Check(testkit.Rows("0")) +} + +func TestIssue20161(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test;`) + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(raw JSON);`) + tk.MustExec(`insert into t(raw) values('["a","ab"]'), ('["a"]'), (null);`) + tk.MustQuery(`SELECT JSON_SEARCH(raw,'one','c') FROM t;`). + Check(testkit.Rows("", "", "")) +} + +func TestCollationIndexJoin(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b char(10), key(b)) collate utf8mb4_general_ci") + tk.MustExec("create table t2(a int, b char(10), key(b)) collate ascii_bin") + tk.MustExec("insert into t1 values (1, 'a')") + tk.MustExec("insert into t2 values (1, 'A')") + + tk.MustQuery("select /*+ inl_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) + tk.MustQuery("select /*+ hash_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) + tk.MustQuery("select /*+ merge_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) + tk.MustQuery("select /*+ inl_hash_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) + tk.MustQuery("select /*+ inl_hash_join(t2) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Optimizer Hint /*+ INL_HASH_JOIN(t2) */ is inapplicable")) + tk.MustQuery("select /*+ inl_merge_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) + tk.MustQuery("select /*+ inl_merge_join(t2) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Optimizer Hint /*+ INL_MERGE_JOIN(t2) */ is inapplicable")) + + tk.MustExec("drop table if exists a, b") + tk.MustExec("create table a(i int, k varbinary(40), v int, primary key(i, k) clustered)") + tk.MustExec("create table b(i int, k varchar(40), v int, primary key(i, k) clustered)") + tk.MustExec("insert into a select 3, 'nice mccarthy', 10") + tk.MustQuery("select * from a, b where a.i = b.i and a.k = b.k").Check(testkit.Rows()) + + tk.MustExec("drop table if exists a, b") + tk.MustExec("create table a(i int NOT NULL, k varbinary(40) NOT NULL, v int, key idx1(i, k))") + tk.MustExec("create table b(i int NOT NULL, k varchar(40) NOT NULL, v int, key idx1(i, k))") + tk.MustExec("insert into a select 3, 'nice mccarthy', 10") + tk.MustQuery(" select /*+ inl_join(b) */ b.i from a, b where a.i = b.i and a.k = b.k").Check(testkit.Rows()) +} + +func TestCollationMergeJoin(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE `t` (" + + " `col_10` blob DEFAULT NULL," + + " `col_11` decimal(17,5) NOT NULL," + + " `col_13` varchar(381) COLLATE utf8mb4_unicode_ci NOT NULL DEFAULT 'Yr'," + + " PRIMARY KEY (`col_13`,`col_11`) CLUSTERED," + + " KEY `idx_4` (`col_10`(3))" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") + tk.MustExec("insert into t values ('a', 12523, 'A');") + tk.MustExec("insert into t values ('A', 2, 'a');") + tk.MustExec("insert into t values ('a', 23, 'A');") + tk.MustExec("insert into t values ('a', 23, 'h2');") + tk.MustExec("insert into t values ('a', 23, 'h3');") + tk.MustExec("insert into t values ('a', 23, 'h4');") + tk.MustExec("insert into t values ('a', 23, 'h5');") + tk.MustExec("insert into t values ('a', 23, 'h6');") + tk.MustExec("insert into t values ('a', 23, 'h7');") + tk.MustQuery("select /*+ MERGE_JOIN(t) */ t.* from t where col_13 in ( select col_10 from t where t.col_13 in ( 'a', 'b' ) ) order by col_10 ;").Check( + testkit.Rows("\x41 2.00000 a", "\x61 23.00000 A", "\x61 12523.00000 A")) +} + +func TestIssue20876(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("drop table if exists t;") + tk.MustExec("CREATE TABLE `t` (" + + " `a` char(10) COLLATE utf8mb4_unicode_ci NOT NULL," + + " `b` char(20) COLLATE utf8mb4_general_ci NOT NULL," + + " `c` int(11) NOT NULL," + + " PRIMARY KEY (`a`,`b`,`c`)," + + " KEY `idx` (`a`)" + + ")") + tk.MustExec("insert into t values ('#', 'C', 10), ('$', 'c', 20), ('$', 'c', 30), ('a', 'a', 10), ('A', 'A', 30)") + tk.MustExec("analyze table t") + tk.MustQuery("select * from t where a='#';").Check(testkit.Rows("# C 10")) +} + +func TestLikeWithCollation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk.MustQuery(`select 'a' like 'A' collate utf8mb4_unicode_ci;`).Check(testkit.Rows("1")) + tk.MustGetErrMsg(`select 'a' collate utf8mb4_bin like 'A' collate utf8mb4_unicode_ci;`, "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_unicode_ci,EXPLICIT) for operation 'like'") + tk.MustQuery(`select '😛' collate utf8mb4_general_ci like '😋';`).Check(testkit.Rows("1")) + tk.MustQuery(`select '😛' collate utf8mb4_general_ci = '😋';`).Check(testkit.Rows("1")) + tk.MustQuery(`select '😛' collate utf8mb4_unicode_ci like '😋';`).Check(testkit.Rows("0")) + tk.MustQuery(`select '😛' collate utf8mb4_unicode_ci = '😋';`).Check(testkit.Rows("1")) +} + +func TestCollationUnion(t *testing.T) { + // For issue 19694. + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + + tk.MustQuery("select cast('2010-09-09' as date) a union select '2010-09-09 ' order by a;").Check(testkit.Rows("2010-09-09", "2010-09-09 ")) + res := tk.MustQuery("select cast('2010-09-09' as date) a union select '2010-09-09 ';") + require.Len(t, res.Rows(), 2) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + res = tk.MustQuery("select cast('2010-09-09' as date) a union select '2010-09-09 ';") + require.Len(t, res.Rows(), 1) +} + +func TestCollationPrefixClusteredIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (k char(20), v int, primary key (k(4)) clustered, key (k)) collate utf8mb4_general_ci;") + tk.MustExec("insert into t values('01233', 1);") + tk.MustExec("create index idx on t(k(2))") + tk.MustQuery("select * from t use index(k_2);").Check(testkit.Rows("01233 1")) + tk.MustQuery("select * from t use index(idx);").Check(testkit.Rows("01233 1")) + tk.MustExec("admin check table t;") +} + +func TestIssue23805(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk.MustExec("CREATE TABLE `tbl_5` (" + + " `col_25` time NOT NULL DEFAULT '05:35:58'," + + " `col_26` blob NOT NULL," + + " `col_27` double NOT NULL," + + " `col_28` char(83) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL," + + " `col_29` timestamp NOT NULL," + + " `col_30` varchar(36) COLLATE utf8mb4_general_ci NOT NULL DEFAULT 'ywzIn'," + + " `col_31` binary(85) DEFAULT 'OIstcXsGmAyc'," + + " `col_32` datetime NOT NULL DEFAULT '2024-08-02 00:00:00'," + + " PRIMARY KEY (`col_26`(3),`col_27`) /*T![clustered_index] CLUSTERED */," + + " UNIQUE KEY `idx_10` (`col_26`(5)));") + tk.MustExec("insert ignore into tbl_5 set col_28 = 'ZmZIdSnq' , col_25 = '18:50:52.00' on duplicate key update col_26 = 'y';\n") +} + +func TestIssue26662(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(a varchar(36) NOT NULL) ENGINE = InnoDB DEFAULT CHARSET = utf8 COLLATE = utf8_general_ci;") + tk.MustExec("set names utf8;") + tk.MustQuery("select t2.b from (select t1.a as b from t1 union all select t1.a as b from t1) t2 where case when (t2.b is not null) then t2.b else '' end > '1234567';"). + Check(testkit.Rows()) +} + +func TestCollationForBinaryLiteral(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE t (`COL1` tinyblob NOT NULL, `COL2` binary(1) NOT NULL, `COL3` bigint(11) NOT NULL, PRIMARY KEY (`COL1`(5),`COL2`,`COL3`) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("insert into t values(0x1E,0xEC,6966939640596047133);") + tk.MustQuery("select * from t where col1 not in (0x1B,0x20) order by col1").Check(testkit.Rows("\x1e \xec 6966939640596047133")) + tk.MustExec("drop table t") +} diff --git a/expression/integration_test.go b/expression/integration_test.go index 7f01c659c126d..bbf0961e63509 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -49,7 +49,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/sem" @@ -3979,6 +3978,8 @@ func (s *testIntegrationSuite) TestCompareBuiltin(c *C) { result.Check(testkit.Rows("1 1 1")) result = tk.MustQuery(`select INTERVAL(100, NULL, NULL, NULL, NULL, NULL, 100);`) result.Check(testkit.Rows("6")) + result = tk.MustQuery(`SELECT INTERVAL(0,(1*5)/2) + INTERVAL(5,4,3);`) + result.Check(testkit.Rows("2")) // for greatest result = tk.MustQuery(`select greatest(1, 2, 3), greatest("a", "b", "c"), greatest(1.1, 1.2, 1.3), greatest("123a", 1, 2)`) @@ -6580,7 +6581,7 @@ func (s *testIntegrationSerialSuite) TestCacheConstEval(c *C) { tk.MustExec("admin reload expr_pushdown_blacklist") } -func (s *testSuite) TestIssue20071(c *C) { +func (s *testIntegrationSuite) TestIssue20071(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists table_30_utf8_4") tk.MustExec("drop table if exists t") @@ -6593,7 +6594,7 @@ func (s *testSuite) TestIssue20071(c *C) { tk.MustExec("select a from table_30_utf8_4 order by a") } -func (s *testSuite) TestVirtualGeneratedColumnAndLimit(c *C) { +func (s *testIntegrationSuite) TestVirtualGeneratedColumnAndLimit(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int, b int as (a + 1));") @@ -6602,242 +6603,6 @@ func (s *testSuite) TestVirtualGeneratedColumnAndLimit(c *C) { tk.MustQuery("select /*+ LIMIT_TO_COP() */ b from t order by b limit 1;").Check(testkit.Rows("2")) } -func (s *testIntegrationSerialSuite) TestCollationBasic(c *C) { - tk := testkit.NewTestKit(c, s.store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk.MustExec("use test") - tk.MustExec("create table t_ci(a varchar(10) collate utf8mb4_general_ci, unique key(a))") - tk.MustExec("insert into t_ci values ('a')") - tk.MustQuery("select * from t_ci").Check(testkit.Rows("a")) - tk.MustQuery("select * from t_ci").Check(testkit.Rows("a")) - tk.MustQuery("select * from t_ci where a='a'").Check(testkit.Rows("a")) - tk.MustQuery("select * from t_ci where a='A'").Check(testkit.Rows("a")) - tk.MustQuery("select * from t_ci where a='a '").Check(testkit.Rows("a")) - tk.MustQuery("select * from t_ci where a='a '").Check(testkit.Rows("a")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a varchar(10) primary key,b int)") - tk.MustExec("insert into t values ('a', 1), ('b', 3), ('a', 2) on duplicate key update b = b + 1;") - tk.MustExec("set autocommit=0") - tk.MustExec("insert into t values ('a', 1), ('b', 3), ('a', 2) on duplicate key update b = b + 1;") - tk.MustQuery("select * from t").Check(testkit.Rows("a 4", "b 4")) - tk.MustExec("set autocommit=1") - tk.MustQuery("select * from t").Check(testkit.Rows("a 4", "b 4")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a varchar(10),b int, key tk (a))") - tk.MustExec("insert into t values ('', 1), ('', 3)") - tk.MustExec("set autocommit=0") - tk.MustExec("update t set b = b + 1") - tk.MustQuery("select * from t").Check(testkit.Rows(" 2", " 4")) - tk.MustExec("set autocommit=1") - tk.MustQuery("select * from t").Check(testkit.Rows(" 2", " 4")) - - tk.MustExec("drop table t_ci") - tk.MustExec("create table t_ci(id bigint primary key, a varchar(10) collate utf8mb4_general_ci, unique key(a, id))") - tk.MustExec("insert into t_ci values (1, 'a')") - tk.MustQuery("select a from t_ci").Check(testkit.Rows("a")) - tk.MustQuery("select a from t_ci").Check(testkit.Rows("a")) - tk.MustQuery("select a from t_ci where a='a'").Check(testkit.Rows("a")) - tk.MustQuery("select a from t_ci where a='A'").Check(testkit.Rows("a")) - tk.MustQuery("select a from t_ci where a='a '").Check(testkit.Rows("a")) - tk.MustQuery("select a from t_ci where a='a '").Check(testkit.Rows("a")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c set('A', 'B') collate utf8mb4_general_ci);") - tk.MustExec("insert into t values('a');") - tk.MustExec("insert into t values('B');") - tk.MustQuery("select c from t where c = 'a';").Check(testkit.Rows("A")) - tk.MustQuery("select c from t where c = 'A';").Check(testkit.Rows("A")) - tk.MustQuery("select c from t where c = 'b';").Check(testkit.Rows("B")) - tk.MustQuery("select c from t where c = 'B';").Check(testkit.Rows("B")) - - tk.MustExec("drop table if exists t1") - tk.MustExec("CREATE TABLE `t1` (" + - " `COL1` varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci NOT NULL," + - " PRIMARY KEY (`COL1`(5)) clustered" + - ")") - tk.MustExec("INSERT INTO `t1` VALUES ('Ȇ');") - tk.MustQuery("select * from t1 where col1 not in (0xc484, 0xe5a4bc, 0xc3b3);").Check(testkit.Rows("Ȇ")) - tk.MustQuery("select * from t1 where col1 >= 0xc484 and col1 <= 0xc3b3;").Check(testkit.Rows("Ȇ")) - - tk.MustQuery("select collation(IF('a' < 'B' collate utf8mb4_general_ci, 'smaller', 'greater' collate utf8mb4_unicode_ci));").Check(testkit.Rows("utf8mb4_unicode_ci")) -} - -func (s *testIntegrationSerialSuite) TestWeightString(c *C) { - tk := testkit.NewTestKit(c, s.store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - type testCase struct { - input []string - result []string - resultAsChar1 []string - resultAsChar3 []string - resultAsBinary1 []string - resultAsBinary5 []string - resultExplicitCollateBin []string - } - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (id int, a varchar(20) collate utf8mb4_general_ci)") - cases := testCase{ - input: []string{"aAÁàãăâ", "a", "a ", "中", "中 "}, - result: []string{"\x00A\x00A\x00A\x00A\x00A\x00A\x00A", "\x00A", "\x00A", "\x4E\x2D", "\x4E\x2D"}, - resultAsChar1: []string{"\x00A", "\x00A", "\x00A", "\x4E\x2D", "\x4E\x2D"}, - resultAsChar3: []string{"\x00A\x00A\x00A", "\x00A", "\x00A", "\x4E\x2D", "\x4E\x2D"}, - resultAsBinary1: []string{"a", "a", "a", "\xE4", "\xE4"}, - resultAsBinary5: []string{"aA\xc3\x81\xc3", "a\x00\x00\x00\x00", "a \x00\x00", "中\x00\x00", "中 \x00"}, - resultExplicitCollateBin: []string{"aAÁàãăâ", "a", "a", "中", "中"}, - } - values := make([]string, len(cases.input)) - for i, input := range cases.input { - values[i] = fmt.Sprintf("(%d, '%s')", i, input) - } - tk.MustExec("insert into t values " + strings.Join(values, ",")) - rows := tk.MustQuery("select weight_string(a) from t order by id").Rows() - for i, out := range cases.result { - c.Assert(rows[i][0].(string), Equals, out) - } - rows = tk.MustQuery("select weight_string(a as char(1)) from t order by id").Rows() - for i, out := range cases.resultAsChar1 { - c.Assert(rows[i][0].(string), Equals, out) - } - rows = tk.MustQuery("select weight_string(a as char(3)) from t order by id").Rows() - for i, out := range cases.resultAsChar3 { - c.Assert(rows[i][0].(string), Equals, out) - } - rows = tk.MustQuery("select weight_string(a as binary(1)) from t order by id").Rows() - for i, out := range cases.resultAsBinary1 { - c.Assert(rows[i][0].(string), Equals, out) - } - rows = tk.MustQuery("select weight_string(a as binary(5)) from t order by id").Rows() - for i, out := range cases.resultAsBinary5 { - c.Assert(rows[i][0].(string), Equals, out) - } - c.Assert(tk.MustQuery("select weight_string(NULL);").Rows()[0][0], Equals, "") - c.Assert(tk.MustQuery("select weight_string(7);").Rows()[0][0], Equals, "") - c.Assert(tk.MustQuery("select weight_string(cast(7 as decimal(5)));").Rows()[0][0], Equals, "") - c.Assert(tk.MustQuery("select weight_string(cast(20190821 as date));").Rows()[0][0], Equals, "2019-08-21") - c.Assert(tk.MustQuery("select weight_string(cast(20190821 as date) as binary(5));").Rows()[0][0], Equals, "2019-") - c.Assert(tk.MustQuery("select weight_string(7.0);").Rows()[0][0], Equals, "") - c.Assert(tk.MustQuery("select weight_string(7 AS BINARY(2));").Rows()[0][0], Equals, "7\x00") - // test explicit collation - c.Assert(tk.MustQuery("select weight_string('中 ' collate utf8mb4_general_ci);").Rows()[0][0], Equals, "\x4E\x2D") - c.Assert(tk.MustQuery("select weight_string('中 ' collate utf8mb4_bin);").Rows()[0][0], Equals, "中") - c.Assert(tk.MustQuery("select weight_string('中 ' collate utf8mb4_unicode_ci);").Rows()[0][0], Equals, "\xFB\x40\xCE\x2D") - c.Assert(tk.MustQuery("select collation(a collate utf8mb4_general_ci) from t order by id").Rows()[0][0], Equals, "utf8mb4_general_ci") - c.Assert(tk.MustQuery("select collation('中 ' collate utf8mb4_general_ci);").Rows()[0][0], Equals, "utf8mb4_general_ci") - rows = tk.MustQuery("select weight_string(a collate utf8mb4_bin) from t order by id").Rows() - for i, out := range cases.resultExplicitCollateBin { - c.Assert(rows[i][0].(string), Equals, out) - } - tk.MustGetErrMsg("select weight_string(a collate utf8_general_ci) from t order by id", "[ddl:1253]COLLATION 'utf8_general_ci' is not valid for CHARACTER SET 'utf8mb4'") - tk.MustGetErrMsg("select weight_string('中' collate utf8_bin)", "[ddl:1253]COLLATION 'utf8_bin' is not valid for CHARACTER SET 'utf8mb4'") -} - -func (s *testIntegrationSerialSuite) TestCollationCreateIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a varchar(10) collate utf8mb4_general_ci);") - tk.MustExec("insert into t values ('a');") - tk.MustExec("insert into t values ('A');") - tk.MustExec("insert into t values ('b');") - tk.MustExec("insert into t values ('B');") - tk.MustExec("insert into t values ('a');") - tk.MustExec("insert into t values ('A');") - tk.MustExec("insert into t values ('ß');") - tk.MustExec("insert into t values ('sa');") - tk.MustExec("create index idx on t(a);") - tk.MustQuery("select * from t order by a").Check(testkit.Rows("a", "A", "a", "A", "b", "B", "ß", "sa")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a varchar(10) collate utf8mb4_unicode_ci);") - tk.MustExec("insert into t values ('a');") - tk.MustExec("insert into t values ('A');") - tk.MustExec("insert into t values ('b');") - tk.MustExec("insert into t values ('B');") - tk.MustExec("insert into t values ('a');") - tk.MustExec("insert into t values ('A');") - tk.MustExec("insert into t values ('ß');") - tk.MustExec("insert into t values ('sa');") - tk.MustExec("create index idx on t(a);") - tk.MustQuery("select * from t order by a").Check(testkit.Rows("a", "A", "a", "A", "b", "B", "sa", "ß")) -} - -func (s *testIntegrationSerialSuite) TestCollateConstantPropagation(c *C) { - tk := testkit.NewTestKit(c, s.store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a char(10) collate utf8mb4_bin, b char(10) collate utf8mb4_general_ci);") - tk.MustExec("insert into t values ('a', 'A');") - tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='a' collate utf8mb4_general_ci;").Check(nil) - tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b>='a' collate utf8mb4_general_ci;").Check(nil) - tk.MustExec("drop table t;") - tk.MustExec("create table t (a char(10) collate utf8mb4_general_ci, b char(10) collate utf8mb4_general_ci);") - tk.MustExec("insert into t values ('A', 'a');") - tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='a' collate utf8mb4_bin;").Check(testkit.Rows("A a A a")) - tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b>='a' collate utf8mb4_bin;").Check(testkit.Rows("A a A a")) - tk.MustExec("drop table t;") - tk.MustExec("set names utf8mb4") - tk.MustExec("create table t (a char(10) collate utf8mb4_general_ci, b char(10) collate utf8_general_ci);") - tk.MustExec("insert into t values ('a', 'A');") - tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='A'").Check(testkit.Rows("a A a A")) - tk.MustExec("drop table t;") - tk.MustExec("create table t(a char collate utf8_general_ci, b char collate utf8mb4_general_ci, c char collate utf8_bin);") - tk.MustExec("insert into t values ('b', 'B', 'B');") - tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b=t2.c;").Check(testkit.Rows("b B B b B B")) - tk.MustExec("drop table t;") - tk.MustExec("create table t(a char collate utf8_bin, b char collate utf8_general_ci);") - tk.MustExec("insert into t values ('a', 'A');") - tk.MustQuery("select * from t t1, t t2 where t1.b=t2.b and t2.b=t1.a collate utf8_general_ci;").Check(testkit.Rows("a A a A")) - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") - tk.MustExec("create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_general_ci;") - tk.MustExec("create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_bin;") - tk.MustExec("insert into t1 values ('A', 'a');") - tk.MustExec("insert into t2 values ('a', 'a')") - tk.MustQuery("select * from t1 left join t2 on t1.a = t2.a where t1.a = 'a';").Check(testkit.Rows("A a ")) - tk.MustExec("drop table t;") - tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") - tk.MustExec("create table t(a char collate utf8mb4_bin, b char collate utf8mb4_general_ci);") - tk.MustExec("insert into t values ('a', 'a');") - tk.MustQuery("select * from t t1, t t2 where t2.b = 'A' and lower(concat(t1.a , '' )) = t2.b;").Check(testkit.Rows("a a a a")) - tk.MustExec("drop table t;") - tk.MustExec("create table t(a char collate utf8_unicode_ci, b char collate utf8mb4_unicode_ci, c char collate utf8_bin);") - tk.MustExec("insert into t values ('b', 'B', 'B');") - tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b=t2.c;").Check(testkit.Rows("b B B b B B")) - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("set names utf8mb4 collate utf8mb4_unicode_ci;") - tk.MustExec("create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_unicode_ci;") - tk.MustExec("create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_bin;") - tk.MustExec("insert into t1 values ('A', 'a');") - tk.MustExec("insert into t2 values ('a', 'a')") - tk.MustQuery("select * from t1 left join t2 on t1.a = t2.a where t1.a = 'a';").Check(testkit.Rows("A a ")) - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") - tk.MustExec("create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_general_ci;") - tk.MustExec("create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_unicode_ci;") - tk.MustExec("insert into t1 values ('ß', 's');") - tk.MustExec("insert into t2 values ('s', 's')") - tk.MustQuery("select * from t1 left join t2 on t1.a = t2.a collate utf8mb4_unicode_ci where t1.a = 's';").Check(testkit.Rows("ß s ")) - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("create table t1(a char(10) collate utf8mb4_general_ci, index (a));") - tk.MustExec("create table t2(a char(10) collate utf8_bin, index (a));") - tk.MustExec("insert into t1 values ('a');") - tk.MustExec("insert into t2 values ('A');") - tk.MustExec("set names utf8 collate utf8_general_ci;") - tk.MustQuery("select * from t1, t2 where t1.a=t2.a and t1.a= 'a';").Check(testkit.Rows("a A")) - tk.MustQuery("select * from t1 where a='a' and a = 'A'").Check(testkit.Rows("a")) -} - func (s *testIntegrationSuite2) TestIssue17791(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -6853,517 +6618,6 @@ func (s *testIntegrationSuite2) TestIssue17791(c *C) { tk.MustQuery("SELECT id, pad, expr, NOT 1 BETWEEN -5 AND 5 as expr_in_select FROM t1;").Check(testkit.Rows("1 a 0 0", "2 b 0 0")) } -func (s *testIntegrationSerialSuite) TestMixCollation(c *C) { - tk := testkit.NewTestKit(c, s.store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk.MustGetErrMsg(`select 'a' collate utf8mb4_bin = 'a' collate utf8mb4_general_ci;`, "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation '='") - - tk.MustExec("use test;") - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t ( - mb4general varchar(10) charset utf8mb4 collate utf8mb4_general_ci, - mb4unicode varchar(10) charset utf8mb4 collate utf8mb4_unicode_ci, - mb4bin varchar(10) charset utf8mb4 collate utf8mb4_bin, - general varchar(10) charset utf8 collate utf8_general_ci, - unicode varchar(10) charset utf8 collate utf8_unicode_ci, - utfbin varchar(10) charset utf8 collate utf8_bin, - bin varchar(10) charset binary collate binary, - latin1_bin varchar(10) charset latin1 collate latin1_bin, - ascii_bin varchar(10) charset ascii collate ascii_bin, - i int - );`) - tk.MustExec("insert into t values ('s', 's', 's', 's', 's', 's', 's', 's', 's', 1);") - tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") - - tk.MustQuery("select * from t where mb4unicode = 's' collate utf8mb4_unicode_ci;").Check(testkit.Rows("s s s s s s s s s 1")) - tk.MustQuery(`select * from t t1, t t2 where t1.mb4unicode = t2.mb4general collate utf8mb4_general_ci;`).Check(testkit.Rows("s s s s s s s s s 1 s s s s s s s s s 1")) - tk.MustQuery(`select * from t t1, t t2 where t1.mb4general = t2.mb4unicode collate utf8mb4_general_ci;`).Check(testkit.Rows("s s s s s s s s s 1 s s s s s s s s s 1")) - tk.MustQuery(`select * from t t1, t t2 where t1.mb4general = t2.mb4unicode collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1 s s s s s s s s s 1")) - tk.MustQuery(`select * from t t1, t t2 where t1.mb4unicode = t2.mb4general collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1 s s s s s s s s s 1")) - tk.MustQuery(`select * from t where mb4general = mb4bin collate utf8mb4_general_ci;`).Check(testkit.Rows("s s s s s s s s s 1")) - tk.MustQuery(`select * from t where mb4unicode = mb4general collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1")) - tk.MustQuery(`select * from t where mb4general = mb4unicode collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1")) - tk.MustQuery(`select * from t where mb4unicode = 's' collate utf8mb4_unicode_ci;`).Check(testkit.Rows("s s s s s s s s s 1")) - tk.MustQuery("select * from t where mb4unicode = mb4bin;").Check(testkit.Rows("s s s s s s s s s 1")) - tk.MustQuery("select * from t where general = mb4unicode;").Check(testkit.Rows("s s s s s s s s s 1")) - tk.MustQuery("select * from t where unicode = mb4unicode;").Check(testkit.Rows("s s s s s s s s s 1")) - tk.MustQuery("select * from t where mb4unicode = mb4unicode;").Check(testkit.Rows("s s s s s s s s s 1")) - - tk.MustQuery("select collation(concat(mb4unicode, mb4general collate utf8mb4_unicode_ci)) from t;").Check(testkit.Rows("utf8mb4_unicode_ci")) - tk.MustQuery("select collation(concat(mb4general, mb4unicode, mb4bin)) from t;").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select coercibility(concat(mb4general, mb4unicode, mb4bin)) from t;").Check(testkit.Rows("1")) - tk.MustQuery("select collation(concat(mb4unicode, mb4bin, concat(mb4general))) from t;").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select coercibility(concat(mb4unicode, mb4bin)) from t;").Check(testkit.Rows("2")) - tk.MustQuery("select collation(concat(mb4unicode, mb4bin)) from t;").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select coercibility(concat(mb4bin, concat(mb4general))) from t;").Check(testkit.Rows("2")) - tk.MustQuery("select collation(concaT(mb4bin, cOncAt(mb4general))) from t;").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select coercibility(concat(mb4unicode, mb4bin, concat(mb4general))) from t;").Check(testkit.Rows("2")) - tk.MustQuery("select collation(concat(mb4unicode, mb4bin, concat(mb4general))) from t;").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select coercibility(concat(mb4unicode, mb4general)) from t;").Check(testkit.Rows("1")) - tk.MustQuery("select collation(coalesce(mb4unicode, mb4general)) from t;").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select coercibility(coalesce(mb4unicode, mb4general)) from t;").Check(testkit.Rows("1")) - tk.MustQuery("select collation(CONCAT(concat(mb4unicode), concat(mb4general))) from t;").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select coercibility(cONcat(unicode, general)) from t;").Check(testkit.Rows("1")) - tk.MustQuery("select collation(concAt(unicode, general)) from t;").Check(testkit.Rows("utf8_bin")) - tk.MustQuery("select collation(concat(bin, mb4general)) from t;").Check(testkit.Rows("binary")) - tk.MustQuery("select coercibility(concat(bin, mb4general)) from t;").Check(testkit.Rows("2")) - tk.MustQuery("select collation(concat(mb4unicode, ascii_bin)) from t;").Check(testkit.Rows("utf8mb4_unicode_ci")) - tk.MustQuery("select coercibility(concat(mb4unicode, ascii_bin)) from t;").Check(testkit.Rows("2")) - tk.MustQuery("select collation(concat(mb4unicode, mb4unicode)) from t;").Check(testkit.Rows("utf8mb4_unicode_ci")) - tk.MustQuery("select coercibility(concat(mb4unicode, mb4unicode)) from t;").Check(testkit.Rows("2")) - tk.MustQuery("select collation(concat(bin, bin)) from t;").Check(testkit.Rows("binary")) - tk.MustQuery("select coercibility(concat(bin, bin)) from t;").Check(testkit.Rows("2")) - tk.MustQuery("select collation(concat(latin1_bin, ascii_bin)) from t;").Check(testkit.Rows("latin1_bin")) - tk.MustQuery("select coercibility(concat(latin1_bin, ascii_bin)) from t;").Check(testkit.Rows("2")) - tk.MustQuery("select collation(concat(mb4unicode, bin)) from t;").Check(testkit.Rows("binary")) - tk.MustQuery("select coercibility(concat(mb4unicode, bin)) from t;").Check(testkit.Rows("2")) - tk.MustQuery("select collation(mb4general collate utf8mb4_unicode_ci) from t;").Check(testkit.Rows("utf8mb4_unicode_ci")) - tk.MustQuery("select coercibility(mb4general collate utf8mb4_unicode_ci) from t;").Check(testkit.Rows("0")) - tk.MustQuery("select collation(concat(concat(mb4unicode, mb4general), concat(unicode, general))) from t;").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select coercibility(concat(concat(mb4unicode, mb4general), concat(unicode, general))) from t;").Check(testkit.Rows("1")) - tk.MustQuery("select collation(concat(i, 1)) from t;").Check(testkit.Rows("utf8mb4_general_ci")) - tk.MustQuery("select coercibility(concat(i, 1)) from t;").Check(testkit.Rows("4")) - tk.MustQuery("select collation(concat(i, user())) from t;").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select coercibility(concat(i, user())) from t;").Check(testkit.Rows("3")) - tk.MustGetErrMsg("select * from t where mb4unicode = mb4general;", "[expression:1267]Illegal mix of collations (utf8mb4_unicode_ci,IMPLICIT) and (utf8mb4_general_ci,IMPLICIT) for operation '='") - tk.MustGetErrMsg("select * from t where unicode = general;", "[expression:1267]Illegal mix of collations (utf8_unicode_ci,IMPLICIT) and (utf8_general_ci,IMPLICIT) for operation '='") - tk.MustGetErrMsg("select concat(mb4general) = concat(mb4unicode) from t;", "[expression:1267]Illegal mix of collations (utf8mb4_general_ci,IMPLICIT) and (utf8mb4_unicode_ci,IMPLICIT) for operation '='") - tk.MustGetErrMsg("select * from t t1, t t2 where t1.mb4unicode = t2.mb4general;", "[expression:1267]Illegal mix of collations (utf8mb4_unicode_ci,IMPLICIT) and (utf8mb4_general_ci,IMPLICIT) for operation '='") - tk.MustGetErrMsg("select field('s', mb4general, mb4unicode, mb4bin) from t;", "[expression:1271]Illegal mix of collations for operation 'field'") - tk.MustGetErrMsg("select concat(mb4unicode, mb4general) = mb4unicode from t;", "[expression:1267]Illegal mix of collations (utf8mb4_bin,NONE) and (utf8mb4_unicode_ci,IMPLICIT) for operation '='") - - tk.MustExec("drop table t;") -} - -func (s *testIntegrationSerialSuite) prepare4Join(c *C) *testkit.TestKit { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("USE test") - tk.MustExec("drop table if exists t") - tk.MustExec("drop table if exists t_bin") - tk.MustExec("CREATE TABLE `t` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL)") - tk.MustExec("CREATE TABLE `t_bin` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET binary)") - tk.MustExec("insert into t values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") - tk.MustExec("insert into t_bin values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") - return tk -} - -func (s *testIntegrationSerialSuite) prepare4Join2(c *C) *testkit.TestKit { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("USE test") - tk.MustExec("drop table if exists t1") - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t1 (id int, v varchar(5) character set binary, key(v))") - tk.MustExec("create table t2 (v varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, key(v))") - tk.MustExec("insert into t1 values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") - tk.MustExec("insert into t2 values ('a'), ('À'), ('á'), ('à'), ('b'), ('c'), (' ')") - return tk -} - -func (s *testIntegrationSerialSuite) TestCollateHashJoin(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Join(c) - tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b order by t1.a").Check( - testkit.Rows("1 a", "1 a", "1 a", "1 a", "2 À", "2 À", "2 À", "2 À", "3 á", "3 á", "3 á", "3 á", "4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b order by t1.a").Check( - testkit.Rows("1 a", "2 À", "3 á", "4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( - testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( - testkit.Rows("4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( - testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( - testkit.Rows("4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( - testkit.Rows("2 À", "3 á", "3 á", "4 à", "4 à", "4 à")) - tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( - testkit.Rows()) -} - -func (s *testIntegrationSerialSuite) TestCollateHashJoin2(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Join2(c) - tk.MustQuery("select /*+ TIDB_HJ(t1, t2) */ * from t1, t2 where t1.v=t2.v order by t1.id").Check( - testkit.Rows("1 a a", "2 À À", "3 á á", "4 à à", "5 b b", "6 c c", "7 ")) -} - -func (s *testIntegrationSerialSuite) TestCollateMergeJoin(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Join(c) - tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b order by t1.a").Check( - testkit.Rows("1 a", "1 a", "1 a", "1 a", "2 À", "2 À", "2 À", "2 À", "3 á", "3 á", "3 á", "3 á", "4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b order by t1.a").Check( - testkit.Rows("1 a", "2 À", "3 á", "4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( - testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( - testkit.Rows("4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( - testkit.Rows("4 à", "4 à", "4 à", "4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>3 order by t1.a").Check( - testkit.Rows("4 à", "5 b", "6 c", "7 ")) - tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t t1, t t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( - testkit.Rows("2 À", "3 á", "3 á", "4 à", "4 à", "4 à")) - tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ t1.a, t1.b from t_bin t1, t_bin t2 where t1.b=t2.b and t1.a>t2.a order by t1.a").Check( - testkit.Rows()) -} - -func (s *testIntegrationSerialSuite) TestCollateMergeJoin2(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Join2(c) - tk.MustQuery("select /*+ TIDB_SMJ(t1, t2) */ * from t1, t2 where t1.v=t2.v order by t1.id").Check( - testkit.Rows("1 a a", "2 À À", "3 á á", "4 à à", "5 b b", "6 c c", "7 ")) -} - -func (s *testIntegrationSerialSuite) TestCollateIndexMergeJoin(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, b varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, key(a), key(b))") - tk.MustExec("insert into t values ('a', 'x'), ('x', 'À'), ('á', 'x'), ('à', 'à'), ('à', 'x')") - - tk.MustExec("set tidb_enable_index_merge=1") - tk.MustQuery("select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 'a' or b = 'a'").Sort().Check( - testkit.Rows("a x", "x À", "à x", "à à", "á x")) -} - -func (s *testIntegrationSerialSuite) prepare4Collation(c *C, hasIndex bool) *testkit.TestKit { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("USE test") - tk.MustExec("drop table if exists t") - tk.MustExec("drop table if exists t_bin") - idxSQL := ", key(v)" - if !hasIndex { - idxSQL = "" - } - tk.MustExec(fmt.Sprintf("create table t (id int, v varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL %v)", idxSQL)) - tk.MustExec(fmt.Sprintf("create table t_bin (id int, v varchar(5) CHARACTER SET binary %v)", idxSQL)) - tk.MustExec("insert into t values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") - tk.MustExec("insert into t_bin values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") - return tk -} - -func (s *testIntegrationSerialSuite) TestCollateSelection(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Collation(c, false) - tk.MustQuery("select v from t where v='a' order by id").Check(testkit.Rows("a", "À", "á", "à")) - tk.MustQuery("select v from t_bin where v='a' order by id").Check(testkit.Rows("a")) - tk.MustQuery("select v from t where v<'b' and id<=3").Check(testkit.Rows("a", "À", "á")) - tk.MustQuery("select v from t_bin where v<'b' and id<=3").Check(testkit.Rows("a")) -} - -func (s *testIntegrationSerialSuite) TestCollateSort(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Collation(c, false) - tk.MustQuery("select id from t order by v, id").Check(testkit.Rows("7", "1", "2", "3", "4", "5", "6")) - tk.MustQuery("select id from t_bin order by v, id").Check(testkit.Rows("7", "1", "5", "6", "2", "4", "3")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a char(10) collate utf8mb4_general_ci, key(a))") - tk.MustExec("insert into t values ('a'), ('A'), ('b')") - tk.MustExec("insert into t values ('a'), ('A'), ('b')") - tk.MustExec("insert into t values ('a'), ('A'), ('b')") - tk.MustQuery("select * from t order by a collate utf8mb4_bin").Check(testkit.Rows("A", "A", "A", "a", "a", "a", "b", "b", "b")) - tk.MustQuery("select * from t order by a collate utf8mb4_general_ci").Check(testkit.Rows("a", "A", "a", "A", "a", "A", "b", "b", "b")) - tk.MustQuery("select * from t order by a collate utf8mb4_unicode_ci").Check(testkit.Rows("a", "A", "a", "A", "a", "A", "b", "b", "b")) -} - -func (s *testIntegrationSerialSuite) TestCollateHashAgg(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Collation(c, false) - tk.HasPlan("select distinct(v) from t_bin", "HashAgg") - tk.MustQuery("select distinct(v) from t_bin").Sort().Check(testkit.Rows(" ", "a", "b", "c", "À", "à", "á")) - tk.HasPlan("select distinct(v) from t", "HashAgg") - tk.MustQuery("select distinct(v) from t").Sort().Check(testkit.Rows(" ", "a", "b", "c")) - tk.HasPlan("select v, count(*) from t_bin group by v", "HashAgg") - tk.MustQuery("select v, count(*) from t_bin group by v").Sort().Check(testkit.Rows(" 1", "a 1", "b 1", "c 1", "À 1", "à 1", "á 1")) - tk.HasPlan("select v, count(*) from t group by v", "HashAgg") - tk.MustQuery("select v, count(*) from t group by v").Sort().Check(testkit.Rows(" 1", "a 4", "b 1", "c 1")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a char(10) collate utf8mb4_general_ci, key(a))") - tk.MustExec("insert into t values ('a'), ('A'), ('b')") - tk.MustExec("insert into t values ('a'), ('A'), ('b')") - tk.MustExec("insert into t values ('a'), ('A'), ('b')") - tk.MustExec("insert into t values ('s'), ('ss'), ('ß')") - tk.MustQuery("select count(1) from t group by a collate utf8mb4_bin order by a collate utf8mb4_bin").Check(testkit.Rows("3", "3", "3", "1", "1", "1")) - tk.MustQuery("select count(1) from t group by a collate utf8mb4_unicode_ci order by a collate utf8mb4_unicode_ci").Check(testkit.Rows("6", "3", "1", "2")) - tk.MustQuery("select count(1) from t group by a collate utf8mb4_general_ci order by a collate utf8mb4_general_ci").Check(testkit.Rows("6", "3", "2", "1")) -} - -func (s *testIntegrationSerialSuite) TestCollateStreamAgg(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Collation(c, true) - tk.HasPlan("select distinct(v) from t_bin", "StreamAgg") - tk.MustQuery("select distinct(v) from t_bin").Sort().Check(testkit.Rows(" ", "a", "b", "c", "À", "à", "á")) - tk.HasPlan("select distinct(v) from t", "StreamAgg") - tk.MustQuery("select distinct(v) from t").Sort().Check(testkit.Rows(" ", "a", "b", "c")) - tk.HasPlan("select v, count(*) from t_bin group by v", "StreamAgg") - tk.MustQuery("select v, count(*) from t_bin group by v").Sort().Check(testkit.Rows(" 1", "a 1", "b 1", "c 1", "À 1", "à 1", "á 1")) - tk.HasPlan("select v, count(*) from t group by v", "StreamAgg") - tk.MustQuery("select v, count(*) from t group by v").Sort().Check(testkit.Rows(" 1", "a 4", "b 1", "c 1")) -} - -func (s *testIntegrationSerialSuite) TestCollateIndexReader(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Collation(c, true) - tk.HasPlan("select v from t where v < 'b' order by v", "IndexReader") - tk.MustQuery("select v from t where v < 'b' order by v").Check(testkit.Rows(" ", "a", "À", "á", "à")) - tk.HasPlan("select v from t where v < 'b' and v > ' ' order by v", "IndexReader") - tk.MustQuery("select v from t where v < 'b' and v > ' ' order by v").Check(testkit.Rows("a", "À", "á", "à")) - tk.HasPlan("select v from t_bin where v < 'b' order by v", "IndexReader") - tk.MustQuery("select v from t_bin where v < 'b' order by v").Sort().Check(testkit.Rows(" ", "a")) - tk.HasPlan("select v from t_bin where v < 'b' and v > ' ' order by v", "IndexReader") - tk.MustQuery("select v from t_bin where v < 'b' and v > ' ' order by v").Sort().Check(testkit.Rows("a")) -} - -func (s *testIntegrationSerialSuite) TestCollateIndexLookup(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Collation(c, true) - - tk.HasPlan("select id from t where v < 'b'", "IndexLookUp") - tk.MustQuery("select id from t where v < 'b'").Sort().Check(testkit.Rows("1", "2", "3", "4", "7")) - tk.HasPlan("select id from t where v < 'b' and v > ' '", "IndexLookUp") - tk.MustQuery("select id from t where v < 'b' and v > ' '").Sort().Check(testkit.Rows("1", "2", "3", "4")) - tk.HasPlan("select id from t_bin where v < 'b'", "IndexLookUp") - tk.MustQuery("select id from t_bin where v < 'b'").Sort().Check(testkit.Rows("1", "7")) - tk.HasPlan("select id from t_bin where v < 'b' and v > ' '", "IndexLookUp") - tk.MustQuery("select id from t_bin where v < 'b' and v > ' '").Sort().Check(testkit.Rows("1")) -} - -func (s *testIntegrationSerialSuite) TestIssue16668(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists tx") - tk.MustExec("CREATE TABLE `tx` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL)") - tk.MustExec("insert into tx values (1, 'a'), (2, 'À'), (3, 'á'), (4, 'à'), (5, 'b'), (6, 'c'), (7, ' ')") - tk.MustQuery("select count(distinct(b)) from tx").Check(testkit.Rows("4")) -} - -func (s *testIntegrationSerialSuite) TestIssue27091(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists tx") - tk.MustExec("CREATE TABLE `tx` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL, `c` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin DEFAULT NULL)") - tk.MustExec("insert into tx values (1, 'a', 'a'), (2, 'A ', 'a '), (3, 'A', 'A'), (4, 'a ', 'A ')") - tk.MustQuery("select count(distinct b) from tx").Check(testkit.Rows("1")) - tk.MustQuery("select count(distinct c) from tx").Check(testkit.Rows("2")) - tk.MustQuery("select count(distinct b, c) from tx where a < 3").Check(testkit.Rows("1")) - tk.MustQuery("select approx_count_distinct(b) from tx").Check(testkit.Rows("1")) - tk.MustQuery("select approx_count_distinct(c) from tx").Check(testkit.Rows("2")) - tk.MustQuery("select approx_count_distinct(b, c) from tx where a < 3").Check(testkit.Rows("1")) -} - -func (s *testIntegrationSerialSuite) TestCollateStringFunction(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - - tk.MustQuery("select field('a', 'b', 'a');").Check(testkit.Rows("2")) - tk.MustQuery("select field('a', 'b', 'A');").Check(testkit.Rows("0")) - tk.MustQuery("select field('a', 'b', 'A' collate utf8mb4_bin);").Check(testkit.Rows("0")) - tk.MustQuery("select field('a', 'b', 'a ' collate utf8mb4_bin);").Check(testkit.Rows("2")) - tk.MustQuery("select field('a', 'b', 'A' collate utf8mb4_unicode_ci);").Check(testkit.Rows("2")) - tk.MustQuery("select field('a', 'b', 'a ' collate utf8mb4_unicode_ci);").Check(testkit.Rows("2")) - tk.MustQuery("select field('a', 'b', 'A' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) - tk.MustQuery("select field('a', 'b', 'a ' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) - - tk.MustExec("USE test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a char(10), b char (10)) collate utf8mb4_general_ci") - tk.MustExec("insert into t values ('a', 'A')") - tk.MustQuery("select field(a, b) from t").Check(testkit.Rows("1")) - - tk.MustQuery("select FIND_IN_SET('a','b,a,c,d');").Check(testkit.Rows("2")) - tk.MustQuery("select FIND_IN_SET('a','b,A,c,d');").Check(testkit.Rows("0")) - tk.MustQuery("select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_bin);").Check(testkit.Rows("0")) - tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_bin);").Check(testkit.Rows("2")) - tk.MustQuery("select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) - tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) - - tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") - tk.MustQuery("select collation(cast('a' as char));").Check(testkit.Rows("utf8mb4_general_ci")) - tk.MustQuery("select collation(cast('a' as binary));").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(cast('a' collate utf8mb4_bin as char));").Check(testkit.Rows("utf8mb4_general_ci")) - tk.MustQuery("select collation(cast('a' collate utf8mb4_bin as binary));").Check(testkit.Rows("binary")) - - tk.MustQuery("select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_unicode_ci);").Check(testkit.Rows("2")) - tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_unicode_ci);").Check(testkit.Rows("2")) - - tk.MustExec("select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_bin);") - tk.MustGetErrMsg("select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci);", "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation 'concat'") - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a char)") - tk.MustGetErrMsg("select * from t t1 join t t2 on t1.a collate utf8mb4_bin = t2.a collate utf8mb4_general_ci;", "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation '='") - - tk.MustExec("DROP TABLE IF EXISTS t1;") - tk.MustExec("CREATE TABLE t1 ( a int, p1 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin,p2 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_general_ci , p3 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin,p4 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci ,n1 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_bin,n2 VARCHAR(255) CHARACTER SET utf8 COLLATE utf8_general_ci , n3 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin,n4 VARCHAR(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci );") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values(1,' 0aA1!测试テストמבחן ',' 0aA1!测试テストמבחן ',' 0aA1!测试テストמבחן ',' 0aA1!测试テストמבחן ',' 0Aa1!测试テストמבחן ',' 0Aa1!测试テストמבחן ',' 0Aa1!测试テストמבחן ',' 0Aa1!测试テストמבחן ');") - - tk.MustQuery("select INSTR(p1,n1) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p1,n2) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p1,n3) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p1,n4) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p2,n1) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p2,n2) from t1;").Check(testkit.Rows("1")) - tk.MustQuery("select INSTR(p2,n3) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p2,n4) from t1;").Check(testkit.Rows("1")) - tk.MustQuery("select INSTR(p3,n1) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p3,n2) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p3,n3) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p3,n4) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p4,n1) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p4,n2) from t1;").Check(testkit.Rows("1")) - tk.MustQuery("select INSTR(p4,n3) from t1;").Check(testkit.Rows("0")) - tk.MustQuery("select INSTR(p4,n4) from t1;").Check(testkit.Rows("1")) - - tk.MustExec("truncate table t1;") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (1,'0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן');") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (2,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן');") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (3,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ');") - - tk.MustQuery("select LOCATE(p1,n1) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p1,n2) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p1,n3) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p1,n4) from t1;").Check(testkit.Rows("0", "1", "1")) - tk.MustQuery("select LOCATE(p2,n1) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p2,n2) from t1;").Check(testkit.Rows("0", "1", "1")) - tk.MustQuery("select LOCATE(p2,n3) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p2,n4) from t1;").Check(testkit.Rows("0", "1", "1")) - tk.MustQuery("select LOCATE(p3,n1) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p3,n2) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p3,n3) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p3,n4) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p4,n1) from t1;").Check(testkit.Rows("0", "1", "1")) - tk.MustQuery("select LOCATE(p4,n2) from t1;").Check(testkit.Rows("0", "1", "1")) - tk.MustQuery("select LOCATE(p4,n3) from t1;").Check(testkit.Rows("0", "0", "0")) - tk.MustQuery("select LOCATE(p4,n4) from t1;").Check(testkit.Rows("0", "1", "1")) - - tk.MustQuery("select locate('S', 's' collate utf8mb4_general_ci);").Check(testkit.Rows("1")) - tk.MustQuery("select locate('S', 'a' collate utf8mb4_general_ci);").Check(testkit.Rows("0")) - // MySQL return 0 here, I believe it is a bug in MySQL since 'ß' == 's' under utf8mb4_general_ci collation. - tk.MustQuery("select locate('ß', 's' collate utf8mb4_general_ci);").Check(testkit.Rows("1")) - tk.MustQuery("select locate('S', 's' collate utf8mb4_unicode_ci);").Check(testkit.Rows("1")) - tk.MustQuery("select locate('S', 'a' collate utf8mb4_unicode_ci);").Check(testkit.Rows("0")) - tk.MustQuery("select locate('ß', 'ss' collate utf8mb4_unicode_ci);").Check(testkit.Rows("1")) - - tk.MustExec("truncate table t1;") - tk.MustExec("insert into t1 (a) values (1);") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (2,'0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן');") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (3,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן','0Aa1!测试テストמבחן');") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (4,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ','0Aa1!测试テストמבחן ');") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (5,'0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0Aa1!测试','0Aa1!测试','0Aa1!测试','0Aa1!测试');") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (6,'0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试テストמבחן0aA1!测试','0aA1!测试','0aA1!测试','0aA1!测试','0aA1!测试');") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (7,'0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן');") - tk.MustExec("insert into t1 (a,p1,p2,p3,p4,n1,n2,n3,n4) values (8,'0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ','0aA1!测试テストמבחן ');") - - tk.MustQuery("select p1 REGEXP n1 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p1 REGEXP n2 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p1 REGEXP n3 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p1 REGEXP n4 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) - tk.MustQuery("select p2 REGEXP n1 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p2 REGEXP n2 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) - tk.MustQuery("select p2 REGEXP n3 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p2 REGEXP n4 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) - tk.MustQuery("select p3 REGEXP n1 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p3 REGEXP n2 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p3 REGEXP n3 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p3 REGEXP n4 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p4 REGEXP n1 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) - tk.MustQuery("select p4 REGEXP n2 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) - tk.MustQuery("select p4 REGEXP n3 from t1;").Check(testkit.Rows("", "0", "0", "0", "0", "1", "1", "0")) - tk.MustQuery("select p4 REGEXP n4 from t1;").Check(testkit.Rows("", "1", "1", "0", "1", "1", "1", "0")) - - tk.MustExec("drop table t1;") -} - -func (s *testIntegrationSerialSuite) TestCollateLike(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci") - tk.MustQuery("select 'a' like 'A'").Check(testkit.Rows("1")) - tk.MustQuery("select 'a' like 'A' collate utf8mb4_general_ci").Check(testkit.Rows("1")) - tk.MustQuery("select 'a' like 'À'").Check(testkit.Rows("1")) - tk.MustQuery("select 'a' like '%À'").Check(testkit.Rows("1")) - tk.MustQuery("select 'a' like '%À '").Check(testkit.Rows("0")) - tk.MustQuery("select 'a' like 'À%'").Check(testkit.Rows("1")) - tk.MustQuery("select 'a' like 'À_'").Check(testkit.Rows("0")) - tk.MustQuery("select 'a' like '%À%'").Check(testkit.Rows("1")) - tk.MustQuery("select 'aaa' like '%ÀAa%'").Check(testkit.Rows("1")) - tk.MustExec("set names utf8mb4 collate utf8mb4_bin") - - tk.MustExec("use test;") - tk.MustExec("drop table if exists t_like;") - tk.MustExec("create table t_like(id int, b varchar(20) collate utf8mb4_general_ci);") - tk.MustExec("insert into t_like values (1, 'aaa'), (2, 'abc'), (3, 'aac');") - tk.MustQuery("select b like 'AaÀ' from t_like order by id;").Check(testkit.Rows("1", "0", "0")) - tk.MustQuery("select b like 'Aa_' from t_like order by id;").Check(testkit.Rows("1", "0", "1")) - tk.MustQuery("select b like '_A_' from t_like order by id;").Check(testkit.Rows("1", "0", "1")) - tk.MustQuery("select b from t_like where b like 'Aa_' order by id;").Check(testkit.Rows("aaa", "aac")) - tk.MustQuery("select b from t_like where b like 'A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) - tk.MustQuery("select b from t_like where b like '%A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) - tk.MustExec("alter table t_like add index idx_b(b);") - tk.MustQuery("select b from t_like use index(idx_b) where b like 'Aa_' order by id;").Check(testkit.Rows("aaa", "aac")) - tk.MustQuery("select b from t_like use index(idx_b) where b like 'A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) - tk.MustQuery("select b from t_like use index(idx_b) where b like '%A%' order by id;").Check(testkit.Rows("aaa", "abc", "aac")) -} - -func (s *testIntegrationSerialSuite) TestCollateSubQuery(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := s.prepare4Collation(c, false) - tk.MustQuery("select id from t where v in (select v from t_bin) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) - tk.MustQuery("select id from t_bin where v in (select v from t) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) - tk.MustQuery("select id from t where v not in (select v from t_bin) order by id").Check(testkit.Rows()) - tk.MustQuery("select id from t_bin where v not in (select v from t) order by id").Check(testkit.Rows()) - tk.MustQuery("select id from t where exists (select 1 from t_bin where t_bin.v=t.v) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) - tk.MustQuery("select id from t_bin where exists (select 1 from t where t_bin.v=t.v) order by id").Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7")) - tk.MustQuery("select id from t where not exists (select 1 from t_bin where t_bin.v=t.v) order by id").Check(testkit.Rows()) - tk.MustQuery("select id from t_bin where not exists (select 1 from t where t_bin.v=t.v) order by id").Check(testkit.Rows()) -} - -func (s *testIntegrationSerialSuite) TestCollateDDL(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database t;") - tk.MustExec("use t;") - tk.MustExec("drop database t;") -} - -func (s *testIntegrationSerialSuite) TestNewCollationCheckClusterIndexTable(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("create table t(name char(255) primary key, b int, c int, index idx(name), unique index uidx(name))") - tk.MustExec("insert into t values(\"aaaa\", 1, 1), (\"bbb\", 2, 2), (\"ccc\", 3, 3)") - tk.MustExec("admin check table t") -} - func (s *testIntegrationSuite) TestIssue15986(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -7460,60 +6714,6 @@ func (s *testIntegrationSuite) TestIssue1223(c *C) { `{"test": [null]}`, `{"test": true}`, `{"test": false}`, `""`, "null", `"0"`)) } -func (s *testIntegrationSerialSuite) TestNewCollationWithClusterIndex(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("create table t(d double primary key, a int, name varchar(255), index idx(name(2)), index midx(a, name))") - tk.MustExec("insert into t values(2.11, 1, \"aa\"), (-1, 0, \"abcd\"), (9.99, 0, \"aaaa\")") - tk.MustQuery("select d from t use index(idx) where name=\"aa\"").Check(testkit.Rows("2.11")) -} - -func (s *testIntegrationSerialSuite) TestNewCollationBinaryFlag(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a varchar(2) binary, index (a));") - tk.MustExec("insert into t values ('a ');") - tk.MustQuery("select hex(a) from t;").Check(testkit.Rows("6120")) - tk.MustQuery("select hex(a) from t use index (a);").Check(testkit.Rows("6120")) - - showCreateTable := func(createSQL string) string { - tk.MustExec("drop table if exists t;") - tk.MustExec(createSQL) - s := tk.MustQuery("show create table t;").Rows()[0][1].(string) - return s - } - var sct string - // define case = tuple(table_charset, table_collation, column_charset, column_collation) - // case: (nil, nil, nil, nil) - sct = showCreateTable("create table t(a varchar(10) binary);") - c.Assert(strings.Contains(sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"), IsTrue, Commentf(sct)) - // case: (nil, utf8_general_ci, nil, nil) - sct = showCreateTable("create table t(a varchar(10) binary) collate utf8_general_ci;") - c.Assert(strings.Contains(sct, "varchar(10) COLLATE utf8_bin"), IsTrue, Commentf(sct)) - c.Assert(strings.Contains(sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci"), IsTrue, Commentf(sct)) - // case: (nil, nil, nil, utf8_general_ci) - sct = showCreateTable("create table t(a varchar(10) binary collate utf8_general_ci);") - c.Assert(strings.Contains(sct, "varchar(10) CHARACTER SET utf8 COLLATE utf8_bin"), IsTrue, Commentf(sct)) - // case: (nil, nil, utf8, utf8_general_ci) - sct = showCreateTable("create table t(a varchar(10) binary charset utf8 collate utf8_general_ci);") - c.Assert(strings.Contains(sct, "varchar(10) CHARACTER SET utf8 COLLATE utf8_general_ci"), IsTrue, Commentf(sct)) - // case: (utf8, utf8_general_ci, utf8mb4, utf8mb4_unicode_ci) - sct = showCreateTable("create table t(a varchar(10) binary charset utf8mb4 collate utf8mb4_unicode_ci) charset utf8 collate utf8_general_ci;") - c.Assert(strings.Contains(sct, "varchar(10) CHARACTER SET utf8mb4 COLLATE utf8mb4_unicode_ci"), IsTrue, Commentf(sct)) - c.Assert(strings.Contains(sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci"), IsTrue, Commentf(sct)) - // case: (nil, nil, binary, nil) - sct = showCreateTable("create table t(a varchar(10) binary charset binary);") - c.Assert(strings.Contains(sct, "varbinary(10) DEFAULT NULL"), IsTrue, Commentf(sct)) - c.Assert(strings.Contains(sct, "ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"), IsTrue, Commentf(sct)) -} - func (s *testIntegrationSuite) TestIssue15743(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -7744,21 +6944,6 @@ func (s *testIntegrationSuite) TestIssue17098(c *C) { tk.MustQuery("select collation(t1.a) from t1 union select collation(t2.a) from t2;").Check(testkit.Rows("utf8mb4_bin")) } -func (s *testIntegrationSerialSuite) TestIssue17176(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustGetErrMsg("create table t(a enum('a', 'a ')) charset utf8 collate utf8_bin;", "[types:1291]Column 'a' has duplicated value 'a' in ENUM") - tk.MustGetErrMsg("create table t(a enum('a', 'Á')) charset utf8 collate utf8_general_ci;", "[types:1291]Column 'a' has duplicated value 'Á' in ENUM") - tk.MustGetErrMsg("create table t(a enum('a', 'a ')) charset utf8mb4 collate utf8mb4_bin;", "[types:1291]Column 'a' has duplicated value 'a' in ENUM") - tk.MustExec("create table t(a enum('a', 'A')) charset utf8 collate utf8_bin;") - tk.MustExec("drop table t;") - tk.MustExec("create table t3(a enum('a', 'A')) charset utf8mb4 collate utf8mb4_bin;") -} - func (s *testIntegrationSuite) TestIssue17115(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustQuery("select collation(user());").Check(testkit.Rows("utf8mb4_bin")) @@ -7839,66 +7024,6 @@ func (s *testIntegrationSuite) TestIssue17898(c *C) { tk.MustQuery("select * from t0;").Check(testkit.Rows("0.5 1")) } -func (s *testIntegrationSuite) TestIssue17727(c *C) { - tk := testkit.NewTestKit(c, s.store) - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - var err error - tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ - PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), - }) - c.Assert(err, IsNil) - - tk.MustExec("use test;") - tk.MustExec("DROP TABLE IF EXISTS t1;") - tk.MustExec("CREATE TABLE t1 (id INT NOT NULL PRIMARY KEY auto_increment, a timestamp NOT NULL);") - tk.MustExec("INSERT INTO t1 VALUES (null, '2020-05-30 20:30:00');") - tk.MustExec("PREPARE mystmt FROM 'SELECT * FROM t1 WHERE UNIX_TIMESTAMP(a) >= ?';") - tk.MustExec("SET @a=1590868800;") - tk.MustQuery("EXECUTE mystmt USING @a;").Check(testkit.Rows()) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) - - tk.MustExec("SET @a=1590868801;") - tk.MustQuery("EXECUTE mystmt USING @a;").Check(testkit.Rows()) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) - - tk.MustExec("prepare stmt from 'select unix_timestamp(?)';") - tk.MustExec("set @a = '2020-05-30 20:30:00';") - tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1590841800")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) - - tk.MustExec("set @a = '2020-06-12 13:47:58';") - tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1591940878")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) -} - -func (s *testIntegrationSerialSuite) TestIssue17891(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(id int, value set ('a','b','c') charset utf8mb4 collate utf8mb4_bin default 'a,b ');") - tk.MustExec("drop table t") - tk.MustExec("create table test(id int, value set ('a','b','c') charset utf8mb4 collate utf8mb4_general_ci default 'a,B ,C');") -} - -func (s *testIntegrationSerialSuite) TestIssue20268(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("CREATE TABLE `t` ( `a` enum('a','b') DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;") - tk.MustExec("insert into t values('a');") - tk.MustExec("select * from t where a = 'A';") -} - func (s *testIntegrationSerialSuite) TestIssue17233(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -8043,33 +7168,6 @@ func (s *testIntegrationSuite2) TestSchemaDMLNotChange(c *C) { tk.MustExec("commit") } -func (s *testIntegrationSerialSuite) TestIssue18638(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a varchar(10) collate utf8mb4_bin, b varchar(10) collate utf8mb4_general_ci);") - tk.MustExec("insert into t (a, b) values ('a', 'A');") - tk.MustQuery("select * from t t1, t t2 where t1.a = t2.b collate utf8mb4_general_ci;").Check(testkit.Rows("a A a A")) - tk.MustQuery("select * from t t1 left join t t2 on t1.a = t2.b collate utf8mb4_general_ci;").Check(testkit.Rows("a A a A")) -} - -func (s *testIntegrationSerialSuite) TestCollationText(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a TINYTEXT collate UTF8MB4_GENERAL_CI, UNIQUE KEY `a`(`a`(10)));") - tk.MustExec("insert into t (a) values ('A');") - tk.MustQuery("select * from t t1 inner join t t2 on t1.a = t2.a where t1.a = 'A';").Check(testkit.Rows("A A")) - tk.MustExec("update t set a = 'B';") - tk.MustExec("admin check table t;") -} - func (s *testIntegrationSuite) TestIssue18850(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -8116,21 +7214,6 @@ func (s *testIntegrationSerialSuite) TestIssue18652(c *C) { tk.MustQuery("SELECT * FROM t1 WHERE ( LOG( `col_smallint_key_signed`, -8297584758403770424 ) ) DIV 1").Check(testkit.Rows()) } -func (s *testIntegrationSerialSuite) TestIssue18662(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a varchar(10) collate utf8mb4_bin, b varchar(10) collate utf8mb4_general_ci);") - tk.MustExec("insert into t (a, b) values ('a', 'A');") - tk.MustQuery("select * from t where field('A', a collate utf8mb4_general_ci, b) > 1;").Check(testkit.Rows()) - tk.MustQuery("select * from t where field('A', a, b collate utf8mb4_general_ci) > 1;").Check(testkit.Rows()) - tk.MustQuery("select * from t where field('A' collate utf8mb4_general_ci, a, b) > 1;").Check(testkit.Rows()) - tk.MustQuery("select * from t where field('A', a, b) > 1;").Check(testkit.Rows("a A")) -} - func (s *testIntegrationSerialSuite) TestIssue19045(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -8160,30 +7243,6 @@ func (s *testIntegrationSerialSuite) TestIssue19045(c *C) { tk.MustQuery(`select ( SELECT t1.a FROM t1, t2 WHERE t1.b = t2.a AND t2.b = '03' AND t1.c = a.a) invode from t a ;`).Check(testkit.Rows("a011", "a011")) } -func (s *testIntegrationSerialSuite) TestIssue19116(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") - tk.MustQuery("select collation(concat(1 collate `binary`));").Check(testkit.Rows("binary")) - tk.MustQuery("select coercibility(concat(1 collate `binary`));").Check(testkit.Rows("0")) - tk.MustQuery("select collation(concat(NULL,NULL));").Check(testkit.Rows("binary")) - tk.MustQuery("select coercibility(concat(NULL,NULL));").Check(testkit.Rows("6")) - tk.MustQuery("select collation(concat(1,1));").Check(testkit.Rows("utf8mb4_general_ci")) - tk.MustQuery("select coercibility(concat(1,1));").Check(testkit.Rows("4")) - tk.MustQuery("select collation(1);").Check(testkit.Rows("binary")) - tk.MustQuery("select coercibility(1);").Check(testkit.Rows("5")) - tk.MustQuery("select coercibility(1=1);").Check(testkit.Rows("5")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a datetime)") - tk.MustExec("insert into t values ('2020-02-02')") - tk.MustQuery("select collation(concat(unix_timestamp(a))) from t;").Check(testkit.Rows("utf8mb4_general_ci")) - tk.MustQuery("select coercibility(concat(unix_timestamp(a))) from t;").Check(testkit.Rows("4")) -} - // issues 14448, 19383, 17734 func (s *testIntegrationSerialSuite) TestNoopFunctions(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -8276,38 +7335,6 @@ func (s *testIntegrationSerialSuite) TestIssue18674(c *C) { "-1 1", "0 0", "0 0", "1 1", " 0")) } -func (s *testIntegrationSerialSuite) TestIssue17063(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test;`) - tk.MustExec(`drop table if exists t;`) - tk.MustExec("create table t(a char, b char) collate utf8mb4_general_ci;") - tk.MustExec(`insert into t values('a', 'b');`) - tk.MustExec(`insert into t values('a', 'B');`) - tk.MustQuery(`select * from t where if(a='x', a, b) = 'b';`).Check(testkit.Rows("a b", "a B")) - tk.MustQuery(`select collation(if(a='x', a, b)) from t;`).Check(testkit.Rows("utf8mb4_general_ci", "utf8mb4_general_ci")) - tk.MustQuery(`select coercibility(if(a='x', a, b)) from t;`).Check(testkit.Rows("2", "2")) - tk.MustQuery(`select collation(lag(b, 1, 'B') over w) from t window w as (order by b);`).Check(testkit.Rows("utf8mb4_general_ci", "utf8mb4_general_ci")) - tk.MustQuery(`select coercibility(lag(b, 1, 'B') over w) from t window w as (order by b);`).Check(testkit.Rows("2", "2")) -} - -func (s *testIntegrationSerialSuite) TestIssue11177(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustQuery("SELECT 'lvuleck' BETWEEN '2008-09-16 22:23:50' AND 0;").Check(testkit.Rows("0")) - tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: 'lvuleck'", "Warning 1292 Truncated incorrect DOUBLE value: '2008-09-16 22:23:50'")) - tk.MustQuery("SELECT 'aa' BETWEEN 'bb' AND 0;").Check(testkit.Rows("1")) - tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: 'aa'", "Warning 1292 Truncated incorrect DOUBLE value: 'bb'")) - tk.MustQuery("select 1 between 0 and b'110';").Check(testkit.Rows("1")) - tk.MustQuery("show warnings;").Check(testkit.Rows()) - tk.MustQuery("select 'b' between 'a' and b'110';").Check(testkit.Rows("0")) - tk.MustQuery("show warnings;").Check(testkit.Rows()) -} - func (s *testIntegrationSuite) TestIssue19504(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -8323,51 +7350,6 @@ func (s *testIntegrationSuite) TestIssue19504(c *C) { Check(testkit.Rows("1 1", "0 0", "0 0")) } -func (s *testIntegrationSerialSuite) TestIssue19804(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test;`) - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a set('a', 'b', 'c'));`) - tk.MustGetErrMsg("alter table t change a a set('a', 'b', 'c', 'c');", "[types:1291]Column 'a' has duplicated value 'c' in SET") - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a enum('a', 'b', 'c'));`) - tk.MustGetErrMsg("alter table t change a a enum('a', 'b', 'c', 'c');", "[types:1291]Column 'a' has duplicated value 'c' in ENUM") - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a set('a', 'b', 'c'));`) - tk.MustExec(`alter table t change a a set('a', 'b', 'c', 'd');`) - tk.MustExec(`insert into t values('d');`) - tk.MustGetErrMsg(`alter table t change a a set('a', 'b', 'c', 'e', 'f');`, "[types:1265]Data truncated for column 'a', value is 'd'") -} - -func (s *testIntegrationSerialSuite) TestIssue20209(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test;`) - tk.MustExec(`set @@character_set_client=utf8mb4;`) - tk.MustExec(`set @@collation_connection=utf8_bin;`) - tk.MustExec("CREATE VIEW tview_1 AS SELECT 'a' AS `id`;") -} - -func (s *testIntegrationSerialSuite) TestIssue18949(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test;`) - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a enum('a ', 'b\t', ' c '), b set('a ', 'b\t', ' c '));`) - result := tk.MustQuery("show create table t").Rows()[0][1] - c.Assert(result, Matches, `(?s).*enum\('a','b ',' c'\).*set\('a','b ',' c'\).*`) - tk.MustExec(`alter table t change a aa enum('a ', 'b\t', ' c ');`) - result = tk.MustQuery("show create table t").Rows()[0][1] - c.Assert(result, Matches, `(?s).*enum\('a','b ',' c'\).*set\('a','b ',' c'\).*`) -} - func (s *testIntegrationSuite) TestIssue17767(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -8417,76 +7399,6 @@ func (s *testIntegrationSuite) TestIssue17476(c *C) { tk.MustQuery(`SELECT * FROM (table_int_float_varchar AS tmp3) WHERE (col_varchar_6 AND NULL) IS NULL AND col_int_6=0;`).Check(testkit.Rows("13 0 -0.1 ")) } -func (s *testIntegrationSerialSuite) TestClusteredIndexAndNewCollationIndexEncodeDecodeV5(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("create table t(a int, b char(10) collate utf8mb4_bin, c char(10) collate utf8mb4_general_ci," + - "d varchar(10) collate utf8mb4_bin, e varchar(10) collate utf8mb4_general_ci, f char(10) collate utf8mb4_unicode_ci, g varchar(10) collate utf8mb4_unicode_ci, " + - "primary key(a, b, c, d, e, f, g), key a(a), unique key ua(a), key b(b), unique key ub(b), key c(c), unique key uc(c)," + - "key d(d), unique key ud(d),key e(e), unique key ue(e), key f(f), key g(g), unique key uf(f), unique key ug(g))") - - tk.MustExec("insert into t values (1, '啊 ', '啊 ', '啊 ', '啊 ', '啊 ', '啊 ')") - // Single Read. - tk.MustQuery("select * from t ").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - - tk.MustQuery("select * from t use index(a)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(ua)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(b)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(ub)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(c)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(uc)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(d)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(ud)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(e)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(ue)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(f)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(uf)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(g)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - tk.MustQuery("select * from t use index(ug)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ")) - - tk.MustExec("alter table t add column h varchar(10) collate utf8mb4_general_ci default '🐸'") - tk.MustExec("alter table t add column i varchar(10) collate utf8mb4_general_ci default '🐸'") - tk.MustExec("alter table t add index h(h)") - tk.MustExec("alter table t add unique index uh(h)") - - tk.MustQuery("select * from t use index(h)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(uh)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - - // Double read. - tk.MustQuery("select * from t use index(a)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(ua)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(b)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(ub)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(c)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(uc)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(d)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(ud)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(e)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustQuery("select * from t use index(ue)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸")) - tk.MustExec("admin check table t") - tk.MustExec("admin recover index t a") - tk.MustExec("alter table t add column n char(10) COLLATE utf8mb4_unicode_ci") - tk.MustExec("alter table t add index n(n)") - tk.MustExec("update t set n = '吧';") - tk.MustQuery("select * from t").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸,吧")) - tk.MustQuery("select * from t use index(n)").Check(testutil.RowsWithSep(",", "1,啊,啊,啊 ,啊 ,啊,啊 ,🐸,🐸,吧")) - tk.MustExec("admin check table t") - - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a varchar(255) COLLATE utf8_general_ci primary key clustered, b int) partition by range columns(a) " + - "(partition p0 values less than ('0'), partition p1 values less than MAXVALUE);") - tk.MustExec("alter table t add index b(b);") - tk.MustExec("insert into t values ('0', 1);") - tk.MustQuery("select * from t use index(b);").Check(testkit.Rows("0 1")) - tk.MustQuery("select * from t use index();").Check(testkit.Rows("0 1")) - tk.MustExec("admin check table t") -} - func (s *testIntegrationSuite) TestIssue11645(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) @@ -8563,37 +7475,6 @@ func (s *testIntegrationSuite) TestIssue20730(c *C) { tk.MustQuery("SELECT @sum := IF(@sum=20,4,@sum + tmp.value) sum FROM tmp ORDER BY tmp.id").Check(testkit.Rows("11", "13", "16", "20", "4")) } -func (s *testIntegrationSerialSuite) TestClusteredIndexAndNewCollation(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("CREATE TABLE `t` (" + - "`a` char(10) COLLATE utf8mb4_unicode_ci NOT NULL," + - "`b` char(20) COLLATE utf8mb4_general_ci NOT NULL," + - "`c` int(11) NOT NULL," + - "PRIMARY KEY (`a`,`b`,`c`)," + - "KEY `idx` (`a`))") - - tk.MustExec("begin") - tk.MustExec("insert into t values ('a6', 'b6', 3)") - tk.MustQuery("select * from t").Check(testkit.Rows("a6 b6 3")) - tk.MustQuery("select * from t where a='a6'").Check(testkit.Rows("a6 b6 3")) - tk.MustExec("delete from t") - tk.MustQuery("select * from t").Check(testkit.Rows()) - tk.MustExec("commit") - tk.MustQuery("select * from t").Check(testkit.Rows()) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(`a` char(10) COLLATE utf8mb4_unicode_ci NOT NULL key)") - tk.MustExec("insert into t values ('&');") - tk.MustExec("replace into t values ('&');") - tk.MustQuery("select * from t").Check(testkit.Rows("&")) -} - func (s *testIntegrationSuite) TestIssue20860(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -8603,13 +7484,6 @@ func (s *testIntegrationSuite) TestIssue20860(c *C) { c.Assert(tk.ExecToErr("update t set d = adddate(d, interval 1 day) where id < 10"), NotNil) } -func (s *testIntegrationSerialSuite) TestIssue20608(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustQuery("select '䇇Հ' collate utf8mb4_bin like '___Հ';").Check(testkit.Rows("0")) -} - func (s *testIntegrationSuite2) TestIssue15847(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -8619,19 +7493,6 @@ func (s *testIntegrationSuite2) TestIssue15847(c *C) { tk.MustExec("drop view if exists t15847") } -func (s *testIntegrationSerialSuite) TestIssue20161(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test;`) - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(raw JSON);`) - tk.MustExec(`insert into t(raw) values('["a","ab"]'), ('["a"]'), (null);`) - tk.MustQuery(`SELECT JSON_SEARCH(raw,'one','c') FROM t;`). - Check(testkit.Rows("", "", "")) -} - func (s *testIntegrationSuite) TestIssue10462(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -8736,66 +7597,6 @@ func (s *testIntegrationSuite) TestIssue10467(c *C) { tk.MustQuery("select col, count(1) c from tx2 group by col order by c desc;").Check(testkit.Rows("[922337203685477581] 5", `["3"] 4`, "[3] 3", "[-3] 2", "[922337203685477580] 1")) } -func (s *testIntegrationSerialSuite) TestCollationIndexJoin(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b char(10), key(b)) collate utf8mb4_general_ci") - tk.MustExec("create table t2(a int, b char(10), key(b)) collate ascii_bin") - tk.MustExec("insert into t1 values (1, 'a')") - tk.MustExec("insert into t2 values (1, 'A')") - - tk.MustQuery("select /*+ inl_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) - tk.MustQuery("select /*+ hash_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) - tk.MustQuery("select /*+ merge_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) - tk.MustQuery("select /*+ inl_hash_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) - tk.MustQuery("select /*+ inl_hash_join(t2) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Optimizer Hint /*+ INL_HASH_JOIN(t2) */ is inapplicable")) - tk.MustQuery("select /*+ inl_merge_join(t1) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) - tk.MustQuery("select /*+ inl_merge_join(t2) */ t1.b, t2.b from t1 join t2 where t1.b=t2.b").Check(testkit.Rows("a A")) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Optimizer Hint /*+ INL_MERGE_JOIN(t2) */ is inapplicable")) - - tk.MustExec("drop table if exists a, b") - tk.MustExec("create table a(i int, k varbinary(40), v int, primary key(i, k) clustered)") - tk.MustExec("create table b(i int, k varchar(40), v int, primary key(i, k) clustered)") - tk.MustExec("insert into a select 3, 'nice mccarthy', 10") - tk.MustQuery("select * from a, b where a.i = b.i and a.k = b.k").Check(testkit.Rows()) - - tk.MustExec("drop table if exists a, b") - tk.MustExec("create table a(i int NOT NULL, k varbinary(40) NOT NULL, v int, key idx1(i, k))") - tk.MustExec("create table b(i int NOT NULL, k varchar(40) NOT NULL, v int, key idx1(i, k))") - tk.MustExec("insert into a select 3, 'nice mccarthy', 10") - tk.MustQuery(" select /*+ inl_join(b) */ b.i from a, b where a.i = b.i and a.k = b.k").Check(testkit.Rows()) -} - -func (s *testIntegrationSerialSuite) TestCollationMergeJoin(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("CREATE TABLE `t` (" + - " `col_10` blob DEFAULT NULL," + - " `col_11` decimal(17,5) NOT NULL," + - " `col_13` varchar(381) COLLATE utf8mb4_unicode_ci NOT NULL DEFAULT 'Yr'," + - " PRIMARY KEY (`col_13`,`col_11`) CLUSTERED," + - " KEY `idx_4` (`col_10`(3))" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") - tk.MustExec("insert into t values ('a', 12523, 'A');") - tk.MustExec("insert into t values ('A', 2, 'a');") - tk.MustExec("insert into t values ('a', 23, 'A');") - tk.MustExec("insert into t values ('a', 23, 'h2');") - tk.MustExec("insert into t values ('a', 23, 'h3');") - tk.MustExec("insert into t values ('a', 23, 'h4');") - tk.MustExec("insert into t values ('a', 23, 'h5');") - tk.MustExec("insert into t values ('a', 23, 'h6');") - tk.MustExec("insert into t values ('a', 23, 'h7');") - tk.MustQuery("select /*+ MERGE_JOIN(t) */ t.* from t where col_13 in ( select col_10 from t where t.col_13 in ( 'a', 'b' ) ) order by col_10 ;").Check( - testkit.Rows("\x41 2.00000 a", "\x61 23.00000 A", "\x61 12523.00000 A")) -} - func (s *testIntegrationSuite) TestIssue19892(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) @@ -9034,25 +7835,6 @@ func (s *testIntegrationSuite) TestIssue19892(c *C) { } } -func (s *testIntegrationSerialSuite) TestIssue20876(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("drop table if exists t;") - tk.MustExec("CREATE TABLE `t` (" + - " `a` char(10) COLLATE utf8mb4_unicode_ci NOT NULL," + - " `b` char(20) COLLATE utf8mb4_general_ci NOT NULL," + - " `c` int(11) NOT NULL," + - " PRIMARY KEY (`a`,`b`,`c`)," + - " KEY `idx` (`a`)" + - ")") - tk.MustExec("insert into t values ('#', 'C', 10), ('$', 'c', 20), ('$', 'c', 30), ('a', 'a', 10), ('A', 'A', 30)") - tk.MustExec("analyze table t") - tk.MustQuery("select * from t where a='#';").Check(testkit.Rows("# C 10")) -} - // The actual results do not agree with the test results, It should be modified after the test suite is updated func (s *testIntegrationSuite) TestIssue17726(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -9118,19 +7900,6 @@ func (s *testIntegrationSuite) TestIssue21677(c *C) { tk.MustQuery("select * from t where t. `r10` > 3;").Check(testkit.Rows("1 10")) } -func (s *testIntegrationSerialSuite) TestLikeWithCollation(c *C) { - tk := testkit.NewTestKit(c, s.store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk.MustQuery(`select 'a' like 'A' collate utf8mb4_unicode_ci;`).Check(testkit.Rows("1")) - tk.MustGetErrMsg(`select 'a' collate utf8mb4_bin like 'A' collate utf8mb4_unicode_ci;`, "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_unicode_ci,EXPLICIT) for operation 'like'") - tk.MustQuery(`select '😛' collate utf8mb4_general_ci like '😋';`).Check(testkit.Rows("1")) - tk.MustQuery(`select '😛' collate utf8mb4_general_ci = '😋';`).Check(testkit.Rows("1")) - tk.MustQuery(`select '😛' collate utf8mb4_unicode_ci like '😋';`).Check(testkit.Rows("0")) - tk.MustQuery(`select '😛' collate utf8mb4_unicode_ci = '😋';`).Check(testkit.Rows("1")) -} - func (s *testIntegrationSuite) TestIssue11333(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) @@ -9148,7 +7917,7 @@ func (s *testIntegrationSuite) TestIssue11333(c *C) { tk.MustQuery(`select 0.000000000000000000000000000000000000000000000000000000000000000000000001;`).Check(testkit.Rows("0.000000000000000000000000000000000000000000000000000000000000000000000001")) } -func (s *testSuite) TestIssue12206(c *C) { +func (s *testIntegrationSuite) TestIssue12206(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t12206;") @@ -9323,19 +8092,6 @@ PARTITION BY RANGE (c) ( tk.MustExec("set global tidb_enable_local_txn = off;") } -func (s *testIntegrationSerialSuite) TestCollationUnion(c *C) { - // For issue 19694. - tk := testkit.NewTestKit(c, s.store) - - tk.MustQuery("select cast('2010-09-09' as date) a union select '2010-09-09 ' order by a;").Check(testkit.Rows("2010-09-09", "2010-09-09 ")) - res := tk.MustQuery("select cast('2010-09-09' as date) a union select '2010-09-09 ';") - c.Check(len(res.Rows()), Equals, 2) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - res = tk.MustQuery("select cast('2010-09-09' as date) a union select '2010-09-09 ';") - c.Check(len(res.Rows()), Equals, 1) -} - func (s *testIntegrationSuite) TestIssue22098(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -9565,18 +8321,6 @@ func (s *testIntegrationSuite) TestJiraSetInnoDBDefaultRowFormat(c *C) { } -func (s *testIntegrationSerialSuite) TestCollationForBinaryLiteral(c *C) { - tk := testkit.NewTestKit(c, s.store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("CREATE TABLE t (`COL1` tinyblob NOT NULL, `COL2` binary(1) NOT NULL, `COL3` bigint(11) NOT NULL, PRIMARY KEY (`COL1`(5),`COL2`,`COL3`) /*T![clustered_index] CLUSTERED */)") - tk.MustExec("insert into t values(0x1E,0xEC,6966939640596047133);") - tk.MustQuery("select * from t where col1 not in (0x1B,0x20) order by col1").Check(testkit.Rows("\x1e \xec 6966939640596047133")) - tk.MustExec("drop table t") -} - func (s *testIntegrationSuite) TestIssue23623(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -9595,41 +8339,6 @@ func (s *testIntegrationSuite) TestApproximatePercentile(c *C) { tk.MustQuery("select approx_percentile(a, 10) from t").Check(testkit.Rows("")) } -func (s *testIntegrationSerialSuite) TestCollationPrefixClusteredIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (k char(20), v int, primary key (k(4)) clustered, key (k)) collate utf8mb4_general_ci;") - tk.MustExec("insert into t values('01233', 1);") - tk.MustExec("create index idx on t(k(2))") - tk.MustQuery("select * from t use index(k_2);").Check(testkit.Rows("01233 1")) - tk.MustQuery("select * from t use index(idx);").Check(testkit.Rows("01233 1")) - tk.MustExec("admin check table t;") -} - -func (s *testIntegrationSerialSuite) TestIssue23805(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk.MustExec("CREATE TABLE `tbl_5` (" + - " `col_25` time NOT NULL DEFAULT '05:35:58'," + - " `col_26` blob NOT NULL," + - " `col_27` double NOT NULL," + - " `col_28` char(83) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL," + - " `col_29` timestamp NOT NULL," + - " `col_30` varchar(36) COLLATE utf8mb4_general_ci NOT NULL DEFAULT 'ywzIn'," + - " `col_31` binary(85) DEFAULT 'OIstcXsGmAyc'," + - " `col_32` datetime NOT NULL DEFAULT '2024-08-02 00:00:00'," + - " PRIMARY KEY (`col_26`(3),`col_27`) /*T![clustered_index] CLUSTERED */," + - " UNIQUE KEY `idx_10` (`col_26`(5)));") - tk.MustExec("insert ignore into tbl_5 set col_28 = 'ZmZIdSnq' , col_25 = '18:50:52.00' on duplicate key update col_26 = 'y';\n") -} - func (s *testIntegrationSuite) TestIssue24429(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -10389,18 +9098,6 @@ func (s *testIntegrationSuite) TestTranslate(c *C) { tk.MustQuery("select translate(i, '0123456', 'abcdefg') from t").Check(testkit.Rows("a", "b", "c", "d", "e", "f", "g")) } -func (s *testIntegrationSerialSuite) TestIssue26662(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1(a varchar(36) NOT NULL) ENGINE = InnoDB DEFAULT CHARSET = utf8 COLLATE = utf8_general_ci;") - tk.MustExec("set names utf8;") - tk.MustQuery("select t2.b from (select t1.a as b from t1 union all select t1.a as b from t1) t2 where case when (t2.b is not null) then t2.b else '' end > '1234567';"). - Check(testkit.Rows()) -} - func (s *testIntegrationSuite) TestIssue26958(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") @@ -10669,3 +9366,34 @@ func (s *testIntegrationSuite) TestIssue30101(c *C) { tk.MustExec("insert into t1 values(9223372036854775808, 9223372036854775809);") tk.MustQuery("select greatest(c1, c2) from t1;").Sort().Check(testkit.Rows("9223372036854775809")) } + +func (s *testIntegrationSuite) TestIssue28739(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`USE test`) + tk.MustExec("SET time_zone = 'Europe/Vilnius'") + tk.MustQuery("SELECT UNIX_TIMESTAMP('2020-03-29 03:45:00')").Check(testkit.Rows("1585443600")) + tk.MustQuery("SELECT FROM_UNIXTIME(UNIX_TIMESTAMP('2020-03-29 03:45:00'))").Check(testkit.Rows("2020-03-29 04:00:00")) + tk.MustExec(`DROP TABLE IF EXISTS t`) + tk.MustExec(`CREATE TABLE t (dt DATETIME NULL)`) + defer tk.MustExec(`DROP TABLE t`) + // Test the vector implememtation + tk.MustExec(`INSERT INTO t VALUES ('2021-10-31 02:30:00'), ('2021-03-28 02:30:00'), ('2020-10-04 02:15:00'), ('2020-03-29 03:45:00'), (NULL)`) + tk.MustQuery(`SELECT dt, UNIX_TIMESTAMP(dt) FROM t`).Sort().Check(testkit.Rows( + "2020-03-29 03:45:00 1585443600", + "2020-10-04 02:15:00 1601766900", + "2021-03-28 02:30:00 1616891400", + "2021-10-31 02:30:00 1635636600", + " ")) +} + +func (s *testIntegrationSuite) TestIssue30326(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(1),(1),(2),(2);") + tk.MustExec("set tidb_window_concurrency = 1;") + err := tk.QueryToErr("select (FIRST_VALUE(1) over (partition by v.a)) as c3 from (select a from t where t.a = (select a from t t2 where t.a = t2.a)) as v;") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[executor:1242]Subquery returns more than 1 row") +} diff --git a/expression/main_test.go b/expression/main_test.go index ecbd5f70f93e7..3e99fa8cbf7ab 100644 --- a/expression/main_test.go +++ b/expression/main_test.go @@ -49,6 +49,7 @@ func TestMain(m *testing.M) { timeutil.SetSystemTZ("system") testDataMap.LoadTestSuiteData("testdata", "flag_simplify") + testDataMap.LoadTestSuiteData("testdata", "expression_suite") opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), @@ -76,3 +77,7 @@ func createContext(t *testing.T) *mock.Context { func GetFlagSimplifyData() testdata.TestData { return testDataMap["flag_simplify"] } + +func GetExpressionSuiteData() testdata.TestData { + return testDataMap["expression_suite"] +} diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 4c91489da289e..af92f305943ab 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -35,13 +35,10 @@ import ( func TestInferType(t *testing.T) { t.Parallel() - store, dom, err := newStoreWithBootstrap() - require.NoError(t, err) - defer func() { - dom.Close() - err = store.Close() - require.NoError(t, err) - }() + + store, clean := testkit.CreateMockStore(t) + defer clean() + s := InferTypeSuite{} se, err := session.CreateSession4Test(store) require.NoError(t, err) diff --git a/go.mod b/go.mod index 889d97cc41057..f626c01b4ddd3 100644 --- a/go.mod +++ b/go.mod @@ -53,7 +53,7 @@ require ( github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5 github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e - github.com/pingcap/tipb v0.0.0-20211116093845-e9b045a0bdf8 + github.com/pingcap/tipb v0.0.0-20211201080053-bd104bb270ba github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.9.1 @@ -65,7 +65,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211201083510-3a7675742ee5 + github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211206072923-c0e876615440 github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index 214ab33c99608..59030af41f80a 100644 --- a/go.sum +++ b/go.sum @@ -591,8 +591,8 @@ github.com/pingcap/tidb-dashboard v0.0.0-20211008050453-a25c25809529/go.mod h1:O github.com/pingcap/tidb-dashboard v0.0.0-20211107164327-80363dfbe884/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible h1:c7+izmker91NkjkZ6FgTlmD4k1A5FLOAq+li6Ki2/GY= github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20211116093845-e9b045a0bdf8 h1:Vu/6oq8EFNWgyXRHiclNzTKIu+YKHPCSI/Ba5oVrLtM= -github.com/pingcap/tipb v0.0.0-20211116093845-e9b045a0bdf8/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20211201080053-bd104bb270ba h1:Tt5W/maVBUbG+wxg2nfc88Cqj/HiWYb0TJQ2Rfi0UOQ= +github.com/pingcap/tipb v0.0.0-20211201080053-bd104bb270ba/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -704,8 +704,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211201083510-3a7675742ee5 h1:c0zWUeB8aog6TceRyFEjA2fYUJka8RNGBZWU5l7XF9g= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211201083510-3a7675742ee5/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211206072923-c0e876615440 h1:XHRkMms0v6uxUZqErwZbmAs7baVVyNcOC1oOSz+BGgc= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211206072923-c0e876615440/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr92g0VmZEpkH+40RGQBpFL2121xMs= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= diff --git a/kv/txn.go b/kv/txn.go index 1359e60abb47d..7701dcd870b06 100644 --- a/kv/txn.go +++ b/kv/txn.go @@ -16,10 +16,12 @@ package kv import ( "context" + "errors" "math" "math/rand" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" @@ -58,9 +60,24 @@ func RunInNewTxn(ctx context.Context, store Storage, retryable bool, f func(ctx return err } - err = txn.Commit(ctx) + failpoint.Inject("mockCommitErrorInNewTxn", func(val failpoint.Value) { + if v := val.(string); len(v) > 0 { + switch v { + case "retry_once": + if i == 0 { + err = ErrTxnRetryable + } + case "no_retry": + failpoint.Return(errors.New("mock commit error")) + } + } + }) + if err == nil { - break + err = txn.Commit(ctx) + if err == nil { + break + } } if retryable && IsTxnRetryableError(err) { logutil.BgLogger().Warn("RunInNewTxn", diff --git a/parser/ast/functions.go b/parser/ast/functions.go index e463598838fee..7775f3dbc2029 100644 --- a/parser/ast/functions.go +++ b/parser/ast/functions.go @@ -279,6 +279,7 @@ const ( IsIPv4Mapped = "is_ipv4_mapped" IsIPv6 = "is_ipv6" IsUsedLock = "is_used_lock" + IsUUID = "is_uuid" MasterPosWait = "master_pos_wait" NameConst = "name_const" ReleaseAllLocks = "release_all_locks" diff --git a/parser/auth/auth.go b/parser/auth/auth.go index cb94346f82208..5657c6c276646 100644 --- a/parser/auth/auth.go +++ b/parser/auth/auth.go @@ -41,18 +41,24 @@ func (user *UserIdentity) Restore(ctx *format.RestoreCtx) error { } // String converts UserIdentity to the format user@host. +// It defaults to providing the AuthIdentity (the matching entry in priv tables) +// To use the actual identity use LoginString() func (user *UserIdentity) String() string { // TODO: Escape username and hostname. if user == nil { return "" } + if user.AuthUsername != "" { + return fmt.Sprintf("%s@%s", user.AuthUsername, user.AuthHostname) + } return fmt.Sprintf("%s@%s", user.Username, user.Hostname) } -// AuthIdentityString returns matched identity in user@host format -func (user *UserIdentity) AuthIdentityString() string { +// LoginString returns matched identity in user@host format +// It matches the login user. +func (user *UserIdentity) LoginString() string { // TODO: Escape username and hostname. - return fmt.Sprintf("%s@%s", user.AuthUsername, user.AuthHostname) + return fmt.Sprintf("%s@%s", user.Username, user.Hostname) } type RoleIdentity struct { diff --git a/parser/parser.go b/parser/parser.go index 01b072fc0bf2a..99a2267eb4aa6 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -54,7 +54,7 @@ type yyXError struct { } const ( - yyDefault = 58101 + yyDefault = 58100 yyEOFCode = 57344 account = 57573 action = 57574 @@ -195,7 +195,7 @@ const ( correlation = 57996 cpu = 57636 create = 57383 - createTableSelect = 58085 + createTableSelect = 58084 cross = 57384 csvBackslashEscape = 57637 csvDelimiter = 57638 @@ -325,8 +325,8 @@ const ( help = 57693 hexLit = 58059 highPriority = 57430 - higherThanComma = 58100 - higherThanParenthese = 58094 + higherThanComma = 58099 + higherThanParenthese = 58093 hintComment = 57353 histogram = 57694 history = 57695 @@ -352,7 +352,7 @@ const ( inplace = 57934 insert = 57446 insertMethod = 57705 - insertValues = 58083 + insertValues = 58082 instance = 57706 instant = 57935 int1Type = 57448 @@ -422,25 +422,24 @@ const ( longblobType = 57470 longtextType = 57471 lowPriority = 57472 - lowerThanCharsetKwd = 58086 - lowerThanComma = 58099 - lowerThanCreateTableSelect = 58084 - lowerThanEq = 58096 - lowerThanFunction = 58091 - lowerThanInsertValues = 58082 - lowerThanIntervalKeyword = 58077 - lowerThanKey = 58087 - lowerThanLocal = 58088 - lowerThanNot = 58098 - lowerThanOn = 58095 - lowerThanParenthese = 58093 - lowerThanRemove = 58089 + lowerThanCharsetKwd = 58085 + lowerThanComma = 58098 + lowerThanCreateTableSelect = 58083 + lowerThanEq = 58095 + lowerThanFunction = 58090 + lowerThanInsertValues = 58081 + lowerThanKey = 58086 + lowerThanLocal = 58087 + lowerThanNot = 58097 + lowerThanOn = 58094 + lowerThanParenthese = 58092 + lowerThanRemove = 58088 lowerThanSelectOpt = 58076 - lowerThanSelectStmt = 58081 - lowerThanSetKeyword = 58080 - lowerThanStringLitToken = 58079 - lowerThanValueKeyword = 58078 - lowerThenOrder = 58090 + lowerThanSelectStmt = 58080 + lowerThanSetKeyword = 58079 + lowerThanStringLitToken = 58078 + lowerThanValueKeyword = 58077 + lowerThenOrder = 58089 lsh = 58068 master = 57727 match = 57473 @@ -474,7 +473,7 @@ const ( national = 57746 natural = 57572 ncharType = 57747 - neg = 58097 + neg = 58096 neq = 58069 neqSynonym = 58070 never = 57748 @@ -716,7 +715,7 @@ const ( systemTime = 57873 tableChecksum = 57874 tableKwd = 57534 - tableRefPriority = 58092 + tableRefPriority = 58091 tableSample = 57535 tables = 57875 tablespace = 57876 @@ -1467,10 +1466,10 @@ var ( 57464: 645, // lines (490x) 57371: 646, // by (487x) 58062: 647, // assignmentEq (485x) - 58320: 648, // Identifier (483x) - 58395: 649, // NotKeywordToken (483x) - 58616: 650, // TiDBKeyword (483x) - 58626: 651, // UnReservedKeyword (483x) + 58319: 648, // Identifier (483x) + 58394: 649, // NotKeywordToken (483x) + 58615: 650, // TiDBKeyword (483x) + 58625: 651, // UnReservedKeyword (483x) 57512: 652, // require (482x) 57361: 653, // alter (481x) 64: 654, // '@' (477x) @@ -1515,672 +1514,671 @@ var ( 57539: 693, // tinyblobType (461x) 57540: 694, // tinyIntType (461x) 57541: 695, // tinytextType (461x) - 58581: 696, // SubSelect (208x) - 58635: 697, // UserVariable (171x) - 58557: 698, // SimpleIdent (170x) - 58372: 699, // Literal (168x) - 58571: 700, // StringLiteral (168x) - 58393: 701, // NextValueForSequence (167x) - 58297: 702, // FunctionCallGeneric (166x) - 58298: 703, // FunctionCallKeyword (166x) - 58299: 704, // FunctionCallNonKeyword (166x) - 58300: 705, // FunctionNameConflict (166x) - 58301: 706, // FunctionNameDateArith (166x) - 58302: 707, // FunctionNameDateArithMultiForms (166x) - 58303: 708, // FunctionNameDatetimePrecision (166x) - 58304: 709, // FunctionNameOptionalBraces (166x) - 58305: 710, // FunctionNameSequence (166x) - 58556: 711, // SimpleExpr (166x) - 58582: 712, // SumExpr (166x) - 58584: 713, // SystemVariable (166x) - 58646: 714, // Variable (166x) - 58669: 715, // WindowFuncCall (166x) - 58149: 716, // BitExpr (153x) - 58466: 717, // PredicateExpr (130x) - 58152: 718, // BoolPri (127x) - 58264: 719, // Expression (127x) - 58684: 720, // logAnd (96x) - 58685: 721, // logOr (96x) - 58391: 722, // NUM (96x) - 58254: 723, // EqOpt (86x) - 58594: 724, // TableName (75x) - 58572: 725, // StringName (56x) + 58580: 696, // SubSelect (208x) + 58634: 697, // UserVariable (171x) + 58556: 698, // SimpleIdent (170x) + 58371: 699, // Literal (168x) + 58570: 700, // StringLiteral (168x) + 58392: 701, // NextValueForSequence (167x) + 58296: 702, // FunctionCallGeneric (166x) + 58297: 703, // FunctionCallKeyword (166x) + 58298: 704, // FunctionCallNonKeyword (166x) + 58299: 705, // FunctionNameConflict (166x) + 58300: 706, // FunctionNameDateArith (166x) + 58301: 707, // FunctionNameDateArithMultiForms (166x) + 58302: 708, // FunctionNameDatetimePrecision (166x) + 58303: 709, // FunctionNameOptionalBraces (166x) + 58304: 710, // FunctionNameSequence (166x) + 58555: 711, // SimpleExpr (166x) + 58581: 712, // SumExpr (166x) + 58583: 713, // SystemVariable (166x) + 58645: 714, // Variable (166x) + 58668: 715, // WindowFuncCall (166x) + 58148: 716, // BitExpr (153x) + 58465: 717, // PredicateExpr (130x) + 58151: 718, // BoolPri (127x) + 58263: 719, // Expression (127x) + 58683: 720, // logAnd (96x) + 58684: 721, // logOr (96x) + 58390: 722, // NUM (96x) + 58253: 723, // EqOpt (86x) + 58593: 724, // TableName (75x) + 58571: 725, // StringName (56x) 57549: 726, // unsigned (47x) 57495: 727, // over (45x) 57571: 728, // zerofill (45x) - 58174: 729, // ColumnName (40x) + 58173: 729, // ColumnName (40x) 57400: 730, // deleteKwd (40x) - 58363: 731, // LengthNum (40x) + 58362: 731, // LengthNum (40x) 57404: 732, // distinct (36x) 57405: 733, // distinctRow (36x) - 58674: 734, // WindowingClause (35x) + 58673: 734, // WindowingClause (35x) 57399: 735, // delayed (33x) 57430: 736, // highPriority (33x) 57472: 737, // lowPriority (33x) - 58512: 738, // SelectStmt (29x) - 58513: 739, // SelectStmtBasic (29x) - 58515: 740, // SelectStmtFromDualTable (29x) - 58516: 741, // SelectStmtFromTable (29x) - 58532: 742, // SetOprClause (29x) - 58533: 743, // SetOprClauseList (28x) - 58536: 744, // SetOprStmtWithLimitOrderBy (28x) - 58537: 745, // SetOprStmtWoutLimitOrderBy (28x) + 58511: 738, // SelectStmt (29x) + 58512: 739, // SelectStmtBasic (29x) + 58514: 740, // SelectStmtFromDualTable (29x) + 58515: 741, // SelectStmtFromTable (29x) + 58531: 742, // SetOprClause (29x) + 58532: 743, // SetOprClauseList (28x) + 58535: 744, // SetOprStmtWithLimitOrderBy (28x) + 58536: 745, // SetOprStmtWoutLimitOrderBy (28x) 57353: 746, // hintComment (27x) - 58275: 747, // FieldLen (26x) - 58352: 748, // Int64Num (26x) - 58525: 749, // SelectStmtWithClause (25x) - 58535: 750, // SetOprStmt (25x) - 58675: 751, // WithClause (25x) - 58432: 752, // OptWindowingClause (24x) - 58437: 753, // OrderBy (23x) - 58519: 754, // SelectStmtLimit (23x) + 58274: 747, // FieldLen (26x) + 58351: 748, // Int64Num (26x) + 58524: 749, // SelectStmtWithClause (25x) + 58534: 750, // SetOprStmt (25x) + 58674: 751, // WithClause (25x) + 58431: 752, // OptWindowingClause (24x) + 58436: 753, // OrderBy (23x) + 58518: 754, // SelectStmtLimit (23x) 57527: 755, // sqlBigResult (23x) 57528: 756, // sqlCalcFoundRows (23x) 57529: 757, // sqlSmallResult (23x) - 58231: 758, // DirectPlacementOption (21x) - 58162: 759, // CharsetKw (20x) - 58637: 760, // Username (20x) - 58265: 761, // ExpressionList (17x) - 58461: 762, // PlacementPolicyOption (17x) - 58629: 763, // UpdateStmtNoWith (17x) - 58230: 764, // DeleteWithoutUsingStmt (16x) - 58321: 765, // IfExists (16x) - 58459: 766, // PlacementOption (16x) + 58230: 758, // DirectPlacementOption (21x) + 58161: 759, // CharsetKw (20x) + 58636: 760, // Username (20x) + 58264: 761, // ExpressionList (17x) + 58460: 762, // PlacementPolicyOption (17x) + 58628: 763, // UpdateStmtNoWith (17x) + 58229: 764, // DeleteWithoutUsingStmt (16x) + 58320: 765, // IfExists (16x) + 58458: 766, // PlacementOption (16x) 57537: 767, // terminated (16x) - 58232: 768, // DistinctKwd (15x) - 58322: 769, // IfNotExists (15x) - 58349: 770, // InsertIntoStmt (15x) - 58417: 771, // OptFieldLen (15x) - 58487: 772, // ReplaceIntoStmt (15x) - 58628: 773, // UpdateStmt (15x) - 58233: 774, // DistinctOpt (14x) + 58231: 768, // DistinctKwd (15x) + 58321: 769, // IfNotExists (15x) + 58348: 770, // InsertIntoStmt (15x) + 58416: 771, // OptFieldLen (15x) + 58486: 772, // ReplaceIntoStmt (15x) + 58627: 773, // UpdateStmt (15x) + 58232: 774, // DistinctOpt (14x) 57411: 775, // enclosed (14x) - 58448: 776, // PartitionNameList (14x) - 58659: 777, // WhereClause (14x) - 58660: 778, // WhereClauseOptional (14x) - 58225: 779, // DefaultKwdOpt (13x) + 58447: 776, // PartitionNameList (14x) + 58658: 777, // WhereClause (14x) + 58659: 778, // WhereClauseOptional (14x) + 58224: 779, // DefaultKwdOpt (13x) 57412: 780, // escaped (13x) 57491: 781, // optionally (13x) - 58595: 782, // TableNameList (13x) - 58229: 783, // DeleteWithUsingStmt (12x) - 58263: 784, // ExprOrDefault (12x) - 58357: 785, // JoinTable (12x) - 58411: 786, // OptBinary (12x) - 58503: 787, // RolenameComposed (12x) - 58591: 788, // TableFactor (12x) - 58604: 789, // TableRef (12x) - 58124: 790, // AnalyzeOptionListOpt (11x) - 58228: 791, // DeleteFromStmt (11x) - 58292: 792, // FromOrIn (11x) - 58618: 793, // TimestampUnit (11x) - 58163: 794, // CharsetName (10x) - 58175: 795, // ColumnNameList (10x) - 58396: 796, // NotSym (10x) - 58438: 797, // OrderByOptional (10x) - 58440: 798, // PartDefOption (10x) - 58555: 799, // SignedNum (10x) - 58155: 800, // BuggyDefaultFalseDistinctOpt (9x) - 58215: 801, // DBName (9x) - 58224: 802, // DefaultFalseDistinctOpt (9x) - 58358: 803, // JoinType (9x) + 58594: 782, // TableNameList (13x) + 58228: 783, // DeleteWithUsingStmt (12x) + 58262: 784, // ExprOrDefault (12x) + 58356: 785, // JoinTable (12x) + 58410: 786, // OptBinary (12x) + 58502: 787, // RolenameComposed (12x) + 58590: 788, // TableFactor (12x) + 58603: 789, // TableRef (12x) + 58123: 790, // AnalyzeOptionListOpt (11x) + 58227: 791, // DeleteFromStmt (11x) + 58291: 792, // FromOrIn (11x) + 58617: 793, // TimestampUnit (11x) + 58162: 794, // CharsetName (10x) + 58174: 795, // ColumnNameList (10x) + 58395: 796, // NotSym (10x) + 58437: 797, // OrderByOptional (10x) + 58439: 798, // PartDefOption (10x) + 58554: 799, // SignedNum (10x) + 58154: 800, // BuggyDefaultFalseDistinctOpt (9x) + 58214: 801, // DBName (9x) + 58223: 802, // DefaultFalseDistinctOpt (9x) + 58357: 803, // JoinType (9x) 57466: 804, // load (9x) 57482: 805, // noWriteToBinLog (9x) - 58401: 806, // NumLiteral (9x) - 58502: 807, // Rolename (9x) - 58497: 808, // RoleNameString (9x) - 58120: 809, // AlterTableStmt (8x) - 58214: 810, // CrossOpt (8x) - 58255: 811, // EqOrAssignmentEq (8x) - 58266: 812, // ExpressionListOpt (8x) - 58343: 813, // IndexPartSpecification (8x) - 58359: 814, // KeyOrIndex (8x) - 58520: 815, // SelectStmtLimitOpt (8x) - 58617: 816, // TimeUnit (8x) - 58649: 817, // VariableName (8x) - 58106: 818, // AllOrPartitionNameList (7x) - 58198: 819, // ConstraintKeywordOpt (7x) - 58281: 820, // FieldsOrColumns (7x) - 58290: 821, // ForceOpt (7x) - 58344: 822, // IndexPartSpecificationList (7x) - 58394: 823, // NoWriteToBinLogAliasOpt (7x) - 58470: 824, // Priority (7x) - 58507: 825, // RowFormat (7x) - 58510: 826, // RowValue (7x) - 58530: 827, // SetExpr (7x) - 58541: 828, // ShowDatabaseNameOpt (7x) - 58601: 829, // TableOption (7x) + 58400: 806, // NumLiteral (9x) + 58501: 807, // Rolename (9x) + 58496: 808, // RoleNameString (9x) + 58119: 809, // AlterTableStmt (8x) + 58213: 810, // CrossOpt (8x) + 58254: 811, // EqOrAssignmentEq (8x) + 58265: 812, // ExpressionListOpt (8x) + 58342: 813, // IndexPartSpecification (8x) + 58358: 814, // KeyOrIndex (8x) + 58519: 815, // SelectStmtLimitOpt (8x) + 58616: 816, // TimeUnit (8x) + 58648: 817, // VariableName (8x) + 58105: 818, // AllOrPartitionNameList (7x) + 58197: 819, // ConstraintKeywordOpt (7x) + 58280: 820, // FieldsOrColumns (7x) + 58289: 821, // ForceOpt (7x) + 58343: 822, // IndexPartSpecificationList (7x) + 58393: 823, // NoWriteToBinLogAliasOpt (7x) + 58469: 824, // Priority (7x) + 58506: 825, // RowFormat (7x) + 58509: 826, // RowValue (7x) + 58529: 827, // SetExpr (7x) + 58540: 828, // ShowDatabaseNameOpt (7x) + 58600: 829, // TableOption (7x) 57562: 830, // varying (7x) 57380: 831, // column (6x) - 58169: 832, // ColumnDef (6x) - 58217: 833, // DatabaseOption (6x) - 58220: 834, // DatabaseSym (6x) - 58257: 835, // EscapedTableRef (6x) - 58262: 836, // ExplainableStmt (6x) - 58279: 837, // FieldTerminator (6x) + 58168: 832, // ColumnDef (6x) + 58216: 833, // DatabaseOption (6x) + 58219: 834, // DatabaseSym (6x) + 58256: 835, // EscapedTableRef (6x) + 58261: 836, // ExplainableStmt (6x) + 58278: 837, // FieldTerminator (6x) 57426: 838, // grant (6x) - 58326: 839, // IgnoreOptional (6x) - 58335: 840, // IndexInvisible (6x) - 58340: 841, // IndexNameList (6x) - 58346: 842, // IndexType (6x) - 58449: 843, // PartitionNameListOpt (6x) + 58325: 839, // IgnoreOptional (6x) + 58334: 840, // IndexInvisible (6x) + 58339: 841, // IndexNameList (6x) + 58345: 842, // IndexType (6x) + 58448: 843, // PartitionNameListOpt (6x) 57508: 844, // release (6x) - 58504: 845, // RolenameList (6x) + 58503: 845, // RolenameList (6x) 57523: 846, // show (6x) - 58599: 847, // TableOptimizerHints (6x) - 58638: 848, // UsernameList (6x) - 58676: 849, // WithClustered (6x) - 58104: 850, // AlgorithmClause (5x) - 58145: 851, // BeginTransactionStmt (5x) - 58156: 852, // ByItem (5x) - 58168: 853, // CollationName (5x) - 58172: 854, // ColumnKeywordOpt (5x) - 58188: 855, // CommitStmt (5x) - 58277: 856, // FieldOpt (5x) - 58278: 857, // FieldOpts (5x) - 58318: 858, // IdentList (5x) - 58338: 859, // IndexName (5x) - 58341: 860, // IndexOption (5x) - 58342: 861, // IndexOptionList (5x) + 58598: 847, // TableOptimizerHints (6x) + 58637: 848, // UsernameList (6x) + 58675: 849, // WithClustered (6x) + 58103: 850, // AlgorithmClause (5x) + 58144: 851, // BeginTransactionStmt (5x) + 58155: 852, // ByItem (5x) + 58167: 853, // CollationName (5x) + 58171: 854, // ColumnKeywordOpt (5x) + 58187: 855, // CommitStmt (5x) + 58276: 856, // FieldOpt (5x) + 58277: 857, // FieldOpts (5x) + 58317: 858, // IdentList (5x) + 58337: 859, // IndexName (5x) + 58340: 860, // IndexOption (5x) + 58341: 861, // IndexOptionList (5x) 57438: 862, // infile (5x) - 58368: 863, // LimitOption (5x) - 58376: 864, // LoadDataStmt (5x) - 58380: 865, // LockClause (5x) - 58413: 866, // OptCharsetWithOptBinary (5x) - 58424: 867, // OptNullTreatment (5x) - 58464: 868, // PolicyName (5x) - 58471: 869, // PriorityOpt (5x) - 58506: 870, // RollbackStmt (5x) - 58511: 871, // SelectLockOpt (5x) - 58518: 872, // SelectStmtIntoOption (5x) - 58540: 873, // SetStmt (5x) - 58605: 874, // TableRefs (5x) - 58631: 875, // UserSpec (5x) - 58130: 876, // Assignment (4x) - 58136: 877, // AuthString (4x) - 58147: 878, // BindableStmt (4x) - 58137: 879, // BRIEBooleanOptionName (4x) - 58138: 880, // BRIEIntegerOptionName (4x) - 58139: 881, // BRIEKeywordOptionName (4x) - 58140: 882, // BRIEOption (4x) - 58141: 883, // BRIEOptions (4x) - 58143: 884, // BRIEStringOptionName (4x) - 58157: 885, // ByList (4x) - 58161: 886, // Char (4x) - 58192: 887, // ConfigItemName (4x) - 58196: 888, // Constraint (4x) - 58286: 889, // FloatOpt (4x) - 58347: 890, // IndexTypeName (4x) + 58367: 863, // LimitOption (5x) + 58375: 864, // LoadDataStmt (5x) + 58379: 865, // LockClause (5x) + 58412: 866, // OptCharsetWithOptBinary (5x) + 58423: 867, // OptNullTreatment (5x) + 58463: 868, // PolicyName (5x) + 58470: 869, // PriorityOpt (5x) + 58505: 870, // RollbackStmt (5x) + 58510: 871, // SelectLockOpt (5x) + 58517: 872, // SelectStmtIntoOption (5x) + 58539: 873, // SetStmt (5x) + 58604: 874, // TableRefs (5x) + 58630: 875, // UserSpec (5x) + 58129: 876, // Assignment (4x) + 58135: 877, // AuthString (4x) + 58146: 878, // BindableStmt (4x) + 58136: 879, // BRIEBooleanOptionName (4x) + 58137: 880, // BRIEIntegerOptionName (4x) + 58138: 881, // BRIEKeywordOptionName (4x) + 58139: 882, // BRIEOption (4x) + 58140: 883, // BRIEOptions (4x) + 58142: 884, // BRIEStringOptionName (4x) + 58156: 885, // ByList (4x) + 58160: 886, // Char (4x) + 58191: 887, // ConfigItemName (4x) + 58195: 888, // Constraint (4x) + 58285: 889, // FloatOpt (4x) + 58346: 890, // IndexTypeName (4x) 57490: 891, // option (4x) - 58429: 892, // OptWild (4x) + 58428: 892, // OptWild (4x) 57494: 893, // outer (4x) - 58465: 894, // Precision (4x) - 58479: 895, // ReferDef (4x) - 58493: 896, // RestrictOrCascadeOpt (4x) - 58509: 897, // RowStmt (4x) - 58526: 898, // SequenceOption (4x) + 58464: 894, // Precision (4x) + 58478: 895, // ReferDef (4x) + 58492: 896, // RestrictOrCascadeOpt (4x) + 58508: 897, // RowStmt (4x) + 58525: 898, // SequenceOption (4x) 57532: 899, // statsExtended (4x) - 58586: 900, // TableAsName (4x) - 58587: 901, // TableAsNameOpt (4x) - 58598: 902, // TableNameOptWild (4x) - 58600: 903, // TableOptimizerHintsOpt (4x) - 58602: 904, // TableOptionList (4x) - 58621: 905, // TransactionChar (4x) - 58632: 906, // UserSpecList (4x) - 58670: 907, // WindowName (4x) - 58127: 908, // AsOfClause (3x) - 58131: 909, // AssignmentList (3x) - 58133: 910, // AttributesOpt (3x) - 58153: 911, // Boolean (3x) - 58181: 912, // ColumnOption (3x) - 58184: 913, // ColumnPosition (3x) - 58189: 914, // CommonTableExpr (3x) - 58210: 915, // CreateTableStmt (3x) - 58218: 916, // DatabaseOptionList (3x) - 58226: 917, // DefaultTrueDistinctOpt (3x) - 58251: 918, // EnforcedOrNot (3x) + 58585: 900, // TableAsName (4x) + 58586: 901, // TableAsNameOpt (4x) + 58597: 902, // TableNameOptWild (4x) + 58599: 903, // TableOptimizerHintsOpt (4x) + 58601: 904, // TableOptionList (4x) + 58620: 905, // TransactionChar (4x) + 58631: 906, // UserSpecList (4x) + 58669: 907, // WindowName (4x) + 58126: 908, // AsOfClause (3x) + 58130: 909, // AssignmentList (3x) + 58132: 910, // AttributesOpt (3x) + 58152: 911, // Boolean (3x) + 58180: 912, // ColumnOption (3x) + 58183: 913, // ColumnPosition (3x) + 58188: 914, // CommonTableExpr (3x) + 58209: 915, // CreateTableStmt (3x) + 58217: 916, // DatabaseOptionList (3x) + 58225: 917, // DefaultTrueDistinctOpt (3x) + 58250: 918, // EnforcedOrNot (3x) 57414: 919, // explain (3x) - 58268: 920, // ExtendedPriv (3x) - 58306: 921, // GeneratedAlways (3x) - 58308: 922, // GlobalScope (3x) - 58312: 923, // GroupByClause (3x) - 58330: 924, // IndexHint (3x) - 58334: 925, // IndexHintType (3x) - 58339: 926, // IndexNameAndTypeOpt (3x) + 58267: 920, // ExtendedPriv (3x) + 58305: 921, // GeneratedAlways (3x) + 58307: 922, // GlobalScope (3x) + 58311: 923, // GroupByClause (3x) + 58329: 924, // IndexHint (3x) + 58333: 925, // IndexHintType (3x) + 58338: 926, // IndexNameAndTypeOpt (3x) 57455: 927, // keys (3x) - 58370: 928, // Lines (3x) - 58388: 929, // MaxValueOrExpression (3x) - 58425: 930, // OptOrder (3x) - 58428: 931, // OptTemporary (3x) - 58441: 932, // PartDefOptionList (3x) - 58443: 933, // PartitionDefinition (3x) - 58452: 934, // PasswordExpire (3x) - 58454: 935, // PasswordOrLockOption (3x) - 58463: 936, // PluginNameList (3x) - 58469: 937, // PrimaryOpt (3x) - 58472: 938, // PrivElem (3x) - 58474: 939, // PrivType (3x) + 58369: 928, // Lines (3x) + 58387: 929, // MaxValueOrExpression (3x) + 58424: 930, // OptOrder (3x) + 58427: 931, // OptTemporary (3x) + 58440: 932, // PartDefOptionList (3x) + 58442: 933, // PartitionDefinition (3x) + 58451: 934, // PasswordExpire (3x) + 58453: 935, // PasswordOrLockOption (3x) + 58462: 936, // PluginNameList (3x) + 58468: 937, // PrimaryOpt (3x) + 58471: 938, // PrivElem (3x) + 58473: 939, // PrivType (3x) 57500: 940, // procedure (3x) - 58488: 941, // RequireClause (3x) - 58489: 942, // RequireClauseOpt (3x) - 58491: 943, // RequireListElement (3x) - 58505: 944, // RolenameWithoutIdent (3x) - 58498: 945, // RoleOrPrivElem (3x) - 58517: 946, // SelectStmtGroup (3x) - 58534: 947, // SetOprOpt (3x) - 58585: 948, // TableAliasRefList (3x) - 58588: 949, // TableElement (3x) - 58597: 950, // TableNameListOpt2 (3x) - 58613: 951, // TextString (3x) - 58620: 952, // TraceableStmt (3x) - 58622: 953, // TransactionChars (3x) + 58487: 941, // RequireClause (3x) + 58488: 942, // RequireClauseOpt (3x) + 58490: 943, // RequireListElement (3x) + 58504: 944, // RolenameWithoutIdent (3x) + 58497: 945, // RoleOrPrivElem (3x) + 58516: 946, // SelectStmtGroup (3x) + 58533: 947, // SetOprOpt (3x) + 58584: 948, // TableAliasRefList (3x) + 58587: 949, // TableElement (3x) + 58596: 950, // TableNameListOpt2 (3x) + 58612: 951, // TextString (3x) + 58619: 952, // TraceableStmt (3x) + 58621: 953, // TransactionChars (3x) 57544: 954, // trigger (3x) 57548: 955, // unlock (3x) 57551: 956, // usage (3x) - 58642: 957, // ValuesList (3x) - 58644: 958, // ValuesStmtList (3x) - 58640: 959, // ValueSym (3x) - 58647: 960, // VariableAssignment (3x) - 58667: 961, // WindowFrameStart (3x) - 58103: 962, // AdminStmt (2x) - 58105: 963, // AllColumnsOrPredicateColumnsOpt (2x) - 58107: 964, // AlterDatabaseStmt (2x) - 58108: 965, // AlterImportStmt (2x) - 58109: 966, // AlterInstanceStmt (2x) - 58110: 967, // AlterOrderItem (2x) - 58112: 968, // AlterPolicyStmt (2x) - 58113: 969, // AlterSequenceOption (2x) - 58115: 970, // AlterSequenceStmt (2x) - 58117: 971, // AlterTableSpec (2x) - 58121: 972, // AlterUserStmt (2x) - 58122: 973, // AnalyzeOption (2x) - 58125: 974, // AnalyzeTableStmt (2x) - 58148: 975, // BinlogStmt (2x) - 58142: 976, // BRIEStmt (2x) - 58144: 977, // BRIETables (2x) + 58641: 957, // ValuesList (3x) + 58643: 958, // ValuesStmtList (3x) + 58639: 959, // ValueSym (3x) + 58646: 960, // VariableAssignment (3x) + 58666: 961, // WindowFrameStart (3x) + 58102: 962, // AdminStmt (2x) + 58104: 963, // AllColumnsOrPredicateColumnsOpt (2x) + 58106: 964, // AlterDatabaseStmt (2x) + 58107: 965, // AlterImportStmt (2x) + 58108: 966, // AlterInstanceStmt (2x) + 58109: 967, // AlterOrderItem (2x) + 58111: 968, // AlterPolicyStmt (2x) + 58112: 969, // AlterSequenceOption (2x) + 58114: 970, // AlterSequenceStmt (2x) + 58116: 971, // AlterTableSpec (2x) + 58120: 972, // AlterUserStmt (2x) + 58121: 973, // AnalyzeOption (2x) + 58124: 974, // AnalyzeTableStmt (2x) + 58147: 975, // BinlogStmt (2x) + 58141: 976, // BRIEStmt (2x) + 58143: 977, // BRIETables (2x) 57372: 978, // call (2x) - 58158: 979, // CallStmt (2x) - 58159: 980, // CastType (2x) - 58160: 981, // ChangeStmt (2x) - 58166: 982, // CheckConstraintKeyword (2x) - 58176: 983, // ColumnNameListOpt (2x) - 58179: 984, // ColumnNameOrUserVariable (2x) - 58182: 985, // ColumnOptionList (2x) - 58183: 986, // ColumnOptionListOpt (2x) - 58185: 987, // ColumnSetValue (2x) - 58191: 988, // CompletionTypeWithinTransaction (2x) - 58193: 989, // ConnectionOption (2x) - 58195: 990, // ConnectionOptions (2x) - 58199: 991, // CreateBindingStmt (2x) - 58200: 992, // CreateDatabaseStmt (2x) - 58201: 993, // CreateImportStmt (2x) - 58202: 994, // CreateIndexStmt (2x) - 58203: 995, // CreatePolicyStmt (2x) - 58204: 996, // CreateRoleStmt (2x) - 58206: 997, // CreateSequenceStmt (2x) - 58207: 998, // CreateStatisticsStmt (2x) - 58208: 999, // CreateTableOptionListOpt (2x) - 58211: 1000, // CreateUserStmt (2x) - 58213: 1001, // CreateViewStmt (2x) + 58157: 979, // CallStmt (2x) + 58158: 980, // CastType (2x) + 58159: 981, // ChangeStmt (2x) + 58165: 982, // CheckConstraintKeyword (2x) + 58175: 983, // ColumnNameListOpt (2x) + 58178: 984, // ColumnNameOrUserVariable (2x) + 58181: 985, // ColumnOptionList (2x) + 58182: 986, // ColumnOptionListOpt (2x) + 58184: 987, // ColumnSetValue (2x) + 58190: 988, // CompletionTypeWithinTransaction (2x) + 58192: 989, // ConnectionOption (2x) + 58194: 990, // ConnectionOptions (2x) + 58198: 991, // CreateBindingStmt (2x) + 58199: 992, // CreateDatabaseStmt (2x) + 58200: 993, // CreateImportStmt (2x) + 58201: 994, // CreateIndexStmt (2x) + 58202: 995, // CreatePolicyStmt (2x) + 58203: 996, // CreateRoleStmt (2x) + 58205: 997, // CreateSequenceStmt (2x) + 58206: 998, // CreateStatisticsStmt (2x) + 58207: 999, // CreateTableOptionListOpt (2x) + 58210: 1000, // CreateUserStmt (2x) + 58212: 1001, // CreateViewStmt (2x) 57392: 1002, // databases (2x) - 58222: 1003, // DeallocateStmt (2x) - 58223: 1004, // DeallocateSym (2x) + 58221: 1003, // DeallocateStmt (2x) + 58222: 1004, // DeallocateSym (2x) 57403: 1005, // describe (2x) - 58234: 1006, // DoStmt (2x) - 58235: 1007, // DropBindingStmt (2x) - 58236: 1008, // DropDatabaseStmt (2x) - 58237: 1009, // DropImportStmt (2x) - 58238: 1010, // DropIndexStmt (2x) - 58239: 1011, // DropPolicyStmt (2x) - 58240: 1012, // DropRoleStmt (2x) - 58241: 1013, // DropSequenceStmt (2x) - 58242: 1014, // DropStatisticsStmt (2x) - 58243: 1015, // DropStatsStmt (2x) - 58244: 1016, // DropTableStmt (2x) - 58245: 1017, // DropUserStmt (2x) - 58246: 1018, // DropViewStmt (2x) - 58247: 1019, // DuplicateOpt (2x) - 58249: 1020, // EmptyStmt (2x) - 58250: 1021, // EncryptionOpt (2x) - 58252: 1022, // EnforcedOrNotOpt (2x) - 58256: 1023, // ErrorHandling (2x) - 58258: 1024, // ExecuteStmt (2x) - 58260: 1025, // ExplainStmt (2x) - 58261: 1026, // ExplainSym (2x) - 58270: 1027, // Field (2x) - 58273: 1028, // FieldItem (2x) - 58280: 1029, // Fields (2x) - 58284: 1030, // FlashbackTableStmt (2x) - 58289: 1031, // FlushStmt (2x) - 58295: 1032, // FuncDatetimePrecList (2x) - 58296: 1033, // FuncDatetimePrecListOpt (2x) - 58309: 1034, // GrantProxyStmt (2x) - 58310: 1035, // GrantRoleStmt (2x) - 58311: 1036, // GrantStmt (2x) - 58313: 1037, // HandleRange (2x) - 58315: 1038, // HashString (2x) - 58317: 1039, // HelpStmt (2x) - 58329: 1040, // IndexAdviseStmt (2x) - 58331: 1041, // IndexHintList (2x) - 58332: 1042, // IndexHintListOpt (2x) - 58337: 1043, // IndexLockAndAlgorithmOpt (2x) - 58350: 1044, // InsertValues (2x) - 58354: 1045, // IntoOpt (2x) - 58360: 1046, // KeyOrIndexOpt (2x) + 58233: 1006, // DoStmt (2x) + 58234: 1007, // DropBindingStmt (2x) + 58235: 1008, // DropDatabaseStmt (2x) + 58236: 1009, // DropImportStmt (2x) + 58237: 1010, // DropIndexStmt (2x) + 58238: 1011, // DropPolicyStmt (2x) + 58239: 1012, // DropRoleStmt (2x) + 58240: 1013, // DropSequenceStmt (2x) + 58241: 1014, // DropStatisticsStmt (2x) + 58242: 1015, // DropStatsStmt (2x) + 58243: 1016, // DropTableStmt (2x) + 58244: 1017, // DropUserStmt (2x) + 58245: 1018, // DropViewStmt (2x) + 58246: 1019, // DuplicateOpt (2x) + 58248: 1020, // EmptyStmt (2x) + 58249: 1021, // EncryptionOpt (2x) + 58251: 1022, // EnforcedOrNotOpt (2x) + 58255: 1023, // ErrorHandling (2x) + 58257: 1024, // ExecuteStmt (2x) + 58259: 1025, // ExplainStmt (2x) + 58260: 1026, // ExplainSym (2x) + 58269: 1027, // Field (2x) + 58272: 1028, // FieldItem (2x) + 58279: 1029, // Fields (2x) + 58283: 1030, // FlashbackTableStmt (2x) + 58288: 1031, // FlushStmt (2x) + 58294: 1032, // FuncDatetimePrecList (2x) + 58295: 1033, // FuncDatetimePrecListOpt (2x) + 58308: 1034, // GrantProxyStmt (2x) + 58309: 1035, // GrantRoleStmt (2x) + 58310: 1036, // GrantStmt (2x) + 58312: 1037, // HandleRange (2x) + 58314: 1038, // HashString (2x) + 58316: 1039, // HelpStmt (2x) + 58328: 1040, // IndexAdviseStmt (2x) + 58330: 1041, // IndexHintList (2x) + 58331: 1042, // IndexHintListOpt (2x) + 58336: 1043, // IndexLockAndAlgorithmOpt (2x) + 58349: 1044, // InsertValues (2x) + 58353: 1045, // IntoOpt (2x) + 58359: 1046, // KeyOrIndexOpt (2x) 57456: 1047, // kill (2x) - 58361: 1048, // KillOrKillTiDB (2x) - 58362: 1049, // KillStmt (2x) - 58367: 1050, // LimitClause (2x) + 58360: 1048, // KillOrKillTiDB (2x) + 58361: 1049, // KillStmt (2x) + 58366: 1050, // LimitClause (2x) 57465: 1051, // linear (2x) - 58369: 1052, // LinearOpt (2x) - 58373: 1053, // LoadDataSetItem (2x) - 58377: 1054, // LoadStatsStmt (2x) - 58378: 1055, // LocalOpt (2x) - 58381: 1056, // LockTablesStmt (2x) - 58389: 1057, // MaxValueOrExpressionList (2x) - 58397: 1058, // NowSym (2x) - 58398: 1059, // NowSymFunc (2x) - 58399: 1060, // NowSymOptionFraction (2x) - 58400: 1061, // NumList (2x) - 58403: 1062, // ObjectType (2x) + 58368: 1052, // LinearOpt (2x) + 58372: 1053, // LoadDataSetItem (2x) + 58376: 1054, // LoadStatsStmt (2x) + 58377: 1055, // LocalOpt (2x) + 58380: 1056, // LockTablesStmt (2x) + 58388: 1057, // MaxValueOrExpressionList (2x) + 58396: 1058, // NowSym (2x) + 58397: 1059, // NowSymFunc (2x) + 58398: 1060, // NowSymOptionFraction (2x) + 58399: 1061, // NumList (2x) + 58402: 1062, // ObjectType (2x) 57487: 1063, // of (2x) - 58404: 1064, // OfTablesOpt (2x) - 58405: 1065, // OnCommitOpt (2x) - 58406: 1066, // OnDelete (2x) - 58409: 1067, // OnUpdate (2x) - 58414: 1068, // OptCollate (2x) - 58419: 1069, // OptFull (2x) - 58421: 1070, // OptInteger (2x) - 58434: 1071, // OptionalBraces (2x) - 58433: 1072, // OptionLevel (2x) - 58423: 1073, // OptLeadLagInfo (2x) - 58422: 1074, // OptLLDefault (2x) - 58439: 1075, // OuterOpt (2x) - 58444: 1076, // PartitionDefinitionList (2x) - 58445: 1077, // PartitionDefinitionListOpt (2x) - 58451: 1078, // PartitionOpt (2x) - 58453: 1079, // PasswordOpt (2x) - 58455: 1080, // PasswordOrLockOptionList (2x) - 58456: 1081, // PasswordOrLockOptions (2x) - 58460: 1082, // PlacementOptionList (2x) - 58462: 1083, // PlanReplayerStmt (2x) - 58468: 1084, // PreparedStmt (2x) - 58473: 1085, // PrivLevel (2x) - 58476: 1086, // PurgeImportStmt (2x) - 58477: 1087, // QuickOptional (2x) - 58478: 1088, // RecoverTableStmt (2x) - 58480: 1089, // ReferOpt (2x) - 58482: 1090, // RegexpSym (2x) - 58483: 1091, // RenameTableStmt (2x) - 58484: 1092, // RenameUserStmt (2x) - 58486: 1093, // RepeatableOpt (2x) - 58492: 1094, // RestartStmt (2x) - 58494: 1095, // ResumeImportStmt (2x) + 58403: 1064, // OfTablesOpt (2x) + 58404: 1065, // OnCommitOpt (2x) + 58405: 1066, // OnDelete (2x) + 58408: 1067, // OnUpdate (2x) + 58413: 1068, // OptCollate (2x) + 58418: 1069, // OptFull (2x) + 58420: 1070, // OptInteger (2x) + 58433: 1071, // OptionalBraces (2x) + 58432: 1072, // OptionLevel (2x) + 58422: 1073, // OptLeadLagInfo (2x) + 58421: 1074, // OptLLDefault (2x) + 58438: 1075, // OuterOpt (2x) + 58443: 1076, // PartitionDefinitionList (2x) + 58444: 1077, // PartitionDefinitionListOpt (2x) + 58450: 1078, // PartitionOpt (2x) + 58452: 1079, // PasswordOpt (2x) + 58454: 1080, // PasswordOrLockOptionList (2x) + 58455: 1081, // PasswordOrLockOptions (2x) + 58459: 1082, // PlacementOptionList (2x) + 58461: 1083, // PlanReplayerStmt (2x) + 58467: 1084, // PreparedStmt (2x) + 58472: 1085, // PrivLevel (2x) + 58475: 1086, // PurgeImportStmt (2x) + 58476: 1087, // QuickOptional (2x) + 58477: 1088, // RecoverTableStmt (2x) + 58479: 1089, // ReferOpt (2x) + 58481: 1090, // RegexpSym (2x) + 58482: 1091, // RenameTableStmt (2x) + 58483: 1092, // RenameUserStmt (2x) + 58485: 1093, // RepeatableOpt (2x) + 58491: 1094, // RestartStmt (2x) + 58493: 1095, // ResumeImportStmt (2x) 57514: 1096, // revoke (2x) - 58495: 1097, // RevokeRoleStmt (2x) - 58496: 1098, // RevokeStmt (2x) - 58499: 1099, // RoleOrPrivElemList (2x) - 58500: 1100, // RoleSpec (2x) - 58521: 1101, // SelectStmtOpt (2x) - 58524: 1102, // SelectStmtSQLCache (2x) - 58528: 1103, // SetDefaultRoleOpt (2x) - 58529: 1104, // SetDefaultRoleStmt (2x) - 58539: 1105, // SetRoleStmt (2x) - 58542: 1106, // ShowImportStmt (2x) - 58547: 1107, // ShowProfileType (2x) - 58550: 1108, // ShowStmt (2x) - 58551: 1109, // ShowTableAliasOpt (2x) - 58553: 1110, // ShutdownStmt (2x) - 58554: 1111, // SignedLiteral (2x) - 58558: 1112, // SplitOption (2x) - 58559: 1113, // SplitRegionStmt (2x) - 58563: 1114, // Statement (2x) - 58565: 1115, // StatsOptionsOpt (2x) - 58566: 1116, // StatsPersistentVal (2x) - 58567: 1117, // StatsType (2x) - 58568: 1118, // StopImportStmt (2x) - 58575: 1119, // SubPartDefinition (2x) - 58578: 1120, // SubPartitionMethod (2x) - 58583: 1121, // Symbol (2x) - 58589: 1122, // TableElementList (2x) - 58592: 1123, // TableLock (2x) - 58596: 1124, // TableNameListOpt (2x) - 58603: 1125, // TableOrTables (2x) - 58612: 1126, // TablesTerminalSym (2x) - 58610: 1127, // TableToTable (2x) - 58614: 1128, // TextStringList (2x) - 58619: 1129, // TraceStmt (2x) - 58624: 1130, // TruncateTableStmt (2x) - 58627: 1131, // UnlockTablesStmt (2x) - 58633: 1132, // UserToUser (2x) - 58630: 1133, // UseStmt (2x) - 58645: 1134, // Varchar (2x) - 58648: 1135, // VariableAssignmentList (2x) - 58657: 1136, // WhenClause (2x) - 58662: 1137, // WindowDefinition (2x) - 58665: 1138, // WindowFrameBound (2x) - 58672: 1139, // WindowSpec (2x) - 58677: 1140, // WithGrantOptionOpt (2x) - 58678: 1141, // WithList (2x) - 58682: 1142, // Writeable (2x) - 58102: 1143, // AdminShowSlow (1x) - 58111: 1144, // AlterOrderList (1x) - 58114: 1145, // AlterSequenceOptionList (1x) - 58116: 1146, // AlterTablePartitionOpt (1x) - 58118: 1147, // AlterTableSpecList (1x) - 58119: 1148, // AlterTableSpecListOpt (1x) - 58123: 1149, // AnalyzeOptionList (1x) - 58126: 1150, // AnyOrAll (1x) - 58128: 1151, // AsOfClauseOpt (1x) - 58129: 1152, // AsOpt (1x) - 58134: 1153, // AuthOption (1x) - 58135: 1154, // AuthPlugin (1x) - 58146: 1155, // BetweenOrNotOp (1x) - 58150: 1156, // BitValueType (1x) - 58151: 1157, // BlobType (1x) - 58154: 1158, // BooleanType (1x) + 58494: 1097, // RevokeRoleStmt (2x) + 58495: 1098, // RevokeStmt (2x) + 58498: 1099, // RoleOrPrivElemList (2x) + 58499: 1100, // RoleSpec (2x) + 58520: 1101, // SelectStmtOpt (2x) + 58523: 1102, // SelectStmtSQLCache (2x) + 58527: 1103, // SetDefaultRoleOpt (2x) + 58528: 1104, // SetDefaultRoleStmt (2x) + 58538: 1105, // SetRoleStmt (2x) + 58541: 1106, // ShowImportStmt (2x) + 58546: 1107, // ShowProfileType (2x) + 58549: 1108, // ShowStmt (2x) + 58550: 1109, // ShowTableAliasOpt (2x) + 58552: 1110, // ShutdownStmt (2x) + 58553: 1111, // SignedLiteral (2x) + 58557: 1112, // SplitOption (2x) + 58558: 1113, // SplitRegionStmt (2x) + 58562: 1114, // Statement (2x) + 58564: 1115, // StatsOptionsOpt (2x) + 58565: 1116, // StatsPersistentVal (2x) + 58566: 1117, // StatsType (2x) + 58567: 1118, // StopImportStmt (2x) + 58574: 1119, // SubPartDefinition (2x) + 58577: 1120, // SubPartitionMethod (2x) + 58582: 1121, // Symbol (2x) + 58588: 1122, // TableElementList (2x) + 58591: 1123, // TableLock (2x) + 58595: 1124, // TableNameListOpt (2x) + 58602: 1125, // TableOrTables (2x) + 58611: 1126, // TablesTerminalSym (2x) + 58609: 1127, // TableToTable (2x) + 58613: 1128, // TextStringList (2x) + 58618: 1129, // TraceStmt (2x) + 58623: 1130, // TruncateTableStmt (2x) + 58626: 1131, // UnlockTablesStmt (2x) + 58632: 1132, // UserToUser (2x) + 58629: 1133, // UseStmt (2x) + 58644: 1134, // Varchar (2x) + 58647: 1135, // VariableAssignmentList (2x) + 58656: 1136, // WhenClause (2x) + 58661: 1137, // WindowDefinition (2x) + 58664: 1138, // WindowFrameBound (2x) + 58671: 1139, // WindowSpec (2x) + 58676: 1140, // WithGrantOptionOpt (2x) + 58677: 1141, // WithList (2x) + 58681: 1142, // Writeable (2x) + 58101: 1143, // AdminShowSlow (1x) + 58110: 1144, // AlterOrderList (1x) + 58113: 1145, // AlterSequenceOptionList (1x) + 58115: 1146, // AlterTablePartitionOpt (1x) + 58117: 1147, // AlterTableSpecList (1x) + 58118: 1148, // AlterTableSpecListOpt (1x) + 58122: 1149, // AnalyzeOptionList (1x) + 58125: 1150, // AnyOrAll (1x) + 58127: 1151, // AsOfClauseOpt (1x) + 58128: 1152, // AsOpt (1x) + 58133: 1153, // AuthOption (1x) + 58134: 1154, // AuthPlugin (1x) + 58145: 1155, // BetweenOrNotOp (1x) + 58149: 1156, // BitValueType (1x) + 58150: 1157, // BlobType (1x) + 58153: 1158, // BooleanType (1x) 57370: 1159, // both (1x) - 58164: 1160, // CharsetNameOrDefault (1x) - 58165: 1161, // CharsetOpt (1x) - 58167: 1162, // ClearPasswordExpireOptions (1x) - 58171: 1163, // ColumnFormat (1x) - 58173: 1164, // ColumnList (1x) - 58180: 1165, // ColumnNameOrUserVariableList (1x) - 58177: 1166, // ColumnNameOrUserVarListOpt (1x) - 58178: 1167, // ColumnNameOrUserVarListOptWithBrackets (1x) - 58186: 1168, // ColumnSetValueList (1x) - 58190: 1169, // CompareOp (1x) - 58194: 1170, // ConnectionOptionList (1x) - 58197: 1171, // ConstraintElem (1x) - 58205: 1172, // CreateSequenceOptionListOpt (1x) - 58209: 1173, // CreateTableSelectOpt (1x) - 58212: 1174, // CreateViewSelectOpt (1x) - 58219: 1175, // DatabaseOptionListOpt (1x) - 58221: 1176, // DateAndTimeType (1x) - 58216: 1177, // DBNameList (1x) - 58227: 1178, // DefaultValueExpr (1x) + 58163: 1160, // CharsetNameOrDefault (1x) + 58164: 1161, // CharsetOpt (1x) + 58166: 1162, // ClearPasswordExpireOptions (1x) + 58170: 1163, // ColumnFormat (1x) + 58172: 1164, // ColumnList (1x) + 58179: 1165, // ColumnNameOrUserVariableList (1x) + 58176: 1166, // ColumnNameOrUserVarListOpt (1x) + 58177: 1167, // ColumnNameOrUserVarListOptWithBrackets (1x) + 58185: 1168, // ColumnSetValueList (1x) + 58189: 1169, // CompareOp (1x) + 58193: 1170, // ConnectionOptionList (1x) + 58196: 1171, // ConstraintElem (1x) + 58204: 1172, // CreateSequenceOptionListOpt (1x) + 58208: 1173, // CreateTableSelectOpt (1x) + 58211: 1174, // CreateViewSelectOpt (1x) + 58218: 1175, // DatabaseOptionListOpt (1x) + 58220: 1176, // DateAndTimeType (1x) + 58215: 1177, // DBNameList (1x) + 58226: 1178, // DefaultValueExpr (1x) 57409: 1179, // dual (1x) - 58248: 1180, // ElseOpt (1x) - 58253: 1181, // EnforcedOrNotOrNotNullOpt (1x) - 58259: 1182, // ExplainFormatType (1x) - 58267: 1183, // ExpressionOpt (1x) - 58269: 1184, // FetchFirstOpt (1x) - 58271: 1185, // FieldAsName (1x) - 58272: 1186, // FieldAsNameOpt (1x) - 58274: 1187, // FieldItemList (1x) - 58276: 1188, // FieldList (1x) - 58282: 1189, // FirstOrNext (1x) - 58283: 1190, // FixedPointType (1x) - 58285: 1191, // FlashbackToNewName (1x) - 58287: 1192, // FloatingPointType (1x) - 58288: 1193, // FlushOption (1x) - 58291: 1194, // FromDual (1x) - 58293: 1195, // FulltextSearchModifierOpt (1x) - 58294: 1196, // FuncDatetimePrec (1x) - 58307: 1197, // GetFormatSelector (1x) - 58314: 1198, // HandleRangeList (1x) - 58316: 1199, // HavingClause (1x) - 58319: 1200, // IdentListWithParenOpt (1x) - 58323: 1201, // IfNotRunning (1x) - 58324: 1202, // IfRunning (1x) - 58325: 1203, // IgnoreLines (1x) - 58327: 1204, // ImportTruncate (1x) - 58333: 1205, // IndexHintScope (1x) - 58336: 1206, // IndexKeyTypeOpt (1x) - 58345: 1207, // IndexPartSpecificationListOpt (1x) - 58348: 1208, // IndexTypeOpt (1x) - 58328: 1209, // InOrNotOp (1x) - 58351: 1210, // InstanceOption (1x) - 58353: 1211, // IntegerType (1x) - 58356: 1212, // IsolationLevel (1x) - 58355: 1213, // IsOrNotOp (1x) + 58247: 1180, // ElseOpt (1x) + 58252: 1181, // EnforcedOrNotOrNotNullOpt (1x) + 58258: 1182, // ExplainFormatType (1x) + 58266: 1183, // ExpressionOpt (1x) + 58268: 1184, // FetchFirstOpt (1x) + 58270: 1185, // FieldAsName (1x) + 58271: 1186, // FieldAsNameOpt (1x) + 58273: 1187, // FieldItemList (1x) + 58275: 1188, // FieldList (1x) + 58281: 1189, // FirstOrNext (1x) + 58282: 1190, // FixedPointType (1x) + 58284: 1191, // FlashbackToNewName (1x) + 58286: 1192, // FloatingPointType (1x) + 58287: 1193, // FlushOption (1x) + 58290: 1194, // FromDual (1x) + 58292: 1195, // FulltextSearchModifierOpt (1x) + 58293: 1196, // FuncDatetimePrec (1x) + 58306: 1197, // GetFormatSelector (1x) + 58313: 1198, // HandleRangeList (1x) + 58315: 1199, // HavingClause (1x) + 58318: 1200, // IdentListWithParenOpt (1x) + 58322: 1201, // IfNotRunning (1x) + 58323: 1202, // IfRunning (1x) + 58324: 1203, // IgnoreLines (1x) + 58326: 1204, // ImportTruncate (1x) + 58332: 1205, // IndexHintScope (1x) + 58335: 1206, // IndexKeyTypeOpt (1x) + 58344: 1207, // IndexPartSpecificationListOpt (1x) + 58347: 1208, // IndexTypeOpt (1x) + 58327: 1209, // InOrNotOp (1x) + 58350: 1210, // InstanceOption (1x) + 58352: 1211, // IntegerType (1x) + 58355: 1212, // IsolationLevel (1x) + 58354: 1213, // IsOrNotOp (1x) 57460: 1214, // leading (1x) - 58364: 1215, // LikeEscapeOpt (1x) - 58365: 1216, // LikeOrNotOp (1x) - 58366: 1217, // LikeTableWithOrWithoutParen (1x) - 58371: 1218, // LinesTerminated (1x) - 58374: 1219, // LoadDataSetList (1x) - 58375: 1220, // LoadDataSetSpecOpt (1x) - 58379: 1221, // LocationLabelList (1x) - 58382: 1222, // LockType (1x) - 58383: 1223, // LogTypeOpt (1x) - 58384: 1224, // Match (1x) - 58385: 1225, // MatchOpt (1x) - 58386: 1226, // MaxIndexNumOpt (1x) - 58387: 1227, // MaxMinutesOpt (1x) - 58390: 1228, // NChar (1x) - 58402: 1229, // NumericType (1x) - 58392: 1230, // NVarchar (1x) - 58407: 1231, // OnDeleteUpdateOpt (1x) - 58408: 1232, // OnDuplicateKeyUpdate (1x) - 58410: 1233, // OptBinMod (1x) - 58412: 1234, // OptCharset (1x) - 58415: 1235, // OptErrors (1x) - 58416: 1236, // OptExistingWindowName (1x) - 58418: 1237, // OptFromFirstLast (1x) - 58420: 1238, // OptGConcatSeparator (1x) - 58426: 1239, // OptPartitionClause (1x) - 58427: 1240, // OptTable (1x) - 58430: 1241, // OptWindowFrameClause (1x) - 58431: 1242, // OptWindowOrderByClause (1x) - 58436: 1243, // Order (1x) - 58435: 1244, // OrReplace (1x) + 58363: 1215, // LikeEscapeOpt (1x) + 58364: 1216, // LikeOrNotOp (1x) + 58365: 1217, // LikeTableWithOrWithoutParen (1x) + 58370: 1218, // LinesTerminated (1x) + 58373: 1219, // LoadDataSetList (1x) + 58374: 1220, // LoadDataSetSpecOpt (1x) + 58378: 1221, // LocationLabelList (1x) + 58381: 1222, // LockType (1x) + 58382: 1223, // LogTypeOpt (1x) + 58383: 1224, // Match (1x) + 58384: 1225, // MatchOpt (1x) + 58385: 1226, // MaxIndexNumOpt (1x) + 58386: 1227, // MaxMinutesOpt (1x) + 58389: 1228, // NChar (1x) + 58401: 1229, // NumericType (1x) + 58391: 1230, // NVarchar (1x) + 58406: 1231, // OnDeleteUpdateOpt (1x) + 58407: 1232, // OnDuplicateKeyUpdate (1x) + 58409: 1233, // OptBinMod (1x) + 58411: 1234, // OptCharset (1x) + 58414: 1235, // OptErrors (1x) + 58415: 1236, // OptExistingWindowName (1x) + 58417: 1237, // OptFromFirstLast (1x) + 58419: 1238, // OptGConcatSeparator (1x) + 58425: 1239, // OptPartitionClause (1x) + 58426: 1240, // OptTable (1x) + 58429: 1241, // OptWindowFrameClause (1x) + 58430: 1242, // OptWindowOrderByClause (1x) + 58435: 1243, // Order (1x) + 58434: 1244, // OrReplace (1x) 57444: 1245, // outfile (1x) - 58442: 1246, // PartDefValuesOpt (1x) - 58446: 1247, // PartitionKeyAlgorithmOpt (1x) - 58447: 1248, // PartitionMethod (1x) - 58450: 1249, // PartitionNumOpt (1x) - 58457: 1250, // PerDB (1x) - 58458: 1251, // PerTable (1x) + 58441: 1246, // PartDefValuesOpt (1x) + 58445: 1247, // PartitionKeyAlgorithmOpt (1x) + 58446: 1248, // PartitionMethod (1x) + 58449: 1249, // PartitionNumOpt (1x) + 58456: 1250, // PerDB (1x) + 58457: 1251, // PerTable (1x) 57498: 1252, // precisionType (1x) - 58467: 1253, // PrepareSQL (1x) - 58475: 1254, // ProcedureCall (1x) + 58466: 1253, // PrepareSQL (1x) + 58474: 1254, // ProcedureCall (1x) 57505: 1255, // recursive (1x) - 58481: 1256, // RegexpOrNotOp (1x) - 58485: 1257, // ReorganizePartitionRuleOpt (1x) - 58490: 1258, // RequireList (1x) - 58501: 1259, // RoleSpecList (1x) - 58508: 1260, // RowOrRows (1x) - 58514: 1261, // SelectStmtFieldList (1x) - 58522: 1262, // SelectStmtOpts (1x) - 58523: 1263, // SelectStmtOptsList (1x) - 58527: 1264, // SequenceOptionList (1x) - 58531: 1265, // SetOpr (1x) - 58538: 1266, // SetRoleOpt (1x) - 58543: 1267, // ShowIndexKwd (1x) - 58544: 1268, // ShowLikeOrWhereOpt (1x) - 58545: 1269, // ShowPlacementTarget (1x) - 58546: 1270, // ShowProfileArgsOpt (1x) - 58548: 1271, // ShowProfileTypes (1x) - 58549: 1272, // ShowProfileTypesOpt (1x) - 58552: 1273, // ShowTargetFilterable (1x) + 58480: 1256, // RegexpOrNotOp (1x) + 58484: 1257, // ReorganizePartitionRuleOpt (1x) + 58489: 1258, // RequireList (1x) + 58500: 1259, // RoleSpecList (1x) + 58507: 1260, // RowOrRows (1x) + 58513: 1261, // SelectStmtFieldList (1x) + 58521: 1262, // SelectStmtOpts (1x) + 58522: 1263, // SelectStmtOptsList (1x) + 58526: 1264, // SequenceOptionList (1x) + 58530: 1265, // SetOpr (1x) + 58537: 1266, // SetRoleOpt (1x) + 58542: 1267, // ShowIndexKwd (1x) + 58543: 1268, // ShowLikeOrWhereOpt (1x) + 58544: 1269, // ShowPlacementTarget (1x) + 58545: 1270, // ShowProfileArgsOpt (1x) + 58547: 1271, // ShowProfileTypes (1x) + 58548: 1272, // ShowProfileTypesOpt (1x) + 58551: 1273, // ShowTargetFilterable (1x) 57525: 1274, // spatial (1x) - 58560: 1275, // SplitSyntaxOption (1x) + 58559: 1275, // SplitSyntaxOption (1x) 57530: 1276, // ssl (1x) - 58561: 1277, // Start (1x) - 58562: 1278, // Starting (1x) + 58560: 1277, // Start (1x) + 58561: 1278, // Starting (1x) 57531: 1279, // starting (1x) - 58564: 1280, // StatementList (1x) - 58569: 1281, // StorageMedia (1x) + 58563: 1280, // StatementList (1x) + 58568: 1281, // StorageMedia (1x) 57536: 1282, // stored (1x) - 58570: 1283, // StringList (1x) - 58573: 1284, // StringNameOrBRIEOptionKeyword (1x) - 58574: 1285, // StringType (1x) - 58576: 1286, // SubPartDefinitionList (1x) - 58577: 1287, // SubPartDefinitionListOpt (1x) - 58579: 1288, // SubPartitionNumOpt (1x) - 58580: 1289, // SubPartitionOpt (1x) - 58590: 1290, // TableElementListOpt (1x) - 58593: 1291, // TableLockList (1x) - 58606: 1292, // TableRefsClause (1x) - 58607: 1293, // TableSampleMethodOpt (1x) - 58608: 1294, // TableSampleOpt (1x) - 58609: 1295, // TableSampleUnitOpt (1x) - 58611: 1296, // TableToTableList (1x) - 58615: 1297, // TextType (1x) + 58569: 1283, // StringList (1x) + 58572: 1284, // StringNameOrBRIEOptionKeyword (1x) + 58573: 1285, // StringType (1x) + 58575: 1286, // SubPartDefinitionList (1x) + 58576: 1287, // SubPartDefinitionListOpt (1x) + 58578: 1288, // SubPartitionNumOpt (1x) + 58579: 1289, // SubPartitionOpt (1x) + 58589: 1290, // TableElementListOpt (1x) + 58592: 1291, // TableLockList (1x) + 58605: 1292, // TableRefsClause (1x) + 58606: 1293, // TableSampleMethodOpt (1x) + 58607: 1294, // TableSampleOpt (1x) + 58608: 1295, // TableSampleUnitOpt (1x) + 58610: 1296, // TableToTableList (1x) + 58614: 1297, // TextType (1x) 57543: 1298, // trailing (1x) - 58623: 1299, // TrimDirection (1x) - 58625: 1300, // Type (1x) - 58634: 1301, // UserToUserList (1x) - 58636: 1302, // UserVariableList (1x) - 58639: 1303, // UsingRoles (1x) - 58641: 1304, // Values (1x) - 58643: 1305, // ValuesOpt (1x) - 58650: 1306, // ViewAlgorithm (1x) - 58651: 1307, // ViewCheckOption (1x) - 58652: 1308, // ViewDefiner (1x) - 58653: 1309, // ViewFieldList (1x) - 58654: 1310, // ViewName (1x) - 58655: 1311, // ViewSQLSecurity (1x) + 58622: 1299, // TrimDirection (1x) + 58624: 1300, // Type (1x) + 58633: 1301, // UserToUserList (1x) + 58635: 1302, // UserVariableList (1x) + 58638: 1303, // UsingRoles (1x) + 58640: 1304, // Values (1x) + 58642: 1305, // ValuesOpt (1x) + 58649: 1306, // ViewAlgorithm (1x) + 58650: 1307, // ViewCheckOption (1x) + 58651: 1308, // ViewDefiner (1x) + 58652: 1309, // ViewFieldList (1x) + 58653: 1310, // ViewName (1x) + 58654: 1311, // ViewSQLSecurity (1x) 57563: 1312, // virtual (1x) - 58656: 1313, // VirtualOrStored (1x) - 58658: 1314, // WhenClauseList (1x) - 58661: 1315, // WindowClauseOptional (1x) - 58663: 1316, // WindowDefinitionList (1x) - 58664: 1317, // WindowFrameBetween (1x) - 58666: 1318, // WindowFrameExtent (1x) - 58668: 1319, // WindowFrameUnits (1x) - 58671: 1320, // WindowNameOrSpec (1x) - 58673: 1321, // WindowSpecDetails (1x) - 58679: 1322, // WithReadLockOpt (1x) - 58680: 1323, // WithValidation (1x) - 58681: 1324, // WithValidationOpt (1x) - 58683: 1325, // Year (1x) - 58101: 1326, // $default (0x) + 58655: 1313, // VirtualOrStored (1x) + 58657: 1314, // WhenClauseList (1x) + 58660: 1315, // WindowClauseOptional (1x) + 58662: 1316, // WindowDefinitionList (1x) + 58663: 1317, // WindowFrameBetween (1x) + 58665: 1318, // WindowFrameExtent (1x) + 58667: 1319, // WindowFrameUnits (1x) + 58670: 1320, // WindowNameOrSpec (1x) + 58672: 1321, // WindowSpecDetails (1x) + 58678: 1322, // WithReadLockOpt (1x) + 58679: 1323, // WithValidation (1x) + 58680: 1324, // WithValidationOpt (1x) + 58682: 1325, // Year (1x) + 58100: 1326, // $default (0x) 58061: 1327, // andnot (0x) - 58132: 1328, // AssignmentListOpt (0x) - 58170: 1329, // ColumnDefList (0x) - 58187: 1330, // CommaOpt (0x) - 58085: 1331, // createTableSelect (0x) + 58131: 1328, // AssignmentListOpt (0x) + 58169: 1329, // ColumnDefList (0x) + 58186: 1330, // CommaOpt (0x) + 58084: 1331, // createTableSelect (0x) 58075: 1332, // empty (0x) 57345: 1333, // error (0x) - 58100: 1334, // higherThanComma (0x) - 58094: 1335, // higherThanParenthese (0x) - 58083: 1336, // insertValues (0x) + 58099: 1334, // higherThanComma (0x) + 58093: 1335, // higherThanParenthese (0x) + 58082: 1336, // insertValues (0x) 57352: 1337, // invalid (0x) - 58086: 1338, // lowerThanCharsetKwd (0x) - 58099: 1339, // lowerThanComma (0x) - 58084: 1340, // lowerThanCreateTableSelect (0x) - 58096: 1341, // lowerThanEq (0x) - 58091: 1342, // lowerThanFunction (0x) - 58082: 1343, // lowerThanInsertValues (0x) - 58077: 1344, // lowerThanIntervalKeyword (0x) - 58087: 1345, // lowerThanKey (0x) - 58088: 1346, // lowerThanLocal (0x) - 58098: 1347, // lowerThanNot (0x) - 58095: 1348, // lowerThanOn (0x) - 58093: 1349, // lowerThanParenthese (0x) - 58089: 1350, // lowerThanRemove (0x) - 58076: 1351, // lowerThanSelectOpt (0x) - 58081: 1352, // lowerThanSelectStmt (0x) - 58080: 1353, // lowerThanSetKeyword (0x) - 58079: 1354, // lowerThanStringLitToken (0x) - 58078: 1355, // lowerThanValueKeyword (0x) - 58090: 1356, // lowerThenOrder (0x) - 58097: 1357, // neg (0x) - 57356: 1358, // odbcDateType (0x) - 57358: 1359, // odbcTimestampType (0x) - 57357: 1360, // odbcTimeType (0x) - 58092: 1361, // tableRefPriority (0x) + 58085: 1338, // lowerThanCharsetKwd (0x) + 58098: 1339, // lowerThanComma (0x) + 58083: 1340, // lowerThanCreateTableSelect (0x) + 58095: 1341, // lowerThanEq (0x) + 58090: 1342, // lowerThanFunction (0x) + 58081: 1343, // lowerThanInsertValues (0x) + 58086: 1344, // lowerThanKey (0x) + 58087: 1345, // lowerThanLocal (0x) + 58097: 1346, // lowerThanNot (0x) + 58094: 1347, // lowerThanOn (0x) + 58092: 1348, // lowerThanParenthese (0x) + 58088: 1349, // lowerThanRemove (0x) + 58076: 1350, // lowerThanSelectOpt (0x) + 58080: 1351, // lowerThanSelectStmt (0x) + 58079: 1352, // lowerThanSetKeyword (0x) + 58078: 1353, // lowerThanStringLitToken (0x) + 58077: 1354, // lowerThanValueKeyword (0x) + 58089: 1355, // lowerThenOrder (0x) + 58096: 1356, // neg (0x) + 57356: 1357, // odbcDateType (0x) + 57358: 1358, // odbcTimestampType (0x) + 57357: 1359, // odbcTimeType (0x) + 58091: 1360, // tableRefPriority (0x) } yySymNames = []string{ @@ -3528,7 +3526,6 @@ var ( "lowerThanEq", "lowerThanFunction", "lowerThanInsertValues", - "lowerThanIntervalKeyword", "lowerThanKey", "lowerThanLocal", "lowerThanNot", @@ -7375,12 +7372,12 @@ var ( // 1140 {1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 460: 1249, 1249, 1249, 1249, 465: 1249, 1249, 468: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 476: 1249, 1249, 479: 1249, 1249, 1249, 1249, 1249, 485: 1249, 487: 1249, 1249, 1249, 1249, 1249, 494: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 530: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 3581, 1249, 1249, 1249, 1249, 1249, 1249}, {1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 460: 1252, 1252, 1252, 1252, 465: 1252, 1252, 468: 1252, 1252, 1252, 1252, 1252, 3580, 1252, 476: 1252, 1252, 479: 1252, 1252, 1252, 1252, 1252, 485: 1252, 487: 1252, 1252, 1252, 1252, 1252, 494: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 3576, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 530: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 3577, 3578, 1252, 3581, 1252, 3579, 1252, 1252, 1252, 1252}, - {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 3149, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3593}, + {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 1189, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3593}, {105: 3352, 3348, 108: 3345, 3360, 111: 3347, 3344, 3346, 3350, 3351, 3356, 3355, 3354, 3358, 3359, 3353, 3357, 3349, 491: 3236, 494: 3234, 3235, 3233, 3231, 517: 3342, 3339, 3341, 3340, 3336, 3338, 3337, 3334, 3335, 3333, 3343, 720: 3232, 3230, 793: 3332, 816: 3594}, {1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 460: 1250, 1250, 1250, 1250, 465: 1250, 1250, 468: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 476: 1250, 1250, 479: 1250, 1250, 1250, 1250, 1250, 485: 1250, 487: 1250, 1250, 1250, 1250, 1250, 494: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 530: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250}, // 1145 {1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 460: 1253, 1253, 1253, 1253, 465: 1253, 1253, 468: 1253, 1253, 1253, 1253, 1253, 3580, 1253, 476: 1253, 1253, 479: 1253, 1253, 1253, 1253, 1253, 485: 1253, 487: 1253, 1253, 1253, 1253, 1253, 494: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 3576, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 530: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 3577, 3578, 1253, 3581, 1253, 3579, 1253, 1253, 1253, 1253}, - {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 3149, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3597}, + {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 1189, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3597}, {105: 3352, 3348, 108: 3345, 3360, 111: 3347, 3344, 3346, 3350, 3351, 3356, 3355, 3354, 3358, 3359, 3353, 3357, 3349, 491: 3236, 494: 3234, 3235, 3233, 3231, 517: 3342, 3339, 3341, 3340, 3336, 3338, 3337, 3334, 3335, 3333, 3343, 720: 3232, 3230, 793: 3332, 816: 3598}, {1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 460: 1251, 1251, 1251, 1251, 465: 1251, 1251, 468: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 476: 1251, 1251, 479: 1251, 1251, 1251, 1251, 1251, 485: 1251, 487: 1251, 1251, 1251, 1251, 1251, 494: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 530: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251}, {1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 460: 1254, 1254, 1254, 1254, 465: 1254, 1254, 468: 1254, 1254, 1254, 3575, 3574, 3580, 1254, 476: 1254, 1254, 479: 1254, 1254, 1254, 1254, 1254, 485: 1254, 487: 1254, 1254, 1254, 1254, 1254, 494: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 3576, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 530: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 3577, 3578, 1254, 3581, 1254, 3579, 1254, 1254, 1254, 1254}, @@ -7407,7 +7404,7 @@ var ( {7: 3617, 491: 3236, 494: 3234, 3235, 3233, 3231, 720: 3232, 3230}, {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 3149, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3619, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3618}, {57: 3623, 491: 3236, 494: 3234, 3235, 3233, 3231, 720: 3232, 3230}, - {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 3149, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3620}, + {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 1189, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3620}, // 1170 {105: 3352, 3348, 108: 3345, 3360, 111: 3347, 3344, 3346, 3350, 3351, 3356, 3355, 3354, 3358, 3359, 3353, 3357, 3349, 491: 3236, 494: 3234, 3235, 3233, 3231, 517: 3342, 3339, 3341, 3340, 3336, 3338, 3337, 3334, 3335, 3333, 3343, 720: 3232, 3230, 793: 3332, 816: 3621}, {57: 3622}, diff --git a/parser/parser.y b/parser/parser.y index 8c83474cb584f..56798d26b78b1 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -1399,8 +1399,6 @@ import ( %precedence sqlBigResult %precedence sqlSmallResult %precedence sqlCache sqlNoCache -%precedence lowerThanIntervalKeyword -%precedence interval %precedence next %precedence lowerThanValueKeyword %precedence value @@ -1453,6 +1451,7 @@ import ( %precedence lowerThanNot %right not not2 %right collate +%left interval %right encryption %left labels %precedence quick @@ -6920,7 +6919,7 @@ FunctionNameConflict: | "DAY" | "HOUR" | "IF" -| "INTERVAL" %prec lowerThanIntervalKeyword +| "INTERVAL" | "FORMAT" | "LEFT" | "MICROSECOND" diff --git a/parser/parser_test.go b/parser/parser_test.go index 134a9c79f5a5b..68e185775e18d 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -1494,6 +1494,8 @@ func TestBuiltin(t *testing.T) { {"SELECT LEAST(), LEAST(1, 2, 3);", true, "SELECT LEAST(),LEAST(1, 2, 3)"}, {"SELECT INTERVAL(1, 0, 1, 2)", true, "SELECT INTERVAL(1, 0, 1, 2)"}, + {"SELECT (INTERVAL(1, 0, 1, 2)+5)*7+INTERVAL(1, 0, 1, 2)/2", true, "SELECT (INTERVAL(1, 0, 1, 2)+5)*7+INTERVAL(1, 0, 1, 2)/2"}, + {"SELECT INTERVAL(0, (1*5)/2)+INTERVAL(5, 4, 3)", true, "SELECT INTERVAL(0, (1*5)/2)+INTERVAL(5, 4, 3)"}, {"SELECT DATE_ADD('2008-01-02', INTERVAL INTERVAL(1, 0, 1) DAY);", true, "SELECT DATE_ADD(_UTF8MB4'2008-01-02', INTERVAL INTERVAL(1, 0, 1) DAY)"}, // information functions diff --git a/planner/core/explain.go b/planner/core/explain.go index eda39f413a10e..26bfa775fc417 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -332,8 +332,7 @@ func (p *PhysicalTableReader) ExplainNormalizedInfo() string { return "" } -func (p *PhysicalTableReader) accessObject(sctx sessionctx.Context) string { - ts := p.TablePlans[0].(*PhysicalTableScan) +func getAccessObjectForTableScan(sctx sessionctx.Context, ts *PhysicalTableScan, partitionInfo PartitionInfo) string { pi := ts.Table.GetPartitionInfo() if pi == nil || !sctx.GetSessionVars().UseDynamicPartitionPrune() { return "" @@ -346,7 +345,51 @@ func (p *PhysicalTableReader) accessObject(sctx sessionctx.Context) string { } tbl := tmp.(table.PartitionedTable) - return partitionAccessObject(sctx, tbl, pi, &p.PartitionInfo) + return partitionAccessObject(sctx, tbl, pi, &partitionInfo) +} + +func (p *PhysicalTableReader) accessObject(sctx sessionctx.Context) string { + if !sctx.GetSessionVars().UseDynamicPartitionPrune() { + return "" + } + if len(p.PartitionInfos) == 0 { + ts := p.TablePlans[0].(*PhysicalTableScan) + return getAccessObjectForTableScan(sctx, ts, p.PartitionInfo) + } + if len(p.PartitionInfos) == 1 { + return getAccessObjectForTableScan(sctx, p.PartitionInfos[0].tableScan, p.PartitionInfos[0].partitionInfo) + } + containsPartitionTable := false + for _, info := range p.PartitionInfos { + if info.tableScan.Table.GetPartitionInfo() != nil { + containsPartitionTable = true + break + } + } + if !containsPartitionTable { + return "" + } + var buffer bytes.Buffer + for index, info := range p.PartitionInfos { + if index > 0 { + buffer.WriteString(", ") + } + + tblName := info.tableScan.Table.Name.O + if info.tableScan.TableAsName != nil && info.tableScan.TableAsName.O != "" { + tblName = info.tableScan.TableAsName.O + } + + if info.tableScan.Table.GetPartitionInfo() == nil { + buffer.WriteString("table of ") + buffer.WriteString(tblName) + continue + } + buffer.WriteString(getAccessObjectForTableScan(sctx, info.tableScan, info.partitionInfo)) + buffer.WriteString(" of ") + buffer.WriteString(tblName) + } + return buffer.String() } func partitionAccessObject(sctx sessionctx.Context, tbl table.PartitionedTable, pi *model.PartitionInfo, partTable *PartitionInfo) string { diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index ab57b29c2096d..a370fdaa36697 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1281,11 +1281,11 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { } if v.ExplicitScope && !sysVar.HasNoneScope() { if v.IsGlobal && !sysVar.HasGlobalScope() { - er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "GLOBAL") + er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "SESSION") return } if !v.IsGlobal && !sysVar.HasSessionScope() { - er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "SESSION") + er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "GLOBAL") return } } diff --git a/planner/core/handle_cols.go b/planner/core/handle_cols.go index c1bca6eec7ddf..48d6ab2444edd 100644 --- a/planner/core/handle_cols.go +++ b/planner/core/handle_cols.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" ) // HandleCols is the interface that holds handle columns. @@ -48,7 +49,7 @@ type HandleCols interface { // NumCols returns the number of columns. NumCols() int // Compare compares two datum rows by handle order. - Compare(a, b []types.Datum) (int, error) + Compare(a, b []types.Datum, ctors []collate.Collator) (int, error) // GetFieldTypes return field types of columns. GetFieldsTypes() []*types.FieldType } @@ -145,11 +146,11 @@ func (cb *CommonHandleCols) String() string { } // Compare implements the kv.HandleCols interface. -func (cb *CommonHandleCols) Compare(a, b []types.Datum) (int, error) { - for _, col := range cb.columns { +func (cb *CommonHandleCols) Compare(a, b []types.Datum, ctors []collate.Collator) (int, error) { + for i, col := range cb.columns { aDatum := &a[col.Index] bDatum := &b[col.Index] - cmp, err := aDatum.CompareDatum(cb.sc, bDatum) + cmp, err := aDatum.Compare(cb.sc, bDatum, ctors[i]) if err != nil { return 0, err } @@ -237,7 +238,7 @@ func (ib *IntHandleCols) NumCols() int { } // Compare implements the kv.HandleCols interface. -func (ib *IntHandleCols) Compare(a, b []types.Datum) (int, error) { +func (ib *IntHandleCols) Compare(a, b []types.Datum, ctors []collate.Collator) (int, error) { aInt := a[ib.col.Index].GetInt64() bInt := b[ib.col.Index].GetInt64() if aInt == bInt { diff --git a/planner/core/integration_partition_test.go b/planner/core/integration_partition_test.go index 33e4fb214ba81..b4faf61f64bcb 100644 --- a/planner/core/integration_partition_test.go +++ b/planner/core/integration_partition_test.go @@ -1160,3 +1160,22 @@ func genListPartition(begin, end int) string { buf.WriteString(fmt.Sprintf("%v)", end-1)) return buf.String() } + +func TestIssue27532(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database issue_27532") + defer tk.MustExec(`drop database issue_27532`) + tk.MustExec("use issue_27532") + tk.MustExec(`set tidb_enable_list_partition = 1`) + tk.MustExec(`create table t2 (c1 int primary key, c2 int, c3 int, c4 int, key k2 (c2), key k3 (c3)) partition by hash(c1) partitions 10`) + tk.MustExec(`insert into t2 values (1,1,1,1),(2,2,2,2),(3,3,3,3),(4,4,4,4)`) + tk.MustExec(`set @@tidb_partition_prune_mode="dynamic"`) + tk.MustExec(`set autocommit = 0`) + tk.MustQuery(`select * from t2`).Sort().Check(testkit.Rows("1 1 1 1", "2 2 2 2", "3 3 3 3", "4 4 4 4")) + tk.MustQuery(`select * from t2`).Sort().Check(testkit.Rows("1 1 1 1", "2 2 2 2", "3 3 3 3", "4 4 4 4")) + tk.MustExec(`drop table t2`) +} diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index a4d8945121519..17e84b6efcf47 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -68,6 +68,11 @@ var ( _ PhysicalPlan = &PhysicalTableSample{} ) +type tableScanAndPartitionInfo struct { + tableScan *PhysicalTableScan + partitionInfo PartitionInfo +} + // PhysicalTableReader is the table reader in tidb. type PhysicalTableReader struct { physicalSchemaProducer @@ -86,6 +91,8 @@ type PhysicalTableReader struct { // Used by partition table. PartitionInfo PartitionInfo + // Used by MPP, because MPP plan may contain join/union/union all, it is possible that a physical table reader contains more than 1 table scan + PartitionInfos []tableScanAndPartitionInfo } // PartitionInfo indicates partition helper info in physical plan. diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index fbf6ff47200a7..05ecb5c18eb41 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" tidbutil "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/math" "github.com/pingcap/tidb/util/plancodec" @@ -1213,8 +1214,7 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, } } // The converted result must be same as original datum. - // Compare them based on the dVal's type. - cmp, err := dVal.CompareDatum(stmtCtx, &d) + cmp, err := dVal.Compare(stmtCtx, &d, collate.GetCollator(col.Collate)) if err != nil { return nil, false } else if cmp != 0 { @@ -1235,8 +1235,7 @@ func getPointGetValue(stmtCtx *stmtctx.StatementContext, col *model.ColumnInfo, return nil } // The converted result must be same as original datum. - // Compare them based on the dVal's type. - cmp, err := dVal.CompareDatum(stmtCtx, d) + cmp, err := dVal.Compare(stmtCtx, d, collate.GetCollator(col.Collate)) if err != nil || cmp != 0 { return nil } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index bb57b0fac33da..7c3bbb565c69d 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -140,7 +140,7 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl ranges := detachedResult.Ranges used := make([]int, 0, len(ranges)) for _, r := range ranges { - if r.IsPointNullable(ctx) { + if r.IsPointNullable(ctx.GetSessionVars().StmtCtx) { if !r.HighVal[0].IsNull() { if len(r.HighVal) != len(partIdx) { used = []int{-1} @@ -473,7 +473,7 @@ func (l *listPartitionPruner) locateColumnPartitionsByCondition(cond expression. return nil, true, nil } var locations []tables.ListPartitionLocation - if r.IsPointNullable(l.ctx) { + if r.IsPointNullable(l.ctx.GetSessionVars().StmtCtx) { location, err := colPrune.LocatePartition(sc, r.HighVal[0]) if types.ErrOverflow.Equal(err) { return nil, true, nil // return full-scan if over-flow @@ -555,7 +555,7 @@ func (l *listPartitionPruner) findUsedListPartitions(conds []expression.Expressi } used := make(map[int]struct{}, len(ranges)) for _, r := range ranges { - if r.IsPointNullable(l.ctx) { + if r.IsPointNullable(l.ctx.GetSessionVars().StmtCtx) { if len(r.HighVal) != len(exprCols) { return l.fullRange, nil } diff --git a/planner/core/stats.go b/planner/core/stats.go index 8dabb4a648621..14a6a11a2c2d4 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -408,15 +408,6 @@ func (ds *DataSource) DeriveStats(childStats []*property.StatsInfo, selfSchema * return nil, err } - // TODO: implement UnionScan + IndexMerge - isReadOnlyTxn := true - txn, err := ds.ctx.Txn(false) - if err != nil { - return nil, err - } - if txn.Valid() && !txn.IsReadOnly() { - isReadOnlyTxn = false - } // Consider the IndexMergePath. Now, we just generate `IndexMergePath` in DNF case. isPossibleIdxMerge := len(ds.pushedDownConds) > 0 && len(ds.possibleAccessPaths) > 1 sessionAndStmtPermission := (ds.ctx.GetSessionVars().GetEnableIndexMerge() || len(ds.indexMergeHints) > 0) && !ds.ctx.GetSessionVars().StmtCtx.NoIndexMergeHint @@ -430,8 +421,9 @@ func (ds *DataSource) DeriveStats(childStats []*property.StatsInfo, selfSchema * } } } + readFromTableCache := ds.ctx.GetSessionVars().StmtCtx.ReadFromTableCache - if isPossibleIdxMerge && sessionAndStmtPermission && needConsiderIndexMerge && isReadOnlyTxn && ds.tableInfo.TempTableType != model.TempTableLocal && !readFromTableCache { + if isPossibleIdxMerge && sessionAndStmtPermission && needConsiderIndexMerge && ds.tableInfo.TempTableType != model.TempTableLocal && !readFromTableCache { err := ds.generateAndPruneIndexMergePath(ds.indexMergeHints != nil) if err != nil { return nil, err diff --git a/planner/core/task.go b/planner/core/task.go index 3133d9bcb9ac0..187140c613aa5 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -2254,6 +2254,17 @@ func (t *mppTask) convertToRootTask(ctx sessionctx.Context) *rootTask { return t.copy().(*mppTask).convertToRootTaskImpl(ctx) } +func collectPartitionInfosFromMPPPlan(p *PhysicalTableReader, mppPlan PhysicalPlan) { + switch x := mppPlan.(type) { + case *PhysicalTableScan: + p.PartitionInfos = append(p.PartitionInfos, tableScanAndPartitionInfo{x, x.PartitionInfo}) + default: + for _, ch := range mppPlan.Children() { + collectPartitionInfosFromMPPPlan(p, ch) + } + } +} + func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { sender := PhysicalExchangeSender{ ExchangeType: tipb.ExchangeType_PassThrough, @@ -2266,6 +2277,7 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { StoreType: kv.TiFlash, }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() + collectPartitionInfosFromMPPPlan(p, t.p) cst := t.cst + t.count()*ctx.GetSessionVars().GetNetworkFactor(nil) p.cost = cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 4ad6be46c339f..c4cee0cbf1ce2 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -372,6 +372,35 @@ func TestShowGrants(t *testing.T) { require.Len(t, gs, 3) } +// TestErrorMessage checks that the identity in error messages matches the mysql.user table one. +// MySQL is inconsistent in its error messages, as some match the loginHost and others the +// identity from mysql.user. In TiDB we now use the identity from mysql.user in error messages +// for consistency. +func TestErrorMessage(t *testing.T) { + t.Parallel() + store, clean := newStore(t) + defer clean() + + rootSe := newSession(t, store, dbName) + mustExec(t, rootSe, `CREATE USER wildcard`) + mustExec(t, rootSe, `CREATE USER specifichost@192.168.1.1`) + mustExec(t, rootSe, `GRANT SELECT on test.* TO wildcard`) + mustExec(t, rootSe, `GRANT SELECT on test.* TO specifichost@192.168.1.1`) + + wildSe := newSession(t, store, dbName) + + // The session.Auth() func will populate the AuthUsername and AuthHostname fields. + // We don't have to explicitly specify them. + require.True(t, wildSe.Auth(&auth.UserIdentity{Username: "wildcard", Hostname: "192.168.1.1"}, nil, nil)) + _, err := wildSe.ExecuteInternal(context.Background(), "use mysql;") + require.Equal(t, "[executor:1044]Access denied for user 'wildcard'@'%' to database 'mysql'", err.Error()) + + specificSe := newSession(t, store, dbName) + require.True(t, specificSe.Auth(&auth.UserIdentity{Username: "specifichost", Hostname: "192.168.1.1"}, nil, nil)) + _, err = specificSe.ExecuteInternal(context.Background(), "use mysql;") + require.Equal(t, "[executor:1044]Access denied for user 'specifichost'@'192.168.1.1' to database 'mysql'", err.Error()) +} + func TestShowColumnGrants(t *testing.T) { t.Parallel() store, clean := newStore(t) diff --git a/server/conn.go b/server/conn.go index 113efd1d0a8e1..0209ab77171a5 100644 --- a/server/conn.go +++ b/server/conn.go @@ -2190,10 +2190,15 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool // fetchSize, the desired number of rows to be fetched each time when client uses cursor. func (cc *clientConn) writeChunksWithFetchSize(ctx context.Context, rs ResultSet, serverStatus uint16, fetchSize int) error { fetchedRows := rs.GetFetchedRows() + // if fetchedRows is not enough, getting data from recordSet. + req := rs.NewChunk(nil) for len(fetchedRows) < fetchSize { - // if fetchedRows is not enough, getting data from recordSet. - req := rs.NewChunk(cc.chunkAlloc) + // NOTE: chunk should not be allocated from the allocator + // the allocator will reset every statement + // but it maybe stored in the result set among statements + // ref https://github.com/pingcap/tidb/blob/7fc6ebbda4ddf84c0ba801ca7ebb636b934168cf/server/conn_stmt.go#L233-L239 // Here server.tidbResultSet implements Next method. + req.Reset() if err := rs.Next(ctx, req); err != nil { return err } @@ -2205,7 +2210,6 @@ func (cc *clientConn) writeChunksWithFetchSize(ctx context.Context, rs ResultSet for i := 0; i < rowCount; i++ { fetchedRows = append(fetchedRows, req.GetRow(i)) } - req = chunk.Renew(req, cc.ctx.GetSessionVars().MaxChunkSize) } // tell the client COM_STMT_FETCH has finished by setting proper serverStatus, diff --git a/server/conn_test.go b/server/conn_test.go index 9e2f40d366151..cd35257d72454 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -180,7 +180,6 @@ func TestInitialHandshake(t *testing.T) { var outBuffer bytes.Buffer cfg := newTestConfig() - cfg.Socket = "" cfg.Port = 0 cfg.Status.StatusPort = 0 drv := NewTiDBDriver(store) @@ -489,7 +488,6 @@ func testDispatch(t *testing.T, inputs []dispatchInput, capability uint32) { var outBuffer bytes.Buffer tidbdrv := NewTiDBDriver(store) cfg := newTestConfig() - cfg.Socket = "" cfg.Port, cfg.Status.StatusPort = 0, 0 cfg.Status.ReportStatus = false server, err := NewServer(cfg, tidbdrv) @@ -911,8 +909,6 @@ func TestShowErrors(t *testing.T) { } func TestHandleAuthPlugin(t *testing.T) { - t.Parallel() - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1132,14 +1128,10 @@ func TestHandleAuthPlugin(t *testing.T) { } func TestAuthPlugin2(t *testing.T) { - - t.Parallel() - store, clean := testkit.CreateMockStore(t) defer clean() cfg := newTestConfig() - cfg.Socket = "" cfg.Port = 0 cfg.Status.StatusPort = 0 diff --git a/server/server_test.go b/server/server_test.go index e623af576b1de..034587d7be1f7 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -106,7 +106,7 @@ func (cli *testServerClient) getDSN(overriders ...configOverrider) string { // runTests runs tests using the default database `test`. func (cli *testServerClient) runTests(t *testing.T, overrider configOverrider, tests ...func(dbt *testkit.DBTestKit)) { db, err := sql.Open("mysql", cli.getDSN(overrider)) - require.NoErrorf(t, err, "Error connecting") + require.NoError(t, err) defer func() { err := db.Close() require.NoError(t, err) @@ -124,12 +124,11 @@ func (cli *testServerClient) runTestsOnNewDB(t *testing.T, overrider configOverr config.DBName = "" }) db, err := sql.Open("mysql", dsn) - require.NoErrorf(t, err, "Error connecting") + require.NoError(t, err) defer func() { err := db.Close() require.NoError(t, err) }() - _, err = db.Exec(fmt.Sprintf("DROP DATABASE IF EXISTS `%s`;", dbName)) if err != nil { fmt.Println(err) @@ -167,7 +166,7 @@ func (cli *testServerClient) runTestRegression(t *testing.T, overrider configOve var out bool rows := dbt.MustQuery("SELECT * FROM test") require.Falsef(t, rows.Next(), "unexpected data in empty table") - + require.NoError(t, rows.Close()) // Create Data res := dbt.MustExec("INSERT INTO test VALUES (1)") // res := dbt.mustExec("INSERT INTO test VALUES (?)", 1) @@ -188,7 +187,7 @@ func (cli *testServerClient) runTestRegression(t *testing.T, overrider configOve } else { require.Fail(t, "no data") } - rows.Close() + require.NoError(t, rows.Close()) // Update res = dbt.MustExec("UPDATE test SET val = 0 WHERE val = ?", 1) @@ -206,7 +205,7 @@ func (cli *testServerClient) runTestRegression(t *testing.T, overrider configOve } else { require.Fail(t, "no data") } - rows.Close() + require.NoError(t, rows.Close()) // Delete res = dbt.MustExec("DELETE FROM test WHERE val = 0") @@ -240,7 +239,10 @@ func (cli *testServerClient) runTestPrepareResultFieldType(t *testing.T) { if err != nil { t.Fatal(err) } - defer stmt.Close() + defer func() { + err = stmt.Close() + require.NoError(t, err) + }() row := stmt.QueryRow(param) var result int64 err = row.Scan(&result) @@ -268,6 +270,7 @@ func (cli *testServerClient) runTestSpecialType(t *testing.T) { require.Equal(t, "2012-12-21 12:12:12", outB) require.Equal(t, "04:23:34", outC) require.Equal(t, []byte{8}, outD) + require.NoError(t, rows.Close()) }) } @@ -283,20 +286,21 @@ func (cli *testServerClient) runTestClientWithCollation(t *testing.T) { err := rows.Scan(&name, &collation) require.NoError(t, err) require.Equal(t, "utf8mb4_general_ci", collation) - + require.NoError(t, rows.Close()) // check session variable character_set_client rows = dbt.MustQuery("show variables like 'character_set_client'") require.True(t, rows.Next()) err = rows.Scan(&name, &charset) require.NoError(t, err) require.Equal(t, "utf8mb4", charset) - + require.NoError(t, rows.Close()) // check session variable character_set_results rows = dbt.MustQuery("show variables like 'character_set_results'") require.True(t, rows.Next()) err = rows.Scan(&name, &charset) require.NoError(t, err) require.Equal(t, "utf8mb4", charset) + require.NoError(t, rows.Close()) // check session variable character_set_connection rows = dbt.MustQuery("show variables like 'character_set_connection'") @@ -304,6 +308,7 @@ func (cli *testServerClient) runTestClientWithCollation(t *testing.T) { err = rows.Scan(&name, &charset) require.NoError(t, err) require.Equal(t, "utf8mb4", charset) + require.NoError(t, rows.Close()) }) } @@ -318,6 +323,7 @@ func (cli *testServerClient) runTestPreparedString(t *testing.T) { require.NoError(t, err) require.Equal(t, "abcdeabcde", outA) require.Equal(t, "abcde", outB) + require.NoError(t, rows.Close()) }) } @@ -329,12 +335,11 @@ func (cli *testServerClient) runTestPreparedTimestamp(t *testing.T) { dbt.MustExec("create table test (a timestamp, b time)") dbt.MustExec("set time_zone='+00:00'") insertStmt := dbt.MustPrepare("insert test values (?, ?)") - defer insertStmt.Close() vts := time.Unix(1, 1) vt := time.Unix(-1, 1) dbt.MustExecPrepared(insertStmt, vts, vt) + require.NoError(t, insertStmt.Close()) selectStmt := dbt.MustPrepare("select * from test where a = ? and b = ?") - defer selectStmt.Close() rows := dbt.MustQueryPrepared(selectStmt, vts, vt) require.True(t, rows.Next()) var outA, outB string @@ -342,6 +347,8 @@ func (cli *testServerClient) runTestPreparedTimestamp(t *testing.T) { require.NoError(t, err) require.Equal(t, "1970-01-01 00:00:01", outA) require.Equal(t, "23:59:59", outB) + require.NoError(t, rows.Close()) + require.NoError(t, selectStmt.Close()) }) } @@ -440,13 +447,16 @@ func (cli *testServerClient) runTestLoadDataForSlowLog(t *testing.T, server *Ser rows := dbt.MustQuery("select plan from information_schema.slow_query where query like 'load data local infile % into table t_slow;' order by time desc limit 1") expectedPlan := ".*LoadData.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.* commit_txn.*" checkPlan(rows, expectedPlan) + require.NoError(t, rows.Close()) // Test for record statements_summary for load data statement. rows = dbt.MustQuery("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %' limit 1") checkPlan(rows, expectedPlan) + require.NoError(t, rows.Close()) // Test log normal statement after executing load date. rows = dbt.MustQuery("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1") expectedPlan = ".*Insert.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.*" checkPlan(rows, expectedPlan) + require.NoError(t, rows.Close()) }) } @@ -513,11 +523,13 @@ func (cli *testServerClient) runTestLoadDataAutoRandom(t *testing.T) { dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (c2, c3)", path)) rows := dbt.MustQuery("select count(*) from t") cli.checkRows(t, rows, "50000") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select bit_xor(c2), bit_xor(c3) from t") res := strconv.Itoa(cksum1) res = res + " " res = res + strconv.Itoa(cksum2) cli.checkRows(t, rows, res) + require.NoError(t, rows.Close()) }) } @@ -605,8 +617,10 @@ func (cli *testServerClient) runTestLoadDataForListPartition(t *testing.T) { dbt.MustExec("delete from t") cli.prepareLoadDataFile(t, path, "1 a", "3 c", "4 e") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "3 c", "4 e") + require.NoError(t, rows.Close()) // Test load data meet duplicate error. cli.prepareLoadDataFile(t, path, "1 x", "2 b", "2 x", "7 a") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) @@ -614,6 +628,7 @@ func (cli *testServerClient) runTestLoadDataForListPartition(t *testing.T) { cli.checkRows(t, rows, "Warning 1062 Duplicate entry '1' for key 'idx'", "Warning 1062 Duplicate entry '2' for key 'idx'") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") // Test load data meet no partition warning. @@ -622,8 +637,10 @@ func (cli *testServerClient) runTestLoadDataForListPartition(t *testing.T) { require.NoError(t, err) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, "Warning 1526 Table has no partition for value 100") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "5 a", "7 a") + require.NoError(t, rows.Close()) }) } @@ -654,25 +671,31 @@ func (cli *testServerClient) runTestLoadDataForListPartition2(t *testing.T) { dbt.MustExec("delete from t") cli.prepareLoadDataFile(t, path, "1 a", "3 c", "4 e") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select id,name from t order by id") cli.checkRows(t, rows, "1 a", "3 c", "4 e") // Test load data meet duplicate error. cli.prepareLoadDataFile(t, path, "1 x", "2 b", "2 x", "7 a") + require.NoError(t, rows.Close()) dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, "Warning 1062 Duplicate entry '1-2' for key 'idx'", "Warning 1062 Duplicate entry '2-2' for key 'idx'") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select id,name from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") + require.NoError(t, rows.Close()) // Test load data meet no partition warning. cli.prepareLoadDataFile(t, path, "5 a", "100 x") _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) require.NoError(t, err) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, "Warning 1526 Table has no partition for value 100") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select id,name from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "5 a", "7 a") + require.NoError(t, rows.Close()) }) } @@ -703,8 +726,10 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition(t *testing.T) dbt.MustExec("delete from t") cli.prepareLoadDataFile(t, path, "1 a", "3 c", "4 e") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "3 c", "4 e") + require.NoError(t, rows.Close()) // Test load data meet duplicate error. cli.prepareLoadDataFile(t, path, "1 x", "2 b", "2 x", "7 a") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) @@ -712,16 +737,20 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition(t *testing.T) cli.checkRows(t, rows, "Warning 1062 Duplicate entry '1' for key 'idx'", "Warning 1062 Duplicate entry '2' for key 'idx'") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") // Test load data meet no partition warning. cli.prepareLoadDataFile(t, path, "5 a", "100 x") _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, err) + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, "Warning 1526 Table has no partition for value from column_list") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select id,name from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "5 a", "7 a") + require.NoError(t, rows.Close()) }) } @@ -751,31 +780,38 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition2(t *testing.T dbt.MustExec("delete from t") cli.prepareLoadDataFile(t, path, "w 1 1", "e 5 5", "n 9 9") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "w 1 1", "e 5 5", "n 9 9") // Test load data meet duplicate error. cli.prepareLoadDataFile(t, path, "w 1 2", "w 2 2") _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, err) + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, "Warning 1062 Duplicate entry 'w-1' for key 'idx'") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "w 1 1", "w 2 2", "e 5 5", "n 9 9") // Test load data meet no partition warning. cli.prepareLoadDataFile(t, path, "w 3 3", "w 5 5", "e 8 8") _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, err) + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, "Warning 1526 Table has no partition for value from column_list") cli.prepareLoadDataFile(t, path, "x 1 1", "w 1 1") _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, err) + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, "Warning 1526 Table has no partition for value from column_list", "Warning 1062 Duplicate entry 'w-1' for key 'idx'") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "w 1 1", "w 2 2", "w 3 3", "e 5 5", "e 8 8", "n 9 9") + require.NoError(t, rows.Close()) }) } @@ -903,7 +939,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, "- ", b) require.Equal(t, 5, cc) require.Falsef(t, rows.Next(), "unexpected data") - rows.Close() + require.NoError(t, rows.Close()) // specify faileds and lines dbt.MustExec("delete from test") @@ -942,7 +978,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, "\trow5_col3", b) require.Equal(t, 9, cc) require.Falsef(t, rows.Next(), "unexpected data") - + require.NoError(t, rows.Close()) // infile size more than a packet size(16K) dbt.MustExec("delete from test") _, err = fp.WriteString("\n") @@ -962,7 +998,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, int64(799), affectedRows) rows = dbt.MustQuery("select * from test") require.Truef(t, rows.Next(), "unexpected data") - + require.NoError(t, rows.Close()) // don't support lines terminated is "" dbt.MustExec("set @@tidb_dml_batch_size = 3") _, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test lines terminated by ''") @@ -1020,6 +1056,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, 789, id) require.Falsef(t, rows.Next(), "unexpected data") dbt.MustExec("delete from test") + require.NoError(t, rows.Close()) }) err = fp.Close() @@ -1076,6 +1113,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, "", d.String) require.Falsef(t, rows.Next(), "unexpected data") dbt.MustExec("delete from test") + require.NoError(t, rows.Close()) }) err = fp.Close() @@ -1124,6 +1162,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, `c"d"e`, b.String) require.Falsef(t, rows.Next(), "unexpected data") dbt.MustExec("delete from test") + require.NoError(t, rows.Close()) }) err = fp.Close() @@ -1162,6 +1201,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, "3", c.String) require.Falsef(t, rows.Next(), "unexpected data") dbt.MustExec("delete from test") + require.NoError(t, rows.Close()) }) // unsupport ClientLocalFiles capability @@ -1217,7 +1257,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, 3, a) require.Equal(t, 4, b) require.Falsef(t, rows.Next(), "unexpected data") - + require.NoError(t, rows.Close()) // fail error processing test require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/commitOneTaskErr", "return")) _, err1 = dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ','`) @@ -1269,7 +1309,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, 3, a) require.Equal(t, 4, b) require.Falsef(t, rows.Next(), "unexpected data") - + require.NoError(t, rows.Close()) dbt.MustExec("drop table if exists pn") }) @@ -1316,7 +1356,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Empty(t, b.String) require.Empty(t, c.String) require.Falsef(t, rows.Next(), "unexpected data") - + require.NoError(t, rows.Close()) dbt.MustExec("drop table if exists pn") }) @@ -1363,7 +1403,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.Equal(t, 5, b) require.Equal(t, 600, c) require.Falsef(t, rows.Next(), "unexpected data") - + require.NoError(t, rows.Close()) dbt.MustExec("drop table if exists pn") }) } @@ -1515,10 +1555,9 @@ func (cli *testServerClient) runTestAuth(t *testing.T) { config.Passwd = "456" })) require.NoError(t, err) - _, err = db.Query("USE information_schema;") + _, err = db.Exec("USE information_schema;") require.NotNilf(t, err, "Wrong password should be failed") - err = db.Close() - require.NoError(t, err) + require.NoError(t, db.Close()) // Test for loading active roles. db, err = sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { @@ -1532,6 +1571,7 @@ func (cli *testServerClient) runTestAuth(t *testing.T) { var outA string err = rows.Scan(&outA) require.NoError(t, err) + require.NoError(t, rows.Close()) require.Equal(t, "`authtest_r1`@`%`", outA) err = db.Close() require.NoError(t, err) @@ -1587,16 +1627,10 @@ func (cli *testServerClient) runTestIssue3680(t *testing.T) { func (cli *testServerClient) runTestIssue22646(t *testing.T) { cli.runTests(t, nil, func(dbt *testkit.DBTestKit) { - c1 := make(chan string, 1) - go func() { - dbt.MustExec(``) // empty query. - c1 <- "success" - }() - select { - case res := <-c1: - fmt.Println(res) - case <-time.After(30 * time.Second): - panic("read empty query statement timed out.") + now := time.Now() + dbt.MustExec(``) + if time.Since(now) > 30*time.Second { + t.Fatal("read empty query statement timed out.") } }) } @@ -1894,24 +1928,31 @@ func getStmtCnt(content string) (stmtCnt map[string]int) { const retryTime = 100 -func (cli *testServerClient) waitUntilServerCanConnect() { +func (cli *testServerClient) waitUntilCustomServerCanConnect(overriders ...configOverrider) { // connect server retry := 0 + dsn := cli.getDSN(overriders...) for ; retry < retryTime; retry++ { time.Sleep(time.Millisecond * 10) - db, err := sql.Open("mysql", cli.getDSN()) + db, err := sql.Open("mysql", dsn) if err == nil { - err = db.Close() - if err != nil { - panic(err) + succeed := db.Ping() == nil + if err = db.Close(); err != nil { + log.Error("fail to connect db", zap.String("err", err.Error()), zap.String("DSN", dsn)) + continue + } + if succeed { + break } - break } } if retry == retryTime { - log.Fatal("failed to connect DB in every 10 ms", zap.Int("retryTime", retryTime)) + log.Fatal("failed to connect DB in every 10 ms", zap.String("DSN", dsn), zap.Int("retryTime", retryTime)) } } +func (cli *testServerClient) waitUntilServerCanConnect() { + cli.waitUntilCustomServerCanConnect(nil) +} func (cli *testServerClient) waitUntilServerOnline() { // connect server @@ -1988,9 +2029,9 @@ func (cli *testServerClient) runTestInitConnect(t *testing.T) { db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "init_nonsuper" })) - require.NoErrorf(t, err, "Error connecting") // doesn't fail because of lazy loading - defer db.Close() // may already be closed - _, err = db.Exec("SELECT 1") // fails because of init sql + require.NoError(t, err) // doesn't fail because of lazy loading + defer db.Close() // may already be closed + _, err = db.Exec("SELECT 1") // fails because of init sql require.Error(t, err) } @@ -2033,7 +2074,7 @@ func (cli *testServerClient) runTestInfoschemaClientErrors(t *testing.T) { if rows.Next() { rows.Scan(&errors, &warnings) } - rows.Close() + require.NoError(t, rows.Close()) if test.incrementErrors { errors++ @@ -2049,14 +2090,15 @@ func (cli *testServerClient) runTestInfoschemaClientErrors(t *testing.T) { var fake string rows.Scan(&fake) } - rows.Close() + require.NoError(t, rows.Close()) } + var newErrors, newWarnings int rows = dbt.MustQuery("SELECT SUM(error_count), SUM(warning_count) FROM information_schema."+tbl+" WHERE error_number = ? GROUP BY error_number", test.errCode) if rows.Next() { rows.Scan(&newErrors, &newWarnings) } - rows.Close() + require.NoError(t, rows.Close()) require.Equal(t, errors, newErrors) require.Equalf(t, warnings, newWarnings, "source=information_schema.%s code=%d statement=%s", tbl, test.errCode, test.stmt) } diff --git a/server/statistics_handler_serial_test.go b/server/statistics_handler_serial_test.go index 4f64a9c9b345a..1f4324d3a54f5 100644 --- a/server/statistics_handler_serial_test.go +++ b/server/statistics_handler_serial_test.go @@ -229,4 +229,5 @@ func checkData(t *testing.T, path string, client *testServerClient) { require.Equal(t, "test", tableName) require.Equal(t, int64(3), modifyCount) require.Equal(t, int64(4), count) + require.NoError(t, rows.Close()) } diff --git a/server/tidb_serial_test.go b/server/tidb_serial_test.go index b5f2483584052..5bbf88ad0e392 100644 --- a/server/tidb_serial_test.go +++ b/server/tidb_serial_test.go @@ -106,7 +106,6 @@ func TestTLSAuto(t *testing.T) { } cli := newTestServerClient() cfg := newTestConfig() - cfg.Socket = "" cfg.Port = cli.port cfg.Status.ReportStatus = false cfg.Security.AutoTLS = true @@ -162,7 +161,6 @@ func TestTLSBasic(t *testing.T) { } cli := newTestServerClient() cfg := newTestConfig() - cfg.Socket = "" cfg.Port = cli.port cfg.Status.ReportStatus = false cfg.Security = config.Security{ @@ -234,7 +232,6 @@ func TestTLSVerify(t *testing.T) { // Start the server with TLS & CA, if the client presents its certificate, the certificate will be verified. cli := newTestServerClient() cfg := newTestConfig() - cfg.Socket = "" cfg.Port = cli.port cfg.Status.ReportStatus = false cfg.Security = config.Security{ @@ -302,7 +299,6 @@ func TestErrorNoRollback(t *testing.T) { cli := newTestServerClient() cfg := newTestConfig() - cfg.Socket = "" cfg.Port = cli.port cfg.Status.ReportStatus = false @@ -367,6 +363,7 @@ func TestPrepareCount(t *testing.T) { err = qctx.GetStatement(stmt.ID()).Close() require.NoError(t, err) require.Equal(t, prepareCnt, atomic.LoadInt64(&variable.PreparedStmtCount)) + require.NoError(t, qctx.Close()) } func TestDefaultCharacterAndCollation(t *testing.T) { @@ -422,7 +419,6 @@ func TestReloadTLS(t *testing.T) { // try old cert used in startup configuration. cli := newTestServerClient() cfg := newTestConfig() - cfg.Socket = "" cfg.Port = cli.port cfg.Status.ReportStatus = false cfg.Security = config.Security{ diff --git a/server/tidb_test.go b/server/tidb_test.go index c76ad1957a766..8ea521fda6275 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -76,7 +76,6 @@ func createTidbTestSuite(t *testing.T) (*tidbTestSuite, func()) { require.NoError(t, err) ts.tidbdrv = NewTiDBDriver(ts.store) cfg := newTestConfig() - cfg.Socket = "" cfg.Port = ts.port cfg.Status.ReportStatus = true cfg.Status.StatusPort = ts.statusPort @@ -121,7 +120,7 @@ func createTidbTestTopSQLSuite(t *testing.T) (*tidbTestTopSQLSuite, func()) { // Initialize global variable for top-sql test. db, err := sql.Open("mysql", ts.getDSN()) - require.NoErrorf(t, err, "Error connecting") + require.NoError(t, err) defer func() { err := db.Close() require.NoError(t, err) @@ -242,7 +241,6 @@ func TestStatusPort(t *testing.T) { defer cleanup() cfg := newTestConfig() - cfg.Socket = "" cfg.Port = 0 cfg.Status.ReportStatus = true cfg.Status.StatusPort = ts.statusPort @@ -273,7 +271,6 @@ func TestStatusAPIWithTLS(t *testing.T) { cli := newTestServerClient() cli.statusScheme = "https" cfg := newTestConfig() - cfg.Socket = "" cfg.Port = cli.port cfg.Status.StatusPort = cli.statusPort cfg.Security.ClusterSSLCA = "/tmp/ca-cert-2.pem" @@ -329,7 +326,6 @@ func TestStatusAPIWithTLSCNCheck(t *testing.T) { cli := newTestServerClient() cli.statusScheme = "https" cfg := newTestConfig() - cfg.Socket = "" cfg.Port = cli.port cfg.Status.StatusPort = cli.statusPort cfg.Security.ClusterSSLCA = caPath @@ -453,15 +449,17 @@ func TestSocket(t *testing.T) { time.Sleep(time.Millisecond * 100) defer server.Close() - // a fake server client, config is override, just used to run tests - cli := newTestServerClient() - cli.runTestRegression(t, func(config *mysql.Config) { + confFunc := func(config *mysql.Config) { config.User = "root" config.Net = "unix" config.Addr = socketFile config.DBName = "test" config.Params = map[string]string{"sql_mode": "STRICT_ALL_TABLES"} - }, "SocketRegression") + } + // a fake server client, config is override, just used to run tests + cli := newTestServerClient() + cli.waitUntilCustomServerCanConnect(confFunc) + cli.runTestRegression(t, confFunc, "SocketRegression") } func TestSocketAndIp(t *testing.T) { @@ -498,9 +496,9 @@ func TestSocketAndIp(t *testing.T) { config.User = "root" }, func(dbt *testkit.DBTestKit) { - dbt.MustQuery("DROP USER IF EXISTS 'user1'@'%'") - dbt.MustQuery("DROP USER IF EXISTS 'user1'@'localhost'") - dbt.MustQuery("DROP USER IF EXISTS 'user1'@'127.0.0.1'") + dbt.MustExec("DROP USER IF EXISTS 'user1'@'%'") + dbt.MustExec("DROP USER IF EXISTS 'user1'@'localhost'") + dbt.MustExec("DROP USER IF EXISTS 'user1'@'127.0.0.1'") }) }() cli.runTests(t, func(config *mysql.Config) { @@ -598,8 +596,8 @@ func TestSocketAndIp(t *testing.T) { cli.checkRows(t, rows, "root@localhost") rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.MustQuery("CREATE USER user1@localhost") - dbt.MustQuery("GRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO user1@localhost") + dbt.MustExec("CREATE USER user1@localhost") + dbt.MustExec("GRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO user1@localhost") }) // Test with Network interface connection with all hosts cli.runTests(t, func(config *mysql.Config) { @@ -610,8 +608,10 @@ func TestSocketAndIp(t *testing.T) { rows := dbt.MustQuery("select user()") // NOTICE: this is not compatible with MySQL! (MySQL would report user1@localhost also for 127.0.0.1) cli.checkRows(t, rows, "user1@127.0.0.1") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'127.0.0.1'\nGRANT SELECT,INSERT ON test.* TO 'user1'@'127.0.0.1'") + require.NoError(t, rows.Close()) }) // Test with unix domain socket file connection with all hosts cli.runTests(t, func(config *mysql.Config) { @@ -623,8 +623,10 @@ func TestSocketAndIp(t *testing.T) { func(dbt *testkit.DBTestKit) { rows := dbt.MustQuery("select user()") cli.checkRows(t, rows, "user1@localhost") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'localhost'\nGRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO 'user1'@'localhost'") + require.NoError(t, rows.Close()) }) } @@ -666,9 +668,9 @@ func TestOnlySocket(t *testing.T) { config.Addr = socketFile }, func(dbt *testkit.DBTestKit) { - dbt.MustQuery("DROP USER IF EXISTS 'user1'@'%'") - dbt.MustQuery("DROP USER IF EXISTS 'user1'@'localhost'") - dbt.MustQuery("DROP USER IF EXISTS 'user1'@'127.0.0.1'") + dbt.MustExec("DROP USER IF EXISTS 'user1'@'%'") + dbt.MustExec("DROP USER IF EXISTS 'user1'@'localhost'") + dbt.MustExec("DROP USER IF EXISTS 'user1'@'127.0.0.1'") }) }() cli.runTests(t, func(config *mysql.Config) { @@ -680,10 +682,12 @@ func TestOnlySocket(t *testing.T) { func(dbt *testkit.DBTestKit) { rows := dbt.MustQuery("select user()") cli.checkRows(t, rows, "root@localhost") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.MustQuery("CREATE USER user1@'%'") - dbt.MustQuery("GRANT SELECT ON test.* TO user1@'%'") + require.NoError(t, rows.Close()) + dbt.MustExec("CREATE USER user1@'%'") + dbt.MustExec("GRANT SELECT ON test.* TO user1@'%'") }) // Test with Network interface connection with all hosts, should fail since server not configured db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { @@ -712,8 +716,10 @@ func TestOnlySocket(t *testing.T) { func(dbt *testkit.DBTestKit) { rows := dbt.MustQuery("select user()") cli.checkRows(t, rows, "user1@localhost") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") + require.NoError(t, rows.Close()) }) // Setup user1@127.0.0.1 for loop back network interface access @@ -727,10 +733,12 @@ func TestOnlySocket(t *testing.T) { rows := dbt.MustQuery("select user()") // NOTICE: this is not compatible with MySQL! (MySQL would report user1@localhost also for 127.0.0.1) cli.checkRows(t, rows, "root@localhost") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.MustQuery("CREATE USER user1@127.0.0.1") - dbt.MustQuery("GRANT SELECT,INSERT ON test.* TO user1@'127.0.0.1'") + require.NoError(t, rows.Close()) + dbt.MustExec("CREATE USER user1@127.0.0.1") + dbt.MustExec("GRANT SELECT,INSERT ON test.* TO user1@'127.0.0.1'") }) // Test with unix domain socket file connection with all hosts cli.runTests(t, func(config *mysql.Config) { @@ -742,8 +750,10 @@ func TestOnlySocket(t *testing.T) { func(dbt *testkit.DBTestKit) { rows := dbt.MustQuery("select user()") cli.checkRows(t, rows, "user1@localhost") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") + require.NoError(t, rows.Close()) }) // Setup user1@localhost for socket (and if MySQL compatible; loop back network interface access) @@ -756,10 +766,12 @@ func TestOnlySocket(t *testing.T) { func(dbt *testkit.DBTestKit) { rows := dbt.MustQuery("select user()") cli.checkRows(t, rows, "root@localhost") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.MustQuery("CREATE USER user1@localhost") - dbt.MustQuery("GRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO user1@localhost") + require.NoError(t, rows.Close()) + dbt.MustExec("CREATE USER user1@localhost") + dbt.MustExec("GRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO user1@localhost") }) // Test with unix domain socket file connection with all hosts cli.runTests(t, func(config *mysql.Config) { @@ -771,8 +783,10 @@ func TestOnlySocket(t *testing.T) { func(dbt *testkit.DBTestKit) { rows := dbt.MustQuery("select user()") cli.checkRows(t, rows, "user1@localhost") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'localhost'\nGRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO 'user1'@'localhost'") + require.NoError(t, rows.Close()) }) } @@ -887,7 +901,6 @@ func TestSystemTimeZone(t *testing.T) { tk := testkit.NewTestKit(t, ts.store) cfg := newTestConfig() - cfg.Socket = "" cfg.Port, cfg.Status.StatusPort = 0, 0 cfg.Status.ReportStatus = false server, err := NewServer(cfg, ts.tidbdrv) @@ -1216,7 +1229,6 @@ func TestGracefulShutdown(t *testing.T) { cli := newTestServerClient() cfg := newTestConfig() - cfg.Socket = "" cfg.GracefulWaitBeforeShutdown = 2 // wait before shutdown cfg.Port = 0 cfg.Status.StatusPort = 0 @@ -1287,7 +1299,7 @@ func TestTopSQLCPUProfile(t *testing.T) { defer cleanup() db, err := sql.Open("mysql", ts.getDSN()) - require.NoErrorf(t, err, "Error connecting") + require.NoError(t, err) defer func() { err := db.Close() require.NoError(t, err) @@ -1345,8 +1357,7 @@ func TestTopSQLCPUProfile(t *testing.T) { dbt := testkit.NewDBTestKit(t, db) if strings.HasPrefix(sqlStr, "select") { rows := dbt.MustQuery(sqlStr) - for rows.Next() { - } + require.NoError(t, rows.Close()) } else { // Ignore error here since the error may be write conflict. db.Exec(sqlStr) @@ -1417,11 +1428,11 @@ func TestTopSQLCPUProfile(t *testing.T) { if strings.HasPrefix(prepare, "select") { rows, err := stmt.Query(args...) require.NoError(t, err) - for rows.Next() { - } + require.NoError(t, rows.Close()) } else { // Ignore error here since the error may be write conflict. - stmt.Exec(args...) + _, err = stmt.Exec(args...) + require.NoError(t, err) } }) } @@ -1478,11 +1489,11 @@ func TestTopSQLCPUProfile(t *testing.T) { if strings.HasPrefix(prepare, "select") { rows, err := db.Query(sqlBuf.String()) require.NoErrorf(t, err, "%v", sqlBuf.String()) - for rows.Next() { - } + require.NoError(t, rows.Close()) } else { // Ignore error here since the error may be write conflict. - db.Exec(sqlBuf.String()) + _, err = db.Exec(sqlBuf.String()) + require.NoError(t, err) } }) } @@ -1585,8 +1596,7 @@ func TestTopSQLAgent(t *testing.T) { go ts.loopExec(ctx, t, func(db *sql.DB) { dbt := testkit.NewDBTestKit(t, db) rows := dbt.MustQuery(query) - for rows.Next() { - } + require.NoError(t, rows.Close()) }) } return cancel @@ -1723,8 +1733,10 @@ func TestLocalhostClientMapping(t *testing.T) { dbt := testkit.NewDBTestKit(t, db) rows := dbt.MustQuery("select user()") cli.checkRows(t, rows, "root@localhost") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") + require.NoError(t, rows.Close()) dbt.MustExec("CREATE USER 'localhostuser'@'localhost'") dbt.MustExec("CREATE USER 'localhostuser'@'%'") @@ -1746,8 +1758,10 @@ func TestLocalhostClientMapping(t *testing.T) { rows := dbt.MustQuery("select user()") // NOTICE: this is not compatible with MySQL! (MySQL would report localhostuser@localhost also for 127.0.0.1) cli.checkRows(t, rows, "localhostuser@127.0.0.1") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'localhostuser'@'localhost'\nGRANT SELECT,UPDATE ON test.* TO 'localhostuser'@'localhost'") + require.NoError(t, rows.Close()) }) dbt.MustExec("DROP USER IF EXISTS 'localhostuser'@'localhost'") @@ -1763,12 +1777,14 @@ func TestLocalhostClientMapping(t *testing.T) { func(dbt *testkit.DBTestKit) { rows := dbt.MustQuery("select user()") cli.checkRows(t, rows, "localhostuser@localhost") + require.NoError(t, rows.Close()) rows = dbt.MustQuery("show grants") cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'localhostuser'@'%'\nGRANT SELECT ON test.* TO 'localhostuser'@'%'") + require.NoError(t, rows.Close()) }) // Test if only localhost exists - dbt.MustQuery("DROP USER 'localhostuser'@'%'") + dbt.MustExec("DROP USER 'localhostuser'@'%'") dbSocket, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "localhostuser" config.Net = "unix" diff --git a/server/util_test.go b/server/util_test.go index 35680210b75e4..c60f00bdf4e51 100644 --- a/server/util_test.go +++ b/server/util_test.go @@ -604,5 +604,6 @@ func newTestConfig() *config.Config { cfg.Host = "127.0.0.1" cfg.Status.StatusHost = "127.0.0.1" cfg.Security.AutoTLS = false + cfg.Socket = "" return cfg } diff --git a/session/session_test.go b/session/session_test.go index d0abd9d9f160a..130aa895c01f2 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -759,9 +759,11 @@ func (s *testSessionSuite) TestGetSysVariables(c *C) { tk.MustExec("select @@max_connections") tk.MustExec("select @@global.max_connections") _, err = tk.Exec("select @@session.max_connections") - c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue, Commentf("err %v", err)) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[variable:1238]Variable 'max_connections' is a GLOBAL variable") _, err = tk.Exec("select @@local.max_connections") - c.Assert(terror.ErrorEqual(err, variable.ErrIncorrectScope), IsTrue, Commentf("err %v", err)) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[variable:1238]Variable 'max_connections' is a GLOBAL variable") // Test ScopeNone tk.MustExec("select @@performance_schema_max_mutex_classes") @@ -769,6 +771,10 @@ func (s *testSessionSuite) TestGetSysVariables(c *C) { // For issue 19524, test tk.MustExec("select @@session.performance_schema_max_mutex_classes") tk.MustExec("select @@local.performance_schema_max_mutex_classes") + + _, err = tk.Exec("select @@global.last_insert_id") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[variable:1238]Variable 'last_insert_id' is a SESSION variable") } func (s *testSessionSuite) TestRetryResetStmtCtx(c *C) { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 057ba9a6dcdff..680e6f2c1367b 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1021,7 +1021,7 @@ func (s *SessionVars) CheckAndGetTxnScope() string { // UseDynamicPartitionPrune indicates whether use new dynamic partition prune. func (s *SessionVars) UseDynamicPartitionPrune() bool { - if s.InTxn() { + if s.InTxn() || !s.GetStatusFlag(mysql.ServerStatusAutocommit) { // UnionScan cannot get partition table IDs in dynamic-mode, this is a quick-fix for issues/26719, // please see it for more details. return false diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index a1fcf7e025053..cae6fd4c4aa50 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -19,10 +19,8 @@ import ( "fmt" "math" "runtime" - "sort" "strconv" "strings" - "sync" "sync/atomic" "time" @@ -30,7 +28,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -43,554 +40,6 @@ import ( atomic2 "go.uber.org/atomic" ) -// ScopeFlag is for system variable whether can be changed in global/session dynamically or not. -type ScopeFlag uint8 - -// TypeFlag is the SysVar type, which doesn't exactly match MySQL types. -type TypeFlag byte - -const ( - // ScopeNone means the system variable can not be changed dynamically. - ScopeNone ScopeFlag = 0 - // ScopeGlobal means the system variable can be changed globally. - ScopeGlobal ScopeFlag = 1 << 0 - // ScopeSession means the system variable can only be changed in current session. - ScopeSession ScopeFlag = 1 << 1 - - // TypeStr is the default - TypeStr TypeFlag = 0 - // TypeBool for boolean - TypeBool TypeFlag = 1 - // TypeInt for integer - TypeInt TypeFlag = 2 - // TypeEnum for Enum - TypeEnum TypeFlag = 3 - // TypeFloat for Double - TypeFloat TypeFlag = 4 - // TypeUnsigned for Unsigned integer - TypeUnsigned TypeFlag = 5 - // TypeTime for time of day (a TiDB extension) - TypeTime TypeFlag = 6 - // TypeDuration for a golang duration (a TiDB extension) - TypeDuration TypeFlag = 7 - - // On is the canonical string for ON - On = "ON" - // Off is the canonical string for OFF - Off = "OFF" - // Warn means return warnings - Warn = "WARN" - // IntOnly means enable for int type - IntOnly = "INT_ONLY" -) - -// Global config name list. -const ( - GlobalConfigEnableTopSQL = "enable_resource_metering" -) - -// SysVar is for system variable. -// All the fields of SysVar should be READ ONLY after created. -type SysVar struct { - // Scope is for whether can be changed or not - Scope ScopeFlag - // Name is the variable name. - Name string - // Value is the variable value. - Value string - // Type is the MySQL type (optional) - Type TypeFlag - // MinValue will automatically be validated when specified (optional) - MinValue int64 - // MaxValue will automatically be validated when specified (optional) - MaxValue uint64 - // AutoConvertNegativeBool applies to boolean types (optional) - AutoConvertNegativeBool bool - // ReadOnly applies to all types - ReadOnly bool - // PossibleValues applies to ENUM type - PossibleValues []string - // AllowEmpty is a special TiDB behavior which means "read value from config" (do not use) - AllowEmpty bool - // AllowEmptyAll is a special behavior that only applies to TiDBCapturePlanBaseline, TiDBTxnMode (do not use) - AllowEmptyAll bool - // AllowAutoValue means that the special value "-1" is permitted, even when outside of range. - AllowAutoValue bool - // Validation is a callback after the type validation has been performed, but before the Set function - Validation func(*SessionVars, string, string, ScopeFlag) (string, error) - // SetSession is called after validation but before updating systems[]. It also doubles as an Init function - // and will be called on all variables in builtinGlobalVariable, regardless of their scope. - SetSession func(*SessionVars, string) error - // SetGlobal is called after validation - SetGlobal func(*SessionVars, string) error - // IsHintUpdatable indicate whether it's updatable via SET_VAR() hint (optional) - IsHintUpdatable bool - // Hidden means that it still responds to SET but doesn't show up in SHOW VARIABLES - Hidden bool - // Aliases is a list of sysvars that should also be updated when this sysvar is updated. - // Updating aliases calls the SET function of the aliases, but does not update their aliases (preventing SET recursion) - Aliases []string - // GetSession is a getter function for session scope. - // It can be used by instance-scoped variables to overwrite the previously expected value. - GetSession func(*SessionVars) (string, error) - // GetGlobal is a getter function for global scope. - GetGlobal func(*SessionVars) (string, error) - // skipInit defines if the sysvar should be loaded into the session on init. - // This is only important to set for sysvars that include session scope, - // since global scoped sysvars are not-applicable. - skipInit bool - // IsNoop defines if the sysvar is a noop included for MySQL compatibility - IsNoop bool - // GlobalConfigName is the global config name of this global variable. - // If the global variable has the global config name, - // it should store the global config into PD(etcd) too when set global variable. - GlobalConfigName string -} - -// GetGlobalFromHook calls the GetSession func if it exists. -func (sv *SysVar) GetGlobalFromHook(s *SessionVars) (string, error) { - // Call the Getter if there is one defined. - if sv.GetGlobal != nil { - val, err := sv.GetGlobal(s) - if err != nil { - return val, err - } - // Ensure that the results from the getter are validated - // Since some are read directly from tables. - return sv.ValidateWithRelaxedValidation(s, val, ScopeGlobal), nil - } - if sv.HasNoneScope() { - return sv.Value, nil - } - return s.GlobalVarsAccessor.GetGlobalSysVar(sv.Name) -} - -// GetSessionFromHook calls the GetSession func if it exists. -func (sv *SysVar) GetSessionFromHook(s *SessionVars) (string, error) { - if sv.HasNoneScope() { - return sv.Value, nil - } - // Call the Getter if there is one defined. - if sv.GetSession != nil { - val, err := sv.GetSession(s) - if err != nil { - return val, err - } - // Ensure that the results from the getter are validated - // Since some are read directly from tables. - return sv.ValidateWithRelaxedValidation(s, val, ScopeSession), nil - } - var ( - ok bool - val string - ) - if val, ok = s.stmtVars[sv.Name]; ok { - return val, nil - } - if val, ok = s.systems[sv.Name]; !ok { - return val, errors.New("sysvar has not yet loaded") - } - return val, nil -} - -// SetSessionFromHook calls the SetSession func if it exists. -func (sv *SysVar) SetSessionFromHook(s *SessionVars, val string) error { - if sv.SetSession != nil { - if err := sv.SetSession(s, val); err != nil { - return err - } - } - s.systems[sv.Name] = val - - // Call the Set function on all the aliases for this sysVar - // Skipping the validation function, and not calling aliases of - // aliases. By skipping the validation function it means that things - // like duplicate warnings should not appear. - - if sv.Aliases != nil { - for _, aliasName := range sv.Aliases { - aliasSv := GetSysVar(aliasName) - if aliasSv.SetSession != nil { - if err := aliasSv.SetSession(s, val); err != nil { - return err - } - } - s.systems[aliasSv.Name] = val - } - } - return nil -} - -// SetGlobalFromHook calls the SetGlobal func if it exists. -func (sv *SysVar) SetGlobalFromHook(s *SessionVars, val string, skipAliases bool) error { - if sv.SetGlobal != nil { - return sv.SetGlobal(s, val) - } - - // Call the SetGlobalSysVarOnly function on all the aliases for this sysVar - // which skips the validation function and when SetGlobalFromHook is called again - // it will be with skipAliases=true. This helps break recursion because - // most aliases are reciprocal. - - if !skipAliases && sv.Aliases != nil { - for _, aliasName := range sv.Aliases { - if err := s.GlobalVarsAccessor.SetGlobalSysVarOnly(aliasName, val); err != nil { - return err - } - } - } - return nil -} - -// HasNoneScope returns true if the scope for the sysVar is None. -func (sv *SysVar) HasNoneScope() bool { - return sv.Scope == ScopeNone -} - -// HasSessionScope returns true if the scope for the sysVar includes session. -func (sv *SysVar) HasSessionScope() bool { - return sv.Scope&ScopeSession != 0 -} - -// HasGlobalScope returns true if the scope for the sysVar includes global. -func (sv *SysVar) HasGlobalScope() bool { - return sv.Scope&ScopeGlobal != 0 -} - -// Validate checks if system variable satisfies specific restriction. -func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (string, error) { - // Check that the scope is correct first. - if err := sv.validateScope(scope); err != nil { - return value, err - } - // Normalize the value and apply validation based on type. - // i.e. TypeBool converts 1/on/ON to ON. - normalizedValue, err := sv.ValidateFromType(vars, value, scope) - if err != nil { - return normalizedValue, err - } - // If type validation was successful, call the (optional) validation function - if sv.Validation != nil { - return sv.Validation(vars, normalizedValue, value, scope) - } - return normalizedValue, nil -} - -// ValidateFromType provides automatic validation based on the SysVar's type -func (sv *SysVar) ValidateFromType(vars *SessionVars, value string, scope ScopeFlag) (string, error) { - // The string "DEFAULT" is a special keyword in MySQL, which restores - // the compiled sysvar value. In which case we can skip further validation. - if strings.EqualFold(value, "DEFAULT") { - return sv.Value, nil - } - // Some sysvars in TiDB have a special behavior where the empty string means - // "use the config file value". This needs to be cleaned up once the behavior - // for instance variables is determined. - if value == "" && ((sv.AllowEmpty && scope == ScopeSession) || sv.AllowEmptyAll) { - return value, nil - } - // Provide validation using the SysVar struct - switch sv.Type { - case TypeUnsigned: - return sv.checkUInt64SystemVar(value, vars) - case TypeInt: - return sv.checkInt64SystemVar(value, vars) - case TypeBool: - return sv.checkBoolSystemVar(value, vars) - case TypeFloat: - return sv.checkFloatSystemVar(value, vars) - case TypeEnum: - return sv.checkEnumSystemVar(value, vars) - case TypeTime: - return sv.checkTimeSystemVar(value, vars) - case TypeDuration: - return sv.checkDurationSystemVar(value, vars) - } - return value, nil // typeString -} - -func (sv *SysVar) validateScope(scope ScopeFlag) error { - if sv.ReadOnly || sv.Scope == ScopeNone { - return ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") - } - if scope == ScopeGlobal && !sv.HasGlobalScope() { - return errLocalVariable.FastGenByArgs(sv.Name) - } - if scope == ScopeSession && !sv.HasSessionScope() { - return errGlobalVariable.FastGenByArgs(sv.Name) - } - return nil -} - -// ValidateWithRelaxedValidation normalizes values but can not return errors. -// Normalization+validation needs to be applied when reading values because older versions of TiDB -// may be less sophisticated in normalizing values. But errors should be caught and handled, -// because otherwise there will be upgrade issues. -func (sv *SysVar) ValidateWithRelaxedValidation(vars *SessionVars, value string, scope ScopeFlag) string { - warns := vars.StmtCtx.GetWarnings() - defer func() { - vars.StmtCtx.SetWarnings(warns) // RelaxedValidation = trim warnings too. - }() - normalizedValue, err := sv.ValidateFromType(vars, value, scope) - if err != nil { - return normalizedValue - } - if sv.Validation != nil { - normalizedValue, err = sv.Validation(vars, normalizedValue, value, scope) - if err != nil { - return normalizedValue - } - } - return normalizedValue -} - -const ( - localDayTimeFormat = "15:04" - // FullDayTimeFormat is the full format of analyze start time and end time. - FullDayTimeFormat = "15:04 -0700" -) - -func (sv *SysVar) checkTimeSystemVar(value string, vars *SessionVars) (string, error) { - var t time.Time - var err error - if len(value) <= len(localDayTimeFormat) { - t, err = time.ParseInLocation(localDayTimeFormat, value, vars.Location()) - } else { - t, err = time.ParseInLocation(FullDayTimeFormat, value, vars.Location()) - } - if err != nil { - return "", err - } - return t.Format(FullDayTimeFormat), nil -} - -func (sv *SysVar) checkDurationSystemVar(value string, vars *SessionVars) (string, error) { - d, err := time.ParseDuration(value) - if err != nil { - return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) - } - // Check for min/max violations - if int64(d) < sv.MinValue { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) - return time.Duration(sv.MinValue).String(), nil - } - if uint64(d) > sv.MaxValue { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) - return time.Duration(sv.MaxValue).String(), nil - } - // return a string representation of the duration - return d.String(), nil -} - -func (sv *SysVar) checkUInt64SystemVar(value string, vars *SessionVars) (string, error) { - if sv.AllowAutoValue && value == "-1" { - return value, nil - } - if len(value) == 0 { - return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) - } - if value[0] == '-' { - _, err := strconv.ParseInt(value, 10, 64) - if err != nil { - return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) - } - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) - return fmt.Sprintf("%d", sv.MinValue), nil - } - val, err := strconv.ParseUint(value, 10, 64) - if err != nil { - return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) - } - if val < uint64(sv.MinValue) { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) - return fmt.Sprintf("%d", sv.MinValue), nil - } - if val > sv.MaxValue { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) - return fmt.Sprintf("%d", sv.MaxValue), nil - } - return value, nil -} - -func (sv *SysVar) checkInt64SystemVar(value string, vars *SessionVars) (string, error) { - if sv.AllowAutoValue && value == "-1" { - return value, nil - } - val, err := strconv.ParseInt(value, 10, 64) - if err != nil { - return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) - } - if val < sv.MinValue { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) - return fmt.Sprintf("%d", sv.MinValue), nil - } - if val > int64(sv.MaxValue) { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) - return fmt.Sprintf("%d", sv.MaxValue), nil - } - return value, nil -} - -func (sv *SysVar) checkEnumSystemVar(value string, vars *SessionVars) (string, error) { - // The value could be either a string or the ordinal position in the PossibleValues. - // This allows for the behavior 0 = OFF, 1 = ON, 2 = DEMAND etc. - var iStr string - for i, v := range sv.PossibleValues { - iStr = fmt.Sprintf("%d", i) - if strings.EqualFold(value, v) || strings.EqualFold(value, iStr) { - return v, nil - } - } - return value, ErrWrongValueForVar.GenWithStackByArgs(sv.Name, value) -} - -func (sv *SysVar) checkFloatSystemVar(value string, vars *SessionVars) (string, error) { - if len(value) == 0 { - return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) - } - val, err := strconv.ParseFloat(value, 64) - if err != nil { - return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) - } - if val < float64(sv.MinValue) { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) - return fmt.Sprintf("%d", sv.MinValue), nil - } - if val > float64(sv.MaxValue) { - vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) - return fmt.Sprintf("%d", sv.MaxValue), nil - } - return value, nil -} - -func (sv *SysVar) checkBoolSystemVar(value string, vars *SessionVars) (string, error) { - if strings.EqualFold(value, "ON") { - return On, nil - } else if strings.EqualFold(value, "OFF") { - return Off, nil - } - val, err := strconv.ParseInt(value, 10, 64) - if err == nil { - // There are two types of conversion rules for integer values. - // The default only allows 0 || 1, but a subset of values convert any - // negative integer to 1. - if !sv.AutoConvertNegativeBool { - if val == 0 { - return Off, nil - } else if val == 1 { - return On, nil - } - } else { - if val == 1 || val < 0 { - return On, nil - } else if val == 0 { - return Off, nil - } - } - } - return value, ErrWrongValueForVar.GenWithStackByArgs(sv.Name, value) -} - -// GetNativeValType attempts to convert the val to the approx MySQL non-string type -func (sv *SysVar) GetNativeValType(val string) (types.Datum, byte, uint) { - switch sv.Type { - case TypeUnsigned: - u, err := strconv.ParseUint(val, 10, 64) - if err != nil { - u = 0 - } - return types.NewUintDatum(u), mysql.TypeLonglong, mysql.UnsignedFlag - case TypeBool: - optVal := int64(0) // OFF - if TiDBOptOn(val) { - optVal = 1 - } - return types.NewIntDatum(optVal), mysql.TypeLong, 0 - } - return types.NewStringDatum(val), mysql.TypeVarString, 0 -} - -// SkipInit returns true if when a new session is created we should "skip" copying -// an initial value to it (and call the SetSession func if it exists) -func (sv *SysVar) SkipInit() bool { - if sv.skipInit || sv.IsNoop { - return true - } - // These a special "Global-only" sysvars that for backward compatibility - // are currently cached in the session. Please don't add to this list. - switch sv.Name { - case TiDBEnableChangeMultiSchema, TiDBDDLReorgBatchSize, TiDBEnableAlterPlacement, - TiDBMaxDeltaSchemaCount, InitConnect, MaxPreparedStmtCount, - TiDBDDLReorgWorkerCount, TiDBDDLErrorCountLimit, TiDBRowFormatVersion, - TiDBEnableTelemetry, TiDBEnablePointGetCache: - return false - } - return !sv.HasSessionScope() -} - -var sysVars map[string]*SysVar -var sysVarsLock sync.RWMutex - -// RegisterSysVar adds a sysvar to the SysVars list -func RegisterSysVar(sv *SysVar) { - name := strings.ToLower(sv.Name) - sysVarsLock.Lock() - sysVars[name] = sv - sysVarsLock.Unlock() -} - -// UnregisterSysVar removes a sysvar from the SysVars list -// currently only used in tests. -func UnregisterSysVar(name string) { - name = strings.ToLower(name) - sysVarsLock.Lock() - delete(sysVars, name) - sysVarsLock.Unlock() -} - -// GetSysVar returns sys var info for name as key. -func GetSysVar(name string) *SysVar { - name = strings.ToLower(name) - sysVarsLock.RLock() - defer sysVarsLock.RUnlock() - - return sysVars[name] -} - -// SetSysVar sets a sysvar. In fact, SysVar is immutable. -// SetSysVar is implemented by register a new SysVar with the same name again. -// This will not propagate to the cluster, so it should only be -// used for instance scoped AUTO variables such as system_time_zone. -func SetSysVar(name string, value string) { - old := GetSysVar(name) - tmp := *old - tmp.Value = value - RegisterSysVar(&tmp) -} - -// GetSysVars deep copies the sysVars list under a RWLock -func GetSysVars() map[string]*SysVar { - sysVarsLock.RLock() - defer sysVarsLock.RUnlock() - copy := make(map[string]*SysVar, len(sysVars)) - for name, sv := range sysVars { - tmp := *sv - copy[name] = &tmp - } - return copy -} - -func init() { - sysVars = make(map[string]*SysVar) - for _, v := range defaultSysVars { - RegisterSysVar(v) - } - for _, v := range noopSysVars { - v.IsNoop = true - RegisterSysVar(v) - } -} - var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: SQLSelectLimit, Value: "18446744073709551615", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { result, err := strconv.ParseUint(val, 10, 64) @@ -1883,24 +1332,6 @@ var defaultSysVars = []*SysVar{ }}, } -func collectAllowFuncName4ExpressionIndex() string { - var str []string - for funcName := range GAFunction4ExpressionIndex { - str = append(str, funcName) - } - sort.Strings(str) - return strings.Join(str, ", ") -} - -// GAFunction4ExpressionIndex stores functions GA for expression index. -var GAFunction4ExpressionIndex = map[string]struct{}{ - ast.Lower: {}, - ast.Upper: {}, - ast.MD5: {}, - ast.Reverse: {}, - ast.VitessHash: {}, -} - // FeedbackProbability points to the FeedbackProbability in statistics package. // It's initialized in init() in feedback.go to solve import cycle. var FeedbackProbability *atomic2.Float64 @@ -2208,17 +1639,3 @@ const ( // RandSeed2 is the name of 'rand_seed2' system variable. RandSeed2 = "rand_seed2" ) - -// GlobalVarAccessor is the interface for accessing global scope system and status variables. -type GlobalVarAccessor interface { - // GetGlobalSysVar gets the global system variable value for name. - GetGlobalSysVar(name string) (string, error) - // SetGlobalSysVar sets the global system variable name to value. - SetGlobalSysVar(name string, value string) error - // SetGlobalSysVarOnly sets the global system variable without calling the validation function or updating aliases. - SetGlobalSysVarOnly(name string, value string) error - // GetTiDBTableValue gets a value from mysql.tidb for the key 'name' - GetTiDBTableValue(name string) (string, error) - // SetTiDBTableValue sets a value+comment for the mysql.tidb key 'name' - SetTiDBTableValue(name, value, comment string) error -} diff --git a/sessionctx/variable/variable.go b/sessionctx/variable/variable.go new file mode 100644 index 0000000000000..675ca3bdc0887 --- /dev/null +++ b/sessionctx/variable/variable.go @@ -0,0 +1,589 @@ +// Copyright 2021 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. + +package variable + +import ( + "fmt" + "strconv" + "strings" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/types" +) + +// ScopeFlag is for system variable whether can be changed in global/session dynamically or not. +type ScopeFlag uint8 + +// TypeFlag is the SysVar type, which doesn't exactly match MySQL types. +type TypeFlag byte + +const ( + // ScopeNone means the system variable can not be changed dynamically. + ScopeNone ScopeFlag = 0 + // ScopeGlobal means the system variable can be changed globally. + ScopeGlobal ScopeFlag = 1 << 0 + // ScopeSession means the system variable can only be changed in current session. + ScopeSession ScopeFlag = 1 << 1 + + // TypeStr is the default + TypeStr TypeFlag = 0 + // TypeBool for boolean + TypeBool TypeFlag = 1 + // TypeInt for integer + TypeInt TypeFlag = 2 + // TypeEnum for Enum + TypeEnum TypeFlag = 3 + // TypeFloat for Double + TypeFloat TypeFlag = 4 + // TypeUnsigned for Unsigned integer + TypeUnsigned TypeFlag = 5 + // TypeTime for time of day (a TiDB extension) + TypeTime TypeFlag = 6 + // TypeDuration for a golang duration (a TiDB extension) + TypeDuration TypeFlag = 7 + + // On is the canonical string for ON + On = "ON" + // Off is the canonical string for OFF + Off = "OFF" + // Warn means return warnings + Warn = "WARN" + // IntOnly means enable for int type + IntOnly = "INT_ONLY" +) + +// Global config name list. +const ( + GlobalConfigEnableTopSQL = "enable_resource_metering" +) + +// SysVar is for system variable. +// All the fields of SysVar should be READ ONLY after created. +type SysVar struct { + // Scope is for whether can be changed or not + Scope ScopeFlag + // Name is the variable name. + Name string + // Value is the variable value. + Value string + // Type is the MySQL type (optional) + Type TypeFlag + // MinValue will automatically be validated when specified (optional) + MinValue int64 + // MaxValue will automatically be validated when specified (optional) + MaxValue uint64 + // AutoConvertNegativeBool applies to boolean types (optional) + AutoConvertNegativeBool bool + // ReadOnly applies to all types + ReadOnly bool + // PossibleValues applies to ENUM type + PossibleValues []string + // AllowEmpty is a special TiDB behavior which means "read value from config" (do not use) + AllowEmpty bool + // AllowEmptyAll is a special behavior that only applies to TiDBCapturePlanBaseline, TiDBTxnMode (do not use) + AllowEmptyAll bool + // AllowAutoValue means that the special value "-1" is permitted, even when outside of range. + AllowAutoValue bool + // Validation is a callback after the type validation has been performed, but before the Set function + Validation func(*SessionVars, string, string, ScopeFlag) (string, error) + // SetSession is called after validation but before updating systems[]. It also doubles as an Init function + // and will be called on all variables in builtinGlobalVariable, regardless of their scope. + SetSession func(*SessionVars, string) error + // SetGlobal is called after validation + SetGlobal func(*SessionVars, string) error + // IsHintUpdatable indicate whether it's updatable via SET_VAR() hint (optional) + IsHintUpdatable bool + // Hidden means that it still responds to SET but doesn't show up in SHOW VARIABLES + Hidden bool + // Aliases is a list of sysvars that should also be updated when this sysvar is updated. + // Updating aliases calls the SET function of the aliases, but does not update their aliases (preventing SET recursion) + Aliases []string + // GetSession is a getter function for session scope. + // It can be used by instance-scoped variables to overwrite the previously expected value. + GetSession func(*SessionVars) (string, error) + // GetGlobal is a getter function for global scope. + GetGlobal func(*SessionVars) (string, error) + // skipInit defines if the sysvar should be loaded into the session on init. + // This is only important to set for sysvars that include session scope, + // since global scoped sysvars are not-applicable. + skipInit bool + // IsNoop defines if the sysvar is a noop included for MySQL compatibility + IsNoop bool + // GlobalConfigName is the global config name of this global variable. + // If the global variable has the global config name, + // it should store the global config into PD(etcd) too when set global variable. + GlobalConfigName string +} + +// GetGlobalFromHook calls the GetSession func if it exists. +func (sv *SysVar) GetGlobalFromHook(s *SessionVars) (string, error) { + // Call the Getter if there is one defined. + if sv.GetGlobal != nil { + val, err := sv.GetGlobal(s) + if err != nil { + return val, err + } + // Ensure that the results from the getter are validated + // Since some are read directly from tables. + return sv.ValidateWithRelaxedValidation(s, val, ScopeGlobal), nil + } + if sv.HasNoneScope() { + return sv.Value, nil + } + return s.GlobalVarsAccessor.GetGlobalSysVar(sv.Name) +} + +// GetSessionFromHook calls the GetSession func if it exists. +func (sv *SysVar) GetSessionFromHook(s *SessionVars) (string, error) { + if sv.HasNoneScope() { + return sv.Value, nil + } + // Call the Getter if there is one defined. + if sv.GetSession != nil { + val, err := sv.GetSession(s) + if err != nil { + return val, err + } + // Ensure that the results from the getter are validated + // Since some are read directly from tables. + return sv.ValidateWithRelaxedValidation(s, val, ScopeSession), nil + } + var ( + ok bool + val string + ) + if val, ok = s.stmtVars[sv.Name]; ok { + return val, nil + } + if val, ok = s.systems[sv.Name]; !ok { + return val, errors.New("sysvar has not yet loaded") + } + return val, nil +} + +// SetSessionFromHook calls the SetSession func if it exists. +func (sv *SysVar) SetSessionFromHook(s *SessionVars, val string) error { + if sv.SetSession != nil { + if err := sv.SetSession(s, val); err != nil { + return err + } + } + s.systems[sv.Name] = val + + // Call the Set function on all the aliases for this sysVar + // Skipping the validation function, and not calling aliases of + // aliases. By skipping the validation function it means that things + // like duplicate warnings should not appear. + + if sv.Aliases != nil { + for _, aliasName := range sv.Aliases { + aliasSv := GetSysVar(aliasName) + if aliasSv.SetSession != nil { + if err := aliasSv.SetSession(s, val); err != nil { + return err + } + } + s.systems[aliasSv.Name] = val + } + } + return nil +} + +// SetGlobalFromHook calls the SetGlobal func if it exists. +func (sv *SysVar) SetGlobalFromHook(s *SessionVars, val string, skipAliases bool) error { + if sv.SetGlobal != nil { + return sv.SetGlobal(s, val) + } + + // Call the SetGlobalSysVarOnly function on all the aliases for this sysVar + // which skips the validation function and when SetGlobalFromHook is called again + // it will be with skipAliases=true. This helps break recursion because + // most aliases are reciprocal. + + if !skipAliases && sv.Aliases != nil { + for _, aliasName := range sv.Aliases { + if err := s.GlobalVarsAccessor.SetGlobalSysVarOnly(aliasName, val); err != nil { + return err + } + } + } + return nil +} + +// HasNoneScope returns true if the scope for the sysVar is None. +func (sv *SysVar) HasNoneScope() bool { + return sv.Scope == ScopeNone +} + +// HasSessionScope returns true if the scope for the sysVar includes session. +func (sv *SysVar) HasSessionScope() bool { + return sv.Scope&ScopeSession != 0 +} + +// HasGlobalScope returns true if the scope for the sysVar includes global. +func (sv *SysVar) HasGlobalScope() bool { + return sv.Scope&ScopeGlobal != 0 +} + +// Validate checks if system variable satisfies specific restriction. +func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (string, error) { + // Check that the scope is correct first. + if err := sv.validateScope(scope); err != nil { + return value, err + } + // Normalize the value and apply validation based on type. + // i.e. TypeBool converts 1/on/ON to ON. + normalizedValue, err := sv.ValidateFromType(vars, value, scope) + if err != nil { + return normalizedValue, err + } + // If type validation was successful, call the (optional) validation function + if sv.Validation != nil { + return sv.Validation(vars, normalizedValue, value, scope) + } + return normalizedValue, nil +} + +// ValidateFromType provides automatic validation based on the SysVar's type +func (sv *SysVar) ValidateFromType(vars *SessionVars, value string, scope ScopeFlag) (string, error) { + // The string "DEFAULT" is a special keyword in MySQL, which restores + // the compiled sysvar value. In which case we can skip further validation. + if strings.EqualFold(value, "DEFAULT") { + return sv.Value, nil + } + // Some sysvars in TiDB have a special behavior where the empty string means + // "use the config file value". This needs to be cleaned up once the behavior + // for instance variables is determined. + if value == "" && ((sv.AllowEmpty && scope == ScopeSession) || sv.AllowEmptyAll) { + return value, nil + } + // Provide validation using the SysVar struct + switch sv.Type { + case TypeUnsigned: + return sv.checkUInt64SystemVar(value, vars) + case TypeInt: + return sv.checkInt64SystemVar(value, vars) + case TypeBool: + return sv.checkBoolSystemVar(value, vars) + case TypeFloat: + return sv.checkFloatSystemVar(value, vars) + case TypeEnum: + return sv.checkEnumSystemVar(value, vars) + case TypeTime: + return sv.checkTimeSystemVar(value, vars) + case TypeDuration: + return sv.checkDurationSystemVar(value, vars) + } + return value, nil // typeString +} + +func (sv *SysVar) validateScope(scope ScopeFlag) error { + if sv.ReadOnly || sv.Scope == ScopeNone { + return ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") + } + if scope == ScopeGlobal && !sv.HasGlobalScope() { + return errLocalVariable.FastGenByArgs(sv.Name) + } + if scope == ScopeSession && !sv.HasSessionScope() { + return errGlobalVariable.FastGenByArgs(sv.Name) + } + return nil +} + +// ValidateWithRelaxedValidation normalizes values but can not return errors. +// Normalization+validation needs to be applied when reading values because older versions of TiDB +// may be less sophisticated in normalizing values. But errors should be caught and handled, +// because otherwise there will be upgrade issues. +func (sv *SysVar) ValidateWithRelaxedValidation(vars *SessionVars, value string, scope ScopeFlag) string { + warns := vars.StmtCtx.GetWarnings() + defer func() { + vars.StmtCtx.SetWarnings(warns) // RelaxedValidation = trim warnings too. + }() + normalizedValue, err := sv.ValidateFromType(vars, value, scope) + if err != nil { + return normalizedValue + } + if sv.Validation != nil { + normalizedValue, err = sv.Validation(vars, normalizedValue, value, scope) + if err != nil { + return normalizedValue + } + } + return normalizedValue +} + +const ( + localDayTimeFormat = "15:04" + // FullDayTimeFormat is the full format of analyze start time and end time. + FullDayTimeFormat = "15:04 -0700" +) + +func (sv *SysVar) checkTimeSystemVar(value string, vars *SessionVars) (string, error) { + var t time.Time + var err error + if len(value) <= len(localDayTimeFormat) { + t, err = time.ParseInLocation(localDayTimeFormat, value, vars.Location()) + } else { + t, err = time.ParseInLocation(FullDayTimeFormat, value, vars.Location()) + } + if err != nil { + return "", err + } + return t.Format(FullDayTimeFormat), nil +} + +func (sv *SysVar) checkDurationSystemVar(value string, vars *SessionVars) (string, error) { + d, err := time.ParseDuration(value) + if err != nil { + return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) + } + // Check for min/max violations + if int64(d) < sv.MinValue { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) + return time.Duration(sv.MinValue).String(), nil + } + if uint64(d) > sv.MaxValue { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) + return time.Duration(sv.MaxValue).String(), nil + } + // return a string representation of the duration + return d.String(), nil +} + +func (sv *SysVar) checkUInt64SystemVar(value string, vars *SessionVars) (string, error) { + if sv.AllowAutoValue && value == "-1" { + return value, nil + } + if len(value) == 0 { + return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) + } + if value[0] == '-' { + _, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) + } + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) + return fmt.Sprintf("%d", sv.MinValue), nil + } + val, err := strconv.ParseUint(value, 10, 64) + if err != nil { + return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) + } + if val < uint64(sv.MinValue) { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) + return fmt.Sprintf("%d", sv.MinValue), nil + } + if val > sv.MaxValue { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) + return fmt.Sprintf("%d", sv.MaxValue), nil + } + return value, nil +} + +func (sv *SysVar) checkInt64SystemVar(value string, vars *SessionVars) (string, error) { + if sv.AllowAutoValue && value == "-1" { + return value, nil + } + val, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) + } + if val < sv.MinValue { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) + return fmt.Sprintf("%d", sv.MinValue), nil + } + if val > int64(sv.MaxValue) { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) + return fmt.Sprintf("%d", sv.MaxValue), nil + } + return value, nil +} + +func (sv *SysVar) checkEnumSystemVar(value string, vars *SessionVars) (string, error) { + // The value could be either a string or the ordinal position in the PossibleValues. + // This allows for the behavior 0 = OFF, 1 = ON, 2 = DEMAND etc. + var iStr string + for i, v := range sv.PossibleValues { + iStr = fmt.Sprintf("%d", i) + if strings.EqualFold(value, v) || strings.EqualFold(value, iStr) { + return v, nil + } + } + return value, ErrWrongValueForVar.GenWithStackByArgs(sv.Name, value) +} + +func (sv *SysVar) checkFloatSystemVar(value string, vars *SessionVars) (string, error) { + if len(value) == 0 { + return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) + } + val, err := strconv.ParseFloat(value, 64) + if err != nil { + return value, ErrWrongTypeForVar.GenWithStackByArgs(sv.Name) + } + if val < float64(sv.MinValue) { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) + return fmt.Sprintf("%d", sv.MinValue), nil + } + if val > float64(sv.MaxValue) { + vars.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(sv.Name, value)) + return fmt.Sprintf("%d", sv.MaxValue), nil + } + return value, nil +} + +func (sv *SysVar) checkBoolSystemVar(value string, vars *SessionVars) (string, error) { + if strings.EqualFold(value, "ON") { + return On, nil + } else if strings.EqualFold(value, "OFF") { + return Off, nil + } + val, err := strconv.ParseInt(value, 10, 64) + if err == nil { + // There are two types of conversion rules for integer values. + // The default only allows 0 || 1, but a subset of values convert any + // negative integer to 1. + if !sv.AutoConvertNegativeBool { + if val == 0 { + return Off, nil + } else if val == 1 { + return On, nil + } + } else { + if val == 1 || val < 0 { + return On, nil + } else if val == 0 { + return Off, nil + } + } + } + return value, ErrWrongValueForVar.GenWithStackByArgs(sv.Name, value) +} + +// GetNativeValType attempts to convert the val to the approx MySQL non-string type +func (sv *SysVar) GetNativeValType(val string) (types.Datum, byte, uint) { + switch sv.Type { + case TypeUnsigned: + u, err := strconv.ParseUint(val, 10, 64) + if err != nil { + u = 0 + } + return types.NewUintDatum(u), mysql.TypeLonglong, mysql.UnsignedFlag + case TypeBool: + optVal := int64(0) // OFF + if TiDBOptOn(val) { + optVal = 1 + } + return types.NewIntDatum(optVal), mysql.TypeLong, 0 + } + return types.NewStringDatum(val), mysql.TypeVarString, 0 +} + +// SkipInit returns true if when a new session is created we should "skip" copying +// an initial value to it (and call the SetSession func if it exists) +func (sv *SysVar) SkipInit() bool { + if sv.skipInit || sv.IsNoop { + return true + } + // These a special "Global-only" sysvars that for backward compatibility + // are currently cached in the session. Please don't add to this list. + switch sv.Name { + case TiDBEnableChangeMultiSchema, TiDBDDLReorgBatchSize, TiDBEnableAlterPlacement, + TiDBMaxDeltaSchemaCount, InitConnect, MaxPreparedStmtCount, + TiDBDDLReorgWorkerCount, TiDBDDLErrorCountLimit, TiDBRowFormatVersion, + TiDBEnableTelemetry, TiDBEnablePointGetCache: + return false + } + return !sv.HasSessionScope() +} + +var sysVars map[string]*SysVar +var sysVarsLock sync.RWMutex + +// RegisterSysVar adds a sysvar to the SysVars list +func RegisterSysVar(sv *SysVar) { + name := strings.ToLower(sv.Name) + sysVarsLock.Lock() + sysVars[name] = sv + sysVarsLock.Unlock() +} + +// UnregisterSysVar removes a sysvar from the SysVars list +// currently only used in tests. +func UnregisterSysVar(name string) { + name = strings.ToLower(name) + sysVarsLock.Lock() + delete(sysVars, name) + sysVarsLock.Unlock() +} + +// GetSysVar returns sys var info for name as key. +func GetSysVar(name string) *SysVar { + name = strings.ToLower(name) + sysVarsLock.RLock() + defer sysVarsLock.RUnlock() + + return sysVars[name] +} + +// SetSysVar sets a sysvar. In fact, SysVar is immutable. +// SetSysVar is implemented by register a new SysVar with the same name again. +// This will not propagate to the cluster, so it should only be +// used for instance scoped AUTO variables such as system_time_zone. +func SetSysVar(name string, value string) { + old := GetSysVar(name) + tmp := *old + tmp.Value = value + RegisterSysVar(&tmp) +} + +// GetSysVars deep copies the sysVars list under a RWLock +func GetSysVars() map[string]*SysVar { + sysVarsLock.RLock() + defer sysVarsLock.RUnlock() + copy := make(map[string]*SysVar, len(sysVars)) + for name, sv := range sysVars { + tmp := *sv + copy[name] = &tmp + } + return copy +} + +func init() { + sysVars = make(map[string]*SysVar) + for _, v := range defaultSysVars { + RegisterSysVar(v) + } + for _, v := range noopSysVars { + v.IsNoop = true + RegisterSysVar(v) + } +} + +// GlobalVarAccessor is the interface for accessing global scope system and status variables. +type GlobalVarAccessor interface { + // GetGlobalSysVar gets the global system variable value for name. + GetGlobalSysVar(name string) (string, error) + // SetGlobalSysVar sets the global system variable name to value. + SetGlobalSysVar(name string, value string) error + // SetGlobalSysVarOnly sets the global system variable without calling the validation function or updating aliases. + SetGlobalSysVarOnly(name string, value string) error + // GetTiDBTableValue gets a value from mysql.tidb for the key 'name' + GetTiDBTableValue(name string) (string, error) + // SetTiDBTableValue sets a value+comment for the mysql.tidb key 'name' + SetTiDBTableValue(name, value, comment string) error +} diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index a60dd7f04d2d4..15f13a53521f5 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -16,6 +16,7 @@ package variable import ( "fmt" + "sort" "strconv" "strings" "sync" @@ -23,6 +24,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" @@ -482,3 +484,21 @@ func (v *serverGlobalVariable) GetVal() string { } return v.globalVal } + +func collectAllowFuncName4ExpressionIndex() string { + var str []string + for funcName := range GAFunction4ExpressionIndex { + str = append(str, funcName) + } + sort.Strings(str) + return strings.Join(str, ", ") +} + +// GAFunction4ExpressionIndex stores functions GA for expression index. +var GAFunction4ExpressionIndex = map[string]struct{}{ + ast.Lower: {}, + ast.Upper: {}, + ast.MD5: {}, + ast.Reverse: {}, + ast.VitessHash: {}, +} diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 8c5f1d08a4013..23b2de4333af8 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -906,13 +906,39 @@ func (s *testSerialStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.T } // nolint:unused -func (s *testSerialStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, t string, p string, topn, buckets int) { +func (s *testSerialStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, db, t, p string, topn, buckets int) { + tbl, err := s.do.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(t)) + c.Assert(err, IsNil) + + tblInfo := tbl.Meta() + physicalID := tblInfo.ID + if p != "global" { + for _, def := range tbl.Meta().GetPartitionInfo().Definitions { + if def.Name.L == p { + physicalID = def.ID + } + } + } + tblStats, err := s.do.StatsHandle().TableStatsFromStorage(tblInfo, physicalID, true, 0) + c.Assert(err, IsNil) + delta := buckets/2 + 10 - for _, isIdx := range []int{0, 1} { - c.Assert(len(tk.MustQuery(fmt.Sprintf("show stats_topn where table_name='%v' and partition_name='%v' and is_index=%v", t, p, isIdx)).Rows()), Equals, topn) - numBuckets := len(tk.MustQuery(fmt.Sprintf("show stats_buckets where table_name='%v' and partition_name='%v' and is_index=%v", t, p, isIdx)).Rows()) + for _, idxStats := range tblStats.Indices { + numTopN := idxStats.TopN.Num() + numBuckets := len(idxStats.Buckets) // since the hist-building algorithm doesn't stipulate the final bucket number to be equal to the expected number exactly, // we have to check the results by a range here. + c.Assert(numTopN, Equals, topn) + c.Assert(numBuckets, GreaterEqual, buckets-delta) + c.Assert(numBuckets, LessEqual, buckets+delta) + } + for _, colStats := range tblStats.Columns { + if len(colStats.Buckets) == 0 { + continue // it's not loaded + } + numTopN := colStats.TopN.Num() + numBuckets := len(colStats.Buckets) + c.Assert(numTopN, Equals, topn) c.Assert(numBuckets, GreaterEqual, buckets-delta) c.Assert(numBuckets, LessEqual, buckets+delta) } @@ -947,9 +973,9 @@ func (s *testSerialStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { sql := fmt.Sprintf("analyze table test_gstats_opt with %v topn, %v buckets", ca.topn, ca.buckets) if !ca.err { tk.MustExec(sql) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "global", ca.topn, ca.buckets) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "p0", ca.topn, ca.buckets) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "p1", ca.topn, ca.buckets) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt", "global", ca.topn, ca.buckets) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt", "p0", ca.topn, ca.buckets) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt", "p1", ca.topn, ca.buckets) } else { err := tk.ExecToErr(sql) c.Assert(err, NotNil) @@ -966,25 +992,25 @@ func (s *testSerialStatsSuite) TestAnalyzeGlobalStatsWithOpts2(c *C) { s.prepareForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2") tk.MustExec("analyze table test_gstats_opt2 with 20 topn, 50 buckets, 1000 samples") - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "global", 2, 50) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p0", 1, 50) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p1", 1, 50) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 2, 50) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 1, 50) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 50) // analyze a partition to let its options be different with others' tk.MustExec("analyze table test_gstats_opt2 partition p0 with 10 topn, 20 buckets") - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "global", 10, 20) // use new options - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p0", 10, 20) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p1", 1, 50) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 10, 20) // use new options + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 10, 20) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 50) tk.MustExec("analyze table test_gstats_opt2 partition p1 with 100 topn, 200 buckets") - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "global", 100, 200) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p0", 10, 20) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p1", 100, 200) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 100, 200) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 10, 20) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 100, 200) tk.MustExec("analyze table test_gstats_opt2 partition p0 with 20 topn") // change back to 20 topn - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "global", 20, 256) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p0", 20, 256) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p1", 100, 200) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 20, 256) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 20, 256) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 100, 200) } func (s *testStatsSuite) TestGlobalStatsHealthy(c *C) { diff --git a/statistics/sample.go b/statistics/sample.go index 39065139dd725..6b55ad8367b60 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -78,7 +78,7 @@ func (s *sampleItemSorter) Len() int { func (s *sampleItemSorter) Less(i, j int) bool { var cmp int - cmp, s.err = s.items[i].Value.CompareDatum(s.sc, &s.items[j].Value) + cmp, s.err = s.items[i].Value.Compare(s.sc, &s.items[j].Value, collate.GetBinaryCollator()) if s.err != nil { return true } diff --git a/statistics/table.go b/statistics/table.go index e0fabb74f8bd1..73c1d608fc403 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -33,8 +33,11 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/tracing" "go.uber.org/atomic" + "go.uber.org/zap" ) const ( @@ -331,17 +334,26 @@ func (t *Table) ColumnEqualRowCount(sc *stmtctx.StatementContext, value types.Da // GetRowCountByIntColumnRanges estimates the row count by a slice of IntColumnRange. func (coll *HistColl) GetRowCountByIntColumnRanges(sc *stmtctx.StatementContext, colID int64, intRanges []*ranger.Range) (float64, error) { + var result float64 c, ok := coll.Columns[colID] if !ok || c.IsInvalid(sc, coll.Pseudo) { if len(intRanges) == 0 { return 0, nil } if intRanges[0].LowVal[0].Kind() == types.KindInt64 { - return getPseudoRowCountBySignedIntRanges(intRanges, float64(coll.Count)), nil + result = getPseudoRowCountBySignedIntRanges(intRanges, float64(coll.Count)) + } else { + result = getPseudoRowCountByUnsignedIntRanges(intRanges, float64(coll.Count)) + } + if sc.EnableOptimizerCETrace && ok { + CETraceRange(sc, coll.PhysicalID, []string{c.Info.Name.O}, intRanges, "Column Stats-Pseudo", uint64(result)) } - return getPseudoRowCountByUnsignedIntRanges(intRanges, float64(coll.Count)), nil + return result, nil } result, err := c.GetColumnRowCount(sc, intRanges, coll.Count, true) + if sc.EnableOptimizerCETrace { + CETraceRange(sc, coll.PhysicalID, []string{c.Info.Name.O}, intRanges, "Column Stats", uint64(result)) + } return result, errors.Trace(err) } @@ -349,21 +361,38 @@ func (coll *HistColl) GetRowCountByIntColumnRanges(sc *stmtctx.StatementContext, func (coll *HistColl) GetRowCountByColumnRanges(sc *stmtctx.StatementContext, colID int64, colRanges []*ranger.Range) (float64, error) { c, ok := coll.Columns[colID] if !ok || c.IsInvalid(sc, coll.Pseudo) { - return GetPseudoRowCountByColumnRanges(sc, float64(coll.Count), colRanges, 0) + result, err := GetPseudoRowCountByColumnRanges(sc, float64(coll.Count), colRanges, 0) + if err == nil && sc.EnableOptimizerCETrace && ok { + CETraceRange(sc, coll.PhysicalID, []string{c.Info.Name.O}, colRanges, "Column Stats-Pseudo", uint64(result)) + } + return result, err } result, err := c.GetColumnRowCount(sc, colRanges, coll.Count, false) + if sc.EnableOptimizerCETrace { + CETraceRange(sc, coll.PhysicalID, []string{c.Info.Name.O}, colRanges, "Column Stats", uint64(result)) + } return result, errors.Trace(err) } // GetRowCountByIndexRanges estimates the row count by a slice of Range. func (coll *HistColl) GetRowCountByIndexRanges(sc *stmtctx.StatementContext, idxID int64, indexRanges []*ranger.Range) (float64, error) { - idx := coll.Indices[idxID] - if idx == nil || idx.IsInvalid(coll.Pseudo) { + idx, ok := coll.Indices[idxID] + colNames := make([]string, 0, 8) + if ok { + for _, col := range idx.Info.Columns { + colNames = append(colNames, col.Name.O) + } + } + if !ok || idx.IsInvalid(coll.Pseudo) { colsLen := -1 if idx != nil && idx.Info.Unique { colsLen = len(idx.Info.Columns) } - return getPseudoRowCountByIndexRanges(sc, indexRanges, float64(coll.Count), colsLen) + result, err := getPseudoRowCountByIndexRanges(sc, indexRanges, float64(coll.Count), colsLen) + if err == nil && sc.EnableOptimizerCETrace && ok { + CETraceRange(sc, coll.PhysicalID, colNames, indexRanges, "Index Stats-Pseudo", uint64(result)) + } + return result, err } var result float64 var err error @@ -372,9 +401,43 @@ func (coll *HistColl) GetRowCountByIndexRanges(sc *stmtctx.StatementContext, idx } else { result, err = idx.GetRowCount(sc, coll, indexRanges, coll.Count) } + if sc.EnableOptimizerCETrace { + CETraceRange(sc, coll.PhysicalID, colNames, indexRanges, "Index Stats", uint64(result)) + } return result, errors.Trace(err) } +// CETraceRange appends a list of ranges and related information into CE trace +func CETraceRange(sc *stmtctx.StatementContext, tableID int64, colNames []string, ranges []*ranger.Range, tp string, rowCount uint64) { + allPoint := true + for _, ran := range ranges { + if !ran.IsPointNullable(sc) { + allPoint = false + break + } + } + if allPoint { + tp = tp + "-Point" + } else { + tp = tp + "-Range" + } + expr, err := ranger.RangesToString(sc, ranges, colNames) + if err != nil { + logutil.BgLogger().Debug("[OptimizerTrace] Failed to trace CE of ranges", zap.Error(err)) + } + // We don't need to record meaningless expressions. + if expr == "" || expr == "true" || expr == "false" { + return + } + CERecord := tracing.CETraceRecord{ + TableID: tableID, + Type: tp, + Expr: expr, + RowCount: rowCount, + } + sc.OptimizerCETrace = append(sc.OptimizerCETrace, &CERecord) +} + // PseudoAvgCountPerValue gets a pseudo average count if histogram not exists. func (t *Table) PseudoAvgCountPerValue() float64 { return float64(t.Count) / pseudoEqualRate diff --git a/statistics/testdata/trace_suite_out.json b/statistics/testdata/trace_suite_out.json index d45173d34d24c..9d13e199263a3 100644 --- a/statistics/testdata/trace_suite_out.json +++ b/statistics/testdata/trace_suite_out.json @@ -5,6 +5,41 @@ { "Expr": "a > 0 and a < 2", "Trace": [ + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Point", + "Expr": "((a = 1))", + "RowCount": 4 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Index Stats-Point", + "Expr": "((a = 1))", + "RowCount": 4 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Range", + "Expr": "((a > 0 and a < 2))", + "RowCount": 4 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Point", + "Expr": "((a = 1))", + "RowCount": 4 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Index Stats-Point", + "Expr": "((a = 1))", + "RowCount": 4 + }, { "TableID": 57, "TableName": "", @@ -24,6 +59,27 @@ { "Expr": "a >= 1 and a < 10", "Trace": [ + { + "TableID": 57, + "TableName": "", + "Type": "Index Stats-Range", + "Expr": "((a >= 1 and a < 10))", + "RowCount": 6 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Range", + "Expr": "((a >= 1 and a < 10))", + "RowCount": 6 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Index Stats-Range", + "Expr": "((a >= 1 and a < 10))", + "RowCount": 6 + }, { "TableID": 57, "TableName": "", @@ -43,6 +99,20 @@ { "Expr": "a < 3 or b < 4", "Trace": [ + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Range", + "Expr": "((a < 3))", + "RowCount": 6 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Index Stats-Range", + "Expr": "((a < 3))", + "RowCount": 6 + }, { "TableID": 57, "TableName": "", @@ -64,6 +134,13 @@ "Expr": "`or`(`lt`(test.t.a, 3), `lt`(test.t.b, 4))", "RowCount": 6 }, + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Range", + "Expr": "((b < 4))", + "RowCount": 6 + }, { "TableID": 57, "TableName": "", @@ -99,18 +176,25 @@ "Expr": "`or`(`lt`(test.t.a, 3), `lt`(test.t.b, 4))", "RowCount": 6 }, + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Range", + "Expr": "((b < 4))", + "RowCount": 6 + }, { "TableID": 57, "TableName": "", "Type": "Table Stats-Expression-CNF", - "Expr": "`lt`(test.t.a, 3)", + "Expr": "`lt`(test.t.b, 4)", "RowCount": 6 }, { "TableID": 57, "TableName": "", "Type": "Table Stats-Expression-CNF", - "Expr": "`lt`(test.t.a, 3)", + "Expr": "`lt`(test.t.b, 4)", "RowCount": 6 }, { @@ -120,18 +204,32 @@ "Expr": "`or`(`lt`(test.t.a, 3), `lt`(test.t.b, 4))", "RowCount": 6 }, + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Range", + "Expr": "((a < 3))", + "RowCount": 6 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Index Stats-Range", + "Expr": "((a < 3))", + "RowCount": 6 + }, { "TableID": 57, "TableName": "", "Type": "Table Stats-Expression-CNF", - "Expr": "`lt`(test.t.b, 4)", + "Expr": "`lt`(test.t.a, 3)", "RowCount": 6 }, { "TableID": 57, "TableName": "", "Type": "Table Stats-Expression-CNF", - "Expr": "`lt`(test.t.b, 4)", + "Expr": "`lt`(test.t.a, 3)", "RowCount": 6 }, { @@ -160,6 +258,34 @@ { "Expr": "a = 1 and b = 2", "Trace": [ + { + "TableID": 57, + "TableName": "", + "Type": "Index Stats-Point", + "Expr": "((a = 1) and (b = 2))", + "RowCount": 2 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Point", + "Expr": "((a = 1))", + "RowCount": 4 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Column Stats-Point", + "Expr": "((b = 2))", + "RowCount": 3 + }, + { + "TableID": 57, + "TableName": "", + "Type": "Index Stats-Point", + "Expr": "((a = 1) and (b = 2))", + "RowCount": 2 + }, { "TableID": 57, "TableName": "", diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index e238c85ee2314..854408f082528 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -1914,7 +1914,7 @@ func (w *GCWorker) doGCPlacementRules(dr util.DelRangeTask) (err error) { for _, id := range physicalTableIDs { bundles = append(bundles, placement.NewBundle(id)) } - return infosync.PutRuleBundles(context.TODO(), bundles) + return infosync.PutRuleBundlesWithDefaultRetry(context.TODO(), bundles) } func (w *GCWorker) doGCLabelRules(dr util.DelRangeTask) (err error) { diff --git a/store/mockstore/mockcopr/topn.go b/store/mockstore/mockcopr/topn.go index 68b0c498170e8..bf6833df4f48f 100644 --- a/store/mockstore/mockcopr/topn.go +++ b/store/mockstore/mockcopr/topn.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tipb/go-tipb" ) @@ -49,7 +50,7 @@ func (t *topNSorter) Less(i, j int) bool { v1 := t.rows[i].key[index] v2 := t.rows[j].key[index] - ret, err := v1.CompareDatum(t.sc, &v2) + ret, err := v1.Compare(t.sc, &v2, collate.GetCollator(collate.ProtoToCollation(by.Expr.FieldType.Collate))) if err != nil { t.err = errors.Trace(err) return true @@ -98,7 +99,7 @@ func (t *topNHeap) Less(i, j int) bool { v1 := t.rows[i].key[index] v2 := t.rows[j].key[index] - ret, err := v1.CompareDatum(t.sc, &v2) + ret, err := v1.Compare(t.sc, &v2, collate.GetCollator(collate.ProtoToCollation(by.Expr.FieldType.Collate))) if err != nil { t.err = errors.Trace(err) return true diff --git a/store/mockstore/unistore/cophandler/topn.go b/store/mockstore/unistore/cophandler/topn.go index f459ce16f2513..185317cbec1ee 100644 --- a/store/mockstore/unistore/cophandler/topn.go +++ b/store/mockstore/unistore/cophandler/topn.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" tipb "github.com/pingcap/tipb/go-tipb" ) @@ -51,7 +52,7 @@ func (t *topNSorter) Less(i, j int) bool { v1 := t.rows[i].key[index] v2 := t.rows[j].key[index] - ret, err := v1.CompareDatum(t.sc, &v2) + ret, err := v1.Compare(t.sc, &v2, collate.GetCollator(collate.ProtoToCollation(by.Expr.FieldType.Collate))) if err != nil { t.err = errors.Trace(err) return true @@ -105,7 +106,7 @@ func (t *topNHeap) Less(i, j int) bool { if expression.FieldTypeFromPB(by.GetExpr().GetFieldType()).Tp == mysql.TypeEnum { ret = types.CompareUint64(v1.GetUint64(), v2.GetUint64()) } else { - ret, err = v1.CompareDatum(t.sc, &v2) + ret, err = v1.Compare(t.sc, &v2, collate.GetCollator(collate.ProtoToCollation(by.Expr.FieldType.Collate))) if err != nil { t.err = errors.Trace(err) return true diff --git a/store/mockstore/unistore/tikv/region.go b/store/mockstore/unistore/tikv/region.go index 932841c968927..d854c910047a9 100644 --- a/store/mockstore/unistore/tikv/region.go +++ b/store/mockstore/unistore/tikv/region.go @@ -248,7 +248,13 @@ func (ri *regionCtx) AcquireLatches(hashVals []uint64) { dur := time.Since(start) metrics.LatchWait.Observe(dur.Seconds()) if dur > time.Millisecond*50 { - log.S().Warnf("region %d acquire %d locks takes %v, waitCnt %d", ri.meta.Id, len(hashVals), dur, waitCnt) + var id string + if ri.meta == nil { + id = "unknown" + } else { + id = strconv.FormatUint(ri.meta.Id, 10) + } + log.S().Warnf("region %s acquire %d locks takes %v, waitCnt %d", id, len(hashVals), dur, waitCnt) } } diff --git a/testkit/trequire/trequire.go b/testkit/trequire/trequire.go index c90a7c2339d16..6ffb245413fec 100644 --- a/testkit/trequire/trequire.go +++ b/testkit/trequire/trequire.go @@ -22,13 +22,14 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" ) -// DatumEqual verifies that the actual value is equal to the expected value. +// DatumEqual verifies that the actual value is equal to the expected value. For string datum, they are compared by the binary collation. func DatumEqual(t *testing.T, expected, actual types.Datum, msgAndArgs ...interface{}) { sc := new(stmtctx.StatementContext) - res, err := actual.CompareDatum(sc, &expected) + res, err := actual.Compare(sc, &expected, collate.GetBinaryCollator()) require.NoError(t, err, msgAndArgs) require.Zero(t, res, msgAndArgs) } diff --git a/tools/check/go.mod b/tools/check/go.mod index 0b4f5f590e3a6..9c9c2b8d3da55 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -1,11 +1,12 @@ module github.com/pingcap/tidb/_tools require ( + github.com/AlekSi/gocov-xml v0.0.0-20190121064608-3a14fb1c4737 github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc // indirect github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf // indirect + github.com/axw/gocov v1.0.0 github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03 github.com/dnephin/govet v0.0.0-20171012192244-4a96d43e39d3 - github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf // indirect github.com/kisielk/errcheck v1.2.0 github.com/mdempsky/unconvert v0.0.0-20200228143138-95ecdbfc0b5f // indirect github.com/mgechev/revive v0.0.0-20181210140514-b4cc152955fb @@ -19,6 +20,7 @@ require ( gopkg.in/alecthomas/gometalinter.v3 v3.0.0 // indirect gopkg.in/alecthomas/kingpin.v2 v2.2.6 // indirect gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20170321130658-9670b87a702e // indirect + gotest.tools/gotestsum v1.7.0 // indirect honnef.co/go/tools v0.0.0-20180920025451-e3ad64cb4ed3 ) diff --git a/tools/check/go.sum b/tools/check/go.sum index 604e8fe19c132..ca6214c124823 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -1,20 +1,36 @@ +github.com/AlekSi/gocov-xml v0.0.0-20190121064608-3a14fb1c4737 h1:JZHBkt0GhM+ARQykshqpI49yaWCHQbJonH3XpDTwMZQ= +github.com/AlekSi/gocov-xml v0.0.0-20190121064608-3a14fb1c4737/go.mod h1:w1KSuh2JgIL3nyRiZijboSUwbbxOrTzWwyWVFUHtXBQ= github.com/BurntSushi/toml v0.3.0 h1:e1/Ivsx3Z0FVTV0NSOv/aVgbUWyQuzj7DDnFblkRvsY= github.com/BurntSushi/toml v0.3.0/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf h1:qet1QNfXsQxTZqLG4oE62mJzwPIB8+Tee4RNCL9ulrY= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/axw/gocov v1.0.0 h1:YsqYR66hUmilVr23tu8USgnJIJvnwh3n7j5zRn7x4LU= +github.com/axw/gocov v1.0.0/go.mod h1:LvQpEYiwwIb2nYkXY2fDWhg9/AsYqkhmrCshjlUJECE= github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03 h1:0wUHjDfbCAROEAZ96zAJGwcNMkPIheFaIjtQyv3QqfM= github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03/go.mod h1:uFE9hX+zXEwvyUThZ4gDb9vkAwc5DoHUnRSEpH0VrOs= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dnephin/govet v0.0.0-20171012192244-4a96d43e39d3/go.mod h1:pPTX0MEEoAnfbrAGFj4nSVNhl6YbugRj6eardUZdtGo= +github.com/dnephin/pflag v1.0.7 h1:oxONGlWxhmUct0YzKTgrpQv9AUA1wtPBn7zuSjJqptk= +github.com/dnephin/pflag v1.0.7/go.mod h1:uxE91IoWURlOiTUIA8Mq5ZZkAv3dPUfZNaT80Zm7OQE= github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= +github.com/fatih/color v1.10.0 h1:s36xzo75JdqLaaWoiEHk767eHiwo0598uUxyfiPkDsg= +github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM= github.com/fatih/structtag v1.0.0 h1:pTHj65+u3RKWYPSGaU290FpI/dXxTaHdVwVwbcPKmEc= github.com/fatih/structtag v1.0.0/go.mod h1:IKitwq45uXL/yqi5mYghiD3w9H6eTOvI9vnk8tXMphA= +github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= +github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf h1:7+FW5aGwISbqUtkfmIpZJGRgNFg2ioYPvFaUxdqpDsg= github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= +github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 h1:El6M4kTTCOh6aBiKaUGG7oYTSPP8MxqL4YI3kZKwcP4= +github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510/go.mod h1:pupxD2MaaD3pAXIBCelhxNneeOaAeabZDe5s4K6zSpQ= +github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ= +github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= github.com/kisielk/errcheck v1.2.0 h1:reN85Pxc5larApoH1keMBiu2GWtPqXQ1nc9gx+jOU+E= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v0.0.0-20161130080628-0de1eaf82fa3 h1:s/sV9geKJwXXzcrFiQdiiIFgfesbREplXWR9ZFgnGSQ= @@ -24,8 +40,12 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/mattn/go-colorable v0.0.9 h1:UVL0vNpWh04HeJXV0KLcaT7r06gOH2l4OW6ddYRUIY4= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= +github.com/mattn/go-colorable v0.1.8 h1:c1ghPdyEDarC70ftn0y+A/Ee++9zz8ljHG1b13eJ0s8= +github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-isatty v0.0.4 h1:bnP0vzxcAdeI1zdubAl5PjU6zsERjGZb7raWodagDYs= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= +github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY= +github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-runewidth v0.0.3 h1:a+kO+98RDGEfo6asOGMmpodZq4FNtnGP54yps8BzLR4= github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mdempsky/unconvert v0.0.0-20200228143138-95ecdbfc0b5f h1:Kc3s6QFyh9DLgInXpWKuG+8I7R7lXbnP7mcoOVIt6KY= @@ -55,6 +75,9 @@ github.com/pingcap/errors v0.11.5-0.20211009033009-93128226aaa3/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce h1:Y1kCxlCtlPTMtVcOkjUcuQKh+YrluSo7+7YMCQSzy30= github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/ryanuber/go-glob v0.0.0-20170128012129-256dc444b735 h1:7YvPJVmEeFHR1Tj9sZEYsmarJEQfMVYpd/Vyy/A8dqE= github.com/ryanuber/go-glob v0.0.0-20170128012129-256dc444b735/go.mod h1:807d1WSdnB0XRJzKNil9Om6lcp/3a0v4qIHxIXzX/Yc= @@ -64,44 +87,72 @@ github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyV github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= +github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20170915142106-8351a756f30f/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 h1:SQFwaSi55rU7vdNs9Yr0Z324VNlrF+0wMqRXT4St8ck= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20171026204733-164713f0dfce/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 h1:SrN+KX8Art/Sf4HNj6Zcz06G7VEz+7w9tdXTPOZ7+l4= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1 h1:v+OssWQX+hTHEmOBgwxdZxK4zHq3yOs8F9J7mk0PY8E= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915090833-1cbadb444a80/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/tools v0.0.0-20170915040203-e531a2a1c15f/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180911133044-677d2ff680c1/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563 h1:NIou6eNFigscvKJmsbyez16S2cIS6idossORlFtSt2E= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190617190820-da514acc4774/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190624222133-a101b041ded4/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190925020647-22afafe3322a h1:3GxqzBPBt1O2dIiPnzldQ5d25CAMWJFBZTpqxLPfjs8= golang.org/x/tools v0.0.0-20190925020647-22afafe3322a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200225230052-807dcd883420 h1:4RJNOV+2rLxMEfr6QIpC7GEv9MjD6ApGXTCLrNF9+eA= golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.1.0 h1:po9/4sTYwZU9lPhi1tOrb4hCv3qrhiQ77LZfGa2OjwY= +golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gopkg.in/alecthomas/gometalinter.v2 v2.0.12 h1:/xBWwtjmOmVxn8FXfIk9noV8m2E2Id9jFfUY/Mh9QAI= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= gopkg.in/alecthomas/gometalinter.v3 v3.0.0 h1:tKnpLD70cWDacxrv9JZ4atld7RPoPiHOBfad6mPmyBw= @@ -114,4 +165,7 @@ gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gotest.tools/gotestsum v1.7.0 h1:RwpqwwFKBAa2h+F6pMEGpE707Edld0etUD3GhqqhDNc= +gotest.tools/gotestsum v1.7.0/go.mod h1:V1m4Jw3eBerhI/A6qCxUE07RnCg7ACkKj9BYcAm09V8= +gotest.tools/v3 v3.0.3/go.mod h1:Z7Lb0S5l+klDB31fvDQX8ss/FlKDxtlFlw3Oa8Ymbl8= honnef.co/go/tools v0.0.0-20180920025451-e3ad64cb4ed3/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/types/core_time.go b/types/core_time.go index 161180dd3b166..64820f98aaf79 100644 --- a/types/core_time.go +++ b/types/core_time.go @@ -184,6 +184,59 @@ func (t CoreTime) GoTime(loc *gotime.Location) (gotime.Time, error) { return tm, nil } +// FindZoneTransition check for one Time Zone transition within +/- 4h +// Currently the needed functions are not exported, if gotime.Location.lookup would be exported +// then it would be easy to use that directly +func FindZoneTransition(tIn gotime.Time) (gotime.Time, error) { + // Check most common case first, DST transition on full hour. + // round truncates away from zero! + t2 := tIn.Round(gotime.Hour).Add(-1 * gotime.Hour) + t1 := t2.Add(-1 * gotime.Second) + _, offset1 := t1.Zone() + _, offset2 := t2.Zone() + if offset1 != offset2 { + return t2, nil + } + + // Check if any offset change? + t1 = tIn.Add(-4 * gotime.Hour) + t2 = tIn.Add(4 * gotime.Hour) + _, offset1 = t1.Zone() + _, offset2 = t2.Zone() + if offset1 == offset2 { + return tIn, errors.Trace(ErrWrongValue.GenWithStackByArgs(TimeStr, tIn)) + } + + // Check generic case, like for 'Australia/Lord_Howe' + for t2.After(t1.Add(gotime.Second)) { + t := t1.Add(t2.Sub(t1) / 2).Round(gotime.Second) + _, offset := t.Zone() + if offset == offset1 { + t1 = t + } else { + t2 = t + } + } + return t2, nil +} + +// AdjustedGoTime converts Time to GoTime and adjust for invalid DST times +// like during the DST change with increased offset, +// normally moving to Daylight Saving Time. +// see https://github.com/pingcap/tidb/issues/28739 +func (t CoreTime) AdjustedGoTime(loc *gotime.Location) (gotime.Time, error) { + tm, err := t.GoTime(loc) + if err == nil { + return tm, nil + } + + tAdj, err2 := FindZoneTransition(tm) + if err2 == nil { + return tAdj, nil + } + return tm, err +} + // IsLeapYear returns if it's leap year. func (t CoreTime) IsLeapYear() bool { return isLeapYear(t.getYear()) diff --git a/types/core_time_test.go b/types/core_time_test.go index f34e38ab7f8a2..07e4d040a5c2c 100644 --- a/types/core_time_test.go +++ b/types/core_time_test.go @@ -316,3 +316,81 @@ func TestWeekday(t *testing.T) { require.Equal(t, tt.Expect, weekday.String()) } } + +func TestFindZoneTransition(t *testing.T) { + t.Parallel() + tests := []struct { + TZ string + dt string + Expect string + Success bool + }{ + {"Australia/Lord_Howe", "2020-06-29 03:45:00", "", false}, + {"Australia/Lord_Howe", "2020-10-04 02:15:00", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:29:59", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:29:59.99", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:30:00.0001", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:30:00", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:30:01", "2020-10-04 02:30:00 +11 +1100", true}, + {"Europe/Vilnius", "2020-03-29 03:45:00", "2020-03-29 04:00:00 EEST +0300", true}, + {"Europe/Vilnius", "2020-10-25 03:45:00", "2020-10-25 03:00:00 EET +0200", true}, + {"Europe/Vilnius", "2020-06-29 03:45:00", "", false}, + {"Europe/Amsterdam", "2020-03-29 02:45:00", "2020-03-29 03:00:00 CEST +0200", true}, + {"Europe/Amsterdam", "2020-10-25 02:35:00", "2020-10-25 02:00:00 CET +0100", true}, + {"Europe/Amsterdam", "2020-03-29 02:59:59", "2020-03-29 03:00:00 CEST +0200", true}, + {"Europe/Amsterdam", "2020-03-29 02:59:59.999999999", "2020-03-29 03:00:00 CEST +0200", true}, + {"Europe/Amsterdam", "2020-03-29 03:00:00.000000001", "2020-03-29 03:00:00 CEST +0200", true}, + } + + for _, tt := range tests { + loc, err := time.LoadLocation(tt.TZ) + require.NoError(t, err) + tm, err := time.ParseInLocation("2006-01-02 15:04:05", tt.dt, loc) + require.NoError(t, err) + tp, err := FindZoneTransition(tm) + if !tt.Success { + require.Error(t, err) + } else { + require.NoError(t, err) + require.Equal(t, tt.Expect, tp.Format("2006-01-02 15:04:05.999999999 MST -0700")) + } + } +} + +func TestAdjustedGoTime(t *testing.T) { + t.Parallel() + tests := []struct { + TZ string + dt CoreTime + Expect string + Success bool + }{ + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 01, 59, 59, 997), "2020-10-04 01:59:59.000997 +1030 +1030", true}, + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 02, 00, 00, 0), "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 02, 15, 00, 0), "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 02, 29, 59, 999999), "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 02, 30, 00, 1), "2020-10-04 02:30:00.000001 +11 +1100", true}, + {"Australia/Lord_Howe", FromDate(2020, 06, 29, 03, 45, 00, 0), "2020-06-29 03:45:00 +1030 +1030", true}, + {"Australia/Lord_Howe", FromDate(2020, 04, 04, 01, 45, 00, 0), "2020-04-04 01:45:00 +11 +1100", true}, + {"Europe/Vilnius", FromDate(2020, 03, 29, 03, 45, 00, 0), "2020-03-29 04:00:00 EEST +0300", true}, + {"Europe/Vilnius", FromDate(2020, 03, 29, 03, 59, 59, 456789), "2020-03-29 04:00:00 EEST +0300", true}, + {"Europe/Vilnius", FromDate(2020, 03, 29, 04, 00, 01, 130000), "2020-03-29 04:00:01.13 EEST +0300", true}, + {"Europe/Vilnius", FromDate(2020, 10, 25, 03, 45, 00, 0), "2020-10-25 03:45:00 EET +0200", true}, + {"Europe/Vilnius", FromDate(2020, 06, 29, 03, 45, 00, 0), "2020-06-29 03:45:00 EEST +0300", true}, + {"Europe/Amsterdam", FromDate(2020, 03, 29, 02, 45, 00, 0), "2020-03-29 03:00:00 CEST +0200", true}, + {"Europe/Amsterdam", FromDate(2020, 10, 25, 02, 35, 00, 0), "2020-10-25 02:35:00 CET +0100", true}, + {"UTC", FromDate(2020, 2, 31, 02, 35, 00, 0), "", false}, + } + + for _, tt := range tests { + loc, err := time.LoadLocation(tt.TZ) + require.NoError(t, err) + tp, err := tt.dt.AdjustedGoTime(loc) + if !tt.Success { + require.Error(t, err) + } else { + require.NoError(t, err) + require.Equal(t, tt.Expect, tp.Format("2006-01-02 15:04:05.999999999 MST -0700")) + } + } +} diff --git a/util/collate/collate.go b/util/collate/collate.go index e131134d37739..73bdeeaaa20d5 100644 --- a/util/collate/collate.go +++ b/util/collate/collate.go @@ -323,6 +323,36 @@ func IsBinCollation(collate string) bool { collate == charset.CollationUTF8 || collate == charset.CollationUTF8MB4 } +// CollationToProto converts collation from string to int32(used by protocol). +func CollationToProto(c string) int32 { + if coll, err := charset.GetCollationByName(c); err == nil { + return RewriteNewCollationIDIfNeeded(int32(coll.ID)) + } + v := RewriteNewCollationIDIfNeeded(int32(mysql.DefaultCollationID)) + logutil.BgLogger().Warn( + "Unable to get collation ID by name, use ID of the default collation instead", + zap.String("name", c), + zap.Int32("default collation ID", v), + zap.String("default collation", mysql.DefaultCollationName), + ) + return v +} + +// ProtoToCollation converts collation from int32(used by protocol) to string. +func ProtoToCollation(c int32) string { + coll, err := charset.GetCollationByID(int(RestoreCollationIDIfNeeded(c))) + if err == nil { + return coll.Name + } + logutil.BgLogger().Warn( + "Unable to get collation name from ID, use name of the default collation instead", + zap.Int32("id", c), + zap.Int("default collation ID", mysql.DefaultCollationID), + zap.String("default collation", mysql.DefaultCollationName), + ) + return mysql.DefaultCollationName +} + func init() { newCollatorMap = make(map[string]Collator) newCollatorIDMap = make(map[int]Collator) diff --git a/util/filesort/filesort.go b/util/filesort/filesort.go deleted file mode 100644 index db3ec718aa415..0000000000000 --- a/util/filesort/filesort.go +++ /dev/null @@ -1,625 +0,0 @@ -// Copyright 2017 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. - -package filesort - -import ( - "container/heap" - "encoding/binary" - "io" - "os" - "path/filepath" - "sort" - "strconv" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/codec" -) - -type comparableRow struct { - key []types.Datum - val []types.Datum - handle int64 -} - -type item struct { - index int // source file index - value *comparableRow -} - -// rowHeap maintains a min-heap property of comparableRows. -type rowHeap struct { - sc *stmtctx.StatementContext - ims []*item - byDesc []bool - err error -} - -var headSize = 8 - -func lessThan(sc *stmtctx.StatementContext, i []types.Datum, j []types.Datum, byDesc []bool) (bool, error) { - for k := range byDesc { - v1 := i[k] - v2 := j[k] - - ret, err := v1.CompareDatum(sc, &v2) - if err != nil { - return false, errors.Trace(err) - } - - if byDesc[k] { - ret = -ret - } - - if ret < 0 { - return true, nil - } else if ret > 0 { - return false, nil - } - } - return false, nil -} - -// Len implements heap.Interface Len interface. -func (rh *rowHeap) Len() int { return len(rh.ims) } - -// Swap implements heap.Interface Swap interface. -func (rh *rowHeap) Swap(i, j int) { rh.ims[i], rh.ims[j] = rh.ims[j], rh.ims[i] } - -// Less implements heap.Interface Less interface. -func (rh *rowHeap) Less(i, j int) bool { - l := rh.ims[i].value.key - r := rh.ims[j].value.key - ret, err := lessThan(rh.sc, l, r, rh.byDesc) - if rh.err == nil { - rh.err = err - } - return ret -} - -// Push pushes an element into rowHeap. -func (rh *rowHeap) Push(x interface{}) { - rh.ims = append(rh.ims, x.(*item)) -} - -// Pop pops the last element from rowHeap. -func (rh *rowHeap) Pop() interface{} { - old := rh.ims - n := len(old) - x := old[n-1] - rh.ims = old[0 : n-1] - return x -} - -// FileSorter sorts the given rows according to the byDesc order. -// FileSorter can sort rows that exceed predefined memory capacity. -type FileSorter struct { - sc *stmtctx.StatementContext - byDesc []bool - - workers []*Worker - nWorkers int // number of workers used in async sorting - cWorker int // the next worker to which the sorting job is sent - - mu sync.Mutex - tmpDir string - files []string - nFiles int - cursor int // required when performing full in-memory sort - - rowHeap *rowHeap - fds []*os.File - rowBytes []byte - head []byte - dcod []types.Datum - keySize int - valSize int - maxRowSize int - - wg sync.WaitGroup - closed bool - fetched bool - external bool // mark the necessity of performing external file sort -} - -// Worker sorts file asynchronously. -type Worker struct { - ctx *FileSorter - busy int32 - keySize int - valSize int - rowSize int - bufSize int - buf []*comparableRow - head []byte - err error -} - -// Builder builds a new FileSorter. -type Builder struct { - sc *stmtctx.StatementContext - keySize int - valSize int - bufSize int - nWorkers int - byDesc []bool - tmpDir string -} - -// SetSC sets StatementContext instance which is required in row comparison. -func (b *Builder) SetSC(sc *stmtctx.StatementContext) *Builder { - b.sc = sc - return b -} - -// SetSchema sets the schema of row, including key size and value size. -func (b *Builder) SetSchema(keySize, valSize int) *Builder { - b.keySize = keySize - b.valSize = valSize - return b -} - -// SetBuf sets the number of rows FileSorter can hold in memory at a time. -func (b *Builder) SetBuf(bufSize int) *Builder { - b.bufSize = bufSize - return b -} - -// SetWorkers sets the number of workers used in async sorting. -func (b *Builder) SetWorkers(nWorkers int) *Builder { - b.nWorkers = nWorkers - return b -} - -// SetDesc sets the ordering rule of row comparison. -func (b *Builder) SetDesc(byDesc []bool) *Builder { - b.byDesc = byDesc - return b -} - -// SetDir sets the working directory for FileSorter. -func (b *Builder) SetDir(tmpDir string) *Builder { - b.tmpDir = tmpDir - return b -} - -// Build creates a FileSorter instance using given data. -func (b *Builder) Build() (*FileSorter, error) { - // Sanity checks - if b.sc == nil { - return nil, errors.New("StatementContext is nil") - } - if b.keySize != len(b.byDesc) { - return nil, errors.New("mismatch in key size and byDesc slice") - } - if b.keySize <= 0 { - return nil, errors.New("key size is not positive") - } - if b.valSize <= 0 { - return nil, errors.New("value size is not positive") - } - if b.bufSize <= 0 { - return nil, errors.New("buffer size is not positive") - } - _, err := os.Stat(b.tmpDir) - if err != nil { - if os.IsNotExist(err) { - return nil, errors.New("tmpDir does not exist") - } - return nil, errors.Trace(err) - } - - ws := make([]*Worker, b.nWorkers) - for i := range ws { - ws[i] = &Worker{ - keySize: b.keySize, - valSize: b.valSize, - rowSize: b.keySize + b.valSize + 1, - bufSize: b.bufSize / b.nWorkers, - buf: make([]*comparableRow, 0, b.bufSize/b.nWorkers), - head: make([]byte, headSize), - } - } - - rh := &rowHeap{sc: b.sc, - ims: make([]*item, 0), - byDesc: b.byDesc, - } - - fs := &FileSorter{sc: b.sc, - workers: ws, - nWorkers: b.nWorkers, - cWorker: 0, - - head: make([]byte, headSize), - dcod: make([]types.Datum, 0, b.keySize+b.valSize+1), - keySize: b.keySize, - valSize: b.valSize, - - tmpDir: b.tmpDir, - files: make([]string, 0), - byDesc: b.byDesc, - rowHeap: rh, - } - - for i := 0; i < b.nWorkers; i++ { - fs.workers[i].ctx = fs - } - - return fs, nil -} - -func (fs *FileSorter) getUniqueFileName() string { - fs.mu.Lock() - defer fs.mu.Unlock() - ret := filepath.Join(fs.tmpDir, strconv.Itoa(fs.nFiles)) - fs.nFiles++ - return ret -} - -func (fs *FileSorter) appendFileName(fn string) { - fs.mu.Lock() - defer fs.mu.Unlock() - fs.files = append(fs.files, fn) -} - -func (fs *FileSorter) closeAllFiles() error { - var reportErr error - for _, fd := range fs.fds { - err := fd.Close() - if reportErr == nil { - reportErr = err - } - } - err := os.RemoveAll(fs.tmpDir) - if reportErr == nil { - reportErr = err - } - if reportErr != nil { - return errors.Trace(reportErr) - } - return nil -} - -// internalSort performs full in-memory sort. -func (fs *FileSorter) internalSort() (*comparableRow, error) { - w := fs.workers[fs.cWorker] - - if !fs.fetched { - sort.Sort(w) - if w.err != nil { - return nil, errors.Trace(w.err) - } - fs.fetched = true - } - if fs.cursor < len(w.buf) { - r := w.buf[fs.cursor] - fs.cursor++ - return r, nil - } - return nil, nil -} - -// externalSort performs external file sort. -func (fs *FileSorter) externalSort() (*comparableRow, error) { - if !fs.fetched { - // flush all remaining content to file (if any) - for _, w := range fs.workers { - if atomic.LoadInt32(&(w.busy)) == 0 && len(w.buf) > 0 { - fs.wg.Add(1) - go w.flushToFile() - } - } - - // wait for all workers to finish - fs.wg.Wait() - - // check errors from workers - for _, w := range fs.workers { - if w.err != nil { - return nil, errors.Trace(w.err) - } - if w.rowSize > fs.maxRowSize { - fs.maxRowSize = w.rowSize - } - } - - heap.Init(fs.rowHeap) - if fs.rowHeap.err != nil { - return nil, errors.Trace(fs.rowHeap.err) - } - - fs.rowBytes = make([]byte, fs.maxRowSize) - - err := fs.openAllFiles() - if err != nil { - return nil, errors.Trace(err) - } - - for id := range fs.fds { - row, err := fs.fetchNextRow(id) - if err != nil { - return nil, errors.Trace(err) - } - if row == nil { - return nil, errors.New("file is empty") - } - - im := &item{ - index: id, - value: row, - } - - heap.Push(fs.rowHeap, im) - if fs.rowHeap.err != nil { - return nil, errors.Trace(fs.rowHeap.err) - } - } - - fs.fetched = true - } - - if fs.rowHeap.Len() > 0 { - im := heap.Pop(fs.rowHeap).(*item) - if fs.rowHeap.err != nil { - return nil, errors.Trace(fs.rowHeap.err) - } - - row, err := fs.fetchNextRow(im.index) - if err != nil { - return nil, errors.Trace(err) - } - if row != nil { - nextIm := &item{ - index: im.index, - value: row, - } - - heap.Push(fs.rowHeap, nextIm) - if fs.rowHeap.err != nil { - return nil, errors.Trace(fs.rowHeap.err) - } - } - - return im.value, nil - } - - return nil, nil -} - -func (fs *FileSorter) openAllFiles() error { - for _, fname := range fs.files { - fd, err := os.Open(fname) - if err != nil { - return errors.Trace(err) - } - fs.fds = append(fs.fds, fd) - } - return nil -} - -// fetchNextRow fetches the next row given the source file index. -func (fs *FileSorter) fetchNextRow(index int) (*comparableRow, error) { - n, err := fs.fds[index].Read(fs.head) - if err == io.EOF { - return nil, nil - } - if err != nil { - return nil, errors.Trace(err) - } - if n != headSize { - return nil, errors.New("incorrect header") - } - rowSize := int(binary.BigEndian.Uint64(fs.head)) - - n, err = fs.fds[index].Read(fs.rowBytes) - if err != nil { - return nil, errors.Trace(err) - } - if n != rowSize { - return nil, errors.New("incorrect row") - } - - fs.dcod, err = codec.Decode(fs.rowBytes, fs.keySize+fs.valSize+1) - if err != nil { - return nil, errors.Trace(err) - } - - return &comparableRow{ - key: fs.dcod[:fs.keySize], - val: fs.dcod[fs.keySize : fs.keySize+fs.valSize], - handle: fs.dcod[fs.keySize+fs.valSize:][0].GetInt64(), - }, nil -} - -// Input adds one row into FileSorter. -// Caller should not call Input after calling Output. -func (fs *FileSorter) Input(key []types.Datum, val []types.Datum, handle int64) error { - if fs.closed { - return errors.New("FileSorter has been closed") - } - if fs.fetched { - return errors.New("call input after output") - } - - assigned := false - abortTime := time.Duration(1) * time.Minute // 1 minute - cooldownTime := time.Duration(100) * time.Millisecond // 100 milliseconds - row := &comparableRow{ - key: key, - val: val, - handle: handle, - } - - origin := time.Now() - // assign input row to some worker in a round-robin way - for { - for i := 0; i < fs.nWorkers; i++ { - wid := (fs.cWorker + i) % fs.nWorkers - if atomic.LoadInt32(&(fs.workers[wid].busy)) == 0 { - fs.workers[wid].input(row) - assigned = true - fs.cWorker = wid - break - } - } - if assigned { - break - } - - // all workers are busy now, cooldown and retry - time.Sleep(cooldownTime) - - if time.Since(origin) >= abortTime { - // weird: all workers are busy for at least 1 min - // choose to abort for safety - return errors.New("can not make progress since all workers are busy") - } - } - return nil -} - -// Output gets the next sorted row. -func (fs *FileSorter) Output() ([]types.Datum, []types.Datum, int64, error) { - var ( - r *comparableRow - err error - ) - if fs.closed { - return nil, nil, 0, errors.New("FileSorter has been closed") - } - - if fs.external { - r, err = fs.externalSort() - } else { - r, err = fs.internalSort() - } - - if err != nil { - return nil, nil, 0, errors.Trace(err) - } else if r != nil { - return r.key, r.val, r.handle, nil - } else { - return nil, nil, 0, nil - } -} - -// Close terminates the input or output process and discards all remaining data. -func (fs *FileSorter) Close() error { - if fs.closed { - return nil - } - fs.wg.Wait() - for _, w := range fs.workers { - w.buf = w.buf[:0] - } - fs.closed = true - err := fs.closeAllFiles() - if err != nil { - return errors.Trace(err) - } - return nil -} - -func (w *Worker) Len() int { return len(w.buf) } - -func (w *Worker) Swap(i, j int) { w.buf[i], w.buf[j] = w.buf[j], w.buf[i] } - -func (w *Worker) Less(i, j int) bool { - l := w.buf[i].key - r := w.buf[j].key - ret, err := lessThan(w.ctx.sc, l, r, w.ctx.byDesc) - if w.err == nil { - w.err = errors.Trace(err) - } - return ret -} - -func (w *Worker) input(row *comparableRow) { - w.buf = append(w.buf, row) - - if len(w.buf) > w.bufSize { - atomic.StoreInt32(&(w.busy), int32(1)) - w.ctx.wg.Add(1) - w.ctx.external = true - go w.flushToFile() - } -} - -// flushToFile flushes the buffer to file if it is full. -func (w *Worker) flushToFile() { - defer w.ctx.wg.Done() - var ( - outputByte []byte - prevLen int - ) - - sort.Sort(w) - if w.err != nil { - return - } - - fileName := w.ctx.getUniqueFileName() - - outputFile, err := os.OpenFile(fileName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) - if err != nil { - w.err = errors.Trace(err) - return - } - defer terror.Call(outputFile.Close) - sc := &stmtctx.StatementContext{TimeZone: time.Local} - for _, row := range w.buf { - prevLen = len(outputByte) - outputByte = append(outputByte, w.head...) - outputByte, err = codec.EncodeKey(sc, outputByte, row.key...) - if err != nil { - w.err = errors.Trace(err) - return - } - outputByte, err = codec.EncodeKey(sc, outputByte, row.val...) - if err != nil { - w.err = errors.Trace(err) - return - } - outputByte, err = codec.EncodeKey(sc, outputByte, types.NewIntDatum(row.handle)) - if err != nil { - w.err = errors.Trace(err) - return - } - - if len(outputByte)-prevLen-headSize > w.rowSize { - w.rowSize = len(outputByte) - prevLen - headSize - } - binary.BigEndian.PutUint64(w.head, uint64(len(outputByte)-prevLen-headSize)) - for i := 0; i < headSize; i++ { - outputByte[prevLen+i] = w.head[i] - } - } - - _, err = outputFile.Write(outputByte) - if err != nil { - w.err = errors.Trace(err) - return - } - - w.ctx.appendFileName(fileName) - w.buf = w.buf[:0] - atomic.StoreInt32(&(w.busy), int32(0)) -} diff --git a/util/filesort/filesort_test.go b/util/filesort/filesort_test.go deleted file mode 100644 index 43c7146b018d3..0000000000000 --- a/util/filesort/filesort_test.go +++ /dev/null @@ -1,392 +0,0 @@ -// Copyright 2017 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. - -package filesort - -import ( - "math/rand" - "os" - "testing" - "time" - - "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/types" - "github.com/stretchr/testify/require" -) - -func TestLessThan(t *testing.T) { - t.Parallel() - - sc := new(stmtctx.StatementContext) - - d0 := types.NewDatum(0) - d1 := types.NewDatum(1) - - tests := []struct { - i []types.Datum - j []types.Datum - byDesc []bool - ret bool - }{ - {[]types.Datum{d0}, []types.Datum{d0}, []bool{false}, false}, - {[]types.Datum{d0}, []types.Datum{d1}, []bool{false}, true}, - {[]types.Datum{d1}, []types.Datum{d0}, []bool{false}, false}, - {[]types.Datum{d0}, []types.Datum{d0}, []bool{true}, false}, - {[]types.Datum{d0}, []types.Datum{d1}, []bool{true}, false}, - {[]types.Datum{d1}, []types.Datum{d0}, []bool{true}, true}, - {[]types.Datum{d0, d0}, []types.Datum{d1, d1}, []bool{false, false}, true}, - {[]types.Datum{d0, d1}, []types.Datum{d1, d1}, []bool{false, false}, true}, - {[]types.Datum{d0, d0}, []types.Datum{d1, d1}, []bool{false, false}, true}, - {[]types.Datum{d0, d0}, []types.Datum{d0, d1}, []bool{false, false}, true}, - {[]types.Datum{d0, d1}, []types.Datum{d0, d1}, []bool{false, false}, false}, - {[]types.Datum{d0, d1}, []types.Datum{d0, d0}, []bool{false, false}, false}, - {[]types.Datum{d1, d0}, []types.Datum{d0, d1}, []bool{false, false}, false}, - {[]types.Datum{d1, d1}, []types.Datum{d0, d1}, []bool{false, false}, false}, - {[]types.Datum{d1, d1}, []types.Datum{d0, d0}, []bool{false, false}, false}, - } - - for _, test := range tests { - ret, err := lessThan(sc, test.i, test.j, test.byDesc) - require.NoError(t, err) - require.Equal(t, test.ret, ret) - } -} - -func TestInMemory(t *testing.T) { - t.Parallel() - - seed := rand.NewSource(time.Now().UnixNano()) - r := rand.New(seed) - - sc := new(stmtctx.StatementContext) - keySize := r.Intn(10) + 1 // random int in range [1, 10] - valSize := r.Intn(20) + 1 // random int in range [1, 20] - bufSize := 40 // hold up to 40 items per file - byDesc := make([]bool, keySize) - for i := range byDesc { - byDesc[i] = r.Intn(2) == 0 - } - - var ( - err error - fs *FileSorter - pkey []types.Datum - key []types.Datum - tmpDir string - ret bool - ) - - tmpDir, err = os.MkdirTemp("", "util_filesort_test") - require.NoError(t, err) - - fsBuilder := new(Builder) - fs, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(1).SetDesc(byDesc).SetDir(tmpDir).Build() - require.NoError(t, err) - defer func() { - err := fs.Close() - require.NoError(t, err) - }() - - nRows := r.Intn(bufSize-1) + 1 // random int in range [1, bufSize - 1] - for i := 1; i <= nRows; i++ { - err = fs.Input(nextRow(r, keySize, valSize)) - require.NoError(t, err) - } - - pkey, _, _, err = fs.Output() - require.NoError(t, err) - - for i := 1; i < nRows; i++ { - key, _, _, err = fs.Output() - require.NoError(t, err) - ret, err = lessThan(sc, key, pkey, byDesc) - require.NoError(t, err) - require.False(t, ret) - pkey = key - } -} - -func TestMultipleFiles(t *testing.T) { - t.Parallel() - - seed := rand.NewSource(time.Now().UnixNano()) - r := rand.New(seed) - - sc := new(stmtctx.StatementContext) - keySize := r.Intn(10) + 1 // random int in range [1, 10] - valSize := r.Intn(20) + 1 // random int in range [1, 20] - bufSize := 40 // hold up to 40 items per file - byDesc := make([]bool, keySize) - for i := range byDesc { - byDesc[i] = r.Intn(2) == 0 - } - - var ( - err error - fs *FileSorter - pkey []types.Datum - key []types.Datum - tmpDir string - ret bool - ) - - tmpDir, err = os.MkdirTemp("", "util_filesort_test") - require.NoError(t, err) - - fsBuilder := new(Builder) - - // Test for basic function. - _, err = fsBuilder.Build() - require.EqualError(t, err, "StatementContext is nil") - fsBuilder.SetSC(sc) - _, err = fsBuilder.Build() - require.EqualError(t, err, "key size is not positive") - fsBuilder.SetDesc(byDesc) - _, err = fsBuilder.Build() - require.EqualError(t, err, "mismatch in key size and byDesc slice") - fsBuilder.SetSchema(keySize, valSize) - _, err = fsBuilder.Build() - require.EqualError(t, err, "buffer size is not positive") - fsBuilder.SetBuf(bufSize) - _, err = fsBuilder.Build() - require.EqualError(t, err, "tmpDir does not exist") - fsBuilder.SetDir(tmpDir) - - fs, err = fsBuilder.SetWorkers(1).Build() - require.NoError(t, err) - defer func() { - err := fs.Close() - require.NoError(t, err) - }() - - nRows := (r.Intn(bufSize) + 1) * (r.Intn(10) + 2) - for i := 1; i <= nRows; i++ { - err = fs.Input(nextRow(r, keySize, valSize)) - require.NoError(t, err) - } - - pkey, _, _, err = fs.Output() - require.NoError(t, err) - for i := 1; i < nRows; i++ { - key, _, _, err = fs.Output() - require.NoError(t, err) - ret, err = lessThan(sc, key, pkey, byDesc) - require.NoError(t, err) - require.False(t, ret) - pkey = key - } -} - -func TestMultipleWorkers(t *testing.T) { - t.Parallel() - - seed := rand.NewSource(time.Now().UnixNano()) - r := rand.New(seed) - - sc := new(stmtctx.StatementContext) - keySize := r.Intn(10) + 1 // random int in range [1, 10] - valSize := r.Intn(20) + 1 // random int in range [1, 20] - bufSize := 40 // hold up to 40 items per file - byDesc := make([]bool, keySize) - for i := range byDesc { - byDesc[i] = r.Intn(2) == 0 - } - - var ( - err error - fs *FileSorter - pkey []types.Datum - key []types.Datum - tmpDir string - ret bool - ) - - tmpDir, err = os.MkdirTemp("", "util_filesort_test") - require.NoError(t, err) - - fsBuilder := new(Builder) - fs, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(4).SetDesc(byDesc).SetDir(tmpDir).Build() - require.NoError(t, err) - defer func() { - err := fs.Close() - require.NoError(t, err) - }() - - nRows := (r.Intn(bufSize) + 1) * (r.Intn(10) + 2) - for i := 1; i <= nRows; i++ { - err = fs.Input(nextRow(r, keySize, valSize)) - require.NoError(t, err) - } - - pkey, _, _, err = fs.Output() - require.NoError(t, err) - for i := 1; i < nRows; i++ { - key, _, _, err = fs.Output() - require.NoError(t, err) - ret, err = lessThan(sc, key, pkey, byDesc) - require.NoError(t, err) - require.False(t, ret) - pkey = key - } -} - -func TestClose(t *testing.T) { - t.Parallel() - - seed := rand.NewSource(time.Now().UnixNano()) - r := rand.New(seed) - - sc := new(stmtctx.StatementContext) - keySize := 2 - valSize := 2 - bufSize := 40 - byDesc := []bool{false, false} - - var ( - err error - fs0 *FileSorter - fs1 *FileSorter - tmpDir0 string - tmpDir1 string - errmsg = "FileSorter has been closed" - ) - - // Prepare two FileSorter instances for tests - fsBuilder := new(Builder) - tmpDir0, err = os.MkdirTemp("", "util_filesort_test") - require.NoError(t, err) - fs0, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(1).SetDesc(byDesc).SetDir(tmpDir0).Build() - require.NoError(t, err) - defer func() { - err := fs0.Close() - require.NoError(t, err) - }() - - tmpDir1, err = os.MkdirTemp("", "util_filesort_test") - require.NoError(t, err) - fs1, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(1).SetDesc(byDesc).SetDir(tmpDir1).Build() - require.NoError(t, err) - defer func() { - err := fs1.Close() - require.NoError(t, err) - }() - - // 1. Close after some Input - err = fs0.Input(nextRow(r, keySize, valSize)) - require.NoError(t, err) - - err = fs0.Close() - require.NoError(t, err) - - _, err = os.Stat(tmpDir0) - require.True(t, os.IsNotExist(err)) - - _, _, _, err = fs0.Output() - require.EqualError(t, err, errmsg) - - err = fs0.Input(nextRow(r, keySize, valSize)) - require.EqualError(t, err, errmsg) - - err = fs0.Close() - require.NoError(t, err) - - // 2. Close after some Output - err = fs1.Input(nextRow(r, keySize, valSize)) - require.NoError(t, err) - err = fs1.Input(nextRow(r, keySize, valSize)) - require.NoError(t, err) - - _, _, _, err = fs1.Output() - require.NoError(t, err) - - err = fs1.Close() - require.NoError(t, err) - - _, err = os.Stat(tmpDir1) - require.True(t, os.IsNotExist(err)) - - _, _, _, err = fs1.Output() - require.EqualError(t, err, errmsg) - - err = fs1.Input(nextRow(r, keySize, valSize)) - require.EqualError(t, err, errmsg) - - err = fs1.Close() - require.NoError(t, err) -} - -func TestMismatchedUsage(t *testing.T) { - t.Parallel() - - seed := rand.NewSource(time.Now().UnixNano()) - r := rand.New(seed) - - sc := new(stmtctx.StatementContext) - keySize := 2 - valSize := 2 - bufSize := 40 - byDesc := []bool{false, false} - - var ( - err error - fs0 *FileSorter - fs1 *FileSorter - key []types.Datum - tmpDir string - errmsg = "call input after output" - ) - - // Prepare two FileSorter instances for tests - fsBuilder := new(Builder) - tmpDir, err = os.MkdirTemp("", "util_filesort_test") - require.NoError(t, err) - fs0, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(1).SetDesc(byDesc).SetDir(tmpDir).Build() - require.NoError(t, err) - defer func() { - err := fs0.Close() - require.NoError(t, err) - }() - - tmpDir, err = os.MkdirTemp("", "util_filesort_test") - require.NoError(t, err) - fs1, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(1).SetDesc(byDesc).SetDir(tmpDir).Build() - require.NoError(t, err) - defer func() { - err := fs1.Close() - require.NoError(t, err) - }() - - // 1. call Output after fetched all rows - err = fs0.Input(nextRow(r, keySize, valSize)) - require.NoError(t, err) - - key, _, _, err = fs0.Output() - require.NoError(t, err) - require.NotNil(t, key) - - key, _, _, err = fs0.Output() - require.NoError(t, err) - require.Nil(t, key) - - // 2. call Input after Output - err = fs1.Input(nextRow(r, keySize, valSize)) - require.NoError(t, err) - - key, _, _, err = fs1.Output() - require.NoError(t, err) - require.NotNil(t, key) - - err = fs1.Input(nextRow(r, keySize, valSize)) - require.EqualError(t, err, errmsg) -} diff --git a/util/logutil/log_test.go b/util/logutil/log_test.go index d4f671f05dc92..69eb731dee8c6 100644 --- a/util/logutil/log_test.go +++ b/util/logutil/log_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/log" "github.com/stretchr/testify/require" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) func TestZapLoggerWithKeys(t *testing.T) { @@ -115,13 +116,31 @@ func TestGrpcLoggerCreation(t *testing.T) { } func TestSlowQueryLoggerCreation(t *testing.T) { - level := "warn" + level := "Error" conf := NewLogConfig(level, DefaultLogFormat, "", EmptyFileLogConfig, false) _, prop, err := newSlowQueryLogger(conf) // assert after init slow query logger, the original conf is not changed require.Equal(t, conf.Level, level) require.Nil(t, err) - require.Equal(t, prop.Level.String(), conf.Level) + // slow query logger doesn't use the level of the global log config, and the + // level should be less than WarnLevel which is used by it to log slow query. + require.NotEqual(t, conf.Level, prop.Level.String()) + require.True(t, prop.Level.Level() <= zapcore.WarnLevel) + + level = "warn" + slowQueryFn := "slow-query.log" + fileConf := FileLogConfig{ + log.FileLogConfig{ + Filename: slowQueryFn, + MaxSize: 10, + MaxDays: 10, + MaxBackups: 10, + }, + } + conf = NewLogConfig(level, DefaultLogFormat, slowQueryFn, fileConf, false) + slowQueryConf := newSlowQueryLogConfig(conf) + // slowQueryConf.MaxDays/MaxSize/MaxBackups should be same with global config. + require.Equal(t, fileConf.FileLogConfig, slowQueryConf.File) } func TestGlobalLoggerReplace(t *testing.T) { diff --git a/util/logutil/slow_query_logger.go b/util/logutil/slow_query_logger.go index 2588c36131fd9..d5d59f98aface 100644 --- a/util/logutil/slow_query_logger.go +++ b/util/logutil/slow_query_logger.go @@ -28,17 +28,8 @@ import ( var _pool = buffer.NewPool() func newSlowQueryLogger(cfg *LogConfig) (*zap.Logger, *log.ZapProperties, error) { - - // copy the global log config to slow log config - // if the filename of slow log config is empty, slow log will behave the same as global log. - sqConfig := cfg.Config - if len(cfg.SlowQueryFile) != 0 { - sqConfig.File = cfg.File - sqConfig.File.Filename = cfg.SlowQueryFile - } - // create the slow query logger - sqLogger, prop, err := log.InitLogger(&sqConfig) + sqLogger, prop, err := log.InitLogger(newSlowQueryLogConfig(cfg)) if err != nil { return nil, nil, errors.Trace(err) } @@ -53,6 +44,20 @@ func newSlowQueryLogger(cfg *LogConfig) (*zap.Logger, *log.ZapProperties, error) return sqLogger, prop, nil } +func newSlowQueryLogConfig(cfg *LogConfig) *log.Config { + // copy the global log config to slow log config + // if the filename of slow log config is empty, slow log will behave the same as global log. + sqConfig := cfg.Config + // level of the global log config doesn't affect the slow query logger which determines whether to + // log by execution duration. + sqConfig.Level = LogConfig{}.Level + if len(cfg.SlowQueryFile) != 0 { + sqConfig.File = cfg.File + sqConfig.File.Filename = cfg.SlowQueryFile + } + return &sqConfig +} + type slowLogEncoder struct{} func (e *slowLogEncoder) EncodeEntry(entry zapcore.Entry, _ []zapcore.Field) (*buffer.Buffer, error) { diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 7b422d10243cc..d7ac4c3efbe85 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -448,7 +448,8 @@ func allSinglePoints(sc *stmtctx.StatementContext, points []*point) []*point { if !left.start || right.start || left.excl || right.excl { return nil } - cmp, err := left.value.CompareDatum(sc, &right.value) + // Since the point's collations are equal to the column's collation, we can use any of them. + cmp, err := left.value.Compare(sc, &right.value, collate.GetCollator(left.value.Collation())) if err != nil || cmp != 0 { return nil } diff --git a/util/ranger/points.go b/util/ranger/points.go index a02f77cc08909..d6f88268a2f17 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -467,7 +467,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val } d := types.NewCollateMysqlEnumDatum(tmpEnum, ft.Collate) - if v, err := d.CompareDatum(sc, &val); err == nil { + if v, err := d.Compare(sc, &val, collate.GetCollator(ft.Collate)); err == nil { switch op { case ast.LT: if v < 0 { diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 0e39a228ca9a3..d95517fdcc50e 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -17,6 +17,7 @@ package ranger import ( "bytes" "math" + "regexp" "sort" "unicode/utf8" @@ -25,11 +26,15 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" + "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" ) func validInterval(sctx sessionctx.Context, low, high *point) (bool, error) { @@ -117,7 +122,7 @@ func convertPoint(sctx sessionctx.Context, point *point, tp *types.FieldType) (* return point, errors.Trace(err) } } - valCmpCasted, err := point.value.CompareDatum(sc, &casted) + valCmpCasted, err := point.value.Compare(sc, &casted, collate.GetCollator(tp.Collate)) if err != nil { return point, errors.Trace(err) } @@ -596,3 +601,154 @@ func DetachCondAndBuildRangeForPartition(sctx sessionctx.Context, conditions []e } return d.detachCondAndBuildRangeForCols() } + +// RangesToString print a list of Ranges into a string which can appear in an SQL as a condition. +func RangesToString(sc *stmtctx.StatementContext, rans []*Range, colNames []string) (string, error) { + for _, ran := range rans { + if len(ran.LowVal) != len(ran.HighVal) { + return "", errors.New("range length mismatch") + } + } + var buffer bytes.Buffer + for i, ran := range rans { + buffer.WriteString("(") + for j := range ran.LowVal { + buffer.WriteString("(") + + // The `Exclude` information is only useful for the last columns. + // If it's not the last column, it should always be false, which means it's inclusive. + lowExclude := false + if ran.LowExclude && j == len(ran.LowVal)-1 { + lowExclude = true + } + highExclude := false + if ran.HighExclude && j == len(ran.LowVal)-1 { + highExclude = true + } + + // sanity check: only last column of the `Range` can be an interval + if j < len(ran.LowVal)-1 { + cmp, err := ran.LowVal[j].CompareDatum(sc, &ran.HighVal[j]) + if err != nil { + return "", errors.New("comparing values error: " + err.Error()) + } + if cmp != 0 { + return "", errors.New("unexpected form of range") + } + } + + str, err := RangeSingleColToString(sc, ran.LowVal[j], ran.HighVal[j], lowExclude, highExclude, colNames[j]) + if err != nil { + return "false", err + } + buffer.WriteString(str) + buffer.WriteString(")") + if j < len(ran.LowVal)-1 { + // Conditions on different columns of a range are implicitly connected with AND. + buffer.WriteString(" and ") + } + } + buffer.WriteString(")") + if i < len(rans)-1 { + // Conditions of different ranges are implicitly connected with OR. + buffer.WriteString(" or ") + } + } + result := buffer.String() + + // Simplify some useless conditions. + if matched, err := regexp.MatchString(`^\(*true\)*$`, result); matched || (err != nil) { + return "true", nil + } + return result, nil +} + +// RangeSingleColToString prints a single column of a Range into a string which can appear in an SQL as a condition. +func RangeSingleColToString(sc *stmtctx.StatementContext, lowVal, highVal types.Datum, lowExclude, highExclude bool, colName string) (string, error) { + // case 1: low and high are both special values(null, min not null, max value) + lowKind := lowVal.Kind() + highKind := highVal.Kind() + if (lowKind == types.KindNull || lowKind == types.KindMinNotNull || lowKind == types.KindMaxValue) && + (highKind == types.KindNull || highKind == types.KindMinNotNull || highKind == types.KindMaxValue) { + if lowKind == types.KindNull && highKind == types.KindNull && !lowExclude && !highExclude { + return colName + " is null", nil + } + if lowKind == types.KindNull && highKind == types.KindMaxValue && !lowExclude { + return "true", nil + } + if lowKind == types.KindMinNotNull && highKind == types.KindMaxValue { + return colName + " is not null", nil + } + return "false", nil + } + + var buf bytes.Buffer + restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &buf) + + // case 2: low value and high value are the same, and low value and high value are both inclusive. + cmp, err := lowVal.CompareDatum(sc, &highVal) + if err != nil { + return "false", errors.Trace(err) + } + if cmp == 0 && !lowExclude && !highExclude && !lowVal.IsNull() { + buf.WriteString(colName) + buf.WriteString(" = ") + lowValExpr := driver.ValueExpr{Datum: lowVal} + err := lowValExpr.Restore(restoreCtx) + if err != nil { + return "false", errors.Trace(err) + } + return buf.String(), nil + } + + // case 3: it's an interval. + useOR := false + noLowerPart := false + + // Handle the low value part. + if lowKind == types.KindNull { + buf.WriteString(colName + " is null") + useOR = true + } else if lowKind == types.KindMinNotNull { + noLowerPart = true + } else { + buf.WriteString(colName) + if lowExclude { + buf.WriteString(" > ") + } else { + buf.WriteString(" >= ") + } + lowValExpr := driver.ValueExpr{Datum: lowVal} + err := lowValExpr.Restore(restoreCtx) + if err != nil { + return "false", errors.Trace(err) + } + } + + if !noLowerPart { + if useOR { + buf.WriteString(" or ") + } else { + buf.WriteString(" and ") + } + } + + // Handle the high value part + if highKind == types.KindMaxValue { + buf.WriteString("true") + } else { + buf.WriteString(colName) + if highExclude { + buf.WriteString(" < ") + } else { + buf.WriteString(" <= ") + } + highValExpr := driver.ValueExpr{Datum: highVal} + err := highValExpr.Restore(restoreCtx) + if err != nil { + return "false", errors.Trace(err) + } + } + + return buf.String(), nil +} diff --git a/util/ranger/types.go b/util/ranger/types.go index c950419cf217f..2e8cc1dc6120d 100644 --- a/util/ranger/types.go +++ b/util/ranger/types.go @@ -82,10 +82,10 @@ func (ran *Range) Clone() *Range { // IsPoint returns if the range is a point. func (ran *Range) IsPoint(sctx sessionctx.Context) bool { - return ran.isPoint(sctx, sctx.GetSessionVars().RegardNULLAsPoint) + return ran.isPoint(sctx.GetSessionVars().StmtCtx, sctx.GetSessionVars().RegardNULLAsPoint) } -func (ran *Range) isPoint(sctx sessionctx.Context, regardNullAsPoint bool) bool { +func (ran *Range) isPoint(stmtCtx *stmtctx.StatementContext, regardNullAsPoint bool) bool { if len(ran.LowVal) != len(ran.HighVal) { return false } @@ -95,7 +95,7 @@ func (ran *Range) isPoint(sctx sessionctx.Context, regardNullAsPoint bool) bool if a.Kind() == types.KindMinNotNull || b.Kind() == types.KindMaxValue { return false } - cmp, err := a.CompareDatum(sctx.GetSessionVars().StmtCtx, &b) + cmp, err := a.CompareDatum(stmtCtx, &b) if err != nil { return false } @@ -114,12 +114,13 @@ func (ran *Range) isPoint(sctx sessionctx.Context, regardNullAsPoint bool) bool // IsPointNonNullable returns if the range is a point without NULL. func (ran *Range) IsPointNonNullable(sctx sessionctx.Context) bool { - return ran.isPoint(sctx, false) + return ran.isPoint(sctx.GetSessionVars().StmtCtx, false) } // IsPointNullable returns if the range is a point. -func (ran *Range) IsPointNullable(sctx sessionctx.Context) bool { - return ran.isPoint(sctx, true) +// TODO: unify the parameter type with IsPointNullable and IsPoint +func (ran *Range) IsPointNullable(stmtCtx *stmtctx.StatementContext) bool { + return ran.isPoint(stmtCtx, true) } // IsFullRange check if the range is full scan range