diff --git a/pkg/ccl/changefeedccl/changefeed.go b/pkg/ccl/changefeedccl/changefeed.go index 8ae395b3b554..e80f6ec6174c 100644 --- a/pkg/ccl/changefeedccl/changefeed.go +++ b/pkg/ccl/changefeedccl/changefeed.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" - "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -66,113 +65,15 @@ type emitEntry struct { resolved *jobspb.ResolvedSpan } -func runChangefeedFlow( - ctx context.Context, - execCfg *sql.ExecutorConfig, - details jobspb.ChangefeedDetails, - progress jobspb.Progress, - metrics *Metrics, - resultsCh chan<- tree.Datums, - progressedFn func(context.Context, jobs.HighWaterProgressedFn) error, -) error { - details, err := validateDetails(details) - if err != nil { - return err - } - var highWater hlc.Timestamp - if h := progress.GetHighWater(); h != nil { - highWater = *h - } - trackedSpans, err := fetchSpansForTargets(ctx, execCfg.DB, details.Targets, highWater) - if err != nil { - return err - } - - sink, err := getSink(details.SinkURI, resultsCh) - if err != nil { - return err - } - defer func() { - if err := sink.Close(); err != nil { - log.Warningf(ctx, "failed to close changefeed sink: %+v", err) - } - }() - - if _, ok := sink.(*channelSink); !ok { - // For every sink but channelSink (which uses resultsCh to stream the - // changes), we abuse the job's results channel to make CREATE - // CHANGEFEED wait for this before returning to the user to ensure the - // setup went okay. Job resumption doesn't have the same hack, but at - // the moment ignores results and so is currently okay. Return nil - // instead of anything meaningful so that if we start doing anything - // with the results returned by resumed jobs, then it breaks instead of - // returning nonsense. - resultsCh <- tree.Datums(nil) - } - - sink = makeMetricsSink(metrics, sink) - - // The changefeed flow is intentionally structured as a pull model so it's - // easy to later make it into a DistSQL processor. - // - // TODO(dan): Make this into a DistSQL flow. - buf := makeBuffer() - poller := makePoller(execCfg, details, trackedSpans, highWater, buf) - rowsFn := kvsToRows(execCfg, details, buf.Get) - emitEntriesFn := emitEntries(details, sink, rowsFn) - if err != nil { - return err - } - - g := ctxgroup.WithContext(ctx) - g.GoCtx(poller.Run) - - metrics.mu.Lock() - metricsID := metrics.mu.id - metrics.mu.id++ - metrics.mu.Unlock() - defer func() { - metrics.mu.Lock() - delete(metrics.mu.resolved, metricsID) - metrics.mu.Unlock() - }() - - sf := makeSpanFrontier(trackedSpans...) - g.GoCtx(func(ctx context.Context) error { - for { - resolvedSpans, err := emitEntriesFn(ctx) - if err != nil { - return err - } - newFrontier := false - for _, resolvedSpan := range resolvedSpans { - if sf.Forward(resolvedSpan.Span, resolvedSpan.Timestamp) { - newFrontier = true - } - } - if newFrontier { - metrics.mu.Lock() - metrics.mu.resolved[metricsID] = sf.Frontier() - metrics.mu.Unlock() - if err := emitResolvedTimestamp(ctx, details, sink, progressedFn, sf); err != nil { - return err - } - } - } - }) - - return g.Wait() -} - // kvsToRows gets changed kvs from a closure and converts them into sql rows. It // returns a closure that may be repeatedly called to advance the changefeed. // The returned closure is not threadsafe. func kvsToRows( - execCfg *sql.ExecutorConfig, + leaseManager *sql.LeaseManager, details jobspb.ChangefeedDetails, inputFn func(context.Context) (bufferEntry, error), ) func(context.Context) ([]emitEntry, error) { - rfCache := newRowFetcherCache(execCfg.LeaseManager) + rfCache := newRowFetcherCache(leaseManager) var kvs sqlbase.SpanKVFetcher appendEmitEntryForKV := func( diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go new file mode 100644 index 000000000000..d9a4350293c3 --- /dev/null +++ b/pkg/ccl/changefeedccl/changefeed_dist.go @@ -0,0 +1,207 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package changefeedccl + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/distsqlplan" + "github.com/cockroachdb/cockroach/pkg/sql/distsqlrun" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/hlc" +) + +func init() { + distsqlrun.NewChangeAggregatorProcessor = newChangeAggregatorProcessor + distsqlrun.NewChangeFrontierProcessor = newChangeFrontierProcessor +} + +const ( + changeAggregatorProcName = `changeagg` + changeFrontierProcName = `changefntr` +) + +var changefeedResultTypes = []sqlbase.ColumnType{ + {SemanticType: sqlbase.ColumnType_BYTES}, // resolved span + {SemanticType: sqlbase.ColumnType_STRING}, // topic + {SemanticType: sqlbase.ColumnType_BYTES}, // key + {SemanticType: sqlbase.ColumnType_BYTES}, // value +} + +// distChangefeedFlow plans and runs a distributed changefeed. +// +// One or more ChangeAggregator processors watch table data for changes. These +// transform the changed kvs into changed rows and either emit them to a sink +// (such as kafka) or, if there is no sink, forward them in columns 1,2,3 (where +// they will be eventually returned directly via pgwire). In either case, +// periodically a span will become resolved as of some timestamp, meaning that +// no new rows will ever be emitted at or below that timestamp. These span-level +// resolved timestamps are emitted as a marshaled `jobspb.ResolvedSpan` proto in +// column 0. +// +// The flow will always have exactly one ChangeFrontier processor which all the +// ChangeAggregators feed into. It collects all span-level resolved timestamps +// and aggregates them into a changefeed-level resolved timestamp, which is the +// minimum of the span-level resolved timestamps. This changefeed-level resolved +// timestamp is emitted into the changefeed sink (or returned to the gateway if +// there is no sink) whenever it advances. ChangeFrontier also updates the +// progress of the changefeed's corresponding system job. +// +// TODO(dan): The logic here is the planning for the non-enterprise version of +// changefeeds, which is one ChangeAggregator processor feeding into one +// ChangeFrontier processor with both on the gateway node. Also implement the +// planning logic for the enterprise version, which places ChangeAggregator +// processors on the leaseholder for the spans they're watching. +func distChangefeedFlow( + ctx context.Context, + phs sql.PlanHookState, + jobID int64, + details jobspb.ChangefeedDetails, + progress jobspb.Progress, + resultsCh chan<- tree.Datums, +) error { + var err error + details, err = validateDetails(details) + if err != nil { + return err + } + + var highWater hlc.Timestamp + if h := progress.GetHighWater(); h != nil { + highWater = *h + } + + execCfg := phs.ExecCfg() + trackedSpans, err := fetchSpansForTargets(ctx, execCfg.DB, details.Targets, highWater) + if err != nil { + return err + } + + // TODO(dan): Merge these with the span-level resolved timestamps from the + // job progress. + var watches []distsqlrun.ChangeAggregatorSpec_Watch + for _, span := range trackedSpans { + watches = append(watches, distsqlrun.ChangeAggregatorSpec_Watch{ + Span: span, + InitialResolved: highWater, + }) + } + + gatewayNodeID := execCfg.NodeID.Get() + changeAggregatorProcs := []distsqlplan.Processor{{ + Node: gatewayNodeID, + Spec: distsqlrun.ProcessorSpec{ + Core: distsqlrun.ProcessorCoreUnion{ + ChangeAggregator: &distsqlrun.ChangeAggregatorSpec{ + Watches: watches, + Feed: details, + }, + }, + Output: []distsqlrun.OutputRouterSpec{{Type: distsqlrun.OutputRouterSpec_PASS_THROUGH}}, + }, + }} + changeFrontierSpec := distsqlrun.ChangeFrontierSpec{ + TrackedSpans: trackedSpans, + Feed: details, + JobID: jobID, + } + + var p sql.PhysicalPlan + + stageID := p.NewStageID() + p.ResultRouters = make([]distsqlplan.ProcessorIdx, len(changeAggregatorProcs)) + for i, proc := range changeAggregatorProcs { + proc.Spec.StageID = stageID + pIdx := p.AddProcessor(proc) + p.ResultRouters[i] = pIdx + } + + p.AddSingleGroupStage( + gatewayNodeID, + distsqlrun.ProcessorCoreUnion{ChangeFrontier: &changeFrontierSpec}, + distsqlrun.PostProcessSpec{}, + changefeedResultTypes, + ) + + p.ResultTypes = changefeedResultTypes + p.PlanToStreamColMap = []int{1, 2, 3} + + // Changefeed flows handle transactional consistency themselves. + var noTxn *client.Txn + + dsp := phs.DistSQLPlanner() + evalCtx := phs.ExtendedEvalContext() + planCtx := dsp.NewPlanningCtx(ctx, evalCtx, noTxn) + dsp.FinalizePlan(&planCtx, &p) + + resultRows := makeChangefeedResultWriter(resultsCh) + recv := sql.MakeDistSQLReceiver( + ctx, + resultRows, + tree.Rows, + execCfg.RangeDescriptorCache, + execCfg.LeaseHolderCache, + noTxn, + func(ts hlc.Timestamp) {}, + evalCtx.Tracing, + ) + + var finishedSetupFn func() + if details.SinkURI != `` { + // We abuse the job's results channel to make CREATE CHANGEFEED wait for + // this before returning to the user to ensure the setup went okay. Job + // resumption doesn't have the same hack, but at the moment ignores + // results and so is currently okay. Return nil instead of anything + // meaningful so that if we start doing anything with the results + // returned by resumed jobs, then it breaks instead of returning + // nonsense. + finishedSetupFn = func() { resultsCh <- tree.Datums(nil) } + } + + dsp.Run(&planCtx, noTxn, &p, recv, evalCtx, finishedSetupFn) + return resultRows.Err() +} + +// changefeedResultWriter implements the `distsqlrun.resultWriter` that sends +// the received rows back over the given channel. +type changefeedResultWriter struct { + rowsCh chan<- tree.Datums + rowsAffected int + err error +} + +func makeChangefeedResultWriter(rowsCh chan<- tree.Datums) *changefeedResultWriter { + return &changefeedResultWriter{rowsCh: rowsCh} +} + +func (w *changefeedResultWriter) AddRow(ctx context.Context, row tree.Datums) error { + // Copy the row because it's not guaranteed to exist after this function + // returns. + row = append(tree.Datums(nil), row...) + + select { + case <-ctx.Done(): + return ctx.Err() + case w.rowsCh <- row: + return nil + } +} +func (w *changefeedResultWriter) IncrementRowsAffected(n int) { + w.rowsAffected += n +} +func (w *changefeedResultWriter) SetError(err error) { + w.err = err +} +func (w *changefeedResultWriter) Err() error { + return w.err +} diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go new file mode 100644 index 000000000000..a089af1204a9 --- /dev/null +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -0,0 +1,432 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package changefeedccl + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/distsqlrun" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/pkg/errors" +) + +type changeAggregator struct { + distsqlrun.ProcessorBase + + flowCtx *distsqlrun.FlowCtx + spec distsqlrun.ChangeAggregatorSpec + memAcc mon.BoundAccount + + // cancel shuts down the processor, both the `Next()` flow and the poller. + cancel func() + // poller runs in the background and puts kv changes and resolved spans into + // a buffer, which is used by `Next()`. + poller *poller + // pollerErrCh is written once with the poller error (or nil). + pollerErrCh chan error + + // sink is the Sink to write rows to. Resolved timestamps are never written + // by changeAggregator. + sink Sink + // tickFn is the workhorse behind Next(). It pulls kv changes from the + // buffer that poller fills, handles table leasing, converts them to rows, + // and writes them to the sink. + tickFn func(context.Context) ([]jobspb.ResolvedSpan, error) + // changedRowBuf, if non-nil, contains changed rows to be emitted. Anything + // queued in `resolvedSpanBuf` is dependent on these having been emitted, so + // this one must be empty before moving on to that one. + changedRowBuf *encDatumRowBuffer + // resolvedSpanBuf contains resolved span updates to send to changeFrontier. + // If sink is a bufferSink, it must be emptied before these are sent. + resolvedSpanBuf encDatumRowBuffer +} + +var _ distsqlrun.Processor = &changeAggregator{} +var _ distsqlrun.RowSource = &changeAggregator{} + +func newChangeAggregatorProcessor( + flowCtx *distsqlrun.FlowCtx, + processorID int32, + spec distsqlrun.ChangeAggregatorSpec, + output distsqlrun.RowReceiver, +) (distsqlrun.Processor, error) { + ctx := flowCtx.EvalCtx.Ctx() + memMonitor := distsqlrun.NewMonitor(ctx, flowCtx.EvalCtx.Mon, "changeagg-mem") + ca := &changeAggregator{ + flowCtx: flowCtx, + spec: spec, + memAcc: memMonitor.MakeBoundAccount(), + } + if err := ca.Init( + ca, + &distsqlrun.PostProcessSpec{}, + nil, /* types */ + flowCtx, + processorID, + output, + memMonitor, + distsqlrun.ProcStateOpts{ + TrailingMetaCallback: func() []distsqlrun.ProducerMetadata { + ca.close() + return nil + }, + }, + ); err != nil { + return nil, err + } + + initialHighWater := hlc.Timestamp{WallTime: -1} + var spans []roachpb.Span + for _, watch := range spec.Watches { + spans = append(spans, watch.Span) + if initialHighWater.WallTime == -1 || watch.InitialResolved.Less(initialHighWater) { + initialHighWater = watch.InitialResolved + } + } + + var err error + if ca.sink, err = getSink(spec.Feed.SinkURI, spec.Feed.Targets); err != nil { + return nil, err + } + if b, ok := ca.sink.(*bufferSink); ok { + ca.changedRowBuf = &b.buf + } + // The job registry has a set of metrics used to monitor the various jobs it + // runs. They're all stored as the `metric.Struct` interface because of + // dependency cycles. + metrics := flowCtx.JobRegistry.MetricsStruct().Changefeed.(*Metrics) + ca.sink = makeMetricsSink(metrics, ca.sink) + + buf := makeBuffer() + ca.poller = makePoller( + flowCtx.Settings, flowCtx.ClientDB, flowCtx.ClientDB.Clock(), flowCtx.Gossip, spans, + spec.Feed.Targets, initialHighWater, buf) + rowsFn := kvsToRows(flowCtx.LeaseManager.(*sql.LeaseManager), spec.Feed, buf.Get) + ca.tickFn = emitEntries(spec.Feed, ca.sink, rowsFn) + + return ca, nil +} + +func (ca *changeAggregator) OutputTypes() []sqlbase.ColumnType { + return changefeedResultTypes +} + +// Start is part of the RowSource interface. +func (ca *changeAggregator) Start(ctx context.Context) context.Context { + ctx, ca.cancel = context.WithCancel(ctx) + + ca.pollerErrCh = make(chan error, 1) + go func(ctx context.Context) { + err := ca.poller.Run(ctx) + // Trying to call MoveToDraining here is racy (`MoveToDraining called in + // state stateTrailingMeta`), so return the error via a channel. + ca.pollerErrCh <- err + close(ca.pollerErrCh) + ca.cancel() + }(ctx) + + ctx = ca.StartInternal(ctx, changeAggregatorProcName) + return ctx +} + +func (ca *changeAggregator) close() { + // Wait for the poller to finish shutting down. If the poller errored first, + // then Next will have passed its error to MoveToDraining. Otherwise, the + // error will be related to the forced shutdown of the poller (probably + // context canceled) and we don't care what it is, so throw it away. + <-ca.pollerErrCh + if err := ca.sink.Close(); err != nil { + log.Warningf(ca.Ctx, `error closing sink. goroutines may have leaked: %v`, err) + } + // Need to close the mem accounting while the context is still valid. + ca.memAcc.Close(ca.Ctx) + ca.InternalClose() + ca.MemMonitor.Stop(ca.Ctx) +} + +// Next is part of the RowSource interface. +func (ca *changeAggregator) Next() (sqlbase.EncDatumRow, *distsqlrun.ProducerMetadata) { + for ca.State == distsqlrun.StateRunning { + if !ca.changedRowBuf.IsEmpty() { + return ca.changedRowBuf.Pop(), nil + } else if !ca.resolvedSpanBuf.IsEmpty() { + return ca.resolvedSpanBuf.Pop(), nil + } + + if err := ca.tick(); err != nil { + select { + // If the poller errored first, that's the interesting one, so + // overwrite `err`. + case err = <-ca.pollerErrCh: + default: + } + // Shut down the poller if it wasn't already. + ca.cancel() + + ca.MoveToDraining(err) + break + } + } + return nil, ca.DrainHelper() +} + +func (ca *changeAggregator) tick() error { + resolvedSpans, err := ca.tickFn(ca.Ctx) + if err != nil { + return err + } + + for _, resolvedSpan := range resolvedSpans { + resolvedBytes, err := protoutil.Marshal(&resolvedSpan) + if err != nil { + return err + } + // Enqueue a row to be returned that indicates some span-level resolved + // timestamp has advanced. If any rows were queued in `sink`, they must + // be emitted first. + ca.resolvedSpanBuf.Push(sqlbase.EncDatumRow{ + sqlbase.EncDatum{Datum: tree.NewDBytes(tree.DBytes(resolvedBytes))}, + sqlbase.EncDatum{Datum: tree.DNull}, // topic + sqlbase.EncDatum{Datum: tree.DNull}, // key + sqlbase.EncDatum{Datum: tree.DNull}, // value + }) + } + return nil +} + +// ConsumerDone is part of the RowSource interface. +func (ca *changeAggregator) ConsumerDone() { + ca.MoveToDraining(nil /* err */) +} + +// ConsumerClosed is part of the RowSource interface. +func (ca *changeAggregator) ConsumerClosed() { + // The consumer is done, Next() will not be called again. + ca.InternalClose() +} + +type changeFrontier struct { + distsqlrun.ProcessorBase + + flowCtx *distsqlrun.FlowCtx + spec distsqlrun.ChangeFrontierSpec + memAcc mon.BoundAccount + a sqlbase.DatumAlloc + + // input returns rows from one or more changeAggregator processors + input distsqlrun.RowSource + + // sf contains the current resolved timestamp high-water for the tracked + // span set. + sf *spanFrontier + // sink is the Sink to write resolved timestamps to. Rows are never written + // by changeFrontier. + sink Sink + // jobProgressedFn, if non-nil, is called to checkpoint the changefeed's + // progress in the corresponding system job entry. + jobProgressedFn func(context.Context, jobs.HighWaterProgressedFn) error + // passthroughBuf, in some but not all flows, contains changed row data to + // pass through unchanged to the gateway node. + passthroughBuf encDatumRowBuffer + // resolvedBuf, if non-nil, contains rows indicating a changefeed-level + // resolved timestamp to be returned. It depends on everything in + // `passthroughBuf` being sent, so that one needs to be emptied first. + resolvedBuf *encDatumRowBuffer + // metrics are monitoring counters shared between all changefeeds. + metrics *Metrics + // metricsID is used as the unique id of this changefeed in the + // metrics.MinHighWater map. + metricsID int +} + +var _ distsqlrun.Processor = &changeFrontier{} +var _ distsqlrun.RowSource = &changeFrontier{} + +func newChangeFrontierProcessor( + flowCtx *distsqlrun.FlowCtx, + processorID int32, + spec distsqlrun.ChangeFrontierSpec, + input distsqlrun.RowSource, + output distsqlrun.RowReceiver, +) (distsqlrun.Processor, error) { + ctx := flowCtx.EvalCtx.Ctx() + memMonitor := distsqlrun.NewMonitor(ctx, flowCtx.EvalCtx.Mon, "changefntr-mem") + cf := &changeFrontier{ + flowCtx: flowCtx, + spec: spec, + memAcc: memMonitor.MakeBoundAccount(), + input: input, + sf: makeSpanFrontier(spec.TrackedSpans...), + } + if err := cf.Init( + cf, &distsqlrun.PostProcessSpec{}, + input.OutputTypes(), + flowCtx, + processorID, + output, + memMonitor, + distsqlrun.ProcStateOpts{ + TrailingMetaCallback: func() []distsqlrun.ProducerMetadata { + cf.close() + return nil + }, + InputsToDrain: []distsqlrun.RowSource{cf.input}, + }, + ); err != nil { + return nil, err + } + + var err error + if cf.sink, err = getSink(spec.Feed.SinkURI, spec.Feed.Targets); err != nil { + return nil, err + } + if b, ok := cf.sink.(*bufferSink); ok { + cf.resolvedBuf = &b.buf + } + // The job registry has a set of metrics used to monitor the various jobs it + // runs. They're all stored as the `metric.Struct` interface because of + // dependency cycles. + cf.metrics = flowCtx.JobRegistry.MetricsStruct().Changefeed.(*Metrics) + cf.sink = makeMetricsSink(cf.metrics, cf.sink) + + if spec.JobID != 0 { + job, err := flowCtx.JobRegistry.LoadJob(ctx, spec.JobID) + if err != nil { + return nil, err + } + cf.jobProgressedFn = job.HighWaterProgressed + } + + return cf, nil +} + +func (cf *changeFrontier) OutputTypes() []sqlbase.ColumnType { + return changefeedResultTypes +} + +// Start is part of the RowSource interface. +func (cf *changeFrontier) Start(ctx context.Context) context.Context { + cf.input.Start(ctx) + ctx = cf.StartInternal(ctx, changeFrontierProcName) + + cf.metrics.mu.Lock() + cf.metricsID = cf.metrics.mu.id + cf.metrics.mu.id++ + cf.metrics.mu.Unlock() + go func() { + // Delete this feed from the MinHighwater metric so it's no longer + // considered by the gauge. + // + // TODO(dan): Ideally this would be done in something like `close` but + // there's nothing that's guaranteed to be called when a processor shuts + // down. + <-ctx.Done() + cf.metrics.mu.Lock() + delete(cf.metrics.mu.resolved, cf.metricsID) + cf.metricsID = -1 + cf.metrics.mu.Unlock() + }() + + return ctx +} + +func (cf *changeFrontier) close() { + if err := cf.sink.Close(); err != nil { + log.Warningf(cf.Ctx, `error closing sink. goroutines may have leaked: %v`, err) + } + // Need to close the mem accounting while the context is still valid. + cf.memAcc.Close(cf.Ctx) + cf.InternalClose() + cf.MemMonitor.Stop(cf.Ctx) +} + +// Next is part of the RowSource interface. +func (cf *changeFrontier) Next() (sqlbase.EncDatumRow, *distsqlrun.ProducerMetadata) { + for cf.State == distsqlrun.StateRunning { + if !cf.passthroughBuf.IsEmpty() { + return cf.passthroughBuf.Pop(), nil + } else if !cf.resolvedBuf.IsEmpty() { + return cf.resolvedBuf.Pop(), nil + } + + row, meta := cf.input.Next() + if meta != nil { + if meta.Err != nil { + cf.MoveToDraining(nil /* err */) + } + return nil, meta + } + if row == nil { + cf.MoveToDraining(nil /* err */) + break + } + + if row[0].IsNull() { + // In changefeeds with a sink, this will never happen. But in the + // core changefeed, which returns changed rows directly via pgwire, + // a row with a null resolved_span field is a changed row that needs + // to be forwarded to the gateway. + cf.passthroughBuf.Push(row) + continue + } + + if err := cf.noteResolvedSpan(row[0]); err != nil { + cf.MoveToDraining(err) + break + } + } + return nil, cf.DrainHelper() +} + +func (cf *changeFrontier) noteResolvedSpan(d sqlbase.EncDatum) error { + if err := d.EnsureDecoded(&changefeedResultTypes[0], &cf.a); err != nil { + return err + } + raw, ok := d.Datum.(*tree.DBytes) + if !ok { + return errors.Errorf(`unexpected datum type %T: %s`, d.Datum, d.Datum) + } + var resolved jobspb.ResolvedSpan + if err := protoutil.Unmarshal([]byte(*raw), &resolved); err != nil { + return errors.Wrapf(err, `unmarshalling resolved span: %x`, raw) + } + if cf.sf.Forward(resolved.Span, resolved.Timestamp) { + cf.metrics.mu.Lock() + if cf.metricsID != -1 { + cf.metrics.mu.resolved[cf.metricsID] = cf.sf.Frontier() + } + cf.metrics.mu.Unlock() + if err := emitResolvedTimestamp( + cf.Ctx, cf.spec.Feed, cf.sink, cf.jobProgressedFn, cf.sf, + ); err != nil { + return err + } + } + return nil +} + +// ConsumerDone is part of the RowSource interface. +func (cf *changeFrontier) ConsumerDone() { + cf.MoveToDraining(nil /* err */) +} + +// ConsumerClosed is part of the RowSource interface. +func (cf *changeFrontier) ConsumerClosed() { + // The consumer is done, Next() will not be called again. + cf.InternalClose() +} diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index b30f3070f4ac..33f939e009c1 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -44,7 +44,7 @@ const ( optEnvelopeKeyOnly envelopeType = `key_only` optEnvelopeRow envelopeType = `row` - sinkSchemeChannel = `` + sinkSchemeBuffer = `` sinkSchemeKafka = `kafka` sinkParamTopicPrefix = `topic_prefix` ) @@ -186,13 +186,7 @@ func changefeedPlanHook( } if details.SinkURI == `` { - // The job registry has a set of metrics used to monitor the various - // jobs it runs. They're all stored as the `metric.Struct` interface - // because of dependency cycles. - metrics := p.ExecCfg().JobRegistry.MetricsStruct().Changefeed.(*Metrics) - return runChangefeedFlow( - ctx, p.ExecCfg(), details, progress, metrics, resultsCh, nil, /* progressedFn */ - ) + return distChangefeedFlow(ctx, p, 0 /* jobID */, details, progress, resultsCh) } if err := utilccl.CheckEnterpriseEnabled( @@ -309,14 +303,12 @@ type changefeedResumer struct{} func (b *changefeedResumer) Resume( ctx context.Context, job *jobs.Job, planHookState interface{}, startedCh chan<- tree.Datums, ) error { - execCfg := planHookState.(sql.PlanHookState).ExecCfg() + phs := planHookState.(sql.PlanHookState) details := job.Details().(jobspb.ChangefeedDetails) progress := job.Progress() - metrics := job.RegistryMetrics().Changefeed.(*Metrics) - err := runChangefeedFlow( - ctx, execCfg, details, progress, metrics, startedCh, job.HighWaterProgressed) + err := distChangefeedFlow(ctx, phs, *job.ID(), details, progress, startedCh) if err != nil { - log.Infof(ctx, `CHANGEFEED job %d returning with error: %+v`, *job.ID(), err) + log.Infof(ctx, `CHANGEFEED job %d returning with error: %v`, *job.ID(), err) } return err } diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index ffc1997404c4..8ad15614caa8 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/apd" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -379,7 +378,6 @@ func TestChangefeedComputedColumn(t *testing.T) { func TestChangefeedTruncateRenameDrop(t *testing.T) { defer leaktest.AfterTest(t)() - defer utilccl.TestingEnableEnterprise()() ctx := context.Background() s, sqlDBRaw, _ := serverutils.StartServer(t, base.TestServerArgs{ diff --git a/pkg/ccl/changefeedccl/helpers_test.go b/pkg/ccl/changefeedccl/helpers_test.go index 754545d682a9..a21e912dc767 100644 --- a/pkg/ccl/changefeedccl/helpers_test.go +++ b/pkg/ccl/changefeedccl/helpers_test.go @@ -55,7 +55,6 @@ func createBenchmarkChangefeed( Targets: map[sqlbase.ID]string{tableDesc.ID: tableDesc.Name}, } progress := jobspb.Progress{} - metrics := MakeMetrics().(*Metrics) ctx, cancel := context.WithCancel(ctx) errCh := make(chan error, 1) @@ -63,7 +62,9 @@ func createBenchmarkChangefeed( wg.Add(1) go func() { defer wg.Done() - errCh <- runChangefeedFlow(ctx, execCfg, details, progress, metrics, resultsCh, nil) + errCh <- errors.New(`TODO(dan): The distsql PR (#28555) broke createBenchmarkChangefeed`) + var phs sql.PlanHookState + errCh <- distChangefeedFlow(ctx, phs, 0 /* jobID */, details, progress, resultsCh) }() return func() error { select { diff --git a/pkg/ccl/changefeedccl/poller.go b/pkg/ccl/changefeedccl/poller.go index 0ce9cec9559c..d511c629d700 100644 --- a/pkg/ccl/changefeedccl/poller.go +++ b/pkg/ccl/changefeedccl/poller.go @@ -14,21 +14,18 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/gossip" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/bufalloc" - "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" - "github.com/cockroachdb/cockroach/pkg/ccl/storageccl/engineccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl/intervalccl" + "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" - "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/engine" + "github.com/cockroachdb/cockroach/pkg/util/bufalloc" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -53,20 +50,23 @@ type poller struct { } func makePoller( - execCfg *sql.ExecutorConfig, - details jobspb.ChangefeedDetails, + settings *cluster.Settings, + db *client.DB, + clock *hlc.Clock, + gossip *gossip.Gossip, spans []roachpb.Span, + targets map[sqlbase.ID]string, startTime hlc.Timestamp, buf *buffer, ) *poller { return &poller{ - settings: execCfg.Settings, - db: execCfg.DB, - clock: execCfg.Clock, - gossip: execCfg.Gossip, + settings: settings, + db: db, + clock: clock, + gossip: gossip, highWater: startTime, spans: spans, - targets: details.Targets, + targets: targets, buf: buf, } } diff --git a/pkg/ccl/changefeedccl/sink.go b/pkg/ccl/changefeedccl/sink.go index 4e97627dabe5..d3bd069359a4 100644 --- a/pkg/ccl/changefeedccl/sink.go +++ b/pkg/ccl/changefeedccl/sink.go @@ -46,17 +46,17 @@ type Sink interface { Close() error } -func getSink(sinkURI string, resultsCh chan<- tree.Datums) (Sink, error) { +func getSink(sinkURI string, targets map[sqlbase.ID]string) (Sink, error) { u, err := url.Parse(sinkURI) if err != nil { return nil, err } switch u.Scheme { - case sinkSchemeChannel: - return &channelSink{resultsCh: resultsCh}, nil + case sinkSchemeBuffer: + return &bufferSink{}, nil case sinkSchemeKafka: kafkaTopicPrefix := u.Query().Get(sinkParamTopicPrefix) - return getKafkaSink(kafkaTopicPrefix, u.Host) + return getKafkaSink(kafkaTopicPrefix, u.Host, targets) default: return nil, errors.Errorf(`unsupported sink: %s`, u.Scheme) } @@ -72,7 +72,7 @@ type kafkaSink struct { kafkaTopicPrefix string client sarama.Client producer sarama.AsyncProducer - topicsSeen map[string]struct{} + topics map[string]struct{} stopWorkerCh chan struct{} worker sync.WaitGroup @@ -86,10 +86,15 @@ type kafkaSink struct { } } -func getKafkaSink(kafkaTopicPrefix string, bootstrapServers string) (Sink, error) { +func getKafkaSink( + kafkaTopicPrefix string, bootstrapServers string, targets map[sqlbase.ID]string, +) (Sink, error) { sink := &kafkaSink{ kafkaTopicPrefix: kafkaTopicPrefix, - topicsSeen: make(map[string]struct{}), + } + sink.topics = make(map[string]struct{}) + for _, tableName := range targets { + sink.topics[kafkaTopicPrefix+tableName] = struct{}{} } config := sarama.NewConfig() @@ -192,11 +197,12 @@ func (s *kafkaSink) Close() error { } // EmitRow implements the Sink interface. -func (s *kafkaSink) EmitRow(ctx context.Context, topic string, key, value []byte) error { - topic = s.kafkaTopicPrefix + topic - if _, ok := s.topicsSeen[topic]; !ok { - s.topicsSeen[topic] = struct{}{} +func (s *kafkaSink) EmitRow(ctx context.Context, tableName string, key, value []byte) error { + topic := s.kafkaTopicPrefix + tableName + if _, ok := s.topics[topic]; !ok { + return errors.Errorf(`cannot emit to undeclared topic: %s`, topic) } + msg := &sarama.ProducerMessage{ Topic: topic, Key: sarama.ByteEncoder(key), @@ -210,7 +216,7 @@ func (s *kafkaSink) EmitResolvedTimestamp(ctx context.Context, payload []byte) e // Staleness here does not impact correctness. Some new partitions will miss // this resolved timestamp, but they'll eventually be picked up and get // later ones. - for topic := range s.topicsSeen { + for topic := range s.topics { // TODO(dan): Figure out how expensive this is to call. Maybe we need to // cache it and rate limit? partitions, err := s.client.Partitions(topic) @@ -332,48 +338,66 @@ func (p *changefeedPartitioner) Partition( return p.hash.Partition(message, numPartitions) } -type channelSink struct { - resultsCh chan<- tree.Datums - alloc sqlbase.DatumAlloc +// encDatumRowBuffer is a FIFO of `EncDatumRow`s. +// +// TODO(dan): There's some potential allocation savings here by reusing the same +// backing array. +type encDatumRowBuffer []sqlbase.EncDatumRow + +func (b *encDatumRowBuffer) IsEmpty() bool { + return b == nil || len(*b) == 0 +} +func (b *encDatumRowBuffer) Push(r sqlbase.EncDatumRow) { + *b = append(*b, r) +} +func (b *encDatumRowBuffer) Pop() sqlbase.EncDatumRow { + ret := (*b)[0] + *b = (*b)[1:] + return ret +} + +type bufferSink struct { + buf encDatumRowBuffer + alloc sqlbase.DatumAlloc + closed bool } // EmitRow implements the Sink interface. -func (s *channelSink) EmitRow(ctx context.Context, topic string, key, value []byte) error { - return s.emitDatums(ctx, tree.Datums{ - s.alloc.NewDString(tree.DString(topic)), - s.alloc.NewDBytes(tree.DBytes(key)), - s.alloc.NewDBytes(tree.DBytes(value)), +func (s *bufferSink) EmitRow(_ context.Context, topic string, key, value []byte) error { + if s.closed { + return errors.New(`cannot EmitRow on a closed sink`) + } + s.buf.Push(sqlbase.EncDatumRow{ + {Datum: tree.DNull}, // resolved span + {Datum: s.alloc.NewDString(tree.DString(topic))}, // topic + {Datum: s.alloc.NewDBytes(tree.DBytes(key))}, // key + {Datum: s.alloc.NewDBytes(tree.DBytes(value))}, //value }) + return nil } // EmitResolvedTimestamp implements the Sink interface. -func (s *channelSink) EmitResolvedTimestamp(ctx context.Context, payload []byte) error { - return s.emitDatums(ctx, tree.Datums{ - tree.DNull, - tree.DNull, - s.alloc.NewDBytes(tree.DBytes(payload)), +func (s *bufferSink) EmitResolvedTimestamp(_ context.Context, payload []byte) error { + if s.closed { + return errors.New(`cannot EmitRow on a closed sink`) + } + s.buf.Push(sqlbase.EncDatumRow{ + {Datum: tree.DNull}, // resolved span + {Datum: tree.DNull}, // topic + {Datum: tree.DNull}, // key + {Datum: s.alloc.NewDBytes(tree.DBytes(payload))}, // value }) -} - -func (s *channelSink) emitDatums(ctx context.Context, row tree.Datums) error { - select { - case <-ctx.Done(): - return ctx.Err() - case s.resultsCh <- row: - return nil - } + return nil } // Flush implements the Sink interface. -func (s *channelSink) Flush(_ context.Context) error { +func (s *bufferSink) Flush(_ context.Context) error { return nil } // Close implements the Sink interface. -func (s *channelSink) Close() error { - // nil the channel so any later calls to EmitRow (there shouldn't be any) - // don't work. - s.resultsCh = nil +func (s *bufferSink) Close() error { + s.closed = true return nil } @@ -407,6 +431,7 @@ func (s *metricsSink) EmitResolvedTimestamp(ctx context.Context, payload []byte) } return err } + func (s *metricsSink) Flush(ctx context.Context) error { start := timeutil.Now() err := s.wrapped.Flush(ctx) @@ -415,6 +440,7 @@ func (s *metricsSink) Flush(ctx context.Context) error { } return err } + func (s *metricsSink) Close() error { return s.wrapped.Close() } diff --git a/pkg/ccl/changefeedccl/sink_test.go b/pkg/ccl/changefeedccl/sink_test.go index e28a679dd473..bc6d7cedbc38 100644 --- a/pkg/ccl/changefeedccl/sink_test.go +++ b/pkg/ccl/changefeedccl/sink_test.go @@ -45,8 +45,8 @@ func TestKafkaSink(t *testing.T) { errorsCh: make(chan *sarama.ProducerError, 1), } sink := &kafkaSink{ - producer: p, - topicsSeen: make(map[string]struct{}), + producer: p, + topics: map[string]struct{}{`t`: {}}, } sink.start() defer func() { diff --git a/pkg/internal/client/db.go b/pkg/internal/client/db.go index 61f7ed4a2e54..5a033fd8e9a9 100644 --- a/pkg/internal/client/db.go +++ b/pkg/internal/client/db.go @@ -266,6 +266,11 @@ func (db *DB) GetFactory() TxnSenderFactory { return db.factory } +// Clock returns the DB's hlc.Clock. +func (db *DB) Clock() *hlc.Clock { + return db.clock +} + // NewDB returns a new DB. func NewDB(actx log.AmbientContext, factory TxnSenderFactory, clock *hlc.Clock) *DB { return NewDBWithContext(actx, factory, clock, DefaultDBContext()) diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go index 8cf1a17a0da7..b206fe3eae25 100644 --- a/pkg/jobs/jobs.go +++ b/pkg/jobs/jobs.go @@ -389,13 +389,6 @@ func (j *Job) Details() jobspb.Details { return j.mu.payload.UnwrapDetails() } -// RegistryMetrics returns the metrics for production monitoring of each job -// type. They're all stored as the `metric.Struct` interface because of -// dependency cycles. -func (j *Job) RegistryMetrics() *Metrics { - return &j.registry.metrics -} - // FractionCompleted returns completion according to the in-memory job state. func (j *Job) FractionCompleted() float32 { progress := j.Progress() diff --git a/pkg/server/server.go b/pkg/server/server.go index d852086e3d55..89d5efd84a16 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -502,9 +502,10 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { Metrics: &distSQLMetrics, - JobRegistry: s.jobRegistry, - Gossip: s.gossip, - NodeDialer: s.nodeDialer, + JobRegistry: s.jobRegistry, + Gossip: s.gossip, + NodeDialer: s.nodeDialer, + LeaseManager: s.leaseMgr, } if distSQLTestingKnobs := s.cfg.TestingKnobs.DistSQL; distSQLTestingKnobs != nil { distSQLCfg.TestingKnobs = *distSQLTestingKnobs.(*distsqlrun.TestingKnobs) diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index fda98cc6e441..a6a898cd0f1e 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -517,6 +517,7 @@ func (sc *SchemaChanger) distBackfill( &planCtx, nil, /* txn - the processors manage their own transactions */ &plan, recv, evalCtx, + nil, /* finishedSetupFn */ ) return rw.Err() }); err != nil { diff --git a/pkg/sql/distsql_plan_csv.go b/pkg/sql/distsql_plan_csv.go index 77660b9b203f..aed6da65c8e2 100644 --- a/pkg/sql/distsql_plan_csv.go +++ b/pkg/sql/distsql_plan_csv.go @@ -80,7 +80,7 @@ func PlanAndRunExport( evalCtx.Tracing, ) - dsp.Run(&planCtx, txn, &p, recv, evalCtx) + dsp.Run(&planCtx, txn, &p, recv, evalCtx, nil /* finishedSetupFn */) return resultRows.Err() } @@ -461,7 +461,7 @@ func LoadCSV( defer log.VEventf(ctx, 1, "finished job %s", job.Payload().Description) return db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { - dsp.Run(&planCtx, txn, &p, recv, evalCtx) + dsp.Run(&planCtx, txn, &p, recv, evalCtx, nil /* finishedSetupFn */) return resultRows.Err() }) } @@ -601,7 +601,7 @@ func (dsp *DistSQLPlanner) loadCSVSamplingPlan( log.VEventf(ctx, 1, "begin sampling phase of job %s", job.Payload().Description) // Clear the stage 2 data in case this function is ever restarted (it shouldn't be). samples = nil - dsp.Run(planCtx, nil, &p, recv, evalCtx) + dsp.Run(planCtx, nil, &p, recv, evalCtx, nil /* finishedSetupFn */) if err := rowResultWriter.Err(); err != nil { return nil, err } diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 7dff2d44d53c..91a619ae688c 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -101,6 +101,9 @@ func (dsp *DistSQLPlanner) initRunners() { // txn is the transaction in which the plan will run. If nil, the different // processors are expected to manage their own internal transactions. // +// `finishedSetupFn`, if non-nil, is called synchronously after all the +// processors have successfully started up. +// // All errors encountered are reported to the DistSQLReceiver's resultWriter. // Additionally, if the error is a "communication error" (an error encountered // while using that resultWriter), the error is also stored in @@ -112,6 +115,7 @@ func (dsp *DistSQLPlanner) Run( plan *PhysicalPlan, recv *DistSQLReceiver, evalCtx *extendedEvalContext, + finishedSetupFn func(), ) { ctx := planCtx.ctx @@ -231,6 +235,10 @@ func (dsp *DistSQLPlanner) Run( return } + if finishedSetupFn != nil { + finishedSetupFn() + } + // TODO(radu): this should go through the flow scheduler. if err := flow.StartSync(ctx, func() {}); err != nil { log.Fatalf(ctx, "unexpected error from syncFlow.Start(): %s "+ @@ -580,5 +588,5 @@ func (dsp *DistSQLPlanner) PlanAndRun( return } dsp.FinalizePlan(&planCtx, &physPlan) - dsp.Run(&planCtx, txn, &physPlan, recv, evalCtx) + dsp.Run(&planCtx, txn, &physPlan, recv, evalCtx, nil /* finishedSetupFn */) } diff --git a/pkg/sql/distsqlrun/api.pb.go b/pkg/sql/distsqlrun/api.pb.go index eb3920689731..fa876961def5 100644 --- a/pkg/sql/distsqlrun/api.pb.go +++ b/pkg/sql/distsqlrun/api.pb.go @@ -61,6 +61,8 @@ InterleavedReaderJoinerSpec ProjectSetSpec WindowerSpec + ChangeAggregatorSpec + ChangeFrontierSpec InputStats TableReaderStats HashJoinerStats diff --git a/pkg/sql/distsqlrun/flow.go b/pkg/sql/distsqlrun/flow.go index 10cdd8e9e02e..39580a9df826 100644 --- a/pkg/sql/distsqlrun/flow.go +++ b/pkg/sql/distsqlrun/flow.go @@ -95,6 +95,10 @@ type FlowCtx struct { // whereas this one isn't. executor sqlutil.InternalExecutor + // LeaseManager is a *sql.LeaseManager. It's returned as an `interface{}` + // due to package dependency cycles + LeaseManager interface{} + // nodeID is the ID of the node on which the processors using this FlowCtx // run. nodeID roachpb.NodeID diff --git a/pkg/sql/distsqlrun/flow_diagram.go b/pkg/sql/distsqlrun/flow_diagram.go index 203f3504a391..917e88767411 100644 --- a/pkg/sql/distsqlrun/flow_diagram.go +++ b/pkg/sql/distsqlrun/flow_diagram.go @@ -455,6 +455,20 @@ func (w *WindowerSpec) summary() (string, []string) { return "Windower", details } +// summary implements the diagramCellType interface. +func (s *ChangeAggregatorSpec) summary() (string, []string) { + var details []string + for _, watch := range s.Watches { + details = append(details, watch.Span.String()) + } + return "ChangeAggregator", details +} + +// summary implements the diagramCellType interface. +func (s *ChangeFrontierSpec) summary() (string, []string) { + return "ChangeFrontier", []string{} +} + type diagramCell struct { Title string `json:"title"` Details []string `json:"details"` diff --git a/pkg/sql/distsqlrun/processors.go b/pkg/sql/distsqlrun/processors.go index 7b55f05fb9a0..f3527d055d04 100644 --- a/pkg/sql/distsqlrun/processors.go +++ b/pkg/sql/distsqlrun/processors.go @@ -1062,6 +1062,24 @@ func newProcessor( err := processor.InitWithOutput(post, outputs[0]) return processor, err } + if core.ChangeAggregator != nil { + if err := checkNumInOut(inputs, outputs, 0, 1); err != nil { + return nil, err + } + if NewChangeAggregatorProcessor == nil { + return nil, errors.New("ChangeAggregator processor unimplemented") + } + return NewChangeAggregatorProcessor(flowCtx, processorID, *core.ChangeAggregator, outputs[0]) + } + if core.ChangeFrontier != nil { + if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { + return nil, err + } + if NewChangeFrontierProcessor == nil { + return nil, errors.New("ChangeFrontier processor unimplemented") + } + return NewChangeFrontierProcessor(flowCtx, processorID, *core.ChangeFrontier, inputs[0], outputs[0]) + } return nil, errors.Errorf("unsupported processor core %s", core) } @@ -1089,6 +1107,12 @@ var NewSSTWriterProcessor func(*FlowCtx, int32, SSTWriterSpec, RowSource, RowRec // NewCSVWriterProcessor is externally implemented. var NewCSVWriterProcessor func(*FlowCtx, int32, CSVWriterSpec, RowSource, RowReceiver) (Processor, error) +// NewChangeAggregatorProcessor is externally implemented. +var NewChangeAggregatorProcessor func(*FlowCtx, int32, ChangeAggregatorSpec, RowReceiver) (Processor, error) + +// NewChangeFrontierProcessor is externally implemented. +var NewChangeFrontierProcessor func(*FlowCtx, int32, ChangeFrontierSpec, RowSource, RowReceiver) (Processor, error) + // Equals returns true if two aggregation specifiers are identical (and thus // will always yield the same result). func (a AggregatorSpec_Aggregation) Equals(b AggregatorSpec_Aggregation) bool { diff --git a/pkg/sql/distsqlrun/processors.pb.go b/pkg/sql/distsqlrun/processors.pb.go index 484940c24d86..50ac01a34950 100644 --- a/pkg/sql/distsqlrun/processors.pb.go +++ b/pkg/sql/distsqlrun/processors.pb.go @@ -6,6 +6,7 @@ package distsqlrun import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" +import cockroach_sql_jobs_jobspb "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" import cockroach_roachpb4 "github.com/cockroachdb/cockroach/pkg/roachpb" import cockroach_sql_sqlbase1 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" @@ -535,6 +536,8 @@ type ProcessorCoreUnion struct { ProjectSet *ProjectSetSpec `protobuf:"bytes,22,opt,name=projectSet" json:"projectSet,omitempty"` Windower *WindowerSpec `protobuf:"bytes,23,opt,name=windower" json:"windower,omitempty"` LocalPlanNode *LocalPlanNodeSpec `protobuf:"bytes,24,opt,name=localPlanNode" json:"localPlanNode,omitempty"` + ChangeAggregator *ChangeAggregatorSpec `protobuf:"bytes,25,opt,name=changeAggregator" json:"changeAggregator,omitempty"` + ChangeFrontier *ChangeFrontierSpec `protobuf:"bytes,26,opt,name=changeFrontier" json:"changeFrontier,omitempty"` } func (m *ProcessorCoreUnion) Reset() { *m = ProcessorCoreUnion{} } @@ -1376,6 +1379,50 @@ func (*WindowerSpec_WindowFn) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{29, 2} } +// ChangeAggregatorSpec is the specification for a processor that watches for +// changes in a set of spans. Each span may cross multiple ranges. +type ChangeAggregatorSpec struct { + Watches []ChangeAggregatorSpec_Watch `protobuf:"bytes,1,rep,name=watches" json:"watches"` + // Feed is the specification for this changefeed. + Feed cockroach_sql_jobs_jobspb.ChangefeedDetails `protobuf:"bytes,2,opt,name=feed" json:"feed"` +} + +func (m *ChangeAggregatorSpec) Reset() { *m = ChangeAggregatorSpec{} } +func (m *ChangeAggregatorSpec) String() string { return proto.CompactTextString(m) } +func (*ChangeAggregatorSpec) ProtoMessage() {} +func (*ChangeAggregatorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{30} } + +type ChangeAggregatorSpec_Watch struct { + InitialResolved cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=initial_resolved,json=initialResolved" json:"initial_resolved"` + Span cockroach_roachpb1.Span `protobuf:"bytes,2,opt,name=span" json:"span"` +} + +func (m *ChangeAggregatorSpec_Watch) Reset() { *m = ChangeAggregatorSpec_Watch{} } +func (m *ChangeAggregatorSpec_Watch) String() string { return proto.CompactTextString(m) } +func (*ChangeAggregatorSpec_Watch) ProtoMessage() {} +func (*ChangeAggregatorSpec_Watch) Descriptor() ([]byte, []int) { + return fileDescriptorProcessors, []int{30, 0} +} + +// ChangeFrontierSpec is the specification for a processor that receives +// span-level resolved timestamps, track them, and emits the changefeed-level +// resolved timestamp whenever it changes. +type ChangeFrontierSpec struct { + // TrackedSpans is the entire span set being watched. Once all these spans + // have been resolved at a certain timestamp, then it's safe to resolve the + // changefeed at that timestamp. + TrackedSpans []cockroach_roachpb1.Span `protobuf:"bytes,1,rep,name=tracked_spans,json=trackedSpans" json:"tracked_spans"` + // Feed is the specification for this changefeed. + Feed cockroach_sql_jobs_jobspb.ChangefeedDetails `protobuf:"bytes,2,opt,name=feed" json:"feed"` + // JobID is the id of this changefeed in the system jobs. + JobID int64 `protobuf:"varint,3,opt,name=job_id,json=jobId" json:"job_id"` +} + +func (m *ChangeFrontierSpec) Reset() { *m = ChangeFrontierSpec{} } +func (m *ChangeFrontierSpec) String() string { return proto.CompactTextString(m) } +func (*ChangeFrontierSpec) ProtoMessage() {} +func (*ChangeFrontierSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{31} } + func init() { proto.RegisterType((*ProcessorSpec)(nil), "cockroach.sql.distsqlrun.ProcessorSpec") proto.RegisterType((*PostProcessSpec)(nil), "cockroach.sql.distsqlrun.PostProcessSpec") @@ -1415,6 +1462,9 @@ func init() { proto.RegisterType((*WindowerSpec_Frame_Bound)(nil), "cockroach.sql.distsqlrun.WindowerSpec.Frame.Bound") proto.RegisterType((*WindowerSpec_Frame_Bounds)(nil), "cockroach.sql.distsqlrun.WindowerSpec.Frame.Bounds") proto.RegisterType((*WindowerSpec_WindowFn)(nil), "cockroach.sql.distsqlrun.WindowerSpec.WindowFn") + proto.RegisterType((*ChangeAggregatorSpec)(nil), "cockroach.sql.distsqlrun.ChangeAggregatorSpec") + proto.RegisterType((*ChangeAggregatorSpec_Watch)(nil), "cockroach.sql.distsqlrun.ChangeAggregatorSpec.Watch") + proto.RegisterType((*ChangeFrontierSpec)(nil), "cockroach.sql.distsqlrun.ChangeFrontierSpec") proto.RegisterEnum("cockroach.sql.distsqlrun.ScanVisibility", ScanVisibility_name, ScanVisibility_value) proto.RegisterEnum("cockroach.sql.distsqlrun.SketchType", SketchType_name, SketchType_value) proto.RegisterEnum("cockroach.sql.distsqlrun.AggregatorSpec_Func", AggregatorSpec_Func_name, AggregatorSpec_Func_value) @@ -1810,6 +1860,30 @@ func (m *ProcessorCoreUnion) MarshalTo(dAtA []byte) (int, error) { } i += n27 } + if m.ChangeAggregator != nil { + dAtA[i] = 0xca + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.ChangeAggregator.Size())) + n28, err := m.ChangeAggregator.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n28 + } + if m.ChangeFrontier != nil { + dAtA[i] = 0xd2 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.ChangeFrontier.Size())) + n29, err := m.ChangeFrontier.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n29 + } return i, nil } @@ -1945,11 +2019,11 @@ func (m *TableReaderSpan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Span.Size())) - n28, err := m.Span.MarshalTo(dAtA[i:]) + n30, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n30 return i, nil } @@ -1971,11 +2045,11 @@ func (m *TableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Table.Size())) - n29, err := m.Table.MarshalTo(dAtA[i:]) + n31, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n31 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexIdx)) @@ -2034,47 +2108,47 @@ func (m *JoinReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Table.Size())) - n30, err := m.Table.MarshalTo(dAtA[i:]) + n32, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n30 + i += n32 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexIdx)) if len(m.LookupColumns) > 0 { - dAtA32 := make([]byte, len(m.LookupColumns)*10) - var j31 int + dAtA34 := make([]byte, len(m.LookupColumns)*10) + var j33 int for _, num := range m.LookupColumns { for num >= 1<<7 { - dAtA32[j31] = uint8(uint64(num)&0x7f | 0x80) + dAtA34[j33] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j31++ + j33++ } - dAtA32[j31] = uint8(num) - j31++ + dAtA34[j33] = uint8(num) + j33++ } dAtA[i] = 0x1a i++ - i = encodeVarintProcessors(dAtA, i, uint64(j31)) - i += copy(dAtA[i:], dAtA32[:j31]) + i = encodeVarintProcessors(dAtA, i, uint64(j33)) + i += copy(dAtA[i:], dAtA34[:j33]) } dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n33, err := m.OnExpr.MarshalTo(dAtA[i:]) + n35, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n35 dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexFilterExpr.Size())) - n34, err := m.IndexFilterExpr.MarshalTo(dAtA[i:]) + n36, err := m.IndexFilterExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n36 dAtA[i] = 0x30 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2102,11 +2176,11 @@ func (m *SorterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OutputOrdering.Size())) - n35, err := m.OutputOrdering.MarshalTo(dAtA[i:]) + n37, err := m.OutputOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n35 + i += n37 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OrderingMatchLen)) @@ -2185,30 +2259,30 @@ func (m *ZigzagJoinerSpec) MarshalTo(dAtA []byte) (int, error) { } } if len(m.IndexIds) > 0 { - dAtA37 := make([]byte, len(m.IndexIds)*10) - var j36 int + dAtA39 := make([]byte, len(m.IndexIds)*10) + var j38 int for _, num := range m.IndexIds { for num >= 1<<7 { - dAtA37[j36] = uint8(uint64(num)&0x7f | 0x80) + dAtA39[j38] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j36++ + j38++ } - dAtA37[j36] = uint8(num) - j36++ + dAtA39[j38] = uint8(num) + j38++ } dAtA[i] = 0x1a i++ - i = encodeVarintProcessors(dAtA, i, uint64(j36)) - i += copy(dAtA[i:], dAtA37[:j36]) + i = encodeVarintProcessors(dAtA, i, uint64(j38)) + i += copy(dAtA[i:], dAtA39[:j38]) } dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n38, err := m.OnExpr.MarshalTo(dAtA[i:]) + n40, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n38 + i += n40 dAtA[i] = 0x28 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2265,21 +2339,21 @@ func (m *Columns) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.Columns) > 0 { - dAtA40 := make([]byte, len(m.Columns)*10) - var j39 int + dAtA42 := make([]byte, len(m.Columns)*10) + var j41 int for _, num := range m.Columns { for num >= 1<<7 { - dAtA40[j39] = uint8(uint64(num)&0x7f | 0x80) + dAtA42[j41] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j39++ + j41++ } - dAtA40[j39] = uint8(num) - j39++ + dAtA42[j41] = uint8(num) + j41++ } dAtA[i] = 0xa i++ - i = encodeVarintProcessors(dAtA, i, uint64(j39)) - i += copy(dAtA[i:], dAtA40[:j39]) + i = encodeVarintProcessors(dAtA, i, uint64(j41)) + i += copy(dAtA[i:], dAtA42[:j41]) } return i, nil } @@ -2302,27 +2376,27 @@ func (m *MergeJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.LeftOrdering.Size())) - n41, err := m.LeftOrdering.MarshalTo(dAtA[i:]) + n43, err := m.LeftOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n43 dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.RightOrdering.Size())) - n42, err := m.RightOrdering.MarshalTo(dAtA[i:]) + n44, err := m.RightOrdering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n44 dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n43, err := m.OnExpr.MarshalTo(dAtA[i:]) + n45, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n43 + i += n45 dAtA[i] = 0x30 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2353,47 +2427,47 @@ func (m *HashJoinerSpec) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.LeftEqColumns) > 0 { - dAtA45 := make([]byte, len(m.LeftEqColumns)*10) - var j44 int + dAtA47 := make([]byte, len(m.LeftEqColumns)*10) + var j46 int for _, num := range m.LeftEqColumns { for num >= 1<<7 { - dAtA45[j44] = uint8(uint64(num)&0x7f | 0x80) + dAtA47[j46] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j44++ + j46++ } - dAtA45[j44] = uint8(num) - j44++ + dAtA47[j46] = uint8(num) + j46++ } dAtA[i] = 0xa i++ - i = encodeVarintProcessors(dAtA, i, uint64(j44)) - i += copy(dAtA[i:], dAtA45[:j44]) + i = encodeVarintProcessors(dAtA, i, uint64(j46)) + i += copy(dAtA[i:], dAtA47[:j46]) } if len(m.RightEqColumns) > 0 { - dAtA47 := make([]byte, len(m.RightEqColumns)*10) - var j46 int + dAtA49 := make([]byte, len(m.RightEqColumns)*10) + var j48 int for _, num := range m.RightEqColumns { for num >= 1<<7 { - dAtA47[j46] = uint8(uint64(num)&0x7f | 0x80) + dAtA49[j48] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j46++ + j48++ } - dAtA47[j46] = uint8(num) - j46++ + dAtA49[j48] = uint8(num) + j48++ } dAtA[i] = 0x12 i++ - i = encodeVarintProcessors(dAtA, i, uint64(j46)) - i += copy(dAtA[i:], dAtA47[:j46]) + i = encodeVarintProcessors(dAtA, i, uint64(j48)) + i += copy(dAtA[i:], dAtA49[:j48]) } dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n48, err := m.OnExpr.MarshalTo(dAtA[i:]) + n50, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 + i += n50 dAtA[i] = 0x30 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -2424,21 +2498,21 @@ func (m *AggregatorSpec) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.GroupCols) > 0 { - dAtA50 := make([]byte, len(m.GroupCols)*10) - var j49 int + dAtA52 := make([]byte, len(m.GroupCols)*10) + var j51 int for _, num := range m.GroupCols { for num >= 1<<7 { - dAtA50[j49] = uint8(uint64(num)&0x7f | 0x80) + dAtA52[j51] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j49++ + j51++ } - dAtA50[j49] = uint8(num) - j49++ + dAtA52[j51] = uint8(num) + j51++ } dAtA[i] = 0x12 i++ - i = encodeVarintProcessors(dAtA, i, uint64(j49)) - i += copy(dAtA[i:], dAtA50[:j49]) + i = encodeVarintProcessors(dAtA, i, uint64(j51)) + i += copy(dAtA[i:], dAtA52[:j51]) } if len(m.Aggregations) > 0 { for _, msg := range m.Aggregations { @@ -2453,21 +2527,21 @@ func (m *AggregatorSpec) MarshalTo(dAtA []byte) (int, error) { } } if len(m.OrderedGroupCols) > 0 { - dAtA52 := make([]byte, len(m.OrderedGroupCols)*10) - var j51 int + dAtA54 := make([]byte, len(m.OrderedGroupCols)*10) + var j53 int for _, num := range m.OrderedGroupCols { for num >= 1<<7 { - dAtA52[j51] = uint8(uint64(num)&0x7f | 0x80) + dAtA54[j53] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j51++ + j53++ } - dAtA52[j51] = uint8(num) - j51++ + dAtA54[j53] = uint8(num) + j53++ } dAtA[i] = 0x22 i++ - i = encodeVarintProcessors(dAtA, i, uint64(j51)) - i += copy(dAtA[i:], dAtA52[:j51]) + i = encodeVarintProcessors(dAtA, i, uint64(j53)) + i += copy(dAtA[i:], dAtA54[:j53]) } dAtA[i] = 0x28 i++ @@ -2549,11 +2623,11 @@ func (m *BackfillerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Table.Size())) - n53, err := m.Table.MarshalTo(dAtA[i:]) + n55, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n55 if len(m.Spans) > 0 { for _, msg := range m.Spans { dAtA[i] = 0x1a @@ -2587,11 +2661,11 @@ func (m *BackfillerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.ReadAsOf.Size())) - n54, err := m.ReadAsOf.MarshalTo(dAtA[i:]) + n56, err := m.ReadAsOf.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n56 return i, nil } @@ -2613,11 +2687,11 @@ func (m *FlowSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.FlowID.Size())) - n55, err := m.FlowID.MarshalTo(dAtA[i:]) + n57, err := m.FlowID.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n57 if len(m.Processors) > 0 { for _, msg := range m.Processors { dAtA[i] = 0x12 @@ -2685,11 +2759,11 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Progress.Size())) - n56, err := m.Progress.MarshalTo(dAtA[i:]) + n58, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n58 if len(m.Uri) > 0 { keysForUri := make([]int32, 0, len(m.Uri)) for k := range m.Uri { @@ -2714,11 +2788,11 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Format.Size())) - n57, err := m.Format.MarshalTo(dAtA[i:]) + n59, err := m.Format.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n59 if len(m.Tables) > 0 { keysForTables := make([]string, 0, len(m.Tables)) for k := range m.Tables { @@ -2744,11 +2818,11 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(v.Size())) - n58, err := v.MarshalTo(dAtA[i:]) + n60, err := v.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n58 + i += n60 } } } @@ -2800,11 +2874,11 @@ func (m *SSTWriterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Progress.Size())) - n59, err := m.Progress.MarshalTo(dAtA[i:]) + n61, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n59 + i += n61 return i, nil } @@ -2862,11 +2936,11 @@ func (m *CSVWriterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Options.Size())) - n60, err := m.Options.MarshalTo(dAtA[i:]) + n62, err := m.Options.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n60 + i += n62 dAtA[i] = 0x20 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.ChunkRows)) @@ -3033,11 +3107,11 @@ func (m *InterleavedReaderJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OnExpr.Size())) - n61, err := m.OnExpr.MarshalTo(dAtA[i:]) + n63, err := m.OnExpr.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n61 + i += n63 dAtA[i] = 0x28 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Type)) @@ -3062,30 +3136,30 @@ func (m *InterleavedReaderJoinerSpec_Table) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Desc.Size())) - n62, err := m.Desc.MarshalTo(dAtA[i:]) + n64, err := m.Desc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n62 + i += n64 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.IndexIdx)) dAtA[i] = 0x1a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Post.Size())) - n63, err := m.Post.MarshalTo(dAtA[i:]) + n65, err := m.Post.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n63 + i += n65 dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Ordering.Size())) - n64, err := m.Ordering.MarshalTo(dAtA[i:]) + n66, err := m.Ordering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n64 + i += n66 if len(m.Spans) > 0 { for _, msg := range m.Spans { dAtA[i] = 0x2a @@ -3236,11 +3310,11 @@ func (m *WindowerSpec_Frame) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Bounds.Size())) - n65, err := m.Bounds.MarshalTo(dAtA[i:]) + n67, err := m.Bounds.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n65 + i += n67 return i, nil } @@ -3274,11 +3348,11 @@ func (m *WindowerSpec_Frame_Bound) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.OffsetType.Size())) - n66, err := m.OffsetType.MarshalTo(dAtA[i:]) + n68, err := m.OffsetType.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n66 + i += n68 return i, nil } @@ -3300,20 +3374,20 @@ func (m *WindowerSpec_Frame_Bounds) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Start.Size())) - n67, err := m.Start.MarshalTo(dAtA[i:]) + n69, err := m.Start.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n67 + i += n69 if m.End != nil { dAtA[i] = 0x12 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.End.Size())) - n68, err := m.End.MarshalTo(dAtA[i:]) + n70, err := m.End.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n68 + i += n70 } return i, nil } @@ -3336,11 +3410,11 @@ func (m *WindowerSpec_WindowFn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Func.Size())) - n69, err := m.Func.MarshalTo(dAtA[i:]) + n71, err := m.Func.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n69 + i += n71 dAtA[i] = 0x10 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.ArgIdxStart)) @@ -3350,20 +3424,20 @@ func (m *WindowerSpec_WindowFn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Ordering.Size())) - n70, err := m.Ordering.MarshalTo(dAtA[i:]) + n72, err := m.Ordering.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n70 + i += n72 if m.Frame != nil { dAtA[i] = 0x2a i++ i = encodeVarintProcessors(dAtA, i, uint64(m.Frame.Size())) - n71, err := m.Frame.MarshalTo(dAtA[i:]) + n73, err := m.Frame.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n71 + i += n73 } dAtA[i] = 0x30 i++ @@ -3371,6 +3445,119 @@ func (m *WindowerSpec_WindowFn) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *ChangeAggregatorSpec) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ChangeAggregatorSpec) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Watches) > 0 { + for _, msg := range m.Watches { + dAtA[i] = 0xa + i++ + i = encodeVarintProcessors(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + dAtA[i] = 0x12 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Feed.Size())) + n74, err := m.Feed.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n74 + return i, nil +} + +func (m *ChangeAggregatorSpec_Watch) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ChangeAggregatorSpec_Watch) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.InitialResolved.Size())) + n75, err := m.InitialResolved.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n75 + dAtA[i] = 0x12 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Span.Size())) + n76, err := m.Span.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n76 + return i, nil +} + +func (m *ChangeFrontierSpec) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ChangeFrontierSpec) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.TrackedSpans) > 0 { + for _, msg := range m.TrackedSpans { + dAtA[i] = 0xa + i++ + i = encodeVarintProcessors(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + dAtA[i] = 0x12 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.Feed.Size())) + n77, err := m.Feed.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n77 + dAtA[i] = 0x18 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.JobID)) + return i, nil +} + func encodeVarintProcessors(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -3519,6 +3706,14 @@ func (m *ProcessorCoreUnion) Size() (n int) { l = m.LocalPlanNode.Size() n += 2 + l + sovProcessors(uint64(l)) } + if m.ChangeAggregator != nil { + l = m.ChangeAggregator.Size() + n += 2 + l + sovProcessors(uint64(l)) + } + if m.ChangeFrontier != nil { + l = m.ChangeFrontier.Size() + n += 2 + l + sovProcessors(uint64(l)) + } return n } @@ -4097,6 +4292,45 @@ func (m *WindowerSpec_WindowFn) Size() (n int) { return n } +func (m *ChangeAggregatorSpec) Size() (n int) { + var l int + _ = l + if len(m.Watches) > 0 { + for _, e := range m.Watches { + l = e.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + } + l = m.Feed.Size() + n += 1 + l + sovProcessors(uint64(l)) + return n +} + +func (m *ChangeAggregatorSpec_Watch) Size() (n int) { + var l int + _ = l + l = m.InitialResolved.Size() + n += 1 + l + sovProcessors(uint64(l)) + l = m.Span.Size() + n += 1 + l + sovProcessors(uint64(l)) + return n +} + +func (m *ChangeFrontierSpec) Size() (n int) { + var l int + _ = l + if len(m.TrackedSpans) > 0 { + for _, e := range m.TrackedSpans { + l = e.Size() + n += 1 + l + sovProcessors(uint64(l)) + } + } + l = m.Feed.Size() + n += 1 + l + sovProcessors(uint64(l)) + n += 1 + sovProcessors(uint64(m.JobID)) + return n +} + func sovProcessors(x uint64) (n int) { for { n++ @@ -4177,6 +4411,12 @@ func (this *ProcessorCoreUnion) GetValue() interface{} { if this.LocalPlanNode != nil { return this.LocalPlanNode } + if this.ChangeAggregator != nil { + return this.ChangeAggregator + } + if this.ChangeFrontier != nil { + return this.ChangeFrontier + } return nil } @@ -4226,6 +4466,10 @@ func (this *ProcessorCoreUnion) SetValue(value interface{}) bool { this.Windower = vt case *LocalPlanNodeSpec: this.LocalPlanNode = vt + case *ChangeAggregatorSpec: + this.ChangeAggregator = vt + case *ChangeFrontierSpec: + this.ChangeFrontier = vt default: return false } @@ -5448,39 +5692,105 @@ func (m *ProcessorCoreUnion) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipProcessors(dAtA[iNdEx:]) - if err != nil { - return err + case 25: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ChangeAggregator", wireType) } - if skippy < 0 { + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { return ErrInvalidLengthProcessors } - if (iNdEx + skippy) > l { + postIndex := iNdEx + msglen + if postIndex > l { return io.ErrUnexpectedEOF } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *NoopCoreSpec) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowProcessors + if m.ChangeAggregator == nil { + m.ChangeAggregator = &ChangeAggregatorSpec{} } - if iNdEx >= l { - return io.ErrUnexpectedEOF + if err := m.ChangeAggregator.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 26: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ChangeFrontier", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ChangeFrontier == nil { + m.ChangeFrontier = &ChangeFrontierSpec{} + } + if err := m.ChangeFrontier.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipProcessors(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthProcessors + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *NoopCoreSpec) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF } b := dAtA[iNdEx] iNdEx++ @@ -10999,6 +11309,357 @@ func (m *WindowerSpec_WindowFn) Unmarshal(dAtA []byte) error { } return nil } +func (m *ChangeAggregatorSpec) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ChangeAggregatorSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ChangeAggregatorSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Watches", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Watches = append(m.Watches, ChangeAggregatorSpec_Watch{}) + if err := m.Watches[len(m.Watches)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Feed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Feed.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipProcessors(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthProcessors + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ChangeAggregatorSpec_Watch) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Watch: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Watch: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field InitialResolved", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.InitialResolved.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Span", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Span.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipProcessors(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthProcessors + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ChangeFrontierSpec) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ChangeFrontierSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ChangeFrontierSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TrackedSpans", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TrackedSpans = append(m.TrackedSpans, cockroach_roachpb1.Span{}) + if err := m.TrackedSpans[len(m.TrackedSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Feed", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Feed.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field JobID", wireType) + } + m.JobID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.JobID |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipProcessors(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthProcessors + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipProcessors(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 @@ -11107,243 +11768,255 @@ var ( func init() { proto.RegisterFile("sql/distsqlrun/processors.proto", fileDescriptorProcessors) } var fileDescriptorProcessors = []byte{ - // 3797 bytes of a gzipped FileDescriptorProto + // 3990 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x4d, 0x6c, 0x1b, 0x49, - 0x76, 0x36, 0xc9, 0xe6, 0xdf, 0xe3, 0x8f, 0xda, 0x65, 0x7b, 0xcd, 0xd1, 0x4c, 0x2c, 0xbb, 0xd7, - 0x3b, 0xfe, 0xd9, 0x59, 0x69, 0xec, 0xd9, 0x9d, 0xcc, 0xce, 0xee, 0x62, 0x86, 0xbf, 0x52, 0xcb, - 0x14, 0x29, 0x37, 0x49, 0x79, 0x67, 0x0e, 0xe9, 0xb4, 0xd8, 0x25, 0xaa, 0x47, 0xcd, 0x2e, 0xba, - 0xbb, 0x69, 0x49, 0x46, 0x80, 0x9c, 0x82, 0x1c, 0x13, 0x60, 0x0f, 0xb9, 0x05, 0x7b, 0xd9, 0x73, - 0x2e, 0x39, 0xe4, 0x14, 0x20, 0x40, 0x0e, 0x83, 0x00, 0x41, 0x92, 0xdb, 0x22, 0x08, 0x8c, 0xc4, - 0x7b, 0x08, 0x72, 0xc9, 0x2d, 0x08, 0x30, 0x87, 0x20, 0xa8, 0xbf, 0x66, 0x53, 0xff, 0xb2, 0x07, - 0xbb, 0x17, 0xa2, 0xfb, 0xfd, 0x7c, 0x5d, 0xf5, 0xea, 0xd5, 0x7b, 0xaf, 0x5e, 0x11, 0x96, 0x82, - 0xe7, 0xee, 0x8a, 0xed, 0x04, 0x61, 0xf0, 0xdc, 0xf5, 0xa7, 0xde, 0xca, 0xc4, 0x27, 0x43, 0x1c, - 0x04, 0xc4, 0x0f, 0x96, 0x27, 0x3e, 0x09, 0x09, 0xaa, 0x0c, 0xc9, 0x70, 0xcf, 0x27, 0xd6, 0x70, - 0x77, 0x39, 0x78, 0xee, 0x2e, 0xcf, 0x44, 0x17, 0xaf, 0x32, 0xea, 0x64, 0x7b, 0xc5, 0x9a, 0x38, - 0x5c, 0x78, 0x11, 0x49, 0x92, 0x6d, 0x85, 0x96, 0xa0, 0x5d, 0x97, 0x34, 0xec, 0xfb, 0x11, 0xec, - 0x62, 0x45, 0x52, 0x1d, 0xf2, 0x83, 0x1d, 0xe2, 0x8f, 0xad, 0x50, 0x72, 0xde, 0xa3, 0x23, 0x0a, - 0x9e, 0xbb, 0xdb, 0x56, 0x80, 0x57, 0x82, 0xd0, 0x9f, 0x0e, 0xc3, 0xa9, 0x8f, 0x6d, 0xc1, 0x5d, - 0x8a, 0x73, 0xb1, 0x37, 0x24, 0x36, 0xb6, 0x4d, 0xdb, 0x0a, 0xa7, 0x63, 0x21, 0xf0, 0x6e, 0x5c, - 0xe0, 0x2b, 0xe2, 0x78, 0x66, 0x78, 0x38, 0xc1, 0x82, 0xf9, 0xce, 0x91, 0xd9, 0xc6, 0x86, 0x59, - 0x99, 0x86, 0x8e, 0xbb, 0xb2, 0xeb, 0x0e, 0x57, 0x42, 0x67, 0x8c, 0x83, 0xd0, 0x1a, 0x4f, 0xe4, - 0x04, 0x46, 0x64, 0x44, 0xd8, 0xe3, 0x0a, 0x7d, 0xe2, 0x54, 0xed, 0xcf, 0x52, 0x50, 0xda, 0x94, - 0xc6, 0xea, 0x4d, 0xf0, 0x10, 0xd5, 0x21, 0xed, 0x78, 0x93, 0x69, 0x58, 0x49, 0xdc, 0x4e, 0xdd, - 0x2f, 0x3c, 0xbe, 0xb7, 0x7c, 0x9a, 0xe5, 0x96, 0x75, 0x2a, 0xd6, 0x3b, 0xf4, 0x86, 0x54, 0xaf, - 0xa6, 0x7c, 0xfd, 0x6a, 0xe9, 0x8a, 0xc1, 0x75, 0x51, 0x0b, 0x94, 0x21, 0xf1, 0x71, 0x25, 0x79, - 0x3b, 0x71, 0xbf, 0xf0, 0xf8, 0x83, 0xd3, 0x31, 0xa2, 0x6f, 0xd7, 0x89, 0x8f, 0x07, 0x9e, 0x43, - 0x3c, 0x01, 0xc4, 0xf4, 0xd1, 0x1a, 0x64, 0xc8, 0x34, 0xa4, 0xa3, 0x49, 0xb1, 0xd1, 0x3c, 0x3c, - 0x1d, 0xa9, 0xcb, 0xe4, 0x0c, 0x32, 0x0d, 0xb1, 0x1f, 0x1b, 0x90, 0xd0, 0x47, 0x75, 0x50, 0x26, - 0x24, 0x08, 0x2b, 0x0a, 0x1b, 0xd1, 0x83, 0x33, 0x46, 0x44, 0x82, 0x50, 0x8c, 0x2a, 0x06, 0xc3, - 0x94, 0xd1, 0x43, 0xc8, 0x05, 0xa1, 0x35, 0xc2, 0xa6, 0x63, 0x57, 0xd2, 0xb7, 0x13, 0xf7, 0xd3, - 0xb5, 0x05, 0xca, 0x7d, 0xfd, 0x6a, 0x29, 0xdb, 0xa3, 0x74, 0xbd, 0x61, 0x64, 0x99, 0x80, 0x6e, - 0xa3, 0x8f, 0xa1, 0x18, 0x79, 0x21, 0x95, 0xcf, 0x30, 0xf9, 0x6b, 0x42, 0xbe, 0x10, 0x4d, 0x5c, - 0x6f, 0x18, 0x85, 0x48, 0x50, 0xb7, 0xb5, 0xbf, 0x4a, 0xc2, 0xc2, 0x91, 0x31, 0xa0, 0x1a, 0x64, - 0x76, 0x1c, 0x37, 0xc4, 0x7e, 0x25, 0xc1, 0x86, 0x7f, 0xf7, 0xf4, 0xe1, 0x37, 0x0f, 0x26, 0x3e, - 0x0e, 0x82, 0x99, 0x21, 0x85, 0x26, 0xba, 0x0b, 0x30, 0xf1, 0xc9, 0x57, 0x78, 0x18, 0x3a, 0xc4, - 0x63, 0x0b, 0x93, 0x13, 0x12, 0x31, 0x3a, 0x7a, 0x00, 0x65, 0x6e, 0x30, 0x73, 0x48, 0xdc, 0xe9, - 0xd8, 0x0b, 0x98, 0xe1, 0x4b, 0xb5, 0xa4, 0x9a, 0x30, 0x4a, 0x9c, 0x53, 0xe7, 0x0c, 0xb4, 0x01, - 0x45, 0x1f, 0x7b, 0x36, 0xf6, 0x4d, 0x7c, 0x30, 0xf1, 0x83, 0x8a, 0xc2, 0x56, 0xe8, 0x32, 0x43, - 0x2b, 0x70, 0x7d, 0x4a, 0x0f, 0xd0, 0x7b, 0x90, 0x21, 0x3b, 0x3b, 0x01, 0x0e, 0x99, 0x65, 0x95, - 0x68, 0xf9, 0x18, 0x0d, 0x2d, 0x42, 0xda, 0x75, 0xc6, 0x4e, 0xc8, 0xcc, 0x28, 0x99, 0x9c, 0xa4, - 0xfd, 0x63, 0x09, 0xd0, 0x71, 0x3f, 0x42, 0x9f, 0x82, 0xe2, 0x11, 0x32, 0x11, 0x26, 0x7b, 0xff, - 0xf4, 0x71, 0x75, 0x08, 0x99, 0x50, 0x35, 0x6a, 0x6a, 0x83, 0xe9, 0xa0, 0x27, 0x50, 0x08, 0xad, - 0x6d, 0x17, 0x1b, 0xd8, 0xb2, 0xb1, 0x2f, 0xdc, 0xf8, 0x0c, 0xa7, 0xe9, 0xcf, 0x84, 0x19, 0x4a, - 0x5c, 0x1b, 0xad, 0x01, 0xd0, 0x1d, 0x2c, 0xb0, 0x52, 0x0c, 0xeb, 0xfe, 0xe9, 0x58, 0xeb, 0x91, - 0x2c, 0x83, 0x8a, 0xe9, 0xa2, 0x9f, 0x42, 0x26, 0x20, 0x3e, 0xf5, 0x03, 0xe5, 0x3c, 0x3f, 0xe8, - 0x31, 0x39, 0x86, 0x20, 0x74, 0xe8, 0x38, 0xac, 0xd1, 0xc8, 0xc7, 0x23, 0x2b, 0x24, 0x3e, 0xb3, - 0xf2, 0x99, 0xe3, 0xa8, 0x46, 0xb2, 0x7c, 0x1c, 0x33, 0x5d, 0x54, 0x83, 0x1c, 0x15, 0x74, 0xbc, - 0x61, 0x58, 0xc9, 0x9e, 0x67, 0xde, 0x86, 0x90, 0x64, 0x28, 0x91, 0x1e, 0x35, 0xf1, 0x18, 0xfb, - 0x23, 0x4c, 0xa7, 0x8b, 0xfd, 0x4a, 0xee, 0x3c, 0x13, 0x6f, 0xcc, 0x84, 0xb9, 0x89, 0x63, 0xda, - 0x74, 0x6a, 0xbb, 0x56, 0xb0, 0x2b, 0xb0, 0xf2, 0xe7, 0x4d, 0x6d, 0x2d, 0x92, 0xe5, 0x53, 0x9b, - 0xe9, 0xa2, 0xcf, 0x21, 0xf3, 0xc2, 0x72, 0xa7, 0x38, 0xa8, 0xc0, 0x79, 0x28, 0x5b, 0x4c, 0x2e, - 0xf2, 0x1c, 0xa1, 0x47, 0xc7, 0xb2, 0x6d, 0x0d, 0xf7, 0x76, 0x1c, 0xd7, 0xc5, 0x7e, 0xa5, 0x70, - 0x1e, 0x4a, 0x2d, 0x92, 0xe5, 0x63, 0x99, 0xe9, 0xa2, 0x36, 0x80, 0x8f, 0x2d, 0x5b, 0x1f, 0x4f, - 0x88, 0x1f, 0x56, 0x4a, 0xe7, 0xc5, 0x52, 0x23, 0x92, 0x6d, 0x58, 0xa1, 0xc5, 0xd1, 0x66, 0xfa, - 0xa8, 0x09, 0xf9, 0x5e, 0xaf, 0xff, 0xcc, 0x77, 0xa8, 0xff, 0x94, 0x19, 0xd8, 0x19, 0xc1, 0x3d, - 0x12, 0x65, 0x38, 0x33, 0x4d, 0xf4, 0x19, 0x64, 0x7b, 0xd6, 0x78, 0x42, 0xe7, 0xb6, 0xc0, 0x40, - 0xbe, 0x77, 0x06, 0x08, 0x17, 0x64, 0x10, 0x52, 0x0b, 0x7d, 0x09, 0x2a, 0x7f, 0x9c, 0x39, 0x58, - 0x45, 0x65, 0x48, 0xcb, 0xe7, 0x21, 0x1d, 0x71, 0xc9, 0x63, 0x38, 0x88, 0xc0, 0x4d, 0xc7, 0x0b, - 0xb1, 0xef, 0x62, 0xeb, 0x05, 0xb6, 0xf9, 0xae, 0x11, 0x4e, 0x71, 0x95, 0x7d, 0xe2, 0x47, 0x67, - 0xa5, 0xb3, 0x13, 0x15, 0xd9, 0x97, 0x4e, 0x43, 0x45, 0x7f, 0x08, 0x68, 0x8c, 0x43, 0x8b, 0x66, - 0xe0, 0x3e, 0x0e, 0xc2, 0x1e, 0x0b, 0x68, 0x15, 0xc4, 0xbe, 0xf5, 0xe1, 0x59, 0xce, 0x7c, 0x54, - 0x87, 0x7d, 0xe6, 0x04, 0x2c, 0xb4, 0x03, 0xd7, 0xe3, 0x54, 0x03, 0x0f, 0xb1, 0xf3, 0x02, 0xfb, - 0x95, 0x6b, 0xec, 0x1b, 0x8f, 0x2f, 0xf6, 0x0d, 0xa9, 0xc5, 0xbe, 0x72, 0x22, 0x1e, 0x75, 0x8f, - 0x7a, 0x6f, 0x4b, 0xb8, 0xc7, 0xf5, 0xf3, 0xdc, 0x23, 0x12, 0xe5, 0xee, 0x11, 0xbd, 0xa2, 0x0e, - 0x14, 0x5f, 0x3a, 0xa3, 0x97, 0xd6, 0x48, 0x98, 0xfd, 0x06, 0x43, 0x3a, 0x23, 0x6f, 0x7f, 0x19, - 0x93, 0x66, 0x60, 0x73, 0xfa, 0x74, 0x37, 0x89, 0xf4, 0xd4, 0xc3, 0x61, 0xe5, 0x3b, 0xe7, 0xed, - 0xa6, 0xcd, 0x48, 0x96, 0xfb, 0xff, 0x4c, 0x97, 0x06, 0xad, 0x7d, 0xc7, 0xb3, 0xc9, 0x3e, 0xf6, - 0x2b, 0x37, 0xcf, 0x0b, 0x5a, 0xcf, 0x84, 0x24, 0x0f, 0x5a, 0x52, 0x0f, 0x3d, 0x85, 0x92, 0x4b, - 0x86, 0x96, 0xbb, 0xe9, 0x5a, 0x5e, 0x87, 0xd8, 0xb8, 0x52, 0x61, 0x40, 0xdf, 0x3f, 0x1d, 0xa8, - 0x1d, 0x17, 0x67, 0x68, 0xf3, 0x08, 0x9f, 0x2a, 0x5f, 0xff, 0x72, 0x29, 0xb1, 0xae, 0xe4, 0x32, - 0x6a, 0x76, 0x5d, 0xc9, 0x15, 0xd5, 0x92, 0x56, 0x86, 0x62, 0x3c, 0x25, 0x69, 0x3f, 0x84, 0xef, - 0x9c, 0xec, 0x2f, 0x68, 0x11, 0x92, 0x8e, 0xcd, 0x12, 0x5c, 0xbe, 0x06, 0xa2, 0xb2, 0x48, 0xea, - 0x0d, 0x23, 0xe9, 0xd8, 0xda, 0x1a, 0x54, 0x4e, 0xf3, 0x00, 0xf4, 0x01, 0x40, 0xc0, 0x53, 0xb7, - 0x63, 0x07, 0xac, 0xd0, 0xcb, 0xd7, 0x4a, 0xaf, 0x5f, 0x2d, 0xe5, 0x39, 0xb6, 0xde, 0x08, 0x8c, - 0x3c, 0x17, 0xd0, 0xed, 0x40, 0xfb, 0x45, 0x02, 0xca, 0xf3, 0xb1, 0x0e, 0xd5, 0x21, 0x2b, 0xeb, - 0x03, 0x5e, 0x26, 0x7e, 0xf7, 0x8c, 0xf8, 0x4f, 0xcb, 0x5a, 0xdd, 0xdb, 0x21, 0x22, 0x6b, 0x4b, - 0x4d, 0xf4, 0x2e, 0xe4, 0x7d, 0x6b, 0xdf, 0xdc, 0x3e, 0x0c, 0x71, 0x50, 0x49, 0xde, 0x4e, 0xdd, - 0x2f, 0x1a, 0x39, 0xdf, 0xda, 0xaf, 0xd1, 0x77, 0xb4, 0x04, 0x39, 0x6f, 0x3a, 0x36, 0x7d, 0xb2, - 0x1f, 0xb0, 0x94, 0x29, 0x73, 0x7e, 0xd6, 0x9b, 0x8e, 0x0d, 0xb2, 0x1f, 0x68, 0x0d, 0x58, 0x98, - 0xcb, 0xba, 0x96, 0x87, 0x1e, 0x81, 0x12, 0x4c, 0x2c, 0x4f, 0x64, 0xfc, 0x9b, 0xb1, 0x21, 0x89, - 0x32, 0x7d, 0x99, 0x8a, 0xc9, 0x8a, 0x8e, 0x8a, 0x6a, 0xff, 0x93, 0x3c, 0x02, 0xc3, 0xaa, 0xad, - 0x34, 0x4b, 0xdf, 0xa7, 0x54, 0x0e, 0xa2, 0x2a, 0xe7, 0x39, 0xbf, 0x81, 0x83, 0xa1, 0xef, 0x4c, - 0x42, 0xe2, 0xcb, 0x9a, 0x84, 0xa9, 0xa2, 0x3b, 0x90, 0x77, 0x3c, 0x1b, 0x1f, 0x98, 0x8e, 0x7d, - 0xc0, 0xca, 0x87, 0x92, 0xe0, 0xe7, 0x18, 0x59, 0xb7, 0x0f, 0xd0, 0x2d, 0xc8, 0xfa, 0xf8, 0x05, - 0xf6, 0x03, 0xcc, 0x26, 0x28, 0xab, 0x31, 0x49, 0x44, 0x4d, 0x48, 0xd3, 0x21, 0xca, 0xc2, 0xea, - 0xa2, 0xd5, 0x47, 0x34, 0x41, 0xae, 0x8d, 0xbe, 0x0b, 0xc0, 0xca, 0x24, 0x73, 0xd7, 0xf1, 0x78, - 0x6d, 0x95, 0x12, 0x02, 0x79, 0x46, 0x5f, 0x73, 0xbc, 0x90, 0x5a, 0xdb, 0x09, 0xcc, 0xe1, 0x2e, - 0x1e, 0xee, 0xb1, 0x0a, 0x2b, 0x1a, 0x8c, 0x13, 0xd4, 0x29, 0x11, 0x75, 0x00, 0x5e, 0x38, 0x81, - 0xb3, 0xed, 0xb8, 0x4e, 0x78, 0xc8, 0x72, 0x7e, 0xf9, 0xac, 0x6d, 0xd8, 0x1b, 0x5a, 0xde, 0x56, - 0x24, 0x2f, 0xeb, 0xcc, 0x19, 0x82, 0xf6, 0x0f, 0x29, 0x28, 0xcf, 0x17, 0x3a, 0xbf, 0x2d, 0xb3, - 0x3f, 0x80, 0xb2, 0x4b, 0xc8, 0xde, 0x74, 0x72, 0x52, 0x85, 0xcb, 0x39, 0xb2, 0xc2, 0xad, 0x43, - 0x96, 0x78, 0xac, 0xba, 0x3d, 0xbf, 0xde, 0x3a, 0x5e, 0x77, 0x13, 0x8f, 0xd2, 0xd0, 0x16, 0x5c, - 0xe5, 0x43, 0xe2, 0x75, 0x38, 0x87, 0x4b, 0x5f, 0x1a, 0x6e, 0x81, 0x81, 0xb4, 0x18, 0x06, 0xc3, - 0xfd, 0x31, 0x28, 0xf4, 0x48, 0xc8, 0x96, 0xab, 0xfc, 0x78, 0xe9, 0x14, 0x6b, 0x51, 0x1b, 0xf7, - 0x0f, 0x27, 0x58, 0x3a, 0x3d, 0x55, 0xf9, 0xd6, 0x17, 0xf3, 0x17, 0x09, 0x80, 0x59, 0xbd, 0x89, - 0x9e, 0xc2, 0x82, 0x38, 0x43, 0x10, 0xdf, 0xc6, 0xbe, 0xe3, 0x8d, 0xc4, 0x92, 0x6a, 0x67, 0x9c, - 0xde, 0x84, 0xa4, 0x40, 0x17, 0x87, 0x10, 0x49, 0x45, 0x8f, 0x01, 0x49, 0x2c, 0x73, 0x6c, 0x85, - 0xc3, 0x5d, 0xd3, 0xc5, 0xde, 0xdc, 0x02, 0xab, 0x92, 0xbf, 0x41, 0xd9, 0x6d, 0xec, 0x69, 0xdb, - 0x50, 0x8c, 0x97, 0x9e, 0xe8, 0x1e, 0x2c, 0x30, 0x19, 0x6c, 0x9b, 0xf1, 0xd8, 0x55, 0x32, 0xca, - 0x82, 0x2c, 0x97, 0xfd, 0x01, 0xa8, 0xb2, 0x4a, 0x8d, 0x24, 0x93, 0x4c, 0x72, 0x41, 0xd2, 0x85, - 0xa8, 0xf6, 0x77, 0x49, 0x50, 0x8f, 0x26, 0x30, 0xd4, 0x80, 0x0c, 0xf3, 0x46, 0x19, 0x1b, 0x2f, - 0xe7, 0xc9, 0x42, 0x17, 0xb5, 0x00, 0xf0, 0xf3, 0xb9, 0xef, 0x17, 0x1e, 0xdf, 0x39, 0x23, 0x21, - 0x73, 0x41, 0xb9, 0xb5, 0xf1, 0x73, 0x39, 0x9b, 0xa5, 0xd9, 0x96, 0x88, 0xbb, 0xba, 0xdc, 0x10, - 0xdf, 0x92, 0x97, 0x4b, 0x6f, 0x4c, 0x5f, 0xda, 0x1b, 0x35, 0x07, 0xae, 0x1e, 0x4b, 0x92, 0x48, - 0x83, 0xa2, 0x41, 0xf6, 0x7b, 0x64, 0xea, 0x0f, 0xb1, 0x6e, 0x1f, 0x30, 0x07, 0x2a, 0x19, 0x73, - 0x34, 0xf4, 0x1e, 0xe4, 0x3b, 0x34, 0xb3, 0x4c, 0xa6, 0x61, 0xc0, 0x7d, 0xc1, 0x98, 0x11, 0x10, - 0x02, 0xa5, 0x63, 0x8d, 0x79, 0x6c, 0xcd, 0x1b, 0xec, 0x59, 0xbb, 0x07, 0x59, 0x69, 0x96, 0xf7, - 0xe6, 0x33, 0x18, 0x37, 0x8a, 0x24, 0x69, 0xff, 0x96, 0x84, 0x85, 0x23, 0x07, 0x0e, 0xb4, 0x01, - 0x25, 0x17, 0xef, 0xbc, 0x85, 0x53, 0x17, 0xa9, 0x7a, 0xe4, 0xd2, 0x5d, 0x28, 0xfb, 0xce, 0x68, - 0x37, 0x86, 0x97, 0xbc, 0x24, 0x5e, 0x89, 0xe9, 0x47, 0x80, 0xb1, 0x75, 0x4c, 0xbf, 0xf5, 0x3a, - 0xbe, 0x41, 0x54, 0x79, 0x00, 0x25, 0x6f, 0xea, 0xba, 0x26, 0x7e, 0x3e, 0xb5, 0xa2, 0xc0, 0x22, - 0x13, 0x49, 0x91, 0xb2, 0x9a, 0x82, 0xa3, 0xfd, 0x45, 0x12, 0xca, 0xf3, 0x67, 0x30, 0xf4, 0x10, - 0x16, 0x98, 0x75, 0x63, 0x3e, 0x9f, 0x88, 0xc5, 0x65, 0xbc, 0x13, 0x36, 0x23, 0x97, 0xfe, 0x00, - 0x54, 0x6e, 0xba, 0x23, 0x1b, 0x84, 0x0b, 0x73, 0xb3, 0xce, 0xa4, 0x7f, 0xd7, 0x76, 0xf9, 0x3e, - 0x94, 0xd9, 0x51, 0x75, 0x16, 0x76, 0xe2, 0x86, 0x29, 0x71, 0x9e, 0x0c, 0x28, 0xff, 0x9d, 0x81, - 0xf2, 0xfc, 0x29, 0x07, 0xdd, 0x01, 0x18, 0xf9, 0x84, 0xe7, 0xab, 0xf8, 0x3c, 0xf3, 0x8c, 0x5a, - 0x27, 0x6e, 0x80, 0xfe, 0x00, 0x8a, 0xf2, 0x74, 0xee, 0x10, 0x91, 0xd1, 0x0a, 0x8f, 0x7f, 0x78, - 0xd1, 0xb3, 0x7d, 0xf4, 0x3a, 0x9b, 0xf7, 0x1c, 0x1e, 0xfa, 0x50, 0x84, 0x5f, 0x6c, 0x9b, 0xb1, - 0xa1, 0x28, 0xd1, 0x50, 0x54, 0xc1, 0x5d, 0x8d, 0x46, 0xb4, 0x3a, 0x17, 0x0f, 0x7e, 0x70, 0xe1, - 0x91, 0x1c, 0xb5, 0xde, 0xe2, 0x9f, 0x26, 0xa1, 0x10, 0x1b, 0x1e, 0x05, 0xde, 0x99, 0x7a, 0x43, - 0xb6, 0xf9, 0x2e, 0x03, 0xdc, 0x9a, 0x7a, 0x51, 0x2f, 0x8f, 0x02, 0xa0, 0xdb, 0xb1, 0x1e, 0x46, - 0xbc, 0x1b, 0x36, 0xeb, 0x50, 0xdc, 0x85, 0xb2, 0xc8, 0xd9, 0x43, 0xe2, 0xb2, 0x8a, 0x42, 0xe1, - 0x51, 0x88, 0x53, 0xeb, 0xc4, 0xa5, 0x51, 0xe8, 0x26, 0x0b, 0x24, 0x8c, 0x9d, 0x66, 0x49, 0x22, - 0x33, 0xe4, 0x8c, 0x35, 0xc8, 0x5b, 0xfe, 0x68, 0x3a, 0xc6, 0x5e, 0x18, 0x54, 0x32, 0x97, 0x6e, - 0x8e, 0xcd, 0x94, 0xd7, 0x95, 0x5c, 0x4a, 0x55, 0xb4, 0x5f, 0x25, 0x41, 0xa1, 0xb3, 0x40, 0x2a, - 0x14, 0xab, 0x9d, 0x2f, 0xcc, 0x4e, 0xb7, 0x6f, 0x76, 0x06, 0xed, 0xb6, 0x7a, 0x05, 0x65, 0x21, - 0x55, 0xdd, 0x5a, 0x55, 0x13, 0xa8, 0x08, 0xb9, 0x5a, 0xb7, 0xdb, 0x36, 0xab, 0x9d, 0x86, 0x9a, - 0x44, 0x05, 0xc8, 0xb2, 0xb7, 0xae, 0xa1, 0xa6, 0x50, 0x19, 0xa0, 0xde, 0xed, 0xd4, 0xab, 0x7d, - 0xb3, 0xba, 0xba, 0xaa, 0x2a, 0x28, 0x0f, 0xe9, 0x7a, 0x77, 0xd0, 0xe9, 0xab, 0x69, 0xaa, 0xbe, - 0x51, 0xfd, 0xb9, 0x9a, 0x65, 0x0f, 0x7a, 0x47, 0xcd, 0x21, 0x80, 0x4c, 0xaf, 0xdf, 0x68, 0x34, - 0xb7, 0xd4, 0x3c, 0x25, 0xf6, 0x06, 0x1b, 0x2a, 0x50, 0xb8, 0xde, 0x60, 0xc3, 0xd4, 0x3b, 0x7d, - 0xb5, 0x40, 0xbf, 0xb4, 0x55, 0x35, 0xf4, 0x6a, 0xa7, 0xde, 0x54, 0x8b, 0x94, 0xf5, 0xf3, 0xae, - 0xc1, 0x90, 0x4b, 0xfc, 0x4b, 0x83, 0x4e, 0xdf, 0x34, 0xba, 0xcf, 0x7a, 0x6a, 0x99, 0xe9, 0x3d, - 0x35, 0x1a, 0x7a, 0xab, 0xa5, 0x2e, 0x20, 0x04, 0xe5, 0x96, 0xde, 0xa9, 0xb6, 0xcd, 0x48, 0x5b, - 0xa5, 0x13, 0xe2, 0x34, 0xf1, 0xcd, 0xab, 0xa8, 0x04, 0xf9, 0xaa, 0x61, 0x54, 0xbf, 0x60, 0x88, - 0x88, 0x7e, 0x6c, 0xbd, 0xd7, 0xed, 0xb0, 0xb7, 0x6b, 0x94, 0x49, 0xdf, 0x6a, 0xec, 0xf5, 0x3a, - 0xfd, 0x5c, 0xaf, 0x6f, 0xe8, 0x9d, 0x55, 0xf6, 0x7e, 0x43, 0xfb, 0x00, 0x14, 0xea, 0x45, 0x28, - 0x07, 0x4a, 0x75, 0xd0, 0xef, 0xaa, 0x57, 0xd8, 0x6c, 0xea, 0xd5, 0x76, 0xd5, 0x50, 0x13, 0x54, - 0xba, 0xd3, 0xed, 0x98, 0xe2, 0x3d, 0xa9, 0x7d, 0x93, 0x82, 0xf2, 0x7c, 0x0b, 0x26, 0xf2, 0xdd, - 0x73, 0x5d, 0x6c, 0x5e, 0xef, 0x98, 0xef, 0xce, 0x2a, 0xda, 0xe4, 0x9b, 0x57, 0xb4, 0xd1, 0x29, - 0x20, 0xf5, 0x56, 0xa7, 0x80, 0x47, 0x90, 0xb3, 0xa7, 0x3e, 0xdb, 0x42, 0xcc, 0x8b, 0x53, 0xb5, - 0x1b, 0x94, 0xfd, 0xcd, 0xab, 0xa5, 0x52, 0xe8, 0x8c, 0xf1, 0x72, 0x43, 0x30, 0x8d, 0x48, 0x8c, - 0x1e, 0x1c, 0x86, 0xbb, 0x53, 0x6f, 0xcf, 0x0c, 0x9c, 0x97, 0x78, 0xfe, 0xe0, 0xc0, 0xe8, 0x3d, - 0xe7, 0x25, 0x46, 0x5d, 0x28, 0x92, 0x70, 0x17, 0xfb, 0xa6, 0xa8, 0x78, 0x32, 0x6f, 0x50, 0xf1, - 0x14, 0x18, 0x42, 0x9f, 0x97, 0x3d, 0x9f, 0x41, 0xce, 0xc7, 0x96, 0x5d, 0x0d, 0xba, 0x3b, 0xa2, - 0xb5, 0xf8, 0x7b, 0x31, 0xb0, 0x69, 0xe8, 0xb8, 0xcb, 0xbb, 0xee, 0x70, 0xb9, 0x2f, 0x6f, 0x37, - 0xe4, 0xae, 0x95, 0x4a, 0xda, 0x43, 0xb1, 0xfc, 0x05, 0xc8, 0xea, 0xde, 0x0b, 0xcb, 0x75, 0x6c, - 0xee, 0x01, 0x3c, 0xc2, 0xaa, 0x09, 0xea, 0xf8, 0x3a, 0xad, 0x7d, 0xd4, 0xa4, 0xf6, 0x9b, 0x04, - 0xe4, 0x5a, 0x2e, 0xd9, 0x67, 0xcb, 0xfe, 0x08, 0xb2, 0x3b, 0x2e, 0xd9, 0x37, 0xc5, 0x89, 0xba, - 0x58, 0xab, 0x50, 0xe4, 0x7f, 0x7d, 0xb5, 0x94, 0xa1, 0x22, 0x7a, 0xe3, 0x75, 0xf4, 0x64, 0x64, - 0xa8, 0xa0, 0x6e, 0xa3, 0x0d, 0xd6, 0x9c, 0x10, 0x37, 0x4d, 0xa2, 0x46, 0xbb, 0x77, 0x81, 0xcb, - 0x8e, 0xd8, 0xc5, 0x42, 0x0c, 0x00, 0x0d, 0x20, 0x3b, 0xb2, 0x42, 0xbc, 0x6f, 0x1d, 0xb2, 0xaa, - 0x25, 0x5d, 0xfb, 0x89, 0x58, 0xa3, 0x8f, 0x46, 0x4e, 0xb8, 0x3b, 0xdd, 0x5e, 0x1e, 0x92, 0xf1, - 0x4a, 0x84, 0x6e, 0x6f, 0xcf, 0x9e, 0x57, 0x26, 0x7b, 0xa3, 0x15, 0x79, 0xc8, 0xa5, 0x45, 0x94, - 0xde, 0x30, 0x24, 0x96, 0xb6, 0x0f, 0x85, 0x75, 0xb2, 0xbd, 0xe9, 0x93, 0x11, 0x8d, 0x30, 0xe8, - 0x2e, 0x64, 0xbe, 0x22, 0xdb, 0x72, 0x9a, 0xa9, 0x5a, 0x49, 0x34, 0x0e, 0xd2, 0xeb, 0x64, 0x5b, - 0x6f, 0x18, 0xe9, 0xaf, 0xc8, 0xb6, 0x6e, 0xa3, 0xfb, 0x50, 0x1c, 0x12, 0x2f, 0xf4, 0x9d, 0xed, - 0x69, 0x74, 0x61, 0x90, 0x94, 0xc9, 0x21, 0xce, 0x41, 0x15, 0x50, 0x02, 0x97, 0x84, 0x62, 0xc8, - 0xf2, 0x70, 0xed, 0x92, 0x50, 0xfb, 0xb5, 0x02, 0xe8, 0x78, 0x53, 0x12, 0x7d, 0x0f, 0x0a, 0x01, - 0x6b, 0xdc, 0x71, 0xcf, 0x4a, 0xc6, 0xf4, 0x80, 0x33, 0x98, 0x6b, 0xad, 0x42, 0x6e, 0x22, 0xc6, - 0xcc, 0xd2, 0xee, 0x99, 0x9d, 0xc6, 0xd8, 0x04, 0xa5, 0x47, 0x48, 0x65, 0xb4, 0x0a, 0xa9, 0xa9, - 0xef, 0x54, 0xb2, 0x6c, 0x79, 0x7e, 0x74, 0x99, 0xfe, 0xe9, 0xf2, 0xc0, 0x77, 0x9a, 0x5e, 0xe8, - 0x1f, 0x1a, 0x14, 0x01, 0xfd, 0x0c, 0x32, 0xfc, 0x92, 0x4f, 0x74, 0xab, 0x97, 0x4e, 0xe8, 0x30, - 0xe8, 0xdd, 0x96, 0xe3, 0xe2, 0x16, 0x13, 0x8b, 0x6e, 0x60, 0xd8, 0x1b, 0xda, 0x8a, 0xce, 0x05, - 0x79, 0x36, 0x94, 0x4f, 0x2e, 0x35, 0x14, 0xbe, 0x3f, 0xd8, 0x68, 0x18, 0x6e, 0x22, 0x3a, 0x29, - 0x7c, 0x06, 0xef, 0x04, 0x7b, 0xce, 0xc4, 0x1c, 0x3b, 0x41, 0x40, 0x0f, 0x48, 0x3b, 0xc4, 0xc7, - 0xce, 0xc8, 0x33, 0xf7, 0xf0, 0x21, 0xef, 0x62, 0xcb, 0xd4, 0xf6, 0x1d, 0x2a, 0xb6, 0xc1, 0xa5, - 0x5a, 0x5c, 0xe8, 0x09, 0x3e, 0x0c, 0x16, 0x2d, 0x28, 0xc4, 0xd0, 0x91, 0x0a, 0xa9, 0x3d, 0x7c, - 0xc8, 0xdb, 0x4a, 0x06, 0x7d, 0x44, 0x3f, 0x85, 0x34, 0x6b, 0x6e, 0x5f, 0x2e, 0x90, 0x19, 0x5c, - 0xe9, 0xd3, 0xe4, 0x27, 0x89, 0xc5, 0x8f, 0x21, 0x27, 0x6d, 0x19, 0xc7, 0x4f, 0x73, 0xfc, 0xeb, - 0x71, 0xfc, 0x7c, 0x4c, 0x6f, 0x5d, 0xc9, 0x25, 0xd4, 0x24, 0x4f, 0x80, 0xeb, 0x4a, 0x4e, 0x51, - 0xd3, 0xeb, 0x4a, 0x2e, 0xad, 0x66, 0xb4, 0xbf, 0x49, 0x42, 0x69, 0xae, 0x45, 0x8d, 0xde, 0x87, - 0x82, 0x8d, 0x69, 0xe6, 0xe6, 0x41, 0x8e, 0x37, 0xc5, 0x44, 0x80, 0x89, 0x31, 0xd0, 0x43, 0x28, - 0xed, 0x5b, 0xae, 0x4b, 0xa3, 0x5e, 0xc7, 0xf2, 0x08, 0xef, 0x2e, 0xc9, 0xc8, 0x36, 0xcf, 0x42, - 0xed, 0xf9, 0x16, 0xcc, 0x87, 0x17, 0x6c, 0x97, 0xb3, 0x36, 0x13, 0x3d, 0x70, 0xcc, 0xc7, 0xe0, - 0xb8, 0x43, 0xa7, 0xdf, 0xc2, 0xa1, 0xa9, 0x31, 0xe5, 0x17, 0xe8, 0xee, 0xf3, 0xe8, 0x31, 0x27, - 0x3e, 0x5f, 0x46, 0xa1, 0x66, 0xc6, 0x9e, 0xcd, 0x4c, 0x5a, 0x34, 0xe8, 0xe3, 0xba, 0x92, 0x4b, - 0xaa, 0x29, 0xed, 0xef, 0x13, 0x50, 0x9a, 0x6b, 0xdf, 0x5e, 0xd8, 0x74, 0xf7, 0xa0, 0x48, 0x91, - 0xcd, 0x89, 0x15, 0x86, 0xd8, 0xe7, 0x31, 0x21, 0x12, 0xa4, 0x9c, 0x4d, 0xce, 0x40, 0x3f, 0x83, - 0x2c, 0x99, 0xc8, 0x52, 0xf4, 0x68, 0x0c, 0x97, 0x3b, 0xa5, 0xde, 0xdb, 0xea, 0x72, 0x21, 0xd9, - 0x6c, 0x12, 0x3a, 0xb3, 0xcc, 0xc3, 0xba, 0x7f, 0xca, 0xb1, 0xcc, 0xc3, 0xfa, 0x7f, 0x7f, 0x92, - 0x04, 0xe8, 0xed, 0xe1, 0x70, 0xb8, 0xcb, 0xe6, 0xf0, 0x04, 0x0a, 0x01, 0x7b, 0x33, 0x63, 0xb9, - 0xfb, 0xac, 0xfb, 0x31, 0x26, 0x1c, 0x4b, 0xd9, 0x10, 0x44, 0x14, 0x54, 0x99, 0x1d, 0x0e, 0xf9, - 0xc1, 0x3f, 0xea, 0x59, 0x7e, 0x04, 0x68, 0x84, 0x3d, 0xec, 0x5b, 0x21, 0x36, 0x77, 0x9d, 0x20, - 0x24, 0x23, 0xdf, 0x1a, 0xcf, 0xf5, 0xef, 0xae, 0x4a, 0xfe, 0x9a, 0x64, 0xa3, 0x4f, 0xe0, 0x46, - 0x24, 0x6b, 0x8e, 0xad, 0x03, 0x73, 0x7b, 0x3a, 0xdc, 0xc3, 0x21, 0x9f, 0x9a, 0x6c, 0x60, 0x5c, - 0x8b, 0x44, 0x36, 0xac, 0x83, 0x1a, 0x17, 0x40, 0x77, 0x20, 0x1f, 0x84, 0x56, 0x68, 0xb2, 0x25, - 0x4e, 0xc7, 0xcc, 0x9d, 0xa3, 0x64, 0x76, 0xa6, 0xfd, 0x23, 0x28, 0xc4, 0xae, 0x59, 0x50, 0x0b, - 0x72, 0x7c, 0x22, 0x51, 0xfb, 0xe1, 0x5c, 0x23, 0xc4, 0xb2, 0x51, 0xa4, 0x7b, 0x52, 0x90, 0x2e, - 0x1d, 0x0f, 0xd2, 0xda, 0x7f, 0x25, 0xe1, 0xfa, 0x49, 0x77, 0x33, 0xbf, 0xe5, 0x71, 0xa0, 0x3f, - 0x06, 0xc4, 0xdf, 0xe4, 0x29, 0x2b, 0xd6, 0xee, 0x78, 0xfa, 0xfa, 0xd5, 0x92, 0xb8, 0x2a, 0x12, - 0xe7, 0x2c, 0xbd, 0x11, 0x7c, 0xf3, 0x6a, 0xe9, 0xc7, 0x17, 0xca, 0xaa, 0xb1, 0x3f, 0x62, 0x2c, - 0x4b, 0x6d, 0x43, 0x0d, 0xe6, 0xe0, 0xec, 0x00, 0x59, 0x90, 0x63, 0xe1, 0x98, 0xe6, 0x55, 0xbe, - 0xac, 0x2d, 0xf9, 0xd7, 0x00, 0x16, 0x16, 0xf5, 0xc6, 0x85, 0xf3, 0x78, 0xfc, 0x8b, 0x34, 0x8f, - 0x33, 0x5c, 0xdd, 0xd6, 0xfe, 0x4f, 0x81, 0x77, 0xcf, 0xb8, 0xa4, 0x42, 0x5f, 0x1c, 0xe9, 0x3b, - 0xfd, 0xe4, 0x8d, 0xee, 0xba, 0x78, 0x14, 0x3f, 0xd2, 0x8c, 0x8a, 0xf5, 0xaa, 0x93, 0x27, 0xf5, - 0xaa, 0xe7, 0x9b, 0xcc, 0xa9, 0x93, 0x9b, 0xcc, 0xbf, 0xe3, 0x46, 0xd3, 0xe2, 0xdf, 0x26, 0x21, - 0xcd, 0x26, 0x87, 0x3e, 0x07, 0xc5, 0xc6, 0xc1, 0xf0, 0x8d, 0x3a, 0xcd, 0x4c, 0xf3, 0x22, 0x8d, - 0x66, 0xf9, 0x8f, 0x93, 0xd4, 0xdb, 0xfc, 0xe3, 0xa4, 0x01, 0xb9, 0xa8, 0x3f, 0xa4, 0x5c, 0xb2, - 0x3f, 0x14, 0x69, 0xce, 0x0e, 0x11, 0xe9, 0xb7, 0x39, 0x44, 0x68, 0xff, 0x92, 0x80, 0xf2, 0xfc, - 0x05, 0x1b, 0xfa, 0x1c, 0xd2, 0xfc, 0xdf, 0x1f, 0x89, 0x4b, 0x1f, 0x70, 0xb9, 0x22, 0xea, 0x43, - 0x14, 0x31, 0xed, 0x73, 0xda, 0x9d, 0xf3, 0xdb, 0x30, 0xb6, 0xbe, 0x6a, 0x84, 0x20, 0x9b, 0x3e, - 0xf7, 0x40, 0xf5, 0xa6, 0x63, 0xd6, 0xa7, 0x30, 0x27, 0xd8, 0x37, 0x47, 0xd8, 0xe3, 0xd1, 0xc0, - 0x28, 0x79, 0xd3, 0x71, 0x9d, 0xb8, 0xc1, 0x26, 0xf6, 0x57, 0xb1, 0xa7, 0xfd, 0xb2, 0x08, 0xc5, - 0xf8, 0x65, 0x1f, 0xba, 0x0d, 0x85, 0x89, 0xe5, 0x87, 0x0e, 0x6b, 0x86, 0x1c, 0x8a, 0x16, 0x71, - 0x9c, 0x84, 0x7a, 0x90, 0xe7, 0x17, 0x82, 0xad, 0x68, 0xa4, 0x2b, 0x17, 0xbb, 0x49, 0x14, 0x2f, - 0xad, 0xe8, 0x8c, 0x1f, 0xe1, 0x2c, 0xfe, 0x75, 0x42, 0x9c, 0xee, 0x7b, 0x50, 0x92, 0xbd, 0x17, - 0xdc, 0x7a, 0xd3, 0x4e, 0x87, 0x31, 0x8f, 0x81, 0x9e, 0x02, 0x88, 0x4f, 0x51, 0xc4, 0x24, 0x43, - 0x7c, 0x74, 0xb9, 0x31, 0x53, 0xd4, 0x18, 0x08, 0xbf, 0xb7, 0x5c, 0xfc, 0xcf, 0x34, 0xa4, 0x5b, - 0x3e, 0x2d, 0x37, 0x9e, 0x80, 0x32, 0x26, 0xb6, 0xcc, 0xbc, 0x17, 0x05, 0x67, 0xba, 0xcb, 0x1b, - 0xc4, 0x8e, 0xb6, 0x2a, 0x05, 0x41, 0x4f, 0x21, 0xb3, 0x4d, 0xa6, 0x9e, 0x1d, 0x88, 0x8a, 0xf3, - 0xa3, 0x4b, 0xc1, 0xd5, 0x98, 0xaa, 0x0c, 0x1c, 0x1c, 0x68, 0xf1, 0x7f, 0x13, 0x90, 0x66, 0x0c, - 0xf4, 0x25, 0xe4, 0x19, 0xad, 0x3f, 0x2b, 0x14, 0x3e, 0xbe, 0x3c, 0x7e, 0xcc, 0xfd, 0x66, 0x70, - 0x34, 0x10, 0x3a, 0x5e, 0x68, 0x8a, 0x7f, 0x32, 0xc5, 0x03, 0x43, 0xde, 0xf1, 0xc2, 0x2e, 0xff, - 0x33, 0xd3, 0x1d, 0x28, 0xd2, 0x80, 0x64, 0x4b, 0xb1, 0x14, 0x2b, 0xd1, 0x0a, 0x8c, 0x26, 0x44, - 0xd6, 0xa1, 0xc0, 0x99, 0xbc, 0x9c, 0xe1, 0x5b, 0xff, 0x12, 0x97, 0xac, 0xc0, 0xb5, 0xe9, 0x98, - 0x16, 0xff, 0x32, 0x01, 0x19, 0x6e, 0x12, 0xd4, 0x81, 0x74, 0x10, 0x5a, 0x7e, 0x28, 0x22, 0xdf, - 0xe3, 0xcb, 0x4f, 0x3b, 0x8a, 0x08, 0x14, 0x06, 0x35, 0x66, 0x35, 0xe6, 0x1b, 0xa1, 0xb1, 0xba, - 0x54, 0xbb, 0x07, 0x0a, 0xf5, 0x00, 0x7a, 0x32, 0x37, 0xaa, 0x9d, 0xd5, 0xa6, 0x7a, 0x05, 0xe5, - 0x40, 0x61, 0xdd, 0xa3, 0x04, 0x3d, 0xba, 0xaf, 0x1a, 0xdd, 0xc1, 0x66, 0x4f, 0x4d, 0x6a, 0x2f, - 0x21, 0x1f, 0xd9, 0x1e, 0xdd, 0x84, 0x6b, 0x83, 0x4e, 0xad, 0x3b, 0xe8, 0x34, 0x9a, 0x0d, 0x73, - 0xd3, 0x68, 0xd6, 0x9b, 0x0d, 0xbd, 0xb3, 0xaa, 0x5e, 0x99, 0x67, 0xb4, 0xba, 0xed, 0x76, 0xf7, - 0x19, 0x65, 0x24, 0xd0, 0x75, 0x50, 0xbb, 0xad, 0x56, 0xaf, 0xd9, 0x8f, 0x89, 0x27, 0x63, 0xd4, - 0x99, 0x6c, 0x0a, 0x2d, 0x40, 0xa1, 0x3e, 0x30, 0x8c, 0x26, 0x6f, 0x63, 0xa9, 0xca, 0xe2, 0x3f, - 0x25, 0x21, 0x27, 0xb7, 0x2f, 0x6a, 0xc6, 0xba, 0x90, 0x67, 0x5e, 0xff, 0xcf, 0x4f, 0xfc, 0x68, - 0x0f, 0xf2, 0x7d, 0x28, 0x58, 0xfe, 0x48, 0xb7, 0x0f, 0x7a, 0x6c, 0x51, 0xe2, 0xee, 0x12, 0x67, - 0xa0, 0xdb, 0x90, 0xb3, 0xfc, 0x51, 0x9d, 0x4c, 0x45, 0x72, 0x8d, 0x92, 0x8d, 0xa4, 0x7e, 0x4b, - 0x79, 0xa2, 0x06, 0xe9, 0x1d, 0x5f, 0x96, 0x9a, 0x67, 0xfe, 0xd7, 0xe8, 0xf8, 0x82, 0x1a, 0x5c, - 0x15, 0xdd, 0x87, 0xb9, 0xfe, 0xa8, 0xf8, 0xdf, 0xa3, 0x68, 0x1c, 0xc4, 0x39, 0xda, 0xaf, 0x12, - 0x00, 0xb3, 0xe0, 0x82, 0xca, 0x00, 0x46, 0xf7, 0x99, 0xd9, 0x19, 0x6c, 0xd4, 0x9a, 0x86, 0x70, - 0x81, 0x6a, 0xe7, 0x09, 0xef, 0xd9, 0x35, 0x9a, 0x9d, 0x5e, 0xd3, 0x64, 0xef, 0x49, 0xa4, 0x42, - 0x71, 0xb3, 0x69, 0xd4, 0xd9, 0xda, 0x50, 0x4a, 0x0a, 0x95, 0x20, 0x5f, 0x1f, 0x6c, 0x34, 0xcd, - 0x86, 0xde, 0xeb, 0xf3, 0xde, 0x66, 0xa7, 0xaf, 0xb7, 0x9b, 0xbc, 0xb7, 0xd9, 0xae, 0xae, 0xaa, - 0x19, 0x0a, 0xd7, 0x6e, 0x56, 0x1b, 0x6a, 0x96, 0x2e, 0x6d, 0x4b, 0x37, 0x7a, 0x7d, 0x73, 0xab, - 0xda, 0x1e, 0x34, 0xd5, 0x1c, 0xc5, 0x6f, 0x57, 0xa3, 0xf7, 0x3c, 0x45, 0xeb, 0xf4, 0xd7, 0xc4, - 0x2b, 0x3c, 0xfc, 0x7d, 0x28, 0xcf, 0x5f, 0x81, 0x52, 0x9f, 0xdc, 0x1c, 0xd4, 0xda, 0x7a, 0x5d, - 0xbd, 0x82, 0xde, 0x81, 0x1b, 0xfc, 0xd9, 0xac, 0x76, 0x1a, 0xac, 0x29, 0x2b, 0x58, 0x89, 0x87, - 0x9a, 0x3c, 0xa1, 0x30, 0x7f, 0xbd, 0x0e, 0xea, 0x5a, 0xbb, 0x6d, 0x6e, 0xb6, 0x07, 0x3d, 0xfe, - 0xb3, 0xf5, 0x48, 0xbd, 0x52, 0xbb, 0xfb, 0xf5, 0x7f, 0xdc, 0xba, 0xf2, 0xf5, 0xeb, 0x5b, 0x89, - 0x7f, 0x7e, 0x7d, 0x2b, 0xf1, 0xeb, 0xd7, 0xb7, 0x12, 0xff, 0xfe, 0xfa, 0x56, 0xe2, 0xcf, 0x7f, - 0x73, 0xeb, 0xca, 0x97, 0x30, 0x33, 0xf7, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0x6f, 0x6e, 0xfb, - 0xb0, 0xe0, 0x2c, 0x00, 0x00, + 0x76, 0x36, 0x9b, 0xcd, 0xbf, 0xc7, 0x1f, 0xb5, 0x6b, 0xe4, 0x31, 0x47, 0x33, 0xb1, 0xec, 0x5e, + 0xef, 0xf8, 0x67, 0x67, 0xa5, 0xb1, 0x67, 0x76, 0x32, 0x3b, 0xbb, 0x8b, 0x19, 0xfe, 0x4a, 0x94, + 0x29, 0x52, 0x6e, 0x92, 0xf2, 0xce, 0x1c, 0xd2, 0x69, 0xb2, 0x4b, 0x54, 0x5b, 0xcd, 0x2e, 0xba, + 0xbb, 0x69, 0x49, 0x46, 0x80, 0x9c, 0x82, 0x00, 0xb9, 0x24, 0xc0, 0x1e, 0x72, 0x0b, 0xf6, 0xb2, + 0xe7, 0x5c, 0x72, 0xc8, 0x29, 0xc0, 0x02, 0x41, 0x30, 0xc8, 0x25, 0xc9, 0x6d, 0x11, 0x04, 0x46, + 0xe2, 0x3d, 0x04, 0xb9, 0xe4, 0x16, 0x04, 0x98, 0x43, 0xb2, 0xa8, 0x9f, 0x6e, 0x36, 0xf5, 0x47, + 0xd1, 0x1e, 0xec, 0x5e, 0x24, 0xf6, 0xfb, 0xf9, 0xba, 0xea, 0xd5, 0xab, 0x57, 0xef, 0xbd, 0x6a, + 0x58, 0xf5, 0x9e, 0xd9, 0xeb, 0xa6, 0xe5, 0xf9, 0xde, 0x33, 0xdb, 0x9d, 0x38, 0xeb, 0x63, 0x97, + 0x0c, 0xb0, 0xe7, 0x11, 0xd7, 0x5b, 0x1b, 0xbb, 0xc4, 0x27, 0xa8, 0x38, 0x20, 0x83, 0x03, 0x97, + 0x18, 0x83, 0xfd, 0x35, 0xef, 0x99, 0xbd, 0x36, 0x15, 0x5d, 0x79, 0xfb, 0x29, 0xe9, 0x7b, 0xeb, + 0xf4, 0xcf, 0xb8, 0xcf, 0xfe, 0x71, 0x8d, 0x95, 0xab, 0x4c, 0x7a, 0xdc, 0x5f, 0x37, 0xc6, 0x96, + 0x20, 0xa1, 0x80, 0x64, 0x1a, 0xbe, 0x21, 0x68, 0xcb, 0x01, 0x0d, 0xbb, 0x6e, 0xf8, 0xba, 0x95, + 0x62, 0x40, 0xb5, 0xc8, 0xf7, 0xf7, 0x88, 0x3b, 0x32, 0xfc, 0x80, 0xf3, 0x1e, 0x1d, 0xa9, 0xf7, + 0xcc, 0xee, 0x1b, 0x1e, 0x5e, 0xf7, 0x7c, 0x77, 0x32, 0xf0, 0x27, 0x2e, 0x36, 0x05, 0x77, 0x35, + 0xca, 0xc5, 0xce, 0x80, 0x98, 0xd8, 0xd4, 0x4d, 0xc3, 0x9f, 0x8c, 0x84, 0xc0, 0xbb, 0x51, 0x81, + 0xa7, 0xc4, 0x72, 0x74, 0xff, 0x78, 0x8c, 0x05, 0xf3, 0x9d, 0x13, 0x56, 0x88, 0x0c, 0xb3, 0x38, + 0xf1, 0x2d, 0x7b, 0x7d, 0xdf, 0x1e, 0xac, 0xfb, 0xd6, 0x08, 0x7b, 0xbe, 0x31, 0x1a, 0x07, 0x13, + 0x18, 0x92, 0x21, 0x61, 0x3f, 0xd7, 0xe9, 0x2f, 0x4e, 0x55, 0xff, 0x3c, 0x0e, 0xf9, 0x9d, 0xc0, + 0x88, 0x9d, 0x31, 0x1e, 0xa0, 0x0a, 0x24, 0x2c, 0x67, 0x3c, 0xf1, 0x8b, 0xb1, 0x9b, 0xf1, 0xbb, + 0xd9, 0x87, 0x77, 0xd6, 0xce, 0xb3, 0xe8, 0x5a, 0x83, 0x8a, 0x75, 0x8e, 0x9d, 0x01, 0xd5, 0x2b, + 0xcb, 0x5f, 0xbf, 0x5c, 0xbd, 0xa2, 0x71, 0x5d, 0x54, 0x07, 0x79, 0x40, 0x5c, 0x5c, 0x94, 0x6e, + 0xc6, 0xee, 0x66, 0x1f, 0x7e, 0x70, 0x3e, 0x46, 0xf8, 0xee, 0x0a, 0x71, 0x71, 0xcf, 0xb1, 0x88, + 0x23, 0x80, 0x98, 0x3e, 0xda, 0x84, 0x24, 0x99, 0xf8, 0x74, 0x34, 0x71, 0x36, 0x9a, 0xfb, 0xe7, + 0x23, 0xb5, 0x99, 0x9c, 0x46, 0x26, 0x3e, 0x76, 0x23, 0x03, 0x12, 0xfa, 0xa8, 0x02, 0xf2, 0x98, + 0x78, 0x7e, 0x51, 0x66, 0x23, 0xba, 0x77, 0xc1, 0x88, 0x88, 0xe7, 0x8b, 0x51, 0x45, 0x60, 0x98, + 0x32, 0xba, 0x0f, 0x69, 0xcf, 0x37, 0x86, 0x58, 0xb7, 0xcc, 0x62, 0xe2, 0x66, 0xec, 0x6e, 0xa2, + 0xbc, 0x44, 0xb9, 0xaf, 0x5e, 0xae, 0xa6, 0x3a, 0x94, 0xde, 0xa8, 0x6a, 0x29, 0x26, 0xd0, 0x30, + 0xd1, 0x27, 0x90, 0x0b, 0xbd, 0x93, 0xca, 0x27, 0x99, 0xfc, 0x5b, 0x42, 0x3e, 0x1b, 0x4e, 0xbc, + 0x51, 0xd5, 0xb2, 0xa1, 0x60, 0xc3, 0x54, 0xff, 0x5a, 0x82, 0xa5, 0x13, 0x63, 0x40, 0x65, 0x48, + 0xee, 0x59, 0xb6, 0x8f, 0xdd, 0x62, 0x8c, 0x0d, 0xff, 0xf6, 0xf9, 0xc3, 0xaf, 0x1d, 0x8d, 0x5d, + 0xec, 0x79, 0x53, 0x43, 0x0a, 0x4d, 0x74, 0x1b, 0x60, 0xec, 0x92, 0xa7, 0x78, 0xe0, 0x5b, 0xc4, + 0x61, 0x0b, 0x93, 0x16, 0x12, 0x11, 0x3a, 0xba, 0x07, 0x05, 0x6e, 0x30, 0x7d, 0x40, 0xec, 0xc9, + 0xc8, 0xf1, 0x98, 0xe1, 0xf3, 0x65, 0x49, 0x89, 0x69, 0x79, 0xce, 0xa9, 0x70, 0x06, 0xda, 0x86, + 0x9c, 0x8b, 0x1d, 0x13, 0xbb, 0x3a, 0x3e, 0x1a, 0xbb, 0x5e, 0x51, 0x66, 0x2b, 0xb4, 0xc8, 0xd0, + 0xb2, 0x5c, 0x9f, 0xd2, 0x3d, 0xf4, 0x1e, 0x24, 0xc9, 0xde, 0x9e, 0x87, 0x7d, 0x66, 0x59, 0x39, + 0x5c, 0x3e, 0x46, 0x43, 0x2b, 0x90, 0xb0, 0xad, 0x91, 0xe5, 0x33, 0x33, 0x06, 0x4c, 0x4e, 0x52, + 0xff, 0xbf, 0x00, 0xe8, 0xb4, 0x1f, 0xa1, 0xcf, 0x40, 0x76, 0x08, 0x19, 0x0b, 0x93, 0xbd, 0x7f, + 0xfe, 0xb8, 0x5a, 0x84, 0x8c, 0xa9, 0x1a, 0x35, 0xb5, 0xc6, 0x74, 0xd0, 0x23, 0xc8, 0xfa, 0x46, + 0xdf, 0xc6, 0x1a, 0x36, 0x4c, 0xec, 0x0a, 0x37, 0xbe, 0xc0, 0x69, 0xba, 0x53, 0x61, 0x86, 0x12, + 0xd5, 0x46, 0x9b, 0x00, 0x74, 0x07, 0x0b, 0xac, 0x38, 0xc3, 0xba, 0x7b, 0x3e, 0xd6, 0x56, 0x28, + 0xcb, 0xa0, 0x22, 0xba, 0xe8, 0xc7, 0x90, 0xf4, 0x88, 0x4b, 0xfd, 0x40, 0x9e, 0xe7, 0x07, 0x1d, + 0x26, 0xc7, 0x10, 0x84, 0x0e, 0x1d, 0x87, 0x31, 0x1c, 0xba, 0x78, 0x68, 0xf8, 0xc4, 0x65, 0x56, + 0xbe, 0x70, 0x1c, 0xa5, 0x50, 0x96, 0x8f, 0x63, 0xaa, 0x8b, 0xca, 0x90, 0xa6, 0x82, 0x96, 0x33, + 0xf0, 0x8b, 0xa9, 0x79, 0xe6, 0xad, 0x0a, 0x49, 0x86, 0x12, 0xea, 0x51, 0x13, 0x8f, 0xb0, 0x3b, + 0xc4, 0x74, 0xba, 0xd8, 0x2d, 0xa6, 0xe7, 0x99, 0x78, 0x7b, 0x2a, 0xcc, 0x4d, 0x1c, 0xd1, 0xa6, + 0x53, 0xdb, 0x37, 0xbc, 0x7d, 0x81, 0x95, 0x99, 0x37, 0xb5, 0xcd, 0x50, 0x96, 0x4f, 0x6d, 0xaa, + 0x8b, 0xbe, 0x80, 0xe4, 0x73, 0xc3, 0x9e, 0x60, 0xaf, 0x08, 0xf3, 0x50, 0x76, 0x99, 0x5c, 0xe8, + 0x39, 0x42, 0x8f, 0x8e, 0xa5, 0x6f, 0x0c, 0x0e, 0xf6, 0x2c, 0xdb, 0xc6, 0x6e, 0x31, 0x3b, 0x0f, + 0xa5, 0x1c, 0xca, 0xf2, 0xb1, 0x4c, 0x75, 0x51, 0x13, 0xc0, 0xc5, 0x86, 0xd9, 0x18, 0x8d, 0x89, + 0xeb, 0x17, 0xf3, 0xf3, 0x62, 0xa9, 0x16, 0xca, 0x56, 0x0d, 0xdf, 0xe0, 0x68, 0x53, 0x7d, 0x54, + 0x83, 0x4c, 0xa7, 0xd3, 0x7d, 0xe2, 0x5a, 0xd4, 0x7f, 0x0a, 0x0c, 0xec, 0x82, 0xe0, 0x1e, 0x8a, + 0x32, 0x9c, 0xa9, 0x26, 0xfa, 0x1c, 0x52, 0x1d, 0x63, 0x34, 0xa6, 0x73, 0x5b, 0x62, 0x20, 0xdf, + 0xbd, 0x00, 0x84, 0x0b, 0x32, 0x88, 0x40, 0x0b, 0x7d, 0x05, 0x0a, 0xff, 0x39, 0x75, 0xb0, 0xa2, + 0xc2, 0x90, 0xd6, 0xe6, 0x21, 0x9d, 0x70, 0xc9, 0x53, 0x38, 0x88, 0xc0, 0x75, 0xcb, 0xf1, 0xb1, + 0x6b, 0x63, 0xe3, 0x39, 0x36, 0xf9, 0xae, 0x11, 0x4e, 0x71, 0x95, 0xbd, 0xe2, 0x07, 0x17, 0x1d, + 0x67, 0x67, 0x2a, 0xb2, 0x37, 0x9d, 0x87, 0x8a, 0xfe, 0x10, 0xd0, 0x08, 0xfb, 0x06, 0x3d, 0x81, + 0xbb, 0xd8, 0xf3, 0x3b, 0x2c, 0xa0, 0x15, 0x11, 0x7b, 0xd7, 0x87, 0x17, 0x39, 0xf3, 0x49, 0x1d, + 0xf6, 0x9a, 0x33, 0xb0, 0xd0, 0x1e, 0x2c, 0x47, 0xa9, 0x1a, 0x1e, 0x60, 0xeb, 0x39, 0x76, 0x8b, + 0x6f, 0xb1, 0x77, 0x3c, 0xbc, 0xdc, 0x3b, 0x02, 0x2d, 0xf6, 0x96, 0x33, 0xf1, 0xa8, 0x7b, 0x54, + 0x3a, 0xbb, 0xc2, 0x3d, 0x96, 0xe7, 0xb9, 0x47, 0x28, 0xca, 0xdd, 0x23, 0x7c, 0x44, 0x2d, 0xc8, + 0xbd, 0xb0, 0x86, 0x2f, 0x8c, 0xa1, 0x30, 0xfb, 0x35, 0x86, 0x74, 0xc1, 0xb9, 0xfd, 0x55, 0x44, + 0x9a, 0x81, 0xcd, 0xe8, 0xd3, 0xdd, 0x24, 0x8e, 0xa7, 0x0e, 0xf6, 0x8b, 0x6f, 0xcf, 0xdb, 0x4d, + 0x3b, 0xa1, 0x2c, 0xf7, 0xff, 0xa9, 0x2e, 0x0d, 0x5a, 0x87, 0x96, 0x63, 0x92, 0x43, 0xec, 0x16, + 0xaf, 0xcf, 0x0b, 0x5a, 0x4f, 0x84, 0x24, 0x0f, 0x5a, 0x81, 0x1e, 0x7a, 0x0c, 0x79, 0x9b, 0x0c, + 0x0c, 0x7b, 0xc7, 0x36, 0x9c, 0x16, 0x31, 0x71, 0xb1, 0xc8, 0x80, 0xbe, 0x77, 0x3e, 0x50, 0x33, + 0x2a, 0xce, 0xd0, 0x66, 0x11, 0xe8, 0x76, 0x18, 0xec, 0x1b, 0xce, 0x30, 0xba, 0x1d, 0xde, 0x99, + 0xb7, 0x1d, 0x2a, 0x27, 0x34, 0xf8, 0x76, 0x38, 0x89, 0x83, 0xba, 0x50, 0xe0, 0xb4, 0xba, 0x4b, + 0x1c, 0xdf, 0xc2, 0x6e, 0x71, 0x65, 0x5e, 0x10, 0xa9, 0xcc, 0xc8, 0x33, 0xdc, 0x13, 0x18, 0x9f, + 0xc9, 0x5f, 0xff, 0x7c, 0x35, 0xb6, 0x25, 0xa7, 0x93, 0x4a, 0x6a, 0x4b, 0x4e, 0xe7, 0x94, 0xbc, + 0x5a, 0x80, 0x5c, 0xf4, 0x10, 0x55, 0x3f, 0x86, 0xb7, 0xcf, 0xf6, 0x70, 0xb4, 0x02, 0x92, 0x65, + 0xb2, 0x23, 0x39, 0x53, 0x06, 0x91, 0x0b, 0x49, 0x8d, 0xaa, 0x26, 0x59, 0xa6, 0xba, 0x09, 0xc5, + 0xf3, 0x7c, 0x16, 0x7d, 0x00, 0xe0, 0xf1, 0x64, 0xc3, 0x32, 0x3d, 0x96, 0x9a, 0x66, 0xca, 0xf9, + 0x57, 0x2f, 0x57, 0x33, 0x1c, 0xbb, 0x51, 0xf5, 0xb4, 0x0c, 0x17, 0x68, 0x98, 0x9e, 0xfa, 0xb3, + 0x18, 0x14, 0x66, 0xa3, 0x33, 0xaa, 0x40, 0x2a, 0xc8, 0x68, 0x78, 0x62, 0xfb, 0x9d, 0x0b, 0x4e, + 0x2c, 0x9a, 0x88, 0x37, 0x9c, 0x3d, 0x22, 0xf2, 0x8c, 0x40, 0x13, 0xbd, 0x0b, 0x19, 0xd7, 0x38, + 0xd4, 0xfb, 0xc7, 0x3e, 0xf6, 0x8a, 0xd2, 0xcd, 0xf8, 0xdd, 0x9c, 0x96, 0x76, 0x8d, 0xc3, 0x32, + 0x7d, 0x46, 0xab, 0x90, 0x76, 0x26, 0x23, 0xdd, 0x25, 0x87, 0x1e, 0x3b, 0xe4, 0x83, 0x2c, 0x25, + 0xe5, 0x4c, 0x46, 0x1a, 0x39, 0xf4, 0xd4, 0x2a, 0x2c, 0xcd, 0xe4, 0x09, 0x86, 0x83, 0x1e, 0x80, + 0xec, 0x8d, 0x0d, 0x47, 0xe4, 0x28, 0xd7, 0x23, 0x43, 0x12, 0x85, 0xc5, 0x1a, 0x15, 0x0b, 0x72, + 0x50, 0x2a, 0xaa, 0xfe, 0x8f, 0x74, 0x02, 0x86, 0xe5, 0x87, 0x09, 0x96, 0x70, 0x9c, 0x93, 0xeb, + 0x88, 0x3a, 0x82, 0x67, 0x29, 0x55, 0xec, 0x0d, 0x5c, 0x6b, 0xec, 0x13, 0x37, 0xc8, 0xa2, 0x98, + 0x2a, 0xba, 0x05, 0x19, 0xcb, 0x31, 0xf1, 0x91, 0x6e, 0x99, 0x47, 0x2c, 0xe1, 0xc9, 0x0b, 0x7e, + 0x9a, 0x91, 0x1b, 0xe6, 0x11, 0xba, 0x01, 0x29, 0x17, 0x3f, 0xc7, 0xae, 0x87, 0xd9, 0x04, 0x83, + 0xfc, 0x31, 0x20, 0xa2, 0x1a, 0x24, 0xe8, 0x10, 0x83, 0x54, 0xf0, 0xb2, 0xf9, 0x52, 0x38, 0x41, + 0xae, 0x8d, 0xbe, 0x03, 0xc0, 0x12, 0x3b, 0x7d, 0xdf, 0x72, 0x78, 0x36, 0x18, 0x17, 0x02, 0x19, + 0x46, 0xdf, 0xb4, 0x1c, 0x9f, 0x5a, 0xdb, 0xf2, 0xf4, 0xc1, 0x3e, 0x1e, 0x1c, 0xb0, 0x9c, 0x30, + 0x1c, 0x8c, 0xe5, 0x55, 0x28, 0x11, 0xb5, 0x00, 0x9e, 0x5b, 0x9e, 0xd5, 0xb7, 0x6c, 0xcb, 0x3f, + 0x66, 0x59, 0x4a, 0xe1, 0xa2, 0xc0, 0xd1, 0x19, 0x18, 0xce, 0x6e, 0x28, 0x1f, 0x64, 0xc6, 0x53, + 0x04, 0xf5, 0x1f, 0xe3, 0x50, 0x98, 0x4d, 0xcd, 0x7e, 0x5b, 0x66, 0xbf, 0x07, 0x05, 0x9b, 0x90, + 0x83, 0xc9, 0xf8, 0xac, 0x9c, 0x9c, 0x73, 0x82, 0x9c, 0xbc, 0x02, 0x29, 0xe2, 0xb0, 0x7c, 0x7c, + 0x7e, 0x86, 0x78, 0xba, 0x52, 0x20, 0x0e, 0xa5, 0xa1, 0x5d, 0xb8, 0xca, 0x87, 0xc4, 0x2b, 0x07, + 0x0e, 0x97, 0x58, 0x18, 0x6e, 0x89, 0x81, 0xd4, 0x19, 0x06, 0xc3, 0xfd, 0x21, 0xc8, 0xb4, 0x88, + 0x65, 0xcb, 0x55, 0x78, 0xb8, 0x7a, 0x8e, 0xb5, 0xa8, 0x8d, 0xbb, 0xc7, 0x63, 0x1c, 0x38, 0x3d, + 0x55, 0xf9, 0xd6, 0x17, 0xf3, 0x67, 0x31, 0x80, 0x69, 0x86, 0x8c, 0x1e, 0xc3, 0x92, 0xa8, 0x7a, + 0x88, 0x6b, 0x62, 0xd7, 0x72, 0x86, 0x62, 0x49, 0xd5, 0x0b, 0xea, 0x4d, 0x21, 0x29, 0xd0, 0x45, + 0xd9, 0x14, 0x50, 0xd1, 0x43, 0x40, 0x01, 0x96, 0x3e, 0x32, 0xfc, 0xc1, 0xbe, 0x6e, 0x63, 0x67, + 0x66, 0x81, 0x95, 0x80, 0xbf, 0x4d, 0xd9, 0x4d, 0xec, 0xa8, 0x7d, 0xc8, 0x45, 0x93, 0x65, 0x74, + 0x07, 0x96, 0x98, 0x0c, 0x36, 0xf5, 0x68, 0xec, 0xca, 0x6b, 0x05, 0x41, 0x0e, 0x96, 0xfd, 0x1e, + 0x28, 0x41, 0x5e, 0x1d, 0x4a, 0x4a, 0x4c, 0x72, 0x29, 0xa0, 0x0b, 0x51, 0xf5, 0x97, 0x12, 0x28, + 0x27, 0x8f, 0x5c, 0x54, 0x85, 0x24, 0xf3, 0xc6, 0x20, 0x36, 0x2e, 0xe6, 0xc9, 0x42, 0x17, 0xd5, + 0x01, 0xf0, 0xb3, 0x99, 0xf7, 0x67, 0x1f, 0xde, 0xba, 0xe0, 0xa4, 0xe1, 0x82, 0xc1, 0xd6, 0xc6, + 0xcf, 0x82, 0xd9, 0xac, 0x4e, 0xb7, 0x44, 0xd4, 0xd5, 0x83, 0x0d, 0xf1, 0x2d, 0x79, 0x79, 0xe0, + 0x8d, 0x89, 0x85, 0xbd, 0x51, 0xb5, 0xe0, 0xea, 0xa9, 0x63, 0x1d, 0xa9, 0x90, 0xd3, 0xc8, 0x61, + 0x87, 0x4c, 0xdc, 0x01, 0x6e, 0x98, 0x47, 0xcc, 0x81, 0xf2, 0xda, 0x0c, 0x0d, 0xbd, 0x07, 0x99, + 0x16, 0x3d, 0x59, 0xc6, 0x13, 0xdf, 0xe3, 0xbe, 0xa0, 0x4d, 0x09, 0x08, 0x81, 0xdc, 0x32, 0x46, + 0x3c, 0xb6, 0x66, 0x34, 0xf6, 0x5b, 0xbd, 0x03, 0xa9, 0xc0, 0x2c, 0xef, 0xcd, 0x9e, 0x60, 0xdc, + 0x28, 0x01, 0x49, 0xfd, 0x37, 0x09, 0x96, 0x4e, 0x94, 0x48, 0x68, 0x1b, 0xf2, 0x36, 0xde, 0x7b, + 0x03, 0xa7, 0xce, 0x51, 0xf5, 0xd0, 0xa5, 0xdb, 0x50, 0x70, 0xad, 0xe1, 0x7e, 0x04, 0x4f, 0x5a, + 0x10, 0x2f, 0xcf, 0xf4, 0x43, 0xc0, 0xc8, 0x3a, 0x26, 0xde, 0x78, 0x1d, 0x5f, 0x23, 0xaa, 0xdc, + 0x83, 0xbc, 0x33, 0xb1, 0x6d, 0x1d, 0x3f, 0x9b, 0x18, 0x61, 0x60, 0x09, 0x0e, 0x92, 0x1c, 0x65, + 0xd5, 0x04, 0x47, 0xfd, 0x4b, 0x09, 0x0a, 0xb3, 0x55, 0x23, 0xba, 0x0f, 0x4b, 0xcc, 0xba, 0x11, + 0x9f, 0x8f, 0x45, 0xe2, 0x32, 0xde, 0xf3, 0x6b, 0xa1, 0x4b, 0x7f, 0x00, 0x0a, 0x37, 0xdd, 0x89, + 0x0d, 0xc2, 0x85, 0xb9, 0x59, 0xa7, 0xd2, 0xbf, 0x6b, 0xbb, 0x7c, 0x0f, 0x0a, 0xac, 0xb8, 0x9e, + 0x86, 0x9d, 0xa8, 0x61, 0xf2, 0x9c, 0x17, 0x04, 0x94, 0xff, 0x4e, 0x42, 0x61, 0x36, 0x11, 0x45, + 0xb7, 0x00, 0x86, 0x2e, 0xe1, 0xe7, 0x55, 0x74, 0x9e, 0x19, 0x46, 0xad, 0x10, 0xdb, 0x43, 0x7f, + 0x00, 0xb9, 0xa0, 0x9f, 0x60, 0x11, 0x71, 0xa2, 0x65, 0x1f, 0x7e, 0x7c, 0xd9, 0x6e, 0x44, 0xf8, + 0x38, 0x9d, 0xf7, 0x0c, 0x1e, 0xfa, 0x50, 0x84, 0x5f, 0x6c, 0xea, 0x91, 0xa1, 0xc8, 0xe1, 0x50, + 0x14, 0xc1, 0xdd, 0x08, 0x47, 0xb4, 0x31, 0x13, 0x0f, 0xbe, 0x7f, 0xe9, 0x91, 0x9c, 0xb4, 0xde, + 0xca, 0x9f, 0x4a, 0x90, 0x8d, 0x0c, 0x8f, 0x02, 0xef, 0x4d, 0x9c, 0x01, 0xdb, 0x7c, 0x8b, 0x00, + 0xd7, 0x27, 0x4e, 0xd8, 0x7d, 0xa4, 0x00, 0xe8, 0x66, 0xa4, 0xeb, 0x12, 0xed, 0xdf, 0x4d, 0x7b, + 0x2a, 0xb7, 0xa1, 0x20, 0xce, 0xec, 0x01, 0xb1, 0x59, 0x46, 0x21, 0xf3, 0x28, 0xc4, 0xa9, 0x15, + 0x62, 0xd3, 0x28, 0x74, 0x9d, 0x05, 0x12, 0xc6, 0x4e, 0xb0, 0x43, 0x22, 0x39, 0xe0, 0x8c, 0x4d, + 0xc8, 0x18, 0xee, 0x70, 0x32, 0xc2, 0x8e, 0xef, 0x15, 0x93, 0x0b, 0xb7, 0xf3, 0xa6, 0xca, 0x5b, + 0x72, 0x3a, 0xae, 0xc8, 0xea, 0x2f, 0x24, 0x90, 0xe9, 0x2c, 0x90, 0x02, 0xb9, 0x52, 0xeb, 0x4b, + 0xbd, 0xd5, 0xee, 0xea, 0xad, 0x5e, 0xb3, 0xa9, 0x5c, 0x41, 0x29, 0x88, 0x97, 0x76, 0x37, 0x94, + 0x18, 0xca, 0x41, 0xba, 0xdc, 0x6e, 0x37, 0xf5, 0x52, 0xab, 0xaa, 0x48, 0x28, 0x0b, 0x29, 0xf6, + 0xd4, 0xd6, 0x94, 0x38, 0x2a, 0x00, 0x54, 0xda, 0xad, 0x4a, 0xa9, 0xab, 0x97, 0x36, 0x36, 0x14, + 0x19, 0x65, 0x20, 0x51, 0x69, 0xf7, 0x5a, 0x5d, 0x25, 0x41, 0xd5, 0xb7, 0x4b, 0x3f, 0x55, 0x52, + 0xec, 0x47, 0xa3, 0xa5, 0xa4, 0x11, 0x40, 0xb2, 0xd3, 0xad, 0x56, 0x6b, 0xbb, 0x4a, 0x86, 0x12, + 0x3b, 0xbd, 0x6d, 0x05, 0x28, 0x5c, 0xa7, 0xb7, 0xad, 0x37, 0x5a, 0x5d, 0x25, 0x4b, 0xdf, 0xb4, + 0x5b, 0xd2, 0x1a, 0xa5, 0x56, 0xa5, 0xa6, 0xe4, 0x28, 0xeb, 0xa7, 0x6d, 0x8d, 0x21, 0xe7, 0xf9, + 0x9b, 0x7a, 0xad, 0xae, 0xae, 0xb5, 0x9f, 0x74, 0x94, 0x02, 0xd3, 0x7b, 0xac, 0x55, 0x1b, 0xf5, + 0xba, 0xb2, 0x84, 0x10, 0x14, 0xea, 0x8d, 0x56, 0xa9, 0xa9, 0x87, 0xda, 0x0a, 0x9d, 0x10, 0xa7, + 0x89, 0x77, 0x5e, 0x45, 0x79, 0xc8, 0x94, 0x34, 0xad, 0xf4, 0x25, 0x43, 0x44, 0xf4, 0x65, 0x5b, + 0x9d, 0x76, 0x8b, 0x3d, 0xbd, 0x45, 0x99, 0xf4, 0xa9, 0xcc, 0x1e, 0x97, 0xe9, 0xeb, 0x3a, 0x5d, + 0xad, 0xd1, 0xda, 0x60, 0xcf, 0xd7, 0xd4, 0x0f, 0x40, 0xa6, 0x5e, 0x84, 0xd2, 0x20, 0x97, 0x7a, + 0xdd, 0xb6, 0x72, 0x85, 0xcd, 0xa6, 0x52, 0x6a, 0x96, 0x34, 0x25, 0x46, 0xa5, 0x5b, 0xed, 0x96, + 0x2e, 0x9e, 0x25, 0xf5, 0x9b, 0x38, 0x14, 0x66, 0x9b, 0x46, 0xa1, 0xef, 0xce, 0x75, 0xb1, 0x59, + 0xbd, 0x53, 0xbe, 0x3b, 0xcd, 0x68, 0xa5, 0xd7, 0xcf, 0x68, 0xc3, 0x2a, 0x20, 0xfe, 0x46, 0x55, + 0xc0, 0x03, 0x48, 0x9b, 0x13, 0x97, 0x6d, 0x21, 0xe6, 0xc5, 0xf1, 0xf2, 0x35, 0xca, 0xfe, 0xe6, + 0xe5, 0x6a, 0xde, 0xb7, 0x46, 0x78, 0xad, 0x2a, 0x98, 0x5a, 0x28, 0x46, 0x0b, 0x87, 0xc1, 0xfe, + 0xc4, 0x39, 0xd0, 0x3d, 0xeb, 0x05, 0x9e, 0x2d, 0x1c, 0x18, 0xbd, 0x63, 0xbd, 0xc0, 0xa8, 0x0d, + 0x39, 0xe2, 0xef, 0x63, 0x57, 0x17, 0x19, 0x4f, 0xf2, 0x35, 0x32, 0x9e, 0x2c, 0x43, 0xe8, 0xf2, + 0xb4, 0xe7, 0x73, 0x48, 0xbb, 0xd8, 0x30, 0x4b, 0x5e, 0x7b, 0x4f, 0x34, 0x43, 0x7f, 0x2f, 0x02, + 0x36, 0xf1, 0x2d, 0x7b, 0x6d, 0xdf, 0x1e, 0xac, 0x75, 0x83, 0xfb, 0x98, 0x60, 0xd7, 0x06, 0x4a, + 0xea, 0x7d, 0xb1, 0xfc, 0x59, 0x48, 0x35, 0x9c, 0xe7, 0x86, 0x6d, 0x99, 0xdc, 0x03, 0x78, 0x84, + 0x55, 0x62, 0xd4, 0xf1, 0x1b, 0x34, 0xf7, 0x51, 0x24, 0xf5, 0xd7, 0x31, 0x48, 0xd7, 0x6d, 0x72, + 0xc8, 0x96, 0xfd, 0x01, 0xa4, 0xf6, 0x6c, 0x72, 0xa8, 0x8b, 0x8a, 0x3a, 0x57, 0x2e, 0x52, 0xe4, + 0x7f, 0x7d, 0xb9, 0x9a, 0xa4, 0x22, 0x8d, 0xea, 0xab, 0xf0, 0x97, 0x96, 0xa4, 0x82, 0x0d, 0x13, + 0x6d, 0xb3, 0x76, 0x8a, 0xb8, 0x33, 0x13, 0x39, 0xda, 0x9d, 0x4b, 0x5c, 0xcf, 0x44, 0xae, 0x42, + 0x22, 0x00, 0xa8, 0x07, 0xa9, 0xa1, 0xe1, 0xe3, 0x43, 0xe3, 0x98, 0x65, 0x2d, 0x89, 0xf2, 0x8f, + 0xc4, 0x1a, 0x7d, 0x34, 0xb4, 0xfc, 0xfd, 0x49, 0x7f, 0x6d, 0x40, 0x46, 0xeb, 0x21, 0xba, 0xd9, + 0x9f, 0xfe, 0x5e, 0x1f, 0x1f, 0x0c, 0xd7, 0x83, 0x22, 0x97, 0x26, 0x51, 0x8d, 0xaa, 0x16, 0x60, + 0xa9, 0x87, 0x90, 0xdd, 0x22, 0xfd, 0x1d, 0x97, 0x0c, 0x69, 0x84, 0x41, 0xb7, 0x21, 0xf9, 0x94, + 0xf4, 0x83, 0x69, 0xc6, 0xcb, 0x79, 0xd1, 0x38, 0x48, 0x6c, 0x91, 0x7e, 0xa3, 0xaa, 0x25, 0x9e, + 0x92, 0x7e, 0xc3, 0x44, 0x77, 0x21, 0x37, 0x20, 0x8e, 0xef, 0x5a, 0xfd, 0x49, 0x78, 0xc5, 0x21, + 0x05, 0x87, 0x43, 0x94, 0x83, 0x8a, 0x20, 0x7b, 0x36, 0xf1, 0xc5, 0x90, 0x83, 0xe2, 0xda, 0x26, + 0xbe, 0xfa, 0x2b, 0x19, 0xd0, 0xe9, 0x36, 0x2a, 0xfa, 0x2e, 0x64, 0x3d, 0xd6, 0x6a, 0xe4, 0x9e, + 0x25, 0x45, 0xf4, 0x80, 0x33, 0x98, 0x6b, 0x6d, 0x40, 0x7a, 0x2c, 0xc6, 0xcc, 0x8e, 0xdd, 0x0b, + 0x7b, 0xa3, 0x91, 0x09, 0x06, 0x1e, 0x11, 0x28, 0xa3, 0x0d, 0x88, 0x4f, 0x5c, 0xab, 0x98, 0x62, + 0xcb, 0xf3, 0x83, 0x45, 0x3a, 0xbe, 0x6b, 0x3d, 0xd7, 0xaa, 0x39, 0xbe, 0x7b, 0xac, 0x51, 0x04, + 0xf4, 0x13, 0x48, 0xf2, 0x6b, 0x49, 0xd1, 0x5f, 0x5f, 0x3d, 0xa3, 0xc3, 0xd0, 0x68, 0xd7, 0x2d, + 0x1b, 0xd7, 0x99, 0x58, 0x78, 0x67, 0xc4, 0x9e, 0xd0, 0x6e, 0x58, 0x17, 0x64, 0xd8, 0x50, 0x3e, + 0x5d, 0x68, 0x28, 0x7c, 0x7f, 0xb0, 0xd1, 0x30, 0xdc, 0x58, 0x58, 0x29, 0x7c, 0x0e, 0xef, 0x78, + 0x07, 0xd6, 0x58, 0x1f, 0x59, 0x9e, 0x47, 0x0b, 0xa4, 0x3d, 0xe2, 0x62, 0x6b, 0xe8, 0xe8, 0x07, + 0xf8, 0x98, 0xf7, 0xdd, 0x83, 0xa3, 0xed, 0x6d, 0x2a, 0xb6, 0xcd, 0xa5, 0xea, 0x5c, 0xe8, 0x11, + 0x3e, 0xf6, 0x56, 0x0c, 0xc8, 0x46, 0xd0, 0x91, 0x02, 0xf1, 0x03, 0x7c, 0xcc, 0xdb, 0x4a, 0x1a, + 0xfd, 0x89, 0x7e, 0x0c, 0x09, 0xd6, 0x8e, 0x5f, 0x2c, 0x90, 0x69, 0x5c, 0xe9, 0x33, 0xe9, 0xd3, + 0xd8, 0xca, 0x27, 0x90, 0x0e, 0x6c, 0x19, 0xc5, 0x4f, 0x70, 0xfc, 0xe5, 0x28, 0x7e, 0x26, 0xa2, + 0xb7, 0x25, 0xa7, 0x63, 0x8a, 0xc4, 0x0f, 0xc0, 0x2d, 0x39, 0x2d, 0x2b, 0x89, 0x2d, 0x39, 0x9d, + 0x50, 0x92, 0xea, 0xdf, 0x4a, 0x90, 0x9f, 0x69, 0xaa, 0xa3, 0xf7, 0x21, 0x6b, 0x62, 0x7a, 0x72, + 0xf3, 0x20, 0xc7, 0x9b, 0x62, 0x22, 0xc0, 0x44, 0x18, 0xe8, 0x3e, 0xe4, 0x0f, 0x0d, 0xdb, 0xa6, + 0x51, 0xaf, 0x65, 0x38, 0x84, 0x77, 0x97, 0x82, 0xc8, 0x36, 0xcb, 0x42, 0xcd, 0xd9, 0x16, 0xcc, + 0x87, 0x97, 0x6c, 0xf0, 0xb3, 0x36, 0x13, 0x2d, 0x38, 0x66, 0x63, 0x70, 0xd4, 0xa1, 0x13, 0x6f, + 0xe0, 0xd0, 0xd4, 0x98, 0xc1, 0x1b, 0xe8, 0xee, 0x73, 0x68, 0x99, 0x13, 0x9d, 0x2f, 0xa3, 0x50, + 0x33, 0x63, 0xc7, 0x64, 0x26, 0xcd, 0x69, 0xf4, 0xe7, 0x96, 0x9c, 0x96, 0x94, 0xb8, 0xfa, 0xf7, + 0x31, 0xc8, 0xcf, 0x34, 0x9c, 0x2f, 0x6d, 0xba, 0x3b, 0x90, 0xa3, 0xc8, 0xfa, 0xd8, 0xf0, 0x7d, + 0xec, 0xf2, 0x98, 0x10, 0x0a, 0x52, 0xce, 0x0e, 0x67, 0xa0, 0x9f, 0x40, 0x8a, 0x8c, 0x83, 0x54, + 0xf4, 0x64, 0x0c, 0x0f, 0x76, 0x4a, 0xa5, 0xb3, 0xdb, 0xe6, 0x42, 0x41, 0xb3, 0x49, 0xe8, 0x4c, + 0x4f, 0x1e, 0xd6, 0xfd, 0x93, 0x4f, 0x9d, 0x3c, 0xac, 0xff, 0xf7, 0x27, 0x12, 0x40, 0xe7, 0x00, + 0xfb, 0x83, 0x7d, 0x36, 0x87, 0x47, 0x90, 0xf5, 0xd8, 0x93, 0x1e, 0x39, 0xbb, 0x2f, 0xba, 0xd1, + 0x63, 0xc2, 0x91, 0x23, 0x1b, 0xbc, 0x90, 0x82, 0x8a, 0xd3, 0xe2, 0x90, 0x17, 0xfe, 0x61, 0xcf, + 0xf2, 0x23, 0x40, 0x43, 0xec, 0x60, 0xd7, 0xf0, 0xb1, 0xbe, 0x6f, 0x79, 0x3e, 0x19, 0xba, 0xc6, + 0x68, 0xa6, 0x7f, 0x77, 0x35, 0xe0, 0x6f, 0x06, 0x6c, 0xf4, 0x29, 0x5c, 0x0b, 0x65, 0xf5, 0x91, + 0x71, 0xa4, 0xf7, 0x27, 0x83, 0x03, 0xec, 0xf3, 0xa9, 0x05, 0x0d, 0x8c, 0xb7, 0x42, 0x91, 0x6d, + 0xe3, 0xa8, 0xcc, 0x05, 0xd0, 0x2d, 0xc8, 0x78, 0xbe, 0xe1, 0xeb, 0x6c, 0x89, 0x13, 0x11, 0x73, + 0xa7, 0x29, 0x99, 0xd5, 0xb4, 0x7f, 0x04, 0xd9, 0xc8, 0xc5, 0x10, 0xaa, 0x43, 0x9a, 0x4f, 0x24, + 0x6c, 0x3f, 0xcc, 0x35, 0x42, 0xe4, 0x34, 0x0a, 0x75, 0xcf, 0x0a, 0xd2, 0xf9, 0xd3, 0x41, 0x5a, + 0xfd, 0x2f, 0x09, 0x96, 0xcf, 0xba, 0x4d, 0xfa, 0x2d, 0x8f, 0x03, 0xfd, 0x31, 0x20, 0xfe, 0x14, + 0x54, 0x59, 0x91, 0x76, 0xc7, 0xe3, 0x57, 0x2f, 0x57, 0xc5, 0xe5, 0x96, 0xa8, 0xb3, 0x1a, 0x55, + 0xef, 0x9b, 0x97, 0xab, 0x3f, 0xbc, 0xd4, 0xa9, 0x1a, 0xf9, 0x74, 0x64, 0x2d, 0xd0, 0xd6, 0x14, + 0x6f, 0x06, 0xce, 0xf4, 0x90, 0x01, 0x69, 0x16, 0x8e, 0xe9, 0xb9, 0xca, 0x97, 0xb5, 0x1e, 0x7c, + 0xcc, 0xc0, 0xc2, 0x62, 0xa3, 0x7a, 0xe9, 0x73, 0x3c, 0xfa, 0x46, 0x7a, 0x8e, 0x33, 0xdc, 0x86, + 0xa9, 0xfe, 0x9f, 0x0c, 0xef, 0x5e, 0x70, 0xad, 0x86, 0xbe, 0x3c, 0xd1, 0x77, 0xfa, 0xd1, 0x6b, + 0xdd, 0xce, 0xf1, 0x28, 0x7e, 0xa2, 0x19, 0x15, 0xe9, 0x55, 0x4b, 0x67, 0xf5, 0xaa, 0x67, 0x9b, + 0xcc, 0xf1, 0xb3, 0x9b, 0xcc, 0xbf, 0xe3, 0x46, 0xd3, 0xca, 0xdf, 0x49, 0x90, 0x60, 0x93, 0x43, + 0x5f, 0x80, 0x6c, 0x62, 0x6f, 0xf0, 0x5a, 0x9d, 0x66, 0xa6, 0x79, 0x99, 0x46, 0x73, 0xf0, 0x8d, + 0x4c, 0xfc, 0x4d, 0xbe, 0x91, 0xa9, 0x42, 0x3a, 0xec, 0x0f, 0xc9, 0x0b, 0xf6, 0x87, 0x42, 0xcd, + 0x69, 0x11, 0x91, 0x78, 0x93, 0x22, 0x42, 0xfd, 0x97, 0x18, 0x14, 0x66, 0xaf, 0x04, 0xd1, 0x17, + 0x90, 0xe0, 0xdf, 0xab, 0xc4, 0x16, 0x2e, 0x70, 0xb9, 0x22, 0xea, 0x42, 0x18, 0x31, 0xcd, 0x39, + 0xed, 0xce, 0xd9, 0x6d, 0x18, 0x59, 0x5f, 0x25, 0x44, 0x08, 0x9a, 0x3e, 0x77, 0x40, 0x71, 0x26, + 0x23, 0xd6, 0xa7, 0xd0, 0xc7, 0xd8, 0xd5, 0x87, 0xd8, 0xe1, 0xd1, 0x40, 0xcb, 0x3b, 0x93, 0x51, + 0x85, 0xd8, 0xde, 0x0e, 0x76, 0x37, 0xb0, 0xa3, 0xfe, 0x3c, 0x07, 0xb9, 0xe8, 0xf5, 0x24, 0xba, + 0x09, 0xd9, 0xb1, 0xe1, 0xfa, 0x16, 0x6b, 0x86, 0x1c, 0x8b, 0x16, 0x71, 0x94, 0x84, 0x3a, 0x90, + 0xe1, 0x57, 0x98, 0xf5, 0x70, 0xa4, 0xeb, 0x97, 0xbb, 0xfb, 0x14, 0x0f, 0xf5, 0xb0, 0xc6, 0x0f, + 0x71, 0x56, 0xfe, 0x26, 0x26, 0xaa, 0xfb, 0x0e, 0xe4, 0x83, 0xde, 0x0b, 0xae, 0xbf, 0x6e, 0xa7, + 0x43, 0x9b, 0xc5, 0x40, 0x8f, 0x01, 0xc4, 0xab, 0x28, 0xa2, 0xc4, 0x10, 0x1f, 0x2c, 0x36, 0x66, + 0x8a, 0x1a, 0x01, 0xe1, 0xf7, 0x96, 0x2b, 0xff, 0x99, 0x80, 0x44, 0xdd, 0xa5, 0xe9, 0xc6, 0x23, + 0x90, 0x47, 0xc4, 0x0c, 0x4e, 0xde, 0xcb, 0x82, 0x33, 0xdd, 0xb5, 0x6d, 0x62, 0x86, 0x5b, 0x95, + 0x82, 0xa0, 0xc7, 0x90, 0xec, 0x93, 0x89, 0x63, 0x7a, 0x22, 0xe3, 0xfc, 0x68, 0x21, 0xb8, 0x32, + 0x53, 0x0d, 0x02, 0x07, 0x07, 0x5a, 0xf9, 0xdf, 0x18, 0x24, 0x18, 0x03, 0x7d, 0x05, 0x19, 0x46, + 0xeb, 0x4e, 0x13, 0x85, 0x4f, 0x16, 0xc7, 0x8f, 0xb8, 0xdf, 0x14, 0x8e, 0x06, 0x42, 0xcb, 0xf1, + 0x75, 0xf1, 0xed, 0x55, 0x34, 0x30, 0x64, 0x2c, 0xc7, 0x6f, 0xf3, 0xcf, 0xaf, 0x6e, 0x41, 0x8e, + 0x06, 0x24, 0x33, 0x10, 0x8b, 0xb3, 0x14, 0x2d, 0xcb, 0x68, 0x42, 0x64, 0x0b, 0xb2, 0x9c, 0xc9, + 0xd3, 0x19, 0xbe, 0xf5, 0x17, 0xb8, 0x64, 0x05, 0xae, 0x4d, 0xc7, 0xb4, 0xf2, 0x57, 0x31, 0x48, + 0x72, 0x93, 0xa0, 0x16, 0x24, 0x3c, 0xdf, 0x70, 0x7d, 0x11, 0xf9, 0x1e, 0x2e, 0x3e, 0xed, 0x30, + 0x22, 0x50, 0x18, 0x54, 0x9d, 0xe6, 0x98, 0xaf, 0x85, 0xc6, 0xf2, 0x52, 0xf5, 0x0e, 0xc8, 0xd4, + 0x03, 0x68, 0x65, 0xae, 0x95, 0x5a, 0x1b, 0x35, 0xe5, 0x0a, 0x4a, 0x83, 0xcc, 0xba, 0x47, 0x31, + 0x5a, 0xba, 0x6f, 0x68, 0xed, 0xde, 0x4e, 0x47, 0x91, 0xd4, 0x17, 0x90, 0x09, 0x6d, 0x8f, 0xae, + 0xc3, 0x5b, 0xbd, 0x56, 0xb9, 0xdd, 0x6b, 0x55, 0x6b, 0x55, 0x7d, 0x47, 0xab, 0x55, 0x6a, 0xd5, + 0x46, 0x6b, 0x43, 0xb9, 0x32, 0xcb, 0xa8, 0xb7, 0x9b, 0xcd, 0xf6, 0x13, 0xca, 0x88, 0xa1, 0x65, + 0x50, 0xda, 0xf5, 0x7a, 0xa7, 0xd6, 0x8d, 0x88, 0x4b, 0x11, 0xea, 0x54, 0x36, 0x8e, 0x96, 0x20, + 0x5b, 0xe9, 0x69, 0x5a, 0x8d, 0xb7, 0xb1, 0x14, 0x79, 0xe5, 0x9f, 0x24, 0x48, 0x07, 0xdb, 0x17, + 0xd5, 0x22, 0x5d, 0xc8, 0x0b, 0x3f, 0x58, 0x98, 0x9d, 0xf8, 0xc9, 0x1e, 0xe4, 0xfb, 0x90, 0x35, + 0xdc, 0x61, 0xc3, 0x3c, 0xea, 0xb0, 0x45, 0x89, 0xba, 0x4b, 0x94, 0x81, 0x6e, 0x42, 0xda, 0x70, + 0x87, 0x15, 0x32, 0x11, 0x87, 0x6b, 0x78, 0xd8, 0x04, 0xd4, 0x6f, 0xe9, 0x9c, 0x28, 0x43, 0x62, + 0xcf, 0x0d, 0x52, 0xcd, 0x0b, 0x3f, 0x6c, 0x38, 0xbd, 0xa0, 0x1a, 0x57, 0x45, 0x77, 0x61, 0xa6, + 0x3f, 0x2a, 0xbe, 0xd4, 0x14, 0x8d, 0x83, 0x28, 0x47, 0xfd, 0x45, 0x0c, 0x60, 0x1a, 0x5c, 0x50, + 0x01, 0x40, 0x6b, 0x3f, 0xd1, 0x5b, 0xbd, 0xed, 0x72, 0x4d, 0x13, 0x2e, 0x50, 0x6a, 0x3d, 0xe2, + 0x3d, 0xbb, 0x6a, 0xad, 0xd5, 0xa9, 0xe9, 0xec, 0x59, 0x42, 0x0a, 0xe4, 0x76, 0x6a, 0x5a, 0x85, + 0xad, 0x0d, 0xa5, 0xc4, 0x51, 0x1e, 0x32, 0x95, 0xde, 0x76, 0x4d, 0xaf, 0x36, 0x3a, 0x5d, 0xde, + 0xdb, 0x6c, 0x75, 0x1b, 0xcd, 0x1a, 0xef, 0x6d, 0x36, 0x4b, 0x1b, 0x4a, 0x92, 0xc2, 0x35, 0x6b, + 0xa5, 0xaa, 0x92, 0xa2, 0x4b, 0x5b, 0x6f, 0x68, 0x9d, 0xae, 0xbe, 0x5b, 0x6a, 0xf6, 0x6a, 0x4a, + 0x9a, 0xe2, 0x37, 0x4b, 0xe1, 0x73, 0x86, 0xa2, 0xb5, 0xba, 0x9b, 0xe2, 0x11, 0xd4, 0x5f, 0x4a, + 0xb0, 0x7c, 0xd6, 0x27, 0x22, 0xa8, 0x0b, 0xa9, 0x43, 0x23, 0x9a, 0xe2, 0x7e, 0xbc, 0xd8, 0x37, + 0x26, 0x6b, 0x4f, 0xa8, 0x76, 0x90, 0x4b, 0x09, 0x28, 0x54, 0x07, 0x79, 0x0f, 0x63, 0xf3, 0x9c, + 0xaf, 0x7d, 0xd9, 0xb7, 0xd6, 0xfc, 0xbb, 0x6b, 0x81, 0x49, 0x85, 0xab, 0xd8, 0x37, 0x2c, 0xdb, + 0x0b, 0x9d, 0x0b, 0x63, 0x73, 0xe5, 0xcf, 0x62, 0x90, 0x60, 0x2f, 0x40, 0x2d, 0x50, 0x2c, 0xc7, + 0xf2, 0x2d, 0xc3, 0xd6, 0x5d, 0xec, 0x11, 0xfb, 0x39, 0x36, 0x85, 0xe7, 0x5e, 0xaa, 0xb7, 0xb6, + 0x24, 0x94, 0x35, 0xa1, 0x1b, 0x7e, 0x67, 0x21, 0x5d, 0xfe, 0x3b, 0x8b, 0x7f, 0x88, 0x01, 0x3a, + 0xfd, 0x31, 0x0c, 0x2a, 0x43, 0xde, 0x77, 0x8d, 0xc1, 0x01, 0x36, 0x75, 0x9e, 0xa0, 0x70, 0x3b, + 0xce, 0x81, 0xcc, 0x09, 0x9d, 0x0e, 0x2b, 0xab, 0xbf, 0x25, 0x7b, 0x45, 0xfa, 0x62, 0xf1, 0xf3, + 0xfb, 0x62, 0xf7, 0x7f, 0x1f, 0x0a, 0xb3, 0xf7, 0xe1, 0x34, 0x40, 0xed, 0xf4, 0xca, 0xcd, 0x46, + 0x45, 0xb9, 0x82, 0xde, 0x81, 0x6b, 0xfc, 0xb7, 0x5e, 0x6a, 0x55, 0x59, 0x87, 0x5e, 0xb0, 0x62, + 0xf7, 0xd5, 0xa0, 0x5c, 0x65, 0xc1, 0x6b, 0x19, 0x94, 0xcd, 0x66, 0x53, 0xdf, 0x69, 0xf6, 0x3a, + 0xfc, 0xcf, 0xee, 0x03, 0xe5, 0x4a, 0xf9, 0xf6, 0xd7, 0xff, 0x71, 0xe3, 0xca, 0xd7, 0xaf, 0x6e, + 0xc4, 0xfe, 0xf9, 0xd5, 0x8d, 0xd8, 0xaf, 0x5e, 0xdd, 0x88, 0xfd, 0xfb, 0xab, 0x1b, 0xb1, 0xbf, + 0xf8, 0xf5, 0x8d, 0x2b, 0x5f, 0xc1, 0xd4, 0x95, 0x7e, 0x13, 0x00, 0x00, 0xff, 0xff, 0x0a, 0x3f, + 0x1a, 0xa3, 0xb7, 0x2f, 0x00, 0x00, } diff --git a/pkg/sql/distsqlrun/processors.proto b/pkg/sql/distsqlrun/processors.proto index 808d842bb84b..b1d7c439186a 100644 --- a/pkg/sql/distsqlrun/processors.proto +++ b/pkg/sql/distsqlrun/processors.proto @@ -20,6 +20,7 @@ syntax = "proto2"; package cockroach.sql.distsqlrun; option go_package = "distsqlrun"; +import "jobs/jobspb/jobs.proto"; import "roachpb/api.proto"; import "roachpb/data.proto"; import "roachpb/errors.proto"; @@ -139,6 +140,8 @@ message ProcessorCoreUnion { optional ProjectSetSpec projectSet = 22; optional WindowerSpec windower = 23; optional LocalPlanNodeSpec localPlanNode = 24; + optional ChangeAggregatorSpec changeAggregator = 25; + optional ChangeFrontierSpec changeFrontier = 26; reserved 6, 12; } @@ -958,3 +961,35 @@ message WindowerSpec { // WindowFns is the specification of all window functions to be computed. repeated WindowFn windowFns = 2 [(gogoproto.nullable) = false]; } + +// ChangeAggregatorSpec is the specification for a processor that watches for +// changes in a set of spans. Each span may cross multiple ranges. +message ChangeAggregatorSpec { + message Watch { + optional util.hlc.Timestamp initial_resolved = 1 [(gogoproto.nullable) = false]; + optional roachpb.Span span = 2 [(gogoproto.nullable) = false]; + } + repeated Watch watches = 1 [(gogoproto.nullable) = false]; + + // Feed is the specification for this changefeed. + optional cockroach.sql.jobs.jobspb.ChangefeedDetails feed = 2 [(gogoproto.nullable) = false]; +} + +// ChangeFrontierSpec is the specification for a processor that receives +// span-level resolved timestamps, track them, and emits the changefeed-level +// resolved timestamp whenever it changes. +message ChangeFrontierSpec { + // TrackedSpans is the entire span set being watched. Once all these spans + // have been resolved at a certain timestamp, then it's safe to resolve the + // changefeed at that timestamp. + repeated roachpb.Span tracked_spans = 1 [(gogoproto.nullable) = false]; + + // Feed is the specification for this changefeed. + optional cockroach.sql.jobs.jobspb.ChangefeedDetails feed = 2 [(gogoproto.nullable) = false]; + + // JobID is the id of this changefeed in the system jobs. + optional int64 job_id = 3 [ + (gogoproto.nullable) = false, + (gogoproto.customname) = "JobID" + ]; +} diff --git a/pkg/sql/distsqlrun/server.go b/pkg/sql/distsqlrun/server.go index 021eb2ace1ec..3e679c08d512 100644 --- a/pkg/sql/distsqlrun/server.go +++ b/pkg/sql/distsqlrun/server.go @@ -152,6 +152,10 @@ type ServerConfig struct { // JobRegistry manages jobs being used by this Server. JobRegistry *jobs.Registry + // LeaseManager is a *sql.LeaseManager. It's stored as an `interface{}` due + // to package dependency cycles + LeaseManager interface{} + // A handle to gossip used to broadcast the node's DistSQL version and // draining state. Gossip *gossip.Gossip @@ -415,6 +419,7 @@ func (ds *ServerImpl) setupFlow( txn: txn, ClientDB: ds.DB, executor: ds.Executor, + LeaseManager: ds.ServerConfig.LeaseManager, testingKnobs: ds.TestingKnobs, nodeID: nodeID, TempStorage: ds.TempStorage, diff --git a/pkg/sql/explain_distsql.go b/pkg/sql/explain_distsql.go index 5eaced741fc5..083ad09f7745 100644 --- a/pkg/sql/explain_distsql.go +++ b/pkg/sql/explain_distsql.go @@ -111,7 +111,8 @@ func (n *explainDistSQLNode) startExec(params runParams) error { }, params.extendedEvalCtx.Tracing, ) - distSQLPlanner.Run(&planCtx, params.p.txn, &plan, recv, params.extendedEvalCtx) + distSQLPlanner.Run( + &planCtx, params.p.txn, &plan, recv, params.extendedEvalCtx, nil /* finishedSetupFn */) spans = params.extendedEvalCtx.Tracing.getRecording() if err := params.extendedEvalCtx.Tracing.StopTracing(); err != nil { diff --git a/pkg/sql/scrub.go b/pkg/sql/scrub.go index 99d7c3b3f668..957cf050518a 100644 --- a/pkg/sql/scrub.go +++ b/pkg/sql/scrub.go @@ -562,7 +562,8 @@ func scrubRunDistSQL( p.extendedEvalCtx.Tracing, ) - p.extendedEvalCtx.DistSQLPlanner.Run(planCtx, p.txn, plan, recv, &p.extendedEvalCtx) + p.extendedEvalCtx.DistSQLPlanner.Run( + planCtx, p.txn, plan, recv, &p.extendedEvalCtx, nil /* finishedSetupFn */) if rowResultWriter.Err() != nil { return rows, rowResultWriter.Err() } else if rows.Len() == 0 {