github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/lightning/mydump/parser.go (about)

     1  // Copyright 2019 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package mydump
    15  
    16  import (
    17  	"bytes"
    18  	"fmt"
    19  	"io"
    20  	"regexp"
    21  	"strconv"
    22  	"strings"
    23  	"sync"
    24  	"time"
    25  
    26  	"github.com/pingcap/errors"
    27  	"github.com/pingcap/parser/mysql"
    28  	"github.com/pingcap/tidb/types"
    29  	"go.uber.org/zap"
    30  	"go.uber.org/zap/zapcore"
    31  
    32  	"github.com/pingcap/tidb-lightning/lightning/config"
    33  	"github.com/pingcap/tidb-lightning/lightning/log"
    34  	"github.com/pingcap/tidb-lightning/lightning/metric"
    35  	"github.com/pingcap/tidb-lightning/lightning/worker"
    36  )
    37  
    38  type blockParser struct {
    39  	// states for the lexer
    40  	reader      PooledReader
    41  	buf         []byte
    42  	blockBuf    []byte
    43  	isLastChunk bool
    44  
    45  	// The list of column names of the last INSERT statement.
    46  	columns []string
    47  
    48  	rowPool *sync.Pool
    49  	lastRow Row
    50  	// Current file offset.
    51  	pos int64
    52  
    53  	// cache
    54  	remainBuf *bytes.Buffer
    55  	appendBuf *bytes.Buffer
    56  
    57  	// the Logger associated with this parser for reporting failure
    58  	Logger log.Logger
    59  }
    60  
    61  func makeBlockParser(reader ReadSeekCloser, blockBufSize int64, ioWorkers *worker.Pool) blockParser {
    62  	return blockParser{
    63  		reader:    MakePooledReader(reader, ioWorkers),
    64  		blockBuf:  make([]byte, blockBufSize*config.BufferSizeScale),
    65  		remainBuf: &bytes.Buffer{},
    66  		appendBuf: &bytes.Buffer{},
    67  		Logger:    log.L(),
    68  		rowPool: &sync.Pool{
    69  			New: func() interface{} {
    70  				return make([]types.Datum, 0, 16)
    71  			},
    72  		},
    73  	}
    74  }
    75  
    76  // ChunkParser is a parser of the data files (the file containing only INSERT
    77  // statements).
    78  type ChunkParser struct {
    79  	blockParser
    80  
    81  	escFlavor backslashEscapeFlavor
    82  }
    83  
    84  // Chunk represents a portion of the data file.
    85  type Chunk struct {
    86  	Offset       int64
    87  	EndOffset    int64
    88  	PrevRowIDMax int64
    89  	RowIDMax     int64
    90  	Columns      []string
    91  }
    92  
    93  // Row is the content of a row.
    94  type Row struct {
    95  	RowID int64
    96  	Row   []types.Datum
    97  }
    98  
    99  // MarshalLogArray implements the zapcore.ArrayMarshaler interface
   100  func (row Row) MarshalLogArray(encoder zapcore.ArrayEncoder) error {
   101  	for _, r := range row.Row {
   102  		encoder.AppendString(r.String())
   103  	}
   104  	return nil
   105  }
   106  
   107  type backslashEscapeFlavor uint8
   108  
   109  const (
   110  	backslashEscapeFlavorNone backslashEscapeFlavor = iota
   111  	backslashEscapeFlavorMySQL
   112  	backslashEscapeFlavorMySQLWithNull
   113  )
   114  
   115  type Parser interface {
   116  	Pos() (pos int64, rowID int64)
   117  	SetPos(pos int64, rowID int64) error
   118  	Close() error
   119  	ReadRow() error
   120  	LastRow() Row
   121  	RecycleRow(row Row)
   122  
   123  	// Columns returns the _lower-case_ column names corresponding to values in
   124  	// the LastRow.
   125  	Columns() []string
   126  	// SetColumns set restored column names to parser
   127  	SetColumns([]string)
   128  
   129  	SetLogger(log.Logger)
   130  }
   131  
   132  // NewChunkParser creates a new parser which can read chunks out of a file.
   133  func NewChunkParser(
   134  	sqlMode mysql.SQLMode,
   135  	reader ReadSeekCloser,
   136  	blockBufSize int64,
   137  	ioWorkers *worker.Pool,
   138  ) *ChunkParser {
   139  	escFlavor := backslashEscapeFlavorMySQL
   140  	if sqlMode.HasNoBackslashEscapesMode() {
   141  		escFlavor = backslashEscapeFlavorNone
   142  	}
   143  
   144  	return &ChunkParser{
   145  		blockParser: makeBlockParser(reader, blockBufSize, ioWorkers),
   146  		escFlavor:   escFlavor,
   147  	}
   148  }
   149  
   150  // SetPos changes the reported position and row ID.
   151  func (parser *blockParser) SetPos(pos int64, rowID int64) error {
   152  	p, err := parser.reader.Seek(pos, io.SeekStart)
   153  	if err != nil {
   154  		return errors.Trace(err)
   155  	}
   156  	if p != pos {
   157  		return errors.Errorf("set pos failed, required position: %d, got: %d", pos, p)
   158  	}
   159  	parser.pos = pos
   160  	parser.lastRow.RowID = rowID
   161  	return nil
   162  }
   163  
   164  // Pos returns the current file offset.
   165  func (parser *blockParser) Pos() (int64, int64) {
   166  	return parser.pos, parser.lastRow.RowID
   167  }
   168  
   169  func (parser *blockParser) Close() error {
   170  	return parser.reader.Close()
   171  }
   172  
   173  func (parser *blockParser) Columns() []string {
   174  	return parser.columns
   175  }
   176  
   177  func (parser *blockParser) SetColumns(columns []string) {
   178  	parser.columns = columns
   179  }
   180  
   181  func (parser *blockParser) logSyntaxError() {
   182  	content := parser.buf
   183  	if len(content) > 256 {
   184  		content = content[:256]
   185  	}
   186  	parser.Logger.Error("syntax error",
   187  		zap.Int64("pos", parser.pos),
   188  		zap.ByteString("content", content),
   189  	)
   190  }
   191  
   192  func (parser *blockParser) SetLogger(logger log.Logger) {
   193  	parser.Logger = logger
   194  }
   195  
   196  type token byte
   197  
   198  const (
   199  	tokNil token = iota
   200  	tokRowBegin
   201  	tokRowEnd
   202  	tokValues
   203  	tokNull
   204  	tokTrue
   205  	tokFalse
   206  	tokHexString
   207  	tokBinString
   208  	tokInteger
   209  	tokSingleQuoted
   210  	tokDoubleQuoted
   211  	tokBackQuoted
   212  	tokUnquoted
   213  )
   214  
   215  var tokenDescriptions = [...]string{
   216  	tokNil:          "<Nil>",
   217  	tokRowBegin:     "RowBegin",
   218  	tokRowEnd:       "RowEnd",
   219  	tokValues:       "Values",
   220  	tokNull:         "Null",
   221  	tokTrue:         "True",
   222  	tokFalse:        "False",
   223  	tokHexString:    "HexString",
   224  	tokBinString:    "BinString",
   225  	tokInteger:      "Integer",
   226  	tokSingleQuoted: "SingleQuoted",
   227  	tokDoubleQuoted: "DoubleQuoted",
   228  	tokBackQuoted:   "BackQuoted",
   229  	tokUnquoted:     "Unquoted",
   230  }
   231  
   232  // String implements the fmt.Stringer interface
   233  //
   234  // Mainly used for debugging a token.
   235  func (tok token) String() string {
   236  	t := int(tok)
   237  	if t >= 0 && t < len(tokenDescriptions) {
   238  		if description := tokenDescriptions[t]; description != "" {
   239  			return description
   240  		}
   241  	}
   242  	return fmt.Sprintf("<Unknown(%d)>", t)
   243  }
   244  
   245  func (parser *blockParser) readBlock() error {
   246  	startTime := time.Now()
   247  
   248  	n, err := parser.reader.ReadFull(parser.blockBuf)
   249  
   250  	switch err {
   251  	case io.ErrUnexpectedEOF, io.EOF:
   252  		parser.isLastChunk = true
   253  		fallthrough
   254  	case nil:
   255  		// `parser.buf` reference to `appendBuf.Bytes`, so should use remainBuf to
   256  		// hold the `parser.buf` rest data to prevent slice overlap
   257  		parser.remainBuf.Reset()
   258  		parser.remainBuf.Write(parser.buf)
   259  		parser.appendBuf.Reset()
   260  		parser.appendBuf.Write(parser.remainBuf.Bytes())
   261  		parser.appendBuf.Write(parser.blockBuf[:n])
   262  		parser.buf = parser.appendBuf.Bytes()
   263  		metric.ChunkParserReadBlockSecondsHistogram.Observe(time.Since(startTime).Seconds())
   264  		return nil
   265  	default:
   266  		return errors.Trace(err)
   267  	}
   268  }
   269  
   270  var unescapeRegexp = regexp.MustCompile(`(?s)\\.`)
   271  
   272  func unescape(
   273  	input string,
   274  	delim string,
   275  	escFlavor backslashEscapeFlavor,
   276  ) string {
   277  	if len(delim) > 0 {
   278  		delim2 := delim + delim
   279  		if strings.Index(input, delim2) != -1 {
   280  			input = strings.Replace(input, delim2, delim, -1)
   281  		}
   282  	}
   283  	if escFlavor != backslashEscapeFlavorNone && strings.IndexByte(input, '\\') != -1 {
   284  		input = unescapeRegexp.ReplaceAllStringFunc(input, func(substr string) string {
   285  			switch substr[1] {
   286  			case '0':
   287  				return "\x00"
   288  			case 'b':
   289  				return "\b"
   290  			case 'n':
   291  				return "\n"
   292  			case 'r':
   293  				return "\r"
   294  			case 't':
   295  				return "\t"
   296  			case 'Z':
   297  				return "\x1a"
   298  			default:
   299  				return substr[1:]
   300  			}
   301  		})
   302  	}
   303  	return input
   304  }
   305  
   306  func (parser *ChunkParser) unescapeString(input string) string {
   307  	if len(input) >= 2 {
   308  		switch input[0] {
   309  		case '\'', '"':
   310  			return unescape(input[1:len(input)-1], input[:1], parser.escFlavor)
   311  		case '`':
   312  			return unescape(input[1:len(input)-1], "`", backslashEscapeFlavorNone)
   313  		}
   314  	}
   315  	return input
   316  }
   317  
   318  // ReadRow reads a row from the datafile.
   319  func (parser *ChunkParser) ReadRow() error {
   320  	// This parser will recognize contents like:
   321  	//
   322  	// 		`tableName` (...) VALUES (...) (...) (...)
   323  	//
   324  	// Keywords like INSERT, INTO and separators like ',' and ';' are treated
   325  	// like comments and ignored. Therefore, this parser will accept some
   326  	// nonsense input. The advantage is the parser becomes extremely simple,
   327  	// suitable for us where we just want to quickly and accurately split the
   328  	// file apart, not to validate the content.
   329  
   330  	type state byte
   331  
   332  	const (
   333  		// the state after "INSERT INTO" before the column names or "VALUES"
   334  		stateTableName state = iota
   335  
   336  		// the state while reading the column names
   337  		stateColumns
   338  
   339  		// the state after reading "VALUES"
   340  		stateValues
   341  
   342  		// the state while reading row values
   343  		stateRow
   344  	)
   345  
   346  	// Dry-run sample of the state machine, first row:
   347  	//
   348  	//              Input         Token             State
   349  	//              ~~~~~         ~~~~~             ~~~~~
   350  	//
   351  	//                                              stateValues
   352  	//              INSERT
   353  	//              INTO
   354  	//              `tableName`   tokBackQuoted
   355  	//                                              stateTableName (reset columns)
   356  	//              (             tokRowBegin
   357  	//                                              stateColumns
   358  	//              `a`           tokBackQuoted
   359  	//                                              stateColumns (append column)
   360  	//              ,
   361  	//              `b`           tokBackQuoted
   362  	//                                              stateColumns (append column)
   363  	//              )             tokRowEnd
   364  	//                                              stateValues
   365  	//              VALUES
   366  	//                                              stateValues (no-op)
   367  	//              (             tokRowBegin
   368  	//                                              stateRow (reset row)
   369  	//              1             tokInteger
   370  	//                                              stateRow (append value)
   371  	//              ,
   372  	//              2             tokInteger
   373  	//                                              stateRow (append value)
   374  	//              )             tokRowEnd
   375  	//                                              return
   376  	//
   377  	//
   378  	// Second row:
   379  	//
   380  	//              Input         Token             State
   381  	//              ~~~~~         ~~~~~             ~~~~~
   382  	//
   383  	//                                              stateValues
   384  	//              ,
   385  	//              (             tokRowBegin
   386  	//                                              stateRow (reset row)
   387  	//              3             tokInteger
   388  	//                                              stateRow (append value)
   389  	//              )             tokRowEnd
   390  	//                                              return
   391  	//
   392  	// Third row:
   393  	//
   394  	//              Input         Token             State
   395  	//              ~~~~~         ~~~~~             ~~~~~
   396  	//
   397  	//              ;
   398  	//              INSERT
   399  	//              INTO
   400  	//              `database`    tokBackQuoted
   401  	//                                              stateTableName (reset columns)
   402  	//              .
   403  	//              `tableName`   tokBackQuoted
   404  	//                                              stateTableName (no-op)
   405  	//              VALUES
   406  	//                                              stateValues
   407  	//              (             tokRowBegin
   408  	//                                              stateRow (reset row)
   409  	//              4             tokInteger
   410  	//                                              stateRow (append value)
   411  	//              )             tokRowEnd
   412  	//                                              return
   413  
   414  	row := &parser.lastRow
   415  	st := stateValues
   416  
   417  	for {
   418  		tok, content, err := parser.lex()
   419  		if err != nil {
   420  			if err == io.EOF && st != stateValues {
   421  				return errors.Errorf("syntax error: premature EOF at offset %d", parser.pos)
   422  			}
   423  			return errors.Trace(err)
   424  		}
   425  		switch st {
   426  		case stateTableName:
   427  			switch tok {
   428  			case tokRowBegin:
   429  				st = stateColumns
   430  			case tokValues:
   431  				st = stateValues
   432  			case tokUnquoted, tokDoubleQuoted, tokBackQuoted:
   433  			default:
   434  				return errors.Errorf(
   435  					"syntax error: unexpected %s (%s) at offset %d, expecting %s",
   436  					tok, content, parser.pos, "table name",
   437  				)
   438  			}
   439  		case stateColumns:
   440  			switch tok {
   441  			case tokRowEnd:
   442  				st = stateValues
   443  			case tokUnquoted, tokDoubleQuoted, tokBackQuoted:
   444  				columnName := strings.ToLower(parser.unescapeString(string(content)))
   445  				parser.columns = append(parser.columns, columnName)
   446  			default:
   447  				return errors.Errorf(
   448  					"syntax error: unexpected %s (%s) at offset %d, expecting %s",
   449  					tok, content, parser.pos, "column list",
   450  				)
   451  			}
   452  		case stateValues:
   453  			switch tok {
   454  			case tokRowBegin:
   455  				row.RowID++
   456  				row.Row = parser.acquireDatumSlice()
   457  				st = stateRow
   458  			case tokUnquoted, tokDoubleQuoted, tokBackQuoted:
   459  				parser.columns = nil
   460  				st = stateTableName
   461  			case tokValues:
   462  			default:
   463  				return errors.Errorf(
   464  					"syntax error: unexpected %s (%s) at offset %d, expecting %s",
   465  					tok, content, parser.pos, "start of row",
   466  				)
   467  			}
   468  		case stateRow:
   469  			var value types.Datum
   470  			switch tok {
   471  			case tokRowEnd:
   472  				return nil
   473  			case tokNull:
   474  				value.SetNull()
   475  			case tokTrue:
   476  				value.SetInt64(1)
   477  			case tokFalse:
   478  				value.SetInt64(0)
   479  			case tokInteger:
   480  				c := string(content)
   481  				if strings.HasPrefix(c, "-") {
   482  					i, err := strconv.ParseInt(c, 10, 64)
   483  					if err == nil {
   484  						value.SetInt64(i)
   485  						break
   486  					}
   487  				} else {
   488  					u, err := strconv.ParseUint(c, 10, 64)
   489  					if err == nil {
   490  						value.SetUint64(u)
   491  						break
   492  					}
   493  				}
   494  				// if the integer is too long, fallback to treating it as a
   495  				// string (all types that treats integer specially like BIT
   496  				// can't handle integers more than 64 bits anyway)
   497  				fallthrough
   498  			case tokUnquoted, tokSingleQuoted, tokDoubleQuoted:
   499  				value.SetString(parser.unescapeString(string(content)), "utf8mb4_bin")
   500  			case tokHexString:
   501  				hexLit, err := types.ParseHexStr(string(content))
   502  				if err != nil {
   503  					return err
   504  				}
   505  				value.SetBinaryLiteral(hexLit)
   506  			case tokBinString:
   507  				binLit, err := types.ParseBitStr(string(content))
   508  				if err != nil {
   509  					return err
   510  				}
   511  				value.SetBinaryLiteral(binLit)
   512  			default:
   513  				return errors.Errorf(
   514  					"syntax error: unexpected %s (%s) at offset %d, expecting %s",
   515  					tok, content, parser.pos, "data literal",
   516  				)
   517  			}
   518  			row.Row = append(row.Row, value)
   519  		}
   520  	}
   521  }
   522  
   523  // LastRow is the copy of the row parsed by the last call to ReadRow().
   524  func (parser *blockParser) LastRow() Row {
   525  	return parser.lastRow
   526  }
   527  
   528  // RecycleRow places the row object back into the allocation pool.
   529  func (parser *blockParser) RecycleRow(row Row) {
   530  	parser.rowPool.Put(row.Row[:0])
   531  }
   532  
   533  // acquireDatumSlice allocates an empty []types.Datum
   534  func (parser *blockParser) acquireDatumSlice() []types.Datum {
   535  	return parser.rowPool.Get().([]types.Datum)
   536  }
   537  
   538  // ReadChunks parses the entire file and splits it into continuous chunks of
   539  // size >= minSize.
   540  func ReadChunks(parser Parser, minSize int64) ([]Chunk, error) {
   541  	var chunks []Chunk
   542  
   543  	pos, lastRowID := parser.Pos()
   544  	cur := Chunk{
   545  		Offset:       pos,
   546  		EndOffset:    pos,
   547  		PrevRowIDMax: lastRowID,
   548  		RowIDMax:     lastRowID,
   549  	}
   550  
   551  	for {
   552  		switch err := parser.ReadRow(); errors.Cause(err) {
   553  		case nil:
   554  			cur.EndOffset, cur.RowIDMax = parser.Pos()
   555  			if cur.EndOffset-cur.Offset >= minSize {
   556  				chunks = append(chunks, cur)
   557  				cur.Offset = cur.EndOffset
   558  				cur.PrevRowIDMax = cur.RowIDMax
   559  			}
   560  
   561  		case io.EOF:
   562  			if cur.Offset < cur.EndOffset {
   563  				chunks = append(chunks, cur)
   564  			}
   565  			return chunks, nil
   566  
   567  		default:
   568  			return nil, errors.Trace(err)
   569  		}
   570  	}
   571  }