Skip to content

Commit

Permalink
Delete pkg/parser/mysql/errcode.go and pkg/parser/mysql/errname.go an…
Browse files Browse the repository at this point in the history
…d use error codes and messages in pkg/errno.
  • Loading branch information
wddevries committed Jan 8, 2025
1 parent 76267c3 commit a183b38
Show file tree
Hide file tree
Showing 48 changed files with 1,638 additions and 3,576 deletions.
9 changes: 5 additions & 4 deletions pkg/ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
Expand Down Expand Up @@ -175,10 +176,10 @@ var mockTerrorMap = make(map[string]*terror.Error)

func init() {
// Since terror new action will cause data race with other test suite (getTerrorCode) in parallel, we init it all here.
mockTerrorMap[model.StateNone.String()] = dbterror.ClassDDL.NewStdErr(1, mysql.Message("MockRollingBackInCallBack-"+model.StateNone.String(), nil))
mockTerrorMap[model.StateDeleteOnly.String()] = dbterror.ClassDDL.NewStdErr(1, mysql.Message("MockRollingBackInCallBack-"+model.StateDeleteOnly.String(), nil))
mockTerrorMap[model.StateWriteOnly.String()] = dbterror.ClassDDL.NewStdErr(1, mysql.Message("MockRollingBackInCallBack-"+model.StateWriteOnly.String(), nil))
mockTerrorMap[model.StateWriteReorganization.String()] = dbterror.ClassDDL.NewStdErr(1, mysql.Message("MockRollingBackInCallBack-"+model.StateWriteReorganization.String(), nil))
mockTerrorMap[model.StateNone.String()] = dbterror.ClassDDL.NewStdErr(1, errno.Message("MockRollingBackInCallBack-"+model.StateNone.String(), nil))
mockTerrorMap[model.StateDeleteOnly.String()] = dbterror.ClassDDL.NewStdErr(1, errno.Message("MockRollingBackInCallBack-"+model.StateDeleteOnly.String(), nil))
mockTerrorMap[model.StateWriteOnly.String()] = dbterror.ClassDDL.NewStdErr(1, errno.Message("MockRollingBackInCallBack-"+model.StateWriteOnly.String(), nil))
mockTerrorMap[model.StateWriteReorganization.String()] = dbterror.ClassDDL.NewStdErr(1, errno.Message("MockRollingBackInCallBack-"+model.StateWriteReorganization.String(), nil))
}

// Test issue #20529.
Expand Down
2 changes: 1 addition & 1 deletion pkg/domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -3399,7 +3399,7 @@ var (
ErrInfoSchemaExpired = dbterror.ClassDomain.NewStd(errno.ErrInfoSchemaExpired)
// ErrInfoSchemaChanged returns the error that information schema is changed.
ErrInfoSchemaChanged = dbterror.ClassDomain.NewStdErr(errno.ErrInfoSchemaChanged,
mysql.Message(errno.MySQLErrName[errno.ErrInfoSchemaChanged].Raw+". "+kv.TxnRetryableMark, nil))
errno.Message(errno.MySQLErrName[errno.ErrInfoSchemaChanged].Raw+". "+kv.TxnRetryableMark, nil))
)

// SysProcesses holds the sys processes infos
Expand Down
1 change: 0 additions & 1 deletion pkg/errno/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@ go_library(
],
importpath = "github.com/pingcap/tidb/pkg/errno",
visibility = ["//visibility:public"],
deps = ["//pkg/parser/mysql"],
)

go_test(
Expand Down
2,421 changes: 1,215 additions & 1,206 deletions pkg/errno/errname.go

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion pkg/executor/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -565,7 +565,7 @@ func (e *InsertExec) setMessage() {
numDuplicates = stmtCtx.UpdatedRows()
}
}
msg := fmt.Sprintf(mysql.MySQLErrName[mysql.ErrInsertInfo].Raw, numRecords, numDuplicates, numWarnings)
msg := fmt.Sprintf(errno.MySQLErrName[errno.ErrInsertInfo].Raw, numRecords, numDuplicates, numWarnings)
stmtCtx.SetMessage(msg)
}
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/executor/load_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/br/pkg/storage"
"github.com/pingcap/tidb/pkg/errctx"
"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/executor/importer"
"github.com/pingcap/tidb/pkg/executor/internal/exec"
"github.com/pingcap/tidb/pkg/expression"
Expand Down Expand Up @@ -276,7 +277,7 @@ func (e *LoadDataWorker) setResult(colAssignExprWarnings []contextutil.SQLWarn)
numWarnings = math.MaxUint16
}

msg := fmt.Sprintf(mysql.MySQLErrName[mysql.ErrLoadInfo].Raw, numRecords, numDeletes, numSkipped, numWarnings)
msg := fmt.Sprintf(errno.MySQLErrName[errno.ErrLoadInfo].Raw, numRecords, numDeletes, numSkipped, numWarnings)
warns := make([]contextutil.SQLWarn, numWarnings)
n := copy(warns, stmtCtx.GetWarnings())
for i := 0; i < int(numRecords) && n < len(warns); i++ {
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/replace.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,10 @@ import (
"runtime/trace"
"time"

"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/executor/internal/exec"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/autoid"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/table/tables"
"github.com/pingcap/tidb/pkg/tablecodec"
Expand Down Expand Up @@ -213,7 +213,7 @@ func (e *ReplaceExec) setMessage() {
if e.SelectExec != nil || numRecords > 1 {
numWarnings := stmtCtx.WarningCount()
numDuplicates := stmtCtx.AffectedRows() - numRecords
msg := fmt.Sprintf(mysql.MySQLErrName[mysql.ErrInsertInfo].Raw, numRecords, numDuplicates, numWarnings)
msg := fmt.Sprintf(errno.MySQLErrName[errno.ErrInsertInfo].Raw, numRecords, numDuplicates, numWarnings)
stmtCtx.SetMessage(msg)
}
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/pkg/disttask/importinto"
"github.com/pingcap/tidb/pkg/domain"
"github.com/pingcap/tidb/pkg/domain/infosync"
"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/executor/importer"
"github.com/pingcap/tidb/pkg/executor/internal/exec"
"github.com/pingcap/tidb/pkg/expression"
Expand Down Expand Up @@ -1986,7 +1987,7 @@ func (e *ShowExec) fetchShowWarnings(errOnly bool) error {
if warn != nil {
err = warn.Error()
}
e.appendRow([]any{w.Level, int64(mysql.ErrUnknown), err})
e.appendRow([]any{w.Level, int64(errno.ErrUnknown), err})
}
}
return nil
Expand Down
3 changes: 2 additions & 1 deletion pkg/executor/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"runtime/trace"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/executor/internal/exec"
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/kv"
Expand Down Expand Up @@ -566,7 +567,7 @@ func (e *UpdateExec) setMessage() {
numMatched := e.matched
numChanged := stmtCtx.UpdatedRows()
numWarnings := stmtCtx.WarningCount()
msg := fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUpdateInfo].Raw, numMatched, numChanged, numWarnings)
msg := fmt.Sprintf(errno.MySQLErrName[errno.ErrUpdateInfo].Raw, numMatched, numChanged, numWarnings)
stmtCtx.SetMessage(msg)
}

Expand Down
5 changes: 3 additions & 2 deletions pkg/expression/builtin_miscellaneous.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (

"github.com/google/uuid"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/expression/expropt"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
Expand Down Expand Up @@ -251,9 +252,9 @@ func (b *builtinLockSig) evalInt(ctx EvalContext, row chunk.Row) (int64, bool, e
if err != nil {
if terr, ok := errors.Cause(err).(*terror.Error); ok {
switch terr.Code() {
case mysql.ErrLockWaitTimeout:
case errno.ErrLockWaitTimeout:
return 0, false, nil // Another user has the lock
case mysql.ErrLockDeadlock:
case errno.ErrLockDeadlock:
// Currently this code is not reachable because each Advisory Lock
// Uses a separate session. Deadlock detection does not work across
// independent sessions.
Expand Down
7 changes: 3 additions & 4 deletions pkg/expression/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ package expression
import (
"github.com/pingcap/errors"
mysql "github.com/pingcap/tidb/pkg/errno"
pmysql "github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/dbterror"
)
Expand All @@ -30,7 +29,7 @@ var (
ErrRegexp = dbterror.ClassExpression.NewStd(mysql.ErrRegexp)
ErrOperandColumns = dbterror.ClassExpression.NewStd(mysql.ErrOperandColumns)
ErrCutValueGroupConcat = dbterror.ClassExpression.NewStd(mysql.ErrCutValueGroupConcat)
ErrFunctionsNoopImpl = dbterror.ClassExpression.NewStdErr(mysql.ErrNotSupportedYet, pmysql.Message("function %s has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions", nil))
ErrFunctionsNoopImpl = dbterror.ClassExpression.NewStdErr(mysql.ErrNotSupportedYet, mysql.Message("function %s has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions", nil))
ErrInvalidArgumentForLogarithm = dbterror.ClassExpression.NewStd(mysql.ErrInvalidArgumentForLogarithm)
ErrIncorrectType = dbterror.ClassExpression.NewStd(mysql.ErrIncorrectType)
ErrInvalidTypeForJSON = dbterror.ClassExpression.NewStd(mysql.ErrInvalidTypeForJSON)
Expand All @@ -46,7 +45,7 @@ var (
errZlibZBuf = dbterror.ClassExpression.NewStd(mysql.ErrZlibZBuf)
errIncorrectArgs = dbterror.ClassExpression.NewStd(mysql.ErrWrongArguments)
errUnknownCharacterSet = dbterror.ClassExpression.NewStd(mysql.ErrUnknownCharacterSet)
errDefaultValue = dbterror.ClassExpression.NewStdErr(mysql.ErrInvalidDefault, pmysql.Message("invalid default value", nil))
errDefaultValue = dbterror.ClassExpression.NewStdErr(mysql.ErrInvalidDefault, mysql.Message("invalid default value", nil))
errDeprecatedSyntaxNoReplacement = dbterror.ClassExpression.NewStd(mysql.ErrWarnDeprecatedSyntaxNoReplacement)
errWarnAllowedPacketOverflowed = dbterror.ClassExpression.NewStd(mysql.ErrWarnAllowedPacketOverflowed)
errWarnOptionIgnored = dbterror.ClassExpression.NewStd(mysql.WarnOptionIgnored)
Expand All @@ -64,7 +63,7 @@ var (
// Sequence usage privilege check.
errSequenceAccessDenied = dbterror.ClassExpression.NewStd(mysql.ErrTableaccessDenied)
errUnsupportedJSONComparison = dbterror.ClassExpression.NewStdErr(mysql.ErrNotSupportedYet,
pmysql.Message("comparison of JSON in the LEAST and GREATEST operators", nil))
mysql.Message("comparison of JSON in the LEAST and GREATEST operators", nil))
)

// handleInvalidTimeError reports error or warning depend on the context.
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ go_library(
"//pkg/errno",
"//pkg/meta/model",
"//pkg/parser",
"//pkg/parser/mysql",
"//pkg/parser/terror",
"//pkg/resourcegroup",
"//pkg/types",
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@ package kv
import (
"strings"

"github.com/pingcap/tidb/pkg/errno"
mysql "github.com/pingcap/tidb/pkg/errno"
pmysql "github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/util/dbterror"
)

Expand All @@ -34,7 +34,7 @@ var (
// subject to change it in the future.
ErrTxnRetryable = dbterror.ClassKV.NewStdErr(
mysql.ErrTxnRetryable,
pmysql.Message(
errno.Message(
mysql.MySQLErrName[mysql.ErrTxnRetryable].Raw+TxnRetryableMark,
mysql.MySQLErrName[mysql.ErrTxnRetryable].RedactArgPos,
),
Expand All @@ -57,15 +57,15 @@ var (
// ErrWriteConflict is the error when the commit meets an write conflict error.
ErrWriteConflict = dbterror.ClassKV.NewStdErr(
mysql.ErrWriteConflict,
pmysql.Message(
errno.Message(
mysql.MySQLErrName[mysql.ErrWriteConflict].Raw+" "+TxnRetryableMark,
mysql.MySQLErrName[mysql.ErrWriteConflict].RedactArgPos,
),
)
// ErrWriteConflictInTiDB is the error when the commit meets an write conflict error when local latch is enabled.
ErrWriteConflictInTiDB = dbterror.ClassKV.NewStdErr(
mysql.ErrWriteConflictInTiDB,
pmysql.Message(
errno.Message(
mysql.MySQLErrName[mysql.ErrWriteConflictInTiDB].Raw+" "+TxnRetryableMark,
mysql.MySQLErrName[mysql.ErrWriteConflictInTiDB].RedactArgPos,
),
Expand Down
4 changes: 2 additions & 2 deletions pkg/lightning/common/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ func tryConnectMySQL(cfg *mysql.Config) (*sql.DB, error) {
failpoint.Inject("MustMySQLPassword", func(val failpoint.Value) {
pwd := val.(string)
if cfg.Passwd != pwd {
failpoint.Return(nil, &mysql.MySQLError{Number: tmysql.ErrAccessDenied, Message: "access denied"})
failpoint.Return(nil, &mysql.MySQLError{Number: errno.ErrAccessDenied, Message: "access denied"})
}
failpoint.Return(nil, nil)
})
Expand All @@ -122,7 +122,7 @@ func ConnectMySQL(cfg *mysql.Config) (*sql.DB, error) {
return db, nil
}
// If access is denied and password is encoded by base64, try the decoded string as well.
if mysqlErr, ok := errors.Cause(firstErr).(*mysql.MySQLError); ok && mysqlErr.Number == tmysql.ErrAccessDenied {
if mysqlErr, ok := errors.Cause(firstErr).(*mysql.MySQLError); ok && mysqlErr.Number == errno.ErrAccessDenied {
// If password is encoded by base64, try the decoded string as well.
password, decodeErr := base64.StdEncoding.DecodeString(cfg.Passwd)
if decodeErr == nil && string(password) != cfg.Passwd {
Expand Down
8 changes: 4 additions & 4 deletions pkg/meta/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,9 +134,9 @@ const (

var (
// ErrDBExists is the error for db exists.
ErrDBExists = dbterror.ClassMeta.NewStd(mysql.ErrDBCreateExists)
ErrDBExists = dbterror.ClassMeta.NewStd(errno.ErrDBCreateExists)
// ErrDBNotExists is the error for db not exists.
ErrDBNotExists = dbterror.ClassMeta.NewStd(mysql.ErrBadDB)
ErrDBNotExists = dbterror.ClassMeta.NewStd(errno.ErrBadDB)
// ErrPolicyExists is the error for policy exists.
ErrPolicyExists = dbterror.ClassMeta.NewStd(errno.ErrPlacementPolicyExists)
// ErrPolicyNotExists is the error for policy not exists.
Expand All @@ -146,9 +146,9 @@ var (
// ErrResourceGroupNotExists is the error for resource group not exists.
ErrResourceGroupNotExists = dbterror.ClassMeta.NewStd(errno.ErrResourceGroupNotExists)
// ErrTableExists is the error for table exists.
ErrTableExists = dbterror.ClassMeta.NewStd(mysql.ErrTableExists)
ErrTableExists = dbterror.ClassMeta.NewStd(errno.ErrTableExists)
// ErrTableNotExists is the error for table not exists.
ErrTableNotExists = dbterror.ClassMeta.NewStd(mysql.ErrNoSuchTable)
ErrTableNotExists = dbterror.ClassMeta.NewStd(errno.ErrNoSuchTable)
// ErrDDLReorgElementNotExist is the error for reorg element not exists.
ErrDDLReorgElementNotExist = dbterror.ClassMeta.NewStd(errno.ErrDDLReorgElementNotExist)
// ErrInvalidString is the error for invalid string to parse
Expand Down
1 change: 1 addition & 0 deletions pkg/parser/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
importpath = "github.com/pingcap/tidb/pkg/parser",
visibility = ["//visibility:public"],
deps = [
"//pkg/errno",
"//pkg/parser/ast",
"//pkg/parser/auth",
"//pkg/parser/charset",
Expand Down
1 change: 1 addition & 0 deletions pkg/parser/ast/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ go_library(
importpath = "github.com/pingcap/tidb/pkg/parser/ast",
visibility = ["//visibility:public"],
deps = [
"//pkg/errno",
"//pkg/parser/auth",
"//pkg/parser/charset",
"//pkg/parser/format",
Expand Down
32 changes: 16 additions & 16 deletions pkg/parser/ast/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,9 @@ import (
"strings"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/parser/auth"
"github.com/pingcap/tidb/pkg/parser/format"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/parser/tidb"
"github.com/pingcap/tidb/pkg/parser/types"
Expand Down Expand Up @@ -4004,21 +4004,21 @@ func (n *TruncateTableStmt) Accept(v Visitor) (Node, bool) {
}

var (
ErrNoParts = terror.ClassDDL.NewStd(mysql.ErrNoParts)
ErrPartitionColumnList = terror.ClassDDL.NewStd(mysql.ErrPartitionColumnList)
ErrPartitionRequiresValues = terror.ClassDDL.NewStd(mysql.ErrPartitionRequiresValues)
ErrPartitionsMustBeDefined = terror.ClassDDL.NewStd(mysql.ErrPartitionsMustBeDefined)
ErrPartitionWrongNoPart = terror.ClassDDL.NewStd(mysql.ErrPartitionWrongNoPart)
ErrPartitionWrongNoSubpart = terror.ClassDDL.NewStd(mysql.ErrPartitionWrongNoSubpart)
ErrPartitionWrongValues = terror.ClassDDL.NewStd(mysql.ErrPartitionWrongValues)
ErrRowSinglePartitionField = terror.ClassDDL.NewStd(mysql.ErrRowSinglePartitionField)
ErrSubpartition = terror.ClassDDL.NewStd(mysql.ErrSubpartition)
ErrSystemVersioningWrongPartitions = terror.ClassDDL.NewStd(mysql.ErrSystemVersioningWrongPartitions)
ErrTooManyValues = terror.ClassDDL.NewStd(mysql.ErrTooManyValues)
ErrWrongPartitionTypeExpectedSystemTime = terror.ClassDDL.NewStd(mysql.ErrWrongPartitionTypeExpectedSystemTime)
ErrUnknownCharacterSet = terror.ClassDDL.NewStd(mysql.ErrUnknownCharacterSet)
ErrCoalescePartitionNoPartition = terror.ClassDDL.NewStd(mysql.ErrCoalescePartitionNoPartition)
ErrWrongUsage = terror.ClassDDL.NewStd(mysql.ErrWrongUsage)
ErrNoParts = terror.ClassDDL.NewStd(errno.ErrNoParts)
ErrPartitionColumnList = terror.ClassDDL.NewStd(errno.ErrPartitionColumnList)
ErrPartitionRequiresValues = terror.ClassDDL.NewStd(errno.ErrPartitionRequiresValues)
ErrPartitionsMustBeDefined = terror.ClassDDL.NewStd(errno.ErrPartitionsMustBeDefined)
ErrPartitionWrongNoPart = terror.ClassDDL.NewStd(errno.ErrPartitionWrongNoPart)
ErrPartitionWrongNoSubpart = terror.ClassDDL.NewStd(errno.ErrPartitionWrongNoSubpart)
ErrPartitionWrongValues = terror.ClassDDL.NewStd(errno.ErrPartitionWrongValues)
ErrRowSinglePartitionField = terror.ClassDDL.NewStd(errno.ErrRowSinglePartitionField)
ErrSubpartition = terror.ClassDDL.NewStd(errno.ErrSubpartition)
ErrSystemVersioningWrongPartitions = terror.ClassDDL.NewStd(errno.ErrSystemVersioningWrongPartitions)
ErrTooManyValues = terror.ClassDDL.NewStd(errno.ErrTooManyValues)
ErrWrongPartitionTypeExpectedSystemTime = terror.ClassDDL.NewStd(errno.ErrWrongPartitionTypeExpectedSystemTime)
ErrUnknownCharacterSet = terror.ClassDDL.NewStd(errno.ErrUnknownCharacterSet)
ErrCoalescePartitionNoPartition = terror.ClassDDL.NewStd(errno.ErrCoalescePartitionNoPartition)
ErrWrongUsage = terror.ClassDDL.NewStd(errno.ErrWrongUsage)
)

type SubPartitionDefinition struct {
Expand Down
1 change: 1 addition & 0 deletions pkg/parser/charset/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ go_library(
importpath = "github.com/pingcap/tidb/pkg/parser/charset",
visibility = ["//visibility:public"],
deps = [
"//pkg/errno",
"//pkg/parser/mysql",
"//pkg/parser/terror",
"@com_github_pingcap_errors//:errors",
Expand Down
5 changes: 3 additions & 2 deletions pkg/parser/charset/charset.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,16 +19,17 @@ import (

"github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
"go.uber.org/zap"
)

var (
// ErrUnknownCollation is unknown collation.
ErrUnknownCollation = terror.ClassDDL.NewStd(mysql.ErrUnknownCollation)
ErrUnknownCollation = terror.ClassDDL.NewStd(errno.ErrUnknownCollation)
// ErrCollationCharsetMismatch is collation charset mismatch.
ErrCollationCharsetMismatch = terror.ClassDDL.NewStd(mysql.ErrCollationCharsetMismatch)
ErrCollationCharsetMismatch = terror.ClassDDL.NewStd(errno.ErrCollationCharsetMismatch)
)

var (
Expand Down
4 changes: 2 additions & 2 deletions pkg/parser/charset/encoding_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,14 @@ import (
"strings"
"unsafe"

"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/parser/terror"
"golang.org/x/text/encoding"
"golang.org/x/text/transform"
)

// ErrInvalidCharacterString returns when the string is invalid in the specific charset.
var ErrInvalidCharacterString = terror.ClassParser.NewStd(mysql.ErrInvalidCharacterString)
var ErrInvalidCharacterString = terror.ClassParser.NewStd(errno.ErrInvalidCharacterString)

// encodingBase defines some generic functions.
type encodingBase struct {
Expand Down
Loading

0 comments on commit a183b38

Please sign in to comment.