From e5926f976e36b3b9b04c91186c496130eb42ecf8 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Fri, 19 Feb 2021 19:14:26 -0500 Subject: [PATCH 1/5] kv/kvclient: fix ManualRefresh error handling Fixes #60760. Fallout from #60567. The refreshed BatchRequest was nil on the error path, which was resulting in a nil-pointer exception. This commit fixes this by passing the original BatchRequest to updateStateLocked, like the TxnCoordSender normally does. --- pkg/kv/kvclient/kvcoord/txn_coord_sender.go | 10 ++- .../kvclient/kvcoord/txn_coord_sender_test.go | 77 +++++++++++++++---- 2 files changed, 69 insertions(+), 18 deletions(-) diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go index b69e7e59ae4c..1deaa7727f6f 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go @@ -1158,16 +1158,20 @@ func (tc *TxnCoordSender) ManualRefresh(ctx context.Context) error { // Hijack the pre-emptive refresh code path to perform the refresh but // provide the force flag to ensure that the refresh occurs unconditionally. + // We provide an empty BatchRequest - maybeRefreshPreemptivelyLocked just + // needs the transaction proto. The function then returns a BatchRequest + // with the updated transaction proto. We use this updated proto to call + // into updateStateLocked directly. var ba roachpb.BatchRequest ba.Txn = tc.mu.txn.Clone() const force = true - ba, pErr := tc.interceptorAlloc.txnSpanRefresher.maybeRefreshPreemptivelyLocked(ctx, ba, force) + refreshedBa, pErr := tc.interceptorAlloc.txnSpanRefresher.maybeRefreshPreemptivelyLocked(ctx, ba, force) if pErr != nil { pErr = tc.updateStateLocked(ctx, ba, nil, pErr) } else { var br roachpb.BatchResponse - br.Txn = ba.Txn - pErr = tc.updateStateLocked(ctx, ba, &br, pErr) + br.Txn = refreshedBa.Txn + pErr = tc.updateStateLocked(ctx, ba, &br, nil) } return pErr.GoError() } diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go index 5b8a70ccfdb7..a5492b1914b5 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go @@ -2460,10 +2460,9 @@ func TestTxnManualRefresh(t *testing.T) { r := <-reqCh _, ok := r.ba.GetArg(roachpb.Get) require.True(t, ok) - var br roachpb.BatchResponse + br := r.ba.CreateReply() br.Txn = r.ba.Txn - br.Add(&roachpb.GetResponse{}) - r.respCh <- resp{br: &br} + r.respCh <- resp{br: br} } require.NoError(t, <-errCh) @@ -2474,7 +2473,7 @@ func TestTxnManualRefresh(t *testing.T) { }, }, { - name: "refresh occurs due to read", + name: "refresh occurs successfully due to read", run: func( ctx context.Context, t *testing.T, db *kv.DB, clock *hlc.ManualClock, reqCh <-chan req, @@ -2489,10 +2488,9 @@ func TestTxnManualRefresh(t *testing.T) { r := <-reqCh _, ok := r.ba.GetArg(roachpb.Get) require.True(t, ok) - var br roachpb.BatchResponse + br := r.ba.CreateReply() br.Txn = r.ba.Txn - br.Add(&roachpb.GetResponse{}) - r.respCh <- resp{br: &br} + r.respCh <- resp{br: br} } require.NoError(t, <-errCh) @@ -2503,14 +2501,12 @@ func TestTxnManualRefresh(t *testing.T) { r := <-reqCh _, ok := r.ba.GetArg(roachpb.Put) require.True(t, ok) - var br roachpb.BatchResponse + br := r.ba.CreateReply() br.Txn = r.ba.Txn.Clone() // Push the WriteTimestamp simulating an interaction with the // timestamp cache. - br.Txn.WriteTimestamp = - br.Txn.WriteTimestamp.Add(time.Millisecond.Nanoseconds(), 0) - br.Add(&roachpb.PutResponse{}) - r.respCh <- resp{br: &br} + br.Txn.WriteTimestamp = db.Clock().Now() + r.respCh <- resp{br: br} } require.NoError(t, <-errCh) @@ -2521,10 +2517,9 @@ func TestTxnManualRefresh(t *testing.T) { r := <-reqCh _, ok := r.ba.GetArg(roachpb.Refresh) require.True(t, ok) - var br roachpb.BatchResponse + br := r.ba.CreateReply() br.Txn = r.ba.Txn.Clone() - br.Add(&roachpb.RefreshResponse{}) - r.respCh <- resp{br: &br} + r.respCh <- resp{br: br} } require.NoError(t, <-errCh) @@ -2533,6 +2528,58 @@ func TestTxnManualRefresh(t *testing.T) { require.NoError(t, txn.ManualRefresh(ctx)) }, }, + { + name: "refresh occurs unsuccessfully due to read", + run: func( + ctx context.Context, t *testing.T, db *kv.DB, + clock *hlc.ManualClock, reqCh <-chan req, + ) { + txn := db.NewTxn(ctx, "test") + errCh := make(chan error) + go func() { + _, err := txn.Get(ctx, "foo") + errCh <- err + }() + { + r := <-reqCh + _, ok := r.ba.GetArg(roachpb.Get) + require.True(t, ok) + br := r.ba.CreateReply() + br.Txn = r.ba.Txn + r.respCh <- resp{br: br} + } + require.NoError(t, <-errCh) + + go func() { + errCh <- txn.Put(ctx, "bar", "baz") + }() + { + r := <-reqCh + _, ok := r.ba.GetArg(roachpb.Put) + require.True(t, ok) + br := r.ba.CreateReply() + br.Txn = r.ba.Txn.Clone() + // Push the WriteTimestamp simulating an interaction with the + // timestamp cache. + br.Txn.WriteTimestamp = db.Clock().Now() + r.respCh <- resp{br: br} + } + require.NoError(t, <-errCh) + + go func() { + errCh <- txn.ManualRefresh(ctx) + }() + { + r := <-reqCh + _, ok := r.ba.GetArg(roachpb.Refresh) + require.True(t, ok) + // Rejects the refresh due to a conflicting write. + pErr := roachpb.NewErrorf("encountered recently written key") + r.respCh <- resp{pErr: pErr} + } + require.Regexp(t, `TransactionRetryError: retry txn \(RETRY_SERIALIZABLE - failed preemptive refresh\)`, <-errCh) + }, + }, } run := func(t *testing.T, tc testCase) { stopper := stop.NewStopper() From 1e96a6d17022246fc5b829b8a9759384b97b1aed Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 21 Feb 2021 14:53:07 -0500 Subject: [PATCH 2/5] changefeedccl: move debugging log events to vmodule These were noisy and not very informative. Release note: None --- pkg/ccl/changefeedccl/schemafeed/schema_feed.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go index b1ef0d77e5ea..9db60623f9e0 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go +++ b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go @@ -491,7 +491,7 @@ func (tf *SchemaFeed) validateDescriptor( if err := changefeedbase.ValidateTable(tf.targets, desc); err != nil { return err } - log.Infof(ctx, "validate %v", formatDesc(desc)) + log.VEventf(ctx, 1, "validate %v", formatDesc(desc)) if lastVersion, ok := tf.mu.previousTableVersion[desc.GetID()]; ok { // NB: Writes can occur to a table if desc.GetModificationTime().LessEq(lastVersion.GetModificationTime()) { @@ -517,7 +517,7 @@ func (tf *SchemaFeed) validateDescriptor( After: desc, } shouldFilter, err := tf.filter.shouldFilter(ctx, e) - log.Infof(ctx, "validate shouldFilter %v %v", formatEvent(e), shouldFilter) + log.VEventf(ctx, 1, "validate shouldFilter %v %v", formatEvent(e), shouldFilter) if err != nil { return err } From 29437d43a25615056399082cc6b241c7352f5c8d Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 21 Feb 2021 14:51:53 -0500 Subject: [PATCH 3/5] changefeedtest: deal with serializability violations These seem to pop up. They were not handled properly. Release note: None --- pkg/ccl/changefeedccl/cdctest/BUILD.bazel | 1 + pkg/ccl/changefeedccl/cdctest/testfeed.go | 112 +++++++++++----------- 2 files changed, 58 insertions(+), 55 deletions(-) diff --git a/pkg/ccl/changefeedccl/cdctest/BUILD.bazel b/pkg/ccl/changefeedccl/cdctest/BUILD.bazel index 84e1deda4cdc..4ea859f0f620 100644 --- a/pkg/ccl/changefeedccl/cdctest/BUILD.bazel +++ b/pkg/ccl/changefeedccl/cdctest/BUILD.bazel @@ -25,6 +25,7 @@ go_library( "//pkg/util/randutil", "//pkg/util/retry", "//pkg/util/timeutil", + "@com_github_cockroachdb_cockroach_go//crdb", "@com_github_cockroachdb_errors//:errors", "@com_github_jackc_pgx//:pgx", ], diff --git a/pkg/ccl/changefeedccl/cdctest/testfeed.go b/pkg/ccl/changefeedccl/cdctest/testfeed.go index 960aab76e020..04bbdc1beb5e 100644 --- a/pkg/ccl/changefeedccl/cdctest/testfeed.go +++ b/pkg/ccl/changefeedccl/cdctest/testfeed.go @@ -24,6 +24,7 @@ import ( "strings" "time" + "github.com/cockroachdb/cockroach-go/crdb" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -375,8 +376,8 @@ type TableFeed struct { jobFeed sinkURI string - rows *gosql.Rows - seen map[string]struct{} + toSend []*TestFeedMessage + seen map[string]struct{} } // ResetSeen is useful when manually pausing and resuming a TableFeed. @@ -403,71 +404,72 @@ func (c *TableFeed) Next() (*TestFeedMessage, error) { // by repeatedly fetching and deleting all rows in the table. Then it pages // through the results until they are empty and repeats. for { - if c.rows != nil && c.rows.Next() { - m := &TestFeedMessage{} - var msgID int64 - if err := c.rows.Scan( - &m.Topic, &m.Partition, &msgID, &m.Key, &m.Value, &m.Resolved, - ); err != nil { - return nil, err - } - - // Scan turns NULL bytes columns into a 0-length, non-nil byte - // array, which is pretty unexpected. Nil them out before returning. - // Either key+value or payload will be set, but not both. - if len(m.Key) > 0 || len(m.Value) > 0 { - // TODO(dan): This skips duplicates, since they're allowed by the - // semantics of our changefeeds. Now that we're switching to RangeFeed, - // this can actually happen (usually because of splits) and cause - // flakes. However, we really should be de-duping key+ts, this is too - // coarse. Fixme. - seenKey := m.Topic + m.Partition + string(m.Key) + string(m.Value) - if _, ok := c.seen[seenKey]; ok { - continue - } - c.seen[seenKey] = struct{}{} - - m.Resolved = nil - return m, nil - } - m.Key, m.Value = nil, nil - return m, nil - } - if c.rows != nil { - if err := c.rows.Close(); err != nil { - return nil, err - } - c.rows = nil + if len(c.toSend) > 0 { + toSend := c.toSend[0] + c.toSend = c.toSend[1:] + return toSend, nil } - if err := c.fetchJobError(); err != nil { return nil, c.jobErr } + var toSend []*TestFeedMessage + if err := crdb.ExecuteTx(context.Background(), c.db, nil, func(tx *gosql.Tx) error { + + toSend = nil // reset for this iteration + // TODO(dan): It's a bummer that this mutates the sqlsink table. I + // originally tried paging through message_id by repeatedly generating a + // new high-water with GenerateUniqueInt, but this was racy with rows + // being flushed out by the sink. An alternative is to steal the nanos + // part from `high_water_timestamp` in `crdb_internal.jobs` and run it + // through `builtins.GenerateUniqueID`, but that would mean we're only + // ever running tests on rows that have gotten a resolved timestamp, + // which seems limiting. + rows, err := tx.Query( + `SELECT * FROM [DELETE FROM sqlsink RETURNING *] ORDER BY topic, partition, message_id`) + if err != nil { + return err + } + for rows.Next() { + + m := &TestFeedMessage{} + var msgID int64 + if err := rows.Scan( + &m.Topic, &m.Partition, &msgID, &m.Key, &m.Value, &m.Resolved, + ); err != nil { + return err + } - // TODO(dan): It's a bummer that this mutates the sqlsink table. I - // originally tried paging through message_id by repeatedly generating a - // new high-water with GenerateUniqueInt, but this was racy with rows - // being flushed out by the sink. An alternative is to steal the nanos - // part from `high_water_timestamp` in `crdb_internal.jobs` and run it - // through `builtins.GenerateUniqueID`, but that would mean we're only - // ever running tests on rows that have gotten a resolved timestamp, - // which seems limiting. - var err error - c.rows, err = c.db.Query( - `SELECT * FROM [DELETE FROM sqlsink RETURNING *] ORDER BY topic, partition, message_id`) - if err != nil { + // Scan turns NULL bytes columns into a 0-length, non-nil byte + // array, which is pretty unexpected. Nil them out before returning. + // Either key+value or payload will be set, but not both. + if len(m.Key) > 0 || len(m.Value) > 0 { + // TODO(dan): This skips duplicates, since they're allowed by the + // semantics of our changefeeds. Now that we're switching to RangeFeed, + // this can actually happen (usually because of splits) and cause + // flakes. However, we really should be de-duping key+ts, this is too + // coarse. Fixme. + seenKey := m.Topic + m.Partition + string(m.Key) + string(m.Value) + if _, ok := c.seen[seenKey]; ok { + continue + } + c.seen[seenKey] = struct{}{} + + m.Resolved = nil + } else { + m.Key, m.Value = nil, nil + } + toSend = append(toSend, m) + } + return rows.Err() + }); err != nil { return nil, err } + c.toSend = toSend } } // Close implements the TestFeed interface. func (c *TableFeed) Close() error { - if c.rows != nil { - if err := c.rows.Close(); err != nil { - return errors.Errorf(`could not close rows: %v`, err) - } - } if _, err := c.db.Exec(`CANCEL JOB $1`, c.JobID); err != nil { log.Infof(context.Background(), `could not cancel feed %d: %v`, c.JobID, err) } From 3504342a87deee9b117af32d97650c4c17e84a2c Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Mon, 4 Jan 2021 08:28:38 -0500 Subject: [PATCH 4/5] changefeedccl: support primary key changes This PR does a few things. One is that when errors occur due to unsupported schema changes during the execution of a changefeed, relatively poor handling ensues. Ideally we'd allow the changefeed to run its course right up to that unsupported schema change's timestamp and then ensure that we persist the fact that we've processed all of that data. That would permit a user to then restart a changefeed after the unsupported change. There are some edge cases here worth considering related to off-by-ones in the timestamp management. I probably should go through that exercise before merging this PR. The real feature this work is in support of is to allow for changefeeds to successfully navigate changes to a primary index. This PR works and support changes to the primary key of a table that also include column set changes. Release note (enterprise change): Support primary key changes in `CHANGEFEED`. --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/ccl/changefeedccl/BUILD.bazel | 2 + pkg/ccl/changefeedccl/cdctest/testfeed.go | 6 + pkg/ccl/changefeedccl/changefeed_dist.go | 37 +- .../changefeedccl/changefeed_processors.go | 71 +- pkg/ccl/changefeedccl/changefeed_test.go | 225 ++++- pkg/ccl/changefeedccl/helpers_test.go | 13 +- pkg/ccl/changefeedccl/kvfeed/buffer.go | 19 +- pkg/ccl/changefeedccl/kvfeed/kv_feed.go | 42 +- .../changefeedccl/kvfeed/physical_kv_feed.go | 3 +- pkg/ccl/changefeedccl/kvfeed/scanner.go | 3 +- .../schemafeed/table_event_filter.go | 11 +- pkg/clusterversion/cockroach_versions.go | 8 + pkg/clusterversion/key_string.go | 5 +- pkg/jobs/jobspb/jobs.pb.go | 811 ++++++++++-------- pkg/jobs/jobspb/jobs.proto | 29 +- 17 files changed, 867 insertions(+), 422 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 5426912c80f2..d1b05474e0c6 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -99,4 +99,4 @@ timeseries.storage.resolution_30m.ttl duration 2160h0m0s the maximum age of time trace.debug.enable boolean false if set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.token string if set, traces go to Lightstep using this token trace.zipkin.collector string if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -version version 20.2-36 set the active cluster version in the format '.' +version version 20.2-38 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 2bb3ab92b145..8266c1ba9a0a 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -101,6 +101,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versionversion20.2-36set the active cluster version in the format '.' +versionversion20.2-38set the active cluster version in the format '.' diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index edda27e4b287..02f04b0b17fd 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//pkg/ccl/changefeedccl/changefeeddist", "//pkg/ccl/changefeedccl/kvfeed", "//pkg/ccl/utilccl", + "//pkg/clusterversion", "//pkg/docs", "//pkg/featureflag", "//pkg/geo", @@ -124,6 +125,7 @@ go_test( "//pkg/ccl/importccl", "//pkg/ccl/storageccl", "//pkg/ccl/utilccl", + "//pkg/clusterversion", "//pkg/gossip", "//pkg/jobs", "//pkg/jobs/jobspb", diff --git a/pkg/ccl/changefeedccl/cdctest/testfeed.go b/pkg/ccl/changefeedccl/cdctest/testfeed.go index 04bbdc1beb5e..70fe821e0ced 100644 --- a/pkg/ccl/changefeedccl/cdctest/testfeed.go +++ b/pkg/ccl/changefeedccl/cdctest/testfeed.go @@ -415,6 +415,12 @@ func (c *TableFeed) Next() (*TestFeedMessage, error) { var toSend []*TestFeedMessage if err := crdb.ExecuteTx(context.Background(), c.db, nil, func(tx *gosql.Tx) error { + // Avoid anything that might somehow look like deadlock under stressrace. + _, err := tx.Exec("SET TRANSACTION PRIORITY LOW") + if err != nil { + return err + } + toSend = nil // reset for this iteration // TODO(dan): It's a bummer that this mutates the sqlsink table. I // originally tried paging through message_id by repeatedly generating a diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go index d433362f80ef..fe6836c01adc 100644 --- a/pkg/ccl/changefeedccl/changefeed_dist.go +++ b/pkg/ccl/changefeedccl/changefeed_dist.go @@ -81,19 +81,34 @@ func distChangefeedFlow( } } - spansTS := details.StatementTime + execCfg := execCtx.ExecCfg() var initialHighWater hlc.Timestamp - if h := progress.GetHighWater(); h != nil && !h.IsEmpty() { - initialHighWater = *h - // If we have a high-water set, use it to compute the spans, since the - // ones at the statement time may have been garbage collected by now. - spansTS = initialHighWater - } + var trackedSpans []roachpb.Span + { + spansTS := details.StatementTime + if h := progress.GetHighWater(); h != nil && !h.IsEmpty() { + initialHighWater = *h + // If we have a high-water set, use it to compute the spans, since the + // ones at the statement time may have been garbage collected by now. + spansTS = initialHighWater + } - execCfg := execCtx.ExecCfg() - trackedSpans, err := fetchSpansForTargets(ctx, execCfg.DB, execCfg.Codec, details.Targets, spansTS) - if err != nil { - return err + // We want to fetch the target spans as of the timestamp following the + // highwater unless the highwater corresponds to a timestamp of an initial + // scan. This logic is irritatingly complex but extremely important. Namely, + // we may be here because the schema changed at the current resolved + // timestamp. However, an initial scan should be performed at exactly the + // timestamp specified; initial scans can be created at the timestamp of a + // schema change and thus should see the side-effect of the schema change. + isRestartAfterCheckpointOrNoInitialScan := progress.GetHighWater() != nil + if isRestartAfterCheckpointOrNoInitialScan { + spansTS = spansTS.Next() + } + var err error + trackedSpans, err = fetchSpansForTargets(ctx, execCfg.DB, execCfg.Codec, details.Targets, spansTS) + if err != nil { + return err + } } return changefeeddist.StartDistChangefeed( diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index 53ce638e6614..204aa9871bdd 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeeddist" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvfeed" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" @@ -452,7 +453,7 @@ func (ca *changeAggregator) maybeFlush(resolvedSpan *jobspb.ResolvedSpan) error ca.spansToFlush = append(ca.spansToFlush, resolvedSpan) } - boundaryReached := resolvedSpan != nil && resolvedSpan.BoundaryReached + boundaryReached := resolvedSpan != nil && resolvedSpan.DeprecatedBoundaryReached if len(ca.spansToFlush) == 0 || (timeutil.Since(ca.lastFlush) < ca.flushFrequency && !boundaryReached) { return nil @@ -821,6 +822,10 @@ type changeFrontier struct { // by the KV feed based on OptSchemaChangeEvents and OptSchemaChangePolicy. schemaChangeBoundary hlc.Timestamp + // boundaryType indicates the type of the schemaChangeBoundary and thus the + // action which should be taken when the frontier reaches that boundary. + boundaryType jobspb.ResolvedSpan_BoundaryType + // 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 @@ -1030,11 +1035,31 @@ func (cf *changeFrontier) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetad return cf.ProcessRowHelper(cf.resolvedBuf.Pop()), nil } - if cf.schemaChangeBoundaryReached() && cf.shouldFailOnSchemaChange() { + if cf.schemaChangeBoundaryReached() && + (cf.boundaryType == jobspb.ResolvedSpan_EXIT || + cf.boundaryType == jobspb.ResolvedSpan_RESTART) { + err := pgerror.Newf(pgcode.SchemaChangeOccurred, + "schema change occurred at %v", cf.schemaChangeBoundary.Next().AsOfSystemTime()) + + // Detect whether this boundary should be used to kill or restart the + // changefeed. + if cf.boundaryType == jobspb.ResolvedSpan_RESTART { + // The code to restart the changefeed is only supported once 21.1 is + // activated. + // + // TODO(ajwerner): Remove this gate in 21.2. + if cf.EvalCtx.Settings.Version.IsActive( + cf.Ctx, clusterversion.ChangefeedsSupportPrimaryIndexChanges, + ) { + err = MarkRetryableError(err) + } else { + err = errors.Wrap(err, "primary key change occurred") + } + + } // TODO(ajwerner): make this more useful by at least informing the client // of which tables changed. - cf.MoveToDraining(pgerror.Newf(pgcode.SchemaChangeOccurred, - "schema change occurred at %v", cf.schemaChangeBoundary.Next().AsOfSystemTime())) + cf.MoveToDraining(err) break } @@ -1081,6 +1106,20 @@ func (cf *changeFrontier) noteResolvedSpan(d rowenc.EncDatum) error { `unmarshalling resolved span: %x`, raw) } + // Change aggregators running on v20.2 nodes will not know about the new + // BoundaryType field added in v21.1 and thus will only have the boolean + // populated. This code translates the boolean into the BoundaryType for the + // types that are possible in the mixed version state. + // + // TODO(ajwerner): Remove this code in 21.2. + if resolved.DeprecatedBoundaryReached && resolved.BoundaryType == jobspb.ResolvedSpan_NONE { + if cf.shouldFailOnSchemaChange() { + resolved.BoundaryType = jobspb.ResolvedSpan_EXIT + } else { + resolved.BoundaryType = jobspb.ResolvedSpan_BACKFILL + } + } + // Inserting a timestamp less than the one the changefeed flow started at // could potentially regress the job progress. This is not expected, but it // was a bug at one point, so assert to prevent regressions. @@ -1095,14 +1134,24 @@ func (cf *changeFrontier) noteResolvedSpan(d rowenc.EncDatum) error { return nil } - // We want to ensure that we mark the schemaChangeBoundary and then we want to detect when - // the frontier reaches to or past the schemaChangeBoundary. - if resolved.BoundaryReached && (cf.schemaChangeBoundary.IsEmpty() || resolved.Timestamp.Less(cf.schemaChangeBoundary)) { + // We want to ensure that we mark the schemaChangeBoundary and then we want + // to detect when the frontier reaches to or past the schemaChangeBoundary. + // The behavior when the boundary is reached is controlled by the + // boundaryType. + switch resolved.BoundaryType { + case jobspb.ResolvedSpan_NONE: + if !cf.schemaChangeBoundary.IsEmpty() && cf.schemaChangeBoundary.Less(resolved.Timestamp) { + cf.schemaChangeBoundary = hlc.Timestamp{} + cf.boundaryType = jobspb.ResolvedSpan_NONE + } + case jobspb.ResolvedSpan_BACKFILL, jobspb.ResolvedSpan_EXIT, jobspb.ResolvedSpan_RESTART: + if !cf.schemaChangeBoundary.IsEmpty() && resolved.Timestamp.Less(cf.schemaChangeBoundary) { + return errors.AssertionFailedf("received boundary timestamp %v < %v "+ + "of type %v before reaching existing boundary of type %v", + resolved.Timestamp, cf.schemaChangeBoundary, resolved.BoundaryType, cf.boundaryType) + } cf.schemaChangeBoundary = resolved.Timestamp - } - // If we've moved past a schemaChangeBoundary, make sure to clear it. - if !resolved.BoundaryReached && !cf.schemaChangeBoundary.IsEmpty() && cf.schemaChangeBoundary.Less(resolved.Timestamp) { - cf.schemaChangeBoundary = hlc.Timestamp{} + cf.boundaryType = resolved.BoundaryType } frontierChanged := cf.sf.Forward(resolved.Span, resolved.Timestamp) diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index d57759f36e8e..28aafb7bd2ac 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/cdctest" "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -3048,22 +3049,57 @@ func TestChangefeedHandlesDrainingNodes(t *testing.T) { }) } -// Primary key changes are not currently supported by changefeeds. There used -// to be no detection but rather they would fail with an inscrutible error or -// not all, just swallowing writes. -func TestChangefeedPrimaryKeyChange(t *testing.T) { +// Primary key changes are supported by changefeeds starting in 21.1. This tests +// that basic behavior works. +func TestChangefeedPrimaryKeyChangeWorks(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + skip.UnderRace(t) + skip.UnderShort(t) + + defer jobs.TestingSetAdoptAndCancelIntervals(10*time.Millisecond, 10*time.Millisecond)() + testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) { sqlDB := sqlutils.MakeSQLRunner(db) sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b STRING NOT NULL)`) sqlDB.Exec(t, `INSERT INTO foo VALUES (0, 'initial')`) sqlDB.Exec(t, `UPSERT INTO foo VALUES (0, 'updated')`) - foo := feed(t, f, `CREATE CHANGEFEED FOR foo`) + const baseStmt = `CREATE CHANGEFEED FOR foo WITH resolved = '100ms'` + foo := feed(t, f, baseStmt) defer closeFeed(t, foo) + // maybeHandleRestart deals with the fact that sinkless changefeeds don't + // gracefully handle primary index changes but rather force the client to + // deal with restarting the changefeed as of the last resolved timestamp. + // + // This ends up being pretty sane; sinkless changefeeds already require this + // behavior in the face of other transient failures so clients already need + // to implement this logic. + maybeHandleRestart := func(t *testing.T) (cleanup func()) { + return func() {} + } + if strings.HasSuffix(t.Name(), "sinkless") { + maybeHandleRestart = func(t *testing.T) func() { + var resolved hlc.Timestamp + for { + m, err := foo.Next() + if err != nil { + assert.Contains(t, err.Error(), + fmt.Sprintf("schema change occurred at %s", resolved.Next().AsOfSystemTime())) + break + } + resolved = extractResolvedTimestamp(t, m) + } + const restartStmt = baseStmt + ", cursor = $1" + foo = feed(t, f, restartStmt, resolved.AsOfSystemTime()) + return func() { + closeFeed(t, foo) + } + } + } + // 'initial' is skipped because only the latest value ('updated') is // emitted by the initial scan. assertPayloads(t, foo, []string{ @@ -3076,15 +3112,184 @@ func TestChangefeedPrimaryKeyChange(t *testing.T) { `foo: [2]->{"after": {"a": 2, "b": "b"}}`, }) - // ALTER PRIMARY KEY is totally busted as of writing this. - // The change will occur and all future drops will be hidden - // until the data gets GC'd. Assert that we don't see these updates. sqlDB.Exec(t, `ALTER TABLE foo ALTER PRIMARY KEY USING COLUMNS (b)`) - _, err := foo.Next() - require.Regexp(t, "\"foo\" primary key changed", err) + defer maybeHandleRestart(t)() + sqlDB.Exec(t, `INSERT INTO foo VALUES (3, 'c'), (4, 'd')`) + assertPayloads(t, foo, []string{ + `foo: ["c"]->{"after": {"a": 3, "b": "c"}}`, + `foo: ["d"]->{"after": {"a": 4, "b": "d"}}`, + }) + + // ALTER PRIMARY KEY should work and we should see the changed + // primary key in subsequent writes. + sqlDB.Exec(t, ` +BEGIN; +ALTER TABLE foo ALTER PRIMARY KEY USING COLUMNS (a); +INSERT INTO foo VALUES (5, 'e'); +UPDATE foo SET a = 6 WHERE b = 'a'; +COMMIT; +INSERT INTO foo VALUES (1, 'f'); +`) + // Note that the primary key change is asynchronous and that only the + // subsequent write will be displayed using the new primary key. + assertPayloads(t, foo, []string{ + `foo: ["a"]->{"after": {"a": 6, "b": "a"}}`, + `foo: ["e"]->{"after": {"a": 5, "b": "e"}}`, + }) + defer maybeHandleRestart(t)() + assertPayloads(t, foo, []string{ + `foo: [1]->{"after": {"a": 1, "b": "f"}}`, + }) } t.Run(`sinkless`, sinklessTest(testFn)) t.Run(`enterprise`, enterpriseTest(testFn)) t.Run(`cloudstorage`, cloudStorageTest(testFn)) } + +// Primary key changes are supported by changefeeds starting in 21.1. This test +// specifically focuses on backfill behavior when a single transaction changes +// multiple tables including a primary key change to one and a column change +// requiring a backfill to another. +// +// Note that at time of writing, this change will not end up occurring in the +// same transaction and thus at the same moment but in later code changes, it +// will. +func TestChangefeedPrimaryKeyChangeWorksWithMultipleTables(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderRace(t) + skip.UnderShort(t) + + defer jobs.TestingSetAdoptAndCancelIntervals(10*time.Millisecond, 10*time.Millisecond)() + + testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) { + sqlDB := sqlutils.MakeSQLRunner(db) + sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b STRING NOT NULL)`) + sqlDB.Exec(t, `INSERT INTO foo VALUES (0, 'initial')`) + sqlDB.Exec(t, `UPSERT INTO foo VALUES (0, 'updated')`) + sqlDB.Exec(t, `CREATE TABLE bar (a INT PRIMARY KEY, b STRING NOT NULL)`) + sqlDB.Exec(t, `INSERT INTO bar VALUES (1, 'a')`) + + const baseStmt = `CREATE CHANGEFEED FOR foo, bar WITH resolved = '100ms'` + cf := feed(t, f, baseStmt) + defer closeFeed(t, cf) + + // maybeHandleRestart deals with the fact that sinkless changefeeds don't + // gracefully handle primary index changes but rather force the client to + // deal with restarting the changefeed as of the last resolved timestamp. + // + // This ends up being pretty sane; sinkless changefeeds already require this + // behavior in the face of other transient failures so clients already need + // to implement this logic. + maybeHandleRestart := func(t *testing.T) (cleanup func()) { + return func() {} + } + if strings.HasSuffix(t.Name(), "sinkless") { + maybeHandleRestart = func(t *testing.T) func() { + var resolvedTS hlc.Timestamp + for { + m, err := cf.Next() + if err != nil { + assert.Contains(t, err.Error(), fmt.Sprintf("schema change occurred at %s", resolvedTS.Next().AsOfSystemTime())) + break + } + resolvedTS = extractResolvedTimestamp(t, m) + } + const restartStmt = baseStmt + ", cursor = $1" + cf = feed(t, f, restartStmt, resolvedTS.AsOfSystemTime()) + return func() { + closeFeed(t, cf) + } + } + } + + // 'initial' is skipped because only the latest value ('updated') is + // emitted by the initial scan. + assertPayloads(t, cf, []string{ + `foo: [0]->{"after": {"a": 0, "b": "updated"}}`, + `bar: [1]->{"after": {"a": 1, "b": "a"}}`, + }) + + sqlDB.Exec(t, `INSERT INTO foo VALUES (1, 'a'), (2, 'b')`) + sqlDB.Exec(t, `INSERT INTO bar VALUES (2, 'b'), (3, 'c')`) + assertPayloads(t, cf, []string{ + `foo: [1]->{"after": {"a": 1, "b": "a"}}`, + `foo: [2]->{"after": {"a": 2, "b": "b"}}`, + `bar: [2]->{"after": {"a": 2, "b": "b"}}`, + `bar: [3]->{"after": {"a": 3, "b": "c"}}`, + }) + + sqlDB.Exec(t, ` +BEGIN; +ALTER TABLE foo ALTER PRIMARY KEY USING COLUMNS (b); +INSERT INTO bar VALUES (4, 'd'), (5, 'e'); +INSERT INTO foo VALUES (3, 'c'); +COMMIT; +INSERT INTO foo VALUES (4, 'd'); +INSERT INTO bar VALUES (6, 'f'); +`) + + assertPayloads(t, cf, []string{ + `bar: [4]->{"after": {"a": 4, "b": "d"}}`, + `bar: [5]->{"after": {"a": 5, "b": "e"}}`, + `foo: [3]->{"after": {"a": 3, "b": "c"}}`, + }) + defer maybeHandleRestart(t)() + assertPayloads(t, cf, []string{ + `foo: ["d"]->{"after": {"a": 4, "b": "d"}}`, + `bar: [6]->{"after": {"a": 6, "b": "f"}}`, + }) + } + + t.Run(`sinkless`, sinklessTest(testFn)) + t.Run(`enterprise`, enterpriseTest(testFn)) + t.Run(`cloudstorage`, cloudStorageTest(testFn)) +} + +// Primary key changes are supported by changefeeds starting in 21.1. +func TestChangefeedPrimaryKeyChangeMixedVersion(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderRace(t) + skip.UnderShort(t) + + defer jobs.TestingSetAdoptAndCancelIntervals(10*time.Millisecond, 10*time.Millisecond)() + + testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) { + sqlDB := sqlutils.MakeSQLRunner(db) + sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b STRING NOT NULL)`) + sqlDB.Exec(t, `INSERT INTO foo VALUES (0, 'initial')`) + sqlDB.Exec(t, `UPSERT INTO foo VALUES (0, 'updated')`) + + foo := feed(t, f, `CREATE CHANGEFEED FOR foo`) + defer closeFeed(t, foo) + + // 'initial' is skipped because only the latest value ('updated') is + // emitted by the initial scan. + assertPayloads(t, foo, []string{ + `foo: [0]->{"after": {"a": 0, "b": "updated"}}`, + }) + + // Expect to see an error because primary key changes are not supported + // in the mixed version state. + sqlDB.Exec(t, `ALTER TABLE foo ALTER PRIMARY KEY USING COLUMNS (b)`) + _, err := foo.Next() + require.Regexp(t, `primary key change occurred`, err) + } + + t.Run("enterprise", enterpriseTestWithServerArgs( + func(args *base.TestServerArgs) { + args.Knobs = base.TestingKnobs{ + Server: &server.TestingKnobs{ + BinaryVersionOverride: clusterversion.ByKey(clusterversion.V20_2), + DisableAutomaticVersionUpgrade: 1, + }, + } + }, + testFn, + )) + +} diff --git a/pkg/ccl/changefeedccl/helpers_test.go b/pkg/ccl/changefeedccl/helpers_test.go index 08b8ee3bb5e1..2545c6c2dae0 100644 --- a/pkg/ccl/changefeedccl/helpers_test.go +++ b/pkg/ccl/changefeedccl/helpers_test.go @@ -34,7 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" ) -var testSinkFlushFrequency = 50 * time.Millisecond +var testSinkFlushFrequency = 100 * time.Millisecond func waitForSchemaChange( t testing.TB, sqlDB *sqlutils.SQLRunner, stmt string, arguments ...interface{}, @@ -64,7 +64,11 @@ func readNextMessages(t testing.TB, f cdctest.TestFeed, numMessages int, stripTs for len(actual) < numMessages { m, err := f.Next() if log.V(1) { - log.Infof(context.Background(), `%v %s: %s->%s`, err, m.Topic, m.Key, m.Value) + if m != nil { + log.Infof(context.Background(), `msg %s: %s->%s (%s)`, m.Topic, m.Key, m.Value, m.Resolved) + } else { + log.Infof(context.Background(), `err %v`, err) + } } if err != nil { t.Fatal(err) @@ -194,6 +198,11 @@ func expectResolvedTimestamp(t testing.TB, f cdctest.TestFeed) hlc.Timestamp { } else if m == nil { t.Fatal(`expected message`) } + return extractResolvedTimestamp(t, m) +} + +func extractResolvedTimestamp(t testing.TB, m *cdctest.TestFeedMessage) hlc.Timestamp { + t.Helper() if m.Key != nil { t.Fatalf(`unexpected row %s: %s -> %s`, m.Topic, m.Key, m.Value) } diff --git a/pkg/ccl/changefeedccl/kvfeed/buffer.go b/pkg/ccl/changefeedccl/kvfeed/buffer.go index f3510f0d4e0a..c0380df51784 100644 --- a/pkg/ccl/changefeedccl/kvfeed/buffer.go +++ b/pkg/ccl/changefeedccl/kvfeed/buffer.go @@ -42,7 +42,7 @@ type EventBufferReader interface { // EventBufferWriter is the write portion of the EventBuffer interface. type EventBufferWriter interface { AddKV(ctx context.Context, kv roachpb.KeyValue, prevVal roachpb.Value, backfillTimestamp hlc.Timestamp) error - AddResolved(ctx context.Context, span roachpb.Span, ts hlc.Timestamp, boundaryReached bool) error + AddResolved(ctx context.Context, span roachpb.Span, ts hlc.Timestamp, boundaryType jobspb.ResolvedSpan_BoundaryType) error Close(ctx context.Context) } @@ -165,9 +165,17 @@ func (b *chanBuffer) AddKV( // AddResolved inserts a Resolved timestamp notification in the buffer. func (b *chanBuffer) AddResolved( - ctx context.Context, span roachpb.Span, ts hlc.Timestamp, boundaryReached bool, + ctx context.Context, + span roachpb.Span, + ts hlc.Timestamp, + boundaryType jobspb.ResolvedSpan_BoundaryType, ) error { - return b.addEvent(ctx, Event{resolved: &jobspb.ResolvedSpan{Span: span, Timestamp: ts, BoundaryReached: boundaryReached}}) + return b.addEvent(ctx, Event{resolved: &jobspb.ResolvedSpan{ + Span: span, + Timestamp: ts, + DeprecatedBoundaryReached: boundaryType != jobspb.ResolvedSpan_NONE, + BoundaryType: boundaryType, + }}) } func (b *chanBuffer) Close(_ context.Context) { @@ -273,7 +281,10 @@ func (b *memBuffer) AddKV( // AddResolved inserts a Resolved timestamp notification in the buffer. func (b *memBuffer) AddResolved( - ctx context.Context, span roachpb.Span, ts hlc.Timestamp, boundaryReached bool, + ctx context.Context, + span roachpb.Span, + ts hlc.Timestamp, + boundaryType jobspb.ResolvedSpan_BoundaryType, ) error { b.allocMu.Lock() row := tree.Datums{ diff --git a/pkg/ccl/changefeedccl/kvfeed/kv_feed.go b/pkg/ccl/changefeedccl/kvfeed/kv_feed.go index 670ab8607672..32791d6232f6 100644 --- a/pkg/ccl/changefeedccl/kvfeed/kv_feed.go +++ b/pkg/ccl/changefeedccl/kvfeed/kv_feed.go @@ -57,7 +57,7 @@ type Config struct { // been seen. NeedsInitialScan bool - // InitialHighWater is the timestamp from which new events are guaranteed to + // InitialHighWater is the timestamp after which new events are guaranteed to // be produced. InitialHighWater hlc.Timestamp } @@ -211,22 +211,40 @@ func (f *kvFeed) run(ctx context.Context) (err error) { return err } + boundaryType := jobspb.ResolvedSpan_BACKFILL + if f.schemaChangePolicy == changefeedbase.OptSchemaChangePolicyStop { + boundaryType = jobspb.ResolvedSpan_EXIT + } else if events, err := f.tableFeed.Peek(ctx, highWater.Next()); err == nil && isPrimaryKeyChange(events) { + boundaryType = jobspb.ResolvedSpan_RESTART + } else if err != nil { + return err + } // Resolve all of the spans as a boundary if the policy indicates that // we should do so. - if f.schemaChangePolicy != changefeedbase.OptSchemaChangePolicyNoBackfill { + if f.schemaChangePolicy != changefeedbase.OptSchemaChangePolicyNoBackfill || + boundaryType == jobspb.ResolvedSpan_RESTART { for _, span := range f.spans { - if err := f.sink.AddResolved(ctx, span, highWater, true); err != nil { + if err := f.sink.AddResolved(ctx, span, highWater, boundaryType); err != nil { return err } } } // Exit if the policy says we should. - if f.schemaChangePolicy == changefeedbase.OptSchemaChangePolicyStop { + if boundaryType == jobspb.ResolvedSpan_RESTART || boundaryType == jobspb.ResolvedSpan_EXIT { return schemaChangeDetectedError{highWater.Next()} } } } +func isPrimaryKeyChange(events []schemafeed.TableEvent) bool { + for _, ev := range events { + if schemafeed.IsPrimaryIndexChange(ev) { + return true + } + } + return false +} + func (f *kvFeed) scanIfShould( ctx context.Context, initialScan bool, highWater hlc.Timestamp, ) error { @@ -248,6 +266,17 @@ func (f *kvFeed) scanIfShould( // Only backfill for the tables which have events which may not be all // of the targets. for _, ev := range events { + // If the event corresponds to a primary index change, it does not + // indicate a need for a backfill. Furthermore, if the changefeed was + // started at this timestamp because of a restart due to a primary index + // change, then a backfill should not be performed for that table. + // Below the code detects whether the set of spans to backfill is empty + // and returns early. This is important because a change to a primary + // index may occur in the same transaction as a change requiring a + // backfill. + if schemafeed.IsPrimaryIndexChange(ev) { + continue + } tablePrefix := f.codec.TablePrefix(uint32(ev.After.GetID())) tableSpan := roachpb.Span{Key: tablePrefix, EndKey: tablePrefix.PrefixEnd()} for _, sp := range f.spans { @@ -269,7 +298,8 @@ func (f *kvFeed) scanIfShould( return err } - if !isInitialScan && f.schemaChangePolicy == changefeedbase.OptSchemaChangePolicyNoBackfill { + if (!isInitialScan && f.schemaChangePolicy == changefeedbase.OptSchemaChangePolicyNoBackfill) || + len(spansToBackfill) == 0 { return nil } @@ -401,7 +431,7 @@ func copyFromSourceToSinkUntilTableEvent( // The logic currently doesn't make this clean. resolved := e.Resolved() frontier.Forward(resolved.Span, resolved.Timestamp) - return sink.AddResolved(ctx, resolved.Span, resolved.Timestamp, false) + return sink.AddResolved(ctx, resolved.Span, resolved.Timestamp, jobspb.ResolvedSpan_NONE) default: log.Fatal(ctx, "unknown event type") return nil diff --git a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go index a1edc616dfb9..44443ce7a572 100644 --- a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go +++ b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go @@ -11,6 +11,7 @@ package kvfeed import ( "context" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -101,7 +102,7 @@ func (p *rangefeed) addEventsToBuffer(ctx context.Context) error { // Changefeeds don't care about these at all, so throw them out. continue } - if err := p.memBuf.AddResolved(ctx, t.Span, t.ResolvedTS, false); err != nil { + if err := p.memBuf.AddResolved(ctx, t.Span, t.ResolvedTS, jobspb.ResolvedSpan_NONE); err != nil { return err } default: diff --git a/pkg/ccl/changefeedccl/kvfeed/scanner.go b/pkg/ccl/changefeedccl/kvfeed/scanner.go index a7bd0da15af5..fb7bb464209d 100644 --- a/pkg/ccl/changefeedccl/kvfeed/scanner.go +++ b/pkg/ccl/changefeedccl/kvfeed/scanner.go @@ -14,6 +14,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/gossip" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" @@ -140,7 +141,7 @@ func (p *scanRequestScanner) exportSpan( } } // p.metrics.PollRequestNanosHist.RecordValue(scanDuration.Nanoseconds()) - if err := sink.AddResolved(ctx, span, ts, false); err != nil { + if err := sink.AddResolved(ctx, span, ts, jobspb.ResolvedSpan_NONE); err != nil { return err } if log.V(2) { diff --git a/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go b/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go index a4406fd00b8a..fa47c92224e1 100644 --- a/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go +++ b/pkg/ccl/changefeedccl/schemafeed/table_event_filter.go @@ -34,6 +34,7 @@ var ( tableEventTypeAddColumnWithBackfill: false, tableEventTypeAddColumnNoBackfill: true, tableEventTypeUnknown: true, + tableEventPrimaryKeyChange: false, } columnChangeTableEventFilter = tableEventFilter{ @@ -41,6 +42,7 @@ var ( tableEventTypeAddColumnWithBackfill: false, tableEventTypeAddColumnNoBackfill: false, tableEventTypeUnknown: true, + tableEventPrimaryKeyChange: false, } schemaChangeEventFilters = map[changefeedbase.SchemaChangeEventClass]tableEventFilter{ @@ -76,9 +78,6 @@ func (b tableEventFilter) shouldFilter(ctx context.Context, e TableEvent) (bool, if et == tableEventTruncate { return false, errors.Errorf(`"%s" was truncated`, e.Before.GetName()) } - if et == tableEventPrimaryKeyChange { - return false, errors.Errorf(`"%s" primary key changed`, e.Before.GetName()) - } shouldFilter, ok := b[et] if !ok { return false, errors.AssertionFailedf("policy does not specify how to handle event type %v", et) @@ -137,3 +136,9 @@ func primaryKeyChanged(e TableEvent) bool { return e.Before.GetPrimaryIndexID() != e.After.GetPrimaryIndexID() && pkChangeMutationExists(e.Before) } + +// IsPrimaryIndexChange returns true if the event corresponds to a change +// in the primary index. +func IsPrimaryIndexChange(e TableEvent) bool { + return classifyTableEvent(e) == tableEventPrimaryKeyChange +} diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index acd7ccd8fd86..9f5ad82a58f4 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -255,6 +255,9 @@ const ( // ClosedTimestampsRaftTransport enables the Raft transport for closed // timestamps and disables the previous per-node transport. ClosedTimestampsRaftTransport + // ChangefeedsSupportPrimaryIndexChanges is used to indicate that all + // nodes support detecting and restarting on primary index changes. + ChangefeedsSupportPrimaryIndexChanges // Step (1): Add new versions here. ) @@ -433,6 +436,11 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: ClosedTimestampsRaftTransport, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 36}, }, + { + Key: ChangefeedsSupportPrimaryIndexChanges, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 38}, + }, + // Step (2): Add new versions here. }) diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index ad864ce4c6ca..322e1c0d5987 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -46,11 +46,12 @@ func _() { _ = x[ImplicitColumnPartitioning-35] _ = x[MultiRegionFeatures-36] _ = x[ClosedTimestampsRaftTransport-37] + _ = x[ChangefeedsSupportPrimaryIndexChanges-38] } -const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsSequencesRegclassImplicitColumnPartitioningMultiRegionFeaturesClosedTimestampsRaftTransport" +const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsSequencesRegclassImplicitColumnPartitioningMultiRegionFeaturesClosedTimestampsRaftTransportChangefeedsSupportPrimaryIndexChanges" -var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 200, 217, 231, 295, 312, 321, 346, 366, 378, 383, 392, 420, 449, 471, 481, 496, 542, 592, 608, 629, 667, 709, 725, 761, 778, 804, 823, 852} +var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 200, 217, 231, 295, 312, 321, 346, 366, 378, 383, 392, 420, 449, 471, 481, 496, 542, 592, 608, 629, 667, 709, 725, 761, 778, 804, 823, 852, 889} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 64947911725c..f0e26afd4320 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -58,7 +58,7 @@ func (x EncryptionMode) String() string { return proto.EnumName(EncryptionMode_name, int32(x)) } func (EncryptionMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{0} } type Status int32 @@ -87,7 +87,7 @@ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } func (Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{1} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{1} } type Type int32 @@ -142,7 +142,7 @@ var Type_value = map[string]int32{ } func (Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{2} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{2} } type EncryptionInfo_Scheme int32 @@ -162,7 +162,7 @@ func (x EncryptionInfo_Scheme) String() string { return proto.EnumName(EncryptionInfo_Scheme_name, int32(x)) } func (EncryptionInfo_Scheme) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{2, 0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{2, 0} } type SchemaChangeGCProgress_Status int32 @@ -192,7 +192,45 @@ func (x SchemaChangeGCProgress_Status) String() string { return proto.EnumName(SchemaChangeGCProgress_Status_name, int32(x)) } func (SchemaChangeGCProgress_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{22, 0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{22, 0} +} + +type ResolvedSpan_BoundaryType int32 + +const ( + // NONE indicates that this resolved span does not correspond to a + // boundary. + ResolvedSpan_NONE ResolvedSpan_BoundaryType = 0 + // BACKFILL indicates that this resolved span corresponds to a boundary + // requiring a backfill internally and perhaps indicates the need for a + // protected timestamp. + ResolvedSpan_BACKFILL ResolvedSpan_BoundaryType = 1 + // EXIT indicates that this resolved span corresponds to a boundary which + // should result in the changefeed exiting. + ResolvedSpan_EXIT ResolvedSpan_BoundaryType = 2 + // RESTART indicates that this resolved span corresponds to a boundary which + // should result in the changefeed restarting. + ResolvedSpan_RESTART ResolvedSpan_BoundaryType = 3 +) + +var ResolvedSpan_BoundaryType_name = map[int32]string{ + 0: "NONE", + 1: "BACKFILL", + 2: "EXIT", + 3: "RESTART", +} +var ResolvedSpan_BoundaryType_value = map[string]int32{ + "NONE": 0, + "BACKFILL": 1, + "EXIT": 2, + "RESTART": 3, +} + +func (x ResolvedSpan_BoundaryType) String() string { + return proto.EnumName(ResolvedSpan_BoundaryType_name, int32(x)) +} +func (ResolvedSpan_BoundaryType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_jobs_25813d1ee2eedea8, []int{25, 0} } type Lease struct { @@ -206,7 +244,7 @@ func (m *Lease) Reset() { *m = Lease{} } func (m *Lease) String() string { return proto.CompactTextString(m) } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{0} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -247,7 +285,7 @@ func (m *BackupEncryptionOptions) Reset() { *m = BackupEncryptionOptions func (m *BackupEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions) ProtoMessage() {} func (*BackupEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{1} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{1} } func (m *BackupEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -281,7 +319,7 @@ func (m *BackupEncryptionOptions_KMSInfo) Reset() { *m = BackupEncryptio func (m *BackupEncryptionOptions_KMSInfo) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions_KMSInfo) ProtoMessage() {} func (*BackupEncryptionOptions_KMSInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{1, 0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{1, 0} } func (m *BackupEncryptionOptions_KMSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -321,7 +359,7 @@ func (m *EncryptionInfo) Reset() { *m = EncryptionInfo{} } func (m *EncryptionInfo) String() string { return proto.CompactTextString(m) } func (*EncryptionInfo) ProtoMessage() {} func (*EncryptionInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{2} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{2} } func (m *EncryptionInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -364,7 +402,7 @@ func (m *StreamIngestionDetails) Reset() { *m = StreamIngestionDetails{} func (m *StreamIngestionDetails) String() string { return proto.CompactTextString(m) } func (*StreamIngestionDetails) ProtoMessage() {} func (*StreamIngestionDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{3} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{3} } func (m *StreamIngestionDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -401,7 +439,7 @@ func (m *StreamIngestionProgress) Reset() { *m = StreamIngestionProgress func (m *StreamIngestionProgress) String() string { return proto.CompactTextString(m) } func (*StreamIngestionProgress) ProtoMessage() {} func (*StreamIngestionProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{4} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{4} } func (m *StreamIngestionProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -456,7 +494,7 @@ func (m *BackupDetails) Reset() { *m = BackupDetails{} } func (m *BackupDetails) String() string { return proto.CompactTextString(m) } func (*BackupDetails) ProtoMessage() {} func (*BackupDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{5} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{5} } func (m *BackupDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -488,7 +526,7 @@ func (m *BackupProgress) Reset() { *m = BackupProgress{} } func (m *BackupProgress) String() string { return proto.CompactTextString(m) } func (*BackupProgress) ProtoMessage() {} func (*BackupProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{6} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{6} } func (m *BackupProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -550,7 +588,7 @@ func (m *RestoreDetails) Reset() { *m = RestoreDetails{} } func (m *RestoreDetails) String() string { return proto.CompactTextString(m) } func (*RestoreDetails) ProtoMessage() {} func (*RestoreDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{7} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{7} } func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -587,7 +625,7 @@ func (m *RestoreDetails_DescriptorRewrite) Reset() { *m = RestoreDetails func (m *RestoreDetails_DescriptorRewrite) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_DescriptorRewrite) ProtoMessage() {} func (*RestoreDetails_DescriptorRewrite) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{7, 0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{7, 0} } func (m *RestoreDetails_DescriptorRewrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -620,7 +658,7 @@ func (m *RestoreDetails_BackupLocalityInfo) Reset() { *m = RestoreDetail func (m *RestoreDetails_BackupLocalityInfo) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_BackupLocalityInfo) ProtoMessage() {} func (*RestoreDetails_BackupLocalityInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{7, 1} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{7, 1} } func (m *RestoreDetails_BackupLocalityInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -653,7 +691,7 @@ func (m *RestoreProgress) Reset() { *m = RestoreProgress{} } func (m *RestoreProgress) String() string { return proto.CompactTextString(m) } func (*RestoreProgress) ProtoMessage() {} func (*RestoreProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{8} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{8} } func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -714,7 +752,7 @@ func (m *ImportDetails) Reset() { *m = ImportDetails{} } func (m *ImportDetails) String() string { return proto.CompactTextString(m) } func (*ImportDetails) ProtoMessage() {} func (*ImportDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{9} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{9} } func (m *ImportDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -752,7 +790,7 @@ func (m *ImportDetails_Table) Reset() { *m = ImportDetails_Table{} } func (m *ImportDetails_Table) String() string { return proto.CompactTextString(m) } func (*ImportDetails_Table) ProtoMessage() {} func (*ImportDetails_Table) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{9, 0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{9, 0} } func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -794,7 +832,7 @@ func (m *SequenceValChunk) Reset() { *m = SequenceValChunk{} } func (m *SequenceValChunk) String() string { return proto.CompactTextString(m) } func (*SequenceValChunk) ProtoMessage() {} func (*SequenceValChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{10} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{10} } func (m *SequenceValChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -830,7 +868,7 @@ func (m *SequenceDetails) Reset() { *m = SequenceDetails{} } func (m *SequenceDetails) String() string { return proto.CompactTextString(m) } func (*SequenceDetails) ProtoMessage() {} func (*SequenceDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{11} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{11} } func (m *SequenceDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -865,7 +903,7 @@ func (m *SequenceDetails_SequenceChunks) Reset() { *m = SequenceDetails_ func (m *SequenceDetails_SequenceChunks) String() string { return proto.CompactTextString(m) } func (*SequenceDetails_SequenceChunks) ProtoMessage() {} func (*SequenceDetails_SequenceChunks) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{11, 0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{11, 0} } func (m *SequenceDetails_SequenceChunks) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -911,7 +949,7 @@ func (m *ImportProgress) Reset() { *m = ImportProgress{} } func (m *ImportProgress) String() string { return proto.CompactTextString(m) } func (*ImportProgress) ProtoMessage() {} func (*ImportProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{12} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{12} } func (m *ImportProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -954,7 +992,7 @@ func (m *TypeSchemaChangeDetails) Reset() { *m = TypeSchemaChangeDetails func (m *TypeSchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeDetails) ProtoMessage() {} func (*TypeSchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{13} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{13} } func (m *TypeSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -987,7 +1025,7 @@ func (m *TypeSchemaChangeProgress) Reset() { *m = TypeSchemaChangeProgre func (m *TypeSchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeProgress) ProtoMessage() {} func (*TypeSchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{14} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{14} } func (m *TypeSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1021,7 +1059,7 @@ func (m *NewSchemaChangeDetails) Reset() { *m = NewSchemaChangeDetails{} func (m *NewSchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*NewSchemaChangeDetails) ProtoMessage() {} func (*NewSchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{15} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{15} } func (m *NewSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1055,7 +1093,7 @@ func (m *NewSchemaChangeProgress) Reset() { *m = NewSchemaChangeProgress func (m *NewSchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*NewSchemaChangeProgress) ProtoMessage() {} func (*NewSchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{16} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{16} } func (m *NewSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1088,7 +1126,7 @@ func (m *ResumeSpanList) Reset() { *m = ResumeSpanList{} } func (m *ResumeSpanList) String() string { return proto.CompactTextString(m) } func (*ResumeSpanList) ProtoMessage() {} func (*ResumeSpanList) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{17} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{17} } func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1123,7 +1161,7 @@ func (m *DroppedTableDetails) Reset() { *m = DroppedTableDetails{} } func (m *DroppedTableDetails) String() string { return proto.CompactTextString(m) } func (*DroppedTableDetails) ProtoMessage() {} func (*DroppedTableDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{18} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{18} } func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1186,7 +1224,7 @@ func (m *SchemaChangeGCDetails) Reset() { *m = SchemaChangeGCDetails{} } func (m *SchemaChangeGCDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails) ProtoMessage() {} func (*SchemaChangeGCDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{19} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{19} } func (m *SchemaChangeGCDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1220,7 +1258,7 @@ func (m *SchemaChangeGCDetails_DroppedIndex) Reset() { *m = SchemaChange func (m *SchemaChangeGCDetails_DroppedIndex) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedIndex) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedIndex) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{19, 0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{19, 0} } func (m *SchemaChangeGCDetails_DroppedIndex) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1254,7 +1292,7 @@ func (m *SchemaChangeGCDetails_DroppedID) Reset() { *m = SchemaChangeGCD func (m *SchemaChangeGCDetails_DroppedID) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedID) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedID) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{19, 1} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{19, 1} } func (m *SchemaChangeGCDetails_DroppedID) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1288,7 +1326,7 @@ func (m *SchemaChangeGCDetails_DroppedTenant) Reset() { *m = SchemaChang func (m *SchemaChangeGCDetails_DroppedTenant) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedTenant) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedTenant) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{19, 2} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{19, 2} } func (m *SchemaChangeGCDetails_DroppedTenant) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1352,7 +1390,7 @@ func (m *SchemaChangeDetails) Reset() { *m = SchemaChangeDetails{} } func (m *SchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeDetails) ProtoMessage() {} func (*SchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{20} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{20} } func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1384,7 +1422,7 @@ func (m *SchemaChangeProgress) Reset() { *m = SchemaChangeProgress{} } func (m *SchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeProgress) ProtoMessage() {} func (*SchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{21} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{21} } func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1422,7 +1460,7 @@ func (m *SchemaChangeGCProgress) Reset() { *m = SchemaChangeGCProgress{} func (m *SchemaChangeGCProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress) ProtoMessage() {} func (*SchemaChangeGCProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{22} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{22} } func (m *SchemaChangeGCProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1456,7 +1494,7 @@ func (m *SchemaChangeGCProgress_IndexProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_IndexProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_IndexProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_IndexProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{22, 0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{22, 0} } func (m *SchemaChangeGCProgress_IndexProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1490,7 +1528,7 @@ func (m *SchemaChangeGCProgress_TableProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_TableProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_TableProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_TableProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{22, 1} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{22, 1} } func (m *SchemaChangeGCProgress_TableProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1523,7 +1561,7 @@ func (m *SchemaChangeGCProgress_TenantProgress) Reset() { *m = SchemaCha func (m *SchemaChangeGCProgress_TenantProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_TenantProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_TenantProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{22, 2} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{22, 2} } func (m *SchemaChangeGCProgress_TenantProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1556,7 +1594,7 @@ func (m *ChangefeedTarget) Reset() { *m = ChangefeedTarget{} } func (m *ChangefeedTarget) String() string { return proto.CompactTextString(m) } func (*ChangefeedTarget) ProtoMessage() {} func (*ChangefeedTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{23} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{23} } func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1609,7 +1647,7 @@ func (m *ChangefeedDetails) Reset() { *m = ChangefeedDetails{} } func (m *ChangefeedDetails) String() string { return proto.CompactTextString(m) } func (*ChangefeedDetails) ProtoMessage() {} func (*ChangefeedDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{24} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{24} } func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1635,16 +1673,21 @@ func (m *ChangefeedDetails) XXX_DiscardUnknown() { var xxx_messageInfo_ChangefeedDetails proto.InternalMessageInfo type ResolvedSpan struct { - Span roachpb.Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` - Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` - BoundaryReached bool `protobuf:"varint,3,opt,name=boundary_reached,json=boundaryReached,proto3" json:"boundary_reached,omitempty"` + Span roachpb.Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + // BoundaryReached indicates that this resolved timestamp + // corresponds to a schema change boundary. A schema change boundary is a + // timestamp at which a new version of a table has been committed that either + // indicates a logical backfill or a need for the changefeed to exit. + DeprecatedBoundaryReached bool `protobuf:"varint,3,opt,name=deprecated_boundary_reached,json=deprecatedBoundaryReached,proto3" json:"deprecated_boundary_reached,omitempty"` + BoundaryType ResolvedSpan_BoundaryType `protobuf:"varint,4,opt,name=boundary_type,json=boundaryType,proto3,enum=cockroach.sql.jobs.jobspb.ResolvedSpan_BoundaryType" json:"boundary_type,omitempty"` } func (m *ResolvedSpan) Reset() { *m = ResolvedSpan{} } func (m *ResolvedSpan) String() string { return proto.CompactTextString(m) } func (*ResolvedSpan) ProtoMessage() {} func (*ResolvedSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{25} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{25} } func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1677,7 +1720,7 @@ func (m *ResolvedSpans) Reset() { *m = ResolvedSpans{} } func (m *ResolvedSpans) String() string { return proto.CompactTextString(m) } func (*ResolvedSpans) ProtoMessage() {} func (*ResolvedSpans) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{26} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{26} } func (m *ResolvedSpans) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1720,7 +1763,7 @@ func (m *ChangefeedProgress) Reset() { *m = ChangefeedProgress{} } func (m *ChangefeedProgress) String() string { return proto.CompactTextString(m) } func (*ChangefeedProgress) ProtoMessage() {} func (*ChangefeedProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{27} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{27} } func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1764,7 +1807,7 @@ func (m *CreateStatsDetails) Reset() { *m = CreateStatsDetails{} } func (m *CreateStatsDetails) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails) ProtoMessage() {} func (*CreateStatsDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{28} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{28} } func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1805,7 +1848,7 @@ func (m *CreateStatsDetails_ColStat) Reset() { *m = CreateStatsDetails_C func (m *CreateStatsDetails_ColStat) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails_ColStat) ProtoMessage() {} func (*CreateStatsDetails_ColStat) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{28, 0} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{28, 0} } func (m *CreateStatsDetails_ColStat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1837,7 +1880,7 @@ func (m *CreateStatsProgress) Reset() { *m = CreateStatsProgress{} } func (m *CreateStatsProgress) String() string { return proto.CompactTextString(m) } func (*CreateStatsProgress) ProtoMessage() {} func (*CreateStatsProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{29} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{29} } func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1870,7 +1913,7 @@ func (m *MigrationDetails) Reset() { *m = MigrationDetails{} } func (m *MigrationDetails) String() string { return proto.CompactTextString(m) } func (*MigrationDetails) ProtoMessage() {} func (*MigrationDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{30} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{30} } func (m *MigrationDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1902,7 +1945,7 @@ func (m *MigrationProgress) Reset() { *m = MigrationProgress{} } func (m *MigrationProgress) String() string { return proto.CompactTextString(m) } func (*MigrationProgress) ProtoMessage() {} func (*MigrationProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{31} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{31} } func (m *MigrationProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1972,7 +2015,7 @@ func (m *Payload) Reset() { *m = Payload{} } func (m *Payload) String() string { return proto.CompactTextString(m) } func (*Payload) ProtoMessage() {} func (*Payload) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{32} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{32} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2404,7 +2447,7 @@ func (m *Progress) Reset() { *m = Progress{} } func (m *Progress) String() string { return proto.CompactTextString(m) } func (*Progress) ProtoMessage() {} func (*Progress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{33} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{33} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2900,7 +2943,7 @@ func (m *Job) Reset() { *m = Job{} } func (m *Job) String() string { return proto.CompactTextString(m) } func (*Job) ProtoMessage() {} func (*Job) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_61071a5ccbcb72b9, []int{34} + return fileDescriptor_jobs_25813d1ee2eedea8, []int{34} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2985,6 +3028,7 @@ func init() { proto.RegisterEnum("cockroach.sql.jobs.jobspb.Type", Type_name, Type_value) proto.RegisterEnum("cockroach.sql.jobs.jobspb.EncryptionInfo_Scheme", EncryptionInfo_Scheme_name, EncryptionInfo_Scheme_value) proto.RegisterEnum("cockroach.sql.jobs.jobspb.SchemaChangeGCProgress_Status", SchemaChangeGCProgress_Status_name, SchemaChangeGCProgress_Status_value) + proto.RegisterEnum("cockroach.sql.jobs.jobspb.ResolvedSpan_BoundaryType", ResolvedSpan_BoundaryType_name, ResolvedSpan_BoundaryType_value) } func (this *Lease) Equal(that interface{}) bool { if that == nil { @@ -4913,16 +4957,21 @@ func (m *ResolvedSpan) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n34 - if m.BoundaryReached { + if m.DeprecatedBoundaryReached { dAtA[i] = 0x18 i++ - if m.BoundaryReached { + if m.DeprecatedBoundaryReached { dAtA[i] = 1 } else { dAtA[i] = 0 } i++ } + if m.BoundaryType != 0 { + dAtA[i] = 0x20 + i++ + i = encodeVarintJobs(dAtA, i, uint64(m.BoundaryType)) + } return i, nil } @@ -6615,9 +6664,12 @@ func (m *ResolvedSpan) Size() (n int) { n += 1 + l + sovJobs(uint64(l)) l = m.Timestamp.Size() n += 1 + l + sovJobs(uint64(l)) - if m.BoundaryReached { + if m.DeprecatedBoundaryReached { n += 2 } + if m.BoundaryType != 0 { + n += 1 + sovJobs(uint64(m.BoundaryType)) + } return n } @@ -12919,7 +12971,7 @@ func (m *ResolvedSpan) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field BoundaryReached", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedBoundaryReached", wireType) } var v int for shift := uint(0); ; shift += 7 { @@ -12936,7 +12988,26 @@ func (m *ResolvedSpan) Unmarshal(dAtA []byte) error { break } } - m.BoundaryReached = bool(v != 0) + m.DeprecatedBoundaryReached = bool(v != 0) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BoundaryType", wireType) + } + m.BoundaryType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BoundaryType |= (ResolvedSpan_BoundaryType(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipJobs(dAtA[iNdEx:]) @@ -15267,312 +15338,316 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_61071a5ccbcb72b9) } - -var fileDescriptor_jobs_61071a5ccbcb72b9 = []byte{ - // 4853 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x7b, 0x4b, 0x6c, 0x23, 0x47, - 0x7a, 0xbf, 0x9a, 0xa4, 0xc8, 0xe6, 0x47, 0x91, 0x6a, 0x96, 0x34, 0x33, 0x5c, 0xfe, 0xed, 0xa1, - 0x96, 0x7e, 0xcc, 0xc3, 0x36, 0xe5, 0x1d, 0xff, 0xd7, 0x6b, 0x4f, 0xec, 0xb1, 0xc5, 0x87, 0x24, - 0x52, 0xa3, 0xc7, 0x34, 0xa5, 0xf1, 0x63, 0xe3, 0xed, 0x34, 0xbb, 0x4b, 0x52, 0x47, 0x64, 0x37, - 0xa7, 0xab, 0x39, 0x33, 0xda, 0x00, 0x49, 0xb0, 0x41, 0x80, 0xc5, 0x9c, 0x12, 0x20, 0x9b, 0x4b, - 0x32, 0x40, 0x80, 0xec, 0x02, 0x39, 0x04, 0x08, 0x62, 0x04, 0x49, 0x0e, 0xb9, 0xe5, 0xe2, 0x43, - 0x02, 0xec, 0x25, 0x80, 0x91, 0x03, 0x37, 0x91, 0x2f, 0x39, 0x06, 0xc9, 0x65, 0x31, 0x97, 0x04, - 0xf5, 0xe8, 0x66, 0x93, 0xd4, 0x83, 0x1a, 0xd9, 0x9b, 0xcb, 0x0c, 0xfb, 0xab, 0xaf, 0x7e, 0xf5, - 0xfa, 0xea, 0xf7, 0x7d, 0x5f, 0x55, 0x09, 0x2e, 0xff, 0xa6, 0xd3, 0x22, 0x8b, 0xf4, 0x9f, 0x6e, - 0x8b, 0xfd, 0x57, 0xea, 0xba, 0x8e, 0xe7, 0xa0, 0x6f, 0x19, 0x8e, 0x71, 0xe0, 0x3a, 0xba, 0xb1, - 0x5f, 0x22, 0x0f, 0xda, 0x25, 0x56, 0xc2, 0xb5, 0xf2, 0x97, 0xb0, 0xeb, 0x3a, 0x2e, 0xd5, 0xe7, - 0x3f, 0x78, 0x8d, 0xfc, 0xfc, 0x9e, 0xb3, 0xe7, 0xb0, 0x9f, 0x8b, 0xf4, 0x97, 0x90, 0x22, 0x86, - 0xd1, 0x6d, 0x2d, 0x9a, 0xba, 0xa7, 0x0b, 0x59, 0xce, 0x97, 0x59, 0xce, 0x1b, 0xbb, 0x8e, 0xdb, - 0xd1, 0x3d, 0x1f, 0xe3, 0x25, 0xf2, 0xa0, 0xbd, 0x68, 0xe8, 0x9e, 0xde, 0x76, 0xf6, 0x16, 0x4d, - 0x4c, 0x8c, 0x6e, 0x6b, 0x91, 0x78, 0x6e, 0xcf, 0xf0, 0x7a, 0x2e, 0x36, 0x85, 0x52, 0xe1, 0x18, - 0x25, 0x0f, 0xdb, 0xba, 0xed, 0xf9, 0xf8, 0x3d, 0xcf, 0x6a, 0x2f, 0xee, 0xb7, 0x8d, 0x45, 0xcf, - 0xea, 0x60, 0xe2, 0xe9, 0x9d, 0xae, 0x28, 0xf9, 0x36, 0xad, 0x4a, 0x8c, 0x7d, 0xdc, 0xd1, 0x8d, - 0x7d, 0xdd, 0xde, 0xc3, 0xee, 0x22, 0x6f, 0xc3, 0xe8, 0xb6, 0x84, 0xca, 0xcb, 0x46, 0xbb, 0x47, - 0x3c, 0xec, 0x3e, 0xc4, 0x2e, 0xb1, 0x1c, 0x7b, 0x51, 0x7c, 0x6a, 0xe2, 0x9b, 0x6b, 0x15, 0x7f, - 0x07, 0xa6, 0xef, 0x62, 0x9d, 0x60, 0xf4, 0x29, 0x24, 0x6c, 0xc7, 0xc4, 0x9a, 0x65, 0xe6, 0xa4, - 0x05, 0xe9, 0x7a, 0xba, 0xbc, 0x74, 0xd4, 0x2f, 0xc4, 0x37, 0x1c, 0x13, 0xd7, 0xab, 0xcf, 0xfa, - 0x85, 0xb7, 0xf6, 0x2c, 0x6f, 0xbf, 0xd7, 0x2a, 0x19, 0x4e, 0x67, 0x31, 0x98, 0x51, 0xb3, 0x35, - 0xf8, 0xbd, 0xd8, 0x3d, 0xd8, 0x5b, 0x14, 0xf3, 0x51, 0xe2, 0xd5, 0xd4, 0x38, 0x45, 0xac, 0x9b, - 0x68, 0x1e, 0xa6, 0x71, 0xd7, 0x31, 0xf6, 0x73, 0x91, 0x05, 0xe9, 0x7a, 0x54, 0xe5, 0x1f, 0xb7, - 0x63, 0xff, 0xf1, 0x67, 0x05, 0xa9, 0xf8, 0xb3, 0x08, 0x5c, 0x29, 0xeb, 0xc6, 0x41, 0xaf, 0x5b, - 0xb3, 0x0d, 0xf7, 0xb0, 0xeb, 0x59, 0x8e, 0xbd, 0xc9, 0xfe, 0x25, 0x48, 0x81, 0xe8, 0x01, 0x3e, - 0x64, 0xfd, 0x99, 0x51, 0xe9, 0x4f, 0xf4, 0x3e, 0xc4, 0x3a, 0x8e, 0x89, 0x19, 0x50, 0xe6, 0xd6, - 0x8d, 0xd2, 0x89, 0x8b, 0x5b, 0x1a, 0xa0, 0xad, 0x3b, 0x26, 0x56, 0x59, 0x35, 0xd4, 0x02, 0xf9, - 0xa0, 0x43, 0x34, 0xcb, 0xde, 0x75, 0x72, 0xd1, 0x05, 0xe9, 0x7a, 0xea, 0xd6, 0xed, 0x53, 0x20, - 0x4e, 0xe8, 0x56, 0x69, 0x6d, 0xbd, 0x59, 0xb7, 0x77, 0x9d, 0x72, 0xea, 0xa8, 0x5f, 0x48, 0x88, - 0x0f, 0x35, 0x71, 0xd0, 0x21, 0xf4, 0x47, 0x7e, 0x13, 0x7c, 0x19, 0xed, 0x7f, 0xcf, 0xb5, 0x58, - 0xff, 0x93, 0x2a, 0xfd, 0x89, 0x5e, 0x07, 0x84, 0x39, 0x1e, 0x36, 0x35, 0x6a, 0x49, 0x1a, 0x1d, - 0x60, 0x84, 0x0d, 0x50, 0x09, 0x4a, 0xaa, 0xba, 0xa7, 0xaf, 0xe1, 0x43, 0x3e, 0x43, 0x62, 0x9e, - 0x7e, 0x37, 0x0a, 0x99, 0x41, 0x57, 0x18, 0xfc, 0x2a, 0xc4, 0x99, 0x09, 0x60, 0xd6, 0x42, 0xe6, - 0xd6, 0x9b, 0x13, 0x4d, 0x07, 0xad, 0x5a, 0x6a, 0xb2, 0x7a, 0xaa, 0xa8, 0x8f, 0x10, 0xc4, 0x88, - 0xde, 0xf6, 0x44, 0x47, 0xd8, 0x6f, 0xf4, 0x27, 0x12, 0x2c, 0x8c, 0xf6, 0xa8, 0x7c, 0xb8, 0xb6, - 0xde, 0x5c, 0xd7, 0xa9, 0x1d, 0xad, 0xe1, 0xc3, 0x7a, 0x35, 0x17, 0x5d, 0x88, 0x5e, 0x4f, 0xdd, - 0xda, 0x9c, 0xbc, 0xe1, 0xda, 0x19, 0x88, 0x35, 0xdb, 0x73, 0x0f, 0xd5, 0x33, 0x1b, 0xce, 0x37, - 0xe1, 0x95, 0x89, 0xa0, 0xc2, 0x36, 0x94, 0xe4, 0x36, 0x34, 0x0f, 0xd3, 0x0f, 0xf5, 0x76, 0x0f, - 0x8b, 0xd1, 0xf2, 0x8f, 0xdb, 0x91, 0x77, 0xa4, 0xe2, 0x15, 0x88, 0xf3, 0x89, 0x41, 0x69, 0x48, - 0x2e, 0xd5, 0x9a, 0xb7, 0xbe, 0xfb, 0xf6, 0x4a, 0x65, 0x5d, 0x99, 0x12, 0x4b, 0xf0, 0x3f, 0x12, - 0x5c, 0x6e, 0x7a, 0x2e, 0xd6, 0x3b, 0x75, 0x7b, 0x0f, 0x13, 0x3a, 0xa6, 0x2a, 0xf6, 0x74, 0xab, - 0x4d, 0x90, 0x0d, 0x19, 0xc2, 0x4a, 0x34, 0xdd, 0x34, 0x5d, 0x4c, 0x08, 0x6f, 0xb0, 0xbc, 0xf2, - 0xac, 0x5f, 0xa8, 0x4c, 0xb4, 0x75, 0x0c, 0xa3, 0xbd, 0xc8, 0x21, 0x2c, 0x7b, 0xcf, 0x30, 0xda, - 0x25, 0xde, 0xd2, 0x12, 0x87, 0x53, 0xd3, 0x24, 0xfc, 0x89, 0xbe, 0x03, 0x31, 0xd2, 0xd5, 0x6d, - 0x36, 0x84, 0xd4, 0xad, 0x2b, 0xa1, 0xf9, 0xf7, 0xb7, 0x60, 0xb3, 0xab, 0xdb, 0xe5, 0xd8, 0x17, - 0xfd, 0xc2, 0x94, 0xca, 0x54, 0x51, 0x19, 0x80, 0x78, 0xba, 0xeb, 0x69, 0x94, 0x4b, 0x84, 0xf5, - 0xbf, 0x18, 0xaa, 0x48, 0xb9, 0xa6, 0xb4, 0xdf, 0x36, 0x4a, 0xdb, 0x3e, 0xd7, 0x88, 0xea, 0x49, - 0x56, 0x8d, 0x4a, 0x8b, 0x3a, 0x5c, 0x19, 0x99, 0x80, 0x2d, 0xd7, 0xd9, 0x63, 0x3d, 0x5a, 0x86, - 0x19, 0xa3, 0xe7, 0x39, 0x0f, 0xb1, 0xcb, 0x1b, 0x90, 0x26, 0x6f, 0x20, 0x25, 0x2a, 0xb2, 0x26, - 0xfe, 0x21, 0x0e, 0x69, 0xbe, 0xf1, 0xfc, 0xb9, 0x1d, 0xee, 0xb8, 0xf4, 0x3c, 0x1d, 0x47, 0x77, - 0x40, 0xc6, 0xb6, 0xc9, 0x11, 0x22, 0x93, 0x23, 0x24, 0xb0, 0x6d, 0xb2, 0xfa, 0xdf, 0xe2, 0x3b, - 0x39, 0xca, 0x16, 0x35, 0x71, 0xd4, 0x2f, 0x44, 0x77, 0xd4, 0x3a, 0xdf, 0xd2, 0xef, 0x41, 0xde, - 0xc4, 0x5d, 0x17, 0x1b, 0x3a, 0xdd, 0xd3, 0x2d, 0xd6, 0x75, 0xad, 0xa3, 0xdb, 0xd6, 0x2e, 0x26, - 0x5e, 0x2e, 0xc6, 0x6c, 0x2c, 0x37, 0xd0, 0xe0, 0x63, 0x5b, 0x17, 0xe5, 0xe8, 0xf7, 0x24, 0x98, - 0xeb, 0xb9, 0x16, 0xd1, 0x5a, 0x87, 0x5a, 0xdb, 0x31, 0xf4, 0xb6, 0xe5, 0x1d, 0x6a, 0x07, 0x0f, - 0x73, 0xd3, 0x6c, 0x63, 0xdd, 0x39, 0x93, 0x9d, 0xc4, 0x24, 0x95, 0x76, 0x5c, 0x8b, 0x94, 0x0f, - 0xef, 0x0a, 0x84, 0xb5, 0x87, 0xcc, 0xf8, 0xcb, 0xf3, 0x47, 0xfd, 0x82, 0xb2, 0xa3, 0xd6, 0xc3, - 0x45, 0xf7, 0x55, 0xa5, 0x37, 0xa2, 0x8c, 0xf4, 0x80, 0x96, 0x2c, 0xc7, 0xd6, 0x1c, 0xce, 0x73, - 0xb9, 0x38, 0x9b, 0xa8, 0x5b, 0xe7, 0x67, 0x48, 0x35, 0x8b, 0xc7, 0xb8, 0xfc, 0x0f, 0x25, 0xc8, - 0x53, 0x97, 0x83, 0x0d, 0x3a, 0x4d, 0x81, 0x3f, 0xd3, 0x5c, 0x6c, 0x38, 0xae, 0x99, 0x4b, 0xd0, - 0x79, 0x2a, 0x37, 0xff, 0x75, 0x52, 0x4f, 0xc3, 0x3c, 0x63, 0xaf, 0x67, 0x99, 0xa5, 0x9d, 0x9d, - 0x7a, 0xf5, 0xa8, 0x5f, 0xc8, 0x6d, 0xf9, 0xe0, 0xc1, 0x22, 0xaa, 0x0c, 0x5a, 0xcd, 0x75, 0x4f, - 0x28, 0x41, 0xef, 0x40, 0xc6, 0x70, 0xda, 0x6d, 0x6c, 0xb0, 0x61, 0xef, 0xa8, 0xf5, 0x9c, 0xcc, - 0x16, 0x38, 0x7b, 0xd4, 0x2f, 0xa4, 0x2b, 0x41, 0x09, 0x5d, 0xea, 0xb4, 0x11, 0xfe, 0x44, 0x2a, - 0xcc, 0x86, 0x26, 0x8c, 0xf9, 0x93, 0x24, 0x9b, 0xad, 0x1b, 0x13, 0x53, 0xa1, 0x9a, 0xc1, 0x43, - 0xdf, 0xf9, 0x0a, 0x5c, 0x3a, 0x76, 0x15, 0xcf, 0xa2, 0xb0, 0x64, 0x98, 0xc2, 0x14, 0xc8, 0xf0, - 0x45, 0xf1, 0x37, 0x66, 0xf1, 0xf3, 0x0c, 0x64, 0x54, 0x4c, 0x3c, 0xc7, 0xc5, 0xfe, 0x8e, 0xfa, - 0x5c, 0x82, 0x39, 0x1a, 0x6f, 0xb8, 0x56, 0xd7, 0x73, 0x5c, 0xcd, 0xc5, 0x8f, 0x5c, 0xcb, 0xc3, - 0x24, 0x17, 0x61, 0x46, 0xb7, 0x74, 0xca, 0x10, 0x86, 0x81, 0x4a, 0xd5, 0x00, 0x44, 0x15, 0x18, - 0xdc, 0xee, 0xee, 0xfc, 0xe8, 0x17, 0x85, 0xdb, 0x13, 0xad, 0xe3, 0x78, 0x08, 0x54, 0xaa, 0x57, - 0x55, 0x64, 0x8e, 0x01, 0xa3, 0x17, 0x20, 0x46, 0xed, 0x96, 0xb9, 0x9c, 0x64, 0x59, 0x3e, 0xea, - 0x17, 0x62, 0xd4, 0xb2, 0x55, 0x26, 0x1d, 0xda, 0xe0, 0xb1, 0xe7, 0xd8, 0xe0, 0x2b, 0x90, 0xf2, - 0xf4, 0x56, 0x1b, 0x6b, 0xb4, 0x65, 0x22, 0xb6, 0xdf, 0xab, 0x23, 0x33, 0x41, 0x1e, 0xb4, 0x5b, - 0x3a, 0xc1, 0xa5, 0x6d, 0xaa, 0x19, 0x1a, 0x3b, 0x78, 0xbe, 0x80, 0xa0, 0x45, 0x48, 0x51, 0x2e, - 0x73, 0x2d, 0x13, 0x6b, 0x66, 0x8b, 0xed, 0xa1, 0x64, 0x39, 0x73, 0xd4, 0x2f, 0xc0, 0xa6, 0x10, - 0x57, 0xcb, 0x2a, 0xf8, 0x2a, 0xd5, 0x16, 0xf2, 0x60, 0x5e, 0x90, 0x46, 0xb0, 0xff, 0x99, 0x3d, - 0x25, 0x58, 0x17, 0xde, 0x9b, 0x7c, 0x31, 0xf8, 0xba, 0xfb, 0xc6, 0xc3, 0x22, 0x14, 0x3e, 0x48, - 0xd4, 0x1a, 0x2b, 0x41, 0xaf, 0x41, 0xb6, 0xeb, 0xe2, 0xae, 0xee, 0x62, 0xcd, 0x70, 0x3a, 0xdd, - 0x36, 0xf6, 0xb0, 0xc9, 0xac, 0x5f, 0x56, 0x15, 0x51, 0x50, 0xf1, 0xe5, 0xe8, 0x15, 0xea, 0xdd, - 0x74, 0x8f, 0x06, 0x4e, 0x04, 0xbb, 0x54, 0x33, 0xc9, 0x34, 0xd3, 0x4c, 0x5a, 0x17, 0x42, 0xf4, - 0x16, 0x5c, 0x1a, 0xac, 0x1b, 0xd1, 0xba, 0xbd, 0x56, 0xdb, 0x22, 0xfb, 0xd8, 0xcc, 0x01, 0xd3, - 0x9e, 0x0f, 0x15, 0x6e, 0xf9, 0x65, 0xe8, 0x70, 0xc8, 0x14, 0x0d, 0x3a, 0x31, 0xfa, 0x1e, 0xce, - 0xa5, 0x16, 0xa4, 0xeb, 0xd3, 0xe5, 0xd5, 0x67, 0xfd, 0x42, 0x75, 0x62, 0x3b, 0x22, 0xb8, 0xb3, - 0xe8, 0xb9, 0x18, 0x87, 0xcc, 0xb2, 0x22, 0xf0, 0xc2, 0x16, 0xe5, 0xcb, 0x90, 0x0a, 0x30, 0xd8, - 0x82, 0xb9, 0x99, 0xe7, 0x66, 0xbb, 0x10, 0x0a, 0x5a, 0x82, 0x04, 0x0f, 0xe1, 0x49, 0x2e, 0xcd, - 0x16, 0xf0, 0xdb, 0x27, 0xd9, 0x10, 0xd3, 0x0a, 0xad, 0x92, 0x5f, 0x0f, 0x55, 0x01, 0xbc, 0xc3, - 0xae, 0x6f, 0x89, 0x19, 0x86, 0xf2, 0xca, 0x49, 0x28, 0x87, 0xdd, 0xb0, 0x21, 0x26, 0x3d, 0xf1, - 0x4d, 0x50, 0x03, 0x66, 0x78, 0x7e, 0x20, 0x70, 0x66, 0x19, 0xce, 0xb5, 0x13, 0x70, 0x58, 0xd8, - 0xa3, 0x87, 0x90, 0x52, 0x24, 0x90, 0x10, 0xb4, 0x05, 0x19, 0x1a, 0xab, 0x52, 0x4d, 0x81, 0xa6, - 0x30, 0xb4, 0x1b, 0x27, 0xa0, 0x55, 0x85, 0x72, 0x08, 0x2f, 0x6d, 0x86, 0x64, 0x24, 0xff, 0xdf, - 0x12, 0x64, 0xc7, 0xc8, 0x03, 0x6d, 0x43, 0x24, 0x48, 0x3f, 0x28, 0xa7, 0x47, 0x58, 0xea, 0x71, - 0x11, 0x22, 0x89, 0x58, 0x26, 0xda, 0x83, 0x24, 0x35, 0x67, 0xdb, 0xa3, 0xb9, 0x4d, 0x84, 0x81, - 0x37, 0x8e, 0xfa, 0x05, 0x79, 0x8b, 0x09, 0x2f, 0xdc, 0x84, 0xcc, 0xc1, 0xeb, 0x26, 0x2a, 0x40, - 0xca, 0x73, 0x34, 0xfc, 0xd8, 0x22, 0x9e, 0x65, 0xef, 0xb1, 0x60, 0x41, 0x56, 0xc1, 0x73, 0x6a, - 0x42, 0x92, 0xff, 0xd3, 0x08, 0xa0, 0xf1, 0x5d, 0x8a, 0xfe, 0x5e, 0x82, 0x17, 0xfc, 0x18, 0xc0, - 0x71, 0xad, 0x3d, 0xcb, 0xd6, 0xdb, 0x43, 0xc1, 0x80, 0xc4, 0x66, 0xfb, 0xd3, 0x8b, 0x50, 0x81, - 0x08, 0x10, 0x36, 0x05, 0xfc, 0x68, 0xa0, 0xf0, 0x02, 0xf5, 0xa0, 0x3c, 0x50, 0x18, 0x53, 0xb9, - 0xaf, 0xe6, 0x7a, 0x27, 0x54, 0xce, 0xaf, 0xc1, 0x8b, 0xa7, 0x02, 0x9f, 0xc7, 0x77, 0xe5, 0x7f, - 0x24, 0xc1, 0x95, 0x13, 0x3c, 0x4a, 0x18, 0x27, 0xcd, 0x71, 0xee, 0x85, 0x71, 0x52, 0xb7, 0x7e, - 0xed, 0x02, 0x5e, 0x2b, 0xd4, 0x89, 0x46, 0x4c, 0x96, 0x94, 0x48, 0xf1, 0x4d, 0x98, 0x15, 0x95, - 0x82, 0x00, 0xf7, 0x45, 0x80, 0x7d, 0x6b, 0x6f, 0x5f, 0x7b, 0xa4, 0x7b, 0xd8, 0x15, 0x39, 0x69, - 0x92, 0x4a, 0x3e, 0xa2, 0x82, 0xe2, 0xbf, 0xc8, 0x90, 0xae, 0x77, 0xba, 0x8e, 0xeb, 0xf9, 0x5e, - 0xf6, 0x2e, 0xc4, 0x99, 0x5f, 0x20, 0x62, 0xfd, 0x4a, 0xa7, 0xf4, 0x70, 0xa8, 0x26, 0xf7, 0x2f, - 0x82, 0x16, 0x04, 0x46, 0xe0, 0xfe, 0x22, 0xc7, 0xba, 0xbf, 0xf7, 0x21, 0xce, 0x0f, 0x20, 0x44, - 0x60, 0x5f, 0x38, 0x26, 0x23, 0xa8, 0x6f, 0x2e, 0x5b, 0x6d, 0xbc, 0xcc, 0xd4, 0x7c, 0x70, 0x5e, - 0x09, 0xbd, 0x0a, 0x32, 0x21, 0x9e, 0x46, 0xac, 0x1f, 0x72, 0xef, 0x19, 0xe5, 0xb9, 0x6d, 0xb3, - 0xb9, 0xdd, 0xb4, 0x7e, 0x88, 0xd5, 0x04, 0x21, 0x1e, 0xfd, 0x81, 0xf2, 0x20, 0x3f, 0xd2, 0xdb, - 0x6d, 0xe6, 0x65, 0xa7, 0x59, 0x2e, 0x1f, 0x7c, 0x0f, 0x6f, 0xb3, 0xf8, 0x37, 0xbb, 0xcd, 0x84, - 0xc3, 0xec, 0xea, 0xde, 0x3e, 0x8b, 0x1c, 0x93, 0x2a, 0x70, 0xd1, 0x96, 0xee, 0xed, 0xa3, 0x1c, - 0x24, 0x88, 0x4e, 0x7d, 0x17, 0xc9, 0xc9, 0x0b, 0xd1, 0xeb, 0x33, 0xaa, 0xff, 0x89, 0xae, 0x02, - 0xf3, 0xbc, 0xfc, 0x93, 0x39, 0xb1, 0xa8, 0x1a, 0x92, 0xb0, 0x79, 0x38, 0xb0, 0xba, 0xda, 0xee, - 0x01, 0xe1, 0x4e, 0x4b, 0xcc, 0xc3, 0x81, 0xd5, 0x5d, 0x5e, 0x23, 0x6a, 0x82, 0x16, 0x2e, 0x1f, - 0x10, 0x74, 0x0d, 0x66, 0x2d, 0x96, 0x01, 0x69, 0xa6, 0xe5, 0x62, 0xc3, 0x6b, 0x1f, 0x32, 0x87, - 0x25, 0xab, 0x19, 0x2e, 0xae, 0x0a, 0x29, 0xba, 0x01, 0xca, 0xa8, 0x9b, 0x65, 0x8e, 0x46, 0x56, - 0x67, 0x47, 0xbc, 0x2c, 0x55, 0xe5, 0x4b, 0x1d, 0x72, 0x9c, 0x69, 0xae, 0xca, 0xe5, 0x03, 0x9f, - 0x59, 0x82, 0xb9, 0xae, 0xee, 0x12, 0xac, 0xb5, 0x7a, 0xb6, 0xd9, 0xc6, 0x1a, 0xe7, 0xea, 0x5c, - 0x86, 0x69, 0x67, 0x59, 0x51, 0x99, 0x95, 0x70, 0x5a, 0x3f, 0x2b, 0xf6, 0xbe, 0xfc, 0x7f, 0x10, - 0x7b, 0xe7, 0x7f, 0x16, 0x81, 0x69, 0x66, 0xe7, 0xe8, 0x36, 0xc4, 0xe8, 0x32, 0x8b, 0xcc, 0x6e, - 0xd2, 0x98, 0x8b, 0xd5, 0x41, 0x08, 0x62, 0xb6, 0xde, 0xc1, 0x39, 0xc4, 0x8c, 0x80, 0xfd, 0x46, - 0x57, 0x20, 0x41, 0xf0, 0x03, 0xed, 0xa1, 0xde, 0xce, 0xcd, 0xb1, 0x15, 0x8e, 0x13, 0xfc, 0xe0, - 0xbe, 0xde, 0x46, 0x97, 0x20, 0x6e, 0x11, 0xcd, 0xc6, 0x8f, 0x72, 0xf3, 0x6c, 0xa6, 0xa6, 0x2d, - 0xb2, 0x81, 0x1f, 0x31, 0xda, 0xd6, 0xdd, 0x3d, 0xec, 0x69, 0x86, 0xd3, 0x26, 0xb9, 0x4b, 0x74, - 0x83, 0xd1, 0x90, 0x8e, 0x8a, 0x2a, 0x4e, 0x9b, 0xa0, 0xff, 0x07, 0xc9, 0x47, 0x3a, 0xd1, 0x70, - 0xa7, 0xeb, 0x1d, 0xb2, 0xc9, 0x92, 0xa9, 0xd9, 0x93, 0x1a, 0xfd, 0x6e, 0xc4, 0xe4, 0x88, 0x12, - 0x6d, 0xc4, 0xe4, 0xa8, 0x12, 0x6b, 0xc4, 0xe4, 0x98, 0x32, 0xdd, 0x88, 0xc9, 0xd3, 0x4a, 0xbc, - 0x11, 0x93, 0xe3, 0x4a, 0xa2, 0x11, 0x93, 0x13, 0x8a, 0xdc, 0x88, 0xc9, 0xb2, 0x92, 0x6c, 0xc4, - 0xe4, 0xa4, 0x02, 0x8d, 0x98, 0x0c, 0x4a, 0xaa, 0x11, 0x93, 0x53, 0xca, 0x4c, 0x23, 0x26, 0xcf, - 0x28, 0xe9, 0x46, 0x4c, 0x4e, 0x2b, 0x99, 0x46, 0x4c, 0xce, 0x28, 0xb3, 0x8d, 0x98, 0x3c, 0xab, - 0x28, 0x8d, 0x98, 0xac, 0x28, 0xd9, 0x46, 0x4c, 0xce, 0x2a, 0xa8, 0xf8, 0xb9, 0x04, 0x4a, 0x13, - 0x3f, 0xe8, 0x61, 0xdb, 0xc0, 0xf7, 0xf5, 0x76, 0x65, 0xbf, 0x67, 0x1f, 0xa0, 0x57, 0x61, 0xd6, - 0xa0, 0x3f, 0x34, 0x9e, 0x18, 0xd3, 0xa1, 0x4a, 0x6c, 0xa8, 0x69, 0x26, 0x6e, 0x52, 0x29, 0x1d, - 0xf1, 0x8b, 0x00, 0x42, 0x8f, 0xee, 0x6c, 0x7e, 0xfa, 0x96, 0xe4, 0x2a, 0x74, 0x3b, 0x8f, 0xc0, - 0xb8, 0xce, 0x23, 0x46, 0x1f, 0x43, 0x30, 0xaa, 0xf3, 0x08, 0x2d, 0xc2, 0xbc, 0x8d, 0x1f, 0x7b, - 0xda, 0xa8, 0x32, 0xa3, 0x0a, 0x35, 0x4b, 0xcb, 0x2a, 0xe1, 0x0a, 0xc5, 0x7f, 0x8e, 0xc0, 0xac, - 0xdf, 0x69, 0x9f, 0x0e, 0x77, 0x41, 0xa1, 0xcb, 0x62, 0x99, 0x9a, 0xe7, 0x70, 0x24, 0x9f, 0x18, - 0xdf, 0x3f, 0x85, 0x18, 0x47, 0x50, 0xe8, 0x77, 0xdd, 0xdc, 0x76, 0x58, 0x73, 0xdc, 0x35, 0xa8, - 0x69, 0x12, 0x96, 0xe5, 0x77, 0x20, 0xe3, 0x57, 0xe2, 0x12, 0x54, 0x81, 0xf8, 0x50, 0x7b, 0xaf, - 0x4d, 0xd0, 0x9e, 0x3f, 0xd5, 0xaa, 0xa8, 0x9a, 0xff, 0x2d, 0x40, 0xe3, 0x6d, 0x87, 0xdd, 0xd2, - 0x34, 0x77, 0x4b, 0x9b, 0xc3, 0x6e, 0xe9, 0xdd, 0xf3, 0x8d, 0x2d, 0xd4, 0xed, 0x70, 0x56, 0xf7, - 0x8f, 0x11, 0xc8, 0x70, 0x17, 0x11, 0xb8, 0xa3, 0xd7, 0x20, 0xcb, 0x48, 0xcb, 0xb2, 0xf7, 0xb4, - 0xae, 0x10, 0xb2, 0xf1, 0x45, 0x54, 0xc5, 0x2f, 0x08, 0x94, 0x5f, 0x82, 0xb4, 0x8b, 0x75, 0x73, - 0xa0, 0x18, 0x61, 0x8a, 0x33, 0x54, 0x18, 0x28, 0xbd, 0x02, 0x19, 0xe6, 0x0d, 0x07, 0x5a, 0x51, - 0xa6, 0x95, 0x66, 0xd2, 0x40, 0xad, 0x0c, 0x69, 0xd2, 0xd5, 0xed, 0x81, 0x56, 0x8c, 0x4d, 0xea, - 0x19, 0x67, 0x50, 0x33, 0xb4, 0x4e, 0xd8, 0x97, 0xba, 0x98, 0xf4, 0x3a, 0x58, 0xeb, 0x3a, 0x3c, - 0xd9, 0x8a, 0xaa, 0x49, 0x2e, 0xd9, 0x72, 0x08, 0xda, 0x61, 0xa6, 0xc2, 0xe6, 0x42, 0x33, 0xf9, - 0xe4, 0xe4, 0xe2, 0xac, 0x95, 0x9b, 0x93, 0x4f, 0xa7, 0x3a, 0x4b, 0x86, 0x05, 0xc5, 0xbf, 0x96, - 0xe0, 0x0a, 0x0d, 0x98, 0x39, 0x2b, 0x56, 0xd8, 0xb9, 0xb9, 0x6f, 0x9d, 0x3a, 0x24, 0x58, 0xd0, - 0x1d, 0xc4, 0x9f, 0xab, 0x47, 0xfd, 0x42, 0x9c, 0x6a, 0x5f, 0xd8, 0x73, 0xc5, 0x29, 0x70, 0x9d, - 0xa5, 0x47, 0x9e, 0xab, 0xdb, 0xc4, 0xa2, 0x89, 0x02, 0x5d, 0xb6, 0x0e, 0xee, 0xb4, 0xb0, 0xcb, - 0x17, 0x63, 0x46, 0x9d, 0x1f, 0x2a, 0x5c, 0xe7, 0x65, 0xc5, 0x3c, 0xe4, 0x46, 0xbb, 0x1c, 0x64, - 0xf6, 0xbf, 0x0e, 0x97, 0x37, 0xf0, 0xa3, 0xe3, 0x46, 0x53, 0x86, 0x04, 0xe7, 0x2f, 0xdf, 0xe4, - 0xaf, 0x8f, 0xb2, 0x6a, 0xf8, 0xea, 0xa0, 0xc4, 0x7a, 0xba, 0xcd, 0x2a, 0xa8, 0x7e, 0xc5, 0xe2, - 0xa7, 0x70, 0x65, 0x04, 0x3d, 0x58, 0xbe, 0x0f, 0x20, 0x4e, 0x33, 0x3f, 0x11, 0xd9, 0x64, 0xc6, - 0xb3, 0x8a, 0x71, 0xf4, 0x26, 0xd5, 0x57, 0x45, 0xb5, 0xa2, 0xca, 0x8e, 0x24, 0x7a, 0x1d, 0x4c, - 0x2d, 0xe4, 0xae, 0x45, 0x3c, 0xf4, 0x21, 0xcc, 0x08, 0x8b, 0xa0, 0x86, 0xe2, 0x77, 0xfb, 0x0c, - 0xa3, 0x4a, 0xb9, 0x01, 0x08, 0x29, 0xfe, 0x8d, 0x04, 0x73, 0x55, 0xd7, 0xe9, 0x76, 0xb1, 0x29, - 0x7c, 0x05, 0x9f, 0x0b, 0xdf, 0x45, 0x48, 0x21, 0x17, 0xb1, 0x01, 0x91, 0x7a, 0x55, 0xe4, 0x02, - 0x77, 0x2e, 0x9a, 0x62, 0xd4, 0xab, 0xe8, 0x5d, 0x3e, 0x21, 0x3d, 0xc2, 0xf8, 0x33, 0x33, 0x96, - 0xf4, 0x0d, 0x99, 0x29, 0x53, 0x54, 0x45, 0x85, 0xe2, 0x4f, 0x13, 0x70, 0x29, 0x3c, 0xc9, 0x2b, - 0x15, 0xbf, 0xe3, 0x9f, 0x41, 0xc2, 0xb2, 0x4d, 0xfc, 0x18, 0x4f, 0xc4, 0x93, 0xc7, 0x41, 0x94, - 0xc4, 0x7c, 0xd4, 0x29, 0x8c, 0x9f, 0x66, 0x0a, 0x4c, 0xf4, 0x71, 0x10, 0x9e, 0xf2, 0x63, 0x9f, - 0xdb, 0xcf, 0x8d, 0x5e, 0x1d, 0x09, 0x55, 0x87, 0x22, 0x41, 0xe6, 0x50, 0xbe, 0xa1, 0x48, 0xb0, - 0x09, 0x59, 0xcb, 0xf6, 0xb0, 0xdb, 0xc6, 0xfa, 0x43, 0x1a, 0xd8, 0xd0, 0xe6, 0xc5, 0xe9, 0xcf, - 0xa4, 0x61, 0x84, 0x12, 0x02, 0xe0, 0xe1, 0xc8, 0x67, 0x30, 0x17, 0x06, 0xf5, 0x97, 0xe0, 0xf4, - 0x13, 0x21, 0x36, 0xc3, 0x03, 0x58, 0xff, 0xe0, 0x25, 0x04, 0x54, 0x17, 0xd3, 0x7e, 0x1f, 0xe2, - 0x3c, 0xd1, 0x17, 0xc7, 0xab, 0x77, 0x9e, 0x77, 0xda, 0xf9, 0x01, 0x82, 0x2a, 0xd0, 0xf2, 0x7f, - 0x2c, 0xc1, 0x4c, 0x78, 0xb9, 0x91, 0x05, 0x32, 0xeb, 0xbb, 0x4f, 0x69, 0xd1, 0xf2, 0x06, 0x8d, - 0x65, 0x59, 0x21, 0x5b, 0x83, 0x0f, 0x9e, 0x7b, 0x0d, 0x38, 0x84, 0x30, 0xa5, 0xba, 0x49, 0x03, - 0x24, 0xd3, 0x75, 0xba, 0x83, 0xe3, 0xf5, 0xa8, 0x2a, 0x53, 0x01, 0x8d, 0xf9, 0xf2, 0xbf, 0x0d, - 0xc9, 0xc0, 0x50, 0x42, 0x19, 0x7e, 0xf4, 0x6b, 0xcc, 0xf0, 0x4f, 0x6d, 0xbf, 0x0a, 0xe9, 0xa1, - 0x19, 0x43, 0x97, 0x83, 0x3e, 0xc4, 0xca, 0x71, 0xde, 0x87, 0x33, 0x51, 0x8a, 0xbf, 0x88, 0xc3, - 0xdc, 0x71, 0x4c, 0xfb, 0x09, 0x28, 0x21, 0xde, 0xd2, 0xda, 0x16, 0xf1, 0xc4, 0x7e, 0xba, 0x71, - 0x7a, 0x42, 0x1a, 0x22, 0x3f, 0x61, 0x2d, 0x19, 0x77, 0x98, 0x12, 0xbf, 0x0f, 0x19, 0x93, 0x77, - 0x5c, 0x13, 0x1b, 0x35, 0x7a, 0x66, 0x1e, 0x79, 0x0c, 0x01, 0x0a, 0xf4, 0xb4, 0x19, 0x2a, 0x22, - 0xec, 0xde, 0xc1, 0x47, 0x0f, 0xce, 0x76, 0x2c, 0x93, 0xed, 0x9e, 0x74, 0xb9, 0x79, 0xd4, 0x2f, - 0x64, 0x05, 0x96, 0x7f, 0x98, 0x73, 0xe1, 0x95, 0xca, 0x9a, 0x23, 0x80, 0x26, 0xf5, 0xba, 0xb4, - 0x9c, 0x36, 0x3c, 0x3d, 0xf0, 0xba, 0x74, 0x1f, 0x5d, 0xdc, 0xeb, 0xd2, 0x9f, 0x75, 0x13, 0xfd, - 0xbe, 0x04, 0x59, 0x7e, 0xb2, 0xdb, 0xe9, 0x79, 0x3a, 0x3f, 0xae, 0xf7, 0xf3, 0xd3, 0x4f, 0x8e, - 0xfa, 0x85, 0x59, 0x36, 0x21, 0xeb, 0xa2, 0x8c, 0x35, 0x5b, 0x7e, 0xde, 0x66, 0x07, 0x28, 0x22, - 0x67, 0x0b, 0x04, 0x26, 0x5a, 0x83, 0x0c, 0x4f, 0xb6, 0xfd, 0x0b, 0x78, 0x96, 0xb8, 0xa6, 0xcb, - 0x2f, 0x3f, 0xeb, 0x17, 0x16, 0x8e, 0xb1, 0x2c, 0x9e, 0xa7, 0xdf, 0xe7, 0xba, 0x6a, 0x7a, 0x37, - 0xfc, 0x89, 0x0c, 0x48, 0x07, 0xa6, 0x71, 0xd8, 0x15, 0x79, 0xee, 0xc5, 0x5d, 0xd9, 0x8c, 0x6f, - 0x23, 0x14, 0x13, 0xed, 0xc1, 0xac, 0xdf, 0x08, 0x77, 0xe8, 0x24, 0x97, 0xfc, 0x5a, 0x9a, 0xf1, - 0xcd, 0x9a, 0x8f, 0x9a, 0x88, 0x23, 0x97, 0xcb, 0x30, 0x7f, 0x6c, 0x94, 0xf3, 0x65, 0x1c, 0x2e, - 0x0f, 0x13, 0x61, 0x10, 0x87, 0x68, 0xa3, 0x1e, 0xf2, 0x83, 0x89, 0xc9, 0xd4, 0xc7, 0xe0, 0x64, - 0xe6, 0x7f, 0x8d, 0xfa, 0xc8, 0xcf, 0x46, 0x7c, 0xe4, 0x73, 0xe0, 0x33, 0xf3, 0x1a, 0xc1, 0xf7, - 0x1d, 0xe5, 0xc7, 0x81, 0x2f, 0xe0, 0xa7, 0x36, 0x1f, 0x3e, 0x07, 0x3c, 0xab, 0xef, 0x7f, 0x06, - 0xde, 0xe0, 0x9f, 0x24, 0x48, 0x0f, 0x8d, 0xec, 0x57, 0xe9, 0x0e, 0xb6, 0x82, 0x68, 0x88, 0x3f, - 0xd3, 0x78, 0xe7, 0xfc, 0xc3, 0x1a, 0x0e, 0x92, 0xf2, 0x7f, 0x27, 0x41, 0x7a, 0x68, 0x22, 0xbf, - 0x21, 0x47, 0xf2, 0xf5, 0xf7, 0xbc, 0x05, 0x99, 0xe1, 0x25, 0x0a, 0xb5, 0x21, 0x7d, 0x3d, 0x6d, - 0x14, 0xbf, 0x07, 0x71, 0x2e, 0x41, 0x08, 0x32, 0x1f, 0x2d, 0xd5, 0xb7, 0xeb, 0x1b, 0x2b, 0xda, - 0xf2, 0xa6, 0xaa, 0xad, 0x54, 0x94, 0x29, 0x34, 0x03, 0x72, 0xb5, 0x76, 0xb7, 0x46, 0x85, 0x8a, - 0x84, 0x52, 0x90, 0x60, 0x5f, 0xb5, 0xaa, 0x12, 0x29, 0x96, 0x41, 0xe1, 0xd8, 0xbb, 0x98, 0x3a, - 0x06, 0x1a, 0xf7, 0xa3, 0x12, 0xcc, 0xb1, 0x20, 0xbd, 0x43, 0xe3, 0x37, 0xea, 0x0a, 0xb5, 0x50, - 0xf4, 0x9c, 0x0d, 0x8a, 0xa8, 0x53, 0xdc, 0xd0, 0x3b, 0xb8, 0xf8, 0xb7, 0x31, 0xc8, 0x0e, 0x40, - 0x7c, 0xb7, 0xf8, 0x2a, 0xc8, 0xc4, 0xb2, 0x0f, 0xb4, 0xc1, 0xa5, 0x39, 0x3f, 0x48, 0xb3, 0xec, - 0x83, 0x1d, 0xb5, 0xae, 0x26, 0x68, 0xe1, 0x8e, 0x6b, 0xa1, 0x06, 0xc4, 0x9c, 0xae, 0xe7, 0xe7, - 0x90, 0x6f, 0x9f, 0x32, 0x15, 0x63, 0x6d, 0x94, 0x36, 0xbb, 0x9e, 0x38, 0x01, 0x60, 0x18, 0xe8, - 0x2f, 0xa5, 0x41, 0xd6, 0xc3, 0xb3, 0xc5, 0x77, 0xcf, 0x85, 0xc7, 0x27, 0x40, 0xdc, 0x60, 0x7e, - 0x44, 0x37, 0xea, 0xb3, 0x7e, 0x21, 0x3b, 0x3a, 0x41, 0xe4, 0x82, 0x57, 0x9b, 0x7e, 0x17, 0x51, - 0x83, 0x5f, 0xaa, 0x0d, 0x26, 0x9a, 0x39, 0x84, 0x09, 0xef, 0x2d, 0xd3, 0x43, 0x0b, 0x91, 0xdf, - 0x83, 0x99, 0x70, 0xef, 0x8f, 0x39, 0x2d, 0x5f, 0x1a, 0x3e, 0x96, 0x78, 0x6d, 0xa2, 0x99, 0x11, - 0x29, 0x61, 0xe8, 0x88, 0xfe, 0x7b, 0x90, 0x0c, 0xa6, 0xfd, 0x3c, 0x67, 0xfb, 0x9c, 0xe3, 0x83, - 0xc3, 0xb2, 0x69, 0x25, 0x5e, 0xfc, 0x2b, 0x09, 0x66, 0x54, 0x4c, 0x9c, 0xf6, 0x43, 0x6c, 0xd2, - 0x98, 0x27, 0x78, 0xd3, 0x22, 0x4d, 0xfe, 0xa6, 0x65, 0x09, 0x92, 0xc1, 0x69, 0xe6, 0x79, 0xde, - 0x75, 0x0c, 0x6a, 0xa1, 0x1b, 0xa0, 0xb4, 0x9c, 0x9e, 0x6d, 0xea, 0xee, 0xa1, 0xe6, 0x62, 0xdd, - 0xd8, 0xc7, 0xa6, 0xb8, 0xb9, 0x99, 0xf5, 0xe5, 0x2a, 0x17, 0x17, 0x31, 0xa4, 0xc3, 0x1d, 0xa6, - 0x24, 0x44, 0x63, 0x36, 0x26, 0x18, 0x4a, 0x5b, 0xaf, 0x9d, 0x1e, 0xfa, 0x05, 0x08, 0xfe, 0x62, - 0xba, 0x61, 0xd4, 0xe2, 0x8f, 0x23, 0x80, 0x06, 0x6b, 0x10, 0x62, 0xbc, 0xd1, 0xc6, 0x22, 0x17, - 0x6f, 0x0c, 0xfd, 0xe4, 0xf4, 0xa3, 0xe1, 0x28, 0x3b, 0x1a, 0x66, 0x9b, 0xe1, 0x57, 0x7a, 0x3c, - 0x2c, 0x62, 0x82, 0xff, 0x8a, 0x01, 0xaa, 0xb8, 0x58, 0xf7, 0x30, 0x25, 0x38, 0x72, 0x5a, 0x4a, - 0x5f, 0x86, 0x69, 0x9e, 0xff, 0x45, 0xce, 0x93, 0xff, 0x89, 0x49, 0xe1, 0x55, 0xd1, 0x0f, 0x60, - 0xc6, 0x70, 0xda, 0xbd, 0x8e, 0xad, 0xb1, 0x8b, 0x6d, 0x11, 0x6f, 0x7f, 0xf7, 0xb4, 0xbd, 0x32, - 0xd6, 0xb9, 0x52, 0xc5, 0x69, 0xd3, 0xef, 0xe0, 0x6d, 0x13, 0x03, 0x64, 0x1a, 0xe8, 0x05, 0x48, - 0x06, 0xfb, 0x96, 0x45, 0xda, 0x49, 0x75, 0x20, 0x40, 0xb7, 0x60, 0x5a, 0x27, 0x9a, 0xb3, 0xcb, - 0x42, 0xe1, 0xb3, 0x0c, 0x59, 0x8d, 0xe9, 0x64, 0x73, 0x17, 0xbd, 0x05, 0xe9, 0xdd, 0x07, 0x3c, - 0x3d, 0xe0, 0x3c, 0xcd, 0xdf, 0x1b, 0xcc, 0x1e, 0xf5, 0x0b, 0xa9, 0xe5, 0x7b, 0x6c, 0xb0, 0x94, - 0xa5, 0xd5, 0xd4, 0xee, 0x83, 0xe0, 0x03, 0xdd, 0x84, 0x6c, 0x47, 0x7f, 0xac, 0xed, 0xba, 0xba, - 0x21, 0xe2, 0xe1, 0x36, 0x27, 0x1f, 0x49, 0x9d, 0xed, 0xe8, 0x8f, 0x97, 0x85, 0xbc, 0x6e, 0xb6, - 0x71, 0xfe, 0x3f, 0x25, 0x48, 0x88, 0x11, 0xa1, 0x2e, 0x80, 0x98, 0x1e, 0xcb, 0xe4, 0xa6, 0x9e, - 0x2e, 0xdf, 0x3b, 0xea, 0x17, 0x92, 0x15, 0x26, 0xad, 0x57, 0xc9, 0xb3, 0x7e, 0xe1, 0xc3, 0xe7, - 0xe5, 0x46, 0x1f, 0x44, 0x4d, 0xf2, 0x46, 0xea, 0x26, 0x3b, 0xb7, 0xdc, 0xd7, 0x89, 0xb6, 0x6f, - 0x11, 0xcf, 0xd9, 0x73, 0xf5, 0x0e, 0x5b, 0x5c, 0x59, 0x9d, 0xd9, 0xd7, 0xc9, 0xaa, 0x2f, 0x43, - 0x79, 0x1a, 0xd9, 0x3c, 0xe4, 0xef, 0x12, 0xf8, 0xce, 0x0d, 0xbe, 0xd1, 0x2d, 0xb8, 0x14, 0x54, - 0xd6, 0xe8, 0xa0, 0x5b, 0x3d, 0xe3, 0x00, 0x33, 0x87, 0x43, 0xa9, 0x71, 0x2e, 0x28, 0x5c, 0xd7, - 0x1f, 0x97, 0x79, 0x51, 0xf1, 0x12, 0xcc, 0x85, 0x96, 0x35, 0x88, 0x43, 0x31, 0x28, 0xeb, 0xd6, - 0x9e, 0xab, 0x87, 0x9f, 0xfd, 0xdd, 0x83, 0xd9, 0x91, 0x67, 0xb5, 0x82, 0xbd, 0xc2, 0xe7, 0x6d, - 0xc3, 0xef, 0x70, 0x4b, 0x15, 0xfe, 0xe9, 0x47, 0xf6, 0x19, 0x63, 0xe8, 0xbb, 0x38, 0x07, 0xd9, - 0xa0, 0x99, 0xa0, 0xed, 0x5f, 0xce, 0x40, 0x62, 0x4b, 0x3f, 0x6c, 0x3b, 0xba, 0x89, 0x16, 0x20, - 0xe5, 0xbf, 0x65, 0xf0, 0xdb, 0x4b, 0xaa, 0x61, 0x11, 0xb2, 0x20, 0xd3, 0x23, 0xd8, 0xa5, 0xf6, - 0xa0, 0xb1, 0x57, 0xbe, 0x9c, 0x8e, 0xcb, 0xe5, 0x67, 0xfd, 0xc2, 0x9d, 0xc9, 0x96, 0x07, 0x1b, - 0x3d, 0xd7, 0xf2, 0x0e, 0x4b, 0xcd, 0x7b, 0x77, 0x77, 0x04, 0x14, 0xdd, 0xc4, 0x8e, 0x9a, 0xee, - 0x85, 0x3f, 0xc5, 0xcb, 0x10, 0x3a, 0xd5, 0x5a, 0xc7, 0x32, 0x5c, 0x87, 0xf8, 0x17, 0x08, 0x42, - 0xba, 0xce, 0x84, 0xe8, 0x1a, 0xcc, 0xee, 0x5a, 0x36, 0xbb, 0xbc, 0xf2, 0xf5, 0xf8, 0xdd, 0x41, - 0xc6, 0x17, 0x0b, 0xc5, 0x87, 0x90, 0x09, 0xbd, 0x06, 0xa1, 0x66, 0x16, 0x67, 0x66, 0xb6, 0x79, - 0xd4, 0x2f, 0xa4, 0x07, 0xdb, 0x96, 0x9b, 0xda, 0x45, 0xdc, 0x70, 0x7a, 0xd0, 0x0c, 0x35, 0xb4, - 0x79, 0x98, 0x66, 0x6f, 0xc0, 0xf9, 0x03, 0x30, 0x95, 0x7f, 0xa0, 0xb7, 0x61, 0xba, 0x8d, 0x75, - 0x82, 0xc5, 0xdb, 0xae, 0x85, 0x53, 0x88, 0x80, 0x3d, 0xa2, 0x56, 0xb9, 0x3a, 0x2a, 0x43, 0x9c, - 0x5f, 0x47, 0xb2, 0x4b, 0xc4, 0xf1, 0xd3, 0xd7, 0x13, 0x9f, 0xf1, 0xad, 0x4e, 0xa9, 0xa2, 0x26, - 0xaa, 0x41, 0xc2, 0xe5, 0x37, 0xd0, 0xec, 0x6a, 0xf1, 0xcc, 0xf3, 0x84, 0xd0, 0x05, 0xf7, 0xea, - 0x94, 0xea, 0xd7, 0x45, 0xdb, 0xfe, 0x33, 0x10, 0xee, 0x51, 0xc4, 0x2b, 0x97, 0xd2, 0x84, 0x31, - 0xe7, 0x00, 0x70, 0x08, 0x85, 0x0e, 0xd0, 0x62, 0xd7, 0x11, 0xec, 0x86, 0xf2, 0xf4, 0x01, 0x0e, - 0x5d, 0x6d, 0xd3, 0x01, 0xf2, 0x9a, 0x68, 0x03, 0xc0, 0x08, 0xbc, 0x1c, 0xbb, 0xbb, 0x4c, 0xdd, - 0x7a, 0xfd, 0x3c, 0x01, 0xdb, 0xea, 0x94, 0x1a, 0x42, 0x40, 0xf7, 0x20, 0x65, 0x0c, 0xb6, 0x6d, - 0x6e, 0x96, 0x01, 0xbe, 0x71, 0x2e, 0xee, 0x5e, 0xa5, 0x7c, 0x3d, 0x90, 0x0e, 0xf3, 0xb5, 0x32, - 0xca, 0xd7, 0x35, 0x48, 0x8b, 0xa3, 0x1f, 0xfe, 0xe7, 0x03, 0xb9, 0x2c, 0x73, 0x17, 0x61, 0x2b, - 0xf1, 0xff, 0xc0, 0xa0, 0x54, 0xb3, 0x0d, 0xc7, 0xc4, 0x66, 0x8d, 0x7e, 0xab, 0xe2, 0xa4, 0x9b, - 0x7d, 0x10, 0xb4, 0x02, 0x19, 0xa3, 0x8d, 0x75, 0xbb, 0xd7, 0xf5, 0x71, 0xd0, 0x84, 0x38, 0x69, - 0x51, 0x4f, 0x00, 0x6d, 0x00, 0xda, 0x65, 0x4f, 0x47, 0xc2, 0xbd, 0x62, 0x57, 0xa0, 0x93, 0x80, - 0x29, 0xac, 0xae, 0x3a, 0xe8, 0x19, 0x7a, 0x19, 0xd2, 0xb6, 0x63, 0x1b, 0xba, 0x6d, 0xe0, 0x36, - 0xf3, 0xac, 0xfc, 0xd6, 0x74, 0x58, 0x88, 0x3e, 0x85, 0x0c, 0x19, 0xca, 0x52, 0x72, 0x97, 0x58, - 0x8b, 0x6f, 0x9e, 0xf7, 0x5c, 0x73, 0x75, 0x4a, 0x1d, 0x41, 0x42, 0xbf, 0x01, 0x8a, 0x37, 0x72, - 0xf9, 0xc1, 0xee, 0x5f, 0x4f, 0x7f, 0xa6, 0x75, 0xc2, 0x15, 0xcf, 0xea, 0x94, 0x3a, 0x86, 0x86, - 0x3e, 0x83, 0x59, 0x32, 0xfc, 0xa0, 0x39, 0x77, 0x85, 0x35, 0xf0, 0x9d, 0x53, 0x4f, 0xf0, 0x8f, - 0x7b, 0x03, 0xbe, 0x3a, 0xa5, 0x8e, 0x62, 0x51, 0x78, 0x7b, 0xf8, 0x0e, 0x25, 0x97, 0x3b, 0x13, - 0xfe, 0xf8, 0x3b, 0x1d, 0x0a, 0x3f, 0x82, 0x85, 0xd6, 0x20, 0xd9, 0xf1, 0x7d, 0x45, 0xee, 0x5b, - 0x67, 0x06, 0xf6, 0xa3, 0xee, 0x6b, 0x75, 0x4a, 0x1d, 0xd4, 0x2f, 0x27, 0x21, 0x21, 0xee, 0xda, - 0x82, 0x1b, 0xec, 0x84, 0x22, 0x17, 0x7f, 0x29, 0x83, 0x1c, 0xc4, 0xa0, 0x8b, 0x80, 0x82, 0xa8, - 0x61, 0xf0, 0x6c, 0x90, 0xba, 0xa0, 0xc8, 0xea, 0x94, 0x9a, 0xf5, 0xcb, 0x06, 0x2f, 0x07, 0xaf, - 0xc1, 0x6c, 0xc7, 0x31, 0xad, 0x5d, 0x6b, 0x40, 0xfc, 0xfc, 0x64, 0x35, 0xe3, 0x8b, 0x05, 0xf1, - 0xdf, 0x19, 0x7a, 0x5d, 0x33, 0xc9, 0xeb, 0x74, 0xda, 0xfb, 0xe0, 0xf9, 0x0d, 0x75, 0x44, 0x6e, - 0xcf, 0x66, 0xd7, 0x6a, 0x22, 0xbb, 0xe6, 0xf1, 0x55, 0x5a, 0x48, 0x45, 0x82, 0x5c, 0x19, 0x61, - 0xe6, 0x1b, 0x67, 0x32, 0xb3, 0x3f, 0xf6, 0x55, 0x29, 0xa0, 0xe6, 0xe5, 0x51, 0x6a, 0xbe, 0x79, - 0x36, 0x35, 0x87, 0x60, 0x02, 0x6e, 0xde, 0x39, 0x96, 0x9b, 0x17, 0x27, 0xdc, 0x38, 0x21, 0xc4, - 0x61, 0x72, 0xae, 0x8c, 0x90, 0xf3, 0x8d, 0x33, 0xc9, 0x39, 0x3c, 0x46, 0xc1, 0xce, 0x9b, 0xc7, - 0xb0, 0xf3, 0x1b, 0x13, 0xb1, 0x73, 0x08, 0x2c, 0x4c, 0xcf, 0xea, 0x71, 0xf4, 0x5c, 0x9a, 0x8c, - 0x9e, 0x43, 0x90, 0x43, 0xfc, 0xfc, 0xfd, 0x31, 0xee, 0x51, 0xce, 0xde, 0xbc, 0xc7, 0x1e, 0xa9, - 0xac, 0x4a, 0x63, 0xe4, 0xa3, 0x1f, 0x43, 0x3e, 0x59, 0x06, 0xff, 0xd6, 0x39, 0xc8, 0x27, 0xd4, - 0xc0, 0x38, 0xfb, 0x7c, 0x0c, 0x33, 0x61, 0xc6, 0x60, 0xaf, 0x58, 0x4e, 0xe7, 0xb6, 0x13, 0xfe, - 0xfa, 0x82, 0xd9, 0x40, 0xa8, 0x08, 0xfd, 0x60, 0x9c, 0x78, 0xe6, 0xce, 0x04, 0x3f, 0xe1, 0xba, - 0x77, 0x55, 0x1a, 0x67, 0x9e, 0xbb, 0x61, 0xe6, 0x99, 0x3f, 0xd3, 0x77, 0x8f, 0x45, 0xb4, 0xab, - 0x52, 0x98, 0x7a, 0x00, 0x64, 0xff, 0x35, 0x41, 0x88, 0x86, 0x8a, 0x7f, 0x24, 0x41, 0xb4, 0xe1, - 0xb4, 0x50, 0x66, 0x70, 0xd6, 0xc7, 0x4e, 0xe9, 0x3e, 0x18, 0xa8, 0x8b, 0x6c, 0xef, 0xa5, 0x53, - 0xda, 0x0e, 0xce, 0x46, 0x83, 0x4a, 0xe8, 0x3d, 0x48, 0x74, 0x79, 0x34, 0x2d, 0x98, 0xa6, 0x78, - 0x5a, 0x7d, 0xae, 0xa9, 0xfa, 0x55, 0x6e, 0xde, 0x08, 0xff, 0x21, 0xd6, 0xba, 0x63, 0x62, 0x94, - 0x01, 0xd8, 0xd2, 0x09, 0xe9, 0xee, 0xbb, 0x3a, 0xc1, 0xca, 0x14, 0x4a, 0x40, 0x74, 0x6d, 0xbd, - 0xa9, 0x48, 0x37, 0x3f, 0x0e, 0x9f, 0xcc, 0x55, 0xd5, 0xa5, 0xfa, 0x46, 0x7d, 0x63, 0x45, 0xdb, - 0x58, 0x5a, 0xaf, 0x35, 0x95, 0x29, 0x94, 0x83, 0xf9, 0x8f, 0x96, 0xea, 0xdb, 0xe2, 0xa8, 0x4e, - 0xab, 0x6f, 0x6c, 0xd7, 0xd4, 0xfb, 0x4b, 0x77, 0x15, 0x09, 0x5d, 0x06, 0xa4, 0x6e, 0x56, 0xd6, - 0x9a, 0xd5, 0xb2, 0x56, 0xd9, 0x5c, 0xdf, 0x5a, 0xaa, 0x6c, 0xd7, 0x37, 0x37, 0x94, 0x08, 0x92, - 0x21, 0x56, 0xdd, 0xdc, 0xa8, 0x29, 0x70, 0xf3, 0x27, 0x31, 0x88, 0x51, 0x5b, 0x43, 0x2f, 0x43, - 0x6a, 0x67, 0xa3, 0xb9, 0x55, 0xab, 0xd4, 0x97, 0xeb, 0xb5, 0xaa, 0x32, 0x95, 0x9f, 0x7b, 0xf2, - 0x74, 0x61, 0x96, 0x16, 0xed, 0xd8, 0xa4, 0x8b, 0x0d, 0x46, 0xb2, 0x28, 0x0f, 0xf1, 0xf2, 0x52, - 0x65, 0x6d, 0x67, 0x4b, 0x91, 0xf2, 0x99, 0x27, 0x4f, 0x17, 0x80, 0x2a, 0x70, 0x82, 0x43, 0x2f, - 0x40, 0x42, 0xad, 0x35, 0xb7, 0x37, 0xd5, 0x9a, 0x12, 0xc9, 0xcf, 0x3e, 0x79, 0xba, 0x90, 0xa2, - 0x85, 0x82, 0xb7, 0xd0, 0x35, 0x48, 0x37, 0x2b, 0xab, 0xb5, 0xf5, 0x25, 0xad, 0xb2, 0xba, 0xb4, - 0xb1, 0x52, 0x53, 0xa2, 0xf9, 0xf9, 0x27, 0x4f, 0x17, 0x94, 0x51, 0x43, 0xa7, 0x4d, 0xd4, 0xd7, - 0xb7, 0x36, 0xd5, 0x6d, 0x25, 0x36, 0x68, 0x82, 0xf3, 0x0b, 0x2a, 0x02, 0xf0, 0xda, 0xcb, 0xb5, - 0x5a, 0x55, 0x99, 0xce, 0xa3, 0x27, 0x4f, 0x17, 0x32, 0xb4, 0x7c, 0x40, 0x1b, 0xe8, 0x15, 0x98, - 0xa9, 0xa8, 0xb5, 0xa5, 0xed, 0x9a, 0xd6, 0xdc, 0x5e, 0xda, 0x6e, 0x2a, 0xf1, 0xc1, 0x48, 0x42, - 0x54, 0x80, 0x4a, 0x90, 0x5d, 0xda, 0xd9, 0xde, 0xd4, 0x86, 0x74, 0x13, 0xf9, 0x2b, 0x4f, 0x9e, - 0x2e, 0xcc, 0x51, 0xdd, 0xa5, 0x9e, 0xe7, 0x84, 0xf5, 0x5f, 0x07, 0x65, 0xa8, 0xff, 0xda, 0x4a, - 0x45, 0x91, 0xf3, 0x97, 0x9f, 0x3c, 0x5d, 0x40, 0xa3, 0x43, 0x58, 0xa9, 0xa0, 0xff, 0x0f, 0x97, - 0xb7, 0x3f, 0xd9, 0xaa, 0x55, 0x6b, 0xcd, 0x8a, 0x36, 0x3c, 0xec, 0x64, 0x3e, 0xf7, 0xe4, 0xe9, - 0xc2, 0x3c, 0xad, 0x33, 0x36, 0xf4, 0x37, 0x40, 0x69, 0x6e, 0xab, 0xb5, 0xa5, 0x75, 0xad, 0xbe, - 0xb1, 0x52, 0x6b, 0xb2, 0xc5, 0x82, 0x41, 0x97, 0x46, 0x36, 0x2d, 0x1d, 0xc2, 0x46, 0xed, 0xa3, - 0x11, 0xfc, 0xd4, 0x40, 0x7f, 0x64, 0x1f, 0xa2, 0x05, 0x48, 0xae, 0xd7, 0x57, 0xd4, 0x25, 0x86, - 0x3b, 0x93, 0xcf, 0x3e, 0x79, 0xba, 0x90, 0xa6, 0x7a, 0xc1, 0xae, 0xca, 0xcb, 0x3f, 0xfe, 0xf3, - 0xab, 0x53, 0x7f, 0xf1, 0xd3, 0xab, 0x53, 0xe5, 0xeb, 0x5f, 0xfc, 0xfb, 0xd5, 0xa9, 0x2f, 0x8e, - 0xae, 0x4a, 0x3f, 0x3f, 0xba, 0x2a, 0x7d, 0x79, 0x74, 0x55, 0xfa, 0xb7, 0xa3, 0xab, 0xd2, 0x1f, - 0x7c, 0x75, 0x75, 0xea, 0xe7, 0x5f, 0x5d, 0x9d, 0xfa, 0xf2, 0xab, 0xab, 0x53, 0x9f, 0xc6, 0xb9, - 0x5d, 0xb7, 0xe2, 0x2c, 0x35, 0x7c, 0xeb, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xca, 0x90, 0x8b, - 0x16, 0x39, 0x3b, 0x00, 0x00, +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_25813d1ee2eedea8) } + +var fileDescriptor_jobs_25813d1ee2eedea8 = []byte{ + // 4919 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0x4b, 0x6c, 0x23, 0x47, + 0x7a, 0x56, 0x93, 0x14, 0xd9, 0xfc, 0xf9, 0x50, 0xb3, 0xa4, 0x99, 0xa1, 0xb9, 0xf6, 0x50, 0xcb, + 0xb5, 0x3d, 0x0f, 0xaf, 0xa9, 0x5d, 0x79, 0x9f, 0x13, 0x7b, 0x6c, 0xbe, 0x24, 0x51, 0x1a, 0x3d, + 0xa6, 0x29, 0x8d, 0x1f, 0x1b, 0x6f, 0xa7, 0xd9, 0x5d, 0x92, 0x3a, 0x22, 0xbb, 0x39, 0x5d, 0xcd, + 0x99, 0xd1, 0x06, 0x48, 0x82, 0x0d, 0x02, 0x2c, 0xe6, 0x94, 0x00, 0xd9, 0x5c, 0x92, 0x01, 0x02, + 0x64, 0x17, 0xc8, 0x21, 0x97, 0x18, 0x41, 0x92, 0x43, 0x6e, 0xb9, 0xf8, 0x90, 0x00, 0x7b, 0x09, + 0x60, 0xe4, 0xc0, 0x4d, 0xe4, 0x4b, 0x8e, 0x41, 0x72, 0x59, 0xcc, 0x25, 0x41, 0x3d, 0xba, 0xd9, + 0xa4, 0x5e, 0xd4, 0xc8, 0xde, 0x5c, 0x66, 0xd8, 0x7f, 0xfd, 0xf5, 0xd5, 0xeb, 0xaf, 0xef, 0xff, + 0xeb, 0xaf, 0x12, 0x5c, 0xfd, 0x6d, 0xa7, 0x4d, 0x16, 0xe8, 0x3f, 0xbd, 0x36, 0xfb, 0xaf, 0xdc, + 0x73, 0x1d, 0xcf, 0x41, 0x2f, 0x19, 0x8e, 0x71, 0xe0, 0x3a, 0xba, 0xb1, 0x5f, 0x26, 0x0f, 0x3b, + 0x65, 0x56, 0xc2, 0xb5, 0x0a, 0x57, 0xb0, 0xeb, 0x3a, 0x2e, 0xd5, 0xe7, 0x3f, 0x78, 0x8d, 0xc2, + 0xdc, 0x9e, 0xb3, 0xe7, 0xb0, 0x9f, 0x0b, 0xf4, 0x97, 0x90, 0x22, 0x86, 0xd1, 0x6b, 0x2f, 0x98, + 0xba, 0xa7, 0x0b, 0x59, 0xde, 0x97, 0x59, 0xce, 0x9b, 0xbb, 0x8e, 0xdb, 0xd5, 0x3d, 0x1f, 0xe3, + 0x6b, 0xe4, 0x61, 0x67, 0xc1, 0xd0, 0x3d, 0xbd, 0xe3, 0xec, 0x2d, 0x98, 0x98, 0x18, 0xbd, 0xf6, + 0x02, 0xf1, 0xdc, 0xbe, 0xe1, 0xf5, 0x5d, 0x6c, 0x0a, 0xa5, 0xe2, 0x09, 0x4a, 0x1e, 0xb6, 0x75, + 0xdb, 0xf3, 0xf1, 0xfb, 0x9e, 0xd5, 0x59, 0xd8, 0xef, 0x18, 0x0b, 0x9e, 0xd5, 0xc5, 0xc4, 0xd3, + 0xbb, 0x3d, 0x51, 0xf2, 0x55, 0x5a, 0x95, 0x18, 0xfb, 0xb8, 0xab, 0x1b, 0xfb, 0xba, 0xbd, 0x87, + 0xdd, 0x05, 0xde, 0x86, 0xd1, 0x6b, 0x0b, 0x95, 0x57, 0x8d, 0x4e, 0x9f, 0x78, 0xd8, 0x7d, 0x84, + 0x5d, 0x62, 0x39, 0xf6, 0x82, 0xf8, 0xd4, 0xc4, 0x37, 0xd7, 0x2a, 0xfd, 0x1e, 0x4c, 0xdf, 0xc3, + 0x3a, 0xc1, 0xe8, 0x23, 0x48, 0xd8, 0x8e, 0x89, 0x35, 0xcb, 0xcc, 0x4b, 0xf3, 0xd2, 0xcd, 0x4c, + 0xb5, 0x72, 0x34, 0x28, 0xc6, 0x37, 0x1c, 0x13, 0x37, 0xeb, 0xcf, 0x07, 0xc5, 0xb7, 0xf6, 0x2c, + 0x6f, 0xbf, 0xdf, 0x2e, 0x1b, 0x4e, 0x77, 0x21, 0x98, 0x51, 0xb3, 0x3d, 0xfc, 0xbd, 0xd0, 0x3b, + 0xd8, 0x5b, 0x10, 0xf3, 0x51, 0xe6, 0xd5, 0xd4, 0x38, 0x45, 0x6c, 0x9a, 0x68, 0x0e, 0xa6, 0x71, + 0xcf, 0x31, 0xf6, 0xf3, 0x91, 0x79, 0xe9, 0x66, 0x54, 0xe5, 0x1f, 0x77, 0x62, 0xff, 0xf9, 0x17, + 0x45, 0xa9, 0xf4, 0xf3, 0x08, 0x5c, 0xab, 0xea, 0xc6, 0x41, 0xbf, 0xd7, 0xb0, 0x0d, 0xf7, 0xb0, + 0xe7, 0x59, 0x8e, 0xbd, 0xc9, 0xfe, 0x25, 0x48, 0x81, 0xe8, 0x01, 0x3e, 0x64, 0xfd, 0x49, 0xab, + 0xf4, 0x27, 0x7a, 0x07, 0x62, 0x5d, 0xc7, 0xc4, 0x0c, 0x28, 0xbb, 0x78, 0xab, 0x7c, 0xea, 0xe2, + 0x96, 0x87, 0x68, 0xeb, 0x8e, 0x89, 0x55, 0x56, 0x0d, 0xb5, 0x41, 0x3e, 0xe8, 0x12, 0xcd, 0xb2, + 0x77, 0x9d, 0x7c, 0x74, 0x5e, 0xba, 0x99, 0x5a, 0xbc, 0x73, 0x06, 0xc4, 0x29, 0xdd, 0x2a, 0xaf, + 0xad, 0xb7, 0x9a, 0xf6, 0xae, 0x53, 0x4d, 0x1d, 0x0d, 0x8a, 0x09, 0xf1, 0xa1, 0x26, 0x0e, 0xba, + 0x84, 0xfe, 0x28, 0x6c, 0x82, 0x2f, 0xa3, 0xfd, 0xef, 0xbb, 0x16, 0xeb, 0x7f, 0x52, 0xa5, 0x3f, + 0xd1, 0xd7, 0x01, 0x61, 0x8e, 0x87, 0x4d, 0x8d, 0x5a, 0x92, 0x46, 0x07, 0x18, 0x61, 0x03, 0x54, + 0x82, 0x92, 0xba, 0xee, 0xe9, 0x6b, 0xf8, 0x90, 0xcf, 0x90, 0x98, 0xa7, 0xdf, 0x8f, 0x42, 0x76, + 0xd8, 0x15, 0x06, 0xbf, 0x02, 0x71, 0x66, 0x02, 0x98, 0xb5, 0x90, 0x5d, 0xfc, 0xc6, 0x44, 0xd3, + 0x41, 0xab, 0x96, 0x5b, 0xac, 0x9e, 0x2a, 0xea, 0x23, 0x04, 0x31, 0xa2, 0x77, 0x3c, 0xd1, 0x11, + 0xf6, 0x1b, 0xfd, 0x99, 0x04, 0xf3, 0xe3, 0x3d, 0xaa, 0x1e, 0xae, 0xad, 0xb7, 0xd6, 0x75, 0x6a, + 0x47, 0x6b, 0xf8, 0xb0, 0x59, 0xcf, 0x47, 0xe7, 0xa3, 0x37, 0x53, 0x8b, 0x9b, 0x93, 0x37, 0xdc, + 0x38, 0x07, 0xb1, 0x61, 0x7b, 0xee, 0xa1, 0x7a, 0x6e, 0xc3, 0x85, 0x16, 0xbc, 0x36, 0x11, 0x54, + 0xd8, 0x86, 0x92, 0xdc, 0x86, 0xe6, 0x60, 0xfa, 0x91, 0xde, 0xe9, 0x63, 0x31, 0x5a, 0xfe, 0x71, + 0x27, 0xf2, 0x3d, 0xa9, 0x74, 0x0d, 0xe2, 0x7c, 0x62, 0x50, 0x06, 0x92, 0x95, 0x46, 0x6b, 0xf1, + 0xdb, 0xdf, 0x59, 0xae, 0xad, 0x2b, 0x53, 0x62, 0x09, 0xfe, 0x57, 0x82, 0xab, 0x2d, 0xcf, 0xc5, + 0x7a, 0xb7, 0x69, 0xef, 0x61, 0x42, 0xc7, 0x54, 0xc7, 0x9e, 0x6e, 0x75, 0x08, 0xb2, 0x21, 0x4b, + 0x58, 0x89, 0xa6, 0x9b, 0xa6, 0x8b, 0x09, 0xe1, 0x0d, 0x56, 0x97, 0x9f, 0x0f, 0x8a, 0xb5, 0x89, + 0xb6, 0x8e, 0x61, 0x74, 0x16, 0x38, 0x84, 0x65, 0xef, 0x19, 0x46, 0xa7, 0xcc, 0x5b, 0xaa, 0x70, + 0x38, 0x35, 0x43, 0xc2, 0x9f, 0xe8, 0x9b, 0x10, 0x23, 0x3d, 0xdd, 0x66, 0x43, 0x48, 0x2d, 0x5e, + 0x0b, 0xcd, 0xbf, 0xbf, 0x05, 0x5b, 0x3d, 0xdd, 0xae, 0xc6, 0x3e, 0x1d, 0x14, 0xa7, 0x54, 0xa6, + 0x8a, 0xaa, 0x00, 0xc4, 0xd3, 0x5d, 0x4f, 0xa3, 0x5c, 0x22, 0xac, 0xff, 0x95, 0x50, 0x45, 0xca, + 0x35, 0xe5, 0xfd, 0x8e, 0x51, 0xde, 0xf6, 0xb9, 0x46, 0x54, 0x4f, 0xb2, 0x6a, 0x54, 0x5a, 0xd2, + 0xe1, 0xda, 0xd8, 0x04, 0x6c, 0xb9, 0xce, 0x1e, 0xeb, 0xd1, 0x12, 0xa4, 0x8d, 0xbe, 0xe7, 0x3c, + 0xc2, 0x2e, 0x6f, 0x40, 0x9a, 0xbc, 0x81, 0x94, 0xa8, 0xc8, 0x9a, 0xf8, 0xc7, 0x38, 0x64, 0xf8, + 0xc6, 0xf3, 0xe7, 0x76, 0xb4, 0xe3, 0xd2, 0x8b, 0x74, 0x1c, 0xdd, 0x05, 0x19, 0xdb, 0x26, 0x47, + 0x88, 0x4c, 0x8e, 0x90, 0xc0, 0xb6, 0xc9, 0xea, 0xbf, 0xc4, 0x77, 0x72, 0x94, 0x2d, 0x6a, 0xe2, + 0x68, 0x50, 0x8c, 0xee, 0xa8, 0x4d, 0xbe, 0xa5, 0xdf, 0x86, 0x82, 0x89, 0x7b, 0x2e, 0x36, 0x74, + 0xba, 0xa7, 0xdb, 0xac, 0xeb, 0x5a, 0x57, 0xb7, 0xad, 0x5d, 0x4c, 0xbc, 0x7c, 0x8c, 0xd9, 0x58, + 0x7e, 0xa8, 0xc1, 0xc7, 0xb6, 0x2e, 0xca, 0xd1, 0x1f, 0x48, 0x30, 0xdb, 0x77, 0x2d, 0xa2, 0xb5, + 0x0f, 0xb5, 0x8e, 0x63, 0xe8, 0x1d, 0xcb, 0x3b, 0xd4, 0x0e, 0x1e, 0xe5, 0xa7, 0xd9, 0xc6, 0xba, + 0x7b, 0x2e, 0x3b, 0x89, 0x49, 0x2a, 0xef, 0xb8, 0x16, 0xa9, 0x1e, 0xde, 0x13, 0x08, 0x6b, 0x8f, + 0x98, 0xf1, 0x57, 0xe7, 0x8e, 0x06, 0x45, 0x65, 0x47, 0x6d, 0x86, 0x8b, 0x1e, 0xa8, 0x4a, 0x7f, + 0x4c, 0x19, 0xe9, 0x01, 0x2d, 0x59, 0x8e, 0xad, 0x39, 0x9c, 0xe7, 0xf2, 0x71, 0x36, 0x51, 0x8b, + 0x17, 0x67, 0x48, 0x35, 0x87, 0x8f, 0x71, 0xf9, 0x1f, 0x4b, 0x50, 0xa0, 0x2e, 0x07, 0x1b, 0x74, + 0x9a, 0x02, 0x7f, 0xa6, 0xb9, 0xd8, 0x70, 0x5c, 0x33, 0x9f, 0xa0, 0xf3, 0x54, 0x6d, 0xfd, 0xdb, + 0xa4, 0x9e, 0x86, 0x79, 0xc6, 0x7e, 0xdf, 0x32, 0xcb, 0x3b, 0x3b, 0xcd, 0xfa, 0xd1, 0xa0, 0x98, + 0xdf, 0xf2, 0xc1, 0x83, 0x45, 0x54, 0x19, 0xb4, 0x9a, 0xef, 0x9d, 0x52, 0x82, 0xbe, 0x07, 0x59, + 0xc3, 0xe9, 0x74, 0xb0, 0xc1, 0x86, 0xbd, 0xa3, 0x36, 0xf3, 0x32, 0x5b, 0xe0, 0xdc, 0xd1, 0xa0, + 0x98, 0xa9, 0x05, 0x25, 0x74, 0xa9, 0x33, 0x46, 0xf8, 0x13, 0xa9, 0x30, 0x13, 0x9a, 0x30, 0xe6, + 0x4f, 0x92, 0x6c, 0xb6, 0x6e, 0x4d, 0x4c, 0x85, 0x6a, 0x16, 0x8f, 0x7c, 0x17, 0x6a, 0x70, 0xe5, + 0xc4, 0x55, 0x3c, 0x8f, 0xc2, 0x92, 0x61, 0x0a, 0x53, 0x20, 0xcb, 0x17, 0xc5, 0xdf, 0x98, 0xa5, + 0x4f, 0xb2, 0x90, 0x55, 0x31, 0xf1, 0x1c, 0x17, 0xfb, 0x3b, 0xea, 0x13, 0x09, 0x66, 0x69, 0xbc, + 0xe1, 0x5a, 0x3d, 0xcf, 0x71, 0x35, 0x17, 0x3f, 0x76, 0x2d, 0x0f, 0x93, 0x7c, 0x84, 0x19, 0x5d, + 0xe5, 0x8c, 0x21, 0x8c, 0x02, 0x95, 0xeb, 0x01, 0x88, 0x2a, 0x30, 0xb8, 0xdd, 0xdd, 0xfd, 0xf1, + 0x2f, 0x8b, 0x77, 0x26, 0x5a, 0xc7, 0xe3, 0x21, 0x50, 0xb9, 0x59, 0x57, 0x91, 0x79, 0x0c, 0x18, + 0xbd, 0x0c, 0x31, 0x6a, 0xb7, 0xcc, 0xe5, 0x24, 0xab, 0xf2, 0xd1, 0xa0, 0x18, 0xa3, 0x96, 0xad, + 0x32, 0xe9, 0xc8, 0x06, 0x8f, 0xbd, 0xc0, 0x06, 0x5f, 0x86, 0x94, 0xa7, 0xb7, 0x3b, 0x58, 0xa3, + 0x2d, 0x13, 0xb1, 0xfd, 0x5e, 0x1f, 0x9b, 0x09, 0xf2, 0xb0, 0xd3, 0xd6, 0x09, 0x2e, 0x6f, 0x53, + 0xcd, 0xd0, 0xd8, 0xc1, 0xf3, 0x05, 0x04, 0x2d, 0x40, 0x8a, 0x72, 0x99, 0x6b, 0x99, 0x58, 0x33, + 0xdb, 0x6c, 0x0f, 0x25, 0xab, 0xd9, 0xa3, 0x41, 0x11, 0x36, 0x85, 0xb8, 0x5e, 0x55, 0xc1, 0x57, + 0xa9, 0xb7, 0x91, 0x07, 0x73, 0x82, 0x34, 0x82, 0xfd, 0xcf, 0xec, 0x29, 0xc1, 0xba, 0xf0, 0xf6, + 0xe4, 0x8b, 0xc1, 0xd7, 0xdd, 0x37, 0x1e, 0x16, 0xa1, 0xf0, 0x41, 0xa2, 0xf6, 0xb1, 0x12, 0xf4, + 0x06, 0xe4, 0x7a, 0x2e, 0xee, 0xe9, 0x2e, 0xd6, 0x0c, 0xa7, 0xdb, 0xeb, 0x60, 0x0f, 0x9b, 0xcc, + 0xfa, 0x65, 0x55, 0x11, 0x05, 0x35, 0x5f, 0x8e, 0x5e, 0xa3, 0xde, 0x4d, 0xf7, 0x68, 0xe0, 0x44, + 0xb0, 0x4b, 0x35, 0x93, 0x4c, 0x33, 0xc3, 0xa4, 0x4d, 0x21, 0x44, 0x6f, 0xc1, 0x95, 0xe1, 0xba, + 0x11, 0xad, 0xd7, 0x6f, 0x77, 0x2c, 0xb2, 0x8f, 0xcd, 0x3c, 0x30, 0xed, 0xb9, 0x50, 0xe1, 0x96, + 0x5f, 0x86, 0x0e, 0x47, 0x4c, 0xd1, 0xa0, 0x13, 0xa3, 0xef, 0xe1, 0x7c, 0x6a, 0x5e, 0xba, 0x39, + 0x5d, 0x5d, 0x79, 0x3e, 0x28, 0xd6, 0x27, 0xb6, 0x23, 0x82, 0xbb, 0x0b, 0x9e, 0x8b, 0x71, 0xc8, + 0x2c, 0x6b, 0x02, 0x2f, 0x6c, 0x51, 0xbe, 0x0c, 0xa9, 0x00, 0xc3, 0x2d, 0x98, 0x4f, 0xbf, 0x30, + 0xdb, 0x85, 0x50, 0x50, 0x05, 0x12, 0x3c, 0x84, 0x27, 0xf9, 0x0c, 0x5b, 0xc0, 0xaf, 0x9e, 0x66, + 0x43, 0x4c, 0x2b, 0xb4, 0x4a, 0x7e, 0x3d, 0x54, 0x07, 0xf0, 0x0e, 0x7b, 0xbe, 0x25, 0x66, 0x19, + 0xca, 0x6b, 0xa7, 0xa1, 0x1c, 0xf6, 0xc2, 0x86, 0x98, 0xf4, 0xc4, 0x37, 0x41, 0xab, 0x90, 0xe6, + 0xe7, 0x03, 0x81, 0x33, 0xc3, 0x70, 0x6e, 0x9c, 0x82, 0xc3, 0xc2, 0x1e, 0x3d, 0x84, 0x94, 0x22, + 0x81, 0x84, 0xa0, 0x2d, 0xc8, 0xd2, 0x58, 0x95, 0x6a, 0x0a, 0x34, 0x85, 0xa1, 0xdd, 0x3a, 0x05, + 0xad, 0x2e, 0x94, 0x43, 0x78, 0x19, 0x33, 0x24, 0x23, 0x85, 0xff, 0x91, 0x20, 0x77, 0x8c, 0x3c, + 0xd0, 0x36, 0x44, 0x82, 0xe3, 0x07, 0xe5, 0xf4, 0x08, 0x3b, 0x7a, 0x5c, 0x86, 0x48, 0x22, 0x96, + 0x89, 0xf6, 0x20, 0x49, 0xcd, 0xd9, 0xf6, 0xe8, 0xd9, 0x26, 0xc2, 0xc0, 0x57, 0x8f, 0x06, 0x45, + 0x79, 0x8b, 0x09, 0x2f, 0xdd, 0x84, 0xcc, 0xc1, 0x9b, 0x26, 0x2a, 0x42, 0xca, 0x73, 0x34, 0xfc, + 0xc4, 0x22, 0x9e, 0x65, 0xef, 0xb1, 0x60, 0x41, 0x56, 0xc1, 0x73, 0x1a, 0x42, 0x52, 0xf8, 0xf3, + 0x08, 0xa0, 0xe3, 0xbb, 0x14, 0xfd, 0x83, 0x04, 0x2f, 0xfb, 0x31, 0x80, 0xe3, 0x5a, 0x7b, 0x96, + 0xad, 0x77, 0x46, 0x82, 0x01, 0x89, 0xcd, 0xf6, 0x47, 0x97, 0xa1, 0x02, 0x11, 0x20, 0x6c, 0x0a, + 0xf8, 0xf1, 0x40, 0xe1, 0x65, 0xea, 0x41, 0x79, 0xa0, 0x70, 0x4c, 0xe5, 0x81, 0x9a, 0xef, 0x9f, + 0x52, 0xb9, 0xb0, 0x06, 0xaf, 0x9c, 0x09, 0x7c, 0x11, 0xdf, 0x55, 0xf8, 0xb1, 0x04, 0xd7, 0x4e, + 0xf1, 0x28, 0x61, 0x9c, 0x0c, 0xc7, 0xb9, 0x1f, 0xc6, 0x49, 0x2d, 0xfe, 0xc6, 0x25, 0xbc, 0x56, + 0xa8, 0x13, 0xab, 0x31, 0x59, 0x52, 0x22, 0xa5, 0x6f, 0xc0, 0x8c, 0xa8, 0x14, 0x04, 0xb8, 0xaf, + 0x00, 0xec, 0x5b, 0x7b, 0xfb, 0xda, 0x63, 0xdd, 0xc3, 0xae, 0x38, 0x93, 0x26, 0xa9, 0xe4, 0x7d, + 0x2a, 0x28, 0xfd, 0xab, 0x0c, 0x99, 0x66, 0xb7, 0xe7, 0xb8, 0x9e, 0xef, 0x65, 0xef, 0x41, 0x9c, + 0xf9, 0x05, 0x22, 0xd6, 0xaf, 0x7c, 0x46, 0x0f, 0x47, 0x6a, 0x72, 0xff, 0x22, 0x68, 0x41, 0x60, + 0x04, 0xee, 0x2f, 0x72, 0xa2, 0xfb, 0x7b, 0x07, 0xe2, 0x3c, 0x01, 0x21, 0x02, 0xfb, 0xe2, 0x09, + 0x27, 0x82, 0xe6, 0xe6, 0x92, 0xd5, 0xc1, 0x4b, 0x4c, 0xcd, 0x07, 0xe7, 0x95, 0xd0, 0xeb, 0x20, + 0x13, 0xe2, 0x69, 0xc4, 0xfa, 0x11, 0xf7, 0x9e, 0x51, 0x7e, 0xb6, 0x6d, 0xb5, 0xb6, 0x5b, 0xd6, + 0x8f, 0xb0, 0x9a, 0x20, 0xc4, 0xa3, 0x3f, 0x50, 0x01, 0xe4, 0xc7, 0x7a, 0xa7, 0xc3, 0xbc, 0xec, + 0x34, 0x3b, 0xcb, 0x07, 0xdf, 0xa3, 0xdb, 0x2c, 0xfe, 0xe5, 0x6e, 0x33, 0xe1, 0x30, 0x7b, 0xba, + 0xb7, 0xcf, 0x22, 0xc7, 0xa4, 0x0a, 0x5c, 0xb4, 0xa5, 0x7b, 0xfb, 0x28, 0x0f, 0x09, 0xa2, 0x53, + 0xdf, 0x45, 0xf2, 0xf2, 0x7c, 0xf4, 0x66, 0x5a, 0xf5, 0x3f, 0xd1, 0x75, 0x60, 0x9e, 0x97, 0x7f, + 0x32, 0x27, 0x16, 0x55, 0x43, 0x12, 0x36, 0x0f, 0x07, 0x56, 0x4f, 0xdb, 0x3d, 0x20, 0xdc, 0x69, + 0x89, 0x79, 0x38, 0xb0, 0x7a, 0x4b, 0x6b, 0x44, 0x4d, 0xd0, 0xc2, 0xa5, 0x03, 0x82, 0x6e, 0xc0, + 0x8c, 0xc5, 0x4e, 0x40, 0x9a, 0x69, 0xb9, 0xd8, 0xf0, 0x3a, 0x87, 0xcc, 0x61, 0xc9, 0x6a, 0x96, + 0x8b, 0xeb, 0x42, 0x8a, 0x6e, 0x81, 0x32, 0xee, 0x66, 0x99, 0xa3, 0x91, 0xd5, 0x99, 0x31, 0x2f, + 0x4b, 0x55, 0xf9, 0x52, 0x87, 0x1c, 0x67, 0x86, 0xab, 0x72, 0xf9, 0xd0, 0x67, 0x96, 0x61, 0xb6, + 0xa7, 0xbb, 0x04, 0x6b, 0xed, 0xbe, 0x6d, 0x76, 0xb0, 0xc6, 0xb9, 0x3a, 0x9f, 0x65, 0xda, 0x39, + 0x56, 0x54, 0x65, 0x25, 0x9c, 0xd6, 0xcf, 0x8b, 0xbd, 0xaf, 0xfe, 0x3f, 0xc4, 0xde, 0x85, 0x9f, + 0x47, 0x60, 0x9a, 0xd9, 0x39, 0xba, 0x03, 0x31, 0xba, 0xcc, 0xe2, 0x64, 0x37, 0x69, 0xcc, 0xc5, + 0xea, 0x20, 0x04, 0x31, 0x5b, 0xef, 0xe2, 0x3c, 0x62, 0x46, 0xc0, 0x7e, 0xa3, 0x6b, 0x90, 0x20, + 0xf8, 0xa1, 0xf6, 0x48, 0xef, 0xe4, 0x67, 0xd9, 0x0a, 0xc7, 0x09, 0x7e, 0xf8, 0x40, 0xef, 0xa0, + 0x2b, 0x10, 0xb7, 0x88, 0x66, 0xe3, 0xc7, 0xf9, 0x39, 0x36, 0x53, 0xd3, 0x16, 0xd9, 0xc0, 0x8f, + 0x19, 0x6d, 0xeb, 0xee, 0x1e, 0xf6, 0x34, 0xc3, 0xe9, 0x90, 0xfc, 0x15, 0xba, 0xc1, 0x68, 0x48, + 0x47, 0x45, 0x35, 0xa7, 0x43, 0xd0, 0x57, 0x20, 0xf9, 0x58, 0x27, 0x1a, 0xee, 0xf6, 0xbc, 0x43, + 0x36, 0x59, 0x32, 0x35, 0x7b, 0xd2, 0xa0, 0xdf, 0xab, 0x31, 0x39, 0xa2, 0x44, 0x57, 0x63, 0x72, + 0x54, 0x89, 0xad, 0xc6, 0xe4, 0x98, 0x32, 0xbd, 0x1a, 0x93, 0xa7, 0x95, 0xf8, 0x6a, 0x4c, 0x8e, + 0x2b, 0x89, 0xd5, 0x98, 0x9c, 0x50, 0xe4, 0xd5, 0x98, 0x2c, 0x2b, 0xc9, 0xd5, 0x98, 0x9c, 0x54, + 0x60, 0x35, 0x26, 0x83, 0x92, 0x5a, 0x8d, 0xc9, 0x29, 0x25, 0xbd, 0x1a, 0x93, 0xd3, 0x4a, 0x66, + 0x35, 0x26, 0x67, 0x94, 0xec, 0x6a, 0x4c, 0xce, 0x2a, 0x33, 0xab, 0x31, 0x79, 0x46, 0x51, 0x56, + 0x63, 0xb2, 0xa2, 0xe4, 0x56, 0x63, 0x72, 0x4e, 0x41, 0xa5, 0x4f, 0x24, 0x50, 0x5a, 0xf8, 0x61, + 0x1f, 0xdb, 0x06, 0x7e, 0xa0, 0x77, 0x6a, 0xfb, 0x7d, 0xfb, 0x00, 0xbd, 0x0e, 0x33, 0x06, 0xfd, + 0xa1, 0xf1, 0x83, 0x31, 0x1d, 0xaa, 0xc4, 0x86, 0x9a, 0x61, 0xe2, 0x16, 0x95, 0xd2, 0x11, 0xbf, + 0x02, 0x20, 0xf4, 0xe8, 0xce, 0xe6, 0xd9, 0xb7, 0x24, 0x57, 0xa1, 0xdb, 0x79, 0x0c, 0xc6, 0x75, + 0x1e, 0x33, 0xfa, 0x18, 0x81, 0x51, 0x9d, 0xc7, 0x68, 0x01, 0xe6, 0x6c, 0xfc, 0xc4, 0xd3, 0xc6, + 0x95, 0x19, 0x55, 0xa8, 0x39, 0x5a, 0x56, 0x0b, 0x57, 0x28, 0xfd, 0x4b, 0x04, 0x66, 0xfc, 0x4e, + 0xfb, 0x74, 0xb8, 0x0b, 0x0a, 0x5d, 0x16, 0xcb, 0xd4, 0x3c, 0x87, 0x23, 0xf9, 0xc4, 0xf8, 0xce, + 0x19, 0xc4, 0x38, 0x86, 0x42, 0xbf, 0x9b, 0xe6, 0xb6, 0xc3, 0x9a, 0xe3, 0xae, 0x41, 0xcd, 0x90, + 0xb0, 0xac, 0xb0, 0x03, 0x59, 0xbf, 0x12, 0x97, 0xa0, 0x1a, 0xc4, 0x47, 0xda, 0x7b, 0x63, 0x82, + 0xf6, 0xfc, 0xa9, 0x56, 0x45, 0xd5, 0xc2, 0xef, 0x00, 0x3a, 0xde, 0x76, 0xd8, 0x2d, 0x4d, 0x73, + 0xb7, 0xb4, 0x39, 0xea, 0x96, 0xbe, 0x7f, 0xb1, 0xb1, 0x85, 0xba, 0x1d, 0x3e, 0xd5, 0xfd, 0x53, + 0x04, 0xb2, 0xdc, 0x45, 0x04, 0xee, 0xe8, 0x0d, 0xc8, 0x31, 0xd2, 0xb2, 0xec, 0x3d, 0xad, 0x27, + 0x84, 0x6c, 0x7c, 0x11, 0x55, 0xf1, 0x0b, 0x02, 0xe5, 0xaf, 0x41, 0xc6, 0xc5, 0xba, 0x39, 0x54, + 0x8c, 0x30, 0xc5, 0x34, 0x15, 0x06, 0x4a, 0xaf, 0x41, 0x96, 0x79, 0xc3, 0xa1, 0x56, 0x94, 0x69, + 0x65, 0x98, 0x34, 0x50, 0xab, 0x42, 0x86, 0xf4, 0x74, 0x7b, 0xa8, 0x15, 0x63, 0x93, 0x7a, 0x4e, + 0x0e, 0x2a, 0x4d, 0xeb, 0x84, 0x7d, 0xa9, 0x8b, 0x49, 0xbf, 0x8b, 0xb5, 0x9e, 0xc3, 0x0f, 0x5b, + 0x51, 0x35, 0xc9, 0x25, 0x5b, 0x0e, 0x41, 0x3b, 0xcc, 0x54, 0xd8, 0x5c, 0x68, 0x26, 0x9f, 0x9c, + 0x7c, 0x9c, 0xb5, 0x72, 0x7b, 0xf2, 0xe9, 0x54, 0x67, 0xc8, 0xa8, 0xa0, 0xf4, 0x37, 0x12, 0x5c, + 0xa3, 0x01, 0x33, 0x67, 0xc5, 0x1a, 0xcb, 0x9b, 0xfb, 0xd6, 0xa9, 0x43, 0x82, 0x05, 0xdd, 0x41, + 0xfc, 0xb9, 0x72, 0x34, 0x28, 0xc6, 0xa9, 0xf6, 0xa5, 0x3d, 0x57, 0x9c, 0x02, 0x37, 0xd9, 0xf1, + 0xc8, 0x73, 0x75, 0x9b, 0x58, 0xf4, 0xa0, 0x40, 0x97, 0xad, 0x8b, 0xbb, 0x6d, 0xec, 0xf2, 0xc5, + 0x48, 0xab, 0x73, 0x23, 0x85, 0xeb, 0xbc, 0xac, 0x54, 0x80, 0xfc, 0x78, 0x97, 0x83, 0x93, 0xfd, + 0x6f, 0xc2, 0xd5, 0x0d, 0xfc, 0xf8, 0xa4, 0xd1, 0x54, 0x21, 0xc1, 0xf9, 0xcb, 0x37, 0xf9, 0x9b, + 0xe3, 0xac, 0x1a, 0xbe, 0x3a, 0x28, 0xb3, 0x9e, 0x6e, 0xb3, 0x0a, 0xaa, 0x5f, 0xb1, 0xf4, 0x11, + 0x5c, 0x1b, 0x43, 0x0f, 0x96, 0xef, 0x5d, 0x88, 0xd3, 0x93, 0x9f, 0x88, 0x6c, 0xb2, 0xc7, 0x4f, + 0x15, 0xc7, 0xd1, 0x5b, 0x54, 0x5f, 0x15, 0xd5, 0x4a, 0x2a, 0x4b, 0x49, 0xf4, 0xbb, 0x98, 0x5a, + 0xc8, 0x3d, 0x8b, 0x78, 0xe8, 0x3d, 0x48, 0x0b, 0x8b, 0xa0, 0x86, 0xe2, 0x77, 0xfb, 0x1c, 0xa3, + 0x4a, 0xb9, 0x01, 0x08, 0x29, 0xfd, 0xad, 0x04, 0xb3, 0x75, 0xd7, 0xe9, 0xf5, 0xb0, 0x29, 0x7c, + 0x05, 0x9f, 0x0b, 0xdf, 0x45, 0x48, 0x21, 0x17, 0xb1, 0x01, 0x91, 0x66, 0x5d, 0x9c, 0x05, 0xee, + 0x5e, 0xf6, 0x88, 0xd1, 0xac, 0xa3, 0xef, 0xf3, 0x09, 0xe9, 0x13, 0xc6, 0x9f, 0xd9, 0x63, 0x87, + 0xbe, 0x11, 0x33, 0x65, 0x8a, 0xaa, 0xa8, 0x50, 0xfa, 0x59, 0x02, 0xae, 0x84, 0x27, 0x79, 0xb9, + 0xe6, 0x77, 0xfc, 0x63, 0x48, 0x58, 0xb6, 0x89, 0x9f, 0xe0, 0x89, 0x78, 0xf2, 0x24, 0x88, 0xb2, + 0x98, 0x8f, 0x26, 0x85, 0xf1, 0x8f, 0x99, 0x02, 0x13, 0x7d, 0x10, 0x84, 0xa7, 0x3c, 0xed, 0x73, + 0xe7, 0x85, 0xd1, 0xeb, 0x63, 0xa1, 0xea, 0x48, 0x24, 0xc8, 0x1c, 0xca, 0x97, 0x14, 0x09, 0xb6, + 0x20, 0x67, 0xd9, 0x1e, 0x76, 0x3b, 0x58, 0x7f, 0x44, 0x03, 0x1b, 0xda, 0xbc, 0xc8, 0xfe, 0x4c, + 0x1a, 0x46, 0x28, 0x21, 0x00, 0x1e, 0x8e, 0x7c, 0x0c, 0xb3, 0x61, 0x50, 0x7f, 0x09, 0xce, 0xce, + 0x08, 0xb1, 0x19, 0x1e, 0xc2, 0xfa, 0x89, 0x97, 0x10, 0x50, 0x53, 0x4c, 0xfb, 0x03, 0x88, 0xf3, + 0x83, 0xbe, 0x48, 0xaf, 0xde, 0x7d, 0xd1, 0x69, 0xe7, 0x09, 0x04, 0x55, 0xa0, 0x15, 0xfe, 0x54, + 0x82, 0x74, 0x78, 0xb9, 0x91, 0x05, 0x32, 0xeb, 0xbb, 0x4f, 0x69, 0xd1, 0xea, 0x06, 0x8d, 0x65, + 0x59, 0x21, 0x5b, 0x83, 0x77, 0x5f, 0x78, 0x0d, 0x38, 0x84, 0x30, 0xa5, 0xa6, 0x49, 0x03, 0x24, + 0xd3, 0x75, 0x7a, 0xc3, 0xf4, 0x7a, 0x54, 0x95, 0xa9, 0x80, 0xc6, 0x7c, 0x85, 0xdf, 0x85, 0x64, + 0x60, 0x28, 0xa1, 0x13, 0x7e, 0xf4, 0x0b, 0x3c, 0xe1, 0x9f, 0xd9, 0x7e, 0x1d, 0x32, 0x23, 0x33, + 0x86, 0xae, 0x06, 0x7d, 0x88, 0x55, 0xe3, 0xbc, 0x0f, 0xe7, 0xa2, 0x94, 0x7e, 0x19, 0x87, 0xd9, + 0x93, 0x98, 0xf6, 0x43, 0x50, 0x42, 0xbc, 0xa5, 0x75, 0x2c, 0xe2, 0x89, 0xfd, 0x74, 0xeb, 0xec, + 0x03, 0x69, 0x88, 0xfc, 0x84, 0xb5, 0x64, 0xdd, 0x51, 0x4a, 0xfc, 0x01, 0x64, 0x4d, 0xde, 0x71, + 0x4d, 0x6c, 0xd4, 0xe8, 0xb9, 0xe7, 0xc8, 0x13, 0x08, 0x50, 0xa0, 0x67, 0xcc, 0x50, 0x11, 0x61, + 0xf7, 0x0e, 0x3e, 0x7a, 0x90, 0xdb, 0xb1, 0x4c, 0xb6, 0x7b, 0x32, 0xd5, 0xd6, 0xd1, 0xa0, 0x98, + 0x13, 0x58, 0x7e, 0x32, 0xe7, 0xd2, 0x2b, 0x95, 0x33, 0xc7, 0x00, 0x4d, 0xea, 0x75, 0x69, 0x39, + 0x6d, 0x78, 0x7a, 0xe8, 0x75, 0xe9, 0x3e, 0xba, 0xbc, 0xd7, 0xa5, 0x3f, 0x9b, 0x26, 0xfa, 0x43, + 0x09, 0x72, 0x3c, 0xb3, 0xdb, 0xed, 0x7b, 0x3a, 0x4f, 0xd7, 0xfb, 0xe7, 0xd3, 0x0f, 0x8f, 0x06, + 0xc5, 0x19, 0x36, 0x21, 0xeb, 0xa2, 0x8c, 0x35, 0x5b, 0x7d, 0xd1, 0x66, 0x87, 0x28, 0xe2, 0xcc, + 0x16, 0x08, 0x4c, 0xb4, 0x06, 0x59, 0x7e, 0xd8, 0xf6, 0x2f, 0xe0, 0xd9, 0xc1, 0x35, 0x53, 0x7d, + 0xf5, 0xf9, 0xa0, 0x38, 0x7f, 0x82, 0x65, 0xf1, 0x73, 0xfa, 0x03, 0xae, 0xab, 0x66, 0x76, 0xc3, + 0x9f, 0xc8, 0x80, 0x4c, 0x60, 0x1a, 0x87, 0x3d, 0x71, 0xce, 0xbd, 0xbc, 0x2b, 0x4b, 0xfb, 0x36, + 0x42, 0x31, 0xd1, 0x1e, 0xcc, 0xf8, 0x8d, 0x70, 0x87, 0x4e, 0xf2, 0xc9, 0x2f, 0xa4, 0x19, 0xdf, + 0xac, 0xf9, 0xa8, 0x89, 0x48, 0xb9, 0x5c, 0x85, 0xb9, 0x13, 0xa3, 0x9c, 0xcf, 0xe2, 0x70, 0x75, + 0x94, 0x08, 0x83, 0x38, 0x44, 0x1b, 0xf7, 0x90, 0xef, 0x4e, 0x4c, 0xa6, 0x3e, 0x06, 0x27, 0x33, + 0xff, 0x6b, 0xdc, 0x47, 0x7e, 0x3c, 0xe6, 0x23, 0x5f, 0x00, 0x9f, 0x99, 0xd7, 0x18, 0xbe, 0xef, + 0x28, 0x3f, 0x08, 0x7c, 0x01, 0xcf, 0xda, 0xbc, 0xf7, 0x02, 0xf0, 0xac, 0xbe, 0xff, 0x19, 0x78, + 0x83, 0x7f, 0x96, 0x20, 0x33, 0x32, 0xb2, 0x5f, 0xa7, 0x3b, 0xd8, 0x0a, 0xa2, 0x21, 0xfe, 0x4c, + 0xe3, 0x7b, 0x17, 0x1f, 0xd6, 0x68, 0x90, 0x54, 0xf8, 0x7b, 0x09, 0x32, 0x23, 0x13, 0xf9, 0x25, + 0x39, 0x92, 0x2f, 0xbe, 0xe7, 0x6d, 0xc8, 0x8e, 0x2e, 0x51, 0xa8, 0x0d, 0xe9, 0x8b, 0x69, 0xa3, + 0xf4, 0x5d, 0x88, 0x73, 0x09, 0x42, 0x90, 0x7d, 0xbf, 0xd2, 0xdc, 0x6e, 0x6e, 0x2c, 0x6b, 0x4b, + 0x9b, 0xaa, 0xb6, 0x5c, 0x53, 0xa6, 0x50, 0x1a, 0xe4, 0x7a, 0xe3, 0x5e, 0x83, 0x0a, 0x15, 0x09, + 0xa5, 0x20, 0xc1, 0xbe, 0x1a, 0x75, 0x25, 0x52, 0xaa, 0x82, 0xc2, 0xb1, 0x77, 0x31, 0x75, 0x0c, + 0x34, 0xee, 0x47, 0x65, 0x98, 0x65, 0x41, 0x7a, 0x97, 0xc6, 0x6f, 0xd4, 0x15, 0x6a, 0xa1, 0xe8, + 0x39, 0x17, 0x14, 0x51, 0xa7, 0xb8, 0xa1, 0x77, 0x71, 0xe9, 0xef, 0x62, 0x90, 0x1b, 0x82, 0xf8, + 0x6e, 0xf1, 0x75, 0x90, 0x89, 0x65, 0x1f, 0x68, 0xc3, 0x4b, 0x73, 0x9e, 0x48, 0xb3, 0xec, 0x83, + 0x1d, 0xb5, 0xa9, 0x26, 0x68, 0xe1, 0x8e, 0x6b, 0xa1, 0x55, 0x88, 0x39, 0x3d, 0xcf, 0x3f, 0x43, + 0x7e, 0xe7, 0x8c, 0xa9, 0x38, 0xd6, 0x46, 0x79, 0xb3, 0xe7, 0x89, 0x0c, 0x00, 0xc3, 0x40, 0x7f, + 0x2d, 0x0d, 0x4f, 0x3d, 0xfc, 0xb4, 0xf8, 0xfd, 0x0b, 0xe1, 0xf1, 0x09, 0x10, 0x37, 0x98, 0xef, + 0xd3, 0x8d, 0xfa, 0x7c, 0x50, 0xcc, 0x8d, 0x4f, 0x10, 0xb9, 0xe4, 0xd5, 0xa6, 0xdf, 0x45, 0xb4, + 0xca, 0x2f, 0xd5, 0x86, 0x13, 0xcd, 0x1c, 0xc2, 0x84, 0xf7, 0x96, 0x99, 0x91, 0x85, 0x28, 0xec, + 0x41, 0x3a, 0xdc, 0xfb, 0x13, 0xb2, 0xe5, 0x95, 0xd1, 0xb4, 0xc4, 0x1b, 0x13, 0xcd, 0x8c, 0x38, + 0x12, 0x86, 0x52, 0xf4, 0xdf, 0x85, 0x64, 0x30, 0xed, 0x17, 0xc9, 0xed, 0x73, 0x8e, 0x0f, 0x92, + 0x65, 0xd3, 0x4a, 0xbc, 0x34, 0x88, 0x40, 0x5a, 0xc5, 0xc4, 0xe9, 0x3c, 0xc2, 0x26, 0x8d, 0x79, + 0x82, 0x37, 0x2d, 0xd2, 0xe4, 0x6f, 0x5a, 0x2a, 0x90, 0x0c, 0xb2, 0x99, 0x17, 0x79, 0xd7, 0x31, + 0xac, 0x85, 0xee, 0xc2, 0x57, 0xc2, 0xcf, 0x37, 0x9c, 0xbe, 0x6d, 0xea, 0xee, 0xa1, 0xe6, 0x62, + 0xdd, 0xd8, 0xc7, 0xa6, 0xb8, 0xc4, 0x79, 0x29, 0xf4, 0x7e, 0x43, 0x68, 0xa8, 0x5c, 0x01, 0x7d, + 0x08, 0x99, 0xa0, 0x12, 0xf5, 0xc5, 0x2c, 0x82, 0xca, 0x2e, 0x7e, 0xeb, 0xec, 0xe8, 0x2f, 0x18, + 0x75, 0xd9, 0xc7, 0xa3, 0x3e, 0x57, 0x4d, 0xb7, 0x43, 0x5f, 0xa5, 0x77, 0x20, 0x1d, 0x2e, 0x45, + 0x32, 0xc4, 0x36, 0x36, 0x37, 0x1a, 0x7c, 0x4f, 0x57, 0x2b, 0xb5, 0xb5, 0xa5, 0xe6, 0xbd, 0x7b, + 0x8a, 0x44, 0xe5, 0x8d, 0x0f, 0x9a, 0xdb, 0x4a, 0x84, 0xee, 0x6e, 0xb5, 0xd1, 0xda, 0xae, 0xa8, + 0xdb, 0x4a, 0xb4, 0x84, 0x21, 0x13, 0x6e, 0x89, 0x72, 0x26, 0x0d, 0x31, 0x99, 0x60, 0xe4, 0x94, + 0x7d, 0x63, 0xc2, 0xbe, 0xfa, 0xb6, 0xe7, 0x86, 0x51, 0x4b, 0x3f, 0x89, 0x00, 0x1a, 0x9a, 0x4c, + 0x88, 0xa0, 0xc7, 0x1b, 0x8b, 0x5c, 0xbe, 0x31, 0xf4, 0xd3, 0xb3, 0x33, 0xd9, 0x51, 0x96, 0xc9, + 0x66, 0x7b, 0xf7, 0xd7, 0x9a, 0xcd, 0x16, 0x21, 0xcc, 0x7f, 0xc7, 0x00, 0xd5, 0x5c, 0xac, 0x7b, + 0x98, 0xf2, 0x31, 0x39, 0x2b, 0x03, 0x51, 0x85, 0x69, 0x7e, 0x5c, 0x8d, 0x5c, 0xe4, 0xb8, 0x2a, + 0x26, 0x85, 0x57, 0x45, 0x3f, 0x84, 0xb4, 0xe1, 0x74, 0xfa, 0x5d, 0x5b, 0x63, 0xf7, 0xf0, 0xe2, + 0x78, 0xf0, 0xed, 0xb3, 0xb6, 0xf6, 0xb1, 0xce, 0x95, 0x6b, 0x4e, 0x87, 0x7e, 0x07, 0x4f, 0xb1, + 0x18, 0x20, 0xd3, 0x40, 0x2f, 0x43, 0x32, 0xa0, 0x19, 0x66, 0xd6, 0x49, 0x75, 0x28, 0x40, 0x8b, + 0x30, 0xad, 0x13, 0xcd, 0xd9, 0x65, 0x91, 0xfb, 0x79, 0xfb, 0x4e, 0x8d, 0xe9, 0x64, 0x73, 0x17, + 0xbd, 0x05, 0x99, 0xdd, 0x87, 0xfc, 0x34, 0xc3, 0xdd, 0x0a, 0x7f, 0x1e, 0x31, 0x73, 0x34, 0x28, + 0xa6, 0x96, 0xee, 0xb3, 0xc1, 0x52, 0xa7, 0xa2, 0xa6, 0x76, 0x1f, 0x06, 0x1f, 0xe8, 0x36, 0xe4, + 0xba, 0xfa, 0x13, 0x6d, 0xd7, 0xd5, 0x0d, 0x11, 0xbe, 0x77, 0x38, 0x57, 0x4a, 0xea, 0x4c, 0x57, + 0x7f, 0xb2, 0x24, 0xe4, 0x4d, 0xb3, 0x83, 0x0b, 0xff, 0x25, 0x41, 0x42, 0x8c, 0x08, 0xf5, 0x00, + 0xc4, 0xf4, 0x58, 0x26, 0x37, 0xf5, 0x4c, 0xf5, 0xfe, 0xd1, 0xa0, 0x98, 0xac, 0x31, 0x69, 0xb3, + 0x4e, 0x9e, 0x0f, 0x8a, 0xef, 0xbd, 0x28, 0x95, 0xfb, 0x20, 0x6a, 0x92, 0x37, 0xd2, 0x34, 0x59, + 0x9a, 0x75, 0x5f, 0x27, 0xda, 0xbe, 0x45, 0x3c, 0x67, 0xcf, 0xd5, 0xbb, 0x6c, 0x71, 0x65, 0x35, + 0xbd, 0xaf, 0x93, 0x15, 0x5f, 0x86, 0x0a, 0x34, 0x10, 0x7b, 0xc4, 0x9f, 0x51, 0x70, 0x76, 0x09, + 0xbe, 0xd1, 0x22, 0x5c, 0x09, 0x2a, 0x6b, 0x74, 0xd0, 0xed, 0xbe, 0x71, 0x80, 0x99, 0x7f, 0xa4, + 0x4c, 0x3e, 0x1b, 0x14, 0xae, 0xeb, 0x4f, 0xaa, 0xbc, 0xa8, 0x74, 0x05, 0x66, 0x43, 0xcb, 0x1a, + 0x84, 0xcd, 0x18, 0x94, 0x75, 0x6b, 0xcf, 0xd5, 0xc3, 0xaf, 0x14, 0xef, 0xc3, 0xcc, 0xd8, 0x2b, + 0x60, 0x41, 0xb6, 0xe1, 0xf4, 0xe0, 0xe8, 0xb3, 0xe1, 0x72, 0x8d, 0x7f, 0xfa, 0x07, 0x91, 0xac, + 0x31, 0xf2, 0x5d, 0x9a, 0x85, 0x5c, 0xd0, 0x4c, 0xd0, 0xf6, 0xaf, 0xd2, 0x90, 0xd8, 0xd2, 0x0f, + 0x3b, 0x8e, 0x6e, 0xa2, 0x79, 0x48, 0xf9, 0x4f, 0x2f, 0xfc, 0xf6, 0x92, 0x6a, 0x58, 0x84, 0x2c, + 0xc8, 0xf6, 0x09, 0x76, 0xa9, 0x3d, 0x68, 0xec, 0x51, 0x32, 0xf7, 0x1e, 0xd5, 0xea, 0xf3, 0x41, + 0xf1, 0xee, 0x64, 0xcb, 0x83, 0x8d, 0xbe, 0x6b, 0x79, 0x87, 0xe5, 0xd6, 0xfd, 0x7b, 0x3b, 0x02, + 0x8a, 0x6e, 0x62, 0x47, 0xcd, 0xf4, 0xc3, 0x9f, 0xe2, 0x21, 0x0b, 0x9d, 0x6a, 0xad, 0x6b, 0x19, + 0xae, 0x43, 0xfc, 0xfb, 0x0e, 0x21, 0x5d, 0x67, 0x42, 0x74, 0x03, 0x66, 0x76, 0x2d, 0x9b, 0xdd, + 0xb5, 0xf9, 0x7a, 0xfc, 0xaa, 0x23, 0xeb, 0x8b, 0x85, 0xe2, 0x23, 0xc8, 0x86, 0x1e, 0xaf, 0x50, + 0x33, 0x8b, 0x33, 0x33, 0xdb, 0x3c, 0x1a, 0x14, 0x33, 0xc3, 0x6d, 0xcb, 0x4d, 0xed, 0x32, 0x51, + 0x43, 0x66, 0xd8, 0x0c, 0x35, 0xb4, 0x39, 0x98, 0x66, 0x4f, 0xd6, 0xf9, 0x7b, 0x35, 0x95, 0x7f, + 0xa0, 0xef, 0xc0, 0x74, 0x07, 0xeb, 0x04, 0x8b, 0xa7, 0x68, 0xf3, 0x67, 0x10, 0x01, 0x7b, 0xf3, + 0xad, 0x72, 0x75, 0x54, 0x85, 0x38, 0xbf, 0x3d, 0x65, 0x77, 0x9e, 0xc7, 0x93, 0xc5, 0xa7, 0xbe, + 0x3a, 0x5c, 0x99, 0x52, 0x45, 0x4d, 0xd4, 0x80, 0x84, 0xcb, 0x2f, 0xcc, 0xd9, 0x4d, 0xe8, 0xb9, + 0xe9, 0x8f, 0xd0, 0x7d, 0xfc, 0xca, 0x94, 0xea, 0xd7, 0x45, 0xdb, 0xfe, 0xab, 0x15, 0xee, 0x51, + 0xc4, 0xa3, 0x9c, 0xf2, 0x84, 0x21, 0xf2, 0x10, 0x70, 0x04, 0x85, 0x0e, 0xd0, 0x62, 0xb7, 0x27, + 0xec, 0x42, 0xf5, 0xec, 0x01, 0x8e, 0xdc, 0xc4, 0xd3, 0x01, 0xf2, 0x9a, 0x68, 0x03, 0xc0, 0x08, + 0xbc, 0x1c, 0xbb, 0x6a, 0x4d, 0x2d, 0x7e, 0xfd, 0x22, 0xf1, 0xe5, 0xca, 0x94, 0x1a, 0x42, 0x40, + 0xf7, 0x21, 0x65, 0x0c, 0xb7, 0x6d, 0x7e, 0x86, 0x01, 0xbe, 0x79, 0x21, 0xee, 0x5e, 0xa1, 0x7c, + 0x3d, 0x94, 0x8e, 0xf2, 0xb5, 0x32, 0xce, 0xd7, 0x0d, 0xc8, 0x88, 0x4c, 0x15, 0xff, 0x6b, 0x87, + 0x7c, 0x8e, 0xb9, 0x8b, 0xb0, 0x95, 0xf8, 0x7f, 0x0f, 0x51, 0x6e, 0xd8, 0x86, 0x63, 0x62, 0xb3, + 0x41, 0xbf, 0x55, 0x91, 0x98, 0x67, 0x1f, 0x04, 0x2d, 0x43, 0xd6, 0xe8, 0x60, 0xdd, 0xee, 0xf7, + 0x7c, 0x1c, 0x34, 0x21, 0x4e, 0x46, 0xd4, 0x13, 0x40, 0x1b, 0x80, 0x76, 0xd9, 0x4b, 0x97, 0x70, + 0xaf, 0xd8, 0x8d, 0xed, 0x24, 0x60, 0x0a, 0xab, 0xab, 0x0e, 0x7b, 0x86, 0x5e, 0x85, 0x8c, 0xed, + 0xd8, 0x86, 0x6e, 0x1b, 0xb8, 0xc3, 0x3c, 0x2b, 0xbf, 0xe4, 0x1d, 0x15, 0xa2, 0x8f, 0x20, 0x4b, + 0x46, 0x0e, 0x55, 0xf9, 0x2b, 0xac, 0xc5, 0x6f, 0x5c, 0x34, 0x0d, 0xbb, 0x32, 0xa5, 0x8e, 0x21, + 0xa1, 0xdf, 0x02, 0xc5, 0x1b, 0xbb, 0xab, 0x61, 0xd7, 0xc5, 0x67, 0xbf, 0x2a, 0x3b, 0xe5, 0x46, + 0x6a, 0x65, 0x4a, 0x3d, 0x86, 0x86, 0x3e, 0x86, 0x19, 0x32, 0xfa, 0xfe, 0x3a, 0x7f, 0x8d, 0x35, + 0xf0, 0xcd, 0x33, 0x2f, 0x1c, 0x4e, 0x7a, 0xb2, 0xbe, 0x32, 0xa5, 0x8e, 0x63, 0x51, 0x78, 0x7b, + 0xf4, 0xca, 0x27, 0x9f, 0x3f, 0x17, 0xfe, 0xe4, 0x2b, 0x28, 0x0a, 0x3f, 0x86, 0x85, 0xd6, 0x20, + 0xd9, 0xf5, 0x7d, 0x45, 0xfe, 0xa5, 0x73, 0xcf, 0x21, 0xe3, 0xee, 0x6b, 0x65, 0x4a, 0x1d, 0xd6, + 0xaf, 0x26, 0x21, 0x21, 0xae, 0x06, 0x83, 0x0b, 0xf7, 0x84, 0x22, 0x97, 0x7e, 0x25, 0x83, 0x1c, + 0xc4, 0xa0, 0x0b, 0x80, 0x82, 0xa8, 0x61, 0xf8, 0xca, 0x91, 0xba, 0xa0, 0xc8, 0xca, 0x94, 0x9a, + 0xf3, 0xcb, 0x86, 0x0f, 0x1d, 0x6f, 0xc0, 0x4c, 0xd7, 0x31, 0xad, 0x5d, 0x6b, 0x48, 0xfc, 0x3c, + 0x11, 0x9c, 0xf5, 0xc5, 0x82, 0xf8, 0xef, 0x8e, 0x3c, 0x06, 0x9a, 0xe4, 0x31, 0x3d, 0xed, 0x7d, + 0xf0, 0x5a, 0x88, 0x3a, 0x22, 0xb7, 0x6f, 0xb3, 0x5b, 0x40, 0x91, 0x0c, 0xe0, 0xf1, 0x55, 0x46, + 0x48, 0xc5, 0x79, 0xbe, 0x36, 0xc6, 0xcc, 0xb7, 0xce, 0x65, 0x66, 0x7f, 0xec, 0x2b, 0x52, 0x40, + 0xcd, 0x4b, 0xe3, 0xd4, 0x7c, 0xfb, 0x7c, 0x6a, 0x0e, 0xc1, 0x04, 0xdc, 0xbc, 0x73, 0x22, 0x37, + 0x2f, 0x4c, 0xb8, 0x71, 0x42, 0x88, 0xa3, 0xe4, 0x5c, 0x1b, 0x23, 0xe7, 0x5b, 0xe7, 0x92, 0x73, + 0x78, 0x8c, 0x82, 0x9d, 0x37, 0x4f, 0x60, 0xe7, 0x37, 0x27, 0x62, 0xe7, 0x10, 0x58, 0x98, 0x9e, + 0xd5, 0x93, 0xe8, 0xb9, 0x3c, 0x19, 0x3d, 0x87, 0x20, 0x47, 0xf8, 0xf9, 0x07, 0xc7, 0xb8, 0x47, + 0x39, 0x7f, 0xf3, 0x9e, 0x98, 0x01, 0x5a, 0x91, 0x8e, 0x91, 0x8f, 0x7e, 0x02, 0xf9, 0xe4, 0x18, + 0xfc, 0x5b, 0x17, 0x20, 0x9f, 0x50, 0x03, 0xc7, 0xd9, 0xe7, 0x03, 0x48, 0x87, 0x19, 0x83, 0x3d, + 0xba, 0x39, 0x9b, 0xdb, 0x4e, 0xf9, 0x63, 0x11, 0x66, 0x03, 0xa1, 0x22, 0xf4, 0xc3, 0xe3, 0xc4, + 0x33, 0x7b, 0x2e, 0xf8, 0x29, 0xb7, 0xd3, 0x2b, 0xd2, 0x71, 0xe6, 0xb9, 0x17, 0x66, 0x9e, 0xb9, + 0x73, 0x7d, 0xf7, 0xb1, 0x88, 0x76, 0x45, 0x0a, 0x53, 0x0f, 0x80, 0xec, 0x3f, 0x7e, 0x08, 0xd1, + 0x50, 0xe9, 0x4f, 0x24, 0x88, 0xae, 0x3a, 0x6d, 0x94, 0x1d, 0xa6, 0x26, 0x59, 0x52, 0xf1, 0xdd, + 0xa1, 0xba, 0x38, 0xed, 0x7d, 0xed, 0x8c, 0xb6, 0x83, 0x54, 0x6e, 0x50, 0x09, 0xbd, 0x0d, 0x89, + 0x1e, 0x8f, 0xa6, 0x05, 0xd3, 0x94, 0xce, 0xaa, 0xcf, 0x35, 0x55, 0xbf, 0xca, 0xed, 0x5b, 0xe1, + 0xbf, 0x1b, 0x5b, 0x77, 0x4c, 0x8c, 0xb2, 0x00, 0x5b, 0x3a, 0x21, 0xbd, 0x7d, 0x57, 0x27, 0x58, + 0x99, 0x42, 0x09, 0x88, 0xae, 0xad, 0xb7, 0x14, 0xe9, 0xf6, 0x07, 0xe1, 0x44, 0x62, 0x5d, 0xad, + 0x34, 0x37, 0x9a, 0x1b, 0xcb, 0xda, 0x46, 0x65, 0xbd, 0xd1, 0x52, 0xa6, 0x50, 0x1e, 0xe6, 0xde, + 0xaf, 0x34, 0xb7, 0x45, 0x66, 0x51, 0x6b, 0x6e, 0x6c, 0x37, 0xd4, 0x07, 0x95, 0x7b, 0x8a, 0x84, + 0xae, 0x02, 0x52, 0x37, 0x6b, 0x6b, 0xad, 0x7a, 0x55, 0xab, 0x6d, 0xae, 0x6f, 0x55, 0x6a, 0xdb, + 0xcd, 0xcd, 0x0d, 0x25, 0x82, 0x64, 0x88, 0xd5, 0x37, 0x37, 0x1a, 0x0a, 0xdc, 0xfe, 0x69, 0x0c, + 0x62, 0x2c, 0x87, 0xf1, 0x2a, 0xa4, 0x76, 0x36, 0x5a, 0x5b, 0x8d, 0x5a, 0x73, 0xa9, 0xd9, 0xa8, + 0x2b, 0x53, 0x85, 0xd9, 0xa7, 0xcf, 0xe6, 0x67, 0x68, 0xd1, 0x8e, 0x4d, 0x7a, 0xd8, 0x60, 0x24, + 0x8b, 0x0a, 0x10, 0xaf, 0x56, 0x6a, 0x6b, 0x3b, 0x5b, 0x8a, 0x54, 0xc8, 0x3e, 0x7d, 0x36, 0x0f, + 0x54, 0x81, 0x13, 0x1c, 0x7a, 0x99, 0xe7, 0x38, 0x36, 0xd5, 0x86, 0x12, 0x29, 0xcc, 0x3c, 0x7d, + 0x36, 0x9f, 0x62, 0xa9, 0x13, 0x41, 0x52, 0x37, 0x20, 0xd3, 0xaa, 0xad, 0x34, 0xd6, 0x2b, 0x5a, + 0x6d, 0xa5, 0xb2, 0xb1, 0xdc, 0x50, 0xa2, 0x85, 0xb9, 0xa7, 0xcf, 0xe6, 0x95, 0x71, 0x43, 0xa7, + 0x4d, 0x34, 0xd7, 0xb7, 0x36, 0xd5, 0x6d, 0x25, 0x36, 0x6c, 0x82, 0xf3, 0x0b, 0x2a, 0x01, 0xf0, + 0xda, 0x4b, 0x8d, 0x46, 0x5d, 0x99, 0x2e, 0xa0, 0xa7, 0xcf, 0xe6, 0xb3, 0xb4, 0x7c, 0x48, 0x1b, + 0xe8, 0x35, 0x48, 0xd7, 0xd4, 0x46, 0x65, 0xbb, 0xa1, 0xb5, 0xb6, 0x2b, 0xdb, 0x2d, 0x25, 0x3e, + 0x1c, 0x49, 0x88, 0x0a, 0x50, 0x19, 0x72, 0x95, 0x9d, 0xed, 0x4d, 0x6d, 0x44, 0x37, 0x51, 0xb8, + 0xf6, 0xf4, 0xd9, 0xfc, 0x2c, 0xd5, 0xad, 0xf4, 0x3d, 0x27, 0xac, 0xff, 0x75, 0x50, 0x46, 0xfa, + 0xaf, 0x2d, 0xd7, 0x14, 0xb9, 0x70, 0xf5, 0xe9, 0xb3, 0x79, 0x34, 0x3e, 0x84, 0xe5, 0x1a, 0xfa, + 0x16, 0x5c, 0xdd, 0xfe, 0x70, 0xab, 0x51, 0x6f, 0xb4, 0x6a, 0xda, 0xe8, 0xb0, 0x93, 0x85, 0xfc, + 0xd3, 0x67, 0xf3, 0x73, 0xb4, 0xce, 0xb1, 0xa1, 0xbf, 0x09, 0x4a, 0x6b, 0x5b, 0x6d, 0x54, 0xd6, + 0xb5, 0xe6, 0xc6, 0x72, 0xa3, 0xc5, 0x16, 0x0b, 0x86, 0x5d, 0x1a, 0xdb, 0xb4, 0x74, 0x08, 0x1b, + 0x8d, 0xf7, 0xc7, 0xf0, 0x53, 0x43, 0xfd, 0xb1, 0x7d, 0x88, 0xe6, 0x21, 0xb9, 0xde, 0x5c, 0x56, + 0x2b, 0x0c, 0x37, 0x5d, 0xc8, 0x3d, 0x7d, 0x36, 0x9f, 0xa1, 0x7a, 0xc1, 0xae, 0x2a, 0xc8, 0x3f, + 0xf9, 0xcb, 0xeb, 0x53, 0x7f, 0xf5, 0xb3, 0xeb, 0x53, 0xd5, 0x9b, 0x9f, 0xfe, 0xc7, 0xf5, 0xa9, + 0x4f, 0x8f, 0xae, 0x4b, 0xbf, 0x38, 0xba, 0x2e, 0x7d, 0x76, 0x74, 0x5d, 0xfa, 0xf7, 0xa3, 0xeb, + 0xd2, 0x1f, 0x7d, 0x7e, 0x7d, 0xea, 0x17, 0x9f, 0x5f, 0x9f, 0xfa, 0xec, 0xf3, 0xeb, 0x53, 0x1f, + 0xc5, 0xb9, 0x5d, 0xb7, 0xe3, 0xec, 0x68, 0xf8, 0xd6, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x20, + 0xb8, 0x57, 0x0a, 0xe8, 0x3b, 0x00, 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 192d94ade282..64b0aac0d781 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -523,7 +523,34 @@ message ChangefeedDetails { message ResolvedSpan { roachpb.Span span = 1 [(gogoproto.nullable) = false]; util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; - bool boundary_reached = 3; + + // BoundaryReached indicates that this resolved timestamp + // corresponds to a schema change boundary. A schema change boundary is a + // timestamp at which a new version of a table has been committed that either + // indicates a logical backfill or a need for the changefeed to exit. + bool deprecated_boundary_reached = 3; + + enum BoundaryType { + + // NONE indicates that this resolved span does not correspond to a + // boundary. + NONE = 0; + + // BACKFILL indicates that this resolved span corresponds to a boundary + // requiring a backfill internally and perhaps indicates the need for a + // protected timestamp. + BACKFILL = 1; + + // EXIT indicates that this resolved span corresponds to a boundary which + // should result in the changefeed exiting. + EXIT = 2; + + // RESTART indicates that this resolved span corresponds to a boundary which + // should result in the changefeed restarting. + RESTART = 3; + } + + BoundaryType boundary_type = 4; } message ResolvedSpans { From 0b463289ddf59b7569d1dede9ae2a8f916a1fad2 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Sun, 21 Feb 2021 23:42:52 -0500 Subject: [PATCH 5/5] Revert "vendor: bump pebble to 959663f8" Informs #60828. This reverts commit d8c3eef24e342885a3bcac32c23da9b8d0c2f8cc. --- DEPS.bzl | 4 ++-- WORKSPACE | 4 ++-- go.mod | 4 ++-- go.sum | 8 ++++---- vendor | 2 +- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 7871566b039b..cf8a8d57da48 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -428,8 +428,8 @@ def go_deps(): name = "com_github_cockroachdb_pebble", build_file_proto_mode = "disable_global", importpath = "github.com/cockroachdb/pebble", - sum = "h1:H3Cj4eJkj6nqLPogxjA8VTcUTg258Ra+vG0uxJc5SBg=", - version = "v0.0.0-20210219204632-959663f8ccbf", + sum = "h1:EyzONynthydmrlGVcEiyNmbLwDejSGb9Rzyn1NcEtNw=", + version = "v0.0.0-20210217155127-444296cfa2bb", ) go_repository( name = "com_github_cockroachdb_redact", diff --git a/WORKSPACE b/WORKSPACE index cc00170574a4..adc2ff4fb89f 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -48,8 +48,8 @@ go_repository( name = "org_golang_x_sys", build_file_proto_mode = "disable_global", importpath = "golang.org/x/sys", - sum = "h1:mDSj8NPponP6fRpRDblAGl5bpSHjPulHtk5lGl0gLSY=", - version = "v0.0.0-20210219172841-57ea560cfca1", + sum = "h1:2/QtM1mL37YmcsT8HaDNHDgTqqFVw+zr8UzMiBVLzYU=", + version = "v0.0.0-20210217105451-b926d437f341", ) go_repository( diff --git a/go.mod b/go.mod index 55fb091ab23a..528f9f3b2740 100644 --- a/go.mod +++ b/go.mod @@ -40,7 +40,7 @@ require ( github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55 github.com/cockroachdb/gostdlib v1.13.0 github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f - github.com/cockroachdb/pebble v0.0.0-20210219204632-959663f8ccbf + github.com/cockroachdb/pebble v0.0.0-20210217155127-444296cfa2bb github.com/cockroachdb/redact v1.0.9 github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 @@ -154,7 +154,7 @@ require ( golang.org/x/oauth2 v0.0.0-20190115181402-5dab4167f31c golang.org/x/perf v0.0.0-20180704124530-6e6d33e29852 golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 - golang.org/x/sys v0.0.0-20210219172841-57ea560cfca1 + golang.org/x/sys v0.0.0-20210217105451-b926d437f341 golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1 golang.org/x/text v0.3.5 golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 diff --git a/go.sum b/go.sum index ab3875fff515..0d8f130eea7d 100644 --- a/go.sum +++ b/go.sum @@ -174,8 +174,8 @@ github.com/cockroachdb/grpc-gateway v1.14.6-0.20200519165156-52697fc4a249 h1:pZu github.com/cockroachdb/grpc-gateway v1.14.6-0.20200519165156-52697fc4a249/go.mod h1:UJ0EZAp832vCd54Wev9N1BMKEyvcZ5+IM0AwDrnlkEc= github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= -github.com/cockroachdb/pebble v0.0.0-20210219204632-959663f8ccbf h1:H3Cj4eJkj6nqLPogxjA8VTcUTg258Ra+vG0uxJc5SBg= -github.com/cockroachdb/pebble v0.0.0-20210219204632-959663f8ccbf/go.mod h1:1XpB4cLQcF189RAcWi4gUc110zJgtOfT7SVNGY8sOe0= +github.com/cockroachdb/pebble v0.0.0-20210217155127-444296cfa2bb h1:EyzONynthydmrlGVcEiyNmbLwDejSGb9Rzyn1NcEtNw= +github.com/cockroachdb/pebble v0.0.0-20210217155127-444296cfa2bb/go.mod h1:1XpB4cLQcF189RAcWi4gUc110zJgtOfT7SVNGY8sOe0= github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/redact v1.0.9 h1:sjlUvGorKMIVQfo+w2RqDi5eewCHn453C/vdIXMzjzI= @@ -901,8 +901,8 @@ golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20201009025420-dfb3f7c4e634/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210219172841-57ea560cfca1 h1:mDSj8NPponP6fRpRDblAGl5bpSHjPulHtk5lGl0gLSY= -golang.org/x/sys v0.0.0-20210219172841-57ea560cfca1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210217105451-b926d437f341 h1:2/QtM1mL37YmcsT8HaDNHDgTqqFVw+zr8UzMiBVLzYU= +golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221 h1:/ZHdbVpdR/jk3g30/d4yUL0JU9kksj8+F/bnQUVLGDM= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1 h1:v+OssWQX+hTHEmOBgwxdZxK4zHq3yOs8F9J7mk0PY8E= diff --git a/vendor b/vendor index fd22480758bf..ad59c46f9a36 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit fd22480758bf52f005623ed6eb57c197c3506ecd +Subproject commit ad59c46f9a36acd3a14fda6ae5d3a9f25b3e88a6