Skip to content

Commit

Permalink
*: Refine error message about "Out of range value for column" (#6334)
Browse files Browse the repository at this point in the history
  • Loading branch information
spongedu authored and zz-jason committed May 9, 2018
1 parent 6248a74 commit 3db7594
Show file tree
Hide file tree
Showing 5 changed files with 36 additions and 28 deletions.
2 changes: 1 addition & 1 deletion ddl/ddl_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -816,7 +816,7 @@ func (s *testDBSuite) TestIssue6101(c *C) {
s.tk.MustExec("create table t1 (quantity decimal(2) unsigned);")
_, err := s.tk.Exec("insert into t1 values (500), (-500), (~0), (-1);")
terr := errors.Trace(err).(*errors.Err).Cause().(*terror.Error)
c.Assert(terr.Code(), Equals, terror.ErrCode(tmysql.ErrDataOutOfRange))
c.Assert(terr.Code(), Equals, terror.ErrCode(tmysql.ErrWarnDataOutOfRange))
s.tk.MustExec("drop table t1")

s.tk.MustExec("set sql_mode=''")
Expand Down
3 changes: 3 additions & 0 deletions executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -1460,6 +1460,9 @@ func (e *InsertValues) handleErr(col *table.Column, rowIdx int, err error) error
return resetErrDataTooLong(col.Name.O, rowIdx+1, err)
}

if types.ErrOverflow.Equal(err) {
return types.ErrWarnDataOutOfRange.GenByArgs(col.Name.O, int64(rowIdx+1))
}
return e.filterErr(err)
}

