@@ -489,9 +489,9 @@ type LoadDataInfo struct {
489489 columns []* table.Column
490490}
491491
492- // SetBatchCount sets the number of rows to insert in a batch.
493- func (e * LoadDataInfo ) SetBatchCount (limit int64 ) {
494- e .insertVal .batchRows = limit
492+ // SetMaxRowsInBatch sets the max number of rows to insert in a batch.
493+ func (e * LoadDataInfo ) SetMaxRowsInBatch (limit uint64 ) {
494+ e .insertVal .maxRowsInBatch = limit
495495}
496496
497497// getValidData returns prevData and curData that starts from starting symbol.
@@ -606,6 +606,7 @@ func (e *LoadDataInfo) InsertData(prevData, curData []byte) ([]byte, bool, error
606606 isEOF = true
607607 prevData , curData = curData , prevData
608608 }
609+ rows := make ([][]types.Datum , 0 , e .insertVal .maxRowsInBatch )
609610 for len (curData ) > 0 {
610611 line , curData , hasStarting = e .getLine (prevData , curData )
611612 prevData = nil
@@ -631,15 +632,22 @@ func (e *LoadDataInfo) InsertData(prevData, curData []byte) ([]byte, bool, error
631632 if err != nil {
632633 return nil , false , errors .Trace (err )
633634 }
634- e . insertData (cols )
635- e .insertVal .currRow ++
636- if e .insertVal .batchRows != 0 && e .insertVal .currRow % e .insertVal .batchRows == 0 {
635+ rows = append ( rows , e . colsToRow (cols ) )
636+ e .insertVal .rowCount ++
637+ if e .insertVal .maxRowsInBatch != 0 && e .insertVal .rowCount % e .insertVal .maxRowsInBatch == 0 {
637638 reachLimit = true
638639 log .Infof ("This insert rows has reached the batch %d, current total rows %d" ,
639- e .insertVal .batchRows , e .insertVal .currRow )
640+ e .insertVal .maxRowsInBatch , e .insertVal .rowCount )
640641 break
641642 }
642643 }
644+ rows , err := batchMarkDupRows (e .Ctx , e .Table , rows )
645+ if err != nil {
646+ return nil , reachLimit , errors .Trace (err )
647+ }
648+ for _ , row := range rows {
649+ e .insertData (row )
650+ }
643651 if e .insertVal .lastInsertID != 0 {
644652 e .insertVal .ctx .GetSessionVars ().SetLastInsertID (e .insertVal .lastInsertID )
645653 }
@@ -715,7 +723,7 @@ func escapeChar(c byte) byte {
715723 return c
716724}
717725
718- func (e * LoadDataInfo ) insertData (cols []string ) {
726+ func (e * LoadDataInfo ) colsToRow (cols []string ) types. DatumRow {
719727 for i := 0 ; i < len (e .row ); i ++ {
720728 if i >= len (cols ) {
721729 e .row [i ].SetString ("" )
@@ -727,9 +735,16 @@ func (e *LoadDataInfo) insertData(cols []string) {
727735 if err != nil {
728736 warnLog := fmt .Sprintf ("Load Data: insert data:%v failed:%v" , e .row , errors .ErrorStack (err ))
729737 e .insertVal .handleLoadDataWarnings (err , warnLog )
738+ return nil
739+ }
740+ return row
741+ }
742+
743+ func (e * LoadDataInfo ) insertData (row types.DatumRow ) {
744+ if row == nil {
730745 return
731746 }
732- _ , err = e .Table .AddRecord (e .insertVal .ctx , row , false )
747+ _ , err : = e .Table .AddRecord (e .insertVal .ctx , row , false )
733748 if err != nil {
734749 warnLog := fmt .Sprintf ("Load Data: insert data:%v failed:%v" , row , errors .ErrorStack (err ))
735750 e .insertVal .handleLoadDataWarnings (err , warnLog )
@@ -817,8 +832,8 @@ type defaultVal struct {
817832type InsertValues struct {
818833 baseExecutor
819834
820- currRow int64
821- batchRows int64
835+ rowCount uint64
836+ maxRowsInBatch uint64
822837 lastInsertID uint64
823838 needFillDefaultValues bool
824839
@@ -869,7 +884,7 @@ func (e *InsertExec) exec(goCtx goctx.Context, rows [][]types.Datum) (Row, error
869884 // Using BatchGet in insert ignore to mark rows as duplicated before we add records to the table.
870885 if e .IgnoreErr {
871886 var err error
872- rows , err = e . batchMarkDupRows (rows )
887+ rows , err = batchMarkDupRows (e . ctx , e . Table , rows )
873888 if err != nil {
874889 return nil , errors .Trace (err )
875890 }
@@ -928,12 +943,12 @@ type keyWithDupError struct {
928943 dupErr error
929944}
930945
931- func ( e * InsertExec ) getRecordIDs ( rows [][]types.Datum ) ([]int64 , error ) {
946+ func getRecordIDs ( ctx context. Context , t table. Table , rows [][]types.Datum ) ([]int64 , error ) {
932947 recordIDs := make ([]int64 , 0 , len (rows ))
933- if e . Table .Meta ().PKIsHandle {
948+ if t .Meta ().PKIsHandle {
934949 var handleCol * table.Column
935- for _ , col := range e . Table .Cols () {
936- if col .IsPKHandleColumn (e . Table .Meta ()) {
950+ for _ , col := range t .Cols () {
951+ if col .IsPKHandleColumn (t .Meta ()) {
937952 handleCol = col
938953 break
939954 }
@@ -943,7 +958,7 @@ func (e *InsertExec) getRecordIDs(rows [][]types.Datum) ([]int64, error) {
943958 }
944959 } else {
945960 for range rows {
946- recordID , err := e . Table . AllocAutoID (e . ctx )
961+ recordID , err := t . AllocAutoID (ctx )
947962 if err != nil {
948963 return nil , errors .Trace (err )
949964 }
@@ -955,30 +970,30 @@ func (e *InsertExec) getRecordIDs(rows [][]types.Datum) ([]int64, error) {
955970
956971// getKeysNeedCheck gets keys converted from to-be-insert rows to record keys and unique index keys,
957972// which need to be checked whether they are duplicate keys.
958- func ( e * InsertExec ) getKeysNeedCheck ( rows [][]types.Datum ) ([][]keyWithDupError , error ) {
973+ func getKeysNeedCheck ( ctx context. Context , t table. Table , rows [][]types.Datum ) ([][]keyWithDupError , error ) {
959974 nUnique := 0
960- for _ , v := range e . Table .WritableIndices () {
975+ for _ , v := range t .WritableIndices () {
961976 if v .Meta ().Unique {
962977 nUnique ++
963978 }
964979 }
965980 rowWithKeys := make ([][]keyWithDupError , 0 , len (rows ))
966981
967982 // get recordIDs
968- recordIDs , err := e . getRecordIDs (rows )
983+ recordIDs , err := getRecordIDs (ctx , t , rows )
969984 if err != nil {
970985 return nil , errors .Trace (err )
971986 }
972987
973988 for i , row := range rows {
974989 keysWithErr := make ([]keyWithDupError , 0 , nUnique + 1 )
975990 // append record keys and errors
976- if e . Table .Meta ().PKIsHandle {
977- keysWithErr = append (keysWithErr , keyWithDupError {e . Table .RecordKey (recordIDs [i ]), kv .ErrKeyExists .FastGen ("Duplicate entry '%d' for key 'PRIMARY'" , recordIDs [i ])})
991+ if t .Meta ().PKIsHandle {
992+ keysWithErr = append (keysWithErr , keyWithDupError {t .RecordKey (recordIDs [i ]), kv .ErrKeyExists .FastGen ("Duplicate entry '%d' for key 'PRIMARY'" , recordIDs [i ])})
978993 }
979994
980995 // append unique keys and errors
981- for _ , v := range e . Table .WritableIndices () {
996+ for _ , v := range t .WritableIndices () {
982997 if ! v .Meta ().Unique {
983998 continue
984999 }
@@ -989,7 +1004,7 @@ func (e *InsertExec) getKeysNeedCheck(rows [][]types.Datum) ([][]keyWithDupError
9891004 }
9901005 var key []byte
9911006 var distinct bool
992- key , distinct , err = v .GenIndexKey (e . ctx .GetSessionVars ().StmtCtx ,
1007+ key , distinct , err = v .GenIndexKey (ctx .GetSessionVars ().StmtCtx ,
9931008 colVals , recordIDs [i ], nil )
9941009 if err != nil {
9951010 return nil , errors .Trace (err )
@@ -1007,9 +1022,9 @@ func (e *InsertExec) getKeysNeedCheck(rows [][]types.Datum) ([][]keyWithDupError
10071022// batchMarkDupRows marks rows with duplicate errors as nil.
10081023// All duplicate rows were marked and appended as duplicate warnings
10091024// to the statement context in batch.
1010- func ( e * InsertExec ) batchMarkDupRows ( rows [][]types.Datum ) ([][]types.Datum , error ) {
1025+ func batchMarkDupRows ( ctx context. Context , t table. Table , rows [][]types.Datum ) ([][]types.Datum , error ) {
10111026 // get keys need to be checked
1012- rowWithKeys , err := e . getKeysNeedCheck (rows )
1027+ rowWithKeys , err := getKeysNeedCheck (ctx , t , rows )
10131028
10141029 // batch get values
10151030 nKeys := 0
@@ -1022,7 +1037,7 @@ func (e *InsertExec) batchMarkDupRows(rows [][]types.Datum) ([][]types.Datum, er
10221037 batchKeys = append (batchKeys , k .key )
10231038 }
10241039 }
1025- values , err := e . ctx .Txn ().GetSnapshot ().BatchGet (batchKeys )
1040+ values , err := ctx .Txn ().GetSnapshot ().BatchGet (batchKeys )
10261041 if err != nil {
10271042 return nil , errors .Trace (err )
10281043 }
@@ -1033,7 +1048,7 @@ func (e *InsertExec) batchMarkDupRows(rows [][]types.Datum) ([][]types.Datum, er
10331048 if _ , found := values [string (k .key )]; found {
10341049 // If duplicate keys were found in BatchGet, mark row = nil.
10351050 rows [i ] = nil
1036- e . ctx .GetSessionVars ().StmtCtx .AppendWarning (k .dupErr )
1051+ ctx .GetSessionVars ().StmtCtx .AppendWarning (k .dupErr )
10371052 break
10381053 }
10391054 }
@@ -1048,7 +1063,7 @@ func (e *InsertExec) batchMarkDupRows(rows [][]types.Datum) ([][]types.Datum, er
10481063 }
10491064
10501065 // this statement was already been checked
1051- e . ctx .GetSessionVars ().StmtCtx .BatchCheck = true
1066+ ctx .GetSessionVars ().StmtCtx .BatchCheck = true
10521067 return rows , nil
10531068}
10541069
@@ -1240,7 +1255,7 @@ func (e *InsertValues) getRows(cols []*table.Column, ignoreErr bool) (rows [][]t
12401255 if err = e .checkValueCount (length , len (list ), len (e .GenColumns ), i , cols ); err != nil {
12411256 return nil , errors .Trace (err )
12421257 }
1243- e .currRow = int64 (i )
1258+ e .rowCount = uint64 (i )
12441259 rows [i ], err = e .getRow (cols , list , ignoreErr )
12451260 if err != nil {
12461261 return nil , errors .Trace (err )
@@ -1320,7 +1335,7 @@ func (e *InsertValues) getRowsSelect(goCtx goctx.Context, cols []*table.Column,
13201335 if innerRow == nil {
13211336 break
13221337 }
1323- e .currRow = int64 (len (rows ))
1338+ e .rowCount = uint64 (len (rows ))
13241339 row , err := e .fillRowData (cols , innerRow , ignoreErr )
13251340 if err != nil {
13261341 return nil , errors .Trace (err )
@@ -1350,7 +1365,7 @@ func (e *InsertValues) getRowsSelectChunk(goCtx goctx.Context, cols []*table.Col
13501365
13511366 for innerChunkRow := chk .Begin (); innerChunkRow != chk .End (); innerChunkRow = innerChunkRow .Next () {
13521367 innerRow := innerChunkRow .GetDatumRow (fields )
1353- e .currRow = int64 (len (rows ))
1368+ e .rowCount = uint64 (len (rows ))
13541369 row , err := e .fillRowData (cols , innerRow , ignoreErr )
13551370 if err != nil {
13561371 return nil , errors .Trace (err )
@@ -1519,7 +1534,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(row []types.Datum, i int, c *tab
15191534 return errors .Trace (err )
15201535 }
15211536 // It's compatible with mysql. So it sets last insert id to the first row.
1522- if e .currRow == 0 {
1537+ if e .rowCount == 0 {
15231538 e .lastInsertID = uint64 (recordID )
15241539 }
15251540 }
0 commit comments