From 6449b99df97704b7977b0da82a52b0ad0e62cb24 Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Fri, 6 Nov 2020 09:34:01 -0500 Subject: [PATCH] importccl: add `nextval` support for IMPORT INTO CSV Previously, `nextval` was not supported as a default expression for a non-targeted import into column. This change adds that functionality for a CSV import. There is a lot of great discussion about the approach to this problem at https://github.com/cockroachdb/cockroach/issues/48253#issuecomment-671559236. At a high level, on encountering a nextval(seqname) for the first time, IMPORT will reserve a chunk of values for this sequence, and tie those values to the (fileIdx, rowNum) which is a unique reference to a particular row in a distributed import. The size of this chunk grows exponentially based on how many times a single processor encounters a nextval call for that particular sequence. The reservation of the chunk piggy backs on existing methods which provide atomic, non-transactional guarantees when it comes to increasing the value of a sequence. Information about the reserved chunks is stored in the import job progress details so as to ensure the following property: If the import job were to be paused and then resumed, assuming all the rows imported were not checkpointed, we need to ensure that the nextval value for a previously processed (fileIdx, rowNum) is identical to the value computed in the first run of the import job. This property is necessary to prevent duplicate entries with the same key but different value. We use the jobs progress details to check if we have a previously reserved chunk of sequence values which can be used for the current (fileIdx, rowNum). Release note (sql change): IMPORT INTO for CSV now supports nextval as a default expression of a non-targeted column. --- pkg/ccl/importccl/import_processor.go | 17 +- pkg/ccl/importccl/import_processor_test.go | 7 +- pkg/ccl/importccl/import_stmt.go | 3 +- pkg/ccl/importccl/import_stmt_test.go | 278 +++- pkg/ccl/importccl/read_import_avro_test.go | 3 +- pkg/ccl/importccl/read_import_base.go | 25 +- pkg/ccl/importccl/read_import_csv.go | 15 +- pkg/ccl/importccl/read_import_mysql.go | 2 +- pkg/ccl/importccl/read_import_pgdump.go | 3 +- pkg/ccl/importccl/read_import_workload.go | 3 +- pkg/jobs/jobspb/jobs.pb.go | 1560 +++++++++++++----- pkg/jobs/jobspb/jobs.proto | 33 +- pkg/jobs/registry.go | 13 + pkg/sql/distsql_plan_csv.go | 7 + pkg/sql/execinfrapb/processors_bulk_io.pb.go | 30 +- pkg/sql/execinfrapb/processors_bulk_io.proto | 1 - pkg/sql/row/BUILD.bazel | 10 + pkg/sql/row/expr_walker.go | 345 +++- pkg/sql/row/expr_walker_test.go | 225 +++ pkg/sql/row/row_converter.go | 57 +- pkg/sql/rowexec/bulk_row_writer.go | 2 +- 21 files changed, 2149 insertions(+), 490 deletions(-) create mode 100644 pkg/sql/row/expr_walker_test.go diff --git a/pkg/ccl/importccl/import_processor.go b/pkg/ccl/importccl/import_processor.go index b1a1c2bbeefb..4ebe56aa50e3 100644 --- a/pkg/ccl/importccl/import_processor.go +++ b/pkg/ccl/importccl/import_processor.go @@ -56,6 +56,8 @@ type readImportDataProcessor struct { progCh chan execinfrapb.RemoteProducerMetadata_BulkProcessorProgress + seqChunkProvider *row.SeqChunkProvider + importErr error summary *roachpb.BulkOpSummary } @@ -83,6 +85,15 @@ func newReadImportDataProcessor( }); err != nil { return nil, err } + + // Load the import job running the import in case any of the columns have a + // default expression which uses sequences. In this case we need to update the + // job progress within the import processor. + if cp.flowCtx.Cfg.JobRegistry != nil { + cp.seqChunkProvider = &row.SeqChunkProvider{JobID: cp.spec.Progress.JobID, + Registry: cp.flowCtx.Cfg.JobRegistry} + } + return cp, nil } @@ -92,7 +103,8 @@ func (idp *readImportDataProcessor) Start(ctx context.Context) context.Context { // which is closed only after the go routine returns. go func() { defer close(idp.progCh) - idp.summary, idp.importErr = runImport(ctx, idp.flowCtx, &idp.spec, idp.progCh) + idp.summary, idp.importErr = runImport(ctx, idp.flowCtx, &idp.spec, idp.progCh, + idp.seqChunkProvider) }() return idp.StartInternal(ctx, readImportDataProcessorName) } @@ -158,6 +170,7 @@ func makeInputConverter( spec *execinfrapb.ReadImportDataSpec, evalCtx *tree.EvalContext, kvCh chan row.KVBatch, + seqChunkProvider *row.SeqChunkProvider, ) (inputConverter, error) { injectTimeIntoEvalCtx(evalCtx, spec.WalltimeNanos) var singleTable *tabledesc.Immutable @@ -215,7 +228,7 @@ func makeInputConverter( } return newCSVInputReader( kvCh, spec.Format.Csv, spec.WalltimeNanos, int(spec.ReaderParallelism), - singleTable, singleTableTargetCols, evalCtx), nil + singleTable, singleTableTargetCols, evalCtx, seqChunkProvider), nil case roachpb.IOFileFormat_MysqlOutfile: return newMysqloutfileReader( spec.Format.MysqlOut, kvCh, spec.WalltimeNanos, diff --git a/pkg/ccl/importccl/import_processor_test.go b/pkg/ccl/importccl/import_processor_test.go index 49714ef07b80..2c0ff14a8539 100644 --- a/pkg/ccl/importccl/import_processor_test.go +++ b/pkg/ccl/importccl/import_processor_test.go @@ -116,7 +116,8 @@ func TestConverterFlushesBatches(t *testing.T) { } kvCh := make(chan row.KVBatch, batchSize) - conv, err := makeInputConverter(ctx, converterSpec, &evalCtx, kvCh) + conv, err := makeInputConverter(ctx, converterSpec, &evalCtx, kvCh, + nil /* seqChunkProvider */) if err != nil { t.Fatalf("makeInputConverter() error = %v", err) } @@ -419,7 +420,7 @@ func TestImportHonorsResumePosition(t *testing.T) { } }() - _, err := runImport(ctx, flowCtx, spec, progCh) + _, err := runImport(ctx, flowCtx, spec, progCh, nil /* seqChunkProvider */) if err != nil { t.Fatal(err) @@ -496,7 +497,7 @@ func TestImportHandlesDuplicateKVs(t *testing.T) { } }() - _, err := runImport(ctx, flowCtx, spec, progCh) + _, err := runImport(ctx, flowCtx, spec, progCh, nil /* seqChunkProvider */) require.True(t, errors.HasType(err, &kvserverbase.DuplicateKeyError{})) }) } diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index 4840b479508e..5736c87d26da 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -1437,7 +1437,8 @@ func (r *importResumer) Resume( } } - res, err := sql.DistIngest(ctx, p, r.job, tables, files, format, details.Walltime, r.testingKnobs.alwaysFlushJobProgress) + res, err := sql.DistIngest(ctx, p, r.job, tables, files, format, details.Walltime, + r.testingKnobs.alwaysFlushJobProgress) if err != nil { return err } diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go index f54e33a95b11..09f80d421ac8 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -42,6 +42,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/distsql" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/row" @@ -3770,15 +3772,6 @@ func TestImportDefault(t *testing.T) { format: "CSV", expectedResults: [][]string{{"1", "102"}, {"2", "102"}}, }, - { - name: "nextval", - sequence: "testseq", - data: "1\n2", - create: "a INT, b INT DEFAULT nextval('testseq')", - targetCols: "a", - format: "CSV", - expectedError: "unsafe for import", - }, // TODO (anzoteh96): add AVRO format, and also MySQL and PGDUMP once // IMPORT INTO are supported for these file formats. { @@ -4050,6 +4043,273 @@ func TestImportDefault(t *testing.T) { }) } +func TestImportDefaultNextVal(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + defer setImportReaderParallelism(1)() + skip.UnderStressRace(t, "test hits a timeout before a successful run") + + const nodes = 3 + numFiles := 1 + rowsPerFile := 1000 + rowsPerRaceFile := 16 + testFiles := makeCSVData(t, numFiles, rowsPerFile, numFiles, rowsPerRaceFile) + + ctx := context.Background() + baseDir := filepath.Join("testdata", "csv") + tc := testcluster.StartTestCluster(t, nodes, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ExternalIODir: baseDir}}) + defer tc.Stopper().Stop(ctx) + conn := tc.Conns[0] + + sqlDB := sqlutils.MakeSQLRunner(conn) + + type seqMetadata struct { + start int + increment int + expectedImportChunkAllocs int + // We process fewer rows under race. + expectedImportChunkAllocsUnderRace int + } + + t.Run("nextval", func(t *testing.T) { + testCases := []struct { + name string + create string + targetCols []string + seqToNumNextval map[string]seqMetadata + insertData string + }{ + { + name: "simple-nextval", + create: "a INT, b INT DEFAULT nextval('myseq'), c STRING", + targetCols: []string{"a", "c"}, + // 1000 rows means we will allocate 3 chunks of 10, 100, 1000. + // The 2 inserts will add 6 more nextval calls. + // First insert: 1->3 + // Import: 3->1113 + // Second insert 1113->1116 + seqToNumNextval: map[string]seqMetadata{"myseq": {1, 1, 1116, 116}}, + insertData: `(1, 'cat'), (2, 'him'), (3, 'meme')`, + }, + { + name: "simple-nextval-with-increment-and-start", + create: "a INT, b INT DEFAULT nextval('myseq'), c STRING", + targetCols: []string{"a", "c"}, + // 1000 rows means we will allocate 3 chunks of 10, 100, 1000. + // The 2 inserts will add 6 more nextval calls. + // First insert: 100->120 + // Import: 120->11220 + // Second insert: 11220->11250 + seqToNumNextval: map[string]seqMetadata{"myseq": {100, 10, 11250, 1250}}, + insertData: `(1, 'cat'), (2, 'him'), (3, 'meme')`, + }, + { + name: "two-nextval-diff-seq", + create: "a INT, b INT DEFAULT nextval('myseq') + nextval('myseq2'), c STRING", + targetCols: []string{"a", "c"}, + seqToNumNextval: map[string]seqMetadata{"myseq": {1, 1, 1116, 116}, + "myseq2": {1, 1, 1116, 116}}, + insertData: `(1, 'cat'), (2, 'him'), (3, 'meme')`, + }, + // TODO(adityamaru): Unskip once #56387 is fixed. + //{ + // name: "two-nextval-same-seq", + // create: "a INT, b INT DEFAULT nextval('myseq') + nextval('myseq'), + // c STRING", + // targetCols: []string{"a", "c"}, + // seqToNumNextval: map[string]int{"myseq": 1, "myseq2": 1}, + // expectedImportChunkAllocs: 1110, + //}, + { + name: "two-nextval-cols-same-seq", + create: "a INT, b INT DEFAULT nextval('myseq'), c STRING, d INT DEFAULT nextval('myseq')", + targetCols: []string{"a", "c"}, + // myseq will allocate 10, 100, 1000, 10000 for the 2000 rows. + // 2 inserts will consume 12 more nextval calls. + // First insert: 1->6 + // Import: 6->11116 + // Second insert: 11116->11122 + seqToNumNextval: map[string]seqMetadata{"myseq": {1, 1, 11122, 122}}, + insertData: `(1, 'cat'), (2, 'him'), (3, 'meme')`, + }, + } + + for _, test := range testCases { + t.Run(test.name, func(t *testing.T) { + defer sqlDB.Exec(t, `DROP TABLE t`) + for seqName := range test.seqToNumNextval { + sqlDB.Exec(t, fmt.Sprintf(`DROP SEQUENCE IF EXISTS %s`, seqName)) + sqlDB.Exec(t, fmt.Sprintf(`CREATE SEQUENCE %s START %d INCREMENT %d`, seqName, + test.seqToNumNextval[seqName].start, test.seqToNumNextval[seqName].increment)) + } + sqlDB.Exec(t, fmt.Sprintf(`CREATE TABLE t (%s)`, test.create)) + sqlDB.Exec(t, fmt.Sprintf(`INSERT INTO t (%s) VALUES %s`, + strings.Join(test.targetCols, ", "), test.insertData)) + sqlDB.Exec(t, fmt.Sprintf(`IMPORT INTO t (%s) CSV DATA (%s)`, + strings.Join(test.targetCols, ", "), strings.Join(testFiles.files, ", "))) + sqlDB.Exec(t, fmt.Sprintf(`INSERT INTO t (%s) VALUES %s`, + strings.Join(test.targetCols, ", "), test.insertData)) + + for seqName := range test.seqToNumNextval { + var seqVal int + sqlDB.QueryRow(t, fmt.Sprintf(`SELECT last_value from %s`, seqName)).Scan(&seqVal) + expectedVal := test.seqToNumNextval[seqName].expectedImportChunkAllocs + if util.RaceEnabled { + expectedVal = test.seqToNumNextval[seqName].expectedImportChunkAllocsUnderRace + } + require.Equal(t, expectedVal, seqVal) + } + }) + } + }) +} + +func TestImportDefaultWithResume(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + defer setImportReaderParallelism(1)() + const batchSize = 5 + defer TestingSetParallelImporterReaderBatchSize(batchSize)() + defer row.TestingSetDatumRowConverterBatchSize(2 * batchSize)() + jobs.DefaultAdoptInterval = 100 * time.Millisecond + + s, db, _ := serverutils.StartServer(t, + base.TestServerArgs{ + Knobs: base.TestingKnobs{ + RegistryLiveness: jobs.NewFakeNodeLiveness(1), + DistSQL: &execinfra.TestingKnobs{ + BulkAdderFlushesEveryBatch: true, + }, + }, + }) + registry := s.JobRegistry().(*jobs.Registry) + ctx := context.Background() + defer s.Stopper().Stop(ctx) + + sqlDB := sqlutils.MakeSQLRunner(db) + testCases := []struct { + name string + create string + targetCols string + format string + sequence string + }{ + { + name: "nextval", + create: "a INT, b STRING, c INT PRIMARY KEY DEFAULT nextval('mysequence')", + targetCols: "a, b", + sequence: "mysequence", + }, + } + for _, test := range testCases { + t.Run(test.name, func(t *testing.T) { + defer fmt.Sprintf(`DROP SEQUENCE IF EXISTS %s`, test.sequence) + defer sqlDB.Exec(t, `DROP TABLE t`) + + sqlDB.Exec(t, fmt.Sprintf(`CREATE SEQUENCE %s`, test.sequence)) + sqlDB.Exec(t, fmt.Sprintf(`CREATE TABLE t (%s)`, test.create)) + + jobCtx, cancelImport := context.WithCancel(ctx) + jobIDCh := make(chan int64) + var jobID int64 = -1 + + registry.TestingResumerCreationKnobs = map[jobspb.Type]func(raw jobs.Resumer) jobs.Resumer{ + // Arrange for our special job resumer to be + // returned the very first time we start the import. + jobspb.TypeImport: func(raw jobs.Resumer) jobs.Resumer { + resumer := raw.(*importResumer) + resumer.testingKnobs.ignoreProtectedTimestamps = true + resumer.testingKnobs.alwaysFlushJobProgress = true + resumer.testingKnobs.afterImport = func(summary backupccl.RowCount) error { + return nil + } + if jobID == -1 { + return &cancellableImportResumer{ + ctx: jobCtx, + jobIDCh: jobIDCh, + wrapped: resumer, + } + } + return resumer + }, + } + + expectedNumRows := 10*batchSize + 1 + testBarrier, csvBarrier := newSyncBarrier() + csv1 := newCsvGenerator(0, expectedNumRows, &intGenerator{}, &strGenerator{}) + csv1.addBreakpoint(7*batchSize, func() (bool, error) { + defer csvBarrier.Enter()() + return false, nil + }) + + // Convince distsql to use our "external" storage implementation. + storage := newGeneratedStorage(csv1) + s.DistSQLServer().(*distsql.ServerImpl).ServerConfig.ExternalStorage = storage.externalStorageFactory() + + // Execute import; ignore any errors returned + // (since we're aborting the first import run.). + go func() { + _, _ = sqlDB.DB.ExecContext(ctx, + fmt.Sprintf(`IMPORT INTO t (%s) CSV DATA ($1)`, test.targetCols), storage.getGeneratorURIs()[0]) + }() + jobID = <-jobIDCh + + // Wait until we are blocked handling breakpoint. + unblockImport := testBarrier.Enter() + // Wait until we have recorded some job progress. + js := queryJobUntil(t, sqlDB.DB, jobID, func(js jobState) bool { + return js.prog.ResumePos[0] > 0 + }) + + // Pause the job; + if err := registry.PauseRequested(ctx, nil, jobID); err != nil { + t.Fatal(err) + } + // Send cancellation and unblock breakpoint. + cancelImport() + unblockImport() + + // Get number of sequence value chunks which have been reserved. + js = queryJobUntil(t, sqlDB.DB, jobID, func(js jobState) bool { + return jobs.StatusPaused == js.status + }) + // We expect two chunk entries since our breakpoint is at 7*batchSize. + // [1, 10] and [11, 100] + var id int32 + sqlDB.QueryRow(t, fmt.Sprintf(`SELECT id FROM system.namespace WHERE name='%s'`, + test.sequence)).Scan(&id) + seqDetailsOnPause := js.prog.SequenceDetails + chunksOnPause := seqDetailsOnPause[0].SeqIdToChunks[id].Chunks + require.Equal(t, len(chunksOnPause), 2) + require.Equal(t, chunksOnPause[0].ChunkStartVal, int64(1)) + require.Equal(t, chunksOnPause[0].ChunkSize, int64(10)) + require.Equal(t, chunksOnPause[1].ChunkStartVal, int64(11)) + require.Equal(t, chunksOnPause[1].ChunkSize, int64(100)) + + // Just to be doubly sure, check the sequence value before and after + // resumption to make sure it hasn't changed. + var seqValOnPause int64 + sqlDB.QueryRow(t, fmt.Sprintf(`SELECT last_value FROM %s`, test.sequence)).Scan(&seqValOnPause) + + // Unpause the job and wait for it to complete. + if err := registry.Unpause(ctx, nil, jobID); err != nil { + t.Fatal(err) + } + js = queryJobUntil(t, sqlDB.DB, jobID, func(js jobState) bool { return jobs.StatusSucceeded == js.status }) + // No additional chunks should have been allocated on job resumption since + // we already have enough chunks of the sequence values to cover all the + // rows. + seqDetailsOnSuccess := js.prog.SequenceDetails + require.Equal(t, seqDetailsOnPause, seqDetailsOnSuccess) + + var seqValOnSuccess int64 + sqlDB.QueryRow(t, fmt.Sprintf(`SELECT last_value FROM %s`, + test.sequence)).Scan(&seqValOnSuccess) + require.Equal(t, seqValOnPause, seqValOnSuccess) + }) + } +} + func TestImportComputed(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/ccl/importccl/read_import_avro_test.go b/pkg/ccl/importccl/read_import_avro_test.go index c818b185d57a..b8e0b09e3743 100644 --- a/pkg/ccl/importccl/read_import_avro_test.go +++ b/pkg/ccl/importccl/read_import_avro_test.go @@ -253,7 +253,8 @@ func (th *testHelper) newRecordStream( require.NoError(t, err) conv, err := row.NewDatumRowConverter( - context.Background(), th.schemaTable, nil, th.evalCtx.Copy(), nil) + context.Background(), th.schemaTable, nil, th.evalCtx.Copy(), nil, + nil /* seqChunkProvider */) require.NoError(t, err) return &testRecordStream{ producer: producer, diff --git a/pkg/ccl/importccl/read_import_base.go b/pkg/ccl/importccl/read_import_base.go index 9ef0537c0344..7f2dbcee48fa 100644 --- a/pkg/ccl/importccl/read_import_base.go +++ b/pkg/ccl/importccl/read_import_base.go @@ -47,6 +47,7 @@ func runImport( flowCtx *execinfra.FlowCtx, spec *execinfrapb.ReadImportDataSpec, progCh chan execinfrapb.RemoteProducerMetadata_BulkProcessorProgress, + seqChunkProvider *row.SeqChunkProvider, ) (*roachpb.BulkOpSummary, error) { // Used to send ingested import rows to the KV layer. kvCh := make(chan row.KVBatch, 10) @@ -69,7 +70,11 @@ func runImport( flowCtx.TypeResolverFactory.Descriptors.ReleaseAll(ctx) } - conv, err := makeInputConverter(ctx, spec, flowCtx.NewEvalCtx(), kvCh) + evalCtx := flowCtx.NewEvalCtx() + // TODO(adityamaru): Should we just plumb the flowCtx instead of this + // assignment. + evalCtx.DB = flowCtx.Cfg.DB + conv, err := makeInputConverter(ctx, spec, evalCtx, kvCh, seqChunkProvider) if err != nil { return nil, err } @@ -407,13 +412,14 @@ func newImportRowError(err error, row string, num int64) error { // parallelImportContext describes state associated with the import. type parallelImportContext struct { - walltime int64 // Import time stamp. - numWorkers int // Parallelism - batchSize int // Number of records to batch - evalCtx *tree.EvalContext // Evaluation context. - tableDesc *tabledesc.Immutable // Table descriptor we're importing into. - targetCols tree.NameList // List of columns to import. nil if importing all columns. - kvCh chan row.KVBatch // Channel for sending KV batches. + walltime int64 // Import time stamp. + numWorkers int // Parallelism. + batchSize int // Number of records to batch. + evalCtx *tree.EvalContext // Evaluation context. + tableDesc *tabledesc.Immutable // Table descriptor we're importing into. + targetCols tree.NameList // List of columns to import. nil if importing all columns. + kvCh chan row.KVBatch // Channel for sending KV batches. + seqChunkProvider *row.SeqChunkProvider // Used to reserve chunks of sequence values. } // importFileContext describes state specific to a file being imported. @@ -441,7 +447,8 @@ func makeDatumConverter( ctx context.Context, importCtx *parallelImportContext, fileCtx *importFileContext, ) (*row.DatumRowConverter, error) { conv, err := row.NewDatumRowConverter( - ctx, importCtx.tableDesc, importCtx.targetCols, importCtx.evalCtx, importCtx.kvCh) + ctx, importCtx.tableDesc, importCtx.targetCols, importCtx.evalCtx, importCtx.kvCh, + importCtx.seqChunkProvider) if err == nil { conv.KvBatch.Source = fileCtx.source } diff --git a/pkg/ccl/importccl/read_import_csv.go b/pkg/ccl/importccl/read_import_csv.go index bcb2785b1293..d68bc01cad3f 100644 --- a/pkg/ccl/importccl/read_import_csv.go +++ b/pkg/ccl/importccl/read_import_csv.go @@ -42,19 +42,22 @@ func newCSVInputReader( tableDesc *tabledesc.Immutable, targetCols tree.NameList, evalCtx *tree.EvalContext, + seqChunkProvider *row.SeqChunkProvider, ) *csvInputReader { numExpectedDataCols := len(targetCols) if numExpectedDataCols == 0 { numExpectedDataCols = len(tableDesc.VisibleColumns()) } + return &csvInputReader{ importCtx: ¶llelImportContext{ - walltime: walltime, - numWorkers: parallelism, - evalCtx: evalCtx, - tableDesc: tableDesc, - targetCols: targetCols, - kvCh: kvCh, + walltime: walltime, + numWorkers: parallelism, + evalCtx: evalCtx, + tableDesc: tableDesc, + targetCols: targetCols, + kvCh: kvCh, + seqChunkProvider: seqChunkProvider, }, numExpectedDataCols: numExpectedDataCols, opts: opts, diff --git a/pkg/ccl/importccl/read_import_mysql.go b/pkg/ccl/importccl/read_import_mysql.go index affa7c109fa7..b1acee028de2 100644 --- a/pkg/ccl/importccl/read_import_mysql.go +++ b/pkg/ccl/importccl/read_import_mysql.go @@ -74,7 +74,7 @@ func newMysqldumpReader( continue } conv, err := row.NewDatumRowConverter(ctx, tabledesc.NewImmutable(*table.Desc), - nil /* targetColNames */, evalCtx, kvCh) + nil /* targetColNames */, evalCtx, kvCh, nil /* seqChunkProvider */) if err != nil { return nil, err } diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go index 471fc7372452..0456d815c3c6 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -611,7 +611,8 @@ func newPgDumpReader( for i, col := range tableDesc.VisibleColumns() { colSubMap[col.Name] = i } - conv, err := row.NewDatumRowConverter(ctx, tableDesc, targetCols, evalCtx, kvCh) + conv, err := row.NewDatumRowConverter(ctx, tableDesc, targetCols, evalCtx, kvCh, + nil /* seqChunkProvider */) if err != nil { return nil, err } diff --git a/pkg/ccl/importccl/read_import_workload.go b/pkg/ccl/importccl/read_import_workload.go index e4ea324d1a4e..65bac517563e 100644 --- a/pkg/ccl/importccl/read_import_workload.go +++ b/pkg/ccl/importccl/read_import_workload.go @@ -221,7 +221,8 @@ func NewWorkloadKVConverter( // // This worker needs its own EvalContext and DatumAlloc. func (w *WorkloadKVConverter) Worker(ctx context.Context, evalCtx *tree.EvalContext) error { - conv, err := row.NewDatumRowConverter(ctx, w.tableDesc, nil /* targetColNames */, evalCtx, w.kvCh) + conv, err := row.NewDatumRowConverter(ctx, w.tableDesc, nil /* targetColNames */, evalCtx, + w.kvCh, nil /* seqChunkProvider */) if err != nil { return err } diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index d7a2369212a4..22096fa0178b 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -55,7 +55,7 @@ func (x EncryptionMode) String() string { return proto.EnumName(EncryptionMode_name, int32(x)) } func (EncryptionMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_8c0ac68f84c44742, []int{0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{0} } type Status int32 @@ -84,7 +84,7 @@ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } func (Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_8c0ac68f84c44742, []int{1} + return fileDescriptor_jobs_98971d9474e7d77d, []int{1} } type Type int32 @@ -130,7 +130,7 @@ var Type_value = map[string]int32{ } func (Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_8c0ac68f84c44742, []int{2} + return fileDescriptor_jobs_98971d9474e7d77d, []int{2} } type EncryptionInfo_Scheme int32 @@ -150,7 +150,7 @@ func (x EncryptionInfo_Scheme) String() string { return proto.EnumName(EncryptionInfo_Scheme_name, int32(x)) } func (EncryptionInfo_Scheme) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_8c0ac68f84c44742, []int{2, 0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{2, 0} } type SchemaChangeGCProgress_Status int32 @@ -180,7 +180,7 @@ func (x SchemaChangeGCProgress_Status) String() string { return proto.EnumName(SchemaChangeGCProgress_Status_name, int32(x)) } func (SchemaChangeGCProgress_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_8c0ac68f84c44742, []int{16, 0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{18, 0} } type Lease struct { @@ -194,7 +194,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_8c0ac68f84c44742, []int{0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{0} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -235,7 +235,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_8c0ac68f84c44742, []int{1} + return fileDescriptor_jobs_98971d9474e7d77d, []int{1} } func (m *BackupEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -269,7 +269,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_8c0ac68f84c44742, []int{1, 0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{1, 0} } func (m *BackupEncryptionOptions_KMSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -309,7 +309,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_8c0ac68f84c44742, []int{2} + return fileDescriptor_jobs_98971d9474e7d77d, []int{2} } func (m *EncryptionInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -364,7 +364,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_8c0ac68f84c44742, []int{3} + return fileDescriptor_jobs_98971d9474e7d77d, []int{3} } func (m *BackupDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -396,7 +396,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_8c0ac68f84c44742, []int{4} + return fileDescriptor_jobs_98971d9474e7d77d, []int{4} } func (m *BackupProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -458,7 +458,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_8c0ac68f84c44742, []int{5} + return fileDescriptor_jobs_98971d9474e7d77d, []int{5} } func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -495,7 +495,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_8c0ac68f84c44742, []int{5, 0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{5, 0} } func (m *RestoreDetails_DescriptorRewrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -528,7 +528,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_8c0ac68f84c44742, []int{5, 1} + return fileDescriptor_jobs_98971d9474e7d77d, []int{5, 1} } func (m *RestoreDetails_BackupLocalityInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -561,7 +561,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_8c0ac68f84c44742, []int{6} + return fileDescriptor_jobs_98971d9474e7d77d, []int{6} } func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -622,7 +622,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_8c0ac68f84c44742, []int{7} + return fileDescriptor_jobs_98971d9474e7d77d, []int{7} } func (m *ImportDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -660,7 +660,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_8c0ac68f84c44742, []int{7, 0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{7, 0} } func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -685,6 +685,119 @@ func (m *ImportDetails_Table) XXX_DiscardUnknown() { var xxx_messageInfo_ImportDetails_Table proto.InternalMessageInfo +// SequenceValChunks represents a single chunk of sequence values allocated +// during an IMPORT. +type SequenceValChunk struct { + ChunkStartVal int64 `protobuf:"varint,1,opt,name=chunk_start_val,json=chunkStartVal,proto3" json:"chunk_start_val,omitempty"` + ChunkSize int64 `protobuf:"varint,2,opt,name=chunk_size,json=chunkSize,proto3" json:"chunk_size,omitempty"` + // The first row in the file being imported from which the current chunk of + // sequence values is being used. + ChunkStartRow int64 `protobuf:"varint,3,opt,name=chunk_start_row,json=chunkStartRow,proto3" json:"chunk_start_row,omitempty"` + // The row in the file being imported at which the import will need to use a + // new chunk of sequence values. + NextChunkStartRow int64 `protobuf:"varint,4,opt,name=next_chunk_start_row,json=nextChunkStartRow,proto3" json:"next_chunk_start_row,omitempty"` +} + +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_98971d9474e7d77d, []int{8} +} +func (m *SequenceValChunk) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SequenceValChunk) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SequenceValChunk) XXX_Merge(src proto.Message) { + xxx_messageInfo_SequenceValChunk.Merge(dst, src) +} +func (m *SequenceValChunk) XXX_Size() int { + return m.Size() +} +func (m *SequenceValChunk) XXX_DiscardUnknown() { + xxx_messageInfo_SequenceValChunk.DiscardUnknown(m) +} + +var xxx_messageInfo_SequenceValChunk proto.InternalMessageInfo + +// SequenceDetails represents information about the sequences processed in a +// single file during IMPORT. +type SequenceDetails struct { + // Mapping from sequence ID to allocated sequence chunks. + SeqIdToChunks map[int32]*SequenceDetails_SequenceChunks `protobuf:"bytes,1,rep,name=seq_id_to_chunks,json=seqIdToChunks,proto3" json:"seq_id_to_chunks,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +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_98971d9474e7d77d, []int{9} +} +func (m *SequenceDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SequenceDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SequenceDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_SequenceDetails.Merge(dst, src) +} +func (m *SequenceDetails) XXX_Size() int { + return m.Size() +} +func (m *SequenceDetails) XXX_DiscardUnknown() { + xxx_messageInfo_SequenceDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_SequenceDetails proto.InternalMessageInfo + +// SequenceChunks represents all the chunks reserved for a particular sequence +// during an IMPORT. +type SequenceDetails_SequenceChunks struct { + Chunks []*SequenceValChunk `protobuf:"bytes,1,rep,name=chunks,proto3" json:"chunks,omitempty"` +} + +func (m *SequenceDetails_SequenceChunks) Reset() { *m = SequenceDetails_SequenceChunks{} } +func (m *SequenceDetails_SequenceChunks) String() string { return proto.CompactTextString(m) } +func (*SequenceDetails_SequenceChunks) ProtoMessage() {} +func (*SequenceDetails_SequenceChunks) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_98971d9474e7d77d, []int{9, 0} +} +func (m *SequenceDetails_SequenceChunks) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SequenceDetails_SequenceChunks) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SequenceDetails_SequenceChunks) XXX_Merge(src proto.Message) { + xxx_messageInfo_SequenceDetails_SequenceChunks.Merge(dst, src) +} +func (m *SequenceDetails_SequenceChunks) XXX_Size() int { + return m.Size() +} +func (m *SequenceDetails_SequenceChunks) XXX_DiscardUnknown() { + xxx_messageInfo_SequenceDetails_SequenceChunks.DiscardUnknown(m) +} + +var xxx_messageInfo_SequenceDetails_SequenceChunks proto.InternalMessageInfo + type ImportProgress struct { SamplingProgress []float32 `protobuf:"fixed32,1,rep,packed,name=sampling_progress,json=samplingProgress,proto3" json:"sampling_progress,omitempty"` ReadProgress []float32 `protobuf:"fixed32,2,rep,packed,name=read_progress,json=readProgress,proto3" json:"read_progress,omitempty"` @@ -697,13 +810,16 @@ type ImportProgress struct { // been flushed, we can advance the count here and then on resume skip over // that many rows without needing to convert/process them at all. ResumePos []int64 `protobuf:"varint,5,rep,packed,name=resume_pos,json=resumePos,proto3" json:"resume_pos,omitempty"` + // Holds metadata related to sequences for every file processed during an + // IMPORT. + SequenceDetails []*SequenceDetails `protobuf:"bytes,6,rep,name=sequence_details,json=sequenceDetails,proto3" json:"sequence_details,omitempty"` } 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_8c0ac68f84c44742, []int{8} + return fileDescriptor_jobs_98971d9474e7d77d, []int{10} } func (m *ImportProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -737,7 +853,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_8c0ac68f84c44742, []int{9} + return fileDescriptor_jobs_98971d9474e7d77d, []int{11} } func (m *TypeSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -770,7 +886,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_8c0ac68f84c44742, []int{10} + return fileDescriptor_jobs_98971d9474e7d77d, []int{12} } func (m *TypeSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -803,7 +919,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_8c0ac68f84c44742, []int{11} + return fileDescriptor_jobs_98971d9474e7d77d, []int{13} } func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -838,7 +954,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_8c0ac68f84c44742, []int{12} + return fileDescriptor_jobs_98971d9474e7d77d, []int{14} } func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -896,7 +1012,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_8c0ac68f84c44742, []int{13} + return fileDescriptor_jobs_98971d9474e7d77d, []int{15} } func (m *SchemaChangeGCDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -930,7 +1046,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_8c0ac68f84c44742, []int{13, 0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{15, 0} } func (m *SchemaChangeGCDetails_DroppedIndex) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -964,7 +1080,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_8c0ac68f84c44742, []int{13, 1} + return fileDescriptor_jobs_98971d9474e7d77d, []int{15, 1} } func (m *SchemaChangeGCDetails_DroppedID) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1028,7 +1144,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_8c0ac68f84c44742, []int{14} + return fileDescriptor_jobs_98971d9474e7d77d, []int{16} } func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1060,7 +1176,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_8c0ac68f84c44742, []int{15} + return fileDescriptor_jobs_98971d9474e7d77d, []int{17} } func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1096,7 +1212,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_8c0ac68f84c44742, []int{16} + return fileDescriptor_jobs_98971d9474e7d77d, []int{18} } func (m *SchemaChangeGCProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1130,7 +1246,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_8c0ac68f84c44742, []int{16, 0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{18, 0} } func (m *SchemaChangeGCProgress_IndexProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1164,7 +1280,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_8c0ac68f84c44742, []int{16, 1} + return fileDescriptor_jobs_98971d9474e7d77d, []int{18, 1} } func (m *SchemaChangeGCProgress_TableProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1197,7 +1313,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_8c0ac68f84c44742, []int{17} + return fileDescriptor_jobs_98971d9474e7d77d, []int{19} } func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1250,7 +1366,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_8c0ac68f84c44742, []int{18} + return fileDescriptor_jobs_98971d9474e7d77d, []int{20} } func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1285,7 +1401,7 @@ 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_8c0ac68f84c44742, []int{19} + return fileDescriptor_jobs_98971d9474e7d77d, []int{21} } func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1328,7 +1444,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_8c0ac68f84c44742, []int{20} + return fileDescriptor_jobs_98971d9474e7d77d, []int{22} } func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1372,7 +1488,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_8c0ac68f84c44742, []int{21} + return fileDescriptor_jobs_98971d9474e7d77d, []int{23} } func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1413,7 +1529,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_8c0ac68f84c44742, []int{21, 0} + return fileDescriptor_jobs_98971d9474e7d77d, []int{23, 0} } func (m *CreateStatsDetails_ColStat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1445,7 +1561,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_8c0ac68f84c44742, []int{22} + return fileDescriptor_jobs_98971d9474e7d77d, []int{24} } func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1512,7 +1628,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_8c0ac68f84c44742, []int{23} + return fileDescriptor_jobs_98971d9474e7d77d, []int{25} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1851,7 +1967,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_8c0ac68f84c44742, []int{24} + return fileDescriptor_jobs_98971d9474e7d77d, []int{26} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2257,7 +2373,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_8c0ac68f84c44742, []int{25} + return fileDescriptor_jobs_98971d9474e7d77d, []int{27} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2299,6 +2415,10 @@ func init() { proto.RegisterType((*RestoreProgress)(nil), "cockroach.sql.jobs.jobspb.RestoreProgress") proto.RegisterType((*ImportDetails)(nil), "cockroach.sql.jobs.jobspb.ImportDetails") proto.RegisterType((*ImportDetails_Table)(nil), "cockroach.sql.jobs.jobspb.ImportDetails.Table") + proto.RegisterType((*SequenceValChunk)(nil), "cockroach.sql.jobs.jobspb.SequenceValChunk") + proto.RegisterType((*SequenceDetails)(nil), "cockroach.sql.jobs.jobspb.SequenceDetails") + proto.RegisterMapType((map[int32]*SequenceDetails_SequenceChunks)(nil), "cockroach.sql.jobs.jobspb.SequenceDetails.SeqIdToChunksEntry") + proto.RegisterType((*SequenceDetails_SequenceChunks)(nil), "cockroach.sql.jobs.jobspb.SequenceDetails.SequenceChunks") proto.RegisterType((*ImportProgress)(nil), "cockroach.sql.jobs.jobspb.ImportProgress") proto.RegisterType((*TypeSchemaChangeDetails)(nil), "cockroach.sql.jobs.jobspb.TypeSchemaChangeDetails") proto.RegisterType((*TypeSchemaChangeProgress)(nil), "cockroach.sql.jobs.jobspb.TypeSchemaChangeProgress") @@ -3258,6 +3378,124 @@ func (m *ImportDetails_Table) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *SequenceValChunk) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SequenceValChunk) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.ChunkStartVal != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintJobs(dAtA, i, uint64(m.ChunkStartVal)) + } + if m.ChunkSize != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintJobs(dAtA, i, uint64(m.ChunkSize)) + } + if m.ChunkStartRow != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintJobs(dAtA, i, uint64(m.ChunkStartRow)) + } + if m.NextChunkStartRow != 0 { + dAtA[i] = 0x20 + i++ + i = encodeVarintJobs(dAtA, i, uint64(m.NextChunkStartRow)) + } + return i, nil +} + +func (m *SequenceDetails) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SequenceDetails) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.SeqIdToChunks) > 0 { + keysForSeqIdToChunks := make([]int32, 0, len(m.SeqIdToChunks)) + for k := range m.SeqIdToChunks { + keysForSeqIdToChunks = append(keysForSeqIdToChunks, int32(k)) + } + github_com_gogo_protobuf_sortkeys.Int32s(keysForSeqIdToChunks) + for _, k := range keysForSeqIdToChunks { + dAtA[i] = 0xa + i++ + v := m.SeqIdToChunks[int32(k)] + msgSize := 0 + if v != nil { + msgSize = v.Size() + msgSize += 1 + sovJobs(uint64(msgSize)) + } + mapSize := 1 + sovJobs(uint64(k)) + msgSize + i = encodeVarintJobs(dAtA, i, uint64(mapSize)) + dAtA[i] = 0x8 + i++ + i = encodeVarintJobs(dAtA, i, uint64(k)) + if v != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintJobs(dAtA, i, uint64(v.Size())) + n13, err := v.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n13 + } + } + } + return i, nil +} + +func (m *SequenceDetails_SequenceChunks) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SequenceDetails_SequenceChunks) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Chunks) > 0 { + for _, msg := range m.Chunks { + dAtA[i] = 0xa + i++ + i = encodeVarintJobs(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + func (m *ImportProgress) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -3278,8 +3516,8 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintJobs(dAtA, i, uint64(len(m.SamplingProgress)*4)) for _, num := range m.SamplingProgress { - f13 := math.Float32bits(float32(num)) - encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f13)) + f14 := math.Float32bits(float32(num)) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f14)) i += 4 } } @@ -3288,8 +3526,8 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintJobs(dAtA, i, uint64(len(m.ReadProgress)*4)) for _, num := range m.ReadProgress { - f14 := math.Float32bits(float32(num)) - encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f14)) + f15 := math.Float32bits(float32(num)) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f15)) i += 4 } } @@ -3298,8 +3536,8 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintJobs(dAtA, i, uint64(len(m.WriteProgress)*4)) for _, num := range m.WriteProgress { - f15 := math.Float32bits(float32(num)) - encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f15)) + f16 := math.Float32bits(float32(num)) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f16)) i += 4 } } @@ -3316,22 +3554,34 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { } } if len(m.ResumePos) > 0 { - dAtA17 := make([]byte, len(m.ResumePos)*10) - var j16 int + dAtA18 := make([]byte, len(m.ResumePos)*10) + var j17 int for _, num1 := range m.ResumePos { num := uint64(num1) for num >= 1<<7 { - dAtA17[j16] = uint8(uint64(num)&0x7f | 0x80) + dAtA18[j17] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j16++ + j17++ } - dAtA17[j16] = uint8(num) - j16++ + dAtA18[j17] = uint8(num) + j17++ } dAtA[i] = 0x2a i++ - i = encodeVarintJobs(dAtA, i, uint64(j16)) - i += copy(dAtA[i:], dAtA17[:j16]) + i = encodeVarintJobs(dAtA, i, uint64(j17)) + i += copy(dAtA[i:], dAtA18[:j17]) + } + if len(m.SequenceDetails) > 0 { + for _, msg := range m.SequenceDetails { + dAtA[i] = 0x32 + i++ + i = encodeVarintJobs(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } } return i, nil } @@ -3489,11 +3739,11 @@ func (m *SchemaChangeGCDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintJobs(dAtA, i, uint64(m.InterleavedTable.Size())) - n18, err := m.InterleavedTable.MarshalTo(dAtA[i:]) + n19, err := m.InterleavedTable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n18 + i += n19 } if len(m.InterleavedIndexes) > 0 { for _, msg := range m.InterleavedIndexes { @@ -3626,38 +3876,38 @@ func (m *SchemaChangeDetails) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(m.FormatVersion)) } if len(m.DroppedTypes) > 0 { - dAtA20 := make([]byte, len(m.DroppedTypes)*10) - var j19 int + dAtA21 := make([]byte, len(m.DroppedTypes)*10) + var j20 int for _, num := range m.DroppedTypes { for num >= 1<<7 { - dAtA20[j19] = uint8(uint64(num)&0x7f | 0x80) + dAtA21[j20] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j19++ + j20++ } - dAtA20[j19] = uint8(num) - j19++ + dAtA21[j20] = uint8(num) + j20++ } dAtA[i] = 0x42 i++ - i = encodeVarintJobs(dAtA, i, uint64(j19)) - i += copy(dAtA[i:], dAtA20[:j19]) + i = encodeVarintJobs(dAtA, i, uint64(j20)) + i += copy(dAtA[i:], dAtA21[:j20]) } if len(m.DroppedSchemas) > 0 { - dAtA22 := make([]byte, len(m.DroppedSchemas)*10) - var j21 int + dAtA23 := make([]byte, len(m.DroppedSchemas)*10) + var j22 int for _, num := range m.DroppedSchemas { for num >= 1<<7 { - dAtA22[j21] = uint8(uint64(num)&0x7f | 0x80) + dAtA23[j22] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j21++ + j22++ } - dAtA22[j21] = uint8(num) - j21++ + dAtA23[j22] = uint8(num) + j22++ } dAtA[i] = 0x4a i++ - i = encodeVarintJobs(dAtA, i, uint64(j21)) - i += copy(dAtA[i:], dAtA22[:j21]) + i = encodeVarintJobs(dAtA, i, uint64(j22)) + i += copy(dAtA[i:], dAtA23[:j22]) } return i, nil } @@ -3868,21 +4118,21 @@ func (m *ChangefeedDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64((&v).Size())) - n23, err := (&v).MarshalTo(dAtA[i:]) + n24, err := (&v).MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n23 + i += n24 } } dAtA[i] = 0x3a i++ i = encodeVarintJobs(dAtA, i, uint64(m.StatementTime.Size())) - n24, err := m.StatementTime.MarshalTo(dAtA[i:]) + n25, err := m.StatementTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n25 return i, nil } @@ -3904,19 +4154,19 @@ func (m *ResolvedSpan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintJobs(dAtA, i, uint64(m.Span.Size())) - n25, err := m.Span.MarshalTo(dAtA[i:]) + n26, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n25 + i += n26 dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Timestamp.Size())) - n26, err := m.Timestamp.MarshalTo(dAtA[i:]) + n27, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n27 if m.BoundaryReached { dAtA[i] = 0x18 i++ @@ -3960,11 +4210,11 @@ func (m *ChangefeedProgress) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.ProtectedTimestampRecord.Size())) - n27, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) + n28, err := m.ProtectedTimestampRecord.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n28 return i, nil } @@ -3992,11 +4242,11 @@ func (m *CreateStatsDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Table.Size())) - n28, err := m.Table.MarshalTo(dAtA[i:]) + n29, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n29 if len(m.ColumnStats) > 0 { for _, msg := range m.ColumnStats { dAtA[i] = 0x1a @@ -4019,11 +4269,11 @@ func (m *CreateStatsDetails) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintJobs(dAtA, i, uint64(m.AsOf.Size())) - n29, err := m.AsOf.MarshalTo(dAtA[i:]) + n30, err := m.AsOf.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n30 } if len(m.FQTableName) > 0 { dAtA[i] = 0x32 @@ -4056,21 +4306,21 @@ func (m *CreateStatsDetails_ColStat) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.ColumnIDs) > 0 { - dAtA31 := make([]byte, len(m.ColumnIDs)*10) - var j30 int + dAtA32 := make([]byte, len(m.ColumnIDs)*10) + var j31 int for _, num := range m.ColumnIDs { for num >= 1<<7 { - dAtA31[j30] = uint8(uint64(num)&0x7f | 0x80) + dAtA32[j31] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j30++ + j31++ } - dAtA31[j30] = uint8(num) - j30++ + dAtA32[j31] = uint8(num) + j31++ } dAtA[i] = 0xa i++ - i = encodeVarintJobs(dAtA, i, uint64(j30)) - i += copy(dAtA[i:], dAtA31[:j30]) + i = encodeVarintJobs(dAtA, i, uint64(j31)) + i += copy(dAtA[i:], dAtA32[:j31]) } if m.HasHistogram { dAtA[i] = 0x10 @@ -4156,21 +4406,21 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(m.FinishedMicros)) } if len(m.DescriptorIDs) > 0 { - dAtA33 := make([]byte, len(m.DescriptorIDs)*10) - var j32 int + dAtA34 := make([]byte, len(m.DescriptorIDs)*10) + var j33 int for _, num := range m.DescriptorIDs { for num >= 1<<7 { - dAtA33[j32] = uint8(uint64(num)&0x7f | 0x80) + dAtA34[j33] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j32++ + j33++ } - dAtA33[j32] = uint8(num) - j32++ + dAtA34[j33] = uint8(num) + j33++ } dAtA[i] = 0x32 i++ - i = encodeVarintJobs(dAtA, i, uint64(j32)) - i += copy(dAtA[i:], dAtA33[:j32]) + i = encodeVarintJobs(dAtA, i, uint64(j33)) + i += copy(dAtA[i:], dAtA34[:j33]) } if len(m.Error) > 0 { dAtA[i] = 0x42 @@ -4182,18 +4432,18 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Lease.Size())) - n34, err := m.Lease.MarshalTo(dAtA[i:]) + n35, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n35 } if m.Details != nil { - nn35, err := m.Details.MarshalTo(dAtA[i:]) + nn36, err := m.Details.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn35 + i += nn36 } if len(m.Statement) > 0 { dAtA[i] = 0x82 @@ -4237,11 +4487,11 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.FinalResumeError.Size())) - n36, err := m.FinalResumeError.MarshalTo(dAtA[i:]) + n37, err := m.FinalResumeError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n36 + i += n37 } if m.Noncancelable { dAtA[i] = 0xa0 @@ -4264,11 +4514,11 @@ func (m *Payload_Backup) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Backup.Size())) - n37, err := m.Backup.MarshalTo(dAtA[i:]) + n38, err := m.Backup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n37 + i += n38 } return i, nil } @@ -4278,11 +4528,11 @@ func (m *Payload_Restore) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Restore.Size())) - n38, err := m.Restore.MarshalTo(dAtA[i:]) + n39, err := m.Restore.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n38 + i += n39 } return i, nil } @@ -4292,11 +4542,11 @@ func (m *Payload_SchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChange.Size())) - n39, err := m.SchemaChange.MarshalTo(dAtA[i:]) + n40, err := m.SchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n40 } return i, nil } @@ -4306,11 +4556,11 @@ func (m *Payload_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Import.Size())) - n40, err := m.Import.MarshalTo(dAtA[i:]) + n41, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n40 + i += n41 } return i, nil } @@ -4320,11 +4570,11 @@ func (m *Payload_Changefeed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Changefeed.Size())) - n41, err := m.Changefeed.MarshalTo(dAtA[i:]) + n42, err := m.Changefeed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n42 } return i, nil } @@ -4334,11 +4584,11 @@ func (m *Payload_CreateStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintJobs(dAtA, i, uint64(m.CreateStats.Size())) - n42, err := m.CreateStats.MarshalTo(dAtA[i:]) + n43, err := m.CreateStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n43 } return i, nil } @@ -4350,11 +4600,11 @@ func (m *Payload_SchemaChangeGC) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChangeGC.Size())) - n43, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) + n44, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n43 + i += n44 } return i, nil } @@ -4366,11 +4616,11 @@ func (m *Payload_TypeSchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.TypeSchemaChange.Size())) - n44, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) + n45, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n44 + i += n45 } return i, nil } @@ -4390,11 +4640,11 @@ func (m *Progress) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Progress != nil { - nn45, err := m.Progress.MarshalTo(dAtA[i:]) + nn46, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn45 + i += nn46 } if m.ModifiedMicros != 0 { dAtA[i] = 0x10 @@ -4408,11 +4658,11 @@ func (m *Progress) MarshalTo(dAtA []byte) (int, error) { i += copy(dAtA[i:], m.RunningStatus) } if m.Details != nil { - nn46, err := m.Details.MarshalTo(dAtA[i:]) + nn47, err := m.Details.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn46 + i += nn47 } return i, nil } @@ -4431,11 +4681,11 @@ func (m *Progress_HighWater) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.HighWater.Size())) - n47, err := m.HighWater.MarshalTo(dAtA[i:]) + n48, err := m.HighWater.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n47 + i += n48 } return i, nil } @@ -4445,11 +4695,11 @@ func (m *Progress_Backup) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Backup.Size())) - n48, err := m.Backup.MarshalTo(dAtA[i:]) + n49, err := m.Backup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 + i += n49 } return i, nil } @@ -4459,11 +4709,11 @@ func (m *Progress_Restore) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Restore.Size())) - n49, err := m.Restore.MarshalTo(dAtA[i:]) + n50, err := m.Restore.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n50 } return i, nil } @@ -4473,11 +4723,11 @@ func (m *Progress_SchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChange.Size())) - n50, err := m.SchemaChange.MarshalTo(dAtA[i:]) + n51, err := m.SchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n51 } return i, nil } @@ -4487,11 +4737,11 @@ func (m *Progress_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Import.Size())) - n51, err := m.Import.MarshalTo(dAtA[i:]) + n52, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n52 } return i, nil } @@ -4501,11 +4751,11 @@ func (m *Progress_Changefeed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Changefeed.Size())) - n52, err := m.Changefeed.MarshalTo(dAtA[i:]) + n53, err := m.Changefeed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n53 } return i, nil } @@ -4515,11 +4765,11 @@ func (m *Progress_CreateStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintJobs(dAtA, i, uint64(m.CreateStats.Size())) - n53, err := m.CreateStats.MarshalTo(dAtA[i:]) + n54, err := m.CreateStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n54 } return i, nil } @@ -4531,11 +4781,11 @@ func (m *Progress_SchemaChangeGC) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChangeGC.Size())) - n54, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) + n55, err := m.SchemaChangeGC.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n55 } return i, nil } @@ -4547,11 +4797,11 @@ func (m *Progress_TypeSchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintJobs(dAtA, i, uint64(m.TypeSchemaChange.Size())) - n55, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) + n56, err := m.TypeSchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n56 } return i, nil } @@ -4579,21 +4829,21 @@ func (m *Job) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Progress.Size())) - n56, err := m.Progress.MarshalTo(dAtA[i:]) + n57, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n57 } if m.Payload != nil { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Payload.Size())) - n57, err := m.Payload.MarshalTo(dAtA[i:]) + n58, err := m.Payload.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n58 } return i, nil } @@ -4971,6 +5221,64 @@ func (m *ImportDetails_Table) Size() (n int) { return n } +func (m *SequenceValChunk) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ChunkStartVal != 0 { + n += 1 + sovJobs(uint64(m.ChunkStartVal)) + } + if m.ChunkSize != 0 { + n += 1 + sovJobs(uint64(m.ChunkSize)) + } + if m.ChunkStartRow != 0 { + n += 1 + sovJobs(uint64(m.ChunkStartRow)) + } + if m.NextChunkStartRow != 0 { + n += 1 + sovJobs(uint64(m.NextChunkStartRow)) + } + return n +} + +func (m *SequenceDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.SeqIdToChunks) > 0 { + for k, v := range m.SeqIdToChunks { + _ = k + _ = v + l = 0 + if v != nil { + l = v.Size() + l += 1 + sovJobs(uint64(l)) + } + mapEntrySize := 1 + sovJobs(uint64(k)) + l + n += mapEntrySize + 1 + sovJobs(uint64(mapEntrySize)) + } + } + return n +} + +func (m *SequenceDetails_SequenceChunks) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Chunks) > 0 { + for _, e := range m.Chunks { + l = e.Size() + n += 1 + l + sovJobs(uint64(l)) + } + } + return n +} + func (m *ImportProgress) Size() (n int) { if m == nil { return 0 @@ -4999,6 +5307,12 @@ func (m *ImportProgress) Size() (n int) { } n += 1 + sovJobs(uint64(l)) + l } + if len(m.SequenceDetails) > 0 { + for _, e := range m.SequenceDetails { + l = e.Size() + n += 1 + l + sovJobs(uint64(l)) + } + } return n } @@ -8234,7 +8548,376 @@ func (m *ImportDetails_Table) Unmarshal(dAtA []byte) error { } return nil } -func (m *ImportProgress) Unmarshal(dAtA []byte) error { +func (m *SequenceValChunk) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SequenceValChunk: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SequenceValChunk: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunkStartVal", wireType) + } + m.ChunkStartVal = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ChunkStartVal |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunkSize", wireType) + } + m.ChunkSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ChunkSize |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunkStartRow", wireType) + } + m.ChunkStartRow = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ChunkStartRow |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NextChunkStartRow", wireType) + } + m.NextChunkStartRow = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NextChunkStartRow |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthJobs + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SequenceDetails) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SequenceDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SequenceDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SeqIdToChunks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SeqIdToChunks == nil { + m.SeqIdToChunks = make(map[int32]*SequenceDetails_SequenceChunks) + } + var mapkey int32 + var mapvalue *SequenceDetails_SequenceChunks + for iNdEx < postIndex { + entryPreIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + if fieldNum == 1 { + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapkey |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + } else if fieldNum == 2 { + var mapmsglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + mapmsglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if mapmsglen < 0 { + return ErrInvalidLengthJobs + } + postmsgIndex := iNdEx + mapmsglen + if mapmsglen < 0 { + return ErrInvalidLengthJobs + } + if postmsgIndex > l { + return io.ErrUnexpectedEOF + } + mapvalue = &SequenceDetails_SequenceChunks{} + if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex + } else { + iNdEx = entryPreIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthJobs + } + if (iNdEx + skippy) > postIndex { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + m.SeqIdToChunks[mapkey] = mapvalue + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthJobs + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SequenceDetails_SequenceChunks) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SequenceChunks: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SequenceChunks: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Chunks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Chunks = append(m.Chunks, &SequenceValChunk{}) + if err := m.Chunks[len(m.Chunks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthJobs + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ImportProgress) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8520,6 +9203,37 @@ func (m *ImportProgress) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field ResumePos", wireType) } + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SequenceDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SequenceDetails = append(m.SequenceDetails, &SequenceDetails{}) + if err := m.SequenceDetails[len(m.SequenceDetails)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipJobs(dAtA[iNdEx:]) @@ -12352,270 +13066,282 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_8c0ac68f84c44742) } - -var fileDescriptor_jobs_8c0ac68f84c44742 = []byte{ - // 4191 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xcd, 0x6f, 0x23, 0x47, - 0x76, 0x17, 0x3f, 0x44, 0x36, 0x1f, 0x45, 0xaa, 0x59, 0xd2, 0xcc, 0x70, 0x19, 0x5b, 0x54, 0x68, - 0x7b, 0x3d, 0x33, 0xb6, 0x29, 0xaf, 0x26, 0xeb, 0xf5, 0x4e, 0xec, 0xf1, 0x8a, 0x1f, 0x92, 0x48, - 0x8d, 0x3e, 0xa6, 0x29, 0x8d, 0xd7, 0x5e, 0x38, 0x9d, 0x26, 0xbb, 0x24, 0x75, 0xd4, 0xec, 0xee, - 0xe9, 0x6a, 0xce, 0x8c, 0x16, 0x41, 0x12, 0x6c, 0x90, 0x60, 0x31, 0xa7, 0x04, 0xd8, 0xe4, 0x90, - 0x64, 0x80, 0x00, 0xc9, 0x02, 0x39, 0x04, 0x08, 0xb0, 0x08, 0x92, 0x1c, 0xf2, 0x07, 0xf8, 0x90, - 0xc3, 0x5e, 0x02, 0x2c, 0x72, 0xe0, 0x26, 0xf4, 0x25, 0x87, 0x04, 0x58, 0x24, 0xb7, 0x39, 0x05, - 0xf5, 0xd1, 0xcd, 0x26, 0xf5, 0x45, 0x8d, 0xec, 0xcd, 0x45, 0x62, 0xbd, 0x7a, 0xf5, 0xab, 0xaf, - 0xf7, 0x7e, 0xef, 0x55, 0x55, 0xc3, 0xf5, 0xdf, 0xb2, 0xdb, 0x64, 0x89, 0xfe, 0x71, 0xda, 0xec, - 0x5f, 0xd9, 0x71, 0x6d, 0xcf, 0x46, 0x5f, 0xeb, 0xd8, 0x9d, 0x23, 0xd7, 0xd6, 0x3a, 0x87, 0x65, - 0xf2, 0xc8, 0x2c, 0xb3, 0x1a, 0xae, 0x55, 0xb8, 0x86, 0x5d, 0xd7, 0x76, 0xa9, 0x3e, 0xff, 0xc1, - 0x5b, 0x14, 0xe6, 0x0f, 0xec, 0x03, 0x9b, 0xfd, 0x5c, 0xa2, 0xbf, 0x84, 0x14, 0x31, 0x0c, 0xa7, - 0xbd, 0xa4, 0x6b, 0x9e, 0x26, 0x64, 0x79, 0x5f, 0x66, 0xd8, 0xef, 0xec, 0xdb, 0x6e, 0x57, 0xf3, - 0x7c, 0x8c, 0xd7, 0xc8, 0x23, 0x73, 0xa9, 0xa3, 0x79, 0x9a, 0x69, 0x1f, 0x2c, 0xe9, 0x98, 0x74, - 0x9c, 0xf6, 0x12, 0xf1, 0xdc, 0x5e, 0xc7, 0xeb, 0xb9, 0x58, 0x17, 0x4a, 0xc5, 0x53, 0x94, 0x3c, - 0x6c, 0x69, 0x96, 0xe7, 0xe3, 0xf7, 0x3c, 0xc3, 0x5c, 0x3a, 0x34, 0x3b, 0x4b, 0x9e, 0xd1, 0xc5, - 0xc4, 0xd3, 0xba, 0x0e, 0xaf, 0x29, 0xfd, 0x2e, 0x4c, 0xdf, 0xc7, 0x1a, 0xc1, 0xe8, 0x53, 0x48, - 0x5a, 0xb6, 0x8e, 0x55, 0x43, 0xcf, 0x47, 0x16, 0x23, 0x37, 0x33, 0x95, 0x95, 0x41, 0xbf, 0x98, - 0xd8, 0xb2, 0x75, 0xdc, 0xa8, 0xbd, 0xe8, 0x17, 0xef, 0x1c, 0x18, 0xde, 0x61, 0xaf, 0x5d, 0xee, - 0xd8, 0xdd, 0xa5, 0x60, 0x21, 0xf4, 0xf6, 0xf0, 0xf7, 0x92, 0x73, 0x74, 0xb0, 0x24, 0xa6, 0x51, - 0xe6, 0xcd, 0x94, 0x04, 0x45, 0x6c, 0xe8, 0x68, 0x1e, 0xa6, 0xb1, 0x63, 0x77, 0x0e, 0xf3, 0xd1, - 0xc5, 0xc8, 0xcd, 0x98, 0xc2, 0x0b, 0x77, 0xe3, 0xff, 0xf9, 0x97, 0xc5, 0x48, 0xe9, 0xc7, 0x51, - 0xb8, 0x51, 0xd1, 0x3a, 0x47, 0x3d, 0xa7, 0x6e, 0x75, 0xdc, 0x63, 0xc7, 0x33, 0x6c, 0x6b, 0x9b, - 0xfd, 0x25, 0x48, 0x86, 0xd8, 0x11, 0x3e, 0x66, 0xe3, 0x99, 0x51, 0xe8, 0x4f, 0xf4, 0x21, 0xc4, - 0xbb, 0xb6, 0x8e, 0x19, 0x50, 0x76, 0xf9, 0x56, 0xf9, 0xcc, 0x3d, 0x29, 0x0f, 0xd1, 0x36, 0x6d, - 0x1d, 0x2b, 0xac, 0x19, 0x6a, 0x83, 0x74, 0xd4, 0x25, 0xaa, 0x61, 0xed, 0xdb, 0xf9, 0xd8, 0x62, - 0xe4, 0x66, 0x7a, 0xf9, 0xee, 0x39, 0x10, 0x67, 0x0c, 0xab, 0xbc, 0xb1, 0xd9, 0x6a, 0x58, 0xfb, - 0x76, 0x25, 0x3d, 0xe8, 0x17, 0x93, 0xa2, 0xa0, 0x24, 0x8f, 0xba, 0x84, 0xfe, 0x28, 0x6c, 0x83, - 0x2f, 0xa3, 0xe3, 0xef, 0xb9, 0x06, 0x1b, 0x7f, 0x4a, 0xa1, 0x3f, 0xd1, 0xdb, 0x80, 0x30, 0xc7, - 0xc3, 0xba, 0x4a, 0x0d, 0x40, 0xa5, 0x13, 0x8c, 0xb2, 0x09, 0xca, 0x41, 0x4d, 0x4d, 0xf3, 0xb4, - 0x0d, 0x7c, 0xcc, 0x57, 0x48, 0xac, 0xd3, 0xef, 0xc5, 0x20, 0x3b, 0x1c, 0x0a, 0x83, 0x5f, 0x87, - 0x04, 0xe9, 0x1c, 0xe2, 0x2e, 0x66, 0x3d, 0x64, 0x97, 0xdf, 0x9d, 0x68, 0x39, 0x68, 0xd3, 0x72, - 0x8b, 0xb5, 0x53, 0x44, 0x7b, 0x84, 0x20, 0x4e, 0x34, 0xd3, 0x13, 0x03, 0x61, 0xbf, 0xd1, 0x9f, - 0x47, 0x60, 0x71, 0x7c, 0x44, 0x95, 0xe3, 0x8d, 0xcd, 0xd6, 0xa6, 0x46, 0x3c, 0xec, 0x6e, 0xe0, - 0xe3, 0x46, 0x2d, 0x1f, 0x5b, 0x8c, 0xdd, 0x4c, 0x2f, 0x6f, 0x4f, 0xde, 0x71, 0xfd, 0x02, 0xc4, - 0xba, 0xe5, 0xb9, 0xc7, 0xca, 0x85, 0x1d, 0x17, 0x5a, 0xf0, 0xc6, 0x44, 0x50, 0x61, 0x1b, 0x4a, - 0x71, 0x1b, 0x9a, 0x87, 0xe9, 0xc7, 0x9a, 0xd9, 0xc3, 0x62, 0xb6, 0xbc, 0x70, 0x37, 0xfa, 0x7e, - 0xa4, 0x74, 0x03, 0x12, 0x7c, 0x61, 0x50, 0x06, 0x52, 0x2b, 0xf5, 0xd6, 0xf2, 0x37, 0xdf, 0x5b, - 0xab, 0x6e, 0xca, 0x53, 0x62, 0x0b, 0xfe, 0x39, 0x01, 0x19, 0x6e, 0x13, 0x35, 0xec, 0x69, 0x86, - 0x49, 0x50, 0x05, 0x80, 0x78, 0x9a, 0xeb, 0xa9, 0xd4, 0xad, 0x58, 0x1f, 0xe9, 0xe5, 0x57, 0x43, - 0x8b, 0x41, 0xdd, 0xae, 0x7c, 0x68, 0x76, 0xca, 0xbb, 0xbe, 0xdb, 0x55, 0xe2, 0x9f, 0xf7, 0x8b, - 0x53, 0x4a, 0x8a, 0x35, 0xa3, 0x52, 0x74, 0x0f, 0x24, 0x6c, 0xe9, 0x1c, 0x21, 0x3a, 0x39, 0x42, - 0x12, 0x5b, 0x3a, 0x6b, 0xff, 0x35, 0x6e, 0x64, 0xd4, 0x9c, 0x53, 0x95, 0xe4, 0xa0, 0x5f, 0x8c, - 0xed, 0x29, 0x0d, 0x6e, 0x6d, 0x1f, 0x40, 0x41, 0xc7, 0x8e, 0x8b, 0x3b, 0x1a, 0x35, 0xb7, 0x36, - 0x1b, 0xba, 0xda, 0xd5, 0x2c, 0x63, 0x1f, 0x13, 0x2f, 0x1f, 0x67, 0xd3, 0xcf, 0x0f, 0x35, 0xf8, - 0xdc, 0x36, 0x45, 0x3d, 0xfa, 0xfd, 0x08, 0xcc, 0xf5, 0x5c, 0x83, 0xa8, 0xed, 0x63, 0xd5, 0xb4, - 0x3b, 0x9a, 0x69, 0x78, 0xc7, 0xea, 0xd1, 0xe3, 0xfc, 0x34, 0xdb, 0xf3, 0x7b, 0x17, 0x3a, 0x8e, - 0x58, 0xa4, 0xf2, 0x9e, 0x6b, 0x90, 0xca, 0xf1, 0x7d, 0x81, 0xb0, 0xf1, 0x98, 0xed, 0x4b, 0x65, - 0x7e, 0xd0, 0x2f, 0xca, 0x7b, 0x4a, 0x23, 0x5c, 0xf5, 0x50, 0x91, 0x7b, 0x63, 0xca, 0x48, 0x0b, - 0x3c, 0xc6, 0xb0, 0x2d, 0xd5, 0xe6, 0x2e, 0x98, 0x4f, 0xb0, 0x85, 0x5a, 0xbe, 0xbc, 0xf3, 0x2a, - 0x39, 0x7c, 0x82, 0x66, 0xfe, 0x38, 0x02, 0x05, 0xca, 0x86, 0xb8, 0x43, 0x97, 0x29, 0x60, 0x48, - 0xd5, 0xc5, 0x1d, 0xdb, 0xd5, 0xf3, 0x49, 0xba, 0x4e, 0x95, 0xd6, 0xbf, 0x4d, 0x4a, 0x82, 0x8c, - 0x6b, 0x7b, 0x3d, 0x43, 0x2f, 0xef, 0xed, 0x35, 0x6a, 0x83, 0x7e, 0x31, 0xbf, 0xe3, 0x83, 0x07, - 0x9b, 0xa8, 0x30, 0x68, 0x25, 0xef, 0x9c, 0x51, 0x83, 0xde, 0x87, 0x6c, 0xc7, 0x36, 0x4d, 0xdc, - 0x61, 0xd3, 0xde, 0x53, 0x1a, 0x79, 0x89, 0x6d, 0x70, 0x6e, 0xd0, 0x2f, 0x66, 0xaa, 0x41, 0x0d, - 0xdd, 0xea, 0x4c, 0x27, 0x5c, 0x44, 0x0a, 0xcc, 0x86, 0x16, 0x8c, 0x51, 0x5d, 0x8a, 0xad, 0xd6, - 0xad, 0x89, 0xbd, 0x54, 0xc9, 0xe2, 0x91, 0x72, 0xa1, 0x0a, 0xd7, 0x4e, 0xdd, 0xc5, 0x8b, 0xbc, - 0x2b, 0x15, 0xf6, 0x2e, 0x19, 0xb2, 0x7c, 0x53, 0x76, 0x5c, 0xfb, 0xc0, 0xc5, 0x84, 0x94, 0x7e, - 0x92, 0x85, 0xac, 0x82, 0x89, 0x67, 0xbb, 0xd8, 0xf7, 0xa8, 0x9f, 0x44, 0x60, 0x8e, 0x46, 0x30, - 0xd7, 0x70, 0x3c, 0xdb, 0x55, 0x5d, 0xfc, 0xc4, 0x35, 0x3c, 0x4c, 0xf2, 0x51, 0x66, 0x74, 0x2b, - 0xe7, 0x4c, 0x61, 0x14, 0xa8, 0x5c, 0x0b, 0x40, 0x14, 0x81, 0xc1, 0xed, 0xee, 0xde, 0x0f, 0x7e, - 0x5e, 0xbc, 0x3b, 0xd1, 0x3e, 0x9e, 0x0c, 0xaa, 0xe5, 0x46, 0x4d, 0x41, 0xfa, 0x09, 0x60, 0xf4, - 0x0a, 0xc4, 0xa9, 0xdd, 0x32, 0x36, 0x4c, 0x55, 0xa4, 0x41, 0xbf, 0x18, 0xa7, 0x96, 0xad, 0x30, - 0xe9, 0x88, 0x83, 0xc7, 0x5f, 0xc2, 0xc1, 0xd7, 0x20, 0xed, 0x69, 0x6d, 0x13, 0xab, 0xb4, 0x67, - 0x22, 0xdc, 0xef, 0xeb, 0x63, 0x2b, 0x41, 0x1e, 0x99, 0x6d, 0x8d, 0xe0, 0xf2, 0x2e, 0xd5, 0x0c, - 0xcd, 0x1d, 0x3c, 0x5f, 0x40, 0xd0, 0x12, 0xa4, 0xed, 0xc7, 0xd8, 0x75, 0x0d, 0x1d, 0xab, 0x7a, - 0x9b, 0xf9, 0x50, 0xaa, 0x92, 0x1d, 0xf4, 0x8b, 0xb0, 0x2d, 0xc4, 0xb5, 0x8a, 0x02, 0xbe, 0x4a, - 0xad, 0x8d, 0x3c, 0x98, 0x17, 0xa4, 0x11, 0xf8, 0x3f, 0xb3, 0xa7, 0x24, 0x1b, 0xc2, 0x07, 0x93, - 0x6f, 0x06, 0xdf, 0x77, 0xdf, 0x78, 0x58, 0xf0, 0xe4, 0x93, 0x44, 0xed, 0x13, 0x35, 0xe8, 0x2d, - 0xc8, 0x39, 0x2e, 0x76, 0x34, 0x17, 0xab, 0x1d, 0xbb, 0xeb, 0x98, 0xd8, 0xc3, 0x3a, 0xb3, 0x7e, - 0x49, 0x91, 0x45, 0x45, 0xd5, 0x97, 0xa3, 0x37, 0x20, 0x4b, 0x3c, 0xcd, 0xa3, 0x31, 0x9d, 0x60, - 0x97, 0x6a, 0xa6, 0x98, 0x66, 0x86, 0x49, 0x1b, 0x42, 0x88, 0xee, 0xc0, 0xb5, 0xe1, 0xbe, 0x11, - 0xd5, 0xe9, 0xb5, 0x4d, 0x83, 0x1c, 0x62, 0x3d, 0x0f, 0x4c, 0x7b, 0x3e, 0x54, 0xb9, 0xe3, 0xd7, - 0xa1, 0xe3, 0x11, 0x53, 0xec, 0xd0, 0x85, 0xd1, 0x0e, 0x70, 0x3e, 0xbd, 0x18, 0xb9, 0x39, 0x5d, - 0x59, 0x7f, 0xd1, 0x2f, 0xd6, 0x26, 0xb6, 0x23, 0x82, 0xbb, 0x4b, 0x9e, 0x8b, 0x71, 0xc8, 0x2c, - 0xab, 0x02, 0x2f, 0x6c, 0x51, 0xbe, 0x0c, 0x29, 0x00, 0x43, 0x17, 0xcc, 0xcf, 0xbc, 0x34, 0xdb, - 0x85, 0x50, 0xd0, 0x0a, 0x24, 0x79, 0x52, 0x48, 0xf2, 0x19, 0xb6, 0x81, 0xbf, 0x7a, 0x96, 0x0d, - 0x31, 0xad, 0xd0, 0x2e, 0xf9, 0xed, 0x50, 0x0d, 0xc0, 0x3b, 0x76, 0x7c, 0x4b, 0xcc, 0x32, 0x94, - 0x37, 0xce, 0x42, 0x39, 0x76, 0xc2, 0x86, 0x98, 0xf2, 0x44, 0x99, 0xa0, 0x26, 0xcc, 0xb0, 0xbc, - 0x43, 0x13, 0x38, 0xb3, 0x0c, 0xe7, 0xcd, 0x33, 0x70, 0x58, 0x44, 0xd6, 0x42, 0x48, 0x69, 0x12, - 0x48, 0x08, 0xda, 0x81, 0x2c, 0x4d, 0xa3, 0xa8, 0xa6, 0x40, 0x93, 0x19, 0xda, 0xad, 0x33, 0xd0, - 0x6a, 0x42, 0x39, 0x84, 0x97, 0xd1, 0x43, 0x32, 0x52, 0xf8, 0xdf, 0x08, 0xe4, 0x4e, 0x90, 0x07, - 0xda, 0x85, 0x68, 0x90, 0x19, 0x53, 0x4e, 0x8f, 0xb2, 0xac, 0xf8, 0x2a, 0x44, 0x12, 0x35, 0x74, - 0x74, 0x00, 0x29, 0x6a, 0xce, 0x96, 0x47, 0xd3, 0xee, 0x28, 0x03, 0x6f, 0x0e, 0xfa, 0x45, 0x69, - 0x87, 0x09, 0xaf, 0xdc, 0x85, 0xc4, 0xc1, 0x1b, 0x3a, 0x2a, 0x42, 0xda, 0xb3, 0x55, 0xfc, 0xd4, - 0x20, 0x9e, 0x61, 0x1d, 0xb0, 0x64, 0x41, 0x52, 0xc0, 0xb3, 0xeb, 0x42, 0x52, 0xf8, 0x8b, 0x28, - 0xa0, 0x93, 0x5e, 0x8a, 0xfe, 0x29, 0x02, 0xaf, 0xf8, 0x39, 0x80, 0xed, 0x1a, 0x07, 0x86, 0xa5, - 0x99, 0x23, 0xc9, 0x40, 0x84, 0xad, 0xf6, 0xa7, 0x57, 0xa1, 0x02, 0x91, 0x20, 0x6c, 0x0b, 0xf8, - 0xf1, 0x44, 0xe1, 0x15, 0x1a, 0x41, 0x79, 0xa2, 0x70, 0x42, 0xe5, 0xa1, 0x92, 0xef, 0x9d, 0xd1, - 0xb8, 0xb0, 0x01, 0xaf, 0x9e, 0x0b, 0x7c, 0x99, 0xd8, 0x55, 0xf8, 0x41, 0x04, 0x6e, 0x9c, 0x11, - 0x51, 0xc2, 0x38, 0x19, 0x8e, 0xf3, 0x20, 0x8c, 0x93, 0x5e, 0xfe, 0xf5, 0x2b, 0x44, 0xad, 0xd0, - 0x20, 0x9a, 0x71, 0x29, 0x22, 0x47, 0x4b, 0xef, 0xc2, 0xac, 0x68, 0xe4, 0xc7, 0x51, 0xf4, 0x2a, - 0xc0, 0xa1, 0x71, 0x70, 0xa8, 0x3e, 0xd1, 0x3c, 0xec, 0x8a, 0xe3, 0x52, 0x8a, 0x4a, 0x3e, 0xa6, - 0x82, 0xd2, 0xbf, 0x4a, 0x90, 0x69, 0x74, 0x1d, 0xdb, 0xf5, 0xfc, 0x28, 0x7b, 0x1f, 0x12, 0x2c, - 0x2e, 0x10, 0xb1, 0x7f, 0xe5, 0x73, 0x46, 0x38, 0xd2, 0x92, 0xc7, 0x17, 0x41, 0x0b, 0x02, 0x23, - 0x08, 0x7f, 0xd1, 0x53, 0xc3, 0xdf, 0x87, 0x90, 0xe0, 0x47, 0x5a, 0x71, 0xe2, 0x2a, 0x86, 0xfa, - 0xf2, 0xcf, 0x8b, 0x8d, 0xed, 0x55, 0xc3, 0xc4, 0xab, 0x4c, 0xcd, 0x07, 0xe7, 0x8d, 0xd0, 0xd7, - 0x41, 0x22, 0xc4, 0x53, 0x89, 0xf1, 0x7d, 0x1e, 0x3d, 0x63, 0xfc, 0xd8, 0xd5, 0x6a, 0xed, 0xb6, - 0x8c, 0xef, 0x63, 0x25, 0x49, 0x88, 0x47, 0x7f, 0xa0, 0x02, 0x48, 0x4f, 0x34, 0xd3, 0x64, 0x51, - 0x76, 0x9a, 0x1d, 0x33, 0x83, 0xf2, 0xa8, 0x9b, 0x25, 0xbe, 0x5a, 0x37, 0x13, 0x01, 0xd3, 0xd1, - 0xbc, 0x43, 0x96, 0x39, 0xa6, 0x14, 0xe0, 0xa2, 0x1d, 0xcd, 0x3b, 0x44, 0x79, 0x48, 0x12, 0x8d, - 0xc6, 0x2e, 0x92, 0x97, 0x16, 0x63, 0x37, 0x67, 0x14, 0xbf, 0x88, 0x16, 0x80, 0x45, 0x5e, 0x5e, - 0x64, 0x41, 0x2c, 0xa6, 0x84, 0x24, 0x6c, 0x1d, 0x8e, 0x0c, 0x47, 0xdd, 0x3f, 0x22, 0x3c, 0x68, - 0x89, 0x75, 0x38, 0x32, 0x9c, 0xd5, 0x0d, 0xa2, 0x24, 0x69, 0xe5, 0xea, 0x11, 0x41, 0x6f, 0xc2, - 0xac, 0x61, 0x1d, 0x60, 0xe2, 0xa9, 0xba, 0xe1, 0xe2, 0x8e, 0x67, 0x1e, 0xb3, 0x80, 0x25, 0x29, - 0x59, 0x2e, 0xae, 0x09, 0x29, 0xba, 0x05, 0xf2, 0x78, 0x98, 0x65, 0x81, 0x46, 0x52, 0x66, 0xc7, - 0xa2, 0x2c, 0x55, 0xe5, 0x5b, 0x1d, 0x0a, 0x9c, 0x19, 0xae, 0xca, 0xe5, 0xc3, 0x98, 0x59, 0x86, - 0x39, 0x47, 0x73, 0x09, 0x56, 0xdb, 0x3d, 0x4b, 0x37, 0xb1, 0xca, 0xb9, 0x3a, 0x9f, 0x65, 0xda, - 0x39, 0x56, 0x55, 0x61, 0x35, 0x9c, 0xd6, 0x2f, 0xca, 0xbd, 0xaf, 0xff, 0x3f, 0xe4, 0xde, 0x85, - 0x1f, 0x47, 0x61, 0x9a, 0xd9, 0x39, 0xba, 0x0b, 0x71, 0xba, 0xcd, 0xe2, 0x64, 0x37, 0x69, 0xce, - 0xc5, 0xda, 0xd0, 0x33, 0xb5, 0xa5, 0x75, 0x71, 0x1e, 0x31, 0x23, 0x60, 0xbf, 0xd1, 0x0d, 0x48, - 0x12, 0xfc, 0x48, 0x7d, 0xac, 0x99, 0xf9, 0x39, 0xb6, 0xc3, 0x09, 0x82, 0x1f, 0x3d, 0xd4, 0x4c, - 0x74, 0x0d, 0x12, 0x06, 0x51, 0x2d, 0xfc, 0x24, 0x3f, 0xcf, 0x56, 0x6a, 0xda, 0x20, 0x5b, 0xf8, - 0x09, 0xa3, 0x6d, 0xcd, 0x3d, 0xc0, 0x9e, 0xda, 0xb1, 0x4d, 0x92, 0xbf, 0x46, 0x1d, 0x8c, 0xa6, - 0x74, 0x54, 0x54, 0xb5, 0x4d, 0x82, 0x7e, 0x05, 0x52, 0x4f, 0x34, 0xa2, 0xe2, 0xae, 0xe3, 0x1d, - 0xb3, 0xc5, 0x92, 0xa8, 0xd9, 0x93, 0x3a, 0x2d, 0x37, 0xe3, 0x52, 0x54, 0x8e, 0x35, 0xe3, 0x52, - 0x4c, 0x8e, 0x37, 0xe3, 0x52, 0x5c, 0x9e, 0x6e, 0xc6, 0xa5, 0x69, 0x39, 0xd1, 0x8c, 0x4b, 0x09, - 0x39, 0xd9, 0x8c, 0x4b, 0x49, 0x59, 0x6a, 0xc6, 0x25, 0x49, 0x4e, 0x35, 0xe3, 0x52, 0x4a, 0x86, - 0x66, 0x5c, 0x02, 0x39, 0xdd, 0x8c, 0x4b, 0x69, 0x79, 0xa6, 0x19, 0x97, 0x66, 0xe4, 0x4c, 0x33, - 0x2e, 0x65, 0xe4, 0x6c, 0x33, 0x2e, 0x65, 0xe5, 0xd9, 0x66, 0x5c, 0x9a, 0x95, 0xe5, 0x66, 0x5c, - 0x92, 0xe5, 0x5c, 0x33, 0x2e, 0xe5, 0x64, 0x54, 0xfa, 0xaf, 0x08, 0x64, 0x39, 0x3b, 0x04, 0x4c, - 0xf4, 0x16, 0xe4, 0x98, 0xbd, 0x1a, 0xd6, 0x81, 0xea, 0x08, 0x21, 0xe3, 0x98, 0xa8, 0x22, 0xfb, - 0x15, 0x81, 0xf2, 0x6b, 0x90, 0x71, 0xb1, 0xa6, 0x0f, 0x15, 0xa3, 0x4c, 0x71, 0x86, 0x0a, 0x03, - 0xa5, 0x37, 0x20, 0xcb, 0x88, 0x70, 0xa8, 0x15, 0x63, 0x5a, 0x19, 0x26, 0x0d, 0xd4, 0x2a, 0x90, - 0x21, 0x8e, 0x66, 0x0d, 0xb5, 0xe2, 0x8c, 0xd8, 0x6e, 0x9c, 0x42, 0x36, 0x2d, 0x47, 0xb3, 0x04, - 0xc9, 0xcc, 0xd0, 0x36, 0x61, 0x1a, 0x75, 0x31, 0xe9, 0x75, 0xb1, 0xea, 0xd8, 0x3c, 0xcf, 0x8e, - 0x29, 0x29, 0x2e, 0xd9, 0xb1, 0x49, 0xe9, 0xb7, 0xe1, 0x06, 0xcd, 0x69, 0xb8, 0xe1, 0x56, 0x0f, - 0x35, 0xeb, 0x20, 0x38, 0xb5, 0x68, 0x90, 0x64, 0x79, 0x51, 0x90, 0x22, 0xac, 0x0f, 0xfa, 0xc5, - 0x04, 0xd5, 0xbe, 0x32, 0xb9, 0x24, 0x28, 0x70, 0x43, 0x2f, 0x15, 0x20, 0x3f, 0xde, 0x7b, 0x70, - 0x8e, 0x52, 0xd8, 0x31, 0xaa, 0xd7, 0xc5, 0x74, 0x6a, 0xf7, 0x0d, 0xe2, 0xa1, 0xef, 0xc0, 0x8c, - 0x98, 0x0a, 0x9d, 0xa1, 0x4f, 0xf3, 0x17, 0xac, 0x46, 0xda, 0x0d, 0x40, 0x48, 0xe9, 0xef, 0x23, - 0x30, 0x57, 0x73, 0x6d, 0xc7, 0xc1, 0xba, 0xb0, 0x6f, 0x3e, 0x55, 0xdf, 0xac, 0x23, 0x21, 0xb3, - 0xde, 0x82, 0x68, 0xa3, 0x26, 0xf2, 0x97, 0x7b, 0x57, 0x4d, 0x8b, 0x1a, 0x35, 0xf4, 0x6d, 0x48, - 0xd0, 0xf4, 0xbd, 0x47, 0x58, 0xc8, 0xc8, 0x9e, 0x48, 0x54, 0xc3, 0xe1, 0xa9, 0xc5, 0x14, 0x15, - 0xd1, 0xa0, 0xf4, 0x87, 0x09, 0xb8, 0x16, 0x5e, 0xa3, 0xb5, 0xaa, 0x3f, 0xf0, 0xcf, 0x20, 0x69, - 0x58, 0x3a, 0x7e, 0x1a, 0x04, 0xbd, 0x0f, 0xcf, 0x43, 0x3d, 0x0d, 0xa2, 0x2c, 0xd6, 0xa3, 0x41, - 0x61, 0xfc, 0xd4, 0x58, 0x60, 0xa2, 0xef, 0x06, 0x21, 0x95, 0x1f, 0x55, 0xef, 0xbe, 0x34, 0x7a, - 0x6d, 0x2c, 0xbc, 0x8e, 0x44, 0xaf, 0x18, 0x0b, 0x81, 0x5f, 0x4d, 0xf4, 0x6a, 0x41, 0xce, 0xb0, - 0x3c, 0xec, 0x9a, 0x58, 0x7b, 0x4c, 0xc9, 0x98, 0x76, 0x2f, 0x4e, 0xac, 0x93, 0x52, 0x9f, 0x1c, - 0x02, 0xe0, 0x14, 0xfa, 0x19, 0xcc, 0x85, 0x41, 0xfd, 0x2d, 0x38, 0xff, 0x14, 0xcb, 0x56, 0x78, - 0x08, 0xeb, 0x1f, 0x16, 0x43, 0x40, 0x0d, 0x8e, 0x53, 0xf8, 0xd3, 0x08, 0xcc, 0x84, 0xb7, 0x05, - 0x19, 0x20, 0xb1, 0x3e, 0x7c, 0x5f, 0x8c, 0x55, 0xb6, 0x68, 0x9c, 0x64, 0x95, 0x6c, 0xad, 0x3e, - 0x7a, 0xe9, 0xb5, 0xe2, 0x10, 0x62, 0xcb, 0x1b, 0x3a, 0x25, 0x5f, 0xdd, 0xb5, 0x9d, 0xe1, 0xd5, - 0x5d, 0x4c, 0x91, 0xa8, 0x80, 0xc6, 0x93, 0xc2, 0xef, 0x40, 0x2a, 0xd8, 0xd0, 0xd0, 0xe9, 0x21, - 0xf6, 0x25, 0x9e, 0x1e, 0xce, 0xeb, 0xbf, 0xf4, 0xf3, 0x04, 0xcc, 0x9d, 0x46, 0x55, 0x9f, 0x80, - 0x1c, 0x62, 0x06, 0xd5, 0x34, 0x88, 0x27, 0x2c, 0xf6, 0xd6, 0xf9, 0x69, 0x6a, 0x88, 0x5e, 0xc4, - 0x7e, 0x64, 0xdd, 0x51, 0xd2, 0xf9, 0x1e, 0x64, 0x75, 0x3e, 0x65, 0x55, 0xb8, 0x42, 0xec, 0xc2, - 0xec, 0xf2, 0x14, 0x8a, 0x11, 0xe8, 0x19, 0x3d, 0x54, 0x45, 0xd8, 0x6d, 0xa4, 0x8f, 0x1e, 0x9c, - 0xf8, 0x0c, 0x9d, 0xd9, 0x67, 0xa6, 0xd2, 0x1a, 0xf4, 0x8b, 0x39, 0x81, 0xe5, 0x1f, 0xf1, 0xae, - 0xbc, 0xc6, 0x39, 0x7d, 0x0c, 0x50, 0xa7, 0x44, 0x4f, 0xeb, 0x69, 0xc7, 0xd3, 0x43, 0xa2, 0xa7, - 0x96, 0x7a, 0x75, 0xa2, 0xa7, 0x3f, 0x1b, 0x3a, 0xfa, 0x83, 0x08, 0xe4, 0xf8, 0x7d, 0x4f, 0xb7, - 0xe7, 0x69, 0xfc, 0x12, 0xcf, 0xcf, 0x5a, 0x3f, 0x19, 0xf4, 0x8b, 0xb3, 0x6c, 0x41, 0x36, 0x45, - 0x1d, 0xeb, 0xb6, 0xf2, 0xb2, 0xdd, 0x0e, 0x51, 0x44, 0x26, 0x17, 0x08, 0x74, 0xb4, 0x01, 0x59, - 0x9e, 0x82, 0xab, 0x34, 0x09, 0x35, 0x6c, 0x8b, 0xa5, 0xb3, 0x99, 0xca, 0xeb, 0x2f, 0xfa, 0xc5, - 0xc5, 0x53, 0x2c, 0x8b, 0x67, 0xef, 0x0f, 0xb9, 0xae, 0x92, 0xd9, 0x0f, 0x17, 0x51, 0x07, 0x32, - 0x81, 0x69, 0x1c, 0x3b, 0x22, 0xfb, 0xbd, 0x7a, 0xb0, 0x98, 0xf1, 0x6d, 0x84, 0x62, 0xa2, 0x03, - 0x98, 0xf5, 0x3b, 0xe1, 0x69, 0x27, 0xc9, 0xa7, 0xbe, 0x94, 0x6e, 0x7c, 0xb3, 0xe6, 0xb3, 0x26, - 0xe2, 0x20, 0x76, 0x1d, 0xe6, 0x4f, 0x8d, 0xc6, 0x7f, 0x36, 0x0d, 0xd7, 0x47, 0x19, 0x3e, 0xc8, - 0x30, 0xd4, 0xf1, 0x18, 0xf4, 0xd1, 0xc4, 0x51, 0xc2, 0xc7, 0xe0, 0x34, 0xe4, 0x97, 0xc6, 0xa3, - 0xd0, 0x67, 0x63, 0x51, 0xe8, 0x25, 0xf0, 0x99, 0x79, 0x8d, 0xe1, 0x0b, 0xd0, 0xc2, 0xbf, 0x44, - 0x20, 0x33, 0xd2, 0xff, 0x2f, 0x93, 0x6e, 0x77, 0x82, 0xac, 0x80, 0xbf, 0xfe, 0xbd, 0x7f, 0xf9, - 0xb9, 0x8d, 0x26, 0x0b, 0x85, 0x7f, 0x8c, 0x40, 0x66, 0x64, 0xba, 0x5f, 0x11, 0x51, 0x7f, 0xe9, - 0x23, 0x2f, 0x7d, 0x0b, 0x12, 0x5c, 0x82, 0x10, 0x64, 0x3f, 0x5e, 0x69, 0xec, 0x36, 0xb6, 0xd6, - 0xd4, 0xd5, 0x6d, 0x45, 0x5d, 0xab, 0xca, 0x53, 0x68, 0x06, 0xa4, 0x5a, 0xfd, 0x7e, 0x9d, 0x0a, - 0xe5, 0x08, 0x4a, 0x43, 0x92, 0x95, 0xea, 0x35, 0x39, 0x5a, 0xaa, 0x80, 0xcc, 0xb1, 0xf7, 0x31, - 0xa5, 0x56, 0x7a, 0x90, 0xa0, 0x67, 0x36, 0x0a, 0x8b, 0xbb, 0x34, 0xc7, 0xa0, 0xc1, 0x44, 0x0d, - 0x65, 0x78, 0xb9, 0xa0, 0x8a, 0x86, 0x95, 0x2d, 0xad, 0x8b, 0x4b, 0xff, 0x10, 0x87, 0xdc, 0x10, - 0xc4, 0x0f, 0x2c, 0xf4, 0x80, 0x6a, 0x58, 0x47, 0xea, 0xf0, 0x31, 0x8a, 0x1f, 0x50, 0x0d, 0xeb, - 0x68, 0x4f, 0x69, 0x28, 0x49, 0x5a, 0xb9, 0xe7, 0x1a, 0xa8, 0x09, 0x71, 0xdb, 0xf1, 0xfc, 0x04, - 0xfd, 0xbd, 0x73, 0x96, 0xe2, 0x44, 0x1f, 0xe5, 0x6d, 0xc7, 0xe3, 0x97, 0x2e, 0x0a, 0xc3, 0x40, - 0x7f, 0x1b, 0x81, 0x24, 0x3f, 0x0d, 0x91, 0x7c, 0x82, 0xe1, 0x7d, 0xfb, 0x52, 0x78, 0x7c, 0x01, - 0xc4, 0xcb, 0xc0, 0xc7, 0xd4, 0xd4, 0x5f, 0xf4, 0x8b, 0xb9, 0xf1, 0x05, 0x22, 0x57, 0x7c, 0x32, - 0xf0, 0x87, 0x88, 0x9a, 0xfc, 0xb2, 0x7a, 0xb8, 0xd0, 0x8c, 0x52, 0x27, 0x7c, 0x0f, 0xc8, 0x8c, - 0x6c, 0x44, 0xe1, 0x00, 0x66, 0xc2, 0xa3, 0x3f, 0xe5, 0x16, 0x6a, 0x65, 0xf4, 0x16, 0xea, 0xad, - 0x89, 0x56, 0x86, 0x63, 0x86, 0xaf, 0xbe, 0xbe, 0x05, 0xa9, 0x60, 0xd9, 0x2f, 0x73, 0x67, 0xc6, - 0x59, 0x32, 0x38, 0x84, 0x4e, 0xcb, 0x89, 0xd2, 0xdf, 0x45, 0x60, 0x46, 0xc1, 0xc4, 0x36, 0x1f, - 0x63, 0x9d, 0x66, 0x0d, 0xe8, 0x1b, 0x10, 0xa7, 0x59, 0x88, 0x38, 0x5f, 0x5f, 0x70, 0x3c, 0x61, - 0xaa, 0x68, 0x05, 0x52, 0xc1, 0x2d, 0xc1, 0x65, 0xde, 0x4b, 0x87, 0xad, 0xd0, 0x2d, 0x90, 0xdb, - 0x76, 0xcf, 0xd2, 0x35, 0xf7, 0x58, 0x75, 0xb1, 0xd6, 0x39, 0xc4, 0xba, 0xb8, 0x11, 0x9d, 0xf5, - 0xe5, 0x0a, 0x17, 0x97, 0x7e, 0x18, 0x05, 0x34, 0x5c, 0x9c, 0x10, 0x4d, 0xd0, 0xdc, 0x87, 0xcd, - 0x43, 0x1c, 0xb0, 0xa2, 0xa7, 0xde, 0x61, 0x8f, 0xa5, 0x50, 0xc1, 0xc4, 0xfd, 0x2d, 0x75, 0x43, - 0x32, 0x82, 0xfe, 0xe4, 0xfc, 0xbb, 0x90, 0x18, 0xbb, 0x0b, 0x61, 0x56, 0xfa, 0x4b, 0xbd, 0x0f, - 0x11, 0xe1, 0xee, 0x7f, 0xe2, 0x80, 0xaa, 0x2e, 0xd6, 0x3c, 0x4c, 0x99, 0x87, 0x9c, 0x77, 0x1e, - 0xac, 0xc0, 0x34, 0x3f, 0x3c, 0x44, 0x2f, 0x73, 0x78, 0x10, 0x8b, 0xc2, 0x9b, 0xa2, 0xdf, 0x80, - 0x99, 0x8e, 0x6d, 0xf6, 0xba, 0x96, 0xca, 0x5e, 0x72, 0x44, 0x2a, 0xf9, 0xcd, 0xf3, 0x8c, 0xf8, - 0xc4, 0xe0, 0xca, 0x55, 0xdb, 0xa4, 0x65, 0xff, 0x7c, 0xcb, 0x01, 0x99, 0x06, 0x7a, 0x05, 0x52, - 0x81, 0x43, 0xb1, 0x24, 0x32, 0xa5, 0x0c, 0x05, 0x68, 0x19, 0xa6, 0x35, 0xa2, 0xda, 0xfb, 0x2c, - 0xcb, 0xbb, 0xc8, 0xc2, 0x94, 0xb8, 0x46, 0xb6, 0xf7, 0xd1, 0x1d, 0xc8, 0xec, 0x3f, 0xe2, 0x99, - 0x2f, 0x27, 0x50, 0xfe, 0xc0, 0x36, 0x3b, 0xe8, 0x17, 0xd3, 0xab, 0x0f, 0xd8, 0x64, 0x29, 0x7d, - 0x2a, 0xe9, 0xfd, 0x47, 0x41, 0x01, 0xdd, 0x86, 0x5c, 0x57, 0x7b, 0xaa, 0xee, 0xbb, 0x5a, 0x47, - 0xa4, 0x7a, 0x26, 0x67, 0x85, 0x88, 0x32, 0xdb, 0xd5, 0x9e, 0xae, 0x0a, 0x79, 0x43, 0x37, 0x71, - 0xe1, 0x17, 0x11, 0x48, 0x8a, 0x19, 0x21, 0x07, 0x40, 0x2c, 0x8f, 0xa1, 0xf3, 0x64, 0x22, 0x53, - 0x79, 0x30, 0xe8, 0x17, 0x53, 0x55, 0x26, 0x6d, 0xd4, 0xc8, 0x8b, 0x7e, 0xf1, 0x3b, 0x2f, 0x4b, - 0x5a, 0x3e, 0x88, 0x92, 0xe2, 0x9d, 0x34, 0x74, 0x76, 0x5b, 0x73, 0xa8, 0x11, 0xf5, 0xd0, 0x20, - 0x9e, 0x7d, 0xe0, 0x6a, 0x5d, 0xb6, 0xb9, 0x92, 0x32, 0x73, 0xa8, 0x91, 0x75, 0x5f, 0x86, 0x0a, - 0x34, 0x1d, 0x78, 0xcc, 0x1f, 0xe2, 0xb8, 0x4b, 0x05, 0x65, 0xb4, 0x0c, 0xd7, 0x82, 0xc6, 0x2a, - 0x9d, 0x74, 0xbb, 0xd7, 0x39, 0xc2, 0x2c, 0x12, 0x50, 0xce, 0x9a, 0x0b, 0x2a, 0x37, 0xb5, 0xa7, - 0x15, 0x5e, 0x55, 0xba, 0x06, 0x73, 0xa1, 0x6d, 0x0d, 0x52, 0xac, 0xff, 0x06, 0x48, 0xee, 0x68, - 0xc7, 0xa6, 0xad, 0xe9, 0x68, 0x11, 0xd2, 0xfe, 0x03, 0x1a, 0x4d, 0x52, 0xb9, 0x1d, 0x86, 0x45, - 0xc8, 0x80, 0x6c, 0x8f, 0x60, 0x97, 0xee, 0x89, 0xca, 0xbe, 0x7a, 0xe2, 0x5c, 0x55, 0xa9, 0xbc, - 0xe8, 0x17, 0xef, 0x4d, 0xb6, 0x44, 0xb8, 0xd3, 0x73, 0x0d, 0xef, 0xb8, 0xdc, 0x7a, 0x70, 0x7f, - 0x4f, 0x40, 0x51, 0x47, 0xb2, 0x95, 0x4c, 0x2f, 0x5c, 0x14, 0xcf, 0x91, 0x74, 0xba, 0x6a, 0xd7, - 0xe8, 0xb8, 0x36, 0xbf, 0xc1, 0x88, 0x31, 0xf2, 0xa6, 0xd2, 0x4d, 0x26, 0x44, 0x6f, 0xc2, 0xec, - 0xbe, 0x61, 0xb1, 0x1b, 0x53, 0x5f, 0x8f, 0xdd, 0x6d, 0x2b, 0x59, 0x5f, 0x2c, 0x14, 0x1f, 0x43, - 0x36, 0xf4, 0x04, 0x49, 0xb7, 0x3a, 0xc1, 0xb6, 0x7a, 0x7b, 0xd0, 0x2f, 0x66, 0x86, 0xae, 0xc3, - 0xb7, 0xfb, 0x2a, 0x31, 0x2a, 0x33, 0xec, 0x86, 0x6e, 0xf6, 0x3c, 0x4c, 0xb3, 0x4f, 0xd9, 0xf8, - 0x57, 0x07, 0x0a, 0x2f, 0xa0, 0xf7, 0x60, 0xda, 0xc4, 0x1a, 0xc1, 0xe2, 0x83, 0x82, 0xc5, 0x73, - 0x9c, 0x91, 0x7d, 0x54, 0xa6, 0x70, 0x75, 0x54, 0x81, 0x04, 0xbf, 0x03, 0x67, 0x37, 0xd7, 0xe9, - 0xe5, 0x9b, 0x93, 0x7e, 0x3b, 0xb2, 0x3e, 0xa5, 0x88, 0x96, 0xa8, 0x0e, 0x49, 0x97, 0x3f, 0x7b, - 0xb0, 0xfb, 0xec, 0x0b, 0x8f, 0xab, 0xa1, 0x57, 0x95, 0xf5, 0x29, 0xc5, 0x6f, 0x8b, 0x76, 0xfd, - 0xb7, 0x47, 0xce, 0xea, 0xe2, 0x69, 0xb5, 0x3c, 0x61, 0x42, 0x36, 0x04, 0x1c, 0x41, 0xa1, 0x13, - 0x34, 0xd8, 0x45, 0x28, 0xbb, 0x16, 0x3f, 0x7f, 0x82, 0x23, 0xef, 0x29, 0x74, 0x82, 0xbc, 0x25, - 0xda, 0x02, 0xe8, 0x04, 0x91, 0x86, 0x5d, 0x98, 0xa7, 0x97, 0xdf, 0xbe, 0x4c, 0x36, 0xb3, 0x3e, - 0xa5, 0x84, 0x10, 0xd0, 0x03, 0x48, 0x77, 0x86, 0xae, 0x93, 0x9f, 0x65, 0x80, 0xef, 0x5c, 0x8a, - 0x3f, 0xd7, 0x29, 0x67, 0x0e, 0xa5, 0xa3, 0x9c, 0x29, 0x8f, 0x73, 0x66, 0x1d, 0x32, 0xe2, 0x66, - 0x81, 0x7f, 0x05, 0x99, 0xcf, 0x31, 0xca, 0x0e, 0x5b, 0x89, 0xff, 0x9d, 0x64, 0xb9, 0x6e, 0x75, - 0x6c, 0x1d, 0xeb, 0x75, 0x5a, 0x56, 0xc4, 0x55, 0x25, 0x2b, 0x10, 0xb4, 0x06, 0xd9, 0x8e, 0x89, - 0x35, 0xab, 0xe7, 0xf8, 0x38, 0x68, 0x42, 0x9c, 0x8c, 0x68, 0x27, 0x80, 0xb6, 0x00, 0xed, 0xb3, - 0xf7, 0xca, 0xf0, 0xa8, 0xd8, 0xbd, 0xfb, 0x24, 0x60, 0x32, 0x6b, 0xab, 0x0c, 0x47, 0x86, 0x5e, - 0x87, 0x8c, 0x65, 0x5b, 0x1d, 0xcd, 0xea, 0x60, 0x93, 0x45, 0x37, 0x7e, 0x55, 0x3f, 0x2a, 0x44, - 0x9f, 0x42, 0x96, 0x8c, 0xa4, 0xf0, 0xf9, 0x6b, 0xac, 0xc7, 0x77, 0x2f, 0x7b, 0x1f, 0xb8, 0x3e, - 0xa5, 0x8c, 0x21, 0xa1, 0xdf, 0x04, 0xd9, 0x1b, 0xbb, 0x03, 0x66, 0x97, 0xfe, 0xe7, 0x7f, 0x1b, - 0x70, 0xc6, 0xa5, 0xf5, 0xfa, 0x94, 0x72, 0x02, 0xad, 0x92, 0x82, 0xa4, 0xce, 0xab, 0x83, 0xb7, - 0x82, 0xa4, 0x2c, 0x95, 0x7e, 0x91, 0x00, 0x29, 0x48, 0x7e, 0x96, 0x00, 0x05, 0xe1, 0x6a, 0xf8, - 0x81, 0x06, 0xe5, 0xdd, 0xe8, 0xfa, 0x94, 0x92, 0xf3, 0xeb, 0x86, 0xdf, 0x68, 0xbc, 0x09, 0xb3, - 0x5d, 0x5b, 0x37, 0xf6, 0x8d, 0x21, 0xdb, 0xf1, 0xdb, 0xaa, 0xac, 0x2f, 0x16, 0x6c, 0x77, 0x6f, - 0xe4, 0x1d, 0x33, 0x36, 0x41, 0xe8, 0x5d, 0x9f, 0x0a, 0x3d, 0x74, 0x52, 0xf6, 0x75, 0x7b, 0x96, - 0x65, 0x58, 0x07, 0xaa, 0x38, 0x6f, 0xf1, 0xc0, 0x9e, 0x11, 0x52, 0x71, 0x64, 0xaa, 0x8e, 0xd1, - 0xd1, 0xad, 0x0b, 0xe9, 0xc8, 0x9f, 0xfb, 0x7a, 0x24, 0xe0, 0xa3, 0xd5, 0x71, 0x3e, 0xba, 0x7d, - 0x31, 0x1f, 0x85, 0x60, 0x02, 0x42, 0xda, 0x3b, 0x95, 0x90, 0x96, 0x26, 0xb4, 0x96, 0x10, 0xe2, - 0x28, 0x23, 0x55, 0xc7, 0x18, 0xe9, 0xd6, 0x85, 0x8c, 0x14, 0x9e, 0xa3, 0xa0, 0xa4, 0xed, 0x53, - 0x28, 0xe9, 0x9d, 0x89, 0x28, 0x29, 0x04, 0x16, 0xe6, 0x24, 0xe5, 0x34, 0x4e, 0x2a, 0x4f, 0xc6, - 0x49, 0x21, 0xc8, 0x11, 0x52, 0xfa, 0xde, 0x09, 0x87, 0x93, 0x19, 0xec, 0x37, 0x2e, 0x7d, 0xc8, - 0x5e, 0x8f, 0x9c, 0xf0, 0x38, 0xed, 0x14, 0x8f, 0xcb, 0x31, 0xf8, 0x3b, 0x97, 0xf0, 0xb8, 0x50, - 0x07, 0x27, 0x5d, 0x0e, 0x40, 0xf2, 0x1f, 0xad, 0x42, 0xee, 0x57, 0xfa, 0x51, 0x04, 0x62, 0x4d, - 0xbb, 0x8d, 0xb2, 0xc3, 0x1b, 0x09, 0x76, 0x97, 0xf0, 0xd1, 0x50, 0x5d, 0xa4, 0xd7, 0xaf, 0x9d, - 0x33, 0x12, 0xbf, 0x67, 0x25, 0x68, 0x84, 0x3e, 0x80, 0xa4, 0xc3, 0x53, 0x27, 0xe1, 0x61, 0xa5, - 0xf3, 0xda, 0x73, 0x4d, 0xc5, 0x6f, 0x72, 0xfb, 0x56, 0xf8, 0x2b, 0xe4, 0x4d, 0x5b, 0xc7, 0x28, - 0x0b, 0xb0, 0xa3, 0x11, 0xe2, 0x1c, 0xba, 0x1a, 0xc1, 0xf2, 0x14, 0x4a, 0x42, 0x6c, 0x63, 0xb3, - 0x25, 0x47, 0x6e, 0x7f, 0x37, 0x7c, 0x47, 0x51, 0x53, 0x56, 0x1a, 0x5b, 0x8d, 0xad, 0x35, 0x75, - 0x6b, 0x65, 0xb3, 0xde, 0x92, 0xa7, 0x50, 0x1e, 0xe6, 0x3f, 0x5e, 0x69, 0xec, 0x8a, 0x4b, 0x0b, - 0xb5, 0xb1, 0xb5, 0x5b, 0x57, 0x1e, 0xae, 0xdc, 0x97, 0x23, 0xe8, 0x3a, 0x20, 0x65, 0xbb, 0xba, - 0xd1, 0xaa, 0x55, 0xd4, 0xea, 0xf6, 0xe6, 0xce, 0x4a, 0x75, 0xb7, 0xb1, 0xbd, 0x25, 0x47, 0x91, - 0x04, 0xf1, 0xda, 0xf6, 0x56, 0x5d, 0x86, 0xdb, 0x3f, 0x8a, 0x41, 0x9c, 0xae, 0x31, 0x7a, 0x1d, - 0xd2, 0x7b, 0x5b, 0xad, 0x9d, 0x7a, 0xb5, 0xb1, 0xda, 0xa8, 0xd7, 0xe4, 0xa9, 0xc2, 0xdc, 0xb3, - 0xe7, 0x8b, 0xb3, 0xb4, 0x6a, 0xcf, 0x22, 0x0e, 0xee, 0x30, 0x72, 0x41, 0x05, 0x48, 0x54, 0x56, - 0xaa, 0x1b, 0x7b, 0x3b, 0x72, 0xa4, 0x90, 0x7d, 0xf6, 0x7c, 0x11, 0xa8, 0x02, 0x77, 0x6c, 0xf4, - 0x0a, 0x24, 0x95, 0x7a, 0x6b, 0x77, 0x5b, 0xa9, 0xcb, 0xd1, 0xc2, 0xec, 0xb3, 0xe7, 0x8b, 0x69, - 0x5a, 0x29, 0xfc, 0x15, 0xbd, 0x09, 0x99, 0x56, 0x75, 0xbd, 0xbe, 0xb9, 0xa2, 0x56, 0xd7, 0x57, - 0xb6, 0xd6, 0xea, 0x72, 0xac, 0x30, 0xff, 0xec, 0xf9, 0xa2, 0x3c, 0xbe, 0xc1, 0xb4, 0x8b, 0xc6, - 0xe6, 0xce, 0xb6, 0xb2, 0x2b, 0xc7, 0x87, 0x5d, 0x70, 0xbf, 0x42, 0x25, 0x00, 0xde, 0x7a, 0xb5, - 0x5e, 0xaf, 0xc9, 0xd3, 0x05, 0xf4, 0xec, 0xf9, 0x62, 0x96, 0xd6, 0x0f, 0xdd, 0x05, 0xbd, 0x01, - 0x33, 0x55, 0xa5, 0xbe, 0xb2, 0x5b, 0x57, 0x5b, 0xbb, 0x2b, 0xbb, 0x2d, 0x39, 0x31, 0x9c, 0x49, - 0xc8, 0x05, 0x50, 0x19, 0x72, 0x2b, 0x7b, 0xbb, 0xdb, 0xea, 0x88, 0x6e, 0xb2, 0x70, 0xe3, 0xd9, - 0xf3, 0xc5, 0x39, 0xaa, 0xbb, 0xd2, 0xf3, 0xec, 0xb0, 0xfe, 0xdb, 0x20, 0x8f, 0x8c, 0x5f, 0x5d, - 0xab, 0xca, 0x52, 0xe1, 0xfa, 0xb3, 0xe7, 0x8b, 0x68, 0x7c, 0x0a, 0x6b, 0x55, 0xf4, 0x6b, 0x70, - 0x7d, 0xf7, 0x93, 0x9d, 0x7a, 0xad, 0xde, 0xaa, 0xaa, 0xa3, 0xd3, 0x4e, 0x15, 0xf2, 0xcf, 0x9e, - 0x2f, 0xce, 0xd3, 0x36, 0xe3, 0xed, 0x0a, 0xd2, 0x0f, 0xff, 0x6a, 0x61, 0xea, 0x6f, 0xfe, 0x7a, - 0x61, 0xaa, 0x72, 0xf3, 0xf3, 0xff, 0x58, 0x98, 0xfa, 0x7c, 0xb0, 0x10, 0xf9, 0xe9, 0x60, 0x21, - 0xf2, 0xb3, 0xc1, 0x42, 0xe4, 0xdf, 0x07, 0x0b, 0x91, 0x3f, 0xfa, 0x62, 0x61, 0xea, 0xa7, 0x5f, - 0x2c, 0x4c, 0xfd, 0xec, 0x8b, 0x85, 0xa9, 0x4f, 0x13, 0xdc, 0xac, 0xda, 0x09, 0x96, 0x86, 0xdf, - 0xf9, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x4d, 0xf4, 0x8b, 0x0b, 0x6c, 0x31, 0x00, 0x00, +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_98971d9474e7d77d) } + +var fileDescriptor_jobs_98971d9474e7d77d = []byte{ + // 4369 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x4b, 0x6c, 0x23, 0x47, + 0x7a, 0x16, 0x1f, 0x22, 0x9b, 0x3f, 0x45, 0xb2, 0x59, 0xd2, 0xcc, 0x70, 0x19, 0x5b, 0x54, 0xe8, + 0xd7, 0xcc, 0xd8, 0xa6, 0xbc, 0x72, 0xd6, 0x6b, 0x4f, 0xec, 0xf1, 0x8a, 0x0f, 0x49, 0xa4, 0x46, + 0x0f, 0x37, 0xa5, 0xf1, 0xda, 0x0b, 0xa7, 0xd3, 0xec, 0x2e, 0x49, 0x1d, 0x91, 0xdd, 0x3d, 0x5d, + 0xcd, 0x99, 0xd1, 0x26, 0x48, 0x82, 0x0d, 0x12, 0x2c, 0xe6, 0x94, 0x00, 0x9b, 0x1c, 0x92, 0x0c, + 0x10, 0x20, 0x59, 0x20, 0x87, 0x00, 0x01, 0x8c, 0x20, 0xc9, 0x21, 0xb7, 0x5c, 0x7c, 0x48, 0x80, + 0xbd, 0x04, 0x58, 0xe4, 0xc0, 0x4d, 0xe4, 0x4b, 0x2e, 0x01, 0x16, 0xc9, 0x6d, 0x4e, 0x41, 0x3d, + 0xba, 0xd9, 0xa4, 0x5e, 0xd4, 0xc8, 0xde, 0x5c, 0x24, 0xd6, 0x5f, 0x7f, 0x7d, 0x55, 0xf5, 0xd7, + 0xff, 0xac, 0x6a, 0xb8, 0xfe, 0x1b, 0x76, 0x87, 0x2c, 0xd2, 0x3f, 0x4e, 0x87, 0xfd, 0xab, 0x38, + 0xae, 0xed, 0xd9, 0xe8, 0x1b, 0xba, 0xad, 0x1f, 0xba, 0xb6, 0xa6, 0x1f, 0x54, 0xc8, 0x83, 0x6e, + 0x85, 0xf5, 0x70, 0xae, 0xe2, 0x35, 0xec, 0xba, 0xb6, 0x4b, 0xf9, 0xf9, 0x0f, 0x3e, 0xa2, 0x38, + 0xb7, 0x6f, 0xef, 0xdb, 0xec, 0xe7, 0x22, 0xfd, 0x25, 0xa8, 0x88, 0x61, 0x38, 0x9d, 0x45, 0x43, + 0xf3, 0x34, 0x41, 0x2b, 0xf8, 0x34, 0xd3, 0x7e, 0x73, 0xcf, 0x76, 0x7b, 0x9a, 0xe7, 0x63, 0xbc, + 0x44, 0x1e, 0x74, 0x17, 0x75, 0xcd, 0xd3, 0xba, 0xf6, 0xfe, 0xa2, 0x81, 0x89, 0xee, 0x74, 0x16, + 0x89, 0xe7, 0xf6, 0x75, 0xaf, 0xef, 0x62, 0x43, 0x30, 0x95, 0x4e, 0x61, 0xf2, 0xb0, 0xa5, 0x59, + 0x9e, 0x8f, 0xdf, 0xf7, 0xcc, 0xee, 0xe2, 0x41, 0x57, 0x5f, 0xf4, 0xcc, 0x1e, 0x26, 0x9e, 0xd6, + 0x73, 0x78, 0x4f, 0xf9, 0x77, 0x60, 0xfa, 0x1e, 0xd6, 0x08, 0x46, 0x9f, 0x42, 0xd2, 0xb2, 0x0d, + 0xac, 0x9a, 0x46, 0x21, 0xb2, 0x10, 0xb9, 0x99, 0xa9, 0x2e, 0x1f, 0x0f, 0x4a, 0x89, 0x4d, 0xdb, + 0xc0, 0xcd, 0xfa, 0xb3, 0x41, 0xe9, 0xed, 0x7d, 0xd3, 0x3b, 0xe8, 0x77, 0x2a, 0xba, 0xdd, 0x5b, + 0x0c, 0x04, 0x61, 0x74, 0x86, 0xbf, 0x17, 0x9d, 0xc3, 0xfd, 0x45, 0xb1, 0x8d, 0x0a, 0x1f, 0xa6, + 0x24, 0x28, 0x62, 0xd3, 0x40, 0x73, 0x30, 0x8d, 0x1d, 0x5b, 0x3f, 0x28, 0x44, 0x17, 0x22, 0x37, + 0x63, 0x0a, 0x6f, 0xdc, 0x89, 0xff, 0xd7, 0x5f, 0x94, 0x22, 0xe5, 0x1f, 0x47, 0xe1, 0x46, 0x55, + 0xd3, 0x0f, 0xfb, 0x4e, 0xc3, 0xd2, 0xdd, 0x23, 0xc7, 0x33, 0x6d, 0x6b, 0x8b, 0xfd, 0x25, 0x48, + 0x86, 0xd8, 0x21, 0x3e, 0x62, 0xeb, 0x99, 0x51, 0xe8, 0x4f, 0xf4, 0x01, 0xc4, 0x7b, 0xb6, 0x81, + 0x19, 0x50, 0x76, 0xe9, 0x56, 0xe5, 0xcc, 0x33, 0xa9, 0x0c, 0xd1, 0x36, 0x6c, 0x03, 0x2b, 0x6c, + 0x18, 0xea, 0x80, 0x74, 0xd8, 0x23, 0xaa, 0x69, 0xed, 0xd9, 0x85, 0xd8, 0x42, 0xe4, 0x66, 0x7a, + 0xe9, 0xce, 0x39, 0x10, 0x67, 0x2c, 0xab, 0xb2, 0xbe, 0xd1, 0x6e, 0x5a, 0x7b, 0x76, 0x35, 0x7d, + 0x3c, 0x28, 0x25, 0x45, 0x43, 0x49, 0x1e, 0xf6, 0x08, 0xfd, 0x51, 0xdc, 0x02, 0x9f, 0x46, 0xd7, + 0xdf, 0x77, 0x4d, 0xb6, 0xfe, 0x94, 0x42, 0x7f, 0xa2, 0x37, 0x00, 0x61, 0x8e, 0x87, 0x0d, 0x95, + 0x2a, 0x80, 0x4a, 0x37, 0x18, 0x65, 0x1b, 0x94, 0x83, 0x9e, 0xba, 0xe6, 0x69, 0xeb, 0xf8, 0x88, + 0x4b, 0x48, 0xc8, 0xe9, 0x77, 0x63, 0x90, 0x1d, 0x2e, 0x85, 0xc1, 0xaf, 0x41, 0x82, 0xe8, 0x07, + 0xb8, 0x87, 0xd9, 0x0c, 0xd9, 0xa5, 0xb7, 0x26, 0x12, 0x07, 0x1d, 0x5a, 0x69, 0xb3, 0x71, 0x8a, + 0x18, 0x8f, 0x10, 0xc4, 0x89, 0xd6, 0xf5, 0xc4, 0x42, 0xd8, 0x6f, 0xf4, 0x67, 0x11, 0x58, 0x18, + 0x5f, 0x51, 0xf5, 0x68, 0x7d, 0xa3, 0xbd, 0xa1, 0x11, 0x0f, 0xbb, 0xeb, 0xf8, 0xa8, 0x59, 0x2f, + 0xc4, 0x16, 0x62, 0x37, 0xd3, 0x4b, 0x5b, 0x93, 0x4f, 0xdc, 0xb8, 0x00, 0xb1, 0x61, 0x79, 0xee, + 0x91, 0x72, 0xe1, 0xc4, 0xc5, 0x36, 0xbc, 0x32, 0x11, 0x54, 0x58, 0x87, 0x52, 0x5c, 0x87, 0xe6, + 0x60, 0xfa, 0xa1, 0xd6, 0xed, 0x63, 0xb1, 0x5b, 0xde, 0xb8, 0x13, 0x7d, 0x37, 0x52, 0xbe, 0x01, + 0x09, 0x2e, 0x18, 0x94, 0x81, 0xd4, 0x72, 0xa3, 0xbd, 0xf4, 0xad, 0x77, 0x56, 0x6b, 0x1b, 0xf2, + 0x94, 0x38, 0x82, 0x7f, 0x4a, 0x40, 0x86, 0xeb, 0x44, 0x1d, 0x7b, 0x9a, 0xd9, 0x25, 0xa8, 0x0a, + 0x40, 0x3c, 0xcd, 0xf5, 0x54, 0x6a, 0x56, 0x6c, 0x8e, 0xf4, 0xd2, 0x8b, 0x21, 0x61, 0x50, 0xb3, + 0xab, 0x1c, 0x74, 0xf5, 0xca, 0x8e, 0x6f, 0x76, 0xd5, 0xf8, 0x17, 0x83, 0xd2, 0x94, 0x92, 0x62, + 0xc3, 0x28, 0x15, 0xdd, 0x05, 0x09, 0x5b, 0x06, 0x47, 0x88, 0x4e, 0x8e, 0x90, 0xc4, 0x96, 0xc1, + 0xc6, 0x7f, 0x83, 0x2b, 0x19, 0x55, 0xe7, 0x54, 0x35, 0x79, 0x3c, 0x28, 0xc5, 0x76, 0x95, 0x26, + 0xd7, 0xb6, 0xf7, 0xa1, 0x68, 0x60, 0xc7, 0xc5, 0xba, 0x46, 0xd5, 0xad, 0xc3, 0x96, 0xae, 0xf6, + 0x34, 0xcb, 0xdc, 0xc3, 0xc4, 0x2b, 0xc4, 0xd9, 0xf6, 0x0b, 0x43, 0x0e, 0xbe, 0xb7, 0x0d, 0xd1, + 0x8f, 0x7e, 0x2f, 0x02, 0xb3, 0x7d, 0xd7, 0x24, 0x6a, 0xe7, 0x48, 0xed, 0xda, 0xba, 0xd6, 0x35, + 0xbd, 0x23, 0xf5, 0xf0, 0x61, 0x61, 0x9a, 0x9d, 0xf9, 0xdd, 0x0b, 0x0d, 0x47, 0x08, 0xa9, 0xb2, + 0xeb, 0x9a, 0xa4, 0x7a, 0x74, 0x4f, 0x20, 0xac, 0x3f, 0x64, 0xe7, 0x52, 0x9d, 0x3b, 0x1e, 0x94, + 0xe4, 0x5d, 0xa5, 0x19, 0xee, 0xba, 0xaf, 0xc8, 0xfd, 0x31, 0x66, 0xa4, 0x05, 0x16, 0x63, 0xda, + 0x96, 0x6a, 0x73, 0x13, 0x2c, 0x24, 0x98, 0xa0, 0x96, 0x2e, 0x6f, 0xbc, 0x4a, 0x1e, 0x9f, 0x70, + 0x33, 0x7f, 0x14, 0x81, 0x22, 0xf5, 0x86, 0x58, 0xa7, 0x62, 0x0a, 0x3c, 0xa4, 0xea, 0x62, 0xdd, + 0x76, 0x8d, 0x42, 0x92, 0xca, 0xa9, 0xda, 0xfe, 0xf7, 0x49, 0x9d, 0x20, 0xf3, 0xb5, 0xfd, 0xbe, + 0x69, 0x54, 0x76, 0x77, 0x9b, 0xf5, 0xe3, 0x41, 0xa9, 0xb0, 0xed, 0x83, 0x07, 0x87, 0xa8, 0x30, + 0x68, 0xa5, 0xe0, 0x9c, 0xd1, 0x83, 0xde, 0x85, 0xac, 0x6e, 0x77, 0xbb, 0x58, 0x67, 0xdb, 0xde, + 0x55, 0x9a, 0x05, 0x89, 0x1d, 0x70, 0xfe, 0x78, 0x50, 0xca, 0xd4, 0x82, 0x1e, 0x7a, 0xd4, 0x19, + 0x3d, 0xdc, 0x44, 0x0a, 0xe4, 0x42, 0x02, 0x63, 0xae, 0x2e, 0xc5, 0xa4, 0x75, 0x6b, 0x62, 0x2b, + 0x55, 0xb2, 0x78, 0xa4, 0x5d, 0xac, 0xc1, 0xb5, 0x53, 0x4f, 0xf1, 0x22, 0xeb, 0x4a, 0x85, 0xad, + 0x4b, 0x86, 0x2c, 0x3f, 0x94, 0x6d, 0xd7, 0xde, 0x77, 0x31, 0x21, 0xe5, 0xcf, 0xb3, 0x90, 0x55, + 0x30, 0xf1, 0x6c, 0x17, 0xfb, 0x16, 0xf5, 0x79, 0x04, 0x66, 0x69, 0x04, 0x73, 0x4d, 0xc7, 0xb3, + 0x5d, 0xd5, 0xc5, 0x8f, 0x5c, 0xd3, 0xc3, 0xa4, 0x10, 0x65, 0x4a, 0xb7, 0x7c, 0xce, 0x16, 0x46, + 0x81, 0x2a, 0xf5, 0x00, 0x44, 0x11, 0x18, 0x5c, 0xef, 0xee, 0xfe, 0xe0, 0x67, 0xa5, 0x3b, 0x13, + 0x9d, 0xe3, 0xc9, 0xa0, 0x5a, 0x69, 0xd6, 0x15, 0x64, 0x9c, 0x00, 0x46, 0x2f, 0x40, 0x9c, 0xea, + 0x2d, 0xf3, 0x86, 0xa9, 0xaa, 0x74, 0x3c, 0x28, 0xc5, 0xa9, 0x66, 0x2b, 0x8c, 0x3a, 0x62, 0xe0, + 0xf1, 0xe7, 0x30, 0xf0, 0x55, 0x48, 0x7b, 0x5a, 0xa7, 0x8b, 0x55, 0x3a, 0x33, 0x11, 0xe6, 0xf7, + 0xea, 0x98, 0x24, 0xc8, 0x83, 0x6e, 0x47, 0x23, 0xb8, 0xb2, 0x43, 0x39, 0x43, 0x7b, 0x07, 0xcf, + 0x27, 0x10, 0xb4, 0x08, 0x69, 0xfb, 0x21, 0x76, 0x5d, 0xd3, 0xc0, 0xaa, 0xd1, 0x61, 0x36, 0x94, + 0xaa, 0x66, 0x8f, 0x07, 0x25, 0xd8, 0x12, 0xe4, 0x7a, 0x55, 0x01, 0x9f, 0xa5, 0xde, 0x41, 0x1e, + 0xcc, 0x09, 0xa7, 0x11, 0xd8, 0x3f, 0xd3, 0xa7, 0x24, 0x5b, 0xc2, 0xfb, 0x93, 0x1f, 0x06, 0x3f, + 0x77, 0x5f, 0x79, 0x58, 0xf0, 0xe4, 0x9b, 0x44, 0x9d, 0x13, 0x3d, 0xe8, 0x75, 0xc8, 0x3b, 0x2e, + 0x76, 0x34, 0x17, 0xab, 0xba, 0xdd, 0x73, 0xba, 0xd8, 0xc3, 0x06, 0xd3, 0x7e, 0x49, 0x91, 0x45, + 0x47, 0xcd, 0xa7, 0xa3, 0x57, 0x20, 0x4b, 0x3c, 0xcd, 0xa3, 0x31, 0x9d, 0x60, 0x97, 0x72, 0xa6, + 0x18, 0x67, 0x86, 0x51, 0x9b, 0x82, 0x88, 0xde, 0x86, 0x6b, 0xc3, 0x73, 0x23, 0xaa, 0xd3, 0xef, + 0x74, 0x4d, 0x72, 0x80, 0x8d, 0x02, 0x30, 0xee, 0xb9, 0x50, 0xe7, 0xb6, 0xdf, 0x87, 0x8e, 0x46, + 0x54, 0x51, 0xa7, 0x82, 0xd1, 0xf6, 0x71, 0x21, 0xbd, 0x10, 0xb9, 0x39, 0x5d, 0x5d, 0x7b, 0x36, + 0x28, 0xd5, 0x27, 0xd6, 0x23, 0x82, 0x7b, 0x8b, 0x9e, 0x8b, 0x71, 0x48, 0x2d, 0x6b, 0x02, 0x2f, + 0xac, 0x51, 0x3e, 0x0d, 0x29, 0x00, 0x43, 0x13, 0x2c, 0xcc, 0x3c, 0xb7, 0xb7, 0x0b, 0xa1, 0xa0, + 0x65, 0x48, 0xf2, 0xa4, 0x90, 0x14, 0x32, 0xec, 0x00, 0x7f, 0xf9, 0x2c, 0x1d, 0x62, 0x5c, 0xa1, + 0x53, 0xf2, 0xc7, 0xa1, 0x3a, 0x80, 0x77, 0xe4, 0xf8, 0x9a, 0x98, 0x65, 0x28, 0xaf, 0x9c, 0x85, + 0x72, 0xe4, 0x84, 0x15, 0x31, 0xe5, 0x89, 0x36, 0x41, 0x2d, 0x98, 0x61, 0x79, 0x87, 0x26, 0x70, + 0x72, 0x0c, 0xe7, 0xb5, 0x33, 0x70, 0x58, 0x44, 0xd6, 0x42, 0x48, 0x69, 0x12, 0x50, 0x08, 0xda, + 0x86, 0x2c, 0x4d, 0xa3, 0x28, 0xa7, 0x40, 0x93, 0x19, 0xda, 0xad, 0x33, 0xd0, 0xea, 0x82, 0x39, + 0x84, 0x97, 0x31, 0x42, 0x34, 0x52, 0xfc, 0xdf, 0x08, 0xe4, 0x4f, 0x38, 0x0f, 0xb4, 0x03, 0xd1, + 0x20, 0x33, 0xa6, 0x3e, 0x3d, 0xca, 0xb2, 0xe2, 0xab, 0x38, 0x92, 0xa8, 0x69, 0xa0, 0x7d, 0x48, + 0x51, 0x75, 0xb6, 0x3c, 0x9a, 0x76, 0x47, 0x19, 0x78, 0xeb, 0x78, 0x50, 0x92, 0xb6, 0x19, 0xf1, + 0xca, 0x53, 0x48, 0x1c, 0xbc, 0x69, 0xa0, 0x12, 0xa4, 0x3d, 0x5b, 0xc5, 0x8f, 0x4d, 0xe2, 0x99, + 0xd6, 0x3e, 0x4b, 0x16, 0x24, 0x05, 0x3c, 0xbb, 0x21, 0x28, 0xc5, 0x3f, 0x8f, 0x02, 0x3a, 0x69, + 0xa5, 0xe8, 0x1f, 0x23, 0xf0, 0x82, 0x9f, 0x03, 0xd8, 0xae, 0xb9, 0x6f, 0x5a, 0x5a, 0x77, 0x24, + 0x19, 0x88, 0x30, 0x69, 0x7f, 0x7a, 0x15, 0x57, 0x20, 0x12, 0x84, 0x2d, 0x01, 0x3f, 0x9e, 0x28, + 0xbc, 0x40, 0x23, 0x28, 0x4f, 0x14, 0x4e, 0xb0, 0xdc, 0x57, 0x0a, 0xfd, 0x33, 0x06, 0x17, 0xd7, + 0xe1, 0xc5, 0x73, 0x81, 0x2f, 0x13, 0xbb, 0x8a, 0x3f, 0x88, 0xc0, 0x8d, 0x33, 0x22, 0x4a, 0x18, + 0x27, 0xc3, 0x71, 0x3e, 0x0a, 0xe3, 0xa4, 0x97, 0x7e, 0xf5, 0x0a, 0x51, 0x2b, 0xb4, 0x88, 0x56, + 0x5c, 0x8a, 0xc8, 0xd1, 0xf2, 0x5b, 0x90, 0x13, 0x83, 0xfc, 0x38, 0x8a, 0x5e, 0x04, 0x38, 0x30, + 0xf7, 0x0f, 0xd4, 0x47, 0x9a, 0x87, 0x5d, 0x51, 0x2e, 0xa5, 0x28, 0xe5, 0x63, 0x4a, 0x28, 0xff, + 0x9b, 0x04, 0x99, 0x66, 0xcf, 0xb1, 0x5d, 0xcf, 0x8f, 0xb2, 0xf7, 0x20, 0xc1, 0xe2, 0x02, 0x11, + 0xe7, 0x57, 0x39, 0x67, 0x85, 0x23, 0x23, 0x79, 0x7c, 0x11, 0x6e, 0x41, 0x60, 0x04, 0xe1, 0x2f, + 0x7a, 0x6a, 0xf8, 0xfb, 0x00, 0x12, 0xbc, 0xa4, 0x15, 0x15, 0x57, 0x29, 0x34, 0x97, 0x5f, 0x2f, + 0x36, 0xb7, 0x56, 0xcc, 0x2e, 0x5e, 0x61, 0x6c, 0x3e, 0x38, 0x1f, 0x84, 0x5e, 0x05, 0x89, 0x10, + 0x4f, 0x25, 0xe6, 0xf7, 0x79, 0xf4, 0x8c, 0xf1, 0xb2, 0xab, 0xdd, 0xde, 0x69, 0x9b, 0xdf, 0xc7, + 0x4a, 0x92, 0x10, 0x8f, 0xfe, 0x40, 0x45, 0x90, 0x1e, 0x69, 0xdd, 0x2e, 0x8b, 0xb2, 0xd3, 0xac, + 0xcc, 0x0c, 0xda, 0xa3, 0x66, 0x96, 0xf8, 0x7a, 0xcd, 0x4c, 0x04, 0x4c, 0x47, 0xf3, 0x0e, 0x58, + 0xe6, 0x98, 0x52, 0x80, 0x93, 0xb6, 0x35, 0xef, 0x00, 0x15, 0x20, 0x49, 0x34, 0x1a, 0xbb, 0x48, + 0x41, 0x5a, 0x88, 0xdd, 0x9c, 0x51, 0xfc, 0x26, 0x9a, 0x07, 0x16, 0x79, 0x79, 0x93, 0x05, 0xb1, + 0x98, 0x12, 0xa2, 0x30, 0x39, 0x1c, 0x9a, 0x8e, 0xba, 0x77, 0x48, 0x78, 0xd0, 0x12, 0x72, 0x38, + 0x34, 0x9d, 0x95, 0x75, 0xa2, 0x24, 0x69, 0xe7, 0xca, 0x21, 0x41, 0xaf, 0x41, 0xce, 0xb4, 0xf6, + 0x31, 0xf1, 0x54, 0xc3, 0x74, 0xb1, 0xee, 0x75, 0x8f, 0x58, 0xc0, 0x92, 0x94, 0x2c, 0x27, 0xd7, + 0x05, 0x15, 0xdd, 0x02, 0x79, 0x3c, 0xcc, 0xb2, 0x40, 0x23, 0x29, 0xb9, 0xb1, 0x28, 0x4b, 0x59, + 0xf9, 0x51, 0x87, 0x02, 0x67, 0x86, 0xb3, 0x72, 0xfa, 0x30, 0x66, 0x56, 0x60, 0xd6, 0xd1, 0x5c, + 0x82, 0xd5, 0x4e, 0xdf, 0x32, 0xba, 0x58, 0xe5, 0xbe, 0xba, 0x90, 0x65, 0xdc, 0x79, 0xd6, 0x55, + 0x65, 0x3d, 0xdc, 0xad, 0x5f, 0x94, 0x7b, 0x5f, 0xff, 0x7f, 0xc8, 0xbd, 0x8b, 0x3f, 0x8e, 0xc2, + 0x34, 0xd3, 0x73, 0x74, 0x07, 0xe2, 0xf4, 0x98, 0x45, 0x65, 0x37, 0x69, 0xce, 0xc5, 0xc6, 0xd0, + 0x9a, 0xda, 0xd2, 0x7a, 0xb8, 0x80, 0x98, 0x12, 0xb0, 0xdf, 0xe8, 0x06, 0x24, 0x09, 0x7e, 0xa0, + 0x3e, 0xd4, 0xba, 0x85, 0x59, 0x76, 0xc2, 0x09, 0x82, 0x1f, 0xdc, 0xd7, 0xba, 0xe8, 0x1a, 0x24, + 0x4c, 0xa2, 0x5a, 0xf8, 0x51, 0x61, 0x8e, 0x49, 0x6a, 0xda, 0x24, 0x9b, 0xf8, 0x11, 0x73, 0xdb, + 0x9a, 0xbb, 0x8f, 0x3d, 0x55, 0xb7, 0xbb, 0xa4, 0x70, 0x8d, 0x1a, 0x18, 0x4d, 0xe9, 0x28, 0xa9, + 0x66, 0x77, 0x09, 0xfa, 0x25, 0x48, 0x3d, 0xd2, 0x88, 0x8a, 0x7b, 0x8e, 0x77, 0xc4, 0x84, 0x25, + 0x51, 0xb5, 0x27, 0x0d, 0xda, 0x6e, 0xc5, 0xa5, 0xa8, 0x1c, 0x6b, 0xc5, 0xa5, 0x98, 0x1c, 0x6f, + 0xc5, 0xa5, 0xb8, 0x3c, 0xdd, 0x8a, 0x4b, 0xd3, 0x72, 0xa2, 0x15, 0x97, 0x12, 0x72, 0xb2, 0x15, + 0x97, 0x92, 0xb2, 0xd4, 0x8a, 0x4b, 0x92, 0x9c, 0x6a, 0xc5, 0xa5, 0x94, 0x0c, 0xad, 0xb8, 0x04, + 0x72, 0xba, 0x15, 0x97, 0xd2, 0xf2, 0x4c, 0x2b, 0x2e, 0xcd, 0xc8, 0x99, 0x56, 0x5c, 0xca, 0xc8, + 0xd9, 0x56, 0x5c, 0xca, 0xca, 0xb9, 0x56, 0x5c, 0xca, 0xc9, 0x72, 0x2b, 0x2e, 0xc9, 0x72, 0xbe, + 0x15, 0x97, 0xf2, 0x32, 0x2a, 0x7f, 0x1e, 0x01, 0xb9, 0x8d, 0x1f, 0xf4, 0xb1, 0xa5, 0xe3, 0xfb, + 0x5a, 0xb7, 0x76, 0xd0, 0xb7, 0x0e, 0xd1, 0xab, 0x90, 0xd3, 0xe9, 0x0f, 0x95, 0x17, 0xc6, 0x74, + 0xab, 0x11, 0xb6, 0xd5, 0x0c, 0x23, 0xb7, 0x29, 0x95, 0xee, 0xf8, 0x45, 0x00, 0xc1, 0x47, 0x2d, + 0x9b, 0x5f, 0x0c, 0xa5, 0x38, 0x0b, 0x35, 0xe7, 0x31, 0x18, 0xd7, 0x7e, 0xc4, 0xdc, 0xc7, 0x08, + 0x8c, 0x62, 0x3f, 0x42, 0x8b, 0x30, 0x67, 0xe1, 0xc7, 0x9e, 0x3a, 0xce, 0xcc, 0x5c, 0x85, 0x92, + 0xa7, 0x7d, 0xb5, 0xf0, 0x80, 0xf2, 0xbf, 0x46, 0x21, 0xe7, 0x2f, 0xda, 0x77, 0x87, 0x7b, 0x20, + 0xd3, 0x63, 0x31, 0x0d, 0xd5, 0xb3, 0x39, 0x92, 0xef, 0x18, 0x3f, 0x38, 0xc7, 0x31, 0x8e, 0xa1, + 0xd0, 0x76, 0xd3, 0xd8, 0xb1, 0xd9, 0x74, 0x3c, 0x34, 0x28, 0x19, 0x12, 0xa6, 0x15, 0x77, 0x21, + 0xeb, 0x0f, 0xe2, 0x14, 0x54, 0x83, 0xc4, 0xc8, 0x7c, 0xaf, 0x4f, 0x30, 0x9f, 0x2f, 0x6a, 0x45, + 0x0c, 0x2d, 0xfe, 0x26, 0xa0, 0x93, 0x73, 0x87, 0xc3, 0xd2, 0x34, 0x0f, 0x4b, 0x5b, 0xa3, 0x61, + 0xe9, 0xbd, 0xcb, 0xed, 0x2d, 0xb4, 0xec, 0x70, 0x55, 0xf7, 0xcf, 0x51, 0xc8, 0xf2, 0x10, 0x11, + 0x84, 0xa3, 0xd7, 0x21, 0xcf, 0x9c, 0x96, 0x69, 0xed, 0xab, 0x8e, 0x20, 0xb2, 0xfd, 0x45, 0x15, + 0xd9, 0xef, 0x08, 0x98, 0x5f, 0x82, 0x8c, 0x8b, 0x35, 0x63, 0xc8, 0x18, 0x65, 0x8c, 0x33, 0x94, + 0x18, 0x30, 0xbd, 0x02, 0x59, 0x16, 0x0d, 0x87, 0x5c, 0x31, 0xc6, 0x95, 0x61, 0xd4, 0x80, 0xad, + 0x0a, 0x19, 0xe2, 0x68, 0xd6, 0x90, 0x2b, 0xce, 0x84, 0x7a, 0xe3, 0x94, 0x88, 0xd3, 0x76, 0x34, + 0x4b, 0x44, 0x9a, 0x19, 0x3a, 0x26, 0x1c, 0x4b, 0x5d, 0x4c, 0xfa, 0x3d, 0xac, 0x3a, 0x36, 0x2f, + 0xb6, 0x62, 0x4a, 0x8a, 0x53, 0xb6, 0x6d, 0x82, 0x76, 0x99, 0xaa, 0x30, 0x59, 0xa8, 0x06, 0x17, + 0x4e, 0x21, 0xc1, 0x66, 0xb9, 0x3d, 0xb9, 0x38, 0x95, 0x1c, 0x19, 0x25, 0x94, 0x7f, 0x0b, 0x6e, + 0xd0, 0x7c, 0x99, 0x3b, 0xc5, 0xda, 0x81, 0x66, 0xed, 0x07, 0xca, 0xa9, 0x41, 0x92, 0xe5, 0xdc, + 0x41, 0xfa, 0xb9, 0x76, 0x3c, 0x28, 0x25, 0x28, 0xf7, 0x95, 0x03, 0x57, 0x82, 0x02, 0x37, 0x8d, + 0x72, 0x11, 0x0a, 0xe3, 0xb3, 0x07, 0x35, 0xba, 0xc2, 0x4a, 0xf4, 0x7e, 0x0f, 0x53, 0x89, 0xdd, + 0x33, 0x89, 0x87, 0xbe, 0x03, 0x33, 0x42, 0x42, 0x54, 0x70, 0xbe, 0xe6, 0x5e, 0x20, 0xe4, 0xb4, + 0x1b, 0x80, 0x90, 0xf2, 0xdf, 0x45, 0x60, 0xb6, 0xee, 0xda, 0x8e, 0x83, 0x0d, 0xe1, 0x3b, 0xf9, + 0x56, 0x7d, 0x97, 0x19, 0x09, 0xb9, 0xcc, 0x4d, 0x88, 0x36, 0xeb, 0x22, 0x37, 0xbe, 0x7b, 0xd5, + 0x94, 0xbb, 0x59, 0x47, 0xef, 0x41, 0x82, 0x96, 0x86, 0x7d, 0xc2, 0xfc, 0x49, 0xf6, 0x44, 0x11, + 0x34, 0x72, 0x6c, 0x8c, 0x51, 0x11, 0x03, 0xca, 0x7f, 0x90, 0x80, 0x6b, 0x61, 0x19, 0xad, 0xd6, + 0xfc, 0x85, 0x7f, 0x06, 0x49, 0xd3, 0x32, 0xf0, 0x63, 0x3c, 0x91, 0xdf, 0x38, 0x0d, 0xa2, 0x22, + 0xe4, 0xd1, 0xa4, 0x30, 0x7e, 0xd9, 0x25, 0x30, 0xd1, 0x77, 0x83, 0x74, 0x8d, 0x5f, 0x83, 0xdc, + 0x79, 0x6e, 0xf4, 0xfa, 0x58, 0xea, 0x36, 0x92, 0x19, 0x31, 0x07, 0xfb, 0x35, 0x65, 0x46, 0x6d, + 0xc8, 0x9b, 0x96, 0x87, 0xdd, 0x2e, 0xd6, 0x1e, 0xd2, 0x40, 0x4f, 0xa7, 0x17, 0xb7, 0x21, 0x93, + 0x86, 0x55, 0x39, 0x04, 0xc0, 0xc3, 0xf3, 0x67, 0x30, 0x1b, 0x06, 0xf5, 0x8f, 0xe0, 0xfc, 0x1b, + 0x12, 0x26, 0xe1, 0x21, 0xac, 0x7f, 0x11, 0x11, 0x02, 0x6a, 0x72, 0x9c, 0xe2, 0x9f, 0x44, 0x60, + 0x26, 0x7c, 0x2c, 0xc8, 0x04, 0x89, 0xcd, 0xe1, 0xdb, 0x62, 0xac, 0xba, 0x49, 0x73, 0x30, 0xd6, + 0xc9, 0x64, 0xf5, 0xe1, 0x73, 0xcb, 0x8a, 0x43, 0x88, 0x23, 0x6f, 0x1a, 0x34, 0xb0, 0x1b, 0xae, + 0xed, 0x0c, 0xaf, 0x85, 0x63, 0x8a, 0x44, 0x09, 0x34, 0x57, 0x29, 0xfe, 0x36, 0xa4, 0x82, 0x03, + 0x0d, 0x55, 0xa6, 0xb1, 0xaf, 0xb0, 0x32, 0x3d, 0x6f, 0xfe, 0xf2, 0xcf, 0x12, 0x30, 0x7b, 0x9a, + 0xab, 0xfa, 0x04, 0xe4, 0x90, 0x67, 0x50, 0xbb, 0x26, 0xf1, 0x84, 0xc6, 0xde, 0x3a, 0xbf, 0x04, + 0x0a, 0xb9, 0x17, 0x71, 0x1e, 0x59, 0x77, 0xd4, 0xe9, 0x7c, 0x0f, 0xb2, 0x06, 0xdf, 0xb2, 0x2a, + 0x4c, 0x21, 0x76, 0x61, 0xe5, 0x72, 0x8a, 0x8b, 0x11, 0xe8, 0x19, 0x23, 0xd4, 0x45, 0xd8, 0x4d, + 0xb7, 0x8f, 0x1e, 0xdc, 0x26, 0x98, 0x06, 0xd3, 0xcf, 0x4c, 0xb5, 0x7d, 0x3c, 0x28, 0xe5, 0x05, + 0x96, 0x7f, 0x7d, 0x70, 0x65, 0x19, 0xe7, 0x8d, 0x31, 0x40, 0x83, 0x3a, 0x7a, 0xda, 0x4f, 0x27, + 0x9e, 0x1e, 0x3a, 0x7a, 0xaa, 0xa9, 0x57, 0x77, 0xf4, 0xf4, 0x67, 0xd3, 0x40, 0xbf, 0x1f, 0x81, + 0x3c, 0xbf, 0x4b, 0xec, 0xf5, 0x3d, 0x8d, 0x5f, 0x10, 0xfb, 0x15, 0xd1, 0x27, 0xc7, 0x83, 0x52, + 0x8e, 0x09, 0x64, 0x43, 0xf4, 0xb1, 0x69, 0xab, 0xcf, 0x3b, 0xed, 0x10, 0x45, 0x54, 0x09, 0x01, + 0xc1, 0x40, 0xeb, 0x90, 0xe5, 0xe5, 0x9d, 0x4a, 0x0b, 0x1c, 0xd3, 0xb6, 0x58, 0xa9, 0x94, 0xa9, + 0xbe, 0xfc, 0x6c, 0x50, 0x5a, 0x38, 0x45, 0xb3, 0x78, 0x65, 0x78, 0x9f, 0xf3, 0x2a, 0x99, 0xbd, + 0x70, 0x13, 0xe9, 0x90, 0x09, 0x54, 0xe3, 0xc8, 0x11, 0x95, 0xd5, 0xd5, 0x83, 0xc5, 0x8c, 0xaf, + 0x23, 0x14, 0x13, 0xed, 0x43, 0xce, 0x9f, 0x84, 0x97, 0x34, 0xa4, 0x90, 0xfa, 0x4a, 0xa6, 0xf1, + 0xd5, 0x9a, 0xef, 0x9a, 0x88, 0x22, 0xff, 0x3a, 0xcc, 0x9d, 0x1a, 0x8d, 0xff, 0x74, 0x1a, 0xae, + 0x8f, 0x7a, 0xf8, 0x20, 0x71, 0x51, 0xc7, 0x63, 0xd0, 0x87, 0x13, 0x47, 0x09, 0x1f, 0x83, 0xbb, + 0x21, 0xbf, 0x35, 0x1e, 0x85, 0x3e, 0x1b, 0x8b, 0x42, 0xcf, 0x81, 0xcf, 0xd4, 0x6b, 0x0c, 0x5f, + 0x80, 0x16, 0xff, 0x25, 0x02, 0x99, 0x91, 0xf9, 0x7f, 0x91, 0xee, 0x76, 0x3b, 0xc8, 0x0a, 0xf8, + 0xcb, 0xf2, 0xbb, 0x97, 0xdf, 0xdb, 0x68, 0xb2, 0x50, 0xfc, 0x87, 0x08, 0x64, 0x46, 0xb6, 0xfb, + 0x35, 0x39, 0xea, 0xaf, 0x7c, 0xe5, 0xe5, 0x6f, 0x43, 0x82, 0x53, 0x10, 0x82, 0xec, 0xc7, 0xcb, + 0xcd, 0x9d, 0xe6, 0xe6, 0xaa, 0xba, 0xb2, 0xa5, 0xa8, 0xab, 0x35, 0x79, 0x0a, 0xcd, 0x80, 0x54, + 0x6f, 0xdc, 0x6b, 0x50, 0xa2, 0x1c, 0x41, 0x69, 0x48, 0xb2, 0x56, 0xa3, 0x2e, 0x47, 0xcb, 0x55, + 0x90, 0x39, 0xf6, 0x1e, 0xa6, 0xae, 0x95, 0x16, 0xa9, 0xa8, 0x02, 0xb3, 0x14, 0x16, 0xf7, 0x68, + 0x8e, 0x41, 0x83, 0x89, 0x1a, 0xca, 0xf0, 0xf2, 0x41, 0x17, 0x0d, 0x2b, 0x9b, 0x5a, 0x0f, 0x97, + 0xff, 0x3e, 0x0e, 0xf9, 0x21, 0x88, 0x1f, 0x58, 0x5e, 0x05, 0x89, 0x98, 0xd6, 0xa1, 0x3a, 0x7c, + 0xe8, 0xe4, 0x97, 0x1f, 0xa6, 0x75, 0xb8, 0xab, 0x34, 0x95, 0x24, 0xed, 0xdc, 0x75, 0x4d, 0xd4, + 0x82, 0xb8, 0xed, 0x78, 0x7e, 0xde, 0xff, 0xce, 0x39, 0xa2, 0x38, 0x31, 0x47, 0x65, 0xcb, 0xf1, + 0x44, 0xd5, 0xc6, 0x30, 0xd0, 0xdf, 0x44, 0x20, 0xc9, 0x2b, 0x6d, 0x3f, 0xc3, 0x7f, 0xef, 0x52, + 0x78, 0x5c, 0x00, 0xe2, 0xd5, 0xe9, 0x63, 0xaa, 0xea, 0xcf, 0x06, 0xa5, 0xfc, 0xb8, 0x80, 0xc8, + 0x15, 0x9f, 0xa3, 0xfc, 0x25, 0xa2, 0x16, 0x7f, 0x08, 0x19, 0x0a, 0x9a, 0xb9, 0xd4, 0x09, 0xdf, + 0x9a, 0x32, 0x23, 0x07, 0x51, 0xdc, 0x87, 0x99, 0xf0, 0xea, 0x4f, 0xb9, 0xe1, 0x5c, 0x1e, 0x2d, + 0x25, 0x5f, 0x9f, 0x48, 0x32, 0x1c, 0x33, 0x7c, 0xad, 0xfa, 0x6d, 0x48, 0x05, 0x62, 0xbf, 0xcc, + 0x7d, 0x2c, 0xf7, 0x92, 0xc1, 0x05, 0xc7, 0xb4, 0x9c, 0x28, 0xff, 0x6d, 0x04, 0x66, 0x14, 0x4c, + 0xec, 0xee, 0x43, 0x6c, 0xd0, 0xac, 0x01, 0x7d, 0x13, 0xe2, 0x34, 0x0b, 0x11, 0x77, 0x37, 0x17, + 0x94, 0x27, 0x8c, 0x15, 0x2d, 0x43, 0x2a, 0xb8, 0x81, 0xba, 0xcc, 0x5b, 0xfc, 0x70, 0x14, 0xba, + 0x05, 0x72, 0xc7, 0xee, 0x5b, 0x86, 0xe6, 0x1e, 0xa9, 0x2e, 0xd6, 0xf4, 0x03, 0x6c, 0x88, 0xdb, + 0xf6, 0x9c, 0x4f, 0x57, 0x38, 0xb9, 0xfc, 0xc3, 0x28, 0xa0, 0xa1, 0x70, 0x42, 0x6e, 0x82, 0xe6, + 0x3e, 0x6c, 0x1f, 0xa2, 0xc0, 0x8a, 0x9e, 0xfa, 0x3e, 0x32, 0x96, 0x42, 0x05, 0x1b, 0xf7, 0x8f, + 0xd4, 0x0d, 0xd1, 0x08, 0xfa, 0xe3, 0xf3, 0xef, 0xd9, 0x62, 0xec, 0x9e, 0x8d, 0x69, 0xe9, 0x2f, + 0xf4, 0xae, 0x4d, 0x84, 0xbb, 0xff, 0x89, 0x03, 0xaa, 0xb9, 0x58, 0xf3, 0x30, 0xf5, 0x3c, 0xe4, + 0xbc, 0x7a, 0xb0, 0x0a, 0xd3, 0xbc, 0x78, 0x88, 0x5e, 0xa6, 0x78, 0x10, 0x42, 0xe1, 0x43, 0xd1, + 0xaf, 0xc1, 0x8c, 0x6e, 0x77, 0xfb, 0x3d, 0x4b, 0x65, 0xaf, 0x84, 0x22, 0x95, 0xfc, 0xd6, 0x79, + 0x4a, 0x7c, 0x62, 0x71, 0x95, 0x9a, 0xdd, 0xa5, 0x6d, 0xbf, 0xbe, 0xe5, 0x80, 0x8c, 0x03, 0xbd, + 0x00, 0xa9, 0xc0, 0xa0, 0x58, 0x12, 0x99, 0x52, 0x86, 0x04, 0xb4, 0x04, 0xd3, 0x1a, 0x51, 0xed, + 0x3d, 0x96, 0xe5, 0x5d, 0xa4, 0x61, 0x4a, 0x5c, 0x23, 0x5b, 0x7b, 0xe8, 0x6d, 0xc8, 0xec, 0x3d, + 0xe0, 0x99, 0x2f, 0x77, 0xa0, 0xfc, 0xf1, 0x36, 0x77, 0x3c, 0x28, 0xa5, 0x57, 0x3e, 0x62, 0x9b, + 0xa5, 0xee, 0x53, 0x49, 0xef, 0x3d, 0x08, 0x1a, 0xe8, 0x36, 0xe4, 0x7b, 0xda, 0x63, 0x75, 0xcf, + 0xd5, 0x74, 0x91, 0xea, 0x75, 0xb9, 0x57, 0x88, 0x28, 0xb9, 0x9e, 0xf6, 0x78, 0x45, 0xd0, 0x9b, + 0x46, 0x17, 0x17, 0x7f, 0x1e, 0x81, 0xa4, 0xd8, 0x11, 0x72, 0x00, 0x84, 0x78, 0x4c, 0x83, 0x27, + 0x13, 0x99, 0xea, 0x47, 0xc7, 0x83, 0x52, 0xaa, 0xc6, 0xa8, 0xcd, 0x3a, 0x79, 0x36, 0x28, 0x7d, + 0xe7, 0x79, 0x9d, 0x96, 0x0f, 0xa2, 0xa4, 0xf8, 0x24, 0x4d, 0x83, 0x5d, 0x02, 0x1d, 0x68, 0x44, + 0x3d, 0x30, 0x89, 0x67, 0xef, 0xbb, 0x5a, 0x8f, 0x1d, 0xae, 0xa4, 0xcc, 0x1c, 0x68, 0x64, 0xcd, + 0xa7, 0xa1, 0x22, 0x4d, 0x07, 0x1e, 0xf2, 0x47, 0x5e, 0x6e, 0x52, 0x41, 0x1b, 0x2d, 0xc1, 0xb5, + 0x60, 0xb0, 0x4a, 0x37, 0xdd, 0xe9, 0xeb, 0x87, 0x98, 0x45, 0x02, 0xea, 0xb3, 0x66, 0x83, 0xce, + 0x0d, 0xed, 0x71, 0x95, 0x77, 0x95, 0xaf, 0xc1, 0x6c, 0xe8, 0x58, 0x83, 0x14, 0xeb, 0xbf, 0x01, + 0x92, 0xdb, 0xda, 0x51, 0xd7, 0xd6, 0x0c, 0xb4, 0x00, 0x69, 0xff, 0x71, 0x96, 0x26, 0xa9, 0x5c, + 0x0f, 0xc3, 0x24, 0x64, 0x42, 0xb6, 0x4f, 0xb0, 0x4b, 0xcf, 0x44, 0x65, 0x5f, 0xd4, 0x71, 0x5f, + 0x55, 0xad, 0x3e, 0x1b, 0x94, 0xee, 0x4e, 0x26, 0x22, 0xac, 0xf7, 0x5d, 0xd3, 0x3b, 0xaa, 0xb4, + 0x3f, 0xba, 0xb7, 0x2b, 0xa0, 0xa8, 0x21, 0xd9, 0x4a, 0xa6, 0x1f, 0x6e, 0x8a, 0xa7, 0x6e, 0xba, + 0x5d, 0xb5, 0x67, 0xea, 0xae, 0x4d, 0xfc, 0x1b, 0x51, 0x41, 0xdd, 0x60, 0x44, 0xf4, 0x1a, 0xe4, + 0xf6, 0x4c, 0x8b, 0xdd, 0xc6, 0xfb, 0x7c, 0xfc, 0x32, 0x34, 0xeb, 0x93, 0x05, 0xe3, 0x43, 0xc8, + 0x86, 0x9e, 0xb7, 0xe9, 0x51, 0x27, 0xd8, 0x51, 0x6f, 0x1d, 0x0f, 0x4a, 0x99, 0xa1, 0xe9, 0xf0, + 0xe3, 0xbe, 0x4a, 0x8c, 0xca, 0x0c, 0xa7, 0xa1, 0x87, 0x3d, 0x07, 0xd3, 0xec, 0x33, 0x49, 0xfe, + 0x45, 0x8b, 0xc2, 0x1b, 0xe8, 0x1d, 0x98, 0xee, 0x62, 0x8d, 0x60, 0xf1, 0xb1, 0xca, 0xc2, 0x39, + 0xc6, 0xc8, 0x3e, 0x58, 0x54, 0x38, 0x3b, 0xaa, 0x42, 0x82, 0xbf, 0xaf, 0xb0, 0x57, 0x91, 0xf4, + 0xd2, 0xcd, 0x49, 0xbf, 0x4b, 0x5a, 0x9b, 0x52, 0xc4, 0x48, 0xd4, 0x80, 0xa4, 0xcb, 0x9f, 0xd4, + 0xd8, 0x5b, 0xc9, 0x85, 0xe5, 0x6a, 0xe8, 0xc5, 0x6e, 0x6d, 0x4a, 0xf1, 0xc7, 0xa2, 0x1d, 0xff, + 0x5d, 0x9b, 0x7b, 0x75, 0xf1, 0x6c, 0x5f, 0x99, 0x30, 0x21, 0x1b, 0x02, 0x8e, 0xa0, 0xd0, 0x0d, + 0x9a, 0xec, 0x7e, 0x95, 0x3d, 0xb9, 0x9c, 0xbf, 0xc1, 0x91, 0xb7, 0x3a, 0xba, 0x41, 0x3e, 0x12, + 0x6d, 0x02, 0xe8, 0x41, 0xa4, 0x61, 0x8f, 0x31, 0xe9, 0xa5, 0x37, 0x2e, 0x93, 0xcd, 0xac, 0x4d, + 0x29, 0x21, 0x04, 0xf4, 0x11, 0xa4, 0xf5, 0xa1, 0xe9, 0x14, 0x72, 0x0c, 0xf0, 0xcd, 0x4b, 0xf9, + 0xcf, 0x35, 0xea, 0x33, 0x87, 0xd4, 0x51, 0x9f, 0x29, 0x8f, 0xfb, 0xcc, 0x06, 0x64, 0xc4, 0xcd, + 0x02, 0xff, 0xc2, 0xb6, 0x90, 0x67, 0x2e, 0x3b, 0xac, 0x25, 0xfe, 0x37, 0xb8, 0x95, 0x86, 0xa5, + 0xdb, 0x06, 0x36, 0x1a, 0xb4, 0xad, 0x88, 0xab, 0x4a, 0xd6, 0x20, 0x68, 0x15, 0xb2, 0x7a, 0x17, + 0x6b, 0x56, 0xdf, 0xf1, 0x71, 0xd0, 0x84, 0x38, 0x19, 0x31, 0x4e, 0x00, 0x6d, 0x02, 0xda, 0x63, + 0x6f, 0xe1, 0xe1, 0x55, 0xb1, 0x37, 0x9d, 0x49, 0xc0, 0x64, 0x36, 0x56, 0x19, 0xae, 0x0c, 0xbd, + 0x0c, 0x19, 0xcb, 0xb6, 0x74, 0xcd, 0xd2, 0x71, 0x97, 0x45, 0x37, 0xfe, 0x0c, 0x34, 0x4a, 0x44, + 0x9f, 0x42, 0x96, 0x8c, 0xa4, 0xf0, 0x85, 0x6b, 0x6c, 0xc6, 0xb7, 0x2e, 0x7b, 0x1f, 0xb8, 0x36, + 0xa5, 0x8c, 0x21, 0xa1, 0x5f, 0x07, 0xd9, 0x1b, 0xbb, 0x03, 0x66, 0x0f, 0x4a, 0xe7, 0x7f, 0x77, + 0x72, 0xc6, 0xa5, 0xf5, 0xda, 0x94, 0x72, 0x02, 0xad, 0x9a, 0x82, 0xa4, 0xb8, 0x31, 0x0f, 0xde, + 0xa1, 0x92, 0xb2, 0x54, 0xfe, 0x79, 0x02, 0xa4, 0x20, 0xf9, 0x59, 0x04, 0x14, 0x84, 0xab, 0xe1, + 0xc7, 0x3f, 0xd4, 0xef, 0x46, 0xd7, 0xa6, 0x94, 0xbc, 0xdf, 0x37, 0xfc, 0xfe, 0xe7, 0x35, 0xc8, + 0xf5, 0x6c, 0xc3, 0xdc, 0x33, 0x87, 0xde, 0x8e, 0xdf, 0x56, 0x65, 0x7d, 0xb2, 0xf0, 0x76, 0x77, + 0x47, 0xde, 0xc8, 0x63, 0x13, 0x84, 0xde, 0xb5, 0xa9, 0xd0, 0x23, 0x3a, 0xf5, 0xbe, 0x6e, 0xdf, + 0xb2, 0x4c, 0x6b, 0x5f, 0x15, 0xf5, 0x16, 0x0f, 0xec, 0x19, 0x41, 0x15, 0x25, 0x53, 0x6d, 0xcc, + 0x1d, 0xdd, 0xba, 0xd0, 0x1d, 0xf9, 0x7b, 0x5f, 0x8b, 0x04, 0xfe, 0x68, 0x65, 0xdc, 0x1f, 0xdd, + 0xbe, 0xd8, 0x1f, 0x85, 0x60, 0x02, 0x87, 0xb4, 0x7b, 0xaa, 0x43, 0x5a, 0x9c, 0x50, 0x5b, 0x42, + 0x88, 0xa3, 0x1e, 0xa9, 0x36, 0xe6, 0x91, 0x6e, 0x5d, 0xe8, 0x91, 0xc2, 0x7b, 0x14, 0x2e, 0x69, + 0xeb, 0x14, 0x97, 0xf4, 0xe6, 0x44, 0x2e, 0x29, 0x04, 0x16, 0xf6, 0x49, 0xca, 0x69, 0x3e, 0xa9, + 0x32, 0x99, 0x4f, 0x0a, 0x41, 0x8e, 0x38, 0xa5, 0xef, 0x9d, 0x30, 0x38, 0x99, 0xc1, 0x7e, 0xf3, + 0xd2, 0x45, 0xf6, 0x5a, 0xe4, 0x84, 0xc5, 0x69, 0xa7, 0x58, 0x5c, 0x9e, 0xc1, 0xbf, 0x7d, 0x09, + 0x8b, 0x0b, 0x4d, 0x70, 0xd2, 0xe4, 0x00, 0x24, 0xff, 0x2d, 0x2c, 0x64, 0x7e, 0xe5, 0x1f, 0x45, + 0x20, 0xd6, 0xb2, 0x3b, 0x28, 0x3b, 0xbc, 0x91, 0x60, 0x77, 0x09, 0x1f, 0x0e, 0xd9, 0x45, 0x7a, + 0xfd, 0xd2, 0x39, 0x2b, 0xf1, 0x67, 0x56, 0x82, 0x41, 0xe8, 0x7d, 0x48, 0x3a, 0x3c, 0x75, 0x12, + 0x16, 0x56, 0x3e, 0x6f, 0x3c, 0xe7, 0x54, 0xfc, 0x21, 0xb7, 0x6f, 0x85, 0xbf, 0x70, 0xdf, 0xb0, + 0x0d, 0x8c, 0xb2, 0x00, 0xdb, 0x1a, 0x21, 0xce, 0x81, 0xab, 0x11, 0x2c, 0x4f, 0xa1, 0x24, 0xc4, + 0xd6, 0x37, 0xda, 0x72, 0xe4, 0xf6, 0x77, 0xc3, 0x77, 0x14, 0x75, 0x65, 0xb9, 0xb9, 0xd9, 0xdc, + 0x5c, 0x55, 0x37, 0x97, 0x37, 0x1a, 0x6d, 0x79, 0x0a, 0x15, 0x60, 0xee, 0xe3, 0xe5, 0xe6, 0x8e, + 0xb8, 0xb4, 0x50, 0x9b, 0x9b, 0x3b, 0x0d, 0xe5, 0xfe, 0xf2, 0x3d, 0x39, 0x82, 0xae, 0x03, 0x52, + 0xb6, 0x6a, 0xeb, 0xed, 0x7a, 0x55, 0xad, 0x6d, 0x6d, 0x6c, 0x2f, 0xd7, 0x76, 0x9a, 0x5b, 0x9b, + 0x72, 0x14, 0x49, 0x10, 0xaf, 0x6f, 0x6d, 0x36, 0x64, 0xb8, 0xfd, 0xa3, 0x18, 0xc4, 0xa9, 0x8c, + 0xd1, 0xcb, 0x90, 0xde, 0xdd, 0x6c, 0x6f, 0x37, 0x6a, 0xcd, 0x95, 0x66, 0xa3, 0x2e, 0x4f, 0x15, + 0x67, 0x9f, 0x3c, 0x5d, 0xc8, 0xd1, 0xae, 0x5d, 0x8b, 0x38, 0x58, 0x67, 0xce, 0x05, 0x15, 0x21, + 0x51, 0x5d, 0xae, 0xad, 0xef, 0x6e, 0xcb, 0x91, 0x62, 0xf6, 0xc9, 0xd3, 0x05, 0xa0, 0x0c, 0xdc, + 0xb0, 0xd1, 0x0b, 0x90, 0x54, 0x1a, 0xed, 0x9d, 0x2d, 0xa5, 0x21, 0x47, 0x8b, 0xb9, 0x27, 0x4f, + 0x17, 0xd2, 0xb4, 0x53, 0xd8, 0x2b, 0x7a, 0x0d, 0x32, 0xed, 0xda, 0x5a, 0x63, 0x63, 0x59, 0xad, + 0xad, 0x2d, 0x6f, 0xae, 0x36, 0xe4, 0x58, 0x71, 0xee, 0xc9, 0xd3, 0x05, 0x79, 0xfc, 0x80, 0xe9, + 0x14, 0xcd, 0x8d, 0xed, 0x2d, 0x65, 0x47, 0x8e, 0x0f, 0xa7, 0xe0, 0x76, 0x85, 0xca, 0x00, 0x7c, + 0xf4, 0x4a, 0xa3, 0x51, 0x97, 0xa7, 0x8b, 0xe8, 0xc9, 0xd3, 0x85, 0x2c, 0xed, 0x1f, 0x9a, 0x0b, + 0x7a, 0x05, 0x66, 0x6a, 0x4a, 0x63, 0x79, 0xa7, 0xa1, 0xb6, 0x77, 0x96, 0x77, 0xda, 0x72, 0x62, + 0xb8, 0x93, 0x90, 0x09, 0xa0, 0x0a, 0xe4, 0x97, 0x77, 0x77, 0xb6, 0xd4, 0x11, 0xde, 0x64, 0xf1, + 0xc6, 0x93, 0xa7, 0x0b, 0xb3, 0x94, 0x77, 0xb9, 0xef, 0xd9, 0x61, 0xfe, 0x37, 0x40, 0x1e, 0x59, + 0xbf, 0xba, 0x5a, 0x93, 0xa5, 0xe2, 0xf5, 0x27, 0x4f, 0x17, 0xd0, 0xf8, 0x16, 0x56, 0x6b, 0xe8, + 0x57, 0xe0, 0xfa, 0xce, 0x27, 0xdb, 0x8d, 0x7a, 0xa3, 0x5d, 0x53, 0x47, 0xb7, 0x9d, 0x2a, 0x16, + 0x9e, 0x3c, 0x5d, 0x98, 0xa3, 0x63, 0xc6, 0xc7, 0x15, 0xa5, 0x1f, 0xfe, 0xe5, 0xfc, 0xd4, 0x5f, + 0xff, 0xd5, 0xfc, 0x54, 0xf5, 0xe6, 0x17, 0xff, 0x39, 0x3f, 0xf5, 0xc5, 0xf1, 0x7c, 0xe4, 0x27, + 0xc7, 0xf3, 0x91, 0x9f, 0x1e, 0xcf, 0x47, 0xfe, 0xe3, 0x78, 0x3e, 0xf2, 0x87, 0x5f, 0xce, 0x4f, + 0xfd, 0xe4, 0xcb, 0xf9, 0xa9, 0x9f, 0x7e, 0x39, 0x3f, 0xf5, 0x69, 0x82, 0xab, 0x55, 0x27, 0xc1, + 0xd2, 0xf0, 0xb7, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x90, 0x45, 0x7d, 0x1b, 0xc8, 0x33, 0x00, + 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 7f983844d911..5c7f7e03ce8f 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -229,6 +229,33 @@ message ImportDetails { ]; } +// SequenceValChunks represents a single chunk of sequence values allocated +// during an IMPORT. +message SequenceValChunk { + int64 chunk_start_val = 1; + int64 chunk_size = 2; + + // The first row in the file being imported from which the current chunk of + // sequence values is being used. + int64 chunk_start_row = 3; + // The row in the file being imported at which the import will need to use a + // new chunk of sequence values. + int64 next_chunk_start_row = 4; +} + +// SequenceDetails represents information about the sequences processed in a +// single file during IMPORT. +message SequenceDetails { + // SequenceChunks represents all the chunks reserved for a particular sequence + // during an IMPORT. + message SequenceChunks { + repeated SequenceValChunk chunks = 1; + } + + // Mapping from sequence ID to allocated sequence chunks. + map seq_id_to_chunks = 1; +} + message ImportProgress { repeated float sampling_progress = 1; repeated float read_progress = 2; @@ -242,6 +269,10 @@ message ImportProgress { // been flushed, we can advance the count here and then on resume skip over // that many rows without needing to convert/process them at all. repeated int64 resume_pos = 5; // Only set by direct import. + + // Holds metadata related to sequences for every file processed during an + // IMPORT. + repeated SequenceDetails sequence_details = 6; } // TypeSchemaChangeDetails is the job detail information for a type schema change job. @@ -303,7 +334,7 @@ message SchemaChangeGCDetails { // The below two fields are used only in the case of TRUNCATE operating on // tables with interleaved indexes. They are only set together. - + // InterleavedTable is the table being truncated. In particular, it is the // TableDescriptor before any of the truncate modifications have been applied. sqlbase.TableDescriptor interleaved_table = 4; diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index e4deea5c2dce..28efd08b175c 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -565,6 +565,19 @@ func (r *Registry) LoadJobWithTxn(ctx context.Context, jobID int64, txn *kv.Txn) return j, nil } +// UpdateJobWithTxn calls the Update method on an existing job with jobID, using +// a transaction passed in the txn argument. Passing a nil transaction means +// that a txn will be automatically created. +func (r *Registry) UpdateJobWithTxn( + ctx context.Context, jobID int64, txn *kv.Txn, updateFunc UpdateFn, +) error { + j := &Job{ + id: &jobID, + registry: r, + } + return j.WithTxn(txn).Update(ctx, updateFunc) +} + // DefaultCancelInterval is a reasonable interval at which to poll this node // for liveness failures and cancel running jobs. var DefaultCancelInterval = 10 * time.Second diff --git a/pkg/sql/distsql_plan_csv.go b/pkg/sql/distsql_plan_csv.go index 71ed376e20cf..25d29ab36174 100644 --- a/pkg/sql/distsql_plan_csv.go +++ b/pkg/sql/distsql_plan_csv.go @@ -227,6 +227,13 @@ func DistIngest( prog := details.(*jobspb.Progress_Import).Import prog.ReadProgress = make([]float32, len(from)) prog.ResumePos = make([]int64, len(from)) + if prog.SequenceDetails == nil { + prog.SequenceDetails = make([]*jobspb.SequenceDetails, len(from)) + for i := range prog.SequenceDetails { + prog.SequenceDetails[i] = &jobspb.SequenceDetails{} + } + } + return 0.0 }, ); err != nil { diff --git a/pkg/sql/execinfrapb/processors_bulk_io.pb.go b/pkg/sql/execinfrapb/processors_bulk_io.pb.go index be5d0a09a07b..7dfb4741fdbc 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.pb.go +++ b/pkg/sql/execinfrapb/processors_bulk_io.pb.go @@ -71,7 +71,7 @@ func (x *FileCompression) UnmarshalJSON(data []byte) error { return nil } func (FileCompression) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{0} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{0} } type BackfillerSpec_Type int32 @@ -110,7 +110,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{0, 0} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{0, 0} } // BackfillerSpec is the specification for a "schema change backfiller". @@ -142,7 +142,7 @@ func (m *BackfillerSpec) Reset() { *m = BackfillerSpec{} } func (m *BackfillerSpec) String() string { return proto.CompactTextString(m) } func (*BackfillerSpec) ProtoMessage() {} func (*BackfillerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{0} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{0} } func (m *BackfillerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -182,7 +182,7 @@ func (m *JobProgress) Reset() { *m = JobProgress{} } func (m *JobProgress) String() string { return proto.CompactTextString(m) } func (*JobProgress) ProtoMessage() {} func (*JobProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{1} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{1} } func (m *JobProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -245,7 +245,7 @@ func (m *ReadImportDataSpec) Reset() { *m = ReadImportDataSpec{} } func (m *ReadImportDataSpec) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec) ProtoMessage() {} func (*ReadImportDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{2} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{2} } func (m *ReadImportDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -283,7 +283,7 @@ func (m *ReadImportDataSpec_ImportTable) Reset() { *m = ReadImportDataSp func (m *ReadImportDataSpec_ImportTable) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec_ImportTable) ProtoMessage() {} func (*ReadImportDataSpec_ImportTable) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{2, 0} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{2, 0} } func (m *ReadImportDataSpec_ImportTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -329,7 +329,7 @@ func (m *BackupDataSpec) Reset() { *m = BackupDataSpec{} } func (m *BackupDataSpec) String() string { return proto.CompactTextString(m) } func (*BackupDataSpec) ProtoMessage() {} func (*BackupDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{3} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{3} } func (m *BackupDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -368,7 +368,7 @@ func (m *RestoreSpanEntry) Reset() { *m = RestoreSpanEntry{} } func (m *RestoreSpanEntry) String() string { return proto.CompactTextString(m) } func (*RestoreSpanEntry) ProtoMessage() {} func (*RestoreSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{4} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{4} } func (m *RestoreSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -406,7 +406,7 @@ func (m *RestoreDataSpec) Reset() { *m = RestoreDataSpec{} } func (m *RestoreDataSpec) String() string { return proto.CompactTextString(m) } func (*RestoreDataSpec) ProtoMessage() {} func (*RestoreDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{5} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{5} } func (m *RestoreDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -440,7 +440,7 @@ func (m *SplitAndScatterSpec) Reset() { *m = SplitAndScatterSpec{} } func (m *SplitAndScatterSpec) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec) ProtoMessage() {} func (*SplitAndScatterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{6} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{6} } func (m *SplitAndScatterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -473,7 +473,7 @@ func (m *SplitAndScatterSpec_RestoreEntryChunk) Reset() { *m = SplitAndS func (m *SplitAndScatterSpec_RestoreEntryChunk) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec_RestoreEntryChunk) ProtoMessage() {} func (*SplitAndScatterSpec_RestoreEntryChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{6, 0} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{6, 0} } func (m *SplitAndScatterSpec_RestoreEntryChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -520,7 +520,7 @@ func (m *CSVWriterSpec) Reset() { *m = CSVWriterSpec{} } func (m *CSVWriterSpec) String() string { return proto.CompactTextString(m) } func (*CSVWriterSpec) ProtoMessage() {} func (*CSVWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{7} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{7} } func (m *CSVWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -555,7 +555,7 @@ func (m *BulkRowWriterSpec) Reset() { *m = BulkRowWriterSpec{} } func (m *BulkRowWriterSpec) String() string { return proto.CompactTextString(m) } func (*BulkRowWriterSpec) ProtoMessage() {} func (*BulkRowWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{8} + return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{8} } func (m *BulkRowWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3929,10 +3929,10 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_e62f5714e36bbbd6) + proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_bf46a169ea516277) } -var fileDescriptor_processors_bulk_io_e62f5714e36bbbd6 = []byte{ +var fileDescriptor_processors_bulk_io_bf46a169ea516277 = []byte{ // 1673 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1b, 0xc7, 0x15, 0xd7, 0x92, 0x4b, 0x8a, 0x7c, 0xb4, 0x24, 0x6a, 0xe2, 0xa4, 0x5b, 0x01, 0x95, 0x04, 0x26, diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 00e6af3e6565..0ae146ea4fd2 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -131,7 +131,6 @@ message ReadImportDataSpec { // User who initiated the import. This is used to check access privileges // when using FileTable ExternalStorage. optional string user_proto = 15 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"]; - // NEXTID: 16 } diff --git a/pkg/sql/row/BUILD.bazel b/pkg/sql/row/BUILD.bazel index f7cb0596bfe8..530c97ff4651 100644 --- a/pkg/sql/row/BUILD.bazel +++ b/pkg/sql/row/BUILD.bazel @@ -21,12 +21,15 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/row", visibility = ["//visibility:public"], deps = [ + "//pkg/jobs", + "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver/concurrency/lock", "//pkg/roachpb", "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", + "//pkg/sql/catalog/catalogkv", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/schemaexpr", @@ -48,6 +51,7 @@ go_library( "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/mon", + "//pkg/util/sequence", "//pkg/util/timeutil", "//pkg/util/unique", "//pkg/util/uuid", @@ -58,6 +62,7 @@ go_library( go_test( name = "row_test", srcs = [ + "expr_walker_test.go", "fetcher_mvcc_test.go", "fetcher_test.go", "main_test.go", @@ -65,6 +70,8 @@ go_test( embed = [":row"], deps = [ "//pkg/base", + "//pkg/jobs", + "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver", @@ -81,6 +88,7 @@ go_test( "//pkg/sql/pgwire/pgerror", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", "//pkg/storage", "//pkg/testutils", "//pkg/testutils/serverutils", @@ -90,8 +98,10 @@ go_test( "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/mon", + "//pkg/util/protoutil", "//pkg/util/randutil", "//vendor/github.com/cockroachdb/errors", "//vendor/github.com/stretchr/testify/assert", + "//vendor/github.com/stretchr/testify/require", ], ) diff --git a/pkg/sql/row/expr_walker.go b/pkg/sql/row/expr_walker.go index 475a644c6be7..3c6294ddf3d7 100644 --- a/pkg/sql/row/expr_walker.go +++ b/pkg/sql/row/expr_walker.go @@ -14,9 +14,20 @@ import ( "context" "math/rand" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/sequence" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" ) @@ -28,6 +39,12 @@ import ( // tune this parameter. const reseedRandEveryN = 1000 +// chunkSizeIncrementRate is the factor by which the size of the chunk of +// sequence values we allocate during an import increases. +const chunkSizeIncrementRate = 10 +const initialChunkSize = 10 +const maxChunkSize = 100000 + type importRand struct { *rand.Rand pos int64 @@ -81,6 +98,16 @@ func makeBuiltinOverride( "import."+builtin.Name, &props, overloads) } +// SequenceMetadata contains information used when processing columns with +// default expressions which use sequences. +type SequenceMetadata struct { + id descpb.ID + seqDesc *tabledesc.Immutable + instancesPerRow int64 + curChunk *jobspb.SequenceValChunk + curVal int64 +} + type overrideVolatility bool const ( @@ -97,20 +124,30 @@ const ( // in the Annotation field of evalCtx when evaluating expressions. const cellInfoAddr tree.AnnotationIdx = iota + 1 -type cellInfoAnnotation struct { - sourceID int32 - rowID int64 +// CellInfoAnnotation encapsulates the AST annotation for the various supported +// default expressions for import. +type CellInfoAnnotation struct { + sourceID int32 + rowID int64 + + // Annotations for unique_rowid(). uniqueRowIDInstance int uniqueRowIDTotal int - randSource *importRand - randInstancePerRow int + + // Annotations for rand() and gen_random_uuid(). + randSource *importRand + randInstancePerRow int + + // Annotations for next_val(). + seqNameToMetadata map[string]*SequenceMetadata + seqChunkProvider *SeqChunkProvider } -func getCellInfoAnnotation(t *tree.Annotations) *cellInfoAnnotation { - return t.Get(cellInfoAddr).(*cellInfoAnnotation) +func getCellInfoAnnotation(t *tree.Annotations) *CellInfoAnnotation { + return t.Get(cellInfoAddr).(*CellInfoAnnotation) } -func (c *cellInfoAnnotation) Reset(sourceID int32, rowID int64) { +func (c *CellInfoAnnotation) reset(sourceID int32, rowID int64) { c.sourceID = sourceID c.rowID = rowID c.uniqueRowIDInstance = 0 @@ -182,14 +219,251 @@ func importGenUUID(evalCtx *tree.EvalContext, args tree.Datums) (tree.Datum, err return tree.NewDUuid(tree.DUuid{UUID: id}), nil } -// Besides overriding, there are also counters that we want to keep track -// of as we walk through the expressions in a row (at datumRowConverter creation -// time). This will be handled by the visitorSideEffect field: it will be -// called with an annotation passed in that changes the counter. In the case of -// unique_rowid, for example, we want to keep track of the total number of -// unique_rowid occurrences in a row. +// SeqChunkProvider uses the import job progress to read and write its sequence +// value chunks. +type SeqChunkProvider struct { + JobID int64 + Registry *jobs.Registry +} + +// RequestChunk updates seqMetadata with information about the chunk of sequence +// values pertaining to the row being processed during an import. The method +// first checks if there is a previously allocated chunk associated with the +// row, and if not goes on to allocate a new chunk. +func (j *SeqChunkProvider) RequestChunk( + evalCtx *tree.EvalContext, c *CellInfoAnnotation, seqMetadata *SequenceMetadata, +) error { + var hasAllocatedChunk bool + return evalCtx.DB.Txn(evalCtx.Context, func(ctx context.Context, txn *kv.Txn) error { + var foundFromPreviouslyAllocatedChunk bool + resolveChunkFunc := func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + progress := md.Progress + + // Check if we have already reserved a chunk corresponding to this row in a + // previous run of the import job. If we have, we must reuse the value of + // the sequence which was designated on this particular invocation of + // nextval(). + var err error + if foundFromPreviouslyAllocatedChunk, err = j.checkForPreviouslyAllocatedChunks( + seqMetadata, c, progress); err != nil { + return err + } else if foundFromPreviouslyAllocatedChunk { + return nil + } + + // Reserve a new sequence value chunk at the KV level. + if !hasAllocatedChunk { + if err := reserveChunkOfSeqVals(evalCtx, c, seqMetadata); err != nil { + return err + } + hasAllocatedChunk = true + } + + // Update job progress with the newly reserved chunk before it can be used by the import. + // It is important that this information is persisted before it is used to + // ensure correct behavior on job resumption. + // We never want to end up in a situation where row x is assigned a different + // sequence value on subsequent import job resumptions. + fileProgress := progress.GetImport().SequenceDetails[c.sourceID] + if fileProgress.SeqIdToChunks == nil { + fileProgress.SeqIdToChunks = make(map[int32]*jobspb.SequenceDetails_SequenceChunks) + } + seqID := seqMetadata.id + if _, ok := fileProgress.SeqIdToChunks[int32(seqID)]; !ok { + fileProgress.SeqIdToChunks[int32(seqID)] = &jobspb.SequenceDetails_SequenceChunks{ + Chunks: make([]*jobspb.SequenceValChunk, 0), + } + } + // We can cleanup some of the older chunks which correspond to rows + // below the resume pos as we are never going to reprocess those + // check pointed rows on job resume. + resumePos := progress.GetImport().ResumePos[c.sourceID] + trim, chunks := 0, fileProgress.SeqIdToChunks[int32(seqID)].Chunks + // If the resumePos is below the max bound of the current chunk we need + // to keep this chunk in case the job is re-resumed. + for ; trim < len(chunks) && chunks[trim].NextChunkStartRow <= resumePos; trim++ { + } + fileProgress.SeqIdToChunks[int32(seqID)].Chunks = + fileProgress.SeqIdToChunks[int32(seqID)].Chunks[trim:] + + fileProgress.SeqIdToChunks[int32(seqID)].Chunks = append( + fileProgress.SeqIdToChunks[int32(seqID)].Chunks, seqMetadata.curChunk) + ju.UpdateProgress(progress) + return nil + } + err := j.Registry.UpdateJobWithTxn(ctx, j.JobID, txn, resolveChunkFunc) + if err != nil { + return err + } + + // Now that the job progress has been written to, we can use the newly + // allocated chunk. + if !foundFromPreviouslyAllocatedChunk { + seqMetadata.curVal = seqMetadata.curChunk.ChunkStartVal + } + return nil + }) +} + +func incrementSequenceByVal( + ctx context.Context, + descriptor *tabledesc.Immutable, + db *kv.DB, + codec keys.SQLCodec, + incrementBy int64, +) (int64, error) { + seqOpts := descriptor.SequenceOpts + var val int64 + var err error + // TODO(adityamaru): Think about virtual sequences. + if seqOpts.Virtual { + return 0, errors.New("virtual sequences are not supported by IMPORT INTO") + } + seqValueKey := codec.SequenceKey(uint32(descriptor.ID)) + val, err = kv.IncrementValRetryable(ctx, db, seqValueKey, incrementBy) + if err != nil { + if errors.HasType(err, (*roachpb.IntegerOverflowError)(nil)) { + return 0, boundsExceededError(descriptor) + } + return 0, err + } + if val > seqOpts.MaxValue || val < seqOpts.MinValue { + return 0, boundsExceededError(descriptor) + } + + return val, nil +} + +func boundsExceededError(descriptor *tabledesc.Immutable) error { + seqOpts := descriptor.SequenceOpts + isAscending := seqOpts.Increment > 0 + + var word string + var value int64 + if isAscending { + word = "maximum" + value = seqOpts.MaxValue + } else { + word = "minimum" + value = seqOpts.MinValue + } + return pgerror.Newf( + pgcode.SequenceGeneratorLimitExceeded, + `reached %s value of sequence %q (%d)`, word, + tree.ErrString((*tree.Name)(&descriptor.Name)), value) +} + +// checkForPreviouslyAllocatedChunks checks if a sequence value has already been +// generated for a the current row being imported. If such a value is found, the +// seqMetadata is updated to reflect this. +// This would be true if the IMPORT job has been re-resumed and there were some +// rows which had not been marked as imported. +func (j *SeqChunkProvider) checkForPreviouslyAllocatedChunks( + seqMetadata *SequenceMetadata, c *CellInfoAnnotation, progress *jobspb.Progress, +) (bool, error) { + var found bool + fileProgress := progress.GetImport().SequenceDetails[c.sourceID] + if fileProgress.SeqIdToChunks == nil { + return found, nil + } + var allocatedSeqChunks *jobspb.SequenceDetails_SequenceChunks + var ok bool + if allocatedSeqChunks, ok = fileProgress.SeqIdToChunks[int32(seqMetadata.id)]; !ok { + return found, nil + } + + for _, chunk := range allocatedSeqChunks.Chunks { + // We have found the chunk of sequence values that was assigned to the + // swath of rows encompassing rowID. + if chunk.ChunkStartRow <= c.rowID && chunk.NextChunkStartRow > c.rowID { + relativeRowIndex := c.rowID - chunk.ChunkStartRow + seqMetadata.curVal = chunk.ChunkStartVal + + seqMetadata.seqDesc.SequenceOpts.Increment*(seqMetadata.instancesPerRow*relativeRowIndex) + found = true + return found, nil + } + } + return found, nil +} + +// reserveChunkOfSeqVals ascertains the size of the next chunk, and reserves it +// at the KV level. The seqMetadata is updated to reflect this. +func reserveChunkOfSeqVals( + evalCtx *tree.EvalContext, c *CellInfoAnnotation, seqMetadata *SequenceMetadata, +) error { + newChunkSize := int64(initialChunkSize) + // If we are allocating a subsequent chunk of sequence values, we attempt + // to reserve a factor of 10 more than reserved the last time so as to + // prevent clobbering the chunk reservation logic which involves writing + // to job progress. + if seqMetadata.curChunk != nil { + newChunkSize = chunkSizeIncrementRate * seqMetadata.curChunk.ChunkSize + if newChunkSize > maxChunkSize { + newChunkSize = maxChunkSize + } + } + + // We want to encompass at least one complete row with our chunk + // allocation. + if newChunkSize < seqMetadata.instancesPerRow { + newChunkSize = seqMetadata.instancesPerRow + } + + incrementValBy := newChunkSize * seqMetadata.seqDesc.SequenceOpts.Increment + // incrementSequenceByVal keeps retrying until it is able to find a slot + // of incrementValBy. + seqVal, err := incrementSequenceByVal(evalCtx.Context, seqMetadata.seqDesc, evalCtx.DB, + evalCtx.Codec, incrementValBy) + if err != nil { + return err + } + + // Update the sequence metadata to reflect the newly reserved chunk. + seqMetadata.curChunk = &jobspb.SequenceValChunk{ + ChunkStartVal: seqVal - incrementValBy + seqMetadata.seqDesc.SequenceOpts.Increment, + ChunkSize: newChunkSize, + ChunkStartRow: c.rowID, + NextChunkStartRow: c.rowID + (newChunkSize / seqMetadata.instancesPerRow), + } + return nil +} + +func importNextVal(evalCtx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + c := getCellInfoAnnotation(evalCtx.Annotations) + seqName := tree.MustBeDString(args[0]) + seqMetadata, ok := c.seqNameToMetadata[string(seqName)] + if !ok { + return nil, errors.Newf("sequence %s not found in annotation", seqName) + } + if c.seqChunkProvider == nil { + return nil, errors.New("no sequence chunk provider configured for the import job") + } + + // If the current importWorker does not have an active chunk for the sequence + // seqName, or the row we are processing is outside the range of rows covered + // by the active chunk, we need to request a chunk. + if seqMetadata.curChunk == nil || c.rowID == seqMetadata.curChunk.NextChunkStartRow { + if err := c.seqChunkProvider.RequestChunk(evalCtx, c, seqMetadata); err != nil { + return nil, err + } + } else { + // The current chunk of sequence values can be used for the row being + // processed. + seqMetadata.curVal += seqMetadata.seqDesc.SequenceOpts.Increment + } + return tree.NewDInt(tree.DInt(seqMetadata.curVal)), nil +} + +// Besides overriding, there are also counters that we want to keep track of as +// we walk through the expressions in a row (at datumRowConverter creation +// time). This will be handled by the visitorSideEffect field: it will be called +// with an annotation, and a FuncExpr. The annotation changes the counter, while +// the FuncExpr is used to extract information from the function. +// +// Egs: In the case of unique_rowid, we want to keep track of the total number +// of unique_rowid occurrences in a row. type customFunc struct { - visitorSideEffect func(annotations *tree.Annotations) + visitorSideEffect func(annotations *tree.Annotations, fn *tree.FuncExpr) error override *tree.FunctionDefinition } @@ -211,8 +485,9 @@ var supportedImportFuncOverrides = map[string]*customFunc{ "timeofday": useDefaultBuiltin, "transaction_timestamp": useDefaultBuiltin, "unique_rowid": { - visitorSideEffect: func(annot *tree.Annotations) { + visitorSideEffect: func(annot *tree.Annotations, _ *tree.FuncExpr) error { getCellInfoAnnotation(annot).uniqueRowIDTotal++ + return nil }, override: makeBuiltinOverride( tree.FunDefs["unique_rowid"], @@ -226,8 +501,9 @@ var supportedImportFuncOverrides = map[string]*customFunc{ ), }, "random": { - visitorSideEffect: func(annot *tree.Annotations) { + visitorSideEffect: func(annot *tree.Annotations, _ *tree.FuncExpr) error { getCellInfoAnnotation(annot).randInstancePerRow++ + return nil }, override: makeBuiltinOverride( tree.FunDefs["random"], @@ -241,8 +517,9 @@ var supportedImportFuncOverrides = map[string]*customFunc{ ), }, "gen_random_uuid": { - visitorSideEffect: func(annot *tree.Annotations) { + visitorSideEffect: func(annot *tree.Annotations, _ *tree.FuncExpr) error { getCellInfoAnnotation(annot).randInstancePerRow++ + return nil }, override: makeBuiltinOverride( tree.FunDefs["gen_random_uuid"], @@ -256,6 +533,32 @@ var supportedImportFuncOverrides = map[string]*customFunc{ }, ), }, + "nextval": { + visitorSideEffect: func(annot *tree.Annotations, fn *tree.FuncExpr) error { + // Get sequence name so that we can update the annotation with the number + // of nextval calls to this sequence in a row. + seqName, err := sequence.GetSequenceFromFunc(fn) + if err != nil { + return err + } + var sequenceMetadata *SequenceMetadata + var ok bool + if sequenceMetadata, ok = getCellInfoAnnotation(annot).seqNameToMetadata[*seqName]; !ok { + return errors.Newf("sequence %s not found in annotation", *seqName) + } + sequenceMetadata.instancesPerRow++ + return nil + }, + override: makeBuiltinOverride( + tree.FunDefs["nextval"], + tree.Overload{ + Types: tree.ArgTypes{{builtins.SequenceNameArg, types.String}}, + ReturnType: tree.FixedReturnType(types.Int), + Info: "Advances the value of the sequence and returns the final value.", + Fn: importNextVal, + }, + ), + }, } func unsafeExpressionError(err error, msg string, expr string) error { @@ -323,7 +626,11 @@ func (v *importDefaultExprVisitor) VisitPost(expr tree.Expr) (newExpr tree.Expr) // unique_rowid function in an expression). v.volatility = overrideVolatile if custom.visitorSideEffect != nil { - custom.visitorSideEffect(v.annotations) + err := custom.visitorSideEffect(v.annotations, fn) + if err != nil { + v.err = errors.Wrapf(err, "function %s failed when invoking side effect", resolvedFnName) + return expr + } } funcExpr := &tree.FuncExpr{ Func: tree.ResolvableFunctionReference{FunctionReference: custom.override}, diff --git a/pkg/sql/row/expr_walker_test.go b/pkg/sql/row/expr_walker_test.go new file mode 100644 index 000000000000..cfac5083bac6 --- /dev/null +++ b/pkg/sql/row/expr_walker_test.go @@ -0,0 +1,225 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +package row + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/stretchr/testify/require" +) + +func createAndIncrementSeqDescriptor( + ctx context.Context, + t *testing.T, + id int, + codec keys.SQLCodec, + incrementBy int64, + seqOpts descpb.TableDescriptor_SequenceOpts, + db *kv.DB, +) tabledesc.Immutable { + desc := tabledesc.MakeImmutable(descpb.TableDescriptor{ + ID: descpb.ID(id), + SequenceOpts: &seqOpts, + }) + seqValueKey := codec.SequenceKey(uint32(desc.ID)) + _, err := kv.IncrementValRetryable( + ctx, db, seqValueKey, incrementBy) + require.NoError(t, err) + return desc +} + +func createMockImportJob( + ctx context.Context, + t *testing.T, + registry *jobs.Registry, + seqIDToAllocatedChunks map[int32]*jobspb.SequenceDetails_SequenceChunks, + resumePos int64, +) *jobs.Job { + seqDetails := []*jobspb.SequenceDetails{{SeqIdToChunks: seqIDToAllocatedChunks}} + mockImportRecord := jobs.Record{ + Details: jobspb.ImportDetails{}, + Progress: jobspb.ImportProgress{ + SequenceDetails: seqDetails, + ResumePos: []int64{resumePos}, + }, + } + mockImportJob, err := registry.CreateJobWithTxn(ctx, mockImportRecord, nil /* txn */) + require.NoError(t, err) + return mockImportJob +} + +// TestJobBackedSeqChunkProvider this is a unit test of the sequence chunk +// provider which is used to populate default expressions during an import. +func TestJobBackedSeqChunkProvider(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + s, sqlDB, db := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + evalCtx := &tree.EvalContext{ + Context: ctx, + DB: db, + Codec: keys.TODOSQLCodec, + InternalExecutor: s.InternalExecutor().(sqlutil.InternalExecutor), + } + + registry := s.JobRegistry().(*jobs.Registry) + testCases := []struct { + name string + // chunks written to progress before we RequestChunk. + allocatedChunks map[int32]*jobspb.SequenceDetails_SequenceChunks + // value by which the sequence is incremented before we RequestChunk. + incrementBy int64 + resumePos int64 + // row being processed during the import. + rowID int64 + instancesPerRow int64 + seqIDToOpts map[int]descpb.TableDescriptor_SequenceOpts + seqIDToExpectedVal map[int]int64 + seqIDToNumChunks map[int]int + }{ + { + // No previously allocated chunks, this is the first row being imported. + name: "first-chunk", + allocatedChunks: nil, + rowID: 0, + instancesPerRow: 1, + seqIDToOpts: map[int]descpb.TableDescriptor_SequenceOpts{55: { + Increment: 1, + MinValue: 1, + MaxValue: 100, + Start: 1, + }}, + seqIDToExpectedVal: map[int]int64{55: 1}, + seqIDToNumChunks: map[int]int{55: 1}, + }, + { + // Import row in already allocated first chunk. Should not allocate a new + // chunk. + name: "row-in-first-chunk", + allocatedChunks: map[int32]*jobspb.SequenceDetails_SequenceChunks{56: {Chunks: []*jobspb.SequenceValChunk{{ + ChunkStartVal: 1, + ChunkStartRow: 0, + ChunkSize: 10, + NextChunkStartRow: 10, + }}}}, + rowID: 8, + instancesPerRow: 1, + seqIDToOpts: map[int]descpb.TableDescriptor_SequenceOpts{56: { + Increment: 1, + MinValue: 1, + MaxValue: 100, + Start: 1, + }}, + seqIDToExpectedVal: map[int]int64{56: 9}, + seqIDToNumChunks: map[int]int{56: 1}, + }, + { + // Import row is greater than the max row covered by the allocated chunk. + // We expect to see another chunk getting allocated. + name: "need-new-chunk", + allocatedChunks: map[int32]*jobspb.SequenceDetails_SequenceChunks{57: {Chunks: []*jobspb.SequenceValChunk{{ + ChunkStartVal: 1, + ChunkStartRow: 0, + ChunkSize: 10, + NextChunkStartRow: 10, + }}}}, + incrementBy: 10, + // rowID is equal to NextChunkStartRow + rowID: 10, + instancesPerRow: 1, + seqIDToOpts: map[int]descpb.TableDescriptor_SequenceOpts{57: { + Increment: 1, + MinValue: 1, + MaxValue: 100, + Start: 1, + }}, + seqIDToExpectedVal: map[int]int64{57: 11}, + seqIDToNumChunks: map[int]int{57: 2}, + }, + { + // Same test case as before, but the resume position means that the first + // chunk should get cleaned up as it covers already processed rows. + name: "cleanup-old-chunks", + allocatedChunks: map[int32]*jobspb.SequenceDetails_SequenceChunks{58: {Chunks: []*jobspb.SequenceValChunk{{ + ChunkStartVal: 1, + ChunkStartRow: 0, + ChunkSize: 10, + NextChunkStartRow: 10, + }}}}, + incrementBy: 10, + resumePos: 10, + rowID: 10, + instancesPerRow: 1, + seqIDToOpts: map[int]descpb.TableDescriptor_SequenceOpts{58: { + Increment: 1, + MinValue: 1, + MaxValue: 100, + Start: 1, + }}, + seqIDToExpectedVal: map[int]int64{58: 11}, + seqIDToNumChunks: map[int]int{58: 1}, + }, + } + + for _, test := range testCases { + t.Run(test.name, func(t *testing.T) { + job := createMockImportJob(ctx, t, registry, test.allocatedChunks, test.resumePos) + j := &SeqChunkProvider{Registry: registry, JobID: *job.ID()} + annot := &CellInfoAnnotation{ + sourceID: 0, + rowID: test.rowID, + } + + for id, val := range test.seqIDToExpectedVal { + seqDesc := createAndIncrementSeqDescriptor(ctx, t, id, keys.TODOSQLCodec, + test.incrementBy, test.seqIDToOpts[id], db) + seqMetadata := &SequenceMetadata{ + id: descpb.ID(id), + seqDesc: &seqDesc, + instancesPerRow: test.instancesPerRow, + curChunk: nil, + curVal: 0, + } + require.NoError(t, j.RequestChunk(evalCtx, annot, seqMetadata)) + getJobProgressQuery := `SELECT progress FROM system.jobs J WHERE J.id = $1` + + var progressBytes []byte + require.NoError(t, sqlDB.QueryRow(getJobProgressQuery, *job.ID()).Scan(&progressBytes)) + var progress jobspb.Progress + require.NoError(t, protoutil.Unmarshal(progressBytes, &progress)) + chunks := progress.GetImport().SequenceDetails[0].SeqIdToChunks[int32(id)].Chunks + + // Ensure that the sequence value for the row is what we expect. + require.Equal(t, val, seqMetadata.curVal) + // Ensure we have as many chunks written to the job progress as we + // expect. + require.Equal(t, test.seqIDToNumChunks[id], len(chunks)) + } + }) + } +} diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index aebe7bac3ad8..fc523c5abd7a 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -13,8 +13,10 @@ package row import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" @@ -26,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -239,6 +242,49 @@ func TestingSetDatumRowConverterBatchSize(newSize int) func() { } } +// getSequenceAnnotation returns a mapping from sequence name to metadata +// related to the sequence which will be used when evaluating the default +// expression using the sequence. +func (c *DatumRowConverter) getSequenceAnnotation( + evalCtx *tree.EvalContext, cols []descpb.ColumnDescriptor, +) (map[string]*SequenceMetadata, error) { + // Identify the sequences used in all the columns. + sequenceIDs := make(map[descpb.ID]struct{}) + for _, col := range cols { + for _, id := range col.UsesSequenceIds { + sequenceIDs[id] = struct{}{} + } + } + + if len(sequenceIDs) == 0 { + return nil, nil + } + + var seqNameToMetadata map[string]*SequenceMetadata + err := evalCtx.DB.Txn(evalCtx.Context, func(ctx context.Context, txn *kv.Txn) error { + seqNameToMetadata = make(map[string]*SequenceMetadata) + txn.SetFixedTimestamp(ctx, hlc.Timestamp{WallTime: evalCtx.TxnTimestamp.UnixNano()}) + for seqID := range sequenceIDs { + seqDesc, err := catalogkv.MustGetTableDescByID(ctx, txn, evalCtx.Codec, seqID) + if err != nil { + return err + } + + seqOpts := seqDesc.SequenceOpts + if seqOpts == nil { + return errors.Newf("descriptor %s is not a sequence", seqDesc.Name) + } + + seqNameToMetadata[seqDesc.Name] = &SequenceMetadata{ + id: seqID, + seqDesc: seqDesc, + } + } + return nil + }) + return seqNameToMetadata, err +} + // NewDatumRowConverter returns an instance of a DatumRowConverter. func NewDatumRowConverter( ctx context.Context, @@ -246,6 +292,7 @@ func NewDatumRowConverter( targetColNames tree.NameList, evalCtx *tree.EvalContext, kvCh chan<- KVBatch, + seqChunkProvider *SeqChunkProvider, ) (*DatumRowConverter, error) { c := &DatumRowConverter{ tableDesc: tableDesc, @@ -314,7 +361,13 @@ func NewDatumRowConverter( // If the DEFAULT expression is immutable, we can store it in the cache so that it // doesn't have to be reevaluated for every row. annot := make(tree.Annotations, 1) - annot.Set(cellInfoAddr, &cellInfoAnnotation{uniqueRowIDInstance: 0}) + var seqNameToMetadata map[string]*SequenceMetadata + seqNameToMetadata, err = c.getSequenceAnnotation(evalCtx, c.cols) + if err != nil { + return nil, err + } + annot.Set(cellInfoAddr, &CellInfoAnnotation{uniqueRowIDInstance: 0, + seqNameToMetadata: seqNameToMetadata, seqChunkProvider: seqChunkProvider}) c.EvalCtx.Annotations = &annot for i := range cols { col := &cols[i] @@ -390,7 +443,7 @@ const rowIDBits = 64 - builtins.NodeIDBits // Row inserts kv operations into the current kv batch, and triggers a SendBatch // if necessary. func (c *DatumRowConverter) Row(ctx context.Context, sourceID int32, rowIndex int64) error { - getCellInfoAnnotation(c.EvalCtx.Annotations).Reset(sourceID, rowIndex) + getCellInfoAnnotation(c.EvalCtx.Annotations).reset(sourceID, rowIndex) for i := range c.cols { col := &c.cols[i] if col.DefaultExpr != nil { diff --git a/pkg/sql/rowexec/bulk_row_writer.go b/pkg/sql/rowexec/bulk_row_writer.go index 8f6ff3701dfd..9b0f1fe4609a 100644 --- a/pkg/sql/rowexec/bulk_row_writer.go +++ b/pkg/sql/rowexec/bulk_row_writer.go @@ -103,7 +103,7 @@ func (sp *bulkRowWriter) work(ctx context.Context) error { var g ctxgroup.Group conv, err := row.NewDatumRowConverter(ctx, - &sp.tableDesc, nil /* targetColNames */, sp.EvalCtx, kvCh) + &sp.tableDesc, nil /* targetColNames */, sp.EvalCtx, kvCh, nil /* seqChunkProvider */) if err != nil { return err }