github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/ccl/changefeedccl/kvfeed/scanner.go (about)

     1  // Copyright 2020 The Cockroach Authors.
     2  //
     3  // Licensed as a CockroachDB Enterprise file under the Cockroach Community
     4  // License (the "License"); you may not use this file except in compliance with
     5  // the License. You may obtain a copy of the License at
     6  //
     7  //     https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
     8  
     9  package kvfeed
    10  
    11  import (
    12  	"context"
    13  	"sync/atomic"
    14  	"time"
    15  
    16  	"github.com/cockroachdb/cockroach/pkg/gossip"
    17  	"github.com/cockroachdb/cockroach/pkg/keys"
    18  	"github.com/cockroachdb/cockroach/pkg/kv"
    19  	"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
    20  	"github.com/cockroachdb/cockroach/pkg/roachpb"
    21  	"github.com/cockroachdb/cockroach/pkg/settings/cluster"
    22  	"github.com/cockroachdb/cockroach/pkg/sql/covering"
    23  	"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
    24  	"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
    25  	"github.com/cockroachdb/cockroach/pkg/util/hlc"
    26  	"github.com/cockroachdb/cockroach/pkg/util/log"
    27  	"github.com/cockroachdb/cockroach/pkg/util/timeutil"
    28  	"github.com/cockroachdb/errors"
    29  )
    30  
    31  type kvScanner interface {
    32  	// Scan will scan all of the KVs in the spans specified by the physical config
    33  	// at the specified timestamp and write them to the buffer.
    34  	Scan(ctx context.Context, sink EventBufferWriter, cfg physicalConfig) error
    35  }
    36  
    37  type scanRequestScanner struct {
    38  	settings *cluster.Settings
    39  	gossip   gossip.DeprecatedGossip
    40  	db       *kv.DB
    41  }
    42  
    43  var _ kvScanner = (*scanRequestScanner)(nil)
    44  
    45  func (p *scanRequestScanner) Scan(
    46  	ctx context.Context, sink EventBufferWriter, cfg physicalConfig,
    47  ) error {
    48  	if log.V(2) {
    49  		log.Infof(ctx, "performing scan on %v at %v withDiff %v",
    50  			cfg.Spans, cfg.Timestamp, cfg.WithDiff)
    51  	}
    52  
    53  	spans, err := getSpansToProcess(ctx, p.db, cfg.Spans)
    54  	if err != nil {
    55  		return err
    56  	}
    57  
    58  	// Export requests for the various watched spans are executed in parallel,
    59  	// with a semaphore-enforced limit based on a cluster setting.
    60  	// The spans here generally correspond with range boundaries.
    61  	approxNodeCount, err := clusterNodeCount(p.gossip)
    62  	if err != nil {
    63  		return err
    64  	}
    65  	maxConcurrentExports := approxNodeCount *
    66  		int(kvserver.ExportRequestsLimit.Get(&p.settings.SV))
    67  	exportsSem := make(chan struct{}, maxConcurrentExports)
    68  	g := ctxgroup.WithContext(ctx)
    69  
    70  	// atomicFinished is used only to enhance debugging messages.
    71  	var atomicFinished int64
    72  
    73  	for _, span := range spans {
    74  		span := span
    75  
    76  		// Wait for our semaphore.
    77  		select {
    78  		case <-ctx.Done():
    79  			return ctx.Err()
    80  		case exportsSem <- struct{}{}:
    81  		}
    82  
    83  		g.GoCtx(func(ctx context.Context) error {
    84  			defer func() { <-exportsSem }()
    85  
    86  			err := p.exportSpan(ctx, span, cfg.Timestamp, cfg.WithDiff, sink)
    87  			finished := atomic.AddInt64(&atomicFinished, 1)
    88  			if log.V(2) {
    89  				log.Infof(ctx, `exported %d of %d: %v`, finished, len(spans), err)
    90  			}
    91  			if err != nil {
    92  				return err
    93  			}
    94  			return nil
    95  		})
    96  	}
    97  	return g.Wait()
    98  }
    99  
   100  func (p *scanRequestScanner) exportSpan(
   101  	ctx context.Context, span roachpb.Span, ts hlc.Timestamp, withDiff bool, sink EventBufferWriter,
   102  ) error {
   103  	txn := p.db.NewTxn(ctx, "changefeed backfill")
   104  	if log.V(2) {
   105  		log.Infof(ctx, `sending ScanRequest %s at %s`, span, ts)
   106  	}
   107  	txn.SetFixedTimestamp(ctx, ts)
   108  	stopwatchStart := timeutil.Now()
   109  	var scanDuration, bufferDuration time.Duration
   110  	const targetBytesPerScan = 16 << 20 // 16 MiB
   111  	for remaining := span; ; {
   112  		start := timeutil.Now()
   113  		b := txn.NewBatch()
   114  		r := roachpb.NewScan(remaining.Key, remaining.EndKey, false /* forUpdate */).(*roachpb.ScanRequest)
   115  		r.ScanFormat = roachpb.BATCH_RESPONSE
   116  		b.Header.TargetBytes = targetBytesPerScan
   117  		// NB: We use a raw request rather than the Scan() method because we want
   118  		// the MVCC timestamps which are encoded in the response but are filtered
   119  		// during result parsing.
   120  		b.AddRawRequest(r)
   121  		if err := txn.Run(ctx, b); err != nil {
   122  			return errors.Wrapf(err, `fetching changes for %s`, span)
   123  		}
   124  		afterScan := timeutil.Now()
   125  		res := b.RawResponse().Responses[0].GetScan()
   126  		if err := slurpScanResponse(ctx, sink, res, ts, withDiff, remaining); err != nil {
   127  			return err
   128  		}
   129  		afterBuffer := timeutil.Now()
   130  		scanDuration += afterScan.Sub(start)
   131  		bufferDuration += afterBuffer.Sub(afterScan)
   132  		if res.ResumeSpan != nil {
   133  			remaining = *res.ResumeSpan
   134  		} else {
   135  			break
   136  		}
   137  	}
   138  	// p.metrics.PollRequestNanosHist.RecordValue(scanDuration.Nanoseconds())
   139  	if err := sink.AddResolved(ctx, span, ts, false); err != nil {
   140  		return err
   141  	}
   142  	if log.V(2) {
   143  		log.Infof(ctx, `finished Scan of %s at %s took %s`,
   144  			span, ts.AsOfSystemTime(), timeutil.Since(stopwatchStart))
   145  	}
   146  	return nil
   147  }
   148  
   149  func getSpansToProcess(
   150  	ctx context.Context, db *kv.DB, targetSpans []roachpb.Span,
   151  ) ([]roachpb.Span, error) {
   152  	var ranges []roachpb.RangeDescriptor
   153  	if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
   154  		var err error
   155  		ranges, err = allRangeDescriptors(ctx, txn)
   156  		return err
   157  	}); err != nil {
   158  		return nil, errors.Wrapf(err, "fetching range descriptors")
   159  	}
   160  
   161  	type spanMarker struct{}
   162  	type rangeMarker struct{}
   163  
   164  	var spanCovering covering.Covering
   165  	for _, span := range targetSpans {
   166  		spanCovering = append(spanCovering, covering.Range{
   167  			Start:   []byte(span.Key),
   168  			End:     []byte(span.EndKey),
   169  			Payload: spanMarker{},
   170  		})
   171  	}
   172  
   173  	var rangeCovering covering.Covering
   174  	for _, rangeDesc := range ranges {
   175  		rangeCovering = append(rangeCovering, covering.Range{
   176  			Start:   []byte(rangeDesc.StartKey),
   177  			End:     []byte(rangeDesc.EndKey),
   178  			Payload: rangeMarker{},
   179  		})
   180  	}
   181  
   182  	chunks := covering.OverlapCoveringMerge(
   183  		[]covering.Covering{spanCovering, rangeCovering},
   184  	)
   185  
   186  	var requests []roachpb.Span
   187  	for _, chunk := range chunks {
   188  		if _, ok := chunk.Payload.([]interface{})[0].(spanMarker); !ok {
   189  			continue
   190  		}
   191  		requests = append(requests, roachpb.Span{Key: chunk.Start, EndKey: chunk.End})
   192  	}
   193  	return requests, nil
   194  }
   195  
   196  // slurpScanResponse iterates the ScanResponse and inserts the contained kvs into
   197  // the KVFeed's buffer.
   198  func slurpScanResponse(
   199  	ctx context.Context,
   200  	sink EventBufferWriter,
   201  	res *roachpb.ScanResponse,
   202  	ts hlc.Timestamp,
   203  	withDiff bool,
   204  	span roachpb.Span,
   205  ) error {
   206  	for _, br := range res.BatchResponses {
   207  		for len(br) > 0 {
   208  			var kv roachpb.KeyValue
   209  			var err error
   210  			kv.Key, kv.Value.Timestamp, kv.Value.RawBytes, br, err = enginepb.ScanDecodeKeyValue(br)
   211  			if err != nil {
   212  				return errors.Wrapf(err, `decoding changes for %s`, span)
   213  			}
   214  			var prevVal roachpb.Value
   215  			if withDiff {
   216  				// Include the same value for the "before" and "after" KV, but
   217  				// interpret them at different timestamp. Specifically, interpret
   218  				// the "before" KV at the timestamp immediately before the schema
   219  				// change. This is handled in kvsToRows.
   220  				prevVal = kv.Value
   221  			}
   222  			if err = sink.AddKV(ctx, kv, prevVal, ts); err != nil {
   223  				return errors.Wrapf(err, `buffering changes for %s`, span)
   224  			}
   225  		}
   226  	}
   227  	return nil
   228  }
   229  
   230  func allRangeDescriptors(ctx context.Context, txn *kv.Txn) ([]roachpb.RangeDescriptor, error) {
   231  	rows, err := txn.Scan(ctx, keys.Meta2Prefix, keys.MetaMax, 0)
   232  	if err != nil {
   233  		return nil, err
   234  	}
   235  
   236  	rangeDescs := make([]roachpb.RangeDescriptor, len(rows))
   237  	for i, row := range rows {
   238  		if err := row.ValueProto(&rangeDescs[i]); err != nil {
   239  			return nil, errors.NewAssertionErrorWithWrappedErrf(err,
   240  				"%s: unable to unmarshal range descriptor", row.Key)
   241  		}
   242  	}
   243  	return rangeDescs, nil
   244  }
   245  
   246  // clusterNodeCount returns the approximate number of nodes in the cluster.
   247  func clusterNodeCount(gw gossip.DeprecatedGossip) (int, error) {
   248  	g, err := gw.OptionalErr(47971)
   249  	if err != nil {
   250  		return 0, err
   251  	}
   252  	var nodes int
   253  	_ = g.IterateInfos(gossip.KeyNodeIDPrefix, func(_ string, _ gossip.Info) error {
   254  		nodes++
   255  		return nil
   256  	})
   257  	return nodes, nil
   258  }