github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/flowinfra/flow_scheduler.go (about)

     1  // Copyright 2016 The Cockroach Authors.
     2  //
     3  // Use of this software is governed by the Business Source License
     4  // included in the file licenses/BSL.txt.
     5  //
     6  // As of the Change Date specified in that file, in accordance with
     7  // the Business Source License, use of this software will be governed
     8  // by the Apache License, Version 2.0, included in the file
     9  // licenses/APL.txt.
    10  
    11  package flowinfra
    12  
    13  import (
    14  	"container/list"
    15  	"context"
    16  	"time"
    17  
    18  	"github.com/cockroachdb/cockroach/pkg/settings"
    19  	"github.com/cockroachdb/cockroach/pkg/settings/cluster"
    20  	"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
    21  	"github.com/cockroachdb/cockroach/pkg/util/log"
    22  	"github.com/cockroachdb/cockroach/pkg/util/stop"
    23  	"github.com/cockroachdb/cockroach/pkg/util/syncutil"
    24  	"github.com/cockroachdb/cockroach/pkg/util/timeutil"
    25  )
    26  
    27  const flowDoneChanSize = 8
    28  
    29  var settingMaxRunningFlows = settings.RegisterPublicIntSetting(
    30  	"sql.distsql.max_running_flows",
    31  	"maximum number of concurrent flows that can be run on a node",
    32  	500,
    33  )
    34  
    35  // FlowScheduler manages running flows and decides when to queue and when to
    36  // start flows. The main interface it presents is ScheduleFlows, which passes a
    37  // flow to be run.
    38  type FlowScheduler struct {
    39  	log.AmbientContext
    40  	stopper    *stop.Stopper
    41  	flowDoneCh chan Flow
    42  	metrics    *execinfra.DistSQLMetrics
    43  
    44  	mu struct {
    45  		syncutil.Mutex
    46  		numRunning      int
    47  		maxRunningFlows int
    48  		queue           *list.List
    49  	}
    50  }
    51  
    52  // flowWithCtx stores a flow to run and a context to run it with.
    53  // TODO(asubiotto): Figure out if asynchronous flow execution can be rearranged
    54  // to avoid the need to store the context.
    55  type flowWithCtx struct {
    56  	ctx         context.Context
    57  	flow        Flow
    58  	enqueueTime time.Time
    59  }
    60  
    61  // NewFlowScheduler creates a new FlowScheduler.
    62  func NewFlowScheduler(
    63  	ambient log.AmbientContext,
    64  	stopper *stop.Stopper,
    65  	settings *cluster.Settings,
    66  	metrics *execinfra.DistSQLMetrics,
    67  ) *FlowScheduler {
    68  	fs := &FlowScheduler{
    69  		AmbientContext: ambient,
    70  		stopper:        stopper,
    71  		flowDoneCh:     make(chan Flow, flowDoneChanSize),
    72  		metrics:        metrics,
    73  	}
    74  	fs.mu.queue = list.New()
    75  	fs.mu.maxRunningFlows = int(settingMaxRunningFlows.Get(&settings.SV))
    76  	settingMaxRunningFlows.SetOnChange(&settings.SV, func() {
    77  		fs.mu.Lock()
    78  		fs.mu.maxRunningFlows = int(settingMaxRunningFlows.Get(&settings.SV))
    79  		fs.mu.Unlock()
    80  	})
    81  	return fs
    82  }
    83  
    84  func (fs *FlowScheduler) canRunFlow(_ Flow) bool {
    85  	// TODO(radu): we will have more complex resource accounting (like memory).
    86  	// For now we just limit the number of concurrent flows.
    87  	return fs.mu.numRunning < fs.mu.maxRunningFlows
    88  }
    89  
    90  // runFlowNow starts the given flow; does not wait for the flow to complete.
    91  func (fs *FlowScheduler) runFlowNow(ctx context.Context, f Flow) error {
    92  	log.VEventf(
    93  		ctx, 1, "flow scheduler running flow %s, currently running %d", f.GetID(), fs.mu.numRunning,
    94  	)
    95  	fs.mu.numRunning++
    96  	fs.metrics.FlowStart()
    97  	if err := f.Start(ctx, func() { fs.flowDoneCh <- f }); err != nil {
    98  		return err
    99  	}
   100  	// TODO(radu): we could replace the WaitGroup with a structure that keeps a
   101  	// refcount and automatically runs Cleanup() when the count reaches 0.
   102  	go func() {
   103  		f.Wait()
   104  		f.Cleanup(ctx)
   105  	}()
   106  	return nil
   107  }
   108  
   109  // ScheduleFlow is the main interface of the flow scheduler: it runs or enqueues
   110  // the given flow.
   111  //
   112  // If the flow can start immediately, errors encountered when starting the flow
   113  // are returned. If the flow is enqueued, these error will be later ignored.
   114  func (fs *FlowScheduler) ScheduleFlow(ctx context.Context, f Flow) error {
   115  	return fs.stopper.RunTaskWithErr(
   116  		ctx, "flowinfra.FlowScheduler: scheduling flow", func(ctx context.Context) error {
   117  			fs.mu.Lock()
   118  			defer fs.mu.Unlock()
   119  
   120  			if fs.canRunFlow(f) {
   121  				return fs.runFlowNow(ctx, f)
   122  			}
   123  			log.VEventf(ctx, 1, "flow scheduler enqueuing flow %s to be run later", f.GetID())
   124  			fs.metrics.FlowsQueued.Inc(1)
   125  			fs.mu.queue.PushBack(&flowWithCtx{
   126  				ctx:         ctx,
   127  				flow:        f,
   128  				enqueueTime: timeutil.Now(),
   129  			})
   130  			return nil
   131  
   132  		})
   133  }
   134  
   135  // Start launches the main loop of the scheduler.
   136  func (fs *FlowScheduler) Start() {
   137  	ctx := fs.AnnotateCtx(context.Background())
   138  	fs.stopper.RunWorker(ctx, func(context.Context) {
   139  		stopped := false
   140  		fs.mu.Lock()
   141  		defer fs.mu.Unlock()
   142  
   143  		for {
   144  			if stopped && fs.mu.numRunning == 0 {
   145  				// TODO(radu): somehow error out the flows that are still in the queue.
   146  				return
   147  			}
   148  			fs.mu.Unlock()
   149  			select {
   150  			case <-fs.flowDoneCh:
   151  				fs.mu.Lock()
   152  				fs.mu.numRunning--
   153  				fs.metrics.FlowStop()
   154  				if !stopped {
   155  					if frElem := fs.mu.queue.Front(); frElem != nil {
   156  						n := frElem.Value.(*flowWithCtx)
   157  						fs.mu.queue.Remove(frElem)
   158  						wait := timeutil.Since(n.enqueueTime)
   159  						log.VEventf(
   160  							n.ctx, 1, "flow scheduler dequeued flow %s, spent %s in queue", n.flow.GetID(), wait,
   161  						)
   162  						fs.metrics.FlowsQueued.Dec(1)
   163  						fs.metrics.QueueWaitHist.RecordValue(int64(wait))
   164  						// Note: we use the flow's context instead of the worker
   165  						// context, to ensure that logging etc is relative to the
   166  						// specific flow.
   167  						if err := fs.runFlowNow(n.ctx, n.flow); err != nil {
   168  							log.Errorf(n.ctx, "error starting queued flow: %s", err)
   169  						}
   170  					}
   171  				}
   172  
   173  			case <-fs.stopper.ShouldStop():
   174  				fs.mu.Lock()
   175  				stopped = true
   176  			}
   177  		}
   178  	})
   179  }