Expand Down
2 changes: 1 addition & 1 deletion executor/write_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ func (s *testSuite) TestInsert(c *C) {
_, err = tk.Exec("insert into t value(0)")
c.Assert(err, IsNil)
_, err = tk.Exec("insert into t value(1)")
c.Assert(types.ErrOverflow.Equal(err), IsTrue)
c.Assert(types.ErrWarnDataOutOfRange.Equal(err), IsTrue)

tk.MustExec("drop table if exists t")
tk.MustExec("create table t(c binary(255))")
Expand Down
52 changes: 26 additions & 26 deletions mysql/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,19 +95,19 @@ var MySQLErrName = map[uint16]string{
ErrNormalShutdown: "%s: Normal shutdown\n",
ErrGotSignal: "%s: Got signal %d. Aborting!\n",
ErrShutdownComplete: "%s: Shutdown complete\n",
ErrForcingClose: "%s: Forcing close of thread %ld user: '%-.48s'\n",
ErrForcingClose: "%s: Forcing close of thread %d user: '%-.48s'\n",
ErrIpsock: "Can't create IP socket",
ErrNoSuchIndex: "Table '%-.192s' has no index like the one used in CREATE INDEX; recreate the table",
ErrWrongFieldTerminators: "Field separator argument is not what is expected; check the manual",
ErrBlobsAndNoTerminated: "You can't use fixed rowlength with BLOBs; please use 'fields terminated by'",
ErrTextFileNotReadable: "The file '%-.128s' must be in the database directory or be readable by all",
ErrFileExists: "File '%-.200s' already exists",
ErrLoadInfo: "Records: %ld Deleted: %ld Skipped: %ld Warnings: %ld",
ErrAlterInfo: "Records: %ld Duplicates: %ld",
ErrLoadInfo: "Records: %d Deleted: %d Skipped: %d Warnings: %d",
ErrAlterInfo: "Records: %d Duplicates: %d",
ErrWrongSubKey: "Incorrect prefix key; the used key part isn't a string, the used length is longer than the key part, or the storage engine doesn't support unique prefix keys",
ErrCantRemoveAllFields: "You can't delete all columns with ALTER TABLE; use DROP TABLE instead",
ErrCantDropFieldOrKey: "Can't DROP '%-.192s'; check that column/key exists",
ErrInsertInfo: "Records: %ld Duplicates: %ld Warnings: %ld",
ErrInsertInfo: "Records: %d Duplicates: %d Warnings: %d",
ErrUpdateTableUsed: "You can't specify target table '%-.192s' for update in FROM clause",
ErrNoSuchThread: "Unknown thread id: %lu",
ErrKillDenied: "You are not owner of thread %lu",
Expand All @@ -133,8 +133,8 @@ var MySQLErrName = map[uint16]string{
ErrUnknownCharacterSet: "Unknown character set: '%-.64s'",
ErrTooManyTables: "Too many tables; MySQL can only use %d tables in a join",
ErrTooManyFields: "Too many columns",
ErrTooBigRowsize: "Row size too large. The maximum row size for the used table type, not counting BLOBs, is %ld. This includes storage overhead, check the manual. You have to change some columns to TEXT or BLOBs",
ErrStackOverrun: "Thread stack overrun: Used: %ld of a %ld stack. Use 'mysqld --threadStack=#' to specify a bigger stack if needed",
ErrTooBigRowsize: "Row size too large. The maximum row size for the used table type, not counting BLOBs, is %d. This includes storage overhead, check the manual. You have to change some columns to TEXT or BLOBs",
ErrStackOverrun: "Thread stack overrun: Used: %d of a %d stack. Use 'mysqld --threadStack=#' to specify a bigger stack if needed",
ErrWrongOuterJoin: "Cross dependency found in OUTER JOIN; examine your ON conditions",
ErrNullColumnInIndex: "Table handler doesn't support NULL in given index. Please change column '%-.192s' to be NOT NULL or use another handler",
ErrCantFindUdf: "Can't load function '%-.192s'",
Expand All @@ -149,9 +149,9 @@ var MySQLErrName = map[uint16]string{
ErrPasswordAnonymousUser: "You are using MySQL as an anonymous user and anonymous users are not allowed to change passwords",
ErrPasswordNotAllowed: "You must have privileges to update tables in the mysql database to be able to change passwords for others",
ErrPasswordNoMatch: "Can't find any matching row in the user table",
ErrUpdateInfo: "Rows matched: %ld Changed: %ld Warnings: %ld",
ErrUpdateInfo: "Rows matched: %d Changed: %d Warnings: %d",
ErrCantCreateThread: "Can't create a new thread (errno %d); if you are not out of available memory, you can consult the manual for a possible OS-dependent bug",
ErrWrongValueCountOnRow: "Column count doesn't match value count at row %ld",
ErrWrongValueCountOnRow: "Column count doesn't match value count at row %d",
ErrCantReopenTable: "Can't reopen table: '%-.192s'",
ErrInvalidUseOfNull: "Invalid use of NULL value",
ErrRegexp: "Got error '%-.64s' from regexp",
Expand All @@ -167,7 +167,7 @@ var MySQLErrName = map[uint16]string{
ErrSyntax: "You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use",
ErrDelayedCantChangeLock: "Delayed insert thread couldn't get requested lock for table %-.192s",
ErrTooManyDelayedThreads: "Too many delayed threads in use",
ErrAbortingConnection: "Aborted connection %ld to db: '%-.192s' user: '%-.48s' (%-.64s)",
ErrAbortingConnection: "Aborted connection %d to db: '%-.192s' user: '%-.48s' (%-.64s)",
ErrNetPacketTooLarge: "Got a packet bigger than 'maxAllowedPacket' bytes",
ErrNetReadErrorFromPipe: "Got a read error from the connection pipe",
ErrNetFcntl: "Got an error from fcntl()",
Expand Down Expand Up @@ -199,7 +199,7 @@ var MySQLErrName = map[uint16]string{
ErrErrorDuringRollback: "Got error %d during ROLLBACK",
ErrErrorDuringFlushLogs: "Got error %d during FLUSHLOGS",
ErrErrorDuringCheckpoint: "Got error %d during CHECKPOINT",
ErrNewAbortingConnection: "Aborted connection %ld to db: '%-.192s' user: '%-.48s' host: '%-.64s' (%-.64s)",
ErrNewAbortingConnection: "Aborted connection %d to db: '%-.192s' user: '%-.48s' host: '%-.64s' (%-.64s)",
ErrDumpNotImplemented: "The storage engine for the table does not support binary table dump",
ErrFlushMasterBinlogClosed: "Binlog closed, cannot RESET MASTER",
ErrIndexRebuild: "Failed rebuilding the index of dumped table '%-.192s'",
Expand Down Expand Up @@ -241,7 +241,7 @@ var MySQLErrName = map[uint16]string{
ErrCantUpdateWithReadlock: "Can't execute the query because you have a conflicting read lock",
ErrMixingNotAllowed: "Mixing of transactional and non-transactional tables is disabled",
ErrDupArgument: "Option '%s' used twice in statement",
ErrUserLimitReached: "User '%-.64s' has exceeded the '%s' resource (current value: %ld)",
ErrUserLimitReached: "User '%-.64s' has exceeded the '%s' resource (current value: %d)",
ErrSpecificAccessDenied: "Access denied; you need (at least one of) the %-.128s privilege(s) for this operation",
ErrLocalVariable: "Variable '%-.64s' is a SESSION variable and can't be used with SET GLOBAL",
ErrGlobalVariable: "Variable '%-.64s' is a GLOBAL variable and should be set with SET GLOBAL",
Expand All @@ -264,7 +264,7 @@ var MySQLErrName = map[uint16]string{
ErrAutoConvert: "Converting column '%s' from %s to %s",
ErrIllegalReference: "Reference '%-.64s' not supported (%s)",
ErrDerivedMustHaveAlias: "Every derived table must have its own alias",
ErrSelectReduced: "Select %u was reduced during optimization",
ErrSelectReduced: "Select %d was reduced during optimization",
ErrTablenameNotAllowedHere: "Table '%-.192s' from one of the SELECTs cannot be used in %-.32s",
ErrNotSupportedAuthMode: "Client does not support authentication protocol requested by server; consider upgrading MySQL client",
ErrSpatialCantHaveNull: "All parts of a SPATIAL index must be NOT NULL",
Expand All @@ -275,12 +275,12 @@ var MySQLErrName = map[uint16]string{
ErrZlibZMem: "ZLIB: Not enough memory",
ErrZlibZBuf: "ZLIB: Not enough room in the output buffer (probably, length of uncompressed data was corrupted)",
ErrZlibZData: "ZLIB: Input data corrupted",
ErrCutValueGroupConcat: "Row %u was cut by GROUPCONCAT()",
ErrWarnTooFewRecords: "Row %ld doesn't contain data for all columns",
ErrWarnTooManyRecords: "Row %ld was truncated; it contained more data than there were input columns",
ErrWarnNullToNotnull: "Column set to default value; NULL supplied to NOT NULL column '%s' at row %ld",
ErrWarnDataOutOfRange: "Out of range value for column '%s' at row %ld",
WarnDataTruncated: "Data truncated for column '%s' at row %ld",
ErrCutValueGroupConcat: "Row %d was cut by GROUPCONCAT()",
ErrWarnTooFewRecords: "Row %d doesn't contain data for all columns",
ErrWarnTooManyRecords: "Row %d was truncated; it contained more data than there were input columns",
ErrWarnNullToNotnull: "Column set to default value; NULL supplied to NOT NULL column '%s' at row %d",
ErrWarnDataOutOfRange: "Out of range value for column '%s' at row %d",
WarnDataTruncated: "Data truncated for column '%s' at row %d",
ErrWarnUsingOtherHandler: "Using storage engine %s for table '%s'",
ErrCantAggregate2collations: "Illegal mix of collations (%s,%s) and (%s,%s) for operation '%s'",
ErrDropUser: "Cannot drop one or more of the requested users",
Expand Down Expand Up @@ -314,9 +314,9 @@ var MySQLErrName = map[uint16]string{
ErrGetErrmsg: "Got error %d '%-.100s' from %s",
ErrGetTemporaryErrmsg: "Got temporary error %d '%-.100s' from %s",
ErrUnknownTimeZone: "Unknown or incorrect time zone: '%-.64s'",
ErrWarnInvalidTimestamp: "Invalid TIMESTAMP value in column '%s' at row %ld",
ErrWarnInvalidTimestamp: "Invalid TIMESTAMP value in column '%s' at row %d",
ErrInvalidCharacterString: "Invalid %s character string: '%.64s'",
ErrWarnAllowedPacketOverflowed: "Result of %s() was larger than maxAllowedPacket (%ld) - truncated",
ErrWarnAllowedPacketOverflowed: "Result of %s() was larger than maxAllowedPacket (%d) - truncated",
ErrConflictingDeclarations: "Conflicting declarations: '%s%s' and '%s%s'",
ErrSpNoRecursiveCreate: "Can't create a %s from within another stored routine",
ErrSpAlreadyExists: "%s %s already exists",
Expand All @@ -333,7 +333,7 @@ var MySQLErrName = map[uint16]string{
ErrUpdateLogDeprecatedIgnored: "The update log is deprecated and replaced by the binary log; SET SQLLOGUPDATE has been ignored.",
ErrUpdateLogDeprecatedTranslated: "The update log is deprecated and replaced by the binary log; SET SQLLOGUPDATE has been translated to SET SQLLOGBIN.",
ErrQueryInterrupted: "Query execution was interrupted",
ErrSpWrongNoOfArgs: "Incorrect number of arguments for %s %s; expected %u, got %u",
ErrSpWrongNoOfArgs: "Incorrect number of arguments for %s %s; expected %d, got %d",
ErrSpCondMismatch: "Undefined CONDITION: %s",
ErrSpNoreturn: "No RETURN found in FUNCTION %s",
ErrSpNoreturnend: "FUNCTION %s ended without RETURN",
Expand Down Expand Up @@ -381,7 +381,7 @@ var MySQLErrName = map[uint16]string{
ErrTrgNoSuchRowInTrg: "There is no %s row in %s trigger",
ErrNoDefaultForField: "Field '%-.192s' doesn't have a default value",
ErrDivisionByZero: "Division by 0",
ErrTruncatedWrongValueForField: "Incorrect %-.32s value: '%-.128s' for column '%.192s' at row %ld",
ErrTruncatedWrongValueForField: "Incorrect %-.32s value: '%-.128s' for column '%.192s' at row %d",
ErrIllegalValueForType: "Illegal %s '%-.192s' value found during parsing",
ErrViewNonupdCheck: "CHECK OPTION on non-updatable view '%-.192s.%-.192s'",
ErrViewCheckFailed: "CHECK OPTION failed '%-.192s.%-.192s'",
Expand Down Expand Up @@ -421,7 +421,7 @@ var MySQLErrName = map[uint16]string{
ErrNonexistingProcGrant: "There is no such grant defined for user '%-.48s' on host '%-.64s' on routine '%-.192s'",
ErrProcAutoGrantFail: "Failed to grant EXECUTE and ALTER ROUTINE privileges",
ErrProcAutoRevokeFail: "Failed to revoke all privileges to dropped routine",
ErrDataTooLong: "Data too long for column '%s' at row %ld",
ErrDataTooLong: "Data too long for column '%s' at row %d",
ErrSpBadSQLstate: "Bad SQLSTATE: '%s'",
ErrStartup: "%s: ready for connections.\nVersion: '%s' socket: '%s' port: %d %s",
ErrLoadFromFixedSizeRowsToVar: "Can't load value from file with fixed size rows to variable",
Expand Down Expand Up @@ -451,7 +451,7 @@ var MySQLErrName = map[uint16]string{
ErrForeignDataStringInvalid: "The data source connection string '%-.64s' is not in the correct format",
ErrCantCreateFederatedTable: "Can't create federated table. Foreign data src : %-.64s",
ErrTrgInWrongSchema: "Trigger in wrong schema",
ErrStackOverrunNeedMore: "Thread stack overrun: %ld bytes used of a %ld byte stack, and %ld bytes needed. Use 'mysqld --threadStack=#' to specify a bigger stack.",
ErrStackOverrunNeedMore: "Thread stack overrun: %d bytes used of a %d byte stack, and %d bytes needed. Use 'mysqld --threadStack=#' to specify a bigger stack.",
ErrTooLongBody: "Routine body for '%-.100s' is too long",
ErrWarnCantDropDefaultKeycache: "Cannot drop default keycache",
ErrTooBigDisplaywidth: "Display width out of range for column '%-.192s' (max = %lu)",
Expand Down Expand Up @@ -586,7 +586,7 @@ var MySQLErrName = map[uint16]string{
ErrCantChangeTxCharacteristics: "Transaction characteristics can't be changed while a transaction is in progress",
ErrDupEntryAutoincrementCase: "ALTER TABLE causes autoIncrement resequencing, resulting in duplicate entry '%-.192s' for key '%-.192s'",
ErrEventModifyQueue: "Internal scheduler error %d",
ErrEventSetVar: "Error during starting/stopping of the scheduler. Error code %u",
ErrEventSetVar: "Error during starting/stopping of the scheduler. Error code %d",
ErrPartitionMerge: "Engine cannot be used in partitioned tables",
ErrCantActivateLog: "Cannot activate '%-.64s' log",
ErrRbrNotAvailable: "The server was not built with row-based replication",
Expand Down Expand Up @@ -744,7 +744,7 @@ var MySQLErrName = map[uint16]string{
ErrUnsupportedEngine: "Storage engine '%s' does not support system tables. [%s.%s]",
ErrBinlogUnsafeAutoincNotFirst: "INSERT into autoincrement field which is not the first part in the composed primary key is unsafe.",
ErrCannotLoadFromTableV2: "Cannot load from %s.%s. The table is probably corrupted",
ErrMasterDelayValueOutOfRange: "The requested value %u for the master delay exceeds the maximum %u",
ErrMasterDelayValueOutOfRange: "The requested value %d for the master delay exceeds the maximum %d",
ErrOnlyFdAndRbrEventsAllowedInBinlogStatement: "Only FormatDescriptionLogEvent and row events are allowed in BINLOG statements (but %s was provided)",
ErrPartitionExchangeDifferentOption: "Non matching attribute '%-.64s' between partition and table",
ErrPartitionExchangePartTable: "Table to exchange with partition is partitioned: '%-.64s'",
Expand Down
5 changes: 5 additions & 0 deletions types/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,9 @@ var (
ErrInvalidDefault = terror.ClassTypes.New(codeInvalidDefault, "Invalid default value for '%s'")
// ErrMBiggerThanD is returned when precision less than the scale.
ErrMBiggerThanD = terror.ClassTypes.New(codeMBiggerThanD, mysql.MySQLErrName[mysql.ErrMBiggerThanD])
// ErrWarnDataOutOfRange is returned when the value in a numeric column that is outside the permissible range of the column data type.
// See https://dev.mysql.com/doc/refman/5.5/en/out-of-range-and-overflow.html for details
ErrWarnDataOutOfRange = terror.ClassTypes.New(codeDataOutOfRange, mysql.MySQLErrName[mysql.ErrWarnDataOutOfRange])
)

const (
Expand All @@ -73,6 +76,7 @@ const (
codeUnknown = terror.ErrCode(mysql.ErrUnknown)
codeInvalidDefault = terror.ErrCode(mysql.ErrInvalidDefault)
codeMBiggerThanD = terror.ErrCode(mysql.ErrMBiggerThanD)
codeDataOutOfRange = terror.ErrCode(mysql.ErrWarnDataOutOfRange)
)

var (
Expand All @@ -99,6 +103,7 @@ func init() {
codeUnknown: mysql.ErrUnknown,
codeInvalidDefault: mysql.ErrInvalidDefault,
codeMBiggerThanD: mysql.ErrMBiggerThanD,
codeDataOutOfRange: mysql.ErrWarnDataOutOfRange,
}
terror.ErrClassToMySQLCodes[terror.ClassTypes] = typesMySQLErrCodes
}

0 comments on commit 3db7594

Please sign in to comment.