github.com/pingcap/tidb-lightning@v5.0.0-rc.0.20210428090220-84b649866577+incompatible/lightning/backend/importer.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 backend
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"strings"
    20  	"sync"
    21  	"time"
    22  
    23  	"github.com/coreos/go-semver/semver"
    24  	"github.com/google/uuid"
    25  	"github.com/pingcap/errors"
    26  	kv "github.com/pingcap/kvproto/pkg/import_kvpb"
    27  	"github.com/pingcap/parser/model"
    28  	"github.com/pingcap/tidb-lightning/lightning/glue"
    29  	"github.com/pingcap/tidb/table"
    30  	"go.uber.org/zap"
    31  	"google.golang.org/grpc"
    32  
    33  	"github.com/pingcap/tidb-lightning/lightning/common"
    34  	"github.com/pingcap/tidb-lightning/lightning/log"
    35  )
    36  
    37  const (
    38  	defaultRetryBackoffTime = time.Second * 3
    39  )
    40  
    41  var (
    42  	requiredTiDBVersion = *semver.New("2.1.0")
    43  	requiredPDVersion   = *semver.New("2.1.0")
    44  	requiredTiKVVersion = *semver.New("2.1.0")
    45  )
    46  
    47  // importer represents a gRPC connection to tikv-importer. This type is
    48  // goroutine safe: you can share this instance and execute any method anywhere.
    49  type importer struct {
    50  	conn   *grpc.ClientConn
    51  	cli    kv.ImportKVClient
    52  	pdAddr string
    53  	tls    *common.TLS
    54  
    55  	mutationPool sync.Pool
    56  }
    57  
    58  // NewImporter creates a new connection to tikv-importer. A single connection
    59  // per tidb-lightning instance is enough.
    60  func NewImporter(ctx context.Context, tls *common.TLS, importServerAddr string, pdAddr string) (Backend, error) {
    61  	conn, err := grpc.DialContext(ctx, importServerAddr, tls.ToGRPCDialOption())
    62  	if err != nil {
    63  		return MakeBackend(nil), errors.Trace(err)
    64  	}
    65  
    66  	return MakeBackend(&importer{
    67  		conn:         conn,
    68  		cli:          kv.NewImportKVClient(conn),
    69  		pdAddr:       pdAddr,
    70  		tls:          tls,
    71  		mutationPool: sync.Pool{New: func() interface{} { return &kv.Mutation{} }},
    72  	}), nil
    73  }
    74  
    75  // NewMockImporter creates an *unconnected* importer based on a custom
    76  // ImportKVClient. This is provided for testing only. Do not use this function
    77  // outside of tests.
    78  func NewMockImporter(cli kv.ImportKVClient, pdAddr string) Backend {
    79  	return MakeBackend(&importer{
    80  		conn:         nil,
    81  		cli:          cli,
    82  		pdAddr:       pdAddr,
    83  		mutationPool: sync.Pool{New: func() interface{} { return &kv.Mutation{} }},
    84  	})
    85  }
    86  
    87  // Close the importer connection.
    88  func (importer *importer) Close() {
    89  	if importer.conn != nil {
    90  		if err := importer.conn.Close(); err != nil {
    91  			log.L().Warn("close importer gRPC connection failed", zap.Error(err))
    92  		}
    93  	}
    94  }
    95  
    96  func (*importer) RetryImportDelay() time.Duration {
    97  	return defaultRetryBackoffTime
    98  }
    99  
   100  func (*importer) MaxChunkSize() int {
   101  	// 31 MB. hardcoded by importer, so do we
   102  	return 31 << 10
   103  }
   104  
   105  func (*importer) ShouldPostProcess() bool {
   106  	return true
   107  }
   108  
   109  // isIgnorableOpenCloseEngineError checks if the error from
   110  // OpenEngine/CloseEngine can be safely ignored.
   111  func isIgnorableOpenCloseEngineError(err error) bool {
   112  	// We allow "FileExists" error. This happens when the engine has been opened
   113  	// and closed before. This error typically arise when resuming from a
   114  	// checkpoint with a partially-imported engine.
   115  	//
   116  	// If the error is legit in a no-checkpoints settings, the later WriteEngine
   117  	// API will bail us out to keep us safe.
   118  	return err == nil || strings.Contains(err.Error(), "FileExists")
   119  }
   120  
   121  func (importer *importer) OpenEngine(ctx context.Context, engineUUID uuid.UUID) error {
   122  	req := &kv.OpenEngineRequest{
   123  		Uuid: engineUUID[:],
   124  	}
   125  
   126  	_, err := importer.cli.OpenEngine(ctx, req)
   127  	if !isIgnorableOpenCloseEngineError(err) {
   128  		return errors.Trace(err)
   129  	}
   130  	return nil
   131  }
   132  
   133  func (importer *importer) CloseEngine(ctx context.Context, engineUUID uuid.UUID) error {
   134  	req := &kv.CloseEngineRequest{
   135  		Uuid: engineUUID[:],
   136  	}
   137  
   138  	_, err := importer.cli.CloseEngine(ctx, req)
   139  	if !isIgnorableOpenCloseEngineError(err) {
   140  		return errors.Trace(err)
   141  	}
   142  	return nil
   143  }
   144  
   145  func (importer *importer) Flush(_ context.Context, _ uuid.UUID) error {
   146  	return nil
   147  }
   148  
   149  func (importer *importer) ImportEngine(ctx context.Context, engineUUID uuid.UUID) error {
   150  	req := &kv.ImportEngineRequest{
   151  		Uuid:   engineUUID[:],
   152  		PdAddr: importer.pdAddr,
   153  	}
   154  
   155  	_, err := importer.cli.ImportEngine(ctx, req)
   156  	return errors.Trace(err)
   157  }
   158  
   159  func (importer *importer) CleanupEngine(ctx context.Context, engineUUID uuid.UUID) error {
   160  	req := &kv.CleanupEngineRequest{
   161  		Uuid: engineUUID[:],
   162  	}
   163  
   164  	_, err := importer.cli.CleanupEngine(ctx, req)
   165  	return errors.Trace(err)
   166  }
   167  
   168  func (importer *importer) WriteRows(
   169  	ctx context.Context,
   170  	engineUUID uuid.UUID,
   171  	tableName string,
   172  	columnNames []string,
   173  	ts uint64,
   174  	rows Rows,
   175  ) (finalErr error) {
   176  	var err error
   177  outside:
   178  	for _, r := range rows.SplitIntoChunks(importer.MaxChunkSize()) {
   179  		for i := 0; i < maxRetryTimes; i++ {
   180  			err = importer.WriteRowsToImporter(ctx, engineUUID, ts, r)
   181  			switch {
   182  			case err == nil:
   183  				continue outside
   184  			case common.IsRetryableError(err):
   185  				// retry next loop
   186  			default:
   187  				return err
   188  			}
   189  		}
   190  		return errors.Annotatef(err, "[%s] write rows reach max retry %d and still failed", tableName, maxRetryTimes)
   191  	}
   192  	return nil
   193  }
   194  
   195  func (importer *importer) WriteRowsToImporter(
   196  	ctx context.Context,
   197  	engineUUID uuid.UUID,
   198  	ts uint64,
   199  	rows Rows,
   200  ) (finalErr error) {
   201  	kvs := rows.(kvPairs)
   202  	if len(kvs) == 0 {
   203  		return nil
   204  	}
   205  
   206  	wstream, err := importer.cli.WriteEngine(ctx)
   207  	if err != nil {
   208  		return errors.Trace(err)
   209  	}
   210  
   211  	logger := log.With(zap.Stringer("engineUUID", engineUUID))
   212  
   213  	defer func() {
   214  		if _, closeErr := wstream.CloseAndRecv(); closeErr != nil {
   215  			if finalErr == nil {
   216  				finalErr = errors.Trace(closeErr)
   217  			} else {
   218  				// just log the close error, we need to propagate the earlier error instead
   219  				logger.Warn("close write stream failed", log.ShortError(closeErr))
   220  			}
   221  		}
   222  	}()
   223  
   224  	// Bind uuid for this write request
   225  	req := &kv.WriteEngineRequest{
   226  		Chunk: &kv.WriteEngineRequest_Head{
   227  			Head: &kv.WriteHead{
   228  				Uuid: engineUUID[:],
   229  			},
   230  		},
   231  	}
   232  	if err := wstream.Send(req); err != nil {
   233  		return errors.Trace(err)
   234  	}
   235  
   236  	// Send kv paris as write request content
   237  	mutations := make([]*kv.Mutation, len(kvs))
   238  	for i, pair := range kvs {
   239  		mutations[i] = importer.mutationPool.Get().(*kv.Mutation)
   240  		mutations[i].Op = kv.Mutation_Put
   241  		mutations[i].Key = pair.Key
   242  		mutations[i].Value = pair.Val
   243  	}
   244  
   245  	req.Reset()
   246  	req.Chunk = &kv.WriteEngineRequest_Batch{
   247  		Batch: &kv.WriteBatch{
   248  			CommitTs:  ts,
   249  			Mutations: mutations,
   250  		},
   251  	}
   252  
   253  	err = wstream.Send(req)
   254  	for _, mutation := range mutations {
   255  		importer.mutationPool.Put(mutation)
   256  	}
   257  
   258  	if err != nil {
   259  		return errors.Trace(err)
   260  	}
   261  
   262  	return nil
   263  }
   264  
   265  func (*importer) MakeEmptyRows() Rows {
   266  	return kvPairs(nil)
   267  }
   268  
   269  func (*importer) NewEncoder(tbl table.Table, options *SessionOptions) (Encoder, error) {
   270  	return NewTableKVEncoder(tbl, options)
   271  }
   272  
   273  func (importer *importer) CheckRequirements(ctx context.Context) error {
   274  	if err := checkTiDBVersionByTLS(ctx, importer.tls, requiredTiDBVersion); err != nil {
   275  		return err
   276  	}
   277  	if err := checkPDVersion(ctx, importer.tls, importer.pdAddr, requiredPDVersion); err != nil {
   278  		return err
   279  	}
   280  	if err := checkTiKVVersion(ctx, importer.tls, importer.pdAddr, requiredTiKVVersion); err != nil {
   281  		return err
   282  	}
   283  	return nil
   284  }
   285  
   286  func checkTiDBVersionByTLS(ctx context.Context, tls *common.TLS, requiredVersion semver.Version) error {
   287  	var status struct{ Version string }
   288  	err := tls.GetJSON(ctx, "/status", &status)
   289  	if err != nil {
   290  		return err
   291  	}
   292  
   293  	return checkTiDBVersion(status.Version, requiredVersion)
   294  }
   295  
   296  func checkTiDBVersion(versionStr string, requiredVersion semver.Version) error {
   297  	version, err := common.ExtractTiDBVersion(versionStr)
   298  	if err != nil {
   299  		return errors.Trace(err)
   300  	}
   301  	return checkVersion("TiDB", requiredVersion, *version)
   302  }
   303  
   304  func checkTiDBVersionBySQL(ctx context.Context, g glue.Glue, requiredVersion semver.Version) error {
   305  	versionStr, err := g.GetSQLExecutor().ObtainStringWithLog(
   306  		ctx,
   307  		"SELECT version();",
   308  		"check TiDB version",
   309  		log.L())
   310  	if err != nil {
   311  		return errors.Trace(err)
   312  	}
   313  
   314  	return checkTiDBVersion(versionStr, requiredVersion)
   315  }
   316  
   317  func checkPDVersion(ctx context.Context, tls *common.TLS, pdAddr string, requiredVersion semver.Version) error {
   318  	version, err := common.FetchPDVersion(ctx, tls, pdAddr)
   319  	if err != nil {
   320  		return errors.Trace(err)
   321  	}
   322  
   323  	return checkVersion("PD", requiredVersion, *version)
   324  }
   325  
   326  func checkTiKVVersion(ctx context.Context, tls *common.TLS, pdAddr string, requiredVersion semver.Version) error {
   327  	return ForAllStores(
   328  		ctx,
   329  		tls.WithHost(pdAddr),
   330  		StoreStateDown,
   331  		func(c context.Context, store *Store) error {
   332  			component := fmt.Sprintf("TiKV (at %s)", store.Address)
   333  			version, err := semver.NewVersion(strings.TrimPrefix(store.Version, "v"))
   334  			if err != nil {
   335  				return errors.Annotate(err, component)
   336  			}
   337  			return checkVersion(component, requiredVersion, *version)
   338  		},
   339  	)
   340  }
   341  
   342  func checkVersion(component string, expected, actual semver.Version) error {
   343  	if actual.Compare(expected) >= 0 {
   344  		return nil
   345  	}
   346  	return errors.Errorf(
   347  		"%s version too old, expected '>=%s', found '%s'",
   348  		component,
   349  		expected,
   350  		actual,
   351  	)
   352  }
   353  
   354  func (importer *importer) FetchRemoteTableModels(ctx context.Context, schema string) ([]*model.TableInfo, error) {
   355  	return fetchRemoteTableModelsFromTLS(ctx, importer.tls, schema)
   356  }
   357  
   358  func (importer *importer) EngineFileSizes() []EngineFileSize {
   359  	return nil
   360  }
   361  
   362  func (importer *importer) FlushEngine(context.Context, uuid.UUID) error {
   363  	return nil
   364  }
   365  
   366  func (importer *importer) FlushAllEngines(context.Context) error {
   367  	return nil
   368  }
   369  
   370  func (importer *importer) ResetEngine(context.Context, uuid.UUID) error {
   371  	return errors.New("cannot reset an engine in importer backend")
   372  }
   373  
   374  func (importer *importer) LocalWriter(ctx context.Context, engineUUID uuid.UUID, maxCacheSize int64) (EngineWriter, error) {
   375  	return &ImporterWriter{importer: importer, engineUUID: engineUUID}, nil
   376  }
   377  
   378  type ImporterWriter struct {
   379  	importer   *importer
   380  	engineUUID uuid.UUID
   381  }
   382  
   383  func (w *ImporterWriter) Close() error {
   384  	return nil
   385  }
   386  
   387  func (w *ImporterWriter) AppendRows(ctx context.Context, tableName string, columnNames []string, ts uint64, rows Rows) error {
   388  	return w.importer.WriteRows(ctx, w.engineUUID, tableName, columnNames, ts, rows)
   389  }