diff --git a/pkg/ccl/backupccl/backup.go b/pkg/ccl/backupccl/backup.go index 2d06e4826542..3488f0eec016 100644 --- a/pkg/ccl/backupccl/backup.go +++ b/pkg/ccl/backupccl/backup.go @@ -52,8 +52,6 @@ const ( // BackupDescriptorCheckpointName is the file name used to store the // serialized BackupDescriptor proto while the backup is in progress. BackupDescriptorCheckpointName = "BACKUP-CHECKPOINT" - // BackupFormatInitialVersion is the first version of backup and its files. - BackupFormatInitialVersion uint32 = 0 // BackupFormatDescriptorTrackingVersion added tracking of complete DBs. BackupFormatDescriptorTrackingVersion uint32 = 1 ) diff --git a/pkg/ccl/backupccl/restore.go b/pkg/ccl/backupccl/restore.go index 5de067bb085f..877949943741 100644 --- a/pkg/ccl/backupccl/restore.go +++ b/pkg/ccl/backupccl/restore.go @@ -906,14 +906,14 @@ func WriteTableDescs( b.CPut(sqlbase.MakeDescMetadataKey(desc.ID), sqlbase.WrapDescriptor(desc), nil) b.CPut(sqlbase.MakeNameMetadataKey(keys.RootNamespaceID, desc.Name), desc.ID, nil) } - for _, table := range tables { - if wrote, ok := wroteDBs[table.ParentID]; ok { - table.Privileges = wrote.GetPrivileges() + for i := range tables { + if wrote, ok := wroteDBs[tables[i].ParentID]; ok { + tables[i].Privileges = wrote.GetPrivileges() } else { - parentDB, err := sqlbase.GetDatabaseDescFromID(ctx, txn, table.ParentID) + parentDB, err := sqlbase.GetDatabaseDescFromID(ctx, txn, tables[i].ParentID) if err != nil { return pgerror.NewAssertionErrorWithWrappedErrf(err, - "failed to lookup parent DB %d", log.Safe(table.ParentID)) + "failed to lookup parent DB %d", log.Safe(tables[i].ParentID)) } // TODO(mberhault): CheckPrivilege wants a planner. if err := sql.CheckPrivilegeForUser(ctx, user, parentDB, privilege.CREATE); err != nil { @@ -921,10 +921,10 @@ func WriteTableDescs( } // Default is to copy privs from restoring parent db, like CREATE TABLE. // TODO(dt): Make this more configurable. - table.Privileges = parentDB.GetPrivileges() + tables[i].Privileges = parentDB.GetPrivileges() } - b.CPut(table.GetDescMetadataKey(), sqlbase.WrapDescriptor(table), nil) - b.CPut(table.GetNameMetadataKey(), table.ID, nil) + b.CPut(tables[i].GetDescMetadataKey(), sqlbase.WrapDescriptor(tables[i]), nil) + b.CPut(tables[i].GetNameMetadataKey(), tables[i].ID, nil) } for _, kv := range extra { b.InitPut(kv.Key, &kv.Value, false) diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index 7306f99b766d..ceda87a118d6 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -9,37 +9,28 @@ package importccl import ( - "bytes" "context" - "fmt" - "io/ioutil" "math" "sort" "strconv" "strings" - "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/internal/client" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -60,7 +51,6 @@ const ( mysqlOutfileEnclose = "fields_enclosed_by" mysqlOutfileEscape = "fields_escaped_by" - importOptionTransform = "transform" importOptionSSTSize = "sstsize" importOptionDecompress = "decompress" importOptionOversample = "oversample" @@ -85,7 +75,6 @@ var importOptionExpectValues = map[string]sql.KVStringOptValidate{ mysqlOutfileEnclose: sql.KVStringOptRequireValue, mysqlOutfileEscape: sql.KVStringOptRequireValue, - importOptionTransform: sql.KVStringOptRequireValue, importOptionSSTSize: sql.KVStringOptRequireValue, importOptionDecompress: sql.KVStringOptRequireValue, importOptionOversample: sql.KVStringOptRequireValue, @@ -97,308 +86,6 @@ var importOptionExpectValues = map[string]sql.KVStringOptValidate{ pgMaxRowSize: sql.KVStringOptRequireValue, } -const ( - // We need to choose arbitrary database and table IDs. These aren't important, - // but they do match what would happen when creating a new database and - // table on an empty cluster. - defaultCSVParentID sqlbase.ID = keys.MinNonPredefinedUserDescID - defaultCSVTableID sqlbase.ID = defaultCSVParentID + 1 -) - -func readCreateTableFromStore( - ctx context.Context, filename string, settings *cluster.Settings, -) (*tree.CreateTable, error) { - store, err := storageccl.ExportStorageFromURI(ctx, filename, settings) - if err != nil { - return nil, err - } - defer store.Close() - reader, err := store.ReadFile(ctx, "") - if err != nil { - return nil, err - } - defer reader.Close() - tableDefStr, err := ioutil.ReadAll(reader) - if err != nil { - return nil, err - } - stmt, err := parser.ParseOne(string(tableDefStr)) - if err != nil { - return nil, err - } - create, ok := stmt.AST.(*tree.CreateTable) - if !ok { - return nil, errors.New("expected CREATE TABLE statement in table file") - } - return create, nil -} - -type fkHandler struct { - allowed bool - skip bool - resolver fkResolver -} - -// NoFKs is used by formats that do not support FKs. -var NoFKs = fkHandler{resolver: make(fkResolver)} - -// MakeSimpleTableDescriptor creates a MutableTableDescriptor from a CreateTable parse -// node without the full machinery. Many parts of the syntax are unsupported -// (see the implementation and TestMakeSimpleTableDescriptorErrors for details), -// but this is enough for our csv IMPORT and for some unit tests. -// -// Any occurrence of SERIAL in the column definitions is handled using -// the CockroachDB legacy behavior, i.e. INT NOT NULL DEFAULT -// unique_rowid(). -func MakeSimpleTableDescriptor( - ctx context.Context, - st *cluster.Settings, - create *tree.CreateTable, - parentID, tableID sqlbase.ID, - fks fkHandler, - walltime int64, -) (*sqlbase.MutableTableDescriptor, error) { - create.HoistConstraints() - if create.IfNotExists { - return nil, pgerror.Unimplemented("import.if-no-exists", "unsupported IF NOT EXISTS") - } - if create.Interleave != nil { - return nil, pgerror.Unimplemented("import.interleave", "interleaved not supported") - } - if create.AsSource != nil { - return nil, pgerror.Unimplemented("import.create-as", "CREATE AS not supported") - } - - filteredDefs := create.Defs[:0] - for i := range create.Defs { - switch def := create.Defs[i].(type) { - case *tree.CheckConstraintTableDef, - *tree.FamilyTableDef, - *tree.IndexTableDef, - *tree.UniqueConstraintTableDef: - // ignore - case *tree.ColumnTableDef: - if def.Computed.Expr != nil { - return nil, pgerror.Unimplementedf("import.computed", "computed columns not supported: %s", tree.AsString(def)) - } - - if err := sql.SimplifySerialInColumnDefWithRowID(ctx, def, &create.Table); err != nil { - return nil, err - } - - case *tree.ForeignKeyConstraintTableDef: - if !fks.allowed { - return nil, pgerror.Unimplemented("import.fk", "this IMPORT format does not support foreign keys") - } - if fks.skip { - continue - } - // Strip the schema/db prefix. - def.Table = tree.MakeUnqualifiedTableName(def.Table.TableName) - - default: - return nil, pgerror.Unimplementedf(fmt.Sprintf("import.%T", def), "unsupported table definition: %s", tree.AsString(def)) - } - // only append this def after we make it past the error checks and continues - filteredDefs = append(filteredDefs, create.Defs[i]) - } - create.Defs = filteredDefs - - semaCtx := tree.SemaContext{} - evalCtx := tree.EvalContext{ - Context: ctx, - Sequence: &importSequenceOperators{}, - } - affected := make(map[sqlbase.ID]*sqlbase.MutableTableDescriptor) - - tableDesc, err := sql.MakeTableDesc( - ctx, - nil, /* txn */ - fks.resolver, - st, - create, - parentID, - tableID, - hlc.Timestamp{WallTime: walltime}, - sqlbase.NewDefaultPrivilegeDescriptor(), - affected, - &semaCtx, - &evalCtx, - ) - if err != nil { - return nil, err - } - if err := fixDescriptorFKState(tableDesc.TableDesc()); err != nil { - return nil, err - } - - return &tableDesc, nil -} - -// fixDescriptorFKState repairs validity and table states set during descriptor -// creation. sql.MakeTableDesc and ResolveFK set the table to the ADD state -// and mark references an validated. This function sets the table to PUBLIC -// and the FKs to unvalidated. -func fixDescriptorFKState(tableDesc *sqlbase.TableDescriptor) error { - tableDesc.State = sqlbase.TableDescriptor_PUBLIC - return tableDesc.ForeachNonDropIndex(func(idx *sqlbase.IndexDescriptor) error { - if idx.ForeignKey.IsSet() { - idx.ForeignKey.Validity = sqlbase.ConstraintValidity_Unvalidated - } - return nil - }) -} - -var ( - errSequenceOperators = errors.New("sequence operations unsupported") - errSchemaResolver = errors.New("schema resolver unsupported") -) - -// Implements the tree.SequenceOperators interface. -type importSequenceOperators struct { -} - -// Implements the tree.EvalDatabase interface. -func (so *importSequenceOperators) ParseQualifiedTableName( - ctx context.Context, sql string, -) (*tree.TableName, error) { - return parser.ParseTableName(sql) -} - -// Implements the tree.EvalDatabase interface. -func (so *importSequenceOperators) ResolveTableName(ctx context.Context, tn *tree.TableName) error { - return errSequenceOperators -} - -// Implements the tree.EvalDatabase interface. -func (so *importSequenceOperators) LookupSchema( - ctx context.Context, dbName, scName string, -) (bool, tree.SchemaMeta, error) { - return false, nil, errSequenceOperators -} - -// Implements the tree.SequenceOperators interface. -func (so *importSequenceOperators) IncrementSequence( - ctx context.Context, seqName *tree.TableName, -) (int64, error) { - return 0, errSequenceOperators -} - -// Implements the tree.SequenceOperators interface. -func (so *importSequenceOperators) GetLatestValueInSessionForSequence( - ctx context.Context, seqName *tree.TableName, -) (int64, error) { - return 0, errSequenceOperators -} - -// Implements the tree.SequenceOperators interface. -func (so *importSequenceOperators) SetSequenceValue( - ctx context.Context, seqName *tree.TableName, newVal int64, isCalled bool, -) error { - return errSequenceOperators -} - -type fkResolver map[string]*sqlbase.MutableTableDescriptor - -var _ sql.SchemaResolver = fkResolver{} - -// Implements the sql.SchemaResolver interface. -func (r fkResolver) Txn() *client.Txn { - return nil -} - -// Implements the sql.SchemaResolver interface. -func (r fkResolver) LogicalSchemaAccessor() sql.SchemaAccessor { - return nil -} - -// Implements the sql.SchemaResolver interface. -func (r fkResolver) CurrentDatabase() string { - return "" -} - -// Implements the sql.SchemaResolver interface. -func (r fkResolver) CurrentSearchPath() sessiondata.SearchPath { - return sessiondata.SearchPath{} -} - -// Implements the sql.SchemaResolver interface. -func (r fkResolver) CommonLookupFlags(required bool) sql.CommonLookupFlags { - return sql.CommonLookupFlags{} -} - -// Implements the sql.SchemaResolver interface. -func (r fkResolver) ObjectLookupFlags(required bool, requireMutable bool) sql.ObjectLookupFlags { - return sql.ObjectLookupFlags{} -} - -// Implements the tree.TableNameExistingResolver interface. -func (r fkResolver) LookupObject( - ctx context.Context, requireMutable bool, dbName, scName, obName string, -) (found bool, objMeta tree.NameResolutionResult, err error) { - if scName != "" { - obName = strings.TrimPrefix(obName, scName+".") - } - tbl, ok := r[obName] - if ok { - return true, tbl, nil - } - names := make([]string, 0, len(r)) - for k := range r { - names = append(names, k) - } - suggestions := strings.Join(names, ",") - return false, nil, errors.Errorf("referenced table %q not found in tables being imported (%s)", obName, suggestions) -} - -// Implements the tree.TableNameTargetResolver interface. -func (r fkResolver) LookupSchema( - ctx context.Context, dbName, scName string, -) (found bool, scMeta tree.SchemaMeta, err error) { - return false, nil, errSchemaResolver -} - -// Implements the sql.SchemaResolver interface. -func (r fkResolver) LookupTableByID(ctx context.Context, id sqlbase.ID) (row.TableEntry, error) { - return row.TableEntry{}, errSchemaResolver -} - -const csvDatabaseName = "csv" - -func finalizeCSVBackup( - ctx context.Context, - backupDesc *backupccl.BackupDescriptor, - parentID sqlbase.ID, - tables map[string]*sqlbase.TableDescriptor, - es storageccl.ExportStorage, - execCfg *sql.ExecutorConfig, -) error { - sort.Sort(backupccl.BackupFileDescriptors(backupDesc.Files)) - - backupDesc.Spans = make([]roachpb.Span, 0, len(tables)) - backupDesc.Descriptors = make([]sqlbase.Descriptor, 1, len(tables)+1) - backupDesc.Descriptors[0] = *sqlbase.WrapDescriptor( - &sqlbase.DatabaseDescriptor{Name: csvDatabaseName, ID: parentID}, - ) - - for _, table := range tables { - backupDesc.Spans = append(backupDesc.Spans, table.TableSpan()) - backupDesc.Descriptors = append(backupDesc.Descriptors, *sqlbase.WrapDescriptor(table)) - } - - backupDesc.FormatVersion = backupccl.BackupFormatInitialVersion - backupDesc.BuildInfo = build.GetInfo() - if execCfg != nil { - backupDesc.NodeID = execCfg.NodeID.Get() - backupDesc.ClusterID = execCfg.ClusterID() - } - descBuf, err := protoutil.Marshal(backupDesc) - if err != nil { - return err - } - return es.WriteFile(ctx, backupccl.BackupDescriptorName, bytes.NewReader(descBuf)) -} - func importJobDescription( orig *tree.Import, defs tree.TableDefs, files []string, opts map[string]string, ) (string, error) { @@ -415,14 +102,6 @@ func importJobDescription( } stmt.Options = nil for k, v := range opts { - switch k { - case importOptionTransform: - clean, err := storageccl.SanitizeExportStorageURI(v) - if err != nil { - return "", err - } - v = clean - } opt := tree.KVOption{Key: tree.Name(k)} val := importOptionExpectValues[k] == sql.KVStringOptRequireValue val = val || (importOptionExpectValues[k] == sql.KVStringOptAny && len(v) > 0) @@ -488,13 +167,9 @@ func importPlanHook( } table := importStmt.Table - transform := opts[importOptionTransform] var parentID sqlbase.ID - if transform != "" { - // If we're not ingesting the data, we don't care what DB we pick. - parentID = defaultCSVParentID - } else if table != nil { + if table != nil { // We have a target table, so it might specify a DB in its name. found, descI, err := table.ResolveTarget(ctx, p, p.SessionData().Database, p.SessionData().SearchPath) @@ -706,14 +381,10 @@ func importPlanHook( return errors.Errorf("Using %q requires all nodes to be upgraded to %s", importOptionDirectIngest, cluster.VersionByKey(cluster.VersionDirectImport)) } - if transform != "" { - return errors.Errorf("cannot use %q and %q options together", importOptionDirectIngest, importOptionTransform) - } } var tableDescs []*sqlbase.TableDescriptor var jobDesc string - var names []string seqVals := make(map[sqlbase.ID]int64) if importStmt.Bundle { store, err := storageccl.ExportStorageFromURI(ctx, files[0], p.ExecCfg().Settings) @@ -751,7 +422,7 @@ func importPlanHook( return err } if tableDescs == nil && table != nil { - names = []string{table.TableName.String()} + return errors.Errorf("table definition not found for %q", table.TableName.String()) } descStr, err := importJobDescription(importStmt, nil, files, opts) @@ -800,21 +471,9 @@ func importPlanHook( jobDesc = descStr } - if transform != "" { - transformStorage, err := storageccl.ExportStorageFromURI(ctx, transform, p.ExecCfg().Settings) - if err != nil { - return err - } - // Delay writing the BACKUP-CHECKPOINT file until as late as possible. - err = backupccl.VerifyUsableExportTarget(ctx, transformStorage, transform) - transformStorage.Close() - if err != nil { - return err - } - telemetry.Count("import.transform") - } else { + if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { for _, tableDesc := range tableDescs { - if err := backupccl.CheckTableExists(ctx, p.Txn(), parentID, tableDesc.Name); err != nil { + if err := backupccl.CheckTableExists(ctx, txn, parentID, tableDesc.Name); err != nil { return err } } @@ -833,20 +492,54 @@ func importPlanHook( TableID: id, ParentID: parentID, } - newSeqVals[id] = seqVals[tableDesc.ID] + if v, ok := seqVals[tableDesc.ID]; ok { + newSeqVals[id] = v + } } seqVals = newSeqVals if err := backupccl.RewriteTableDescs(tableDescs, tableRewrites, ""); err != nil { return err } + + for i := range tableDescs { + tableDescs[i].State = sqlbase.TableDescriptor_IMPORTING + } + + seqValKVs := make([]roachpb.KeyValue, 0, len(seqVals)) + for i := range tableDescs { + if v, ok := seqVals[tableDescs[i].ID]; ok && v != 0 { + key, val, err := sql.MakeSequenceKeyVal(tableDescs[i], v, false) + if err != nil { + return err + } + kv := roachpb.KeyValue{Key: key} + kv.Value.SetInt(val) + seqValKVs = append(seqValKVs, kv) + } + } + + // Write the new TableDescriptors and flip the namespace entries over to + // them. After this call, any queries on a table will be served by the newly + // imported data. + if err := backupccl.WriteTableDescs(ctx, txn, nil, tableDescs, p.User(), p.ExecCfg().Settings, seqValKVs); err != nil { + return pgerror.Wrapf(err, pgerror.CodeDataExceptionError, "creating tables") + } + + // TODO(dt): we should be creating the job with this txn too. Once a job + // is created, the contract is it does its own, explicit cleanup on + // failure (i.e. not just txn rollback) but everything up to and including + // the creation of the job *should* be a single atomic txn. As-is, if we + // fail to creat the job after committing this txn, we've leaving broken + // descs and namespace records. + + return nil + }); err != nil { + return err } tableDetails := make([]jobspb.ImportDetails_Table, 0, len(tableDescs)) for _, tbl := range tableDescs { - tableDetails = append(tableDetails, jobspb.ImportDetails_Table{Desc: tbl, SeqVal: seqVals[tbl.ID]}) - } - for _, name := range names { - tableDetails = append(tableDetails, jobspb.ImportDetails_Table{Name: name}) + tableDetails = append(tableDetails, jobspb.ImportDetails_Table{Desc: tbl, SeqVal: seqVals[tbl.ID], IsNew: true}) } telemetry.CountBucketed("import.files", int64(len(files))) @@ -859,7 +552,6 @@ func importPlanHook( Format: format, ParentID: parentID, Tables: tableDetails, - BackupPath: transform, SSTSize: sstSize, Oversample: oversample, Walltime: walltime, @@ -883,7 +575,6 @@ func doDistributedCSVTransform( p sql.PlanHookState, parentID sqlbase.ID, tables map[string]*sqlbase.TableDescriptor, - transformOnly string, format roachpb.IOFileFormat, walltime int64, sstSize int64, @@ -918,7 +609,6 @@ func doDistributedCSVTransform( sql.NewRowResultWriter(rows), tables, files, - transformOnly, format, walltime, sstSize, @@ -951,73 +641,18 @@ func doDistributedCSVTransform( return roachpb.BulkOpSummary{}, err } - backupDesc := backupccl.BackupDescriptor{ - EndTime: hlc.Timestamp{WallTime: walltime}, - } + var res roachpb.BulkOpSummary n := rows.Len() for i := 0; i < n; i++ { row := rows.At(i) - name := row[0].(*tree.DString) var counts roachpb.BulkOpSummary if err := protoutil.Unmarshal([]byte(*row[1].(*tree.DBytes)), &counts); err != nil { return roachpb.BulkOpSummary{}, err } - backupDesc.EntryCounts.Add(counts) - checksum := row[2].(*tree.DBytes) - spanStart := row[3].(*tree.DBytes) - spanEnd := row[4].(*tree.DBytes) - backupDesc.Files = append(backupDesc.Files, backupccl.BackupDescriptor_File{ - Path: string(*name), - Span: roachpb.Span{ - Key: roachpb.Key(*spanStart), - EndKey: roachpb.Key(*spanEnd), - }, - Sha512: []byte(*checksum), - }) + res.Add(counts) } - if transformOnly == "" { - return backupDesc.EntryCounts, nil - } - - // The returned spans are from the SSTs themselves, and so don't perfectly - // overlap. Sort the files so we can fix the spans to be correctly - // overlapping. This is needed because RESTORE splits at both the start - // and end of each SST, and so there are tiny ranges (like {NULL-/0/0} at - // the start) that get created. During non-transform IMPORT this isn't a - // problem because it only splits on the end key. Replicate that behavior - // here by copying the end key from each span to the start key of the next. - sort.Slice(backupDesc.Files, func(i, j int) bool { - return backupDesc.Files[i].Span.Key.Compare(backupDesc.Files[j].Span.Key) < 0 - }) - - var minTableSpan, maxTableSpan roachpb.Key - for _, tableDesc := range tables { - span := tableDesc.TableSpan() - if minTableSpan == nil || span.Key.Compare(minTableSpan) < 0 { - minTableSpan = span.Key - } - if maxTableSpan == nil || span.EndKey.Compare(maxTableSpan) > 0 { - maxTableSpan = span.EndKey - } - } - backupDesc.Files[0].Span.Key = minTableSpan - for i := 1; i < len(backupDesc.Files); i++ { - backupDesc.Files[i].Span.Key = backupDesc.Files[i-1].Span.EndKey - } - backupDesc.Files[len(backupDesc.Files)-1].Span.EndKey = maxTableSpan - - dest, err := storageccl.ExportStorageConfFromURI(transformOnly) - if err != nil { - return roachpb.BulkOpSummary{}, err - } - es, err := storageccl.MakeExportStorage(ctx, dest, p.ExecCfg().Settings) - if err != nil { - return roachpb.BulkOpSummary{}, err - } - defer es.Close() - - return backupDesc.EntryCounts, finalizeCSVBackup(ctx, &backupDesc, parentID, tables, es, p.ExecCfg()) + return res, nil } type importResumer struct { @@ -1034,10 +669,11 @@ func (r *importResumer) Resume( details := r.job.Details().(jobspb.ImportDetails) p := phs.(sql.PlanHookState) - // TODO(dt): consider looking at the legacy fields used in 2.0. + if details.BackupPath != "" { + return errors.Errorf("transform is no longer supported") + } walltime := details.Walltime - transform := details.BackupPath files := details.URIs parentID := details.ParentID sstSize := details.SSTSize @@ -1083,7 +719,7 @@ func (r *importResumer) Resume( } res, err := doDistributedCSVTransform( - ctx, r.job, files, p, parentID, tables, transform, format, walltime, sstSize, oversample, ingestDirectly, + ctx, r.job, files, p, parentID, tables, format, walltime, sstSize, oversample, ingestDirectly, ) if err != nil { return err @@ -1099,9 +735,6 @@ func (r *importResumer) Resume( // stuff to delete the keys in the background. func (r *importResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) error { details := r.job.Details().(jobspb.ImportDetails) - if details.BackupPath != "" { - return nil - } // Needed to trigger the schema change manager. if err := txn.SetSystemConfigTrigger(); err != nil { @@ -1109,18 +742,30 @@ func (r *importResumer) OnFailOrCancel(ctx context.Context, txn *client.Txn) err } b := txn.NewBatch() for _, tbl := range details.Tables { - tableDesc := tbl.Desc - tableDesc.State = sqlbase.TableDescriptor_DROP - // If the DropTime if set, a table uses RangeClear for fast data removal. This - // operation starts at DropTime + the GC TTL. If we used now() here, it would - // not clean up data until the TTL from the time of the error. Instead, use 1 - // (that is, 1ns past the epoch) to allow this to be cleaned up as soon as - // possible. This is safe since the table data was never visible to users, - // and so we don't need to preserve MVCC semantics. - tableDesc.DropTime = 1 - b.CPut(sqlbase.MakeDescMetadataKey(tableDesc.ID), sqlbase.WrapDescriptor(tableDesc), nil) + tableDesc := *tbl.Desc + tableDesc.Version++ + if tbl.IsNew { + tableDesc.State = sqlbase.TableDescriptor_DROP + // If the DropTime if set, a table uses RangeClear for fast data removal. This + // operation starts at DropTime + the GC TTL. If we used now() here, it would + // not clean up data until the TTL from the time of the error. Instead, use 1 + // (that is, 1ns past the epoch) to allow this to be cleaned up as soon as + // possible. This is safe since the table data was never visible to users, + // and so we don't need to preserve MVCC semantics. + tableDesc.DropTime = 1 + b.CPut(sqlbase.MakeNameMetadataKey(tableDesc.ParentID, tableDesc.Name), nil, tableDesc.ID) + } else { + // IMPORT did not create this table, so we should not drop it. + // TODO(dt): consider trying to delete whatever was ingested before + // returning the table to public. Unfortunately the ingestion isn't + // transactional, so there is no clean way to just rollback our changes, + // but we could iterate by time to delete before returning to public. + tableDesc.Version++ + tableDesc.State = sqlbase.TableDescriptor_PUBLIC + } + b.CPut(sqlbase.MakeDescMetadataKey(tableDesc.ID), sqlbase.WrapDescriptor(&tableDesc), sqlbase.WrapDescriptor(tbl.Desc)) } - return txn.Run(ctx, b) + return errors.Wrap(txn.Run(ctx, b), "rolling back tables") } // OnSuccess is part of the jobs.Resumer interface. @@ -1128,38 +773,26 @@ func (r *importResumer) OnSuccess(ctx context.Context, txn *client.Txn) error { log.Event(ctx, "making tables live") details := r.job.Details().(jobspb.ImportDetails) - if details.BackupPath != "" { - return nil + // Needed to trigger the schema change manager. + if err := txn.SetSystemConfigTrigger(); err != nil { + return err } - - toWrite := make([]*sqlbase.TableDescriptor, len(details.Tables)) - var seqs []roachpb.KeyValue - for i := range details.Tables { - toWrite[i] = details.Tables[i].Desc - toWrite[i].ParentID = details.ParentID - if d := details.Tables[i]; d.SeqVal != 0 { - key, val, err := sql.MakeSequenceKeyVal(d.Desc, d.SeqVal, false) - if err != nil { - return err - } - kv := roachpb.KeyValue{Key: key} - kv.Value.SetInt(val) - seqs = append(seqs, kv) - } + b := txn.NewBatch() + for _, tbl := range details.Tables { + tableDesc := *tbl.Desc + tableDesc.Version++ + tableDesc.State = sqlbase.TableDescriptor_PUBLIC + b.CPut(sqlbase.MakeDescMetadataKey(tableDesc.ID), sqlbase.WrapDescriptor(&tableDesc), sqlbase.WrapDescriptor(tbl.Desc)) } - - // Write the new TableDescriptors and flip the namespace entries over to - // them. After this call, any queries on a table will be served by the newly - // imported data. - if err := backupccl.WriteTableDescs(ctx, txn, nil, toWrite, r.job.Payload().Username, r.settings, seqs); err != nil { - return pgerror.Wrapf(err, pgerror.CodeDataExceptionError, "creating tables") + if err := txn.Run(ctx, b); err != nil { + return errors.Wrap(err, "publishing tables") } // Initiate a run of CREATE STATISTICS. We don't know the actual number of // rows affected per table, so we use a large number because we want to make // sure that stats always get created/refreshed here. - for i := range toWrite { - r.statsRefresher.NotifyMutation(toWrite[i].ID, math.MaxInt32 /* rowsAffected */) + for i := range details.Tables { + r.statsRefresher.NotifyMutation(details.Tables[i].Desc.ID, math.MaxInt32 /* rowsAffected */) } return nil @@ -1169,21 +802,6 @@ func (r *importResumer) OnSuccess(ctx context.Context, txn *client.Txn) error { func (r *importResumer) OnTerminal( ctx context.Context, status jobs.Status, resultsCh chan<- tree.Datums, ) { - details := r.job.Details().(jobspb.ImportDetails) - - if transform := details.BackupPath; transform != "" { - transformStorage, err := storageccl.ExportStorageFromURI(ctx, transform, r.settings) - if err != nil { - log.Warningf(ctx, "unable to create storage: %+v", err) - } else { - // Always attempt to cleanup the checkpoint even if the import failed. - if err := transformStorage.Delete(ctx, backupccl.BackupDescriptorCheckpointName); err != nil { - log.Warningf(ctx, "unable to delete checkpointed backup descriptor: %+v", err) - } - transformStorage.Close() - } - } - if status == jobs.StatusSucceeded { telemetry.CountBucketed("import.rows", r.res.Rows) const mb = 1 << 20 diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go index d422ef0e445a..a6380515c2d9 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -998,14 +998,6 @@ func TestImportCSVStmt(t *testing.T) { ` WITH sstsize = '10K'`, "", }, - { - "schema-in-query-transform-only", - `IMPORT TABLE t (a INT8 PRIMARY KEY, b STRING, INDEX (b), INDEX (a, b)) CSV DATA (%s) WITH delimiter = '|', comment = '#', nullif='', skip = '2', transform = $1`, - nil, - filesWithOpts, - ` WITH comment = '#', delimiter = '|', "nullif" = '', skip = '2', transform = 'nodelocal:///5'`, - "", - }, { "empty-file", `IMPORT TABLE t CREATE USING $1 CSV DATA (%s)`, @@ -1095,17 +1087,9 @@ func TestImportCSVStmt(t *testing.T) { ``, "invalid option \"foo\"", }, - { - "bad-opt-no-arg", - `IMPORT TABLE t (a INT8 PRIMARY KEY, b STRING, INDEX (b), INDEX (a, b)) CSV DATA (%s) WITH transform`, - nil, - files, - ``, - "option \"transform\" requires a value", - }, { "bad-computed-column", - `IMPORT TABLE t (a INT8 PRIMARY KEY, b STRING AS ('hello') STORED, INDEX (b), INDEX (a, b)) CSV DATA (%s) WITH skip = '2', transform = $1`, + `IMPORT TABLE t (a INT8 PRIMARY KEY, b STRING AS ('hello') STORED, INDEX (b), INDEX (a, b)) CSV DATA (%s) WITH skip = '2'`, nil, filesWithOpts, ``, @@ -1113,7 +1097,7 @@ func TestImportCSVStmt(t *testing.T) { }, { "primary-key-dup", - `IMPORT TABLE t CREATE USING $1 CSV DATA (%s) WITH transform = $2`, + `IMPORT TABLE t CREATE USING $1 CSV DATA (%s)`, schema, dups, ``, @@ -1165,12 +1149,6 @@ func TestImportCSVStmt(t *testing.T) { rows, idx, sys, bytes int } - backupPath := fmt.Sprintf("nodelocal:///%d", i) - hasTransform := strings.Contains(tc.query, "transform = $") - if hasTransform { - tc.args = append(tc.args, backupPath) - } - var result int query := fmt.Sprintf(tc.query, strings.Join(tc.files, ", ")) testNum++ @@ -1182,38 +1160,17 @@ func TestImportCSVStmt(t *testing.T) { &unused, &unused, &unused, &restored.rows, &restored.idx, &restored.sys, &restored.bytes, ) - jobPrefix := `IMPORT TABLE ` - if !hasTransform { - jobPrefix += intodb + ".public." - } else { - jobPrefix += `""."".` - } - jobPrefix += `t (a INT8 PRIMARY KEY, b STRING, INDEX (b), INDEX (a, b)) CSV DATA (%s)` + jobPrefix := fmt.Sprintf(`IMPORT TABLE %s.public.t (a INT8 PRIMARY KEY, b STRING, INDEX (b), INDEX (a, b))`, intodb) if err := jobutils.VerifySystemJob(t, sqlDB, testNum, jobspb.TypeImport, jobs.StatusSucceeded, jobs.Record{ Username: security.RootUser, - Description: fmt.Sprintf(jobPrefix+tc.jobOpts, strings.Join(tc.files, ", ")), + Description: fmt.Sprintf(jobPrefix+` CSV DATA (%s)`+tc.jobOpts, strings.Join(tc.files, ", ")), }); err != nil { t.Fatal(err) } isEmpty := len(tc.files) == 1 && tc.files[0] == empty[0] - if hasTransform { - sqlDB.ExpectErr( - t, "does not exist", - `SELECT count(*) FROM t`, - ) - sqlDB.QueryRow( - t, `RESTORE csv.* FROM $1 WITH into_db = $2`, backupPath, intodb, - ).Scan( - &unused, &unused, &unused, &restored.rows, &restored.idx, &restored.sys, &restored.bytes, - ) - if expected, actual := expectedRows, restored.rows; expected != actual && !isEmpty { - t.Fatalf("expected %d rows, got %d", expected, actual) - } - } - if isEmpty { sqlDB.QueryRow(t, `SELECT count(*) FROM t`).Scan(&result) if expect := 0; result != expect { @@ -1394,18 +1351,15 @@ func BenchmarkImport(b *testing.B) { files, _, _ := makeCSVData(b, dir, numFiles, b.N*100) files = nodelocalPrefix(files) - tmp := fmt.Sprintf("nodelocal://%s", filepath.Join(dir, b.Name())) b.ResetTimer() sqlDB.Exec(b, fmt.Sprintf( `IMPORT TABLE t (a INT8 PRIMARY KEY, b STRING, INDEX (b), INDEX (a, b)) - CSV DATA (%s) WITH transform = $1`, + CSV DATA (%s)`, strings.Join(files, ","), - ), - tmp, - ) + )) } func BenchmarkConvertRecord(b *testing.B) { diff --git a/pkg/ccl/importccl/import_table_creation.go b/pkg/ccl/importccl/import_table_creation.go new file mode 100644 index 000000000000..ee2eb107215e --- /dev/null +++ b/pkg/ccl/importccl/import_table_creation.go @@ -0,0 +1,296 @@ +// Copyright 2017 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package importccl + +import ( + "context" + "fmt" + "io/ioutil" + "strings" + + "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" + "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/row" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/pkg/errors" +) + +const ( + // We need to choose arbitrary database and table IDs. These aren't important, + // but they do match what would happen when creating a new database and + // table on an empty cluster. + defaultCSVParentID sqlbase.ID = keys.MinNonPredefinedUserDescID + defaultCSVTableID sqlbase.ID = defaultCSVParentID + 1 +) + +func readCreateTableFromStore( + ctx context.Context, filename string, settings *cluster.Settings, +) (*tree.CreateTable, error) { + store, err := storageccl.ExportStorageFromURI(ctx, filename, settings) + if err != nil { + return nil, err + } + defer store.Close() + reader, err := store.ReadFile(ctx, "") + if err != nil { + return nil, err + } + defer reader.Close() + tableDefStr, err := ioutil.ReadAll(reader) + if err != nil { + return nil, err + } + stmt, err := parser.ParseOne(string(tableDefStr)) + if err != nil { + return nil, err + } + create, ok := stmt.AST.(*tree.CreateTable) + if !ok { + return nil, errors.New("expected CREATE TABLE statement in table file") + } + return create, nil +} + +type fkHandler struct { + allowed bool + skip bool + resolver fkResolver +} + +// NoFKs is used by formats that do not support FKs. +var NoFKs = fkHandler{resolver: make(fkResolver)} + +// MakeSimpleTableDescriptor creates a MutableTableDescriptor from a CreateTable parse +// node without the full machinery. Many parts of the syntax are unsupported +// (see the implementation and TestMakeSimpleTableDescriptorErrors for details), +// but this is enough for our csv IMPORT and for some unit tests. +// +// Any occurrence of SERIAL in the column definitions is handled using +// the CockroachDB legacy behavior, i.e. INT NOT NULL DEFAULT +// unique_rowid(). +func MakeSimpleTableDescriptor( + ctx context.Context, + st *cluster.Settings, + create *tree.CreateTable, + parentID, tableID sqlbase.ID, + fks fkHandler, + walltime int64, +) (*sqlbase.MutableTableDescriptor, error) { + create.HoistConstraints() + if create.IfNotExists { + return nil, pgerror.Unimplemented("import.if-no-exists", "unsupported IF NOT EXISTS") + } + if create.Interleave != nil { + return nil, pgerror.Unimplemented("import.interleave", "interleaved not supported") + } + if create.AsSource != nil { + return nil, pgerror.Unimplemented("import.create-as", "CREATE AS not supported") + } + + filteredDefs := create.Defs[:0] + for i := range create.Defs { + switch def := create.Defs[i].(type) { + case *tree.CheckConstraintTableDef, + *tree.FamilyTableDef, + *tree.IndexTableDef, + *tree.UniqueConstraintTableDef: + // ignore + case *tree.ColumnTableDef: + if def.Computed.Expr != nil { + return nil, pgerror.Unimplementedf("import.computed", "computed columns not supported: %s", tree.AsString(def)) + } + + if err := sql.SimplifySerialInColumnDefWithRowID(ctx, def, &create.Table); err != nil { + return nil, err + } + + case *tree.ForeignKeyConstraintTableDef: + if !fks.allowed { + return nil, pgerror.Unimplemented("import.fk", "this IMPORT format does not support foreign keys") + } + if fks.skip { + continue + } + // Strip the schema/db prefix. + def.Table = tree.MakeUnqualifiedTableName(def.Table.TableName) + + default: + return nil, pgerror.Unimplementedf(fmt.Sprintf("import.%T", def), "unsupported table definition: %s", tree.AsString(def)) + } + // only append this def after we make it past the error checks and continues + filteredDefs = append(filteredDefs, create.Defs[i]) + } + create.Defs = filteredDefs + + semaCtx := tree.SemaContext{} + evalCtx := tree.EvalContext{ + Context: ctx, + Sequence: &importSequenceOperators{}, + } + affected := make(map[sqlbase.ID]*sqlbase.MutableTableDescriptor) + + tableDesc, err := sql.MakeTableDesc( + ctx, + nil, /* txn */ + fks.resolver, + st, + create, + parentID, + tableID, + hlc.Timestamp{WallTime: walltime}, + sqlbase.NewDefaultPrivilegeDescriptor(), + affected, + &semaCtx, + &evalCtx, + ) + if err != nil { + return nil, err + } + if err := fixDescriptorFKState(tableDesc.TableDesc()); err != nil { + return nil, err + } + + return &tableDesc, nil +} + +// fixDescriptorFKState repairs validity and table states set during descriptor +// creation. sql.MakeTableDesc and ResolveFK set the table to the ADD state +// and mark references an validated. This function sets the table to PUBLIC +// and the FKs to unvalidated. +func fixDescriptorFKState(tableDesc *sqlbase.TableDescriptor) error { + tableDesc.State = sqlbase.TableDescriptor_PUBLIC + return tableDesc.ForeachNonDropIndex(func(idx *sqlbase.IndexDescriptor) error { + if idx.ForeignKey.IsSet() { + idx.ForeignKey.Validity = sqlbase.ConstraintValidity_Unvalidated + } + return nil + }) +} + +var ( + errSequenceOperators = errors.New("sequence operations unsupported") + errSchemaResolver = errors.New("schema resolver unsupported") +) + +// Implements the tree.SequenceOperators interface. +type importSequenceOperators struct { +} + +// Implements the tree.EvalDatabase interface. +func (so *importSequenceOperators) ParseQualifiedTableName( + ctx context.Context, sql string, +) (*tree.TableName, error) { + return parser.ParseTableName(sql) +} + +// Implements the tree.EvalDatabase interface. +func (so *importSequenceOperators) ResolveTableName(ctx context.Context, tn *tree.TableName) error { + return errSequenceOperators +} + +// Implements the tree.EvalDatabase interface. +func (so *importSequenceOperators) LookupSchema( + ctx context.Context, dbName, scName string, +) (bool, tree.SchemaMeta, error) { + return false, nil, errSequenceOperators +} + +// Implements the tree.SequenceOperators interface. +func (so *importSequenceOperators) IncrementSequence( + ctx context.Context, seqName *tree.TableName, +) (int64, error) { + return 0, errSequenceOperators +} + +// Implements the tree.SequenceOperators interface. +func (so *importSequenceOperators) GetLatestValueInSessionForSequence( + ctx context.Context, seqName *tree.TableName, +) (int64, error) { + return 0, errSequenceOperators +} + +// Implements the tree.SequenceOperators interface. +func (so *importSequenceOperators) SetSequenceValue( + ctx context.Context, seqName *tree.TableName, newVal int64, isCalled bool, +) error { + return errSequenceOperators +} + +type fkResolver map[string]*sqlbase.MutableTableDescriptor + +var _ sql.SchemaResolver = fkResolver{} + +// Implements the sql.SchemaResolver interface. +func (r fkResolver) Txn() *client.Txn { + return nil +} + +// Implements the sql.SchemaResolver interface. +func (r fkResolver) LogicalSchemaAccessor() sql.SchemaAccessor { + return nil +} + +// Implements the sql.SchemaResolver interface. +func (r fkResolver) CurrentDatabase() string { + return "" +} + +// Implements the sql.SchemaResolver interface. +func (r fkResolver) CurrentSearchPath() sessiondata.SearchPath { + return sessiondata.SearchPath{} +} + +// Implements the sql.SchemaResolver interface. +func (r fkResolver) CommonLookupFlags(required bool) sql.CommonLookupFlags { + return sql.CommonLookupFlags{} +} + +// Implements the sql.SchemaResolver interface. +func (r fkResolver) ObjectLookupFlags(required bool, requireMutable bool) sql.ObjectLookupFlags { + return sql.ObjectLookupFlags{} +} + +// Implements the tree.TableNameExistingResolver interface. +func (r fkResolver) LookupObject( + ctx context.Context, requireMutable bool, dbName, scName, obName string, +) (found bool, objMeta tree.NameResolutionResult, err error) { + if scName != "" { + obName = strings.TrimPrefix(obName, scName+".") + } + tbl, ok := r[obName] + if ok { + return true, tbl, nil + } + names := make([]string, 0, len(r)) + for k := range r { + names = append(names, k) + } + suggestions := strings.Join(names, ",") + return false, nil, errors.Errorf("referenced table %q not found in tables being imported (%s)", obName, suggestions) +} + +// Implements the tree.TableNameTargetResolver interface. +func (r fkResolver) LookupSchema( + ctx context.Context, dbName, scName string, +) (found bool, scMeta tree.SchemaMeta, err error) { + return false, nil, errSchemaResolver +} + +// Implements the sql.SchemaResolver interface. +func (r fkResolver) LookupTableByID(ctx context.Context, id sqlbase.ID) (row.TableEntry, error) { + return row.TableEntry{}, errSchemaResolver +} diff --git a/pkg/ccl/importccl/sst_writer_proc.go b/pkg/ccl/importccl/sst_writer_proc.go index 9e62ab88f59b..6156040bf987 100644 --- a/pkg/ccl/importccl/sst_writer_proc.go +++ b/pkg/ccl/importccl/sst_writer_proc.go @@ -9,7 +9,6 @@ package importccl import ( - "bytes" "context" "fmt" "sort" @@ -66,6 +65,10 @@ func newSSTWriterProcessor( if err := sp.out.Init(&distsqlpb.PostProcessSpec{}, sstOutputTypes, flowCtx.NewEvalCtx(), output); err != nil { return nil, err } + if sp.spec.Destination != "" { + return nil, errors.Errorf("writing external sst is not supported") + } + return sp, nil } @@ -173,53 +176,33 @@ func (sp *sstWriter) Run(ctx context.Context) { end = sst.span.EndKey } - if sp.spec.Destination == "" { - if err := sp.db.AdminSplit(ctx, end, end); err != nil { - return err - } + if err := sp.db.AdminSplit(ctx, end, end); err != nil { + return err + } - log.VEventf(ctx, 1, "scattering key %s", roachpb.PrettyPrintKey(nil, end)) - scatterReq := &roachpb.AdminScatterRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(sst.span), - // TODO(dan): This is a bit of a hack, but it seems to be an - // effective one (see the PR that added it for graphs). As of the - // commit that added this, scatter is not very good at actually - // balancing leases. This is likely for two reasons: 1) there's - // almost certainly some regression in scatter's behavior, it used - // to work much better and 2) scatter has to operate by balancing - // leases for all ranges in a cluster, but in IMPORT, we really - // just want it to be balancing the span being imported into. - RandomizeLeases: true, - } - if _, pErr := client.SendWrapped(ctx, sp.db.NonTransactionalSender(), scatterReq); pErr != nil { - // TODO(dan): Unfortunately, Scatter is still too unreliable to - // fail the IMPORT when Scatter fails. I'm uncomfortable that - // this could break entirely and not start failing the tests, - // but on the bright side, it doesn't affect correctness, only - // throughput. - log.Errorf(ctx, "failed to scatter span %s: %s", roachpb.PrettyPrintKey(nil, end), pErr) - } - if err := bulk.AddSSTable(ctx, sp.db, sst.span.Key, sst.span.EndKey, sst.data); err != nil { - return err - } - } else { - checksum, err = storageccl.SHA512ChecksumData(sst.data) - if err != nil { - return err - } - conf, err := storageccl.ExportStorageConfFromURI(sp.spec.Destination) - if err != nil { - return err - } - es, err := storageccl.MakeExportStorage(ctx, conf, sp.settings) - if err != nil { - return err - } - err = es.WriteFile(ctx, name, bytes.NewReader(sst.data)) - es.Close() - if err != nil { - return err - } + log.VEventf(ctx, 1, "scattering key %s", roachpb.PrettyPrintKey(nil, end)) + scatterReq := &roachpb.AdminScatterRequest{ + RequestHeader: roachpb.RequestHeaderFromSpan(sst.span), + // TODO(dan): This is a bit of a hack, but it seems to be an + // effective one (see the PR that added it for graphs). As of the + // commit that added this, scatter is not very good at actually + // balancing leases. This is likely for two reasons: 1) there's + // almost certainly some regression in scatter's behavior, it used + // to work much better and 2) scatter has to operate by balancing + // leases for all ranges in a cluster, but in IMPORT, we really + // just want it to be balancing the span being imported into. + RandomizeLeases: true, + } + if _, pErr := client.SendWrapped(ctx, sp.db.NonTransactionalSender(), scatterReq); pErr != nil { + // TODO(dan): Unfortunately, Scatter is still too unreliable to + // fail the IMPORT when Scatter fails. I'm uncomfortable that + // this could break entirely and not start failing the tests, + // but on the bright side, it doesn't affect correctness, only + // throughput. + log.Errorf(ctx, "failed to scatter span %s: %s", roachpb.PrettyPrintKey(nil, end), pErr) + } + if err := bulk.AddSSTable(ctx, sp.db, sst.span.Key, sst.span.EndKey, sst.data); err != nil { + return err } countsBytes, err := protoutil.Marshal(&sst.counts) diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 38f4c076ab24..97ac3624d0ee 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -55,7 +55,7 @@ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } func (Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_78ea1a647b3a58c0, []int{0} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{0} } type Type int32 @@ -93,7 +93,7 @@ var Type_value = map[string]int32{ } func (Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_78ea1a647b3a58c0, []int{1} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{1} } type Lease struct { @@ -109,7 +109,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_78ea1a647b3a58c0, []int{0} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{0} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -147,7 +147,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_78ea1a647b3a58c0, []int{1} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{1} } func (m *BackupDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -181,7 +181,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_78ea1a647b3a58c0, []int{2} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{2} } func (m *BackupProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -220,7 +220,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_78ea1a647b3a58c0, []int{3} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{3} } func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -256,7 +256,7 @@ func (m *RestoreDetails_TableRewrite) Reset() { *m = RestoreDetails_Tabl func (m *RestoreDetails_TableRewrite) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_TableRewrite) ProtoMessage() {} func (*RestoreDetails_TableRewrite) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_78ea1a647b3a58c0, []int{3, 0} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{3, 0} } func (m *RestoreDetails_TableRewrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -291,7 +291,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_78ea1a647b3a58c0, []int{4} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{4} } func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -344,7 +344,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_78ea1a647b3a58c0, []int{5} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{5} } func (m *ImportDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -373,6 +373,7 @@ type ImportDetails_Table struct { Desc *sqlbase.TableDescriptor `protobuf:"bytes,1,opt,name=desc,proto3" json:"desc,omitempty"` Name string `protobuf:"bytes,18,opt,name=name,proto3" json:"name,omitempty"` SeqVal int64 `protobuf:"varint,19,opt,name=seq_val,json=seqVal,proto3" json:"seq_val,omitempty"` + IsNew bool `protobuf:"varint,20,opt,name=is_new,json=isNew,proto3" json:"is_new,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_sizecache int32 `json:"-"` } @@ -381,7 +382,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_78ea1a647b3a58c0, []int{5, 0} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{5, 0} } func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -422,7 +423,7 @@ func (m *ImportProgress) Reset() { *m = ImportProgress{} } func (m *ImportProgress) String() string { return proto.CompactTextString(m) } func (*ImportProgress) ProtoMessage() {} func (*ImportProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_78ea1a647b3a58c0, []int{6} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{6} } func (m *ImportProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -457,7 +458,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_78ea1a647b3a58c0, []int{7} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{7} } func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -494,7 +495,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_78ea1a647b3a58c0, []int{8} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{8} } func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -538,7 +539,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_78ea1a647b3a58c0, []int{9} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{9} } func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -572,7 +573,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_78ea1a647b3a58c0, []int{10} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{10} } func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -607,7 +608,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_78ea1a647b3a58c0, []int{11} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{11} } func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -662,7 +663,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_78ea1a647b3a58c0, []int{12} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{12} } func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -698,7 +699,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_78ea1a647b3a58c0, []int{13} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{13} } func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -733,7 +734,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_78ea1a647b3a58c0, []int{14} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{14} } func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -779,7 +780,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_78ea1a647b3a58c0, []int{15} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{15} } func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -814,7 +815,7 @@ func (m *CreateStatsDetails_ColList) Reset() { *m = CreateStatsDetails_C func (m *CreateStatsDetails_ColList) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails_ColList) ProtoMessage() {} func (*CreateStatsDetails_ColList) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_78ea1a647b3a58c0, []int{15, 0} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{15, 0} } func (m *CreateStatsDetails_ColList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -848,7 +849,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_78ea1a647b3a58c0, []int{16} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{16} } func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -903,7 +904,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_78ea1a647b3a58c0, []int{17} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{17} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1182,7 +1183,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_78ea1a647b3a58c0, []int{18} + return fileDescriptor_jobs_6106a12eb5231ca0, []int{18} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1954,6 +1955,18 @@ func (m *ImportDetails_Table) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintJobs(dAtA, i, uint64(m.SeqVal)) } + if m.IsNew { + dAtA[i] = 0xa0 + i++ + dAtA[i] = 0x1 + i++ + if m.IsNew { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -2954,6 +2967,9 @@ func (m *ImportDetails_Table) Size() (n int) { if m.SeqVal != 0 { n += 2 + sovJobs(uint64(m.SeqVal)) } + if m.IsNew { + n += 3 + } return n } @@ -4610,6 +4626,26 @@ func (m *ImportDetails_Table) Unmarshal(dAtA []byte) error { break } } + case 20: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsNew", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.IsNew = bool(v != 0) default: iNdEx = preIndex skippy, err := skipJobs(dAtA[iNdEx:]) @@ -7208,160 +7244,161 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_78ea1a647b3a58c0) } - -var fileDescriptor_jobs_78ea1a647b3a58c0 = []byte{ - // 2428 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0xdf, 0x6f, 0x2b, 0x47, - 0xf5, 0xf7, 0xda, 0x6b, 0x7b, 0x7d, 0xfc, 0x23, 0xeb, 0x49, 0xbe, 0xed, 0xd6, 0x6a, 0x63, 0x7f, - 0x0d, 0x6d, 0xd3, 0x5b, 0x6a, 0x43, 0x2a, 0x5a, 0x5a, 0x41, 0x85, 0x7f, 0xe5, 0xc6, 0x6e, 0x6e, - 0x9c, 0xae, 0x9d, 0x16, 0x8a, 0x60, 0x59, 0x7b, 0x27, 0xf1, 0x12, 0xdb, 0xbb, 0xd9, 0x59, 0xdf, - 0xde, 0x5b, 0x21, 0x21, 0x21, 0x1e, 0x50, 0x9e, 0xf8, 0x03, 0x88, 0x84, 0x04, 0x48, 0xbc, 0x20, - 0xfa, 0x67, 0xdc, 0x17, 0x44, 0x11, 0x2f, 0x20, 0x24, 0x17, 0xcc, 0x0b, 0x7f, 0xc3, 0x7d, 0x42, - 0x33, 0xb3, 0xbb, 0x5e, 0xe7, 0x86, 0xfc, 0x92, 0x78, 0x49, 0xbc, 0x67, 0xce, 0x39, 0x3b, 0xe7, - 0x9c, 0xcf, 0xf9, 0xcc, 0x99, 0x85, 0xe7, 0x7e, 0x64, 0x0d, 0x48, 0x95, 0xfe, 0xb1, 0x07, 0xec, - 0x5f, 0xc5, 0x76, 0x2c, 0xd7, 0x42, 0x2f, 0x0c, 0xad, 0xe1, 0x89, 0x63, 0xe9, 0xc3, 0x51, 0x85, - 0x9c, 0x8e, 0x2b, 0x6c, 0x85, 0x6b, 0x15, 0x36, 0x8e, 0xad, 0x63, 0x8b, 0x69, 0x55, 0xe9, 0x2f, - 0x6e, 0x50, 0x40, 0x4c, 0xd9, 0x1e, 0x54, 0x0d, 0xdd, 0xd5, 0x3d, 0x99, 0xe2, 0xcb, 0x4c, 0xeb, - 0x8d, 0x23, 0xcb, 0x99, 0xe8, 0xae, 0xe7, 0xbe, 0xf0, 0x22, 0x39, 0x1d, 0x57, 0xc9, 0xe9, 0x78, - 0xa0, 0x13, 0x5c, 0x25, 0xae, 0x33, 0x1b, 0xba, 0x33, 0x07, 0x1b, 0xbe, 0xdd, 0xcc, 0x35, 0xc7, - 0xd5, 0xd1, 0x78, 0x58, 0x75, 0xcd, 0x09, 0x26, 0xae, 0x3e, 0xb1, 0xf9, 0x4a, 0xf9, 0x27, 0x10, - 0xdf, 0xc3, 0x3a, 0xc1, 0xe8, 0x63, 0x48, 0x4e, 0x2d, 0x03, 0x6b, 0xa6, 0xa1, 0x08, 0x25, 0x61, - 0x2b, 0x5b, 0xaf, 0x2d, 0xe6, 0xc5, 0xc4, 0xbe, 0x65, 0xe0, 0x76, 0xf3, 0xe9, 0xbc, 0xf8, 0xe6, - 0xb1, 0xe9, 0x8e, 0x66, 0x83, 0xca, 0xd0, 0x9a, 0x54, 0x83, 0x48, 0x8c, 0xc1, 0xf2, 0x77, 0xd5, - 0x3e, 0x39, 0xae, 0x7a, 0xdb, 0xab, 0x70, 0x33, 0x35, 0x41, 0x3d, 0xb6, 0x0d, 0xb4, 0x01, 0x71, - 0x6c, 0x5b, 0xc3, 0x91, 0x12, 0x2d, 0x09, 0x5b, 0x31, 0x95, 0x3f, 0xbc, 0x2b, 0xfe, 0xfb, 0x57, - 0x45, 0xa1, 0xfc, 0x77, 0x01, 0xb2, 0x75, 0x7d, 0x78, 0x32, 0xb3, 0x9b, 0xd8, 0xd5, 0xcd, 0x31, - 0x41, 0x75, 0x00, 0xe2, 0xea, 0x8e, 0xab, 0xd1, 0xbd, 0xb2, 0xcd, 0xa4, 0xb7, 0x5f, 0xaa, 0x2c, - 0xd3, 0x47, 0x63, 0xa9, 0x8c, 0xc6, 0xc3, 0x4a, 0xdf, 0x8f, 0xa5, 0x2e, 0x3e, 0x99, 0x17, 0x23, - 0x6a, 0x8a, 0x99, 0x51, 0x29, 0x7a, 0x0f, 0x24, 0x3c, 0x35, 0xb8, 0x87, 0xe8, 0xcd, 0x3d, 0x24, - 0xf1, 0xd4, 0x60, 0xf6, 0x2f, 0x40, 0x6c, 0xe6, 0x98, 0x4a, 0xac, 0x24, 0x6c, 0xa5, 0xea, 0xc9, - 0xc5, 0xbc, 0x18, 0x3b, 0x54, 0xdb, 0x2a, 0x95, 0xa1, 0xd7, 0x21, 0x3f, 0x60, 0xfb, 0xd5, 0x0c, - 0x4c, 0x86, 0x8e, 0x69, 0xbb, 0x96, 0xa3, 0x88, 0x25, 0x61, 0x2b, 0xa3, 0xca, 0x03, 0x2f, 0x10, - 0x5f, 0x5e, 0x96, 0x21, 0xc7, 0x83, 0x3b, 0x70, 0xac, 0x63, 0x07, 0x13, 0x52, 0xfe, 0x5b, 0x1c, - 0x72, 0x2a, 0x26, 0xae, 0xe5, 0x60, 0x3f, 0xe0, 0x5f, 0x0a, 0x90, 0x73, 0xf5, 0xc1, 0x18, 0x6b, - 0x0e, 0xfe, 0xc4, 0x31, 0x5d, 0x4c, 0x94, 0x68, 0x29, 0xb6, 0x95, 0xde, 0xfe, 0x66, 0xe5, 0xbf, - 0x82, 0xa6, 0xb2, 0xea, 0xa3, 0xd2, 0xa7, 0xf6, 0xaa, 0x67, 0xde, 0x9a, 0xba, 0xce, 0xe3, 0xfa, - 0xdb, 0x3f, 0xfd, 0xe2, 0x86, 0x65, 0x0b, 0x61, 0xa7, 0xd2, 0x6e, 0xaa, 0x59, 0x37, 0xec, 0x0c, - 0xbd, 0x08, 0xe2, 0xcc, 0x31, 0x89, 0x12, 0x2b, 0xc5, 0xb6, 0x52, 0x75, 0x69, 0x31, 0x2f, 0x8a, - 0x87, 0x6a, 0x9b, 0xa8, 0x4c, 0xba, 0x92, 0x69, 0xf1, 0x0e, 0x99, 0xbe, 0x0f, 0x69, 0x1e, 0x3b, - 0xcd, 0x26, 0x51, 0xe2, 0x2c, 0xf0, 0x57, 0x2e, 0x04, 0xee, 0x6f, 0x8e, 0x45, 0xb9, 0x4c, 0xaf, - 0x0a, 0xae, 0x2f, 0x20, 0xa8, 0x0a, 0x69, 0xeb, 0x21, 0x76, 0x1c, 0xd3, 0xc0, 0x9a, 0x31, 0x50, - 0x12, 0xac, 0x74, 0xb9, 0xc5, 0xbc, 0x08, 0x5d, 0x4f, 0xdc, 0xac, 0xab, 0xe0, 0xab, 0x34, 0x07, - 0x85, 0x3f, 0x09, 0x90, 0x09, 0xa7, 0x0d, 0x7d, 0x1f, 0x24, 0xbe, 0x95, 0xa0, 0x07, 0xea, 0x8b, - 0x79, 0x31, 0xc9, 0x74, 0x6e, 0xd1, 0x04, 0x17, 0xb2, 0x99, 0x64, 0x3e, 0xdb, 0x06, 0xfa, 0x21, - 0xa4, 0x6c, 0xdd, 0xc1, 0x53, 0x97, 0xfa, 0x8f, 0x32, 0xff, 0x8d, 0xc5, 0xbc, 0x28, 0x1d, 0x30, - 0xe1, 0xdd, 0x5f, 0x20, 0x71, 0xaf, 0x6d, 0xa3, 0xf0, 0x08, 0xd0, 0xb3, 0x38, 0x40, 0x32, 0xc4, - 0x4e, 0xf0, 0x63, 0x1e, 0x91, 0x4a, 0x7f, 0xa2, 0x3d, 0x88, 0x3f, 0xd4, 0xc7, 0x33, 0xbf, 0x35, - 0xde, 0xba, 0x1b, 0xcc, 0x54, 0xee, 0xe4, 0xdd, 0xe8, 0x37, 0x84, 0x8e, 0x28, 0x09, 0x72, 0xb4, - 0xfc, 0x55, 0x58, 0xf3, 0xf4, 0x7d, 0xb8, 0xa3, 0x97, 0x00, 0x46, 0xe6, 0xf1, 0x48, 0xfb, 0x44, - 0x77, 0xb1, 0xc3, 0xf6, 0x90, 0x51, 0x53, 0x54, 0xf2, 0x11, 0x15, 0x94, 0xbf, 0x88, 0x43, 0xb6, - 0x3d, 0xb1, 0x2d, 0xc7, 0xf5, 0x9b, 0x61, 0x0f, 0x12, 0x2c, 0x61, 0x44, 0x11, 0x18, 0x14, 0x2a, - 0x57, 0x6c, 0x6e, 0xc5, 0x92, 0xef, 0xcd, 0x83, 0x97, 0xe7, 0x23, 0xc0, 0x6e, 0xf4, 0x52, 0xec, - 0x7e, 0x0b, 0x12, 0x9c, 0x45, 0x59, 0xa3, 0xa7, 0xb7, 0x8b, 0xa1, 0x77, 0xf9, 0x54, 0xd6, 0xee, - 0xee, 0x98, 0x63, 0xbc, 0xc3, 0xd4, 0x7c, 0xe7, 0xdc, 0x08, 0xbd, 0x02, 0x12, 0x21, 0xae, 0x46, - 0xcc, 0x4f, 0x39, 0xf4, 0x63, 0xf5, 0x34, 0xc5, 0x4b, 0xaf, 0xd7, 0xef, 0x99, 0x9f, 0x62, 0x35, - 0x49, 0x88, 0x4b, 0x7f, 0xa0, 0x02, 0x48, 0x9f, 0xe8, 0xe3, 0x31, 0x6b, 0x91, 0x38, 0x63, 0xc0, - 0xe0, 0x79, 0x15, 0x14, 0x89, 0xff, 0x01, 0x28, 0x50, 0x11, 0xd2, 0x1e, 0x5f, 0xd9, 0xba, 0x3b, - 0x52, 0x92, 0xb4, 0x2f, 0x54, 0xe0, 0xa2, 0x03, 0xdd, 0x1d, 0x21, 0x05, 0x92, 0x44, 0x9f, 0xd8, - 0x34, 0xe5, 0x52, 0x29, 0xb6, 0x95, 0x51, 0xfd, 0x47, 0xb4, 0x09, 0xac, 0x5f, 0xf8, 0xa3, 0x92, - 0x62, 0x5b, 0x0f, 0x49, 0x58, 0x02, 0x4e, 0x4c, 0x5b, 0x3b, 0x3a, 0x21, 0x0a, 0x94, 0x84, 0x2d, - 0xc9, 0x4b, 0xc0, 0x89, 0x69, 0xef, 0xbc, 0x4f, 0xd4, 0x24, 0x5d, 0xdc, 0x39, 0x21, 0xe8, 0x55, - 0x58, 0x33, 0xa7, 0xc7, 0x98, 0xb8, 0x9a, 0x61, 0x3a, 0x78, 0xe8, 0x8e, 0x1f, 0x2b, 0x69, 0xaa, - 0xae, 0xe6, 0xb8, 0xb8, 0xe9, 0x49, 0x0b, 0x9f, 0x0b, 0x10, 0x67, 0x65, 0x44, 0xef, 0x82, 0x48, - 0x09, 0xc1, 0xa3, 0xff, 0x9b, 0xf2, 0x01, 0xb3, 0x41, 0x08, 0xc4, 0xa9, 0x3e, 0xc1, 0x0a, 0x62, - 0xa1, 0xb2, 0xdf, 0xe8, 0x79, 0x48, 0x12, 0x7c, 0xaa, 0x3d, 0xd4, 0xc7, 0xca, 0x3a, 0x8b, 0x23, - 0x41, 0xf0, 0xe9, 0x87, 0xfa, 0xb8, 0x23, 0x4a, 0x51, 0x39, 0xd6, 0x11, 0xa5, 0x98, 0x2c, 0x76, - 0x44, 0x49, 0x94, 0xe3, 0x1d, 0x51, 0x8a, 0xcb, 0x89, 0x8e, 0x28, 0x25, 0xe4, 0x64, 0x47, 0x94, - 0x92, 0xb2, 0xd4, 0x11, 0x25, 0x49, 0x4e, 0x75, 0x44, 0x29, 0x25, 0x43, 0x47, 0x94, 0x40, 0x4e, - 0x77, 0x44, 0x29, 0x2d, 0x67, 0x3a, 0xa2, 0x94, 0x91, 0xb3, 0x1d, 0x51, 0xca, 0xca, 0xb9, 0x8e, - 0x28, 0xe5, 0xe4, 0xb5, 0x8e, 0x28, 0xad, 0xc9, 0x72, 0x47, 0x94, 0x64, 0x39, 0xdf, 0x11, 0xa5, - 0xbc, 0x8c, 0xca, 0x7f, 0x14, 0x20, 0xc7, 0x71, 0x1a, 0xf4, 0xc4, 0xeb, 0x90, 0x67, 0x09, 0x34, - 0xa7, 0xc7, 0x9a, 0xed, 0x09, 0x19, 0xda, 0xa3, 0xaa, 0xec, 0x2f, 0x04, 0xca, 0x5f, 0x82, 0xac, - 0x83, 0x75, 0x63, 0xa9, 0x18, 0x65, 0x8a, 0x19, 0x2a, 0x0c, 0x94, 0x5e, 0x86, 0x1c, 0x6b, 0xc9, - 0xa5, 0x56, 0x8c, 0x69, 0x65, 0x99, 0x34, 0x50, 0xab, 0x43, 0x96, 0xd8, 0xfa, 0x74, 0xa9, 0x25, - 0xb2, 0x16, 0x7b, 0xfe, 0x12, 0xd8, 0xf7, 0x6c, 0x7d, 0xea, 0xc1, 0x3d, 0x43, 0x6d, 0x82, 0xf3, - 0x4b, 0x65, 0xc7, 0xd7, 0x6c, 0x82, 0xa9, 0xc6, 0x9e, 0x49, 0x5c, 0xf4, 0x6d, 0xc8, 0x38, 0x4c, - 0xa2, 0x51, 0x45, 0xbf, 0x6f, 0xaf, 0x71, 0x9a, 0x76, 0x02, 0x27, 0xa4, 0xfc, 0x7b, 0x01, 0xd6, - 0x9b, 0x8e, 0x65, 0xdb, 0xd8, 0xf0, 0x2a, 0xca, 0xb9, 0xc0, 0x2f, 0xa4, 0x10, 0x2a, 0xe4, 0x7d, - 0x88, 0xb6, 0x9b, 0x1e, 0x7d, 0xbe, 0x7d, 0xd7, 0xee, 0x88, 0xb6, 0x9b, 0xe8, 0x1d, 0x48, 0x10, - 0x57, 0x77, 0x67, 0x84, 0x35, 0x7f, 0x6e, 0xfb, 0xff, 0xaf, 0x20, 0x9a, 0x1e, 0x53, 0x54, 0x3d, - 0x83, 0xf2, 0x9f, 0xa3, 0xb0, 0xde, 0x1b, 0x8e, 0xf0, 0x44, 0x6f, 0x8c, 0xf4, 0xe9, 0x71, 0xb0, - 0xdf, 0xef, 0x82, 0x1c, 0xca, 0x84, 0x36, 0x36, 0x89, 0xeb, 0x9d, 0xe4, 0xaf, 0x5d, 0x4d, 0xb1, - 0xa1, 0x74, 0x7a, 0xf9, 0xc9, 0x39, 0xab, 0x49, 0xfe, 0x1e, 0xe4, 0x0c, 0x9e, 0x21, 0xcd, 0xa3, - 0xc7, 0xd8, 0xb5, 0xf4, 0x78, 0x49, 0x4a, 0x3d, 0xef, 0x59, 0x23, 0xb4, 0x44, 0xd0, 0x8f, 0x61, - 0xdd, 0x77, 0x4e, 0x87, 0x4d, 0x9a, 0x25, 0x4a, 0x47, 0x22, 0x4b, 0xf2, 0xde, 0x62, 0x5e, 0xcc, - 0x7b, 0xae, 0x9a, 0xde, 0xea, 0xdd, 0x79, 0x29, 0x6f, 0x5c, 0xf0, 0x64, 0x78, 0x67, 0xc7, 0x73, - 0xb0, 0x11, 0x4e, 0x69, 0x80, 0xb7, 0x3a, 0xc8, 0x5c, 0x72, 0x84, 0xe9, 0x7e, 0x9d, 0x63, 0xec, - 0xa2, 0x0a, 0xac, 0xd3, 0x4a, 0xe0, 0x09, 0xe5, 0x4d, 0x4a, 0xa3, 0x5a, 0x08, 0x26, 0xf9, 0x60, - 0x89, 0xce, 0x17, 0xfb, 0xfa, 0x04, 0x97, 0xff, 0x20, 0x42, 0x7e, 0xe9, 0xc4, 0xaf, 0x16, 0x65, - 0x2f, 0x73, 0x7a, 0xa2, 0x2d, 0x07, 0x3d, 0xce, 0x5e, 0xe6, 0xf4, 0x84, 0x0e, 0x7b, 0x49, 0xba, - 0x78, 0xe8, 0x98, 0xa8, 0x03, 0xa2, 0x65, 0xbb, 0x7e, 0xb3, 0x5c, 0x75, 0x58, 0x3e, 0xf3, 0x8e, - 0x4a, 0xd7, 0x76, 0xf9, 0x29, 0xac, 0x32, 0x1f, 0xe8, 0xb7, 0x02, 0x24, 0x5d, 0x16, 0x04, 0x51, - 0x12, 0xcc, 0xdf, 0x3b, 0xb7, 0xf2, 0xc7, 0x13, 0xe0, 0x0d, 0x78, 0x07, 0xb4, 0x96, 0x4f, 0xe7, - 0xc5, 0xfc, 0xc5, 0x04, 0x91, 0xbb, 0x4e, 0x7e, 0xfe, 0xde, 0x50, 0x07, 0x72, 0xab, 0x19, 0x66, - 0xe7, 0xc6, 0x0d, 0x67, 0xbb, 0xec, 0x4a, 0x05, 0x0a, 0xc7, 0x74, 0xcc, 0x5a, 0x6e, 0xfb, 0x92, - 0x79, 0xa4, 0xb6, 0x3a, 0x8f, 0xbc, 0x7e, 0xa3, 0x94, 0x70, 0x9f, 0xa1, 0x21, 0xa4, 0xf0, 0x36, - 0xa4, 0x82, 0x7c, 0x87, 0xdf, 0x92, 0xe2, 0x6f, 0xd9, 0x08, 0xbf, 0x25, 0xf5, 0xcc, 0xf4, 0x12, - 0x9c, 0x04, 0x71, 0x39, 0x51, 0xfe, 0x99, 0x00, 0x19, 0x15, 0x13, 0x6b, 0xfc, 0x10, 0x1b, 0xb4, - 0x07, 0xd1, 0xd7, 0x40, 0xa4, 0x3d, 0xed, 0x9d, 0x47, 0xd7, 0x90, 0x1b, 0x53, 0x45, 0x35, 0x48, - 0x05, 0xb7, 0xad, 0xdb, 0x5c, 0x42, 0x96, 0x56, 0x65, 0x1b, 0xd0, 0x32, 0xe0, 0x80, 0xc6, 0xfb, - 0x40, 0xd9, 0x81, 0xed, 0xcd, 0xa3, 0x5c, 0x4e, 0x32, 0xaf, 0x5e, 0x4d, 0x32, 0x41, 0x30, 0x7e, - 0x99, 0x9c, 0x90, 0x8c, 0x78, 0x6d, 0xb8, 0x88, 0x01, 0x6a, 0x38, 0x58, 0x77, 0x31, 0xe5, 0x3c, - 0x72, 0x15, 0x13, 0xd7, 0x21, 0xce, 0xa8, 0xc8, 0x8b, 0xed, 0x86, 0x67, 0xb4, 0xf7, 0x72, 0x6e, - 0x8a, 0x7e, 0x00, 0x99, 0xa1, 0x35, 0x9e, 0x4d, 0x38, 0x59, 0xfa, 0xa4, 0xf6, 0xf5, 0xab, 0x00, - 0xf0, 0xcc, 0xe6, 0x2a, 0x0d, 0x6b, 0x1c, 0x62, 0xce, 0x34, 0x77, 0x48, 0x25, 0x74, 0xfe, 0x4b, - 0x05, 0x60, 0x64, 0x7c, 0x96, 0x52, 0x97, 0x02, 0xb4, 0x0d, 0x71, 0x9d, 0x68, 0xd6, 0x11, 0x9b, - 0xca, 0xae, 0xab, 0x8e, 0x2a, 0xea, 0xa4, 0x7b, 0x84, 0xde, 0x84, 0xec, 0xd1, 0x29, 0xe7, 0x60, - 0xce, 0x3a, 0xfc, 0xa2, 0xb1, 0xb6, 0x98, 0x17, 0xd3, 0x3b, 0x1f, 0xb0, 0x60, 0x29, 0xe7, 0xa8, - 0xe9, 0xa3, 0xd3, 0xe0, 0x01, 0xdd, 0x83, 0xfc, 0x44, 0x7f, 0xa4, 0x1d, 0x39, 0xfa, 0xd0, 0x35, - 0xad, 0xa9, 0x66, 0x1a, 0x63, 0xde, 0x51, 0x82, 0xba, 0x36, 0xd1, 0x1f, 0xed, 0x78, 0xf2, 0xb6, - 0x31, 0xc6, 0x05, 0x1d, 0x92, 0x5e, 0x40, 0xe8, 0x43, 0x88, 0x99, 0x06, 0x3f, 0x50, 0xb3, 0xf5, - 0x26, 0xbd, 0x85, 0xb6, 0x9b, 0xe4, 0xe9, 0xbc, 0xf8, 0xce, 0xad, 0x7b, 0xbb, 0xc1, 0x72, 0xd2, - 0x6e, 0xaa, 0xd4, 0x61, 0xf9, 0xff, 0x60, 0x3d, 0x94, 0xc6, 0x80, 0x6a, 0xff, 0x92, 0x80, 0xe4, - 0x81, 0xfe, 0x78, 0x6c, 0xe9, 0x06, 0x2a, 0x41, 0xda, 0xbf, 0xde, 0x9a, 0xd6, 0xd4, 0xab, 0x7b, - 0x58, 0x44, 0xa7, 0xda, 0x19, 0xc1, 0x0e, 0xcb, 0x01, 0xef, 0xa8, 0xe0, 0x99, 0xce, 0x23, 0xec, - 0x2e, 0x8e, 0x0d, 0x6d, 0x62, 0x0e, 0x1d, 0x8b, 0x9f, 0xb1, 0x31, 0xc6, 0x0c, 0x54, 0xfa, 0x80, - 0x09, 0xe9, 0x5c, 0x78, 0x64, 0x4e, 0x4d, 0x32, 0x5a, 0xea, 0xb1, 0x39, 0x5a, 0xcd, 0xf9, 0x62, - 0x4f, 0xd1, 0x82, 0xdc, 0xf2, 0xb2, 0xad, 0xd1, 0x9c, 0x24, 0x58, 0x4e, 0x76, 0x17, 0xf3, 0x62, - 0x76, 0x89, 0x2d, 0x9e, 0x9d, 0xbb, 0xdd, 0x79, 0x97, 0xfe, 0xdb, 0x06, 0x61, 0x5f, 0x2c, 0x1c, - 0xc7, 0x72, 0x14, 0x89, 0x73, 0x05, 0x7b, 0x40, 0x6f, 0x41, 0x7c, 0x8c, 0x75, 0xc2, 0x47, 0xe1, - 0xf4, 0x76, 0xe9, 0x0a, 0x98, 0xb2, 0x8f, 0x2a, 0x2a, 0x57, 0x47, 0x75, 0x48, 0xf0, 0x79, 0x9b, - 0x4d, 0xc9, 0xe9, 0xed, 0xad, 0x2b, 0x0c, 0x57, 0xbe, 0x85, 0xec, 0x46, 0x54, 0xcf, 0x12, 0xb5, - 0x20, 0xe9, 0xf0, 0xbb, 0x15, 0x9b, 0x9d, 0xaf, 0x1d, 0x29, 0x42, 0xb7, 0xb6, 0xdd, 0x88, 0xea, - 0xdb, 0xa2, 0x3e, 0x64, 0x48, 0xe8, 0x98, 0x55, 0x32, 0xcc, 0xd7, 0x55, 0x53, 0xc4, 0x25, 0x83, - 0xce, 0x2e, 0x1d, 0x0a, 0x43, 0x62, 0x1a, 0xa0, 0xc9, 0x66, 0x5c, 0x25, 0x7b, 0x6d, 0x80, 0x2b, - 0x97, 0x36, 0x1a, 0x20, 0xb7, 0x44, 0xfb, 0x00, 0xc3, 0x80, 0xeb, 0x94, 0x1c, 0xf3, 0xf3, 0x95, - 0xdb, 0x1c, 0x8e, 0xbb, 0x11, 0x35, 0xe4, 0x01, 0x7d, 0x00, 0xe9, 0xe1, 0x12, 0xe4, 0xca, 0x1a, - 0x73, 0xf8, 0xc6, 0xad, 0x98, 0x65, 0x97, 0xb2, 0xc9, 0x52, 0xba, 0xca, 0x26, 0xf2, 0x05, 0x36, - 0xa9, 0xa7, 0x20, 0x69, 0x70, 0xbb, 0xe0, 0x0a, 0x91, 0x94, 0xa5, 0xf2, 0x67, 0x71, 0x90, 0x02, - 0xea, 0xae, 0x02, 0x0a, 0x48, 0x60, 0x68, 0xd1, 0x4b, 0x94, 0x8b, 0xf9, 0xc7, 0x86, 0xe8, 0x6e, - 0x44, 0xcd, 0xfb, 0x6b, 0x0d, 0x7f, 0x89, 0xb6, 0xc8, 0xc4, 0x32, 0xcc, 0x23, 0x73, 0xd9, 0x22, - 0xfc, 0x23, 0x5a, 0xce, 0x17, 0x7b, 0x2d, 0xf2, 0xde, 0xca, 0x45, 0x3b, 0x76, 0x03, 0x42, 0xdb, - 0x8d, 0x84, 0x6e, 0xe2, 0xb4, 0x65, 0x9d, 0xd9, 0x74, 0x4a, 0xef, 0x24, 0xde, 0x58, 0xcc, 0xe9, - 0x32, 0xeb, 0x49, 0xf9, 0x08, 0x8c, 0x1a, 0x17, 0xa0, 0xfc, 0xda, 0xb5, 0x50, 0xf6, 0x63, 0xdf, - 0x15, 0x02, 0x2c, 0xef, 0x5c, 0xc4, 0xf2, 0xbd, 0xeb, 0xb1, 0x1c, 0x72, 0x13, 0x80, 0xf9, 0xf0, - 0x52, 0x30, 0x57, 0x6f, 0x08, 0xe6, 0x90, 0xc7, 0x55, 0x34, 0x37, 0x2e, 0xa0, 0xf9, 0xb5, 0x6b, - 0xd1, 0x1c, 0x8e, 0xd1, 0x83, 0x73, 0xf7, 0x12, 0x38, 0xbf, 0x71, 0x23, 0x38, 0x87, 0x9c, 0x85, - 0xf1, 0xac, 0x5e, 0x86, 0xe7, 0xca, 0xcd, 0xf0, 0x1c, 0x72, 0x19, 0x76, 0x52, 0x07, 0x90, 0xfc, - 0xbb, 0x60, 0x08, 0xbe, 0xf7, 0xbe, 0x03, 0x09, 0xaf, 0xdc, 0x08, 0x72, 0x4d, 0xb5, 0xd6, 0xde, - 0x6f, 0xef, 0xdf, 0xd7, 0xf6, 0x6b, 0x0f, 0x5a, 0x3d, 0x39, 0x82, 0x14, 0xd8, 0xf8, 0xa8, 0xd6, - 0xee, 0x6b, 0x3b, 0x5d, 0x55, 0xbb, 0xdf, 0xd0, 0xda, 0xfb, 0xfd, 0x96, 0xfa, 0x61, 0x6d, 0x4f, - 0x16, 0xd0, 0x73, 0x80, 0xd4, 0x6e, 0xe3, 0xfd, 0x5e, 0xb3, 0xae, 0x35, 0xba, 0x0f, 0x0e, 0x6a, - 0x8d, 0x7e, 0xbb, 0xbb, 0x2f, 0x47, 0x91, 0x04, 0x62, 0xb3, 0xbb, 0xdf, 0x92, 0xe1, 0xde, 0x67, - 0x51, 0x10, 0xfb, 0x8f, 0x6d, 0x8c, 0xbe, 0x0c, 0xe9, 0xc3, 0xfd, 0xde, 0x41, 0xab, 0xd1, 0xde, - 0x69, 0xb7, 0x9a, 0x72, 0xa4, 0xb0, 0x7e, 0x76, 0x5e, 0x5a, 0xa3, 0x4b, 0x87, 0x53, 0x62, 0xe3, - 0x21, 0x83, 0x36, 0x2a, 0x40, 0xa2, 0x5e, 0x6b, 0xbc, 0x7f, 0x78, 0x20, 0x0b, 0x85, 0xdc, 0xd9, - 0x79, 0x09, 0xa8, 0x02, 0x87, 0x15, 0x7a, 0x11, 0x92, 0x6a, 0xab, 0xd7, 0xef, 0xaa, 0x2d, 0x39, - 0x5a, 0x58, 0x3b, 0x3b, 0x2f, 0xa5, 0xe9, 0xa2, 0x87, 0x16, 0xf4, 0x2a, 0x64, 0x7b, 0x8d, 0xdd, - 0xd6, 0x83, 0x9a, 0xd6, 0xd8, 0xad, 0xed, 0xdf, 0x6f, 0xc9, 0xb1, 0xc2, 0xc6, 0xd9, 0x79, 0x49, - 0xa6, 0x3a, 0x61, 0x10, 0xd0, 0x57, 0xb4, 0x1f, 0x1c, 0x74, 0xd5, 0xbe, 0x2c, 0x2e, 0x5f, 0xc1, - 0xab, 0x8a, 0xca, 0x00, 0xdc, 0x7a, 0xa7, 0xd5, 0x6a, 0xca, 0xf1, 0x02, 0x3a, 0x3b, 0x2f, 0xe5, - 0xe8, 0xfa, 0xb2, 0x58, 0xe8, 0x65, 0xc8, 0x34, 0xd4, 0x56, 0xad, 0xdf, 0xd2, 0x7a, 0xfd, 0x5a, - 0xbf, 0x27, 0x27, 0x96, 0x91, 0x84, 0x0a, 0x80, 0x2a, 0x90, 0xaf, 0x1d, 0xf6, 0xbb, 0xda, 0x8a, - 0x6e, 0xb2, 0xf0, 0xfc, 0xd9, 0x79, 0x69, 0x9d, 0xea, 0xd6, 0x66, 0xae, 0x15, 0xd2, 0x2f, 0x48, - 0x3f, 0xff, 0xf5, 0x66, 0xe4, 0x77, 0xbf, 0xd9, 0x8c, 0xd4, 0x4b, 0x4f, 0xfe, 0xb9, 0x19, 0x79, - 0xb2, 0xd8, 0x14, 0x3e, 0x5f, 0x6c, 0x0a, 0x7f, 0x5d, 0x6c, 0x0a, 0xff, 0x58, 0x6c, 0x0a, 0xbf, - 0xf8, 0xd7, 0x66, 0xe4, 0xe3, 0x04, 0x2f, 0xf4, 0x20, 0xc1, 0x3e, 0xe5, 0xbf, 0xf9, 0x9f, 0x00, - 0x00, 0x00, 0xff, 0xff, 0x50, 0x15, 0xb6, 0x29, 0x7b, 0x18, 0x00, 0x00, +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_6106a12eb5231ca0) } + +var fileDescriptor_jobs_6106a12eb5231ca0 = []byte{ + // 2445 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x5f, 0x8f, 0xe3, 0x56, + 0x15, 0x8f, 0x13, 0x27, 0x71, 0x4e, 0xfe, 0x8c, 0x73, 0x67, 0xda, 0xba, 0x51, 0x3b, 0x09, 0x81, + 0xb6, 0xd3, 0x2d, 0x4d, 0x60, 0x2a, 0x5a, 0x5a, 0x41, 0x45, 0xfe, 0xcd, 0x4e, 0xd2, 0xdd, 0x64, + 0xea, 0x64, 0x5a, 0x28, 0x02, 0xe3, 0xc4, 0x77, 0x26, 0x66, 0x92, 0xd8, 0xe3, 0xeb, 0xec, 0x76, + 0x2b, 0x24, 0x24, 0xc4, 0x03, 0xda, 0x27, 0x3e, 0x00, 0x23, 0x21, 0x01, 0x12, 0x2f, 0x88, 0x7e, + 0x8c, 0xbe, 0x20, 0x40, 0xbc, 0x80, 0x90, 0x02, 0x84, 0x07, 0xf8, 0x04, 0x3c, 0xec, 0x13, 0xba, + 0xf7, 0xda, 0x8e, 0x33, 0x3b, 0xcc, 0x3f, 0x89, 0x97, 0x99, 0xf8, 0xdc, 0x73, 0x8e, 0xef, 0x39, + 0xe7, 0x77, 0x7e, 0xf7, 0x5c, 0xc3, 0xb3, 0xdf, 0xb7, 0x86, 0xa4, 0x4a, 0xff, 0xd8, 0x43, 0xf6, + 0xaf, 0x62, 0x3b, 0x96, 0x6b, 0xa1, 0xe7, 0x47, 0xd6, 0xe8, 0xc4, 0xb1, 0xf4, 0xd1, 0xb8, 0x42, + 0x4e, 0x27, 0x15, 0xb6, 0xc2, 0xb5, 0x0a, 0x5b, 0xc7, 0xd6, 0xb1, 0xc5, 0xb4, 0xaa, 0xf4, 0x17, + 0x37, 0x28, 0x20, 0xa6, 0x6c, 0x0f, 0xab, 0x86, 0xee, 0xea, 0x9e, 0x4c, 0xf1, 0x65, 0xa6, 0xf5, + 0xfa, 0x91, 0xe5, 0x4c, 0x75, 0xd7, 0x73, 0x5f, 0x78, 0x81, 0x9c, 0x4e, 0xaa, 0xe4, 0x74, 0x32, + 0xd4, 0x09, 0xae, 0x12, 0xd7, 0x99, 0x8f, 0xdc, 0xb9, 0x83, 0x0d, 0xdf, 0x6e, 0xee, 0x9a, 0x93, + 0xea, 0x78, 0x32, 0xaa, 0xba, 0xe6, 0x14, 0x13, 0x57, 0x9f, 0xda, 0x7c, 0xa5, 0xfc, 0x43, 0x88, + 0xdf, 0xc3, 0x3a, 0xc1, 0xe8, 0x23, 0x48, 0xce, 0x2c, 0x03, 0x6b, 0xa6, 0xa1, 0x08, 0x25, 0x61, + 0x27, 0x5b, 0xaf, 0x2d, 0x17, 0xc5, 0x44, 0xd7, 0x32, 0x70, 0xbb, 0xf9, 0x64, 0x51, 0x7c, 0xe3, + 0xd8, 0x74, 0xc7, 0xf3, 0x61, 0x65, 0x64, 0x4d, 0xab, 0x41, 0x24, 0xc6, 0x70, 0xf5, 0xbb, 0x6a, + 0x9f, 0x1c, 0x57, 0xbd, 0xed, 0x55, 0xb8, 0x99, 0x9a, 0xa0, 0x1e, 0xdb, 0x06, 0xda, 0x82, 0x38, + 0xb6, 0xad, 0xd1, 0x58, 0x89, 0x96, 0x84, 0x9d, 0x98, 0xca, 0x1f, 0xde, 0x11, 0xff, 0xfd, 0xf3, + 0xa2, 0x50, 0xfe, 0xab, 0x00, 0xd9, 0xba, 0x3e, 0x3a, 0x99, 0xdb, 0x4d, 0xec, 0xea, 0xe6, 0x84, + 0xa0, 0x3a, 0x00, 0x71, 0x75, 0xc7, 0xd5, 0xe8, 0x5e, 0xd9, 0x66, 0xd2, 0xbb, 0x2f, 0x56, 0x56, + 0xe9, 0xa3, 0xb1, 0x54, 0xc6, 0x93, 0x51, 0x65, 0xe0, 0xc7, 0x52, 0x17, 0x3f, 0x5b, 0x14, 0x23, + 0x6a, 0x8a, 0x99, 0x51, 0x29, 0x7a, 0x17, 0x24, 0x3c, 0x33, 0xb8, 0x87, 0xe8, 0xf5, 0x3d, 0x24, + 0xf1, 0xcc, 0x60, 0xf6, 0xcf, 0x43, 0x6c, 0xee, 0x98, 0x4a, 0xac, 0x24, 0xec, 0xa4, 0xea, 0xc9, + 0xe5, 0xa2, 0x18, 0x3b, 0x54, 0xdb, 0x2a, 0x95, 0xa1, 0xd7, 0x20, 0x3f, 0x64, 0xfb, 0xd5, 0x0c, + 0x4c, 0x46, 0x8e, 0x69, 0xbb, 0x96, 0xa3, 0x88, 0x25, 0x61, 0x27, 0xa3, 0xca, 0x43, 0x2f, 0x10, + 0x5f, 0x5e, 0x96, 0x21, 0xc7, 0x83, 0x3b, 0x70, 0xac, 0x63, 0x07, 0x13, 0x52, 0xfe, 0x4b, 0x1c, + 0x72, 0x2a, 0x26, 0xae, 0xe5, 0x60, 0x3f, 0xe0, 0x9f, 0x09, 0x90, 0x73, 0xf5, 0xe1, 0x04, 0x6b, + 0x0e, 0x7e, 0xe8, 0x98, 0x2e, 0x26, 0x4a, 0xb4, 0x14, 0xdb, 0x49, 0xef, 0x7e, 0xad, 0xf2, 0x3f, + 0x41, 0x53, 0x59, 0xf7, 0x51, 0x19, 0x50, 0x7b, 0xd5, 0x33, 0x6f, 0xcd, 0x5c, 0xe7, 0x51, 0xfd, + 0xad, 0x1f, 0xfd, 0xed, 0x9a, 0x65, 0x0b, 0x61, 0xa7, 0xd2, 0x6e, 0xaa, 0x59, 0x37, 0xec, 0x0c, + 0xbd, 0x00, 0xe2, 0xdc, 0x31, 0x89, 0x12, 0x2b, 0xc5, 0x76, 0x52, 0x75, 0x69, 0xb9, 0x28, 0x8a, + 0x87, 0x6a, 0x9b, 0xa8, 0x4c, 0xba, 0x96, 0x69, 0xf1, 0x16, 0x99, 0xbe, 0x0b, 0x69, 0x1e, 0x3b, + 0xcd, 0x26, 0x51, 0xe2, 0x2c, 0xf0, 0x97, 0xcf, 0x05, 0xee, 0x6f, 0x8e, 0x45, 0xb9, 0x4a, 0xaf, + 0x0a, 0xae, 0x2f, 0x20, 0xa8, 0x0a, 0x69, 0xeb, 0x01, 0x76, 0x1c, 0xd3, 0xc0, 0x9a, 0x31, 0x54, + 0x12, 0xac, 0x74, 0xb9, 0xe5, 0xa2, 0x08, 0x3d, 0x4f, 0xdc, 0xac, 0xab, 0xe0, 0xab, 0x34, 0x87, + 0x85, 0xdf, 0x0b, 0x90, 0x09, 0xa7, 0x0d, 0x7d, 0x07, 0x24, 0xbe, 0x95, 0xa0, 0x07, 0xea, 0xcb, + 0x45, 0x31, 0xc9, 0x74, 0x6e, 0xd0, 0x04, 0xe7, 0xb2, 0x99, 0x64, 0x3e, 0xdb, 0x06, 0xfa, 0x1e, + 0xa4, 0x6c, 0xdd, 0xc1, 0x33, 0x97, 0xfa, 0x8f, 0x32, 0xff, 0x8d, 0xe5, 0xa2, 0x28, 0x1d, 0x30, + 0xe1, 0xed, 0x5f, 0x20, 0x71, 0xaf, 0x6d, 0xa3, 0xf0, 0x31, 0xa0, 0xa7, 0x71, 0x80, 0x64, 0x88, + 0x9d, 0xe0, 0x47, 0x3c, 0x22, 0x95, 0xfe, 0x44, 0xf7, 0x20, 0xfe, 0x40, 0x9f, 0xcc, 0xfd, 0xd6, + 0x78, 0xf3, 0x76, 0x30, 0x53, 0xb9, 0x93, 0x77, 0xa2, 0x5f, 0x15, 0x3a, 0xa2, 0x24, 0xc8, 0xd1, + 0xf2, 0x97, 0x60, 0xc3, 0xd3, 0xf7, 0xe1, 0x8e, 0x5e, 0x04, 0x18, 0x9b, 0xc7, 0x63, 0xed, 0xa1, + 0xee, 0x62, 0x87, 0xed, 0x21, 0xa3, 0xa6, 0xa8, 0xe4, 0x43, 0x2a, 0x28, 0xff, 0x27, 0x0e, 0xd9, + 0xf6, 0xd4, 0xb6, 0x1c, 0xd7, 0x6f, 0x86, 0x7b, 0x90, 0x60, 0x09, 0x23, 0x8a, 0xc0, 0xa0, 0x50, + 0xb9, 0x64, 0x73, 0x6b, 0x96, 0x7c, 0x6f, 0x1e, 0xbc, 0x3c, 0x1f, 0x01, 0x76, 0xa3, 0x17, 0x62, + 0xf7, 0xeb, 0x90, 0xe0, 0x2c, 0xca, 0x1a, 0x3d, 0xbd, 0x5b, 0x0c, 0xbd, 0xcb, 0xa7, 0xb2, 0x76, + 0x6f, 0xcf, 0x9c, 0xe0, 0x3d, 0xa6, 0xe6, 0x3b, 0xe7, 0x46, 0xe8, 0x65, 0x90, 0x08, 0x71, 0x35, + 0x62, 0x7e, 0xc2, 0xa1, 0x1f, 0xab, 0xa7, 0x29, 0x5e, 0xfa, 0xfd, 0x41, 0xdf, 0xfc, 0x04, 0xab, + 0x49, 0x42, 0x5c, 0xfa, 0x03, 0x15, 0x40, 0x7a, 0xa8, 0x4f, 0x26, 0xac, 0x45, 0xe2, 0x8c, 0x01, + 0x83, 0xe7, 0x75, 0x50, 0x24, 0xfe, 0x0f, 0xa0, 0x40, 0x45, 0x48, 0x7b, 0x7c, 0x65, 0xeb, 0xee, + 0x58, 0x49, 0xd2, 0xbe, 0x50, 0x81, 0x8b, 0x0e, 0x74, 0x77, 0x8c, 0x14, 0x48, 0x12, 0x7d, 0x6a, + 0xd3, 0x94, 0x4b, 0xa5, 0xd8, 0x4e, 0x46, 0xf5, 0x1f, 0xd1, 0x36, 0xb0, 0x7e, 0xe1, 0x8f, 0x4a, + 0x8a, 0x6d, 0x3d, 0x24, 0x61, 0x09, 0x38, 0x31, 0x6d, 0xed, 0xe8, 0x84, 0x28, 0x50, 0x12, 0x76, + 0x24, 0x2f, 0x01, 0x27, 0xa6, 0xbd, 0xf7, 0x1e, 0x51, 0x93, 0x74, 0x71, 0xef, 0x84, 0xa0, 0x57, + 0x60, 0xc3, 0x9c, 0x1d, 0x63, 0xe2, 0x6a, 0x86, 0xe9, 0xe0, 0x91, 0x3b, 0x79, 0xa4, 0xa4, 0xa9, + 0xba, 0x9a, 0xe3, 0xe2, 0xa6, 0x27, 0x2d, 0xfc, 0x4b, 0x80, 0x38, 0x2b, 0x23, 0x7a, 0x07, 0x44, + 0x4a, 0x08, 0x1e, 0xfd, 0x5f, 0x97, 0x0f, 0x98, 0x0d, 0x42, 0x20, 0xce, 0xf4, 0x29, 0x56, 0x10, + 0x0b, 0x95, 0xfd, 0x46, 0xcf, 0x41, 0x92, 0xe0, 0x53, 0xed, 0x81, 0x3e, 0x51, 0x36, 0x59, 0x1c, + 0x09, 0x82, 0x4f, 0x3f, 0xd0, 0x27, 0xe8, 0x19, 0x48, 0x98, 0x44, 0x9b, 0xe1, 0x87, 0xca, 0x16, + 0xdb, 0x52, 0xdc, 0x24, 0x5d, 0xfc, 0xb0, 0x23, 0x4a, 0x51, 0x39, 0xd6, 0x11, 0xa5, 0x98, 0x2c, + 0x76, 0x44, 0x49, 0x94, 0xe3, 0x1d, 0x51, 0x8a, 0xcb, 0x89, 0x8e, 0x28, 0x25, 0xe4, 0x64, 0x47, + 0x94, 0x92, 0xb2, 0xd4, 0x11, 0x25, 0x49, 0x4e, 0x75, 0x44, 0x29, 0x25, 0x43, 0x47, 0x94, 0x40, + 0x4e, 0x77, 0x44, 0x29, 0x2d, 0x67, 0x3a, 0xa2, 0x94, 0x91, 0xb3, 0x1d, 0x51, 0xca, 0xca, 0xb9, + 0x8e, 0x28, 0xe5, 0xe4, 0x8d, 0x8e, 0x28, 0x6d, 0xc8, 0x72, 0x47, 0x94, 0x64, 0x39, 0xdf, 0x11, + 0xa5, 0xbc, 0x8c, 0xca, 0xbf, 0x13, 0x20, 0xc7, 0xe1, 0x1b, 0xb4, 0xca, 0x6b, 0x90, 0x67, 0x79, + 0x35, 0x67, 0xc7, 0x9a, 0xed, 0x09, 0x59, 0x13, 0x44, 0x55, 0xd9, 0x5f, 0x08, 0x94, 0x3f, 0x0f, + 0x59, 0x07, 0xeb, 0xc6, 0x4a, 0x31, 0xca, 0x14, 0x33, 0x54, 0x18, 0x28, 0xbd, 0x04, 0x39, 0xd6, + 0xa9, 0x2b, 0xad, 0x18, 0xd3, 0xca, 0x32, 0x69, 0xa0, 0x56, 0x87, 0x2c, 0xb1, 0xf5, 0xd9, 0x4a, + 0x4b, 0x64, 0x9d, 0xf7, 0xdc, 0x05, 0xdd, 0xd0, 0xb7, 0xf5, 0x99, 0xd7, 0x05, 0x19, 0x6a, 0x13, + 0x1c, 0x6b, 0x2a, 0x3b, 0xd5, 0xe6, 0x53, 0x4c, 0x35, 0xee, 0x99, 0xc4, 0x45, 0xdf, 0x80, 0x8c, + 0xc3, 0x24, 0x1a, 0x55, 0xf4, 0xdb, 0xf9, 0x0a, 0xa7, 0x69, 0x27, 0x70, 0x42, 0xca, 0xbf, 0x11, + 0x60, 0xb3, 0xe9, 0x58, 0xb6, 0x8d, 0x0d, 0xaf, 0xd0, 0x9c, 0x22, 0xfc, 0xfa, 0x0a, 0xa1, 0xfa, + 0xde, 0x85, 0x68, 0xbb, 0xe9, 0xb1, 0xea, 0x5b, 0xb7, 0x6d, 0x9a, 0x68, 0xbb, 0x89, 0xde, 0x86, + 0x04, 0x71, 0x75, 0x77, 0x4e, 0x18, 0x27, 0xe4, 0x76, 0x3f, 0x77, 0x09, 0xff, 0xf4, 0x99, 0xa2, + 0xea, 0x19, 0x94, 0xff, 0x18, 0x85, 0xcd, 0xfe, 0x68, 0x8c, 0xa7, 0x7a, 0x63, 0xac, 0xcf, 0x8e, + 0x83, 0xfd, 0x7e, 0x0b, 0xe4, 0x50, 0x26, 0xb4, 0x89, 0x49, 0x5c, 0xef, 0x80, 0x7f, 0xf5, 0x72, + 0xe6, 0x0d, 0xa5, 0xd3, 0xcb, 0x4f, 0xce, 0x59, 0x4f, 0xf2, 0xb7, 0x21, 0x67, 0xf0, 0x0c, 0x69, + 0x1e, 0x6b, 0xc6, 0xae, 0x64, 0xcd, 0x0b, 0x52, 0xea, 0x79, 0xcf, 0x1a, 0xa1, 0x25, 0x82, 0x7e, + 0x00, 0x9b, 0xbe, 0x73, 0x3a, 0x83, 0xd2, 0x2c, 0x51, 0x96, 0x12, 0x59, 0x92, 0xef, 0x2d, 0x17, + 0xc5, 0xbc, 0xe7, 0xaa, 0xe9, 0xad, 0xde, 0x9e, 0xae, 0xf2, 0xc6, 0x39, 0x4f, 0x86, 0x77, 0xa4, + 0x3c, 0x0b, 0x5b, 0xe1, 0x94, 0x06, 0x78, 0xab, 0x83, 0xcc, 0x25, 0x47, 0x98, 0xee, 0xd7, 0x39, + 0xc6, 0x2e, 0xaa, 0xc0, 0x26, 0xad, 0x04, 0x9e, 0x52, 0x3a, 0xa5, 0xec, 0xaa, 0x85, 0x60, 0x92, + 0x0f, 0x96, 0xe8, 0xd8, 0xd1, 0xd5, 0xa7, 0xb8, 0xfc, 0x5b, 0x11, 0xf2, 0x2b, 0x27, 0x7e, 0xb5, + 0x28, 0xa9, 0x99, 0xb3, 0x13, 0x6d, 0x35, 0xff, 0x71, 0x52, 0x33, 0x67, 0x27, 0x74, 0x06, 0x4c, + 0xd2, 0xc5, 0x43, 0xc7, 0x44, 0x1d, 0x10, 0x2d, 0xdb, 0xf5, 0x9b, 0xe5, 0xb2, 0x33, 0xf4, 0xa9, + 0x77, 0x54, 0x7a, 0xb6, 0xcb, 0x0f, 0x67, 0x95, 0xf9, 0x40, 0xbf, 0x12, 0x20, 0xe9, 0xb2, 0x20, + 0x88, 0x92, 0x60, 0xfe, 0xde, 0xbe, 0x91, 0x3f, 0x9e, 0x00, 0x6f, 0xee, 0x3b, 0xa0, 0xb5, 0x7c, + 0xb2, 0x28, 0xe6, 0xcf, 0x27, 0x88, 0xdc, 0x76, 0x20, 0xf4, 0xf7, 0x86, 0x3a, 0x90, 0x5b, 0xcf, + 0x30, 0x3b, 0x4e, 0xae, 0x39, 0xf2, 0x65, 0xd7, 0x2a, 0x50, 0x38, 0xa6, 0xd3, 0xd7, 0x6a, 0xdb, + 0x17, 0x8c, 0x29, 0xb5, 0xf5, 0x31, 0xe5, 0xb5, 0x6b, 0xa5, 0x84, 0xfb, 0x0c, 0xcd, 0x26, 0x85, + 0xb7, 0x20, 0x15, 0xe4, 0x3b, 0xfc, 0x96, 0x14, 0x7f, 0xcb, 0x56, 0xf8, 0x2d, 0xa9, 0xa7, 0x86, + 0x9a, 0xe0, 0x24, 0x88, 0xcb, 0x89, 0xf2, 0x8f, 0x05, 0xc8, 0xa8, 0x98, 0x58, 0x93, 0x07, 0xd8, + 0xa0, 0x3d, 0x88, 0xbe, 0x0c, 0x22, 0xed, 0x69, 0xef, 0x98, 0xba, 0x82, 0xdc, 0x98, 0x2a, 0xaa, + 0x41, 0x2a, 0xb8, 0x84, 0xdd, 0xe4, 0x6e, 0xb2, 0xb2, 0x2a, 0xdb, 0x80, 0x56, 0x01, 0x07, 0x34, + 0x3e, 0x00, 0xca, 0x0e, 0x6c, 0x6f, 0x1e, 0xe5, 0x72, 0x92, 0x79, 0xe5, 0x72, 0x92, 0x09, 0x82, + 0xf1, 0xcb, 0xe4, 0x84, 0x64, 0xc4, 0x6b, 0xc3, 0x65, 0x0c, 0x50, 0xc3, 0xc1, 0xba, 0x8b, 0x29, + 0xe7, 0x91, 0xcb, 0x98, 0xb8, 0x0e, 0x71, 0x46, 0x45, 0x5e, 0x6c, 0xd7, 0x3c, 0xba, 0xbd, 0x97, + 0x73, 0x53, 0xf4, 0x5d, 0xc8, 0x8c, 0xac, 0xc9, 0x7c, 0xca, 0xc9, 0xd2, 0x27, 0xb5, 0xaf, 0x5c, + 0x06, 0x80, 0xa7, 0x36, 0x57, 0x69, 0x58, 0x93, 0x10, 0x73, 0xa6, 0xb9, 0x43, 0x2a, 0xa1, 0x63, + 0x61, 0x2a, 0x00, 0x23, 0xe3, 0xb3, 0x94, 0xba, 0x12, 0xa0, 0x5d, 0x88, 0xeb, 0x44, 0xb3, 0x8e, + 0xd8, 0xb0, 0x76, 0x55, 0x75, 0x54, 0x51, 0x27, 0xbd, 0x23, 0xf4, 0x06, 0x64, 0x8f, 0x4e, 0x39, + 0x07, 0x73, 0xd6, 0xe1, 0xf7, 0x8f, 0x8d, 0xe5, 0xa2, 0x98, 0xde, 0x7b, 0x9f, 0x05, 0x4b, 0x39, + 0x47, 0x4d, 0x1f, 0x9d, 0x06, 0x0f, 0xe8, 0x0e, 0xe4, 0xa7, 0xfa, 0xc7, 0xda, 0x91, 0xa3, 0x8f, + 0x5c, 0xd3, 0x9a, 0x69, 0xa6, 0x31, 0xe1, 0x1d, 0x25, 0xa8, 0x1b, 0x53, 0xfd, 0xe3, 0x3d, 0x4f, + 0xde, 0x36, 0x26, 0xb8, 0xa0, 0x43, 0xd2, 0x0b, 0x08, 0x7d, 0x00, 0x31, 0xd3, 0xe0, 0x07, 0x6a, + 0xb6, 0xde, 0xa4, 0x97, 0xd3, 0x76, 0x93, 0x3c, 0x59, 0x14, 0xdf, 0xbe, 0x71, 0x6f, 0x37, 0x58, + 0x4e, 0xda, 0x4d, 0x95, 0x3a, 0x2c, 0x3f, 0x03, 0x9b, 0xa1, 0x34, 0x06, 0x54, 0xfb, 0xa7, 0x04, + 0x24, 0x0f, 0xf4, 0x47, 0x13, 0x4b, 0x37, 0x50, 0x09, 0xd2, 0xfe, 0xad, 0xd7, 0xb4, 0x66, 0x5e, + 0xdd, 0xc3, 0x22, 0x3a, 0xec, 0xce, 0x09, 0x76, 0x58, 0x0e, 0x78, 0x47, 0x05, 0xcf, 0x74, 0x1e, + 0x61, 0x57, 0x74, 0x6c, 0x68, 0x53, 0x73, 0xe4, 0x58, 0xfc, 0x8c, 0x8d, 0x31, 0x66, 0xa0, 0xd2, + 0xfb, 0x4c, 0x48, 0xc7, 0xc5, 0x23, 0x73, 0x66, 0x92, 0xf1, 0x4a, 0x8f, 0x8d, 0xd7, 0x6a, 0xce, + 0x17, 0x7b, 0x8a, 0x16, 0xe4, 0x56, 0x77, 0x70, 0x8d, 0xe6, 0x24, 0xc1, 0x72, 0xb2, 0xbf, 0x5c, + 0x14, 0xb3, 0x2b, 0x6c, 0xf1, 0xec, 0xdc, 0xee, 0x2a, 0xbc, 0xf2, 0xdf, 0x36, 0x08, 0xfb, 0x90, + 0xe1, 0x38, 0x96, 0xa3, 0x48, 0x9c, 0x2b, 0xd8, 0x03, 0x7a, 0x13, 0xe2, 0x13, 0xac, 0x13, 0x3e, + 0x21, 0xa7, 0x77, 0x4b, 0x97, 0xc0, 0x94, 0x7d, 0x6b, 0x51, 0xb9, 0x3a, 0xaa, 0x43, 0x82, 0x8f, + 0xe1, 0x6c, 0x78, 0x4e, 0xef, 0xee, 0x5c, 0x62, 0xb8, 0xf6, 0x89, 0x64, 0x3f, 0xa2, 0x7a, 0x96, + 0xa8, 0x05, 0x49, 0x87, 0x5f, 0xb9, 0xd8, 0x48, 0x7d, 0xe5, 0x48, 0x11, 0xba, 0xcc, 0xed, 0x47, + 0x54, 0xdf, 0x16, 0x0d, 0x20, 0x43, 0x42, 0xc7, 0xac, 0x92, 0x61, 0xbe, 0x2e, 0x9b, 0x22, 0x2e, + 0x18, 0x74, 0xf6, 0xe9, 0x50, 0x18, 0x12, 0xd3, 0x00, 0x4d, 0x36, 0xe3, 0x2a, 0xd9, 0x2b, 0x03, + 0x5c, 0xbb, 0xcb, 0xd1, 0x00, 0xb9, 0x25, 0xea, 0x02, 0x8c, 0x02, 0xae, 0x53, 0x72, 0xcc, 0xcf, + 0x17, 0x6f, 0x72, 0x38, 0xee, 0x47, 0xd4, 0x90, 0x07, 0xf4, 0x3e, 0xa4, 0x47, 0x2b, 0x90, 0x2b, + 0x1b, 0xcc, 0xe1, 0xeb, 0x37, 0x62, 0x96, 0x7d, 0xca, 0x26, 0x2b, 0xe9, 0x3a, 0x9b, 0xc8, 0xe7, + 0xd8, 0xa4, 0x9e, 0x82, 0xa4, 0xc1, 0xed, 0x82, 0x2b, 0x44, 0x52, 0x96, 0xca, 0x9f, 0xc6, 0x41, + 0x0a, 0xa8, 0xbb, 0x0a, 0x28, 0x20, 0x81, 0x91, 0x45, 0xef, 0x56, 0x2e, 0xe6, 0xdf, 0x20, 0xa2, + 0xfb, 0x11, 0x35, 0xef, 0xaf, 0x35, 0xfc, 0x25, 0xda, 0x22, 0x53, 0xcb, 0x30, 0x8f, 0xcc, 0x55, + 0x8b, 0xf0, 0x6f, 0x6b, 0x39, 0x5f, 0xec, 0xb5, 0xc8, 0xbb, 0x6b, 0xf7, 0xef, 0xd8, 0x35, 0x08, + 0x6d, 0x3f, 0x12, 0xba, 0xa0, 0xd3, 0x96, 0x75, 0xe6, 0xb3, 0x19, 0xbd, 0x93, 0x78, 0x63, 0x31, + 0xa7, 0xcb, 0xac, 0x27, 0xe5, 0x23, 0x30, 0x6a, 0x9c, 0x83, 0xf2, 0xab, 0x57, 0x42, 0xd9, 0x8f, + 0x7d, 0x5f, 0x08, 0xb0, 0xbc, 0x77, 0x1e, 0xcb, 0x77, 0xae, 0xc6, 0x72, 0xc8, 0x4d, 0x00, 0xe6, + 0xc3, 0x0b, 0xc1, 0x5c, 0xbd, 0x26, 0x98, 0x43, 0x1e, 0xd7, 0xd1, 0xdc, 0x38, 0x87, 0xe6, 0x57, + 0xaf, 0x44, 0x73, 0x38, 0x46, 0x0f, 0xce, 0xbd, 0x0b, 0xe0, 0xfc, 0xfa, 0xb5, 0xe0, 0x1c, 0x72, + 0x16, 0xc6, 0xb3, 0x7a, 0x11, 0x9e, 0x2b, 0xd7, 0xc3, 0x73, 0xc8, 0x65, 0xd8, 0x49, 0x1d, 0x40, + 0xf2, 0xef, 0x82, 0x21, 0xf8, 0xde, 0xf9, 0x26, 0x24, 0xbc, 0x72, 0x23, 0xc8, 0x35, 0xd5, 0x5a, + 0xbb, 0xdb, 0xee, 0xde, 0xd5, 0xba, 0xb5, 0xfb, 0xad, 0xbe, 0x1c, 0x41, 0x0a, 0x6c, 0x7d, 0x58, + 0x6b, 0x0f, 0xb4, 0xbd, 0x9e, 0xaa, 0xdd, 0x6d, 0x68, 0xed, 0xee, 0xa0, 0xa5, 0x7e, 0x50, 0xbb, + 0x27, 0x0b, 0xe8, 0x59, 0x40, 0x6a, 0xaf, 0xf1, 0x5e, 0xbf, 0x59, 0xd7, 0x1a, 0xbd, 0xfb, 0x07, + 0xb5, 0xc6, 0xa0, 0xdd, 0xeb, 0xca, 0x51, 0x24, 0x81, 0xd8, 0xec, 0x75, 0x5b, 0x32, 0xdc, 0xf9, + 0x34, 0x0a, 0xe2, 0xe0, 0x91, 0x8d, 0xd1, 0x17, 0x20, 0x7d, 0xd8, 0xed, 0x1f, 0xb4, 0x1a, 0xed, + 0xbd, 0x76, 0xab, 0x29, 0x47, 0x0a, 0x9b, 0x8f, 0xcf, 0x4a, 0x1b, 0x74, 0xe9, 0x70, 0x46, 0x6c, + 0x3c, 0x62, 0xd0, 0x46, 0x05, 0x48, 0xd4, 0x6b, 0x8d, 0xf7, 0x0e, 0x0f, 0x64, 0xa1, 0x90, 0x7b, + 0x7c, 0x56, 0x02, 0xaa, 0xc0, 0x61, 0x85, 0x5e, 0x80, 0xa4, 0xda, 0xea, 0x0f, 0x7a, 0x6a, 0x4b, + 0x8e, 0x16, 0x36, 0x1e, 0x9f, 0x95, 0xd2, 0x74, 0xd1, 0x43, 0x0b, 0x7a, 0x05, 0xb2, 0xfd, 0xc6, + 0x7e, 0xeb, 0x7e, 0x4d, 0x6b, 0xec, 0xd7, 0xba, 0x77, 0x5b, 0x72, 0xac, 0xb0, 0xf5, 0xf8, 0xac, + 0x24, 0x53, 0x9d, 0x30, 0x08, 0xe8, 0x2b, 0xda, 0xf7, 0x0f, 0x7a, 0xea, 0x40, 0x16, 0x57, 0xaf, + 0xe0, 0x55, 0x45, 0x65, 0x00, 0x6e, 0xbd, 0xd7, 0x6a, 0x35, 0xe5, 0x78, 0x01, 0x3d, 0x3e, 0x2b, + 0xe5, 0xe8, 0xfa, 0xaa, 0x58, 0xe8, 0x25, 0xc8, 0x34, 0xd4, 0x56, 0x6d, 0xd0, 0xd2, 0xfa, 0x83, + 0xda, 0xa0, 0x2f, 0x27, 0x56, 0x91, 0x84, 0x0a, 0x80, 0x2a, 0x90, 0xaf, 0x1d, 0x0e, 0x7a, 0xda, + 0x9a, 0x6e, 0xb2, 0xf0, 0xdc, 0xe3, 0xb3, 0xd2, 0x26, 0xd5, 0xad, 0xcd, 0x5d, 0x2b, 0xa4, 0x5f, + 0x90, 0x7e, 0xf2, 0x8b, 0xed, 0xc8, 0xaf, 0x7f, 0xb9, 0x1d, 0xa9, 0x97, 0x3e, 0xfb, 0xc7, 0x76, + 0xe4, 0xb3, 0xe5, 0xb6, 0xf0, 0x87, 0xe5, 0xb6, 0xf0, 0xe7, 0xe5, 0xb6, 0xf0, 0xf7, 0xe5, 0xb6, + 0xf0, 0xd3, 0x7f, 0x6e, 0x47, 0x3e, 0x4a, 0xf0, 0x42, 0x0f, 0x13, 0xec, 0x0b, 0xff, 0x1b, 0xff, + 0x0d, 0x00, 0x00, 0xff, 0xff, 0x00, 0xe1, 0xca, 0x95, 0x92, 0x18, 0x00, 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 6695c4df4242..24b890ecf7ea 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -75,6 +75,7 @@ message ImportDetails { sqlbase.TableDescriptor desc = 1; string name = 18; int64 seq_val = 19; + bool is_new = 20; reserved 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17; } repeated Table tables = 1 [(gogoproto.nullable) = false]; diff --git a/pkg/sql/distsql_plan_csv.go b/pkg/sql/distsql_plan_csv.go index 086c5e96ddbe..28f1abbb7708 100644 --- a/pkg/sql/distsql_plan_csv.go +++ b/pkg/sql/distsql_plan_csv.go @@ -179,7 +179,6 @@ func LoadCSV( resultRows *RowResultWriter, tables map[string]*sqlbase.TableDescriptor, from []string, - to string, format roachpb.IOFileFormat, walltime int64, splitSize int64, @@ -200,7 +199,6 @@ func LoadCSV( sstSpecs := make([]distsqlpb.SSTWriterSpec, len(nodes)) for i := range nodes { sstSpecs[i] = distsqlpb.SSTWriterSpec{ - Destination: to, WalltimeNanos: walltime, } } diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index 17bf6b5efecb..a5d9dde421db 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -67,7 +67,7 @@ func (x *ConstraintValidity) UnmarshalJSON(data []byte) error { return nil } func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{0} + return fileDescriptor_structured_81f3763b694c15f3, []int{0} } type ForeignKeyReference_Action int32 @@ -112,7 +112,7 @@ func (x *ForeignKeyReference_Action) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Action) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{0, 0} + return fileDescriptor_structured_81f3763b694c15f3, []int{0, 0} } // Match is the algorithm used to compare composite keys. @@ -152,7 +152,7 @@ func (x *ForeignKeyReference_Match) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Match) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{0, 1} + return fileDescriptor_structured_81f3763b694c15f3, []int{0, 1} } // The direction of a column in the index. @@ -189,7 +189,7 @@ func (x *IndexDescriptor_Direction) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{5, 0} + return fileDescriptor_structured_81f3763b694c15f3, []int{5, 0} } // The type of the index. @@ -226,7 +226,7 @@ func (x *IndexDescriptor_Type) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{5, 1} + return fileDescriptor_structured_81f3763b694c15f3, []int{5, 1} } type ConstraintToUpdate_ConstraintType int32 @@ -259,7 +259,7 @@ func (x *ConstraintToUpdate_ConstraintType) UnmarshalJSON(data []byte) error { return nil } func (ConstraintToUpdate_ConstraintType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{6, 0} + return fileDescriptor_structured_81f3763b694c15f3, []int{6, 0} } // A descriptor within a mutation is unavailable for reads, writes @@ -324,7 +324,7 @@ func (x *DescriptorMutation_State) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{7, 0} + return fileDescriptor_structured_81f3763b694c15f3, []int{7, 0} } // Direction of mutation. @@ -367,7 +367,7 @@ func (x *DescriptorMutation_Direction) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{7, 1} + return fileDescriptor_structured_81f3763b694c15f3, []int{7, 1} } // State is set if this TableDescriptor is in the process of being added or deleted. @@ -384,17 +384,21 @@ const ( TableDescriptor_ADD TableDescriptor_State = 1 // Descriptor is being dropped. TableDescriptor_DROP TableDescriptor_State = 2 + // Descriptor is being ingested. + TableDescriptor_IMPORTING TableDescriptor_State = 3 ) var TableDescriptor_State_name = map[int32]string{ 0: "PUBLIC", 1: "ADD", 2: "DROP", + 3: "IMPORTING", } var TableDescriptor_State_value = map[string]int32{ - "PUBLIC": 0, - "ADD": 1, - "DROP": 2, + "PUBLIC": 0, + "ADD": 1, + "DROP": 2, + "IMPORTING": 3, } func (x TableDescriptor_State) Enum() *TableDescriptor_State { @@ -414,7 +418,7 @@ func (x *TableDescriptor_State) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 0} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 0} } // AuditMode indicates which auditing actions to take when this table is used. @@ -451,7 +455,7 @@ func (x *TableDescriptor_AuditMode) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_AuditMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 1} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 1} } type ForeignKeyReference struct { @@ -475,7 +479,7 @@ func (m *ForeignKeyReference) Reset() { *m = ForeignKeyReference{} } func (m *ForeignKeyReference) String() string { return proto.CompactTextString(m) } func (*ForeignKeyReference) ProtoMessage() {} func (*ForeignKeyReference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{0} + return fileDescriptor_structured_81f3763b694c15f3, []int{0} } func (m *ForeignKeyReference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -522,7 +526,7 @@ func (m *ColumnDescriptor) Reset() { *m = ColumnDescriptor{} } func (m *ColumnDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnDescriptor) ProtoMessage() {} func (*ColumnDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{1} + return fileDescriptor_structured_81f3763b694c15f3, []int{1} } func (m *ColumnDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -575,7 +579,7 @@ func (m *ColumnFamilyDescriptor) Reset() { *m = ColumnFamilyDescriptor{} func (m *ColumnFamilyDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnFamilyDescriptor) ProtoMessage() {} func (*ColumnFamilyDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{2} + return fileDescriptor_structured_81f3763b694c15f3, []int{2} } func (m *ColumnFamilyDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -623,7 +627,7 @@ func (m *InterleaveDescriptor) Reset() { *m = InterleaveDescriptor{} } func (m *InterleaveDescriptor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor) ProtoMessage() {} func (*InterleaveDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{3} + return fileDescriptor_structured_81f3763b694c15f3, []int{3} } func (m *InterleaveDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -669,7 +673,7 @@ func (m *InterleaveDescriptor_Ancestor) Reset() { *m = InterleaveDescrip func (m *InterleaveDescriptor_Ancestor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor_Ancestor) ProtoMessage() {} func (*InterleaveDescriptor_Ancestor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{3, 0} + return fileDescriptor_structured_81f3763b694c15f3, []int{3, 0} } func (m *InterleaveDescriptor_Ancestor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -716,7 +720,7 @@ func (m *PartitioningDescriptor) Reset() { *m = PartitioningDescriptor{} func (m *PartitioningDescriptor) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor) ProtoMessage() {} func (*PartitioningDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{4} + return fileDescriptor_structured_81f3763b694c15f3, []int{4} } func (m *PartitioningDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -761,7 +765,7 @@ func (m *PartitioningDescriptor_List) Reset() { *m = PartitioningDescrip func (m *PartitioningDescriptor_List) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_List) ProtoMessage() {} func (*PartitioningDescriptor_List) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{4, 0} + return fileDescriptor_structured_81f3763b694c15f3, []int{4, 0} } func (m *PartitioningDescriptor_List) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -808,7 +812,7 @@ func (m *PartitioningDescriptor_Range) Reset() { *m = PartitioningDescri func (m *PartitioningDescriptor_Range) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_Range) ProtoMessage() {} func (*PartitioningDescriptor_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{4, 1} + return fileDescriptor_structured_81f3763b694c15f3, []int{4, 1} } func (m *PartitioningDescriptor_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -942,7 +946,7 @@ func (m *IndexDescriptor) Reset() { *m = IndexDescriptor{} } func (m *IndexDescriptor) String() string { return proto.CompactTextString(m) } func (*IndexDescriptor) ProtoMessage() {} func (*IndexDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{5} + return fileDescriptor_structured_81f3763b694c15f3, []int{5} } func (m *IndexDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -992,7 +996,7 @@ func (m *ConstraintToUpdate) Reset() { *m = ConstraintToUpdate{} } func (m *ConstraintToUpdate) String() string { return proto.CompactTextString(m) } func (*ConstraintToUpdate) ProtoMessage() {} func (*ConstraintToUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{6} + return fileDescriptor_structured_81f3763b694c15f3, []int{6} } func (m *ConstraintToUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1048,7 +1052,7 @@ func (m *DescriptorMutation) Reset() { *m = DescriptorMutation{} } func (m *DescriptorMutation) String() string { return proto.CompactTextString(m) } func (*DescriptorMutation) ProtoMessage() {} func (*DescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{7} + return fileDescriptor_structured_81f3763b694c15f3, []int{7} } func (m *DescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1332,7 +1336,7 @@ func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } func (m *TableDescriptor) String() string { return proto.CompactTextString(m) } func (*TableDescriptor) ProtoMessage() {} func (*TableDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8} + return fileDescriptor_structured_81f3763b694c15f3, []int{8} } func (m *TableDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1584,7 +1588,7 @@ func (m *TableDescriptor_SchemaChangeLease) Reset() { *m = TableDescript func (m *TableDescriptor_SchemaChangeLease) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SchemaChangeLease) ProtoMessage() {} func (*TableDescriptor_SchemaChangeLease) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 0} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 0} } func (m *TableDescriptor_SchemaChangeLease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1623,7 +1627,7 @@ func (m *TableDescriptor_CheckConstraint) Reset() { *m = TableDescriptor func (m *TableDescriptor_CheckConstraint) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_CheckConstraint) ProtoMessage() {} func (*TableDescriptor_CheckConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 1} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 1} } func (m *TableDescriptor_CheckConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1728,7 +1732,7 @@ func (m *TableDescriptor_NameInfo) Reset() { *m = TableDescriptor_NameIn func (m *TableDescriptor_NameInfo) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_NameInfo) ProtoMessage() {} func (*TableDescriptor_NameInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 2} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 2} } func (m *TableDescriptor_NameInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1770,7 +1774,7 @@ func (m *TableDescriptor_Reference) Reset() { *m = TableDescriptor_Refer func (m *TableDescriptor_Reference) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Reference) ProtoMessage() {} func (*TableDescriptor_Reference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 3} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 3} } func (m *TableDescriptor_Reference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1809,7 +1813,7 @@ func (m *TableDescriptor_MutationJob) Reset() { *m = TableDescriptor_Mut func (m *TableDescriptor_MutationJob) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_MutationJob) ProtoMessage() {} func (*TableDescriptor_MutationJob) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 4} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 4} } func (m *TableDescriptor_MutationJob) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1853,7 +1857,7 @@ func (m *TableDescriptor_SequenceOpts) Reset() { *m = TableDescriptor_Se func (m *TableDescriptor_SequenceOpts) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SequenceOpts) ProtoMessage() {} func (*TableDescriptor_SequenceOpts) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 5} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 5} } func (m *TableDescriptor_SequenceOpts) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1889,7 +1893,7 @@ func (m *TableDescriptor_Replacement) Reset() { *m = TableDescriptor_Rep func (m *TableDescriptor_Replacement) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Replacement) ProtoMessage() {} func (*TableDescriptor_Replacement) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 6} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 6} } func (m *TableDescriptor_Replacement) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1928,7 +1932,7 @@ func (m *TableDescriptor_GCDescriptorMutation) Reset() { *m = TableDescr func (m *TableDescriptor_GCDescriptorMutation) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_GCDescriptorMutation) ProtoMessage() {} func (*TableDescriptor_GCDescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{8, 7} + return fileDescriptor_structured_81f3763b694c15f3, []int{8, 7} } func (m *TableDescriptor_GCDescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1969,7 +1973,7 @@ func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } func (m *DatabaseDescriptor) String() string { return proto.CompactTextString(m) } func (*DatabaseDescriptor) ProtoMessage() {} func (*DatabaseDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{9} + return fileDescriptor_structured_81f3763b694c15f3, []int{9} } func (m *DatabaseDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2029,7 +2033,7 @@ func (m *Descriptor) Reset() { *m = Descriptor{} } func (m *Descriptor) String() string { return proto.CompactTextString(m) } func (*Descriptor) ProtoMessage() {} func (*Descriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_ec887d346a40d82d, []int{10} + return fileDescriptor_structured_81f3763b694c15f3, []int{10} } func (m *Descriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8576,186 +8580,187 @@ var ( ) func init() { - proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_ec887d346a40d82d) -} - -var fileDescriptor_structured_ec887d346a40d82d = []byte{ - // 2825 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x59, 0x4b, 0x6f, 0xe3, 0xc8, - 0x11, 0x36, 0xf5, 0xa4, 0x4a, 0x2f, 0xba, 0xe7, 0xb1, 0x1c, 0xcd, 0xac, 0xad, 0xd1, 0xee, 0x6c, - 0x9c, 0x7d, 0xc8, 0xb3, 0x9e, 0x3c, 0x06, 0x49, 0xb0, 0x88, 0x5e, 0x1e, 0xcb, 0x23, 0x4b, 0x1e, - 0x5a, 0x1e, 0xef, 0x06, 0x9b, 0x08, 0x34, 0xd9, 0xb6, 0xb9, 0x43, 0x91, 0x1a, 0x92, 0x72, 0xec, - 0x7f, 0xb0, 0xa7, 0x20, 0xa7, 0xe4, 0xb6, 0x08, 0x16, 0x39, 0x04, 0xc8, 0x35, 0x87, 0xfc, 0x84, - 0x3d, 0x06, 0x39, 0x05, 0x39, 0x18, 0x89, 0x83, 0x5c, 0x93, 0xfb, 0x20, 0x01, 0x82, 0x6e, 0x76, - 0x53, 0xa4, 0x2d, 0x39, 0xf2, 0xcc, 0x4d, 0xac, 0xee, 0xfa, 0xba, 0xbb, 0xba, 0xea, 0xab, 0xaa, - 0x16, 0xdc, 0x73, 0x5f, 0x9a, 0xab, 0xee, 0x4b, 0x73, 0x5f, 0x75, 0xf1, 0xaa, 0xeb, 0x39, 0x63, - 0xcd, 0x1b, 0x3b, 0x58, 0xaf, 0x8e, 0x1c, 0xdb, 0xb3, 0xd1, 0x2d, 0xcd, 0xd6, 0x5e, 0x38, 0xb6, - 0xaa, 0x1d, 0x55, 0xdd, 0x97, 0x66, 0x95, 0xcd, 0x2b, 0xc9, 0x63, 0xcf, 0x30, 0x57, 0x8f, 0x4c, - 0x6d, 0xd5, 0x33, 0x86, 0xd8, 0xf5, 0xd4, 0xe1, 0xc8, 0x57, 0x28, 0xdd, 0x0d, 0xc3, 0x8d, 0x1c, - 0xe3, 0xd8, 0x30, 0xf1, 0x21, 0x66, 0x83, 0x37, 0x0f, 0xed, 0x43, 0x9b, 0xfe, 0x5c, 0x25, 0xbf, - 0x7c, 0x69, 0xe5, 0x3f, 0x09, 0xb8, 0xb1, 0x6e, 0x3b, 0xd8, 0x38, 0xb4, 0x9e, 0xe2, 0x53, 0x05, - 0x1f, 0x60, 0x07, 0x5b, 0x1a, 0x46, 0x65, 0x48, 0x7a, 0xea, 0xbe, 0x89, 0x65, 0xa1, 0x2c, 0xac, - 0xe4, 0xeb, 0xf0, 0xcd, 0xd9, 0xf2, 0xc2, 0xab, 0xb3, 0xe5, 0x58, 0xbb, 0xa9, 0xf8, 0x03, 0xe8, - 0x01, 0x24, 0x0d, 0x4b, 0xc7, 0x27, 0x72, 0x8c, 0xce, 0x28, 0xb2, 0x19, 0xe9, 0x36, 0x11, 0x92, - 0x69, 0x74, 0x14, 0xc9, 0x90, 0xb0, 0xd4, 0x21, 0x96, 0xe3, 0x65, 0x61, 0x25, 0x53, 0x4f, 0x90, - 0x59, 0x0a, 0x95, 0xa0, 0xa7, 0x20, 0x1e, 0xab, 0xa6, 0xa1, 0x1b, 0xde, 0xa9, 0x9c, 0x28, 0x0b, - 0x2b, 0x85, 0xb5, 0x6f, 0x57, 0xa7, 0x9e, 0xb8, 0xda, 0xb0, 0x2d, 0xd7, 0x73, 0x54, 0xc3, 0xf2, - 0x9e, 0x33, 0x05, 0x06, 0x14, 0x00, 0xa0, 0x87, 0xb0, 0xe8, 0x1e, 0xa9, 0x0e, 0xd6, 0x07, 0x23, - 0x07, 0x1f, 0x18, 0x27, 0x03, 0x13, 0x5b, 0x72, 0xb2, 0x2c, 0xac, 0x24, 0xd9, 0xd4, 0xa2, 0x3f, - 0xbc, 0x4d, 0x47, 0x3b, 0xd8, 0x42, 0x7d, 0xc8, 0xd8, 0xd6, 0x40, 0xc7, 0x26, 0xf6, 0xb0, 0x9c, - 0xa2, 0xeb, 0x7f, 0x3c, 0x63, 0xfd, 0x29, 0x06, 0xaa, 0xd6, 0x34, 0xcf, 0xb0, 0x2d, 0xbe, 0x0f, - 0xdb, 0x6a, 0x52, 0x20, 0x86, 0x3a, 0x1e, 0xe9, 0xaa, 0x87, 0xe5, 0xf4, 0x1b, 0xa3, 0xee, 0x52, - 0x20, 0xd4, 0x81, 0xe4, 0x50, 0xf5, 0xb4, 0x23, 0x59, 0xa4, 0x88, 0x0f, 0xaf, 0x81, 0xb8, 0x45, - 0xf4, 0x18, 0xa0, 0x0f, 0x52, 0xd9, 0x83, 0x94, 0xbf, 0x0e, 0xca, 0x43, 0xa6, 0xdb, 0x1b, 0xd4, - 0x1a, 0xfd, 0x76, 0xaf, 0x2b, 0x2d, 0xa0, 0x1c, 0x88, 0x4a, 0x6b, 0xa7, 0xaf, 0xb4, 0x1b, 0x7d, - 0x49, 0x20, 0x5f, 0x3b, 0xad, 0xfe, 0xa0, 0xbb, 0xdb, 0xe9, 0x48, 0x31, 0x54, 0x84, 0x2c, 0xf9, - 0x6a, 0xb6, 0xd6, 0x6b, 0xbb, 0x9d, 0xbe, 0x14, 0x47, 0x59, 0x48, 0x37, 0x6a, 0x3b, 0x8d, 0x5a, - 0xb3, 0x25, 0x25, 0x4a, 0x89, 0xdf, 0xfd, 0x76, 0x69, 0xa1, 0xf2, 0x10, 0x92, 0x74, 0x39, 0x04, - 0x90, 0xda, 0x69, 0x6f, 0x6d, 0x77, 0x5a, 0xd2, 0x02, 0x12, 0x21, 0xb1, 0x4e, 0x20, 0x04, 0xa2, - 0xb1, 0x5d, 0x53, 0xfa, 0xed, 0x5a, 0x47, 0x8a, 0x31, 0x8d, 0x7f, 0xc5, 0x40, 0x6a, 0xd8, 0xe6, - 0x78, 0x68, 0x35, 0xb1, 0xab, 0x39, 0xc6, 0xc8, 0xb3, 0x9d, 0xc0, 0x65, 0x84, 0x4b, 0x2e, 0xf3, - 0x1e, 0xc4, 0x0c, 0x9d, 0x39, 0xdc, 0x6d, 0x22, 0x3f, 0xa7, 0x2e, 0xf9, 0xea, 0x6c, 0x59, 0xf4, - 0x51, 0xda, 0x4d, 0x25, 0x66, 0xe8, 0xa8, 0x03, 0x09, 0xef, 0x74, 0xe4, 0x3b, 0x5d, 0xae, 0xfe, - 0x98, 0xcc, 0xfc, 0xeb, 0xd9, 0xf2, 0xc3, 0x43, 0xc3, 0x3b, 0x1a, 0xef, 0x57, 0x35, 0x7b, 0xb8, - 0x1a, 0x18, 0x50, 0xdf, 0x9f, 0xfc, 0x5e, 0x1d, 0xbd, 0x38, 0x24, 0xf1, 0xb3, 0x4a, 0x94, 0xdd, - 0x6a, 0x5f, 0xa1, 0x28, 0xa8, 0x0c, 0xa2, 0x35, 0x36, 0x4d, 0x1a, 0x0e, 0xc4, 0x51, 0x45, 0x7e, - 0x3f, 0x5c, 0x8a, 0xee, 0x43, 0x4e, 0xc7, 0x07, 0xea, 0xd8, 0xf4, 0x06, 0xf8, 0x64, 0xe4, 0x50, - 0xc7, 0xcb, 0x28, 0x59, 0x26, 0x6b, 0x9d, 0x8c, 0x1c, 0x74, 0x0f, 0x52, 0x47, 0x86, 0xae, 0x63, - 0x8b, 0xfa, 0x1a, 0x87, 0x60, 0x32, 0xb4, 0x06, 0x8b, 0x63, 0x17, 0xbb, 0x03, 0x17, 0xbf, 0x1c, - 0x93, 0x6b, 0x1b, 0x18, 0xba, 0x2b, 0x43, 0x39, 0xbe, 0x92, 0xaf, 0xa7, 0x58, 0xd8, 0x15, 0xc9, - 0x84, 0x1d, 0x36, 0xde, 0xd6, 0x5d, 0xb2, 0xa8, 0x66, 0x0f, 0x47, 0x63, 0x0f, 0xfb, 0x8b, 0x66, - 0xfd, 0x45, 0x99, 0x8c, 0x2c, 0xba, 0x99, 0x10, 0x45, 0x29, 0xb3, 0x99, 0x10, 0x33, 0x12, 0x6c, - 0x26, 0xc4, 0xb4, 0x24, 0x56, 0xbe, 0x8c, 0xc1, 0x6d, 0xdf, 0x54, 0xeb, 0xea, 0xd0, 0x30, 0x4f, - 0xdf, 0xd4, 0xec, 0x3e, 0x0a, 0x33, 0x3b, 0xdd, 0x11, 0xc1, 0x1e, 0x10, 0x35, 0x57, 0x8e, 0x97, - 0xe3, 0xfe, 0x8e, 0x88, 0xac, 0x4b, 0x44, 0xe8, 0x31, 0x00, 0x9b, 0x42, 0x4e, 0x98, 0xa0, 0x27, - 0xbc, 0x73, 0x7e, 0xb6, 0x9c, 0xe1, 0xf7, 0xe7, 0x46, 0x2e, 0x33, 0xe3, 0x4f, 0x26, 0xc7, 0xed, - 0xc1, 0x22, 0xb7, 0x71, 0x80, 0x40, 0x0d, 0x9d, 0xaf, 0xbf, 0xc3, 0xf6, 0x54, 0x6c, 0xfa, 0x13, - 0xb8, 0x7a, 0x04, 0xaa, 0xa8, 0x47, 0x06, 0xf5, 0xca, 0xef, 0x63, 0x70, 0xb3, 0x6d, 0x79, 0xd8, - 0x31, 0xb1, 0x7a, 0x8c, 0x43, 0x86, 0xf8, 0x14, 0x32, 0xaa, 0xa5, 0x61, 0xd7, 0xb3, 0x1d, 0x57, - 0x16, 0xca, 0xf1, 0x95, 0xec, 0xda, 0x77, 0x66, 0x44, 0xdc, 0x34, 0xfd, 0x6a, 0x8d, 0x29, 0x33, - 0x1b, 0x4e, 0xc0, 0x4a, 0x7f, 0x14, 0x40, 0xe4, 0xa3, 0xe8, 0x21, 0x88, 0x94, 0x49, 0xc9, 0x39, - 0x7c, 0x96, 0xbd, 0xc5, 0xce, 0x91, 0xee, 0x13, 0x39, 0xdd, 0x3f, 0xb9, 0xf9, 0x34, 0x9d, 0xd6, - 0xd6, 0xd1, 0x77, 0x41, 0xa4, 0xa4, 0x3a, 0x08, 0x6e, 0xa3, 0xc4, 0x35, 0x18, 0xeb, 0x86, 0x09, - 0x38, 0x4d, 0xe7, 0xb6, 0x75, 0xd4, 0x98, 0xc6, 0x8d, 0x71, 0xaa, 0xff, 0x16, 0xb7, 0xdc, 0x4e, - 0x94, 0x1d, 0x2f, 0xd1, 0x65, 0xe5, 0x9f, 0x71, 0xb8, 0xbd, 0xad, 0x3a, 0x9e, 0x41, 0x88, 0xc3, - 0xb0, 0x0e, 0x43, 0xf6, 0x7a, 0x00, 0x59, 0x6b, 0x3c, 0x64, 0xb7, 0xe2, 0xb2, 0xb3, 0xf8, 0x67, - 0x07, 0x6b, 0x3c, 0xf4, 0x0d, 0xee, 0x92, 0xa0, 0x34, 0x0d, 0xd7, 0x93, 0x63, 0xd4, 0xa2, 0x6b, - 0x33, 0x2c, 0x3a, 0x7d, 0x8d, 0x6a, 0xc7, 0x70, 0x3d, 0xee, 0x93, 0x04, 0x05, 0xf5, 0x20, 0xe9, - 0xa8, 0xd6, 0x21, 0xa6, 0x4e, 0x96, 0x5d, 0x7b, 0x74, 0x3d, 0x38, 0x85, 0xa8, 0x72, 0x56, 0xa4, - 0x38, 0xa5, 0x5f, 0x0b, 0x90, 0x20, 0xab, 0x5c, 0x11, 0x07, 0xb7, 0x21, 0x75, 0xac, 0x9a, 0x63, - 0xec, 0xd2, 0x33, 0xe4, 0x14, 0xf6, 0x85, 0x7e, 0x0a, 0x45, 0x77, 0xbc, 0x3f, 0x0a, 0x2d, 0x45, - 0xcd, 0x9b, 0x5d, 0xfb, 0xe8, 0x5a, 0xbb, 0x0a, 0x32, 0x55, 0x14, 0xab, 0xf4, 0x02, 0x92, 0x74, - 0xbf, 0x57, 0xec, 0xec, 0x3e, 0xe4, 0x3c, 0x7b, 0x80, 0x4f, 0x34, 0x73, 0xec, 0x1a, 0xc7, 0x98, - 0x7a, 0x47, 0x4e, 0xc9, 0x7a, 0x76, 0x8b, 0x8b, 0xd0, 0x03, 0x28, 0x1c, 0x38, 0xf6, 0x70, 0x60, - 0x58, 0x7c, 0x12, 0x65, 0x47, 0x25, 0x4f, 0xa4, 0x6d, 0x2e, 0xac, 0xfc, 0x57, 0x84, 0x22, 0xf5, - 0xa0, 0xb9, 0x98, 0xe1, 0x41, 0x88, 0x19, 0x6e, 0x45, 0x98, 0x21, 0x70, 0x43, 0x42, 0x0c, 0xf7, - 0x20, 0x35, 0xb6, 0x8c, 0x97, 0x63, 0x7f, 0xcd, 0x80, 0xfc, 0x7c, 0xd9, 0x25, 0xda, 0x48, 0x5c, - 0xa6, 0x8d, 0x0f, 0x01, 0x91, 0x98, 0xc1, 0x83, 0xc8, 0xc4, 0x24, 0x9d, 0x28, 0xd1, 0x91, 0xc6, - 0x4c, 0x92, 0x49, 0x5d, 0x83, 0x64, 0x36, 0x40, 0xc2, 0x27, 0x9e, 0xa3, 0x0e, 0x42, 0xfa, 0x69, - 0xaa, 0xbf, 0x74, 0x7e, 0xb6, 0x5c, 0x68, 0x91, 0xb1, 0xe9, 0x20, 0x05, 0x1c, 0x1a, 0xd3, 0x89, - 0x4f, 0x2c, 0x32, 0x0c, 0xdd, 0x70, 0x30, 0x4d, 0xb7, 0xae, 0x2c, 0x96, 0xe3, 0x57, 0xa4, 0xef, - 0x0b, 0x66, 0xaf, 0x36, 0xb9, 0xa2, 0x22, 0xf9, 0x50, 0x81, 0xc0, 0x45, 0xcf, 0x20, 0x7b, 0xe0, - 0x67, 0xfb, 0xc1, 0x0b, 0x7c, 0x2a, 0x67, 0xa8, 0xbb, 0xbd, 0x3f, 0x7f, 0x5d, 0xc0, 0xe3, 0xf3, - 0x20, 0x18, 0x42, 0xbb, 0x90, 0x77, 0xf8, 0xb0, 0x3e, 0xd8, 0x3f, 0xa5, 0xf9, 0xe7, 0x75, 0x40, - 0x73, 0x13, 0x98, 0xfa, 0x29, 0x7a, 0x06, 0x60, 0x04, 0x2c, 0x49, 0x93, 0x54, 0x76, 0xed, 0x83, - 0x6b, 0xd0, 0x29, 0xdf, 0xe9, 0x04, 0x04, 0xed, 0x41, 0x61, 0xf2, 0x45, 0xb7, 0x9a, 0x7b, 0xcd, - 0xad, 0xe6, 0x43, 0x38, 0xf5, 0x53, 0xd4, 0x87, 0x9b, 0x24, 0x7d, 0xda, 0xae, 0xe1, 0xe1, 0xb0, - 0x0b, 0xe4, 0xa9, 0x0b, 0x54, 0xce, 0xcf, 0x96, 0x51, 0x83, 0x8f, 0x4f, 0x77, 0x03, 0xa4, 0x5d, - 0x18, 0xf7, 0x9d, 0x2a, 0xe2, 0xbc, 0x04, 0xb1, 0x30, 0x71, 0xaa, 0x9d, 0x89, 0xfb, 0x5e, 0x72, - 0xaa, 0x90, 0x6b, 0x13, 0xa4, 0x3d, 0xc8, 0x45, 0x58, 0xa6, 0xf8, 0xfa, 0x2c, 0x13, 0x01, 0x42, - 0x2d, 0x56, 0x30, 0x49, 0xb4, 0xbe, 0xfc, 0x60, 0x4e, 0x07, 0xed, 0x9f, 0x8e, 0xb8, 0x21, 0xa9, - 0x7a, 0x65, 0x09, 0x32, 0x81, 0x8f, 0xa2, 0x34, 0xc4, 0x6b, 0x3b, 0x0d, 0xbf, 0x02, 0x6c, 0xb6, - 0x76, 0x1a, 0x92, 0x50, 0xb9, 0x0f, 0x09, 0xa2, 0x43, 0x2a, 0xc1, 0xf5, 0x9e, 0xb2, 0x57, 0x53, - 0x9a, 0x7e, 0xd5, 0xd9, 0xee, 0x3e, 0x6f, 0x29, 0xfd, 0x56, 0x53, 0x12, 0x48, 0x81, 0x82, 0x26, - 0xf5, 0x7e, 0xdf, 0x66, 0x15, 0xf0, 0x21, 0x14, 0xb5, 0x40, 0x3a, 0xa0, 0x7b, 0x15, 0xca, 0xb1, - 0x95, 0xc2, 0xda, 0xe3, 0xff, 0xdb, 0x33, 0x70, 0x8c, 0xb0, 0x68, 0xb2, 0xf1, 0x82, 0x16, 0x91, - 0x06, 0x5c, 0x17, 0x2b, 0xc7, 0x2e, 0x70, 0x9d, 0x02, 0x49, 0xed, 0x08, 0x6b, 0x2f, 0x18, 0xb7, - 0x7f, 0x6f, 0xc6, 0xc2, 0x34, 0x77, 0x87, 0x8c, 0xd4, 0x20, 0x3a, 0x93, 0xa5, 0x79, 0xd2, 0xa1, - 0x50, 0x95, 0xbb, 0x50, 0x88, 0xee, 0x0a, 0x65, 0x20, 0xd9, 0xd8, 0x68, 0x35, 0x9e, 0x4a, 0x0b, - 0x95, 0x7f, 0x27, 0x00, 0x4d, 0x80, 0xb6, 0xc6, 0x9e, 0x4a, 0xed, 0x5a, 0x83, 0x94, 0xef, 0x48, - 0x94, 0x8f, 0xb3, 0x6b, 0xdf, 0x9a, 0x69, 0x81, 0x68, 0x5d, 0xbd, 0xb1, 0xa0, 0x30, 0x45, 0xf4, - 0x49, 0xb8, 0x77, 0xcb, 0xae, 0xbd, 0x37, 0xdf, 0x7d, 0x6f, 0x2c, 0xf0, 0xa6, 0xee, 0x29, 0x24, - 0x5d, 0x8f, 0x74, 0x38, 0x71, 0xea, 0x2f, 0xab, 0x33, 0xf4, 0x2f, 0x6f, 0xbe, 0xba, 0x43, 0xd4, - 0xb8, 0x0d, 0x28, 0x06, 0xda, 0x83, 0x4c, 0x40, 0x91, 0xac, 0x11, 0x7c, 0x34, 0x3f, 0x60, 0xe0, - 0x6f, 0xbc, 0xda, 0x0a, 0xb0, 0x50, 0x0d, 0xb2, 0x43, 0x36, 0x6d, 0x52, 0x2b, 0x96, 0x59, 0x96, - 0x02, 0x8e, 0x40, 0xb3, 0x55, 0xe8, 0x4b, 0x01, 0xae, 0xd4, 0xd6, 0x49, 0xe9, 0xef, 0xd8, 0xa6, - 0xb9, 0xaf, 0x6a, 0x2f, 0x68, 0x37, 0x17, 0x94, 0xfe, 0x5c, 0x8a, 0x9e, 0x92, 0x5c, 0xc3, 0x6f, - 0x90, 0xf6, 0x67, 0xd9, 0x39, 0xfa, 0x58, 0xee, 0x93, 0x1b, 0x0b, 0x4a, 0x48, 0xbd, 0xf2, 0x63, - 0x48, 0x52, 0x03, 0x91, 0x00, 0xd9, 0xed, 0x3e, 0xed, 0xf6, 0xf6, 0x48, 0x5b, 0x56, 0x84, 0x6c, - 0xb3, 0xd5, 0x69, 0xf5, 0x5b, 0x83, 0x5e, 0xb7, 0xf3, 0x99, 0x24, 0xa0, 0x3b, 0x70, 0x8b, 0x09, - 0x6a, 0xdd, 0xe6, 0x60, 0x4f, 0x69, 0xf3, 0xa1, 0x58, 0x65, 0x25, 0x1c, 0x81, 0x22, 0x24, 0xba, - 0xbd, 0x2e, 0x69, 0xc2, 0x48, 0x2c, 0x36, 0x9b, 0x92, 0x40, 0x63, 0x51, 0xe9, 0x6d, 0x4b, 0xb1, - 0x7a, 0x0e, 0x40, 0x0f, 0xcc, 0xb9, 0x99, 0x10, 0x53, 0x52, 0xba, 0xf2, 0x8b, 0xbb, 0x50, 0xbc, - 0xe0, 0xbf, 0x57, 0x24, 0xff, 0x32, 0x4d, 0xfe, 0x7e, 0x21, 0x29, 0x45, 0x92, 0x7f, 0x8c, 0xe5, - 0xfd, 0x47, 0x90, 0x19, 0xa9, 0x0e, 0xb6, 0x3c, 0x62, 0xff, 0x44, 0xa4, 0x7f, 0x10, 0xb7, 0xe9, - 0x40, 0x30, 0x5d, 0xf4, 0x27, 0xb6, 0x89, 0x52, 0xfa, 0x18, 0x3b, 0x2e, 0xf1, 0x06, 0xff, 0xca, - 0xee, 0xb0, 0xa7, 0x85, 0xc5, 0xc9, 0xae, 0x9e, 0xfb, 0x13, 0x14, 0x3e, 0x13, 0x6d, 0xc3, 0xe2, - 0xd0, 0xd6, 0x8d, 0x03, 0x43, 0xf3, 0xef, 0xdb, 0x33, 0x86, 0x7e, 0xff, 0x9d, 0x5d, 0x7b, 0x3b, - 0x74, 0x1b, 0x63, 0xcf, 0x30, 0xab, 0x47, 0xa6, 0x56, 0xed, 0xf3, 0xa7, 0x13, 0x76, 0x22, 0x29, - 0xac, 0x4d, 0x06, 0xd1, 0x13, 0x48, 0xf3, 0x8a, 0x56, 0xa4, 0xd9, 0x65, 0xde, 0x38, 0x63, 0x88, - 0x5c, 0x1b, 0xad, 0x43, 0xc1, 0xc2, 0x27, 0xe1, 0xae, 0x25, 0x13, 0xf1, 0xc4, 0x5c, 0x17, 0x9f, - 0x4c, 0x6f, 0x59, 0x72, 0xd6, 0x64, 0x44, 0x47, 0xcf, 0x20, 0x3f, 0x72, 0x8c, 0xa1, 0xea, 0x9c, - 0x0e, 0xfc, 0xe0, 0x85, 0xeb, 0x04, 0x6f, 0x40, 0xfb, 0x3e, 0x04, 0x1d, 0x45, 0xeb, 0xe0, 0x37, - 0x09, 0xd8, 0x95, 0xb3, 0xf4, 0x8c, 0xd7, 0x03, 0xe3, 0xca, 0xa8, 0x0e, 0x79, 0x7a, 0xc4, 0xa0, - 0x3b, 0xc9, 0xd1, 0x13, 0x2e, 0xb1, 0x13, 0x66, 0xc9, 0x09, 0xa7, 0x74, 0x28, 0x59, 0x2b, 0x90, - 0xeb, 0x68, 0x13, 0x20, 0x78, 0xb2, 0x22, 0x19, 0xf7, 0xaa, 0x82, 0x66, 0x9b, 0x4f, 0x9c, 0x6c, - 0x49, 0x09, 0x69, 0xa3, 0x2d, 0xc8, 0xf0, 0x20, 0xf6, 0x53, 0xed, 0xec, 0x98, 0xbc, 0x4c, 0x29, - 0x9c, 0x48, 0x02, 0x04, 0xd4, 0x85, 0xa4, 0x89, 0x55, 0x17, 0xb3, 0x7c, 0xfb, 0x78, 0x4e, 0xe6, - 0xdf, 0xd1, 0x8e, 0xf0, 0x50, 0x6d, 0x1c, 0x91, 0xda, 0xbd, 0x43, 0xf4, 0x15, 0x1f, 0x06, 0x75, - 0x41, 0xa2, 0xe6, 0x0a, 0xb3, 0x93, 0x44, 0x2d, 0xf6, 0x2e, 0xb3, 0x58, 0x81, 0x58, 0x6c, 0x26, - 0x43, 0x51, 0x7f, 0xda, 0x9a, 0xb0, 0xd4, 0x8f, 0xa0, 0x70, 0x60, 0x3b, 0x43, 0xd5, 0x1b, 0xf0, - 0xc0, 0x59, 0x9c, 0x54, 0xe4, 0xaf, 0xce, 0x96, 0xf3, 0xeb, 0x74, 0x94, 0x07, 0x4d, 0xfe, 0x20, - 0xfc, 0x89, 0x36, 0x38, 0x99, 0xdf, 0xa0, 0xdc, 0xfb, 0xe1, 0xbc, 0xa7, 0xbb, 0xcc, 0xe4, 0x5d, - 0x48, 0xd1, 0xb4, 0xe6, 0xca, 0x37, 0xa9, 0xcd, 0x5f, 0x33, 0x45, 0x2a, 0x0c, 0x05, 0x7d, 0x0e, - 0x05, 0x9d, 0x48, 0x0c, 0xeb, 0x90, 0x55, 0xfc, 0xb7, 0x28, 0xee, 0xea, 0x9c, 0xb8, 0xa4, 0x1b, - 0x68, 0x5b, 0x07, 0x36, 0x2f, 0xf6, 0x38, 0x98, 0xdf, 0x25, 0xf4, 0x40, 0x3c, 0x50, 0x87, 0x86, - 0x69, 0x60, 0x57, 0xbe, 0x4d, 0x71, 0x3f, 0xba, 0x32, 0xc2, 0x2f, 0x3e, 0x98, 0xf0, 0x54, 0xc0, - 0x41, 0x82, 0x40, 0xa7, 0x82, 0x53, 0x72, 0xa9, 0x6f, 0x5d, 0x0e, 0x74, 0xfe, 0x60, 0x12, 0x79, - 0x3c, 0xa1, 0x81, 0xce, 0xbe, 0x74, 0xf4, 0x0e, 0xc0, 0xb1, 0x81, 0x7f, 0x3e, 0x78, 0x39, 0xc6, - 0xce, 0xa9, 0x2c, 0x87, 0x78, 0x37, 0x43, 0xe4, 0xcf, 0x88, 0x18, 0x7d, 0x0c, 0x19, 0x1d, 0x8f, - 0xb0, 0xa5, 0xbb, 0x3d, 0x4b, 0xbe, 0x43, 0xab, 0xc9, 0x1b, 0xa4, 0xc5, 0x69, 0x72, 0x21, 0xe3, - 0xd5, 0xc9, 0x2c, 0xf4, 0x05, 0xe4, 0xfc, 0x0f, 0xac, 0xf7, 0xac, 0xfa, 0xa9, 0x5c, 0xa2, 0x87, - 0x7e, 0x38, 0xa7, 0x31, 0x27, 0xa5, 0xf3, 0x4d, 0x7e, 0x9e, 0x66, 0x08, 0x4d, 0x89, 0x60, 0xa3, - 0xcf, 0x21, 0xc7, 0xbd, 0x7b, 0xd3, 0xde, 0x77, 0xe5, 0xbb, 0x57, 0x36, 0xfd, 0x17, 0xd7, 0xda, - 0x9a, 0xa8, 0x72, 0xde, 0x0a, 0xa3, 0xa1, 0x4f, 0x21, 0x1f, 0xbc, 0x94, 0xd9, 0x23, 0xcf, 0x95, - 0xef, 0xd1, 0xc0, 0x7c, 0x34, 0xaf, 0xeb, 0x32, 0xdd, 0xde, 0xc8, 0x73, 0x95, 0x9c, 0x1b, 0xfa, - 0x42, 0xf7, 0x21, 0xa3, 0x3b, 0xf6, 0xc8, 0xcf, 0x1f, 0x6f, 0x97, 0x85, 0x95, 0x38, 0xbf, 0x66, - 0x22, 0xa6, 0x89, 0x61, 0x00, 0x05, 0x07, 0x8f, 0x4c, 0x55, 0xc3, 0x43, 0x92, 0xd9, 0xec, 0x03, - 0x79, 0x89, 0xae, 0xbe, 0x36, 0xb7, 0x21, 0x03, 0x65, 0xee, 0x98, 0x21, 0xbc, 0xde, 0x01, 0xda, - 0x05, 0x50, 0xc7, 0xba, 0xe1, 0x0d, 0x86, 0xb6, 0x8e, 0xe5, 0xe5, 0x2b, 0x9f, 0x7c, 0x2f, 0x82, - 0xd7, 0x88, 0xe2, 0x96, 0xad, 0xe3, 0xe0, 0xf1, 0x89, 0x0b, 0xd0, 0xc7, 0x90, 0xa5, 0x47, 0xfb, - 0xc2, 0xde, 0x27, 0xbe, 0x59, 0xa6, 0x87, 0x5b, 0x64, 0x77, 0x99, 0x69, 0x3a, 0xf6, 0x68, 0xd3, - 0xde, 0xa7, 0x1e, 0xc3, 0x7e, 0xea, 0xc8, 0x85, 0xdc, 0xa1, 0x36, 0x98, 0x50, 0xe9, 0x7d, 0x7a, - 0x8b, 0x3f, 0x9c, 0x73, 0x2f, 0x4f, 0x1a, 0x53, 0xc8, 0xf5, 0x06, 0xcf, 0x09, 0x4f, 0x1a, 0x5c, - 0xe6, 0x2a, 0xd9, 0x43, 0x2d, 0xf8, 0x28, 0x7d, 0x2d, 0xc0, 0xe2, 0x25, 0xea, 0x44, 0x3f, 0x83, - 0xb4, 0x65, 0xeb, 0xa1, 0xc7, 0xb2, 0x16, 0x03, 0x4a, 0x75, 0x6d, 0xdd, 0x7f, 0x2b, 0x7b, 0x34, - 0xd7, 0xfb, 0x2e, 0xfd, 0x35, 0xda, 0xaf, 0xfa, 0x6a, 0x4a, 0x8a, 0xa0, 0xb6, 0x75, 0xf4, 0x11, - 0x14, 0xf1, 0xc9, 0xc8, 0x70, 0x42, 0xe5, 0x43, 0x2c, 0x74, 0xfd, 0x85, 0xc9, 0x20, 0x71, 0x82, - 0xd2, 0x9f, 0x05, 0x28, 0x5e, 0xa0, 0x2d, 0x52, 0x29, 0xd1, 0x87, 0xd8, 0x48, 0xa5, 0x44, 0x24, - 0xa1, 0xa6, 0xe2, 0xaa, 0x3f, 0x41, 0xe2, 0x6f, 0xfa, 0x27, 0x48, 0xf4, 0xdd, 0x23, 0x39, 0xff, - 0xbb, 0xc7, 0x66, 0x42, 0x4c, 0x48, 0xc9, 0xd2, 0x67, 0x20, 0x72, 0xca, 0x8c, 0x96, 0x6e, 0xc2, - 0x9c, 0xa5, 0xdb, 0xcc, 0x73, 0x96, 0xbe, 0x12, 0x20, 0x13, 0xfe, 0x77, 0x29, 0x16, 0xa0, 0x4e, - 0xaf, 0x1c, 0x5f, 0xf3, 0xa9, 0x33, 0x6a, 0x81, 0xf8, 0xfc, 0x16, 0x28, 0x1d, 0x43, 0x36, 0xc4, - 0x3a, 0x17, 0x7b, 0x07, 0xe1, 0x35, 0x7a, 0x87, 0x77, 0x21, 0xc5, 0x42, 0xcd, 0x77, 0xa4, 0x3c, - 0xd3, 0x4e, 0xfa, 0x61, 0x96, 0xfc, 0x82, 0x84, 0x58, 0xe9, 0x0f, 0x02, 0xe4, 0xc2, 0x7c, 0x84, - 0x2a, 0x90, 0x31, 0x2c, 0xcd, 0xa1, 0x64, 0x40, 0xd7, 0xe5, 0x2e, 0x38, 0x11, 0x13, 0x96, 0x1a, - 0x1a, 0xd6, 0x80, 0x3e, 0x3f, 0x46, 0xdc, 0x54, 0x1c, 0x1a, 0xd6, 0x73, 0x22, 0xa5, 0x53, 0xd4, - 0x13, 0x36, 0x25, 0x1e, 0x99, 0xa2, 0x9e, 0xf8, 0x53, 0x4a, 0x34, 0xf1, 0x3b, 0x1e, 0xad, 0xcc, - 0xe3, 0xa1, 0x54, 0xee, 0x78, 0x68, 0x09, 0xd2, 0xc7, 0x86, 0xe3, 0x8d, 0x55, 0x93, 0x16, 0xe1, - 0xbc, 0xef, 0xe1, 0xc2, 0xd2, 0x11, 0x64, 0x43, 0x3c, 0x36, 0xc7, 0x85, 0x7e, 0x1f, 0x12, 0x41, - 0x50, 0xcd, 0x59, 0x93, 0x53, 0x85, 0xd2, 0xaf, 0x04, 0xb8, 0x39, 0x8d, 0x49, 0x22, 0x2e, 0xe2, - 0xdb, 0x69, 0x2e, 0x17, 0x89, 0x30, 0x7c, 0x6c, 0x2a, 0xc3, 0x4f, 0x6e, 0x2e, 0x3e, 0xfb, 0xe6, - 0x2a, 0xef, 0xf1, 0x66, 0x0d, 0x20, 0xb5, 0xbd, 0x5b, 0xef, 0xb4, 0x1b, 0x53, 0x1b, 0x2d, 0xd2, - 0x92, 0x05, 0xac, 0x8c, 0x72, 0x20, 0x36, 0xdb, 0x3b, 0xb5, 0x7a, 0xa7, 0xd5, 0x94, 0x16, 0x50, - 0x1e, 0x32, 0x4a, 0xab, 0xd6, 0xa4, 0x1d, 0x9c, 0x24, 0xfc, 0x20, 0xf1, 0xe5, 0x6f, 0x96, 0x05, - 0xbf, 0x15, 0xdb, 0x4c, 0x88, 0x48, 0xba, 0x51, 0xf9, 0x5a, 0x00, 0xd4, 0x54, 0x3d, 0x95, 0x30, - 0xc0, 0x35, 0x7a, 0xb2, 0xd8, 0x15, 0x17, 0x11, 0xad, 0xb3, 0xe3, 0x6f, 0x52, 0x67, 0xfb, 0x1b, - 0xae, 0x7c, 0x25, 0x00, 0x84, 0x36, 0xf7, 0x49, 0xf8, 0xaf, 0xe3, 0xd9, 0x2d, 0xc5, 0x85, 0x6c, - 0xb1, 0xb1, 0xc0, 0xff, 0x58, 0x7e, 0x02, 0xa2, 0xce, 0x8e, 0xcc, 0xbc, 0x65, 0x66, 0xed, 0x7e, - 0xc9, 0x32, 0x1b, 0xe4, 0x1a, 0x99, 0xb4, 0x9e, 0x86, 0xe4, 0xd8, 0x32, 0x6c, 0xeb, 0xfd, 0x66, - 0xf8, 0x49, 0x89, 0xb3, 0x27, 0x31, 0x3e, 0xfd, 0xad, 0x7a, 0x58, 0xf7, 0xbb, 0xec, 0x5d, 0xeb, - 0x38, 0x10, 0x08, 0xa8, 0x00, 0xc0, 0xc6, 0x0d, 0xeb, 0x50, 0x8a, 0xd5, 0xef, 0x7f, 0xf3, 0xf7, - 0xa5, 0x85, 0x6f, 0xce, 0x97, 0x84, 0x3f, 0x9d, 0x2f, 0x09, 0x7f, 0x39, 0x5f, 0x12, 0xfe, 0x76, - 0xbe, 0x24, 0xfc, 0xf2, 0x1f, 0x4b, 0x0b, 0x3f, 0x49, 0xb3, 0x0d, 0xfd, 0x2f, 0x00, 0x00, 0xff, - 0xff, 0xd8, 0x5f, 0x8e, 0x22, 0xd0, 0x1f, 0x00, 0x00, + proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_81f3763b694c15f3) +} + +var fileDescriptor_structured_81f3763b694c15f3 = []byte{ + // 2838 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x59, 0x4b, 0x73, 0xdb, 0xc8, + 0xf1, 0x17, 0xf8, 0x04, 0x9b, 0x2f, 0x68, 0xfc, 0x58, 0x98, 0xf6, 0x4a, 0x34, 0x77, 0xbd, 0x7f, + 0xfd, 0xf7, 0x41, 0x79, 0xe5, 0x6c, 0xe2, 0x4a, 0x52, 0x5b, 0xe1, 0x4b, 0x16, 0x65, 0x8a, 0x94, + 0x21, 0xca, 0xda, 0x4d, 0x6d, 0xc2, 0x82, 0x80, 0x91, 0x84, 0x35, 0x08, 0xd0, 0x00, 0xa8, 0x48, + 0xdf, 0x60, 0x8f, 0x39, 0x25, 0xa7, 0x6c, 0xa5, 0xb6, 0x72, 0x48, 0x55, 0xae, 0x39, 0xe4, 0x23, + 0xec, 0x31, 0x95, 0x53, 0x2a, 0x07, 0x55, 0xa2, 0x54, 0xae, 0xc9, 0xdd, 0x95, 0x54, 0xa5, 0x66, + 0x30, 0x03, 0x02, 0x12, 0xa9, 0x50, 0xf6, 0x8d, 0xe8, 0x99, 0xfe, 0xcd, 0x4c, 0x4f, 0xf7, 0xaf, + 0xbb, 0x87, 0x70, 0xcf, 0x7d, 0x69, 0xae, 0xba, 0x2f, 0xcd, 0x7d, 0xd5, 0xc5, 0xab, 0xae, 0xe7, + 0x8c, 0x35, 0x6f, 0xec, 0x60, 0xbd, 0x3a, 0x72, 0x6c, 0xcf, 0x46, 0xb7, 0x34, 0x5b, 0x7b, 0xe1, + 0xd8, 0xaa, 0x76, 0x54, 0x75, 0x5f, 0x9a, 0x55, 0x36, 0xaf, 0x24, 0x8f, 0x3d, 0xc3, 0x5c, 0x3d, + 0x32, 0xb5, 0x55, 0xcf, 0x18, 0x62, 0xd7, 0x53, 0x87, 0x23, 0x5f, 0xa1, 0x74, 0x37, 0x0c, 0x37, + 0x72, 0x8c, 0x63, 0xc3, 0xc4, 0x87, 0x98, 0x0d, 0xde, 0x3c, 0xb4, 0x0f, 0x6d, 0xfa, 0x73, 0x95, + 0xfc, 0xf2, 0xa5, 0x95, 0x7f, 0x27, 0xe0, 0xc6, 0xba, 0xed, 0x60, 0xe3, 0xd0, 0x7a, 0x8a, 0x4f, + 0x15, 0x7c, 0x80, 0x1d, 0x6c, 0x69, 0x18, 0x95, 0x21, 0xe9, 0xa9, 0xfb, 0x26, 0x96, 0x85, 0xb2, + 0xb0, 0x92, 0xaf, 0xc3, 0xb7, 0x67, 0xcb, 0x0b, 0xaf, 0xce, 0x96, 0x63, 0xed, 0xa6, 0xe2, 0x0f, + 0xa0, 0x07, 0x90, 0x34, 0x2c, 0x1d, 0x9f, 0xc8, 0x31, 0x3a, 0xa3, 0xc8, 0x66, 0xa4, 0xdb, 0x44, + 0x48, 0xa6, 0xd1, 0x51, 0x24, 0x43, 0xc2, 0x52, 0x87, 0x58, 0x8e, 0x97, 0x85, 0x95, 0x4c, 0x3d, + 0x41, 0x66, 0x29, 0x54, 0x82, 0x9e, 0x82, 0x78, 0xac, 0x9a, 0x86, 0x6e, 0x78, 0xa7, 0x72, 0xa2, + 0x2c, 0xac, 0x14, 0xd6, 0xfe, 0xbf, 0x3a, 0xf5, 0xc4, 0xd5, 0x86, 0x6d, 0xb9, 0x9e, 0xa3, 0x1a, + 0x96, 0xf7, 0x9c, 0x29, 0x30, 0xa0, 0x00, 0x00, 0x3d, 0x84, 0x45, 0xf7, 0x48, 0x75, 0xb0, 0x3e, + 0x18, 0x39, 0xf8, 0xc0, 0x38, 0x19, 0x98, 0xd8, 0x92, 0x93, 0x65, 0x61, 0x25, 0xc9, 0xa6, 0x16, + 0xfd, 0xe1, 0x6d, 0x3a, 0xda, 0xc1, 0x16, 0xea, 0x43, 0xc6, 0xb6, 0x06, 0x3a, 0x36, 0xb1, 0x87, + 0xe5, 0x14, 0x5d, 0xff, 0xe3, 0x19, 0xeb, 0x4f, 0x31, 0x50, 0xb5, 0xa6, 0x79, 0x86, 0x6d, 0xf1, + 0x7d, 0xd8, 0x56, 0x93, 0x02, 0x31, 0xd4, 0xf1, 0x48, 0x57, 0x3d, 0x2c, 0xa7, 0xdf, 0x18, 0x75, + 0x97, 0x02, 0xa1, 0x0e, 0x24, 0x87, 0xaa, 0xa7, 0x1d, 0xc9, 0x22, 0x45, 0x7c, 0x78, 0x0d, 0xc4, + 0x2d, 0xa2, 0xc7, 0x00, 0x7d, 0x90, 0xca, 0x1e, 0xa4, 0xfc, 0x75, 0x50, 0x1e, 0x32, 0xdd, 0xde, + 0xa0, 0xd6, 0xe8, 0xb7, 0x7b, 0x5d, 0x69, 0x01, 0xe5, 0x40, 0x54, 0x5a, 0x3b, 0x7d, 0xa5, 0xdd, + 0xe8, 0x4b, 0x02, 0xf9, 0xda, 0x69, 0xf5, 0x07, 0xdd, 0xdd, 0x4e, 0x47, 0x8a, 0xa1, 0x22, 0x64, + 0xc9, 0x57, 0xb3, 0xb5, 0x5e, 0xdb, 0xed, 0xf4, 0xa5, 0x38, 0xca, 0x42, 0xba, 0x51, 0xdb, 0x69, + 0xd4, 0x9a, 0x2d, 0x29, 0x51, 0x4a, 0xfc, 0xf6, 0x37, 0x4b, 0x0b, 0x95, 0x87, 0x90, 0xa4, 0xcb, + 0x21, 0x80, 0xd4, 0x4e, 0x7b, 0x6b, 0xbb, 0xd3, 0x92, 0x16, 0x90, 0x08, 0x89, 0x75, 0x02, 0x21, + 0x10, 0x8d, 0xed, 0x9a, 0xd2, 0x6f, 0xd7, 0x3a, 0x52, 0x8c, 0x69, 0xfc, 0x33, 0x06, 0x52, 0xc3, + 0x36, 0xc7, 0x43, 0xab, 0x89, 0x5d, 0xcd, 0x31, 0x46, 0x9e, 0xed, 0x04, 0x2e, 0x23, 0x5c, 0x72, + 0x99, 0xf7, 0x20, 0x66, 0xe8, 0xcc, 0xe1, 0x6e, 0x13, 0xf9, 0x39, 0x75, 0xc9, 0x57, 0x67, 0xcb, + 0xa2, 0x8f, 0xd2, 0x6e, 0x2a, 0x31, 0x43, 0x47, 0x1d, 0x48, 0x78, 0xa7, 0x23, 0xdf, 0xe9, 0x72, + 0xf5, 0xc7, 0x64, 0xe6, 0x5f, 0xce, 0x96, 0x1f, 0x1e, 0x1a, 0xde, 0xd1, 0x78, 0xbf, 0xaa, 0xd9, + 0xc3, 0xd5, 0xc0, 0x80, 0xfa, 0xfe, 0xe4, 0xf7, 0xea, 0xe8, 0xc5, 0x21, 0x89, 0x9f, 0x55, 0xa2, + 0xec, 0x56, 0xfb, 0x0a, 0x45, 0x41, 0x65, 0x10, 0xad, 0xb1, 0x69, 0xd2, 0x70, 0x20, 0x8e, 0x2a, + 0xf2, 0xfb, 0xe1, 0x52, 0x74, 0x1f, 0x72, 0x3a, 0x3e, 0x50, 0xc7, 0xa6, 0x37, 0xc0, 0x27, 0x23, + 0x87, 0x3a, 0x5e, 0x46, 0xc9, 0x32, 0x59, 0xeb, 0x64, 0xe4, 0xa0, 0x7b, 0x90, 0x3a, 0x32, 0x74, + 0x1d, 0x5b, 0xd4, 0xd7, 0x38, 0x04, 0x93, 0xa1, 0x35, 0x58, 0x1c, 0xbb, 0xd8, 0x1d, 0xb8, 0xf8, + 0xe5, 0x98, 0x5c, 0xdb, 0xc0, 0xd0, 0x5d, 0x19, 0xca, 0xf1, 0x95, 0x7c, 0x3d, 0xc5, 0xc2, 0xae, + 0x48, 0x26, 0xec, 0xb0, 0xf1, 0xb6, 0xee, 0x92, 0x45, 0x35, 0x7b, 0x38, 0x1a, 0x7b, 0xd8, 0x5f, + 0x34, 0xeb, 0x2f, 0xca, 0x64, 0x64, 0xd1, 0xcd, 0x84, 0x28, 0x4a, 0x99, 0xcd, 0x84, 0x98, 0x91, + 0x60, 0x33, 0x21, 0xa6, 0x25, 0xb1, 0xf2, 0x55, 0x0c, 0x6e, 0xfb, 0xa6, 0x5a, 0x57, 0x87, 0x86, + 0x79, 0xfa, 0xa6, 0x66, 0xf7, 0x51, 0x98, 0xd9, 0xe9, 0x8e, 0x08, 0xf6, 0x80, 0xa8, 0xb9, 0x72, + 0xbc, 0x1c, 0xf7, 0x77, 0x44, 0x64, 0x5d, 0x22, 0x42, 0x8f, 0x01, 0xd8, 0x14, 0x72, 0xc2, 0x04, + 0x3d, 0xe1, 0x9d, 0xf3, 0xb3, 0xe5, 0x0c, 0xbf, 0x3f, 0x37, 0x72, 0x99, 0x19, 0x7f, 0x32, 0x39, + 0x6e, 0x0f, 0x16, 0xb9, 0x8d, 0x03, 0x04, 0x6a, 0xe8, 0x7c, 0xfd, 0x1d, 0xb6, 0xa7, 0x62, 0xd3, + 0x9f, 0xc0, 0xd5, 0x23, 0x50, 0x45, 0x3d, 0x32, 0xa8, 0x57, 0x7e, 0x17, 0x83, 0x9b, 0x6d, 0xcb, + 0xc3, 0x8e, 0x89, 0xd5, 0x63, 0x1c, 0x32, 0xc4, 0x67, 0x90, 0x51, 0x2d, 0x0d, 0xbb, 0x9e, 0xed, + 0xb8, 0xb2, 0x50, 0x8e, 0xaf, 0x64, 0xd7, 0xbe, 0x33, 0x23, 0xe2, 0xa6, 0xe9, 0x57, 0x6b, 0x4c, + 0x99, 0xd9, 0x70, 0x02, 0x56, 0xfa, 0x83, 0x00, 0x22, 0x1f, 0x45, 0x0f, 0x41, 0xa4, 0x4c, 0x4a, + 0xce, 0xe1, 0xb3, 0xec, 0x2d, 0x76, 0x8e, 0x74, 0x9f, 0xc8, 0xe9, 0xfe, 0xc9, 0xcd, 0xa7, 0xe9, + 0xb4, 0xb6, 0x8e, 0x3e, 0x01, 0x91, 0x92, 0xea, 0x20, 0xb8, 0x8d, 0x12, 0xd7, 0x60, 0xac, 0x1b, + 0x26, 0xe0, 0x34, 0x9d, 0xdb, 0xd6, 0x51, 0x63, 0x1a, 0x37, 0xc6, 0xa9, 0xfe, 0x5b, 0xdc, 0x72, + 0x3b, 0x51, 0x76, 0xbc, 0x44, 0x97, 0x95, 0x7f, 0xc4, 0xe1, 0xf6, 0xb6, 0xea, 0x78, 0x06, 0x21, + 0x0e, 0xc3, 0x3a, 0x0c, 0xd9, 0xeb, 0x01, 0x64, 0xad, 0xf1, 0x90, 0xdd, 0x8a, 0xcb, 0xce, 0xe2, + 0x9f, 0x1d, 0xac, 0xf1, 0xd0, 0x37, 0xb8, 0x4b, 0x82, 0xd2, 0x34, 0x5c, 0x4f, 0x8e, 0x51, 0x8b, + 0xae, 0xcd, 0xb0, 0xe8, 0xf4, 0x35, 0xaa, 0x1d, 0xc3, 0xf5, 0xb8, 0x4f, 0x12, 0x14, 0xd4, 0x83, + 0xa4, 0xa3, 0x5a, 0x87, 0x98, 0x3a, 0x59, 0x76, 0xed, 0xd1, 0xf5, 0xe0, 0x14, 0xa2, 0xca, 0x59, + 0x91, 0xe2, 0x94, 0x7e, 0x29, 0x40, 0x82, 0xac, 0x72, 0x45, 0x1c, 0xdc, 0x86, 0xd4, 0xb1, 0x6a, + 0x8e, 0xb1, 0x4b, 0xcf, 0x90, 0x53, 0xd8, 0x17, 0xfa, 0x09, 0x14, 0xdd, 0xf1, 0xfe, 0x28, 0xb4, + 0x14, 0x35, 0x6f, 0x76, 0xed, 0xa3, 0x6b, 0xed, 0x2a, 0xc8, 0x54, 0x51, 0xac, 0xd2, 0x0b, 0x48, + 0xd2, 0xfd, 0x5e, 0xb1, 0xb3, 0xfb, 0x90, 0xf3, 0xec, 0x01, 0x3e, 0xd1, 0xcc, 0xb1, 0x6b, 0x1c, + 0x63, 0xea, 0x1d, 0x39, 0x25, 0xeb, 0xd9, 0x2d, 0x2e, 0x42, 0x0f, 0xa0, 0x70, 0xe0, 0xd8, 0xc3, + 0x81, 0x61, 0xf1, 0x49, 0x94, 0x1d, 0x95, 0x3c, 0x91, 0xb6, 0xb9, 0xb0, 0xf2, 0x1f, 0x11, 0x8a, + 0xd4, 0x83, 0xe6, 0x62, 0x86, 0x07, 0x21, 0x66, 0xb8, 0x15, 0x61, 0x86, 0xc0, 0x0d, 0x09, 0x31, + 0xdc, 0x83, 0xd4, 0xd8, 0x32, 0x5e, 0x8e, 0xfd, 0x35, 0x03, 0xf2, 0xf3, 0x65, 0x97, 0x68, 0x23, + 0x71, 0x99, 0x36, 0x3e, 0x04, 0x44, 0x62, 0x06, 0x0f, 0x22, 0x13, 0x93, 0x74, 0xa2, 0x44, 0x47, + 0x1a, 0x33, 0x49, 0x26, 0x75, 0x0d, 0x92, 0xd9, 0x00, 0x09, 0x9f, 0x78, 0x8e, 0x3a, 0x08, 0xe9, + 0xa7, 0xa9, 0xfe, 0xd2, 0xf9, 0xd9, 0x72, 0xa1, 0x45, 0xc6, 0xa6, 0x83, 0x14, 0x70, 0x68, 0x4c, + 0x27, 0x3e, 0xb1, 0xc8, 0x30, 0x74, 0xc3, 0xc1, 0x34, 0xdd, 0xba, 0xb2, 0x58, 0x8e, 0x5f, 0x91, + 0xbe, 0x2f, 0x98, 0xbd, 0xda, 0xe4, 0x8a, 0x8a, 0xe4, 0x43, 0x05, 0x02, 0x17, 0x3d, 0x83, 0xec, + 0x81, 0x9f, 0xed, 0x07, 0x2f, 0xf0, 0xa9, 0x9c, 0xa1, 0xee, 0xf6, 0xfe, 0xfc, 0x75, 0x01, 0x8f, + 0xcf, 0x83, 0x60, 0x08, 0xed, 0x42, 0xde, 0xe1, 0xc3, 0xfa, 0x60, 0xff, 0x94, 0xe6, 0x9f, 0xd7, + 0x01, 0xcd, 0x4d, 0x60, 0xea, 0xa7, 0xe8, 0x19, 0x80, 0x11, 0xb0, 0x24, 0x4d, 0x52, 0xd9, 0xb5, + 0x0f, 0xae, 0x41, 0xa7, 0x7c, 0xa7, 0x13, 0x10, 0xb4, 0x07, 0x85, 0xc9, 0x17, 0xdd, 0x6a, 0xee, + 0x35, 0xb7, 0x9a, 0x0f, 0xe1, 0xd4, 0x4f, 0x51, 0x1f, 0x6e, 0x92, 0xf4, 0x69, 0xbb, 0x86, 0x87, + 0xc3, 0x2e, 0x90, 0xa7, 0x2e, 0x50, 0x39, 0x3f, 0x5b, 0x46, 0x0d, 0x3e, 0x3e, 0xdd, 0x0d, 0x90, + 0x76, 0x61, 0xdc, 0x77, 0xaa, 0x88, 0xf3, 0x12, 0xc4, 0xc2, 0xc4, 0xa9, 0x76, 0x26, 0xee, 0x7b, + 0xc9, 0xa9, 0x42, 0xae, 0x4d, 0x90, 0xf6, 0x20, 0x17, 0x61, 0x99, 0xe2, 0xeb, 0xb3, 0x4c, 0x04, + 0x08, 0xb5, 0x58, 0xc1, 0x24, 0xd1, 0xfa, 0xf2, 0x83, 0x39, 0x1d, 0xb4, 0x7f, 0x3a, 0xe2, 0x86, + 0xa4, 0xea, 0x95, 0x25, 0xc8, 0x04, 0x3e, 0x8a, 0xd2, 0x10, 0xaf, 0xed, 0x34, 0xfc, 0x0a, 0xb0, + 0xd9, 0xda, 0x69, 0x48, 0x42, 0xe5, 0x3e, 0x24, 0x88, 0x0e, 0xa9, 0x04, 0xd7, 0x7b, 0xca, 0x5e, + 0x4d, 0x69, 0xfa, 0x55, 0x67, 0xbb, 0xfb, 0xbc, 0xa5, 0xf4, 0x5b, 0x4d, 0x49, 0x20, 0x05, 0x0a, + 0x9a, 0xd4, 0xfb, 0x7d, 0x9b, 0x55, 0xc0, 0x87, 0x50, 0xd4, 0x02, 0xe9, 0x80, 0xee, 0x55, 0x28, + 0xc7, 0x56, 0x0a, 0x6b, 0x8f, 0xff, 0x67, 0xcf, 0xc0, 0x31, 0xc2, 0xa2, 0xc9, 0xc6, 0x0b, 0x5a, + 0x44, 0x1a, 0x70, 0x5d, 0xac, 0x1c, 0xbb, 0xc0, 0x75, 0x0a, 0x24, 0xb5, 0x23, 0xac, 0xbd, 0x60, + 0xdc, 0xfe, 0xdd, 0x19, 0x0b, 0xd3, 0xdc, 0x1d, 0x32, 0x52, 0x83, 0xe8, 0x4c, 0x96, 0xe6, 0x49, + 0x87, 0x42, 0x55, 0xee, 0x42, 0x21, 0xba, 0x2b, 0x94, 0x81, 0x64, 0x63, 0xa3, 0xd5, 0x78, 0x2a, + 0x2d, 0x54, 0xfe, 0x95, 0x00, 0x34, 0x01, 0xda, 0x1a, 0x7b, 0x2a, 0xb5, 0x6b, 0x0d, 0x52, 0xbe, + 0x23, 0x51, 0x3e, 0xce, 0xae, 0xfd, 0xdf, 0x4c, 0x0b, 0x44, 0xeb, 0xea, 0x8d, 0x05, 0x85, 0x29, + 0xa2, 0x4f, 0xc3, 0xbd, 0x5b, 0x76, 0xed, 0xbd, 0xf9, 0xee, 0x7b, 0x63, 0x81, 0x37, 0x75, 0x4f, + 0x21, 0xe9, 0x7a, 0xa4, 0xc3, 0x89, 0x53, 0x7f, 0x59, 0x9d, 0xa1, 0x7f, 0x79, 0xf3, 0xd5, 0x1d, + 0xa2, 0xc6, 0x6d, 0x40, 0x31, 0xd0, 0x1e, 0x64, 0x02, 0x8a, 0x64, 0x8d, 0xe0, 0xa3, 0xf9, 0x01, + 0x03, 0x7f, 0xe3, 0xd5, 0x56, 0x80, 0x85, 0x6a, 0x90, 0x1d, 0xb2, 0x69, 0x93, 0x5a, 0xb1, 0xcc, + 0xb2, 0x14, 0x70, 0x04, 0x9a, 0xad, 0x42, 0x5f, 0x0a, 0x70, 0xa5, 0xb6, 0x4e, 0x4a, 0x7f, 0xc7, + 0x36, 0xcd, 0x7d, 0x55, 0x7b, 0x41, 0xbb, 0xb9, 0xa0, 0xf4, 0xe7, 0x52, 0xf4, 0x94, 0xe4, 0x1a, + 0x7e, 0x83, 0xb4, 0x3f, 0xcb, 0xce, 0xd1, 0xc7, 0x72, 0x9f, 0xdc, 0x58, 0x50, 0x42, 0xea, 0x95, + 0x1f, 0x41, 0x92, 0x1a, 0x88, 0x04, 0xc8, 0x6e, 0xf7, 0x69, 0xb7, 0xb7, 0x47, 0xda, 0xb2, 0x22, + 0x64, 0x9b, 0xad, 0x4e, 0xab, 0xdf, 0x1a, 0xf4, 0xba, 0x9d, 0xcf, 0x25, 0x01, 0xdd, 0x81, 0x5b, + 0x4c, 0x50, 0xeb, 0x36, 0x07, 0x7b, 0x4a, 0x9b, 0x0f, 0xc5, 0x2a, 0x2b, 0xe1, 0x08, 0x14, 0x21, + 0xd1, 0xed, 0x75, 0x49, 0x13, 0x46, 0x62, 0xb1, 0xd9, 0x94, 0x04, 0x1a, 0x8b, 0x4a, 0x6f, 0x5b, + 0x8a, 0xd5, 0x73, 0x00, 0x7a, 0x60, 0xce, 0xcd, 0x84, 0x98, 0x92, 0xd2, 0x95, 0x5f, 0xdd, 0x85, + 0xe2, 0x05, 0xff, 0xbd, 0x22, 0xf9, 0x97, 0x69, 0xf2, 0xf7, 0x0b, 0x49, 0x29, 0x92, 0xfc, 0x63, + 0x2c, 0xef, 0x3f, 0x82, 0xcc, 0x48, 0x75, 0xb0, 0xe5, 0x11, 0xfb, 0x27, 0x22, 0xfd, 0x83, 0xb8, + 0x4d, 0x07, 0x82, 0xe9, 0xa2, 0x3f, 0xb1, 0x4d, 0x94, 0xd2, 0xc7, 0xd8, 0x71, 0x89, 0x37, 0xf8, + 0x57, 0x76, 0x87, 0x3d, 0x2d, 0x2c, 0x4e, 0x76, 0xf5, 0xdc, 0x9f, 0xa0, 0xf0, 0x99, 0x68, 0x1b, + 0x16, 0x87, 0xb6, 0x6e, 0x1c, 0x18, 0x9a, 0x7f, 0xdf, 0x9e, 0x31, 0xf4, 0xfb, 0xef, 0xec, 0xda, + 0xdb, 0xa1, 0xdb, 0x18, 0x7b, 0x86, 0x59, 0x3d, 0x32, 0xb5, 0x6a, 0x9f, 0x3f, 0x9d, 0xb0, 0x13, + 0x49, 0x61, 0x6d, 0x32, 0x88, 0x9e, 0x40, 0x9a, 0x57, 0xb4, 0x22, 0xcd, 0x2e, 0xf3, 0xc6, 0x19, + 0x43, 0xe4, 0xda, 0x68, 0x1d, 0x0a, 0x16, 0x3e, 0x09, 0x77, 0x2d, 0x99, 0x88, 0x27, 0xe6, 0xba, + 0xf8, 0x64, 0x7a, 0xcb, 0x92, 0xb3, 0x26, 0x23, 0x3a, 0x7a, 0x06, 0xf9, 0x91, 0x63, 0x0c, 0x55, + 0xe7, 0x74, 0xe0, 0x07, 0x2f, 0x5c, 0x27, 0x78, 0x03, 0xda, 0xf7, 0x21, 0xe8, 0x28, 0x5a, 0x07, + 0xbf, 0x49, 0xc0, 0xae, 0x9c, 0xa5, 0x67, 0xbc, 0x1e, 0x18, 0x57, 0x46, 0x75, 0xc8, 0xd3, 0x23, + 0x06, 0xdd, 0x49, 0x8e, 0x9e, 0x70, 0x89, 0x9d, 0x30, 0x4b, 0x4e, 0x38, 0xa5, 0x43, 0xc9, 0x5a, + 0x81, 0x5c, 0x47, 0x9b, 0x00, 0xc1, 0x93, 0x15, 0xc9, 0xb8, 0x57, 0x15, 0x34, 0xdb, 0x7c, 0xe2, + 0x64, 0x4b, 0x4a, 0x48, 0x1b, 0x6d, 0x41, 0x86, 0x07, 0xb1, 0x9f, 0x6a, 0x67, 0xc7, 0xe4, 0x65, + 0x4a, 0xe1, 0x44, 0x12, 0x20, 0xa0, 0x2e, 0x24, 0x4d, 0xac, 0xba, 0x98, 0xe5, 0xdb, 0xc7, 0x73, + 0x32, 0xff, 0x8e, 0x76, 0x84, 0x87, 0x6a, 0xe3, 0x88, 0xd4, 0xee, 0x1d, 0xa2, 0xaf, 0xf8, 0x30, + 0xa8, 0x0b, 0x12, 0x35, 0x57, 0x98, 0x9d, 0x24, 0x6a, 0xb1, 0x77, 0x99, 0xc5, 0x0a, 0xc4, 0x62, + 0x33, 0x19, 0x8a, 0xfa, 0xd3, 0xd6, 0x84, 0xa5, 0x7e, 0x08, 0x85, 0x03, 0xdb, 0x19, 0xaa, 0xde, + 0x80, 0x07, 0xce, 0xe2, 0xa4, 0x22, 0x7f, 0x75, 0xb6, 0x9c, 0x5f, 0xa7, 0xa3, 0x3c, 0x68, 0xf2, + 0x07, 0xe1, 0x4f, 0xb4, 0xc1, 0xc9, 0xfc, 0x06, 0xe5, 0xde, 0x0f, 0xe7, 0x3d, 0xdd, 0x65, 0x26, + 0xef, 0x42, 0x8a, 0xa6, 0x35, 0x57, 0xbe, 0x49, 0x6d, 0xfe, 0x9a, 0x29, 0x52, 0x61, 0x28, 0xe8, + 0x0b, 0x28, 0xe8, 0x44, 0x62, 0x58, 0x87, 0xac, 0xe2, 0xbf, 0x45, 0x71, 0x57, 0xe7, 0xc4, 0x25, + 0xdd, 0x40, 0xdb, 0x3a, 0xb0, 0x79, 0xb1, 0xc7, 0xc1, 0xfc, 0x2e, 0xa1, 0x07, 0xe2, 0x81, 0x3a, + 0x34, 0x4c, 0x03, 0xbb, 0xf2, 0x6d, 0x8a, 0xfb, 0xd1, 0x95, 0x11, 0x7e, 0xf1, 0xc1, 0x84, 0xa7, + 0x02, 0x0e, 0x12, 0x04, 0x3a, 0x15, 0x9c, 0x92, 0x4b, 0x7d, 0xeb, 0x72, 0xa0, 0xf3, 0x07, 0x93, + 0xc8, 0xe3, 0x09, 0x0d, 0x74, 0xf6, 0xa5, 0xa3, 0x77, 0x00, 0x8e, 0x0d, 0xfc, 0xb3, 0xc1, 0xcb, + 0x31, 0x76, 0x4e, 0x65, 0x39, 0xc4, 0xbb, 0x19, 0x22, 0x7f, 0x46, 0xc4, 0xe8, 0x63, 0xc8, 0xe8, + 0x78, 0x84, 0x2d, 0xdd, 0xed, 0x59, 0xf2, 0x1d, 0x5a, 0x4d, 0xde, 0x20, 0x2d, 0x4e, 0x93, 0x0b, + 0x19, 0xaf, 0x4e, 0x66, 0xa1, 0x2f, 0x21, 0xe7, 0x7f, 0x60, 0xbd, 0x67, 0xd5, 0x4f, 0xe5, 0x12, + 0x3d, 0xf4, 0xc3, 0x39, 0x8d, 0x39, 0x29, 0x9d, 0x6f, 0xf2, 0xf3, 0x34, 0x43, 0x68, 0x4a, 0x04, + 0x1b, 0x7d, 0x01, 0x39, 0xee, 0xdd, 0x9b, 0xf6, 0xbe, 0x2b, 0xdf, 0xbd, 0xb2, 0xe9, 0xbf, 0xb8, + 0xd6, 0xd6, 0x44, 0x95, 0xf3, 0x56, 0x18, 0x0d, 0x7d, 0x06, 0xf9, 0xe0, 0xa5, 0xcc, 0x1e, 0x79, + 0xae, 0x7c, 0x8f, 0x06, 0xe6, 0xa3, 0x79, 0x5d, 0x97, 0xe9, 0xf6, 0x46, 0x9e, 0xab, 0xe4, 0xdc, + 0xd0, 0x17, 0xba, 0x0f, 0x19, 0xdd, 0xb1, 0x47, 0x7e, 0xfe, 0x78, 0xbb, 0x2c, 0xac, 0xc4, 0xf9, + 0x35, 0x13, 0x31, 0x4d, 0x0c, 0x03, 0x28, 0x38, 0x78, 0x64, 0xaa, 0x1a, 0x1e, 0x92, 0xcc, 0x66, + 0x1f, 0xc8, 0x4b, 0x74, 0xf5, 0xb5, 0xb9, 0x0d, 0x19, 0x28, 0x73, 0xc7, 0x0c, 0xe1, 0xf5, 0x0e, + 0xd0, 0x2e, 0x80, 0x3a, 0xd6, 0x0d, 0x6f, 0x30, 0xb4, 0x75, 0x2c, 0x2f, 0x5f, 0xf9, 0xe4, 0x7b, + 0x11, 0xbc, 0x46, 0x14, 0xb7, 0x6c, 0x1d, 0x07, 0x8f, 0x4f, 0x5c, 0x80, 0x3e, 0x86, 0x2c, 0x3d, + 0xda, 0x97, 0xf6, 0x3e, 0xf1, 0xcd, 0x32, 0x3d, 0xdc, 0x22, 0xbb, 0xcb, 0x4c, 0xd3, 0xb1, 0x47, + 0x9b, 0xf6, 0x3e, 0xf5, 0x18, 0xf6, 0x53, 0x47, 0x2e, 0xe4, 0x0e, 0xb5, 0xc1, 0x84, 0x4a, 0xef, + 0xd3, 0x5b, 0xfc, 0xc1, 0x9c, 0x7b, 0x79, 0xd2, 0x98, 0x42, 0xae, 0x37, 0x78, 0x4e, 0x78, 0xd2, + 0xe0, 0x32, 0x57, 0xc9, 0x1e, 0x6a, 0xc1, 0x47, 0xe9, 0x1b, 0x01, 0x16, 0x2f, 0x51, 0x27, 0xfa, + 0x29, 0xa4, 0x2d, 0x5b, 0x0f, 0x3d, 0x96, 0xb5, 0x18, 0x50, 0xaa, 0x6b, 0xeb, 0xfe, 0x5b, 0xd9, + 0xa3, 0xb9, 0xde, 0x77, 0xe9, 0xaf, 0xd1, 0x7e, 0xd5, 0x57, 0x53, 0x52, 0x04, 0xb5, 0xad, 0xa3, + 0x8f, 0xa0, 0x88, 0x4f, 0x46, 0x86, 0x13, 0x2a, 0x1f, 0x62, 0xa1, 0xeb, 0x2f, 0x4c, 0x06, 0x89, + 0x13, 0x94, 0xfe, 0x24, 0x40, 0xf1, 0x02, 0x6d, 0x91, 0x4a, 0x89, 0x3e, 0xc4, 0x46, 0x2a, 0x25, + 0x22, 0x09, 0x35, 0x15, 0x57, 0xfd, 0x09, 0x12, 0x7f, 0xd3, 0x3f, 0x41, 0xa2, 0xef, 0x1e, 0xc9, + 0xf9, 0xdf, 0x3d, 0x36, 0x13, 0x62, 0x42, 0x4a, 0x96, 0x3e, 0x07, 0x91, 0x53, 0x66, 0xb4, 0x74, + 0x13, 0xe6, 0x2c, 0xdd, 0x66, 0x9e, 0xb3, 0xf4, 0xb5, 0x00, 0x99, 0xf0, 0xbf, 0x4b, 0xb1, 0x00, + 0x75, 0x7a, 0xe5, 0xf8, 0x9a, 0x4f, 0x9d, 0x51, 0x0b, 0xc4, 0xe7, 0xb7, 0x40, 0xe9, 0x18, 0xb2, + 0x21, 0xd6, 0xb9, 0xd8, 0x3b, 0x08, 0xaf, 0xd1, 0x3b, 0xbc, 0x0b, 0x29, 0x16, 0x6a, 0xbe, 0x23, + 0xe5, 0x99, 0x76, 0xd2, 0x0f, 0xb3, 0xe4, 0x97, 0x24, 0xc4, 0x4a, 0xbf, 0x17, 0x20, 0x17, 0xe6, + 0x23, 0x54, 0x81, 0x8c, 0x61, 0x69, 0x0e, 0x25, 0x03, 0xba, 0x2e, 0x77, 0xc1, 0x89, 0x98, 0xb0, + 0xd4, 0xd0, 0xb0, 0x06, 0xf4, 0xf9, 0x31, 0xe2, 0xa6, 0xe2, 0xd0, 0xb0, 0x9e, 0x13, 0x29, 0x9d, + 0xa2, 0x9e, 0xb0, 0x29, 0xf1, 0xc8, 0x14, 0xf5, 0xc4, 0x9f, 0x52, 0xa2, 0x89, 0xdf, 0xf1, 0x68, + 0x65, 0x1e, 0x0f, 0xa5, 0x72, 0xc7, 0x43, 0x4b, 0x90, 0x3e, 0x36, 0x1c, 0x6f, 0xac, 0x9a, 0xb4, + 0x08, 0xe7, 0x7d, 0x0f, 0x17, 0x96, 0x8e, 0x20, 0x1b, 0xe2, 0xb1, 0x39, 0x2e, 0xf4, 0x7b, 0x90, + 0x08, 0x82, 0x6a, 0xce, 0x9a, 0x9c, 0x2a, 0x94, 0x7e, 0x21, 0xc0, 0xcd, 0x69, 0x4c, 0x12, 0x71, + 0x11, 0xdf, 0x4e, 0x73, 0xb9, 0x48, 0x84, 0xe1, 0x63, 0x53, 0x19, 0x7e, 0x72, 0x73, 0xf1, 0xd9, + 0x37, 0x57, 0xf9, 0x84, 0x37, 0x6b, 0x00, 0xa9, 0xed, 0xdd, 0x7a, 0xa7, 0xdd, 0x98, 0xda, 0x68, + 0xa1, 0x3c, 0x64, 0xda, 0x5b, 0xdb, 0x3d, 0xa5, 0xdf, 0xee, 0x3e, 0x91, 0xe2, 0xa4, 0x43, 0x0b, + 0x48, 0x1a, 0xe5, 0x40, 0x6c, 0xb6, 0x77, 0x6a, 0xf5, 0x4e, 0xab, 0x29, 0x2d, 0x90, 0x99, 0x4a, + 0xab, 0xd6, 0xa4, 0x0d, 0x9d, 0x24, 0x7c, 0x3f, 0xf1, 0xd5, 0xaf, 0x97, 0x05, 0xbf, 0x33, 0xdb, + 0x4c, 0x88, 0x48, 0xba, 0x51, 0xf9, 0x46, 0x00, 0xd4, 0x54, 0x3d, 0x95, 0x10, 0xc2, 0x35, 0x5a, + 0xb4, 0xd8, 0x15, 0xf7, 0x12, 0x2d, 0xbb, 0xe3, 0x6f, 0x52, 0x76, 0xfb, 0x1b, 0xae, 0x7c, 0x2d, + 0x00, 0x84, 0x36, 0xf7, 0x69, 0xf8, 0x9f, 0xe4, 0xd9, 0x1d, 0xc6, 0x85, 0xe4, 0xb1, 0xb1, 0xc0, + 0xff, 0x67, 0x7e, 0x02, 0xa2, 0xce, 0x8e, 0xcc, 0x9c, 0x67, 0x66, 0x29, 0x7f, 0xc9, 0x32, 0x1b, + 0xe4, 0x56, 0x99, 0xb4, 0x9e, 0x86, 0xe4, 0xd8, 0x32, 0x6c, 0xeb, 0xfd, 0x66, 0xf8, 0x85, 0x89, + 0x93, 0x29, 0x31, 0x3e, 0xfd, 0xad, 0x7a, 0x58, 0xf7, 0x9b, 0xee, 0x5d, 0xeb, 0x38, 0x10, 0x08, + 0xa8, 0x00, 0xc0, 0xc6, 0x0d, 0xeb, 0x50, 0x8a, 0xd5, 0xef, 0x7f, 0xfb, 0xb7, 0xa5, 0x85, 0x6f, + 0xcf, 0x97, 0x84, 0x3f, 0x9e, 0x2f, 0x09, 0x7f, 0x3e, 0x5f, 0x12, 0xfe, 0x7a, 0xbe, 0x24, 0xfc, + 0xfc, 0xef, 0x4b, 0x0b, 0x3f, 0x4e, 0xb3, 0x0d, 0xfd, 0x37, 0x00, 0x00, 0xff, 0xff, 0x3e, 0xad, + 0x7f, 0xbf, 0xdf, 0x1f, 0x00, 0x00, } diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index 28e917fbc70e..5858f99820da 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -507,6 +507,8 @@ message TableDescriptor { ADD = 1; // Descriptor is being dropped. DROP = 2; + // Descriptor is being ingested. + IMPORTING = 3; } optional State state = 19 [(gogoproto.nullable) = false];