From c8ef13ba6c7650db905ad86c8efd1071794f17e9 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Mon, 11 Oct 2021 20:51:37 -0700 Subject: [PATCH 1/2] goschedstats: update comment I have checked that the structures still match go1.17.2. Release note: None Fixes #66418. --- pkg/util/goschedstats/runtime_go1.16.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/pkg/util/goschedstats/runtime_go1.16.go b/pkg/util/goschedstats/runtime_go1.16.go index 92a8b8d8c29e..638e012cb9a8 100644 --- a/pkg/util/goschedstats/runtime_go1.16.go +++ b/pkg/util/goschedstats/runtime_go1.16.go @@ -9,9 +9,8 @@ // licenses/APL.txt. // // The structure definitions in this file have been cross-checked against -// go1.16, and go1.17beta1 (needs revalidation). Before allowing newer -// versions, please check that the structures still match with those in -// go/src/runtime. +// go1.16, and go1.17. Before allowing newer versions, please check that the +// structures still match with those in go/src/runtime. //go:build gc && go1.16 && !go1.18 // +build gc,go1.16,!go1.18 From d27dafc5367331bf11037c4cfdc165abdd833881 Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Thu, 30 Sep 2021 12:45:00 -0400 Subject: [PATCH 2/2] sql,importccl: reorganize import files This change only involves moving code into appropriate files. Concretely, it splits up `import_stmt` into the more idiomatic planning, job, processor_planning pattern used by other bulk jobs. Informs: #70050 Release note: None --- pkg/ccl/importccl/BUILD.bazel | 6 +- .../{import_stmt.go => import_job.go} | 3014 +++++------------ pkg/ccl/importccl/import_planning.go | 1288 +++++++ .../importccl/import_processor_planning.go} | 265 +- pkg/sql/BUILD.bazel | 1 - pkg/sql/distsql_running.go | 74 + pkg/sql/distsql_running_test.go | 2 +- pkg/sql/schema_changer.go | 2 +- pkg/sql/testutils.go | 2 +- 9 files changed, 2353 insertions(+), 2301 deletions(-) rename pkg/ccl/importccl/{import_stmt.go => import_job.go} (56%) create mode 100644 pkg/ccl/importccl/import_planning.go rename pkg/{sql/distsql_plan_csv.go => ccl/importccl/import_processor_planning.go} (78%) diff --git a/pkg/ccl/importccl/BUILD.bazel b/pkg/ccl/importccl/BUILD.bazel index d75b7b951f18..c781c34920dc 100644 --- a/pkg/ccl/importccl/BUILD.bazel +++ b/pkg/ccl/importccl/BUILD.bazel @@ -4,8 +4,10 @@ go_library( name = "importccl", srcs = [ "exportcsv.go", + "import_job.go", + "import_planning.go", "import_processor.go", - "import_stmt.go", + "import_processor_planning.go", "import_table_creation.go", "import_type_resolver.go", "read_import_avro.go", @@ -60,6 +62,7 @@ go_library( "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/pgwire/pgnotice", + "//pkg/sql/physicalplan", "//pkg/sql/privilege", "//pkg/sql/row", "//pkg/sql/rowenc", @@ -88,6 +91,7 @@ go_library( "//pkg/workload", "@com_github_cockroachdb_apd_v2//:apd", "@com_github_cockroachdb_errors//:errors", + "@com_github_cockroachdb_logtags//:logtags", "@com_github_lib_pq//oid", "@com_github_linkedin_goavro_v2//:goavro", "@io_vitess_vitess//go/sqltypes", diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_job.go similarity index 56% rename from pkg/ccl/importccl/import_stmt.go rename to pkg/ccl/importccl/import_job.go index cebbc8f098fe..7080b5652ff7 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_job.go @@ -12,19 +12,11 @@ import ( "bytes" "context" "fmt" - "io/ioutil" "math" - "net/url" - "path" - "sort" - "strconv" - "strings" "time" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" - "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -33,34 +25,23 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/telemetry" - "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" - "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/stats" - "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "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/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -70,1483 +51,276 @@ import ( "github.com/cockroachdb/errors" ) -const ( - csvDelimiter = "delimiter" - csvComment = "comment" - csvNullIf = "nullif" - csvSkip = "skip" - csvRowLimit = "row_limit" - csvStrictQuotes = "strict_quotes" - - mysqlOutfileRowSep = "rows_terminated_by" - mysqlOutfileFieldSep = "fields_terminated_by" - mysqlOutfileEnclose = "fields_enclosed_by" - mysqlOutfileEscape = "fields_escaped_by" - - importOptionSSTSize = "sstsize" - importOptionDecompress = "decompress" - importOptionOversample = "oversample" - importOptionSkipFKs = "skip_foreign_keys" - importOptionDisableGlobMatch = "disable_glob_matching" - importOptionSaveRejected = "experimental_save_rejected" - importOptionDetached = "detached" - - pgCopyDelimiter = "delimiter" - pgCopyNull = "nullif" - - optMaxRowSize = "max_row_size" - - // Turn on strict validation when importing avro records. - avroStrict = "strict_validation" - // Default input format is assumed to be OCF (object container file). - // This default can be changed by specified either of these options. - avroBinRecords = "data_as_binary_records" - avroJSONRecords = "data_as_json_records" - // Record separator; default "\n" - avroRecordsSeparatedBy = "records_terminated_by" - // If we are importing avro records (binary or JSON), we must specify schema - // as either an inline JSON schema, or an external schema URI. - avroSchema = "schema" - avroSchemaURI = "schema_uri" - - pgDumpIgnoreAllUnsupported = "ignore_unsupported_statements" - pgDumpIgnoreShuntFileDest = "log_ignored_statements" - pgDumpUnsupportedSchemaStmtLog = "unsupported_schema_stmts" - pgDumpUnsupportedDataStmtLog = "unsupported_data_stmts" - - // RunningStatusImportBundleParseSchema indicates to the user that a bundle format - // schema is being parsed - runningStatusImportBundleParseSchema jobs.RunningStatus = "parsing schema on Import Bundle" -) +type importResumer struct { + job *jobs.Job + settings *cluster.Settings + res backupccl.RowCount -var importOptionExpectValues = map[string]sql.KVStringOptValidate{ - csvDelimiter: sql.KVStringOptRequireValue, - csvComment: sql.KVStringOptRequireValue, - csvNullIf: sql.KVStringOptRequireValue, - csvSkip: sql.KVStringOptRequireValue, - csvRowLimit: sql.KVStringOptRequireValue, - csvStrictQuotes: sql.KVStringOptRequireNoValue, - - mysqlOutfileRowSep: sql.KVStringOptRequireValue, - mysqlOutfileFieldSep: sql.KVStringOptRequireValue, - mysqlOutfileEnclose: sql.KVStringOptRequireValue, - mysqlOutfileEscape: sql.KVStringOptRequireValue, - - importOptionSSTSize: sql.KVStringOptRequireValue, - importOptionDecompress: sql.KVStringOptRequireValue, - importOptionOversample: sql.KVStringOptRequireValue, - importOptionSaveRejected: sql.KVStringOptRequireNoValue, - - importOptionSkipFKs: sql.KVStringOptRequireNoValue, - importOptionDisableGlobMatch: sql.KVStringOptRequireNoValue, - importOptionDetached: sql.KVStringOptRequireNoValue, - - optMaxRowSize: sql.KVStringOptRequireValue, - - avroStrict: sql.KVStringOptRequireNoValue, - avroSchema: sql.KVStringOptRequireValue, - avroSchemaURI: sql.KVStringOptRequireValue, - avroRecordsSeparatedBy: sql.KVStringOptRequireValue, - avroBinRecords: sql.KVStringOptRequireNoValue, - avroJSONRecords: sql.KVStringOptRequireNoValue, - - pgDumpIgnoreAllUnsupported: sql.KVStringOptRequireNoValue, - pgDumpIgnoreShuntFileDest: sql.KVStringOptRequireValue, + testingKnobs struct { + afterImport func(summary backupccl.RowCount) error + alwaysFlushJobProgress bool + } } -var pgDumpMaxLoggedStmts = 1024 +var _ jobs.TraceableJob = &importResumer{} -func testingSetMaxLogIgnoredImportStatements(maxLogSize int) (cleanup func()) { - prevLogSize := pgDumpMaxLoggedStmts - pgDumpMaxLoggedStmts = maxLogSize - return func() { - pgDumpMaxLoggedStmts = prevLogSize - } +func (r *importResumer) ForceRealSpan() bool { + return true } -func makeStringSet(opts ...string) map[string]struct{} { - res := make(map[string]struct{}, len(opts)) - for _, opt := range opts { - res[opt] = struct{}{} - } - return res +var _ jobs.Resumer = &importResumer{} + +type preparedSchemaMetadata struct { + schemaPreparedDetails jobspb.ImportDetails + schemaRewrites backupccl.DescRewriteMap + newSchemaIDToName map[descpb.ID]string + oldSchemaIDToName map[descpb.ID]string + queuedSchemaJobs []jobspb.JobID } -// Options common to all formats. -var allowedCommonOptions = makeStringSet( - importOptionSSTSize, importOptionDecompress, importOptionOversample, - importOptionSaveRejected, importOptionDisableGlobMatch, importOptionDetached) +// Resume is part of the jobs.Resumer interface. +func (r *importResumer) Resume(ctx context.Context, execCtx interface{}) error { + p := execCtx.(sql.JobExecContext) + if err := r.parseBundleSchemaIfNeeded(ctx, p); err != nil { + return err + } -// Format specific allowed options. -var avroAllowedOptions = makeStringSet( - avroStrict, avroBinRecords, avroJSONRecords, - avroRecordsSeparatedBy, avroSchema, avroSchemaURI, optMaxRowSize, csvRowLimit, -) -var csvAllowedOptions = makeStringSet( - csvDelimiter, csvComment, csvNullIf, csvSkip, csvStrictQuotes, csvRowLimit, -) -var mysqlOutAllowedOptions = makeStringSet( - mysqlOutfileRowSep, mysqlOutfileFieldSep, mysqlOutfileEnclose, - mysqlOutfileEscape, csvNullIf, csvSkip, csvRowLimit, -) -var mysqlDumpAllowedOptions = makeStringSet(importOptionSkipFKs, csvRowLimit) -var pgCopyAllowedOptions = makeStringSet(pgCopyDelimiter, pgCopyNull, optMaxRowSize) -var pgDumpAllowedOptions = makeStringSet(optMaxRowSize, importOptionSkipFKs, csvRowLimit, - pgDumpIgnoreAllUnsupported, pgDumpIgnoreShuntFileDest) - -// DROP is required because the target table needs to be take offline during -// IMPORT INTO. -var importIntoRequiredPrivileges = []privilege.Kind{privilege.INSERT, privilege.DROP} - -// File formats supported for IMPORT INTO -var allowedIntoFormats = map[string]struct{}{ - "CSV": {}, - "AVRO": {}, - "DELIMITED": {}, - "PGCOPY": {}, -} + details := r.job.Details().(jobspb.ImportDetails) + files := details.URIs + format := details.Format + + tables := make(map[string]*execinfrapb.ReadImportDataSpec_ImportTable, len(details.Tables)) + if details.Tables != nil { + // Skip prepare stage on job resumption, if it has already been completed. + if !details.PrepareComplete { + var schemaMetadata *preparedSchemaMetadata + if err := sql.DescsTxn(ctx, p.ExecCfg(), func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + ) error { + var preparedDetails jobspb.ImportDetails + schemaMetadata = &preparedSchemaMetadata{ + newSchemaIDToName: make(map[descpb.ID]string), + oldSchemaIDToName: make(map[descpb.ID]string), + } + var err error + curDetails := details + if len(details.Schemas) != 0 { + schemaMetadata, err = r.prepareSchemasForIngestion(ctx, p, curDetails, txn, descsCol) + if err != nil { + return err + } + curDetails = schemaMetadata.schemaPreparedDetails + } -// featureImportEnabled is used to enable and disable the IMPORT feature. -var featureImportEnabled = settings.RegisterBoolSetting( - "feature.import.enabled", - "set to true to enable imports, false to disable; default is true", - featureflag.FeatureFlagEnabledDefault, -).WithPublic() + preparedDetails, err = r.prepareTablesForIngestion(ctx, p, curDetails, txn, descsCol, + schemaMetadata) + if err != nil { + return err + } -func validateFormatOptions( - format string, specified map[string]string, formatAllowed map[string]struct{}, -) error { - for opt := range specified { - if _, ok := formatAllowed[opt]; !ok { - if _, ok = allowedCommonOptions[opt]; !ok { - return errors.Errorf( - "invalid option %q specified for %s import format", opt, format) + // Telemetry for multi-region. + for _, table := range preparedDetails.Tables { + _, dbDesc, err := descsCol.GetImmutableDatabaseByID( + ctx, txn, table.Desc.GetParentID(), tree.DatabaseLookupFlags{Required: true}) + if err != nil { + return err + } + if dbDesc.IsMultiRegion() { + telemetry.Inc(sqltelemetry.ImportIntoMultiRegionDatabaseCounter) + } + } + + // Update the job details now that the schemas and table descs have + // been "prepared". + return r.job.Update(ctx, txn, func( + txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, + ) error { + pl := md.Payload + *pl.GetImport() = preparedDetails + + // Update the set of descriptors for later observability. + // TODO(ajwerner): Do we need this idempotence test? + prev := md.Payload.DescriptorIDs + if prev == nil { + var descriptorIDs []descpb.ID + for _, schema := range preparedDetails.Schemas { + descriptorIDs = append(descriptorIDs, schema.Desc.GetID()) + } + for _, table := range preparedDetails.Tables { + descriptorIDs = append(descriptorIDs, table.Desc.GetID()) + } + pl.DescriptorIDs = descriptorIDs + } + ju.UpdatePayload(pl) + return nil + }) + }); err != nil { + return err } - } - } - return nil -} -func importJobDescription( - p sql.PlanHookState, - orig *tree.Import, - defs tree.TableDefs, - files []string, - opts map[string]string, -) (string, error) { - stmt := *orig - stmt.CreateFile = nil - stmt.CreateDefs = defs - stmt.Files = nil - for _, file := range files { - clean, err := cloud.SanitizeExternalStorageURI(file, nil /* extraParams */) - if err != nil { - return "", err + // Run the queued job which updates the database descriptor to contain the + // newly created schemas. + // NB: Seems like the registry eventually adopts the job anyways but this + // is in keeping with the semantics we use when creating a schema during + // sql execution. Namely, queue job in the txn which creates the schema + // desc and run once the txn has committed. + if err := p.ExecCfg().JobRegistry.Run(ctx, p.ExecCfg().InternalExecutor, + schemaMetadata.queuedSchemaJobs); err != nil { + return err + } + + // Re-initialize details after prepare step. + details = r.job.Details().(jobspb.ImportDetails) + emitImportJobEvent(ctx, p, jobs.StatusRunning, r.job) } - stmt.Files = append(stmt.Files, tree.NewDString(clean)) - } - stmt.Options = nil - for k, v := range opts { - opt := tree.KVOption{Key: tree.Name(k)} - val := importOptionExpectValues[k] == sql.KVStringOptRequireValue - val = val || (importOptionExpectValues[k] == sql.KVStringOptAny && len(v) > 0) - if val { - opt.Value = tree.NewDString(v) + + // Create a mapping from schemaID to schemaName. + schemaIDToName := make(map[descpb.ID]string) + for _, i := range details.Schemas { + schemaIDToName[i.Desc.GetID()] = i.Desc.GetName() } - stmt.Options = append(stmt.Options, opt) - } - sort.Slice(stmt.Options, func(i, j int) bool { return stmt.Options[i].Key < stmt.Options[j].Key }) - ann := p.ExtendedEvalContext().Annotations - return tree.AsStringWithFQNames(&stmt, ann), nil -} -func ensureRequiredPrivileges( - ctx context.Context, - requiredPrivileges []privilege.Kind, - p sql.PlanHookState, - desc *tabledesc.Mutable, -) error { - for _, priv := range requiredPrivileges { - err := p.CheckPrivilege(ctx, desc, priv) - if err != nil { - return err + for _, i := range details.Tables { + var tableName string + if i.Name != "" { + tableName = i.Name + } else if i.Desc != nil { + tableName = i.Desc.Name + } else { + return errors.New("invalid table specification") + } + + // If we are importing from PGDUMP, qualify the table name with the schema + // name since we support non-public schemas. + if details.Format.Format == roachpb.IOFileFormat_PgDump { + schemaName := tree.PublicSchema + if schema, ok := schemaIDToName[i.Desc.GetUnexposedParentSchemaID()]; ok { + schemaName = schema + } + tableName = fmt.Sprintf("%s.%s", schemaName, tableName) + } + tables[tableName] = &execinfrapb.ReadImportDataSpec_ImportTable{ + Desc: i.Desc, + TargetCols: i.TargetCols, + } } } - return nil -} + typeDescs := make([]*descpb.TypeDescriptor, len(details.Types)) + for i, t := range details.Types { + typeDescs[i] = t.Desc + } -// addToFileFormatTelemetry records the different stages of IMPORT on a per file -// format basis. -// -// The current states being counted are: -// attempted: Counted at the very beginning of the IMPORT. -// started: Counted just before the IMPORT job is started. -// failed: Counted when the IMPORT job is failed or canceled. -// succeeded: Counted when the IMPORT job completes successfully. -func addToFileFormatTelemetry(fileFormat, state string) { - telemetry.Count(fmt.Sprintf("%s.%s.%s", "import", strings.ToLower(fileFormat), state)) -} + // If details.Walltime is still 0, then it was not set during + // `prepareTablesForIngestion`. This indicates that we are in an IMPORT INTO, + // and that the walltime was not set in a previous run of IMPORT. + // + // In the case of importing into existing tables we must wait for all nodes + // to see the same version of the updated table descriptor, after which we + // shall chose a ts to import from. + if details.Walltime == 0 { + // Now that we know all the tables are offline, pick a walltime at which we + // will write. + details.Walltime = p.ExecCfg().Clock.Now().WallTime -// resolveUDTsUsedByImportInto resolves all the user defined types that are -// referenced by the table being imported into. -func resolveUDTsUsedByImportInto( - ctx context.Context, p sql.PlanHookState, table *tabledesc.Mutable, -) ([]catalog.TypeDescriptor, error) { - typeDescs := make([]catalog.TypeDescriptor, 0) - var dbDesc catalog.DatabaseDescriptor - err := sql.DescsTxn(ctx, p.ExecCfg(), func( - ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, - ) (err error) { - _, dbDesc, err = descriptors.GetImmutableDatabaseByID(ctx, txn, table.GetParentID(), - tree.DatabaseLookupFlags{ - Required: true, - AvoidCached: true, - }) - if err != nil { - return err - } - typeIDs, _, err := table.GetAllReferencedTypeIDs(dbDesc, - func(id descpb.ID) (catalog.TypeDescriptor, error) { - immutDesc, err := descriptors.GetImmutableTypeByID(ctx, txn, id, tree.ObjectLookupFlags{ - CommonLookupFlags: tree.CommonLookupFlags{ - Required: true, - AvoidCached: true, - }, - }) + // Check if the tables being imported into are starting empty, in which + // case we can cheaply clear-range instead of revert-range to cleanup. + for i := range details.Tables { + if !details.Tables[i].IsNew { + tblDesc := tabledesc.NewBuilder(details.Tables[i].Desc).BuildImmutableTable() + tblSpan := tblDesc.TableSpan(p.ExecCfg().Codec) + res, err := p.ExecCfg().DB.Scan(ctx, tblSpan.Key, tblSpan.EndKey, 1 /* maxRows */) if err != nil { - return nil, err + return errors.Wrap(err, "checking if existing table is empty") } - return immutDesc, nil - }) - if err != nil { - return errors.Wrap(err, "resolving type descriptors") - } - - for _, typeID := range typeIDs { - immutDesc, err := descriptors.GetImmutableTypeByID(ctx, txn, typeID, tree.ObjectLookupFlags{ - CommonLookupFlags: tree.CommonLookupFlags{ - Required: true, - AvoidCached: true, - }, - }) - if err != nil { - return err + details.Tables[i].WasEmpty = len(res) == 0 } - typeDescs = append(typeDescs, immutDesc) } - return err - }) - return typeDescs, err -} -// importPlanHook implements sql.PlanHookFn. -func importPlanHook( - ctx context.Context, stmt tree.Statement, p sql.PlanHookState, -) (sql.PlanHookRowFn, colinfo.ResultColumns, []sql.PlanNode, bool, error) { - importStmt, ok := stmt.(*tree.Import) - if !ok { - return nil, nil, nil, false, nil + if err := r.job.SetDetails(ctx, nil /* txn */, details); err != nil { + return err + } } - if !importStmt.Bundle && !importStmt.Into { - p.BufferClientNotice(ctx, pgnotice.Newf("IMPORT TABLE has been deprecated in 21.2, and will be removed in a future version."+ - " Instead, use CREATE TABLE with the desired schema, and IMPORT INTO the newly created table.")) + res, err := ingestWithRetry(ctx, p, r.job, tables, typeDescs, files, format, details.Walltime, + r.testingKnobs.alwaysFlushJobProgress) + if err != nil { + return err } - addToFileFormatTelemetry(importStmt.FileFormat, "attempted") - - if err := featureflag.CheckEnabled( - ctx, - p.ExecCfg(), - featureImportEnabled, - "IMPORT", - ); err != nil { - return nil, nil, nil, false, err + pkIDs := make(map[uint64]struct{}, len(details.Tables)) + for _, t := range details.Tables { + pkIDs[roachpb.BulkOpSummaryID(uint64(t.Desc.ID), uint64(t.Desc.PrimaryIndex.ID))] = struct{}{} } - - filesFn, err := p.TypeAsStringArray(ctx, importStmt.Files, "IMPORT") - if err != nil { - return nil, nil, nil, false, err + r.res.DataSize = res.DataSize + for id, count := range res.EntryCounts { + if _, ok := pkIDs[id]; ok { + r.res.Rows += count + } else { + r.res.IndexEntries += count + } } - - var createFileFn func() (string, error) - if !importStmt.Bundle && !importStmt.Into && importStmt.CreateDefs == nil { - createFileFn, err = p.TypeAsString(ctx, importStmt.CreateFile, "IMPORT") - if err != nil { - return nil, nil, nil, false, err + if r.testingKnobs.afterImport != nil { + if err := r.testingKnobs.afterImport(r.res); err != nil { + return err } } - optsFn, err := p.TypeAsStringOpts(ctx, importStmt.Options, importOptionExpectValues) - if err != nil { - return nil, nil, nil, false, err + // If the table being imported into referenced UDTs, ensure that a concurrent + // schema change on any of the typeDescs has not modified the type descriptor. If + // it has, it is unsafe to import the data and we fail the import job. + if err := r.checkForUDTModification(ctx, p.ExecCfg()); err != nil { + return err } - opts, optsErr := optsFn() + if err := r.publishSchemas(ctx, p.ExecCfg()); err != nil { + return err + } - var isDetached bool - if _, ok := opts[importOptionDetached]; ok { - isDetached = true + if err := r.publishTables(ctx, p.ExecCfg(), res); err != nil { + return err } - fn := func(ctx context.Context, _ []sql.PlanNode, resultsCh chan<- tree.Datums) error { - // TODO(dan): Move this span into sql. - ctx, span := tracing.ChildSpan(ctx, importStmt.StatementTag()) - defer span.Finish() + // As of 21.2 we do not write a protected timestamp record during IMPORT INTO. + // In case of a mixed version cluster with 21.1 and 21.2 nodes, it is possible + // that the job was planned on an older node and then resumed on a 21.2 node. + // Thus, we still need to clear the timestamp record that was written when the + // IMPORT INTO was planned on the older node. + // + // TODO(adityamaru): Remove in 22.1. + if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return r.releaseProtectedTimestamp(ctx, txn, p.ExecCfg().ProtectedTimestampProvider) + }); err != nil { + log.Errorf(ctx, "failed to release protected timestamp: %v", err) + } - walltime := p.ExecCfg().Clock.Now().WallTime + emitImportJobEvent(ctx, p, jobs.StatusSucceeded, r.job) - if !(p.ExtendedEvalContext().TxnImplicit || isDetached) { - return errors.Errorf("IMPORT cannot be used inside a transaction without DETACHED option") - } + addToFileFormatTelemetry(details.Format.Format.String(), "succeeded") + telemetry.CountBucketed("import.rows", r.res.Rows) + const mb = 1 << 20 + sizeMb := r.res.DataSize / mb + telemetry.CountBucketed("import.size-mb", sizeMb) - if optsErr != nil { - return optsErr - } - - filenamePatterns, err := filesFn() - if err != nil { - return err - } - - // Certain ExternalStorage URIs require super-user access. Check all the - // URIs passed to the IMPORT command. - for _, file := range filenamePatterns { - conf, err := cloud.ExternalStorageConfFromURI(file, p.User()) - if err != nil { - // If it is a workload URI, it won't parse as a storage config, but it - // also doesn't have any auth concerns so just continue. - if _, workloadErr := parseWorkloadConfig(file); workloadErr == nil { - continue - } - return err - } - if !conf.AccessIsWithExplicitAuth() { - err := p.RequireAdminRole(ctx, - fmt.Sprintf("IMPORT from the specified %s URI", conf.Provider.String())) - if err != nil { - return err - } - } - } - - var files []string - if _, ok := opts[importOptionDisableGlobMatch]; ok { - files = filenamePatterns - } else { - for _, file := range filenamePatterns { - uri, err := url.Parse(file) - if err != nil { - return err - } - if strings.Contains(uri.Scheme, "workload") || strings.HasPrefix(uri.Scheme, "http") { - files = append(files, file) - continue - } - prefix := cloud.GetPrefixBeforeWildcard(uri.Path) - if len(prefix) < len(uri.Path) { - pattern := uri.Path[len(prefix):] - uri.Path = prefix - s, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, uri.String(), p.User()) - if err != nil { - return err - } - var expandedFiles []string - if err := s.List(ctx, "", "", func(s string) error { - ok, err := path.Match(pattern, s) - if ok { - uri.Path = prefix + s - expandedFiles = append(expandedFiles, uri.String()) - } - return err - }); err != nil { - return err - } - if len(expandedFiles) < 1 { - return errors.Errorf(`no files matched %q in prefix %q in uri provided: %q`, pattern, prefix, file) - } - files = append(files, expandedFiles...) - } else { - files = append(files, file) - } - } - } - - // Typically the SQL grammar means it is only possible to specifying exactly - // one pgdump/mysqldump URI, but glob-expansion could have changed that. - if importStmt.Bundle && len(files) != 1 { - return pgerror.New(pgcode.FeatureNotSupported, "SQL dump files must be imported individually") - } - - table := importStmt.Table - var db catalog.DatabaseDescriptor - var sc catalog.SchemaDescriptor - if table != nil { - // TODO: As part of work for #34240, we should be operating on - // UnresolvedObjectNames here, rather than TableNames. - // We have a target table, so it might specify a DB in its name. - un := table.ToUnresolvedObjectName() - found, prefix, resPrefix, err := resolver.ResolveTarget(ctx, - un, p, p.SessionData().Database, p.SessionData().SearchPath) - if err != nil { - return pgerror.Wrap(err, pgcode.UndefinedTable, - "resolving target import name") - } - if !found { - // Check if database exists right now. It might not after the import is done, - // but it's better to fail fast than wait until restore. - return pgerror.Newf(pgcode.UndefinedObject, - "database does not exist: %q", table) - } - table.ObjectNamePrefix = prefix - db = resPrefix.Database - sc = resPrefix.Schema - // If this is a non-INTO import that will thus be making a new table, we - // need the CREATE priv in the target DB. - if !importStmt.Into { - if err := p.CheckPrivilege(ctx, db, privilege.CREATE); err != nil { - return err - } - } - - switch sc.SchemaKind() { - case catalog.SchemaVirtual: - return pgerror.Newf(pgcode.InvalidSchemaName, - "cannot import into schema %q", table.SchemaName) - } - } else { - // No target table means we're importing whatever we find into the session - // database, so it must exist. - txn := p.ExtendedEvalContext().Txn - db, err = p.Accessor().GetDatabaseDesc(ctx, txn, p.SessionData().Database, tree.DatabaseLookupFlags{ - AvoidCached: true, - Required: true, - }) - if err != nil { - return pgerror.Wrap(err, pgcode.UndefinedObject, - "could not resolve current database") - } - // If this is a non-INTO import that will thus be making a new table, we - // need the CREATE priv in the target DB. - if !importStmt.Into { - if err := p.CheckPrivilege(ctx, db, privilege.CREATE); err != nil { - return err - } - } - sc = schemadesc.GetPublicSchema() - } - - format := roachpb.IOFileFormat{} - switch importStmt.FileFormat { - case "CSV": - if err = validateFormatOptions(importStmt.FileFormat, opts, csvAllowedOptions); err != nil { - return err - } - format.Format = roachpb.IOFileFormat_CSV - // Set the default CSV separator for the cases when it is not overwritten. - format.Csv.Comma = ',' - if override, ok := opts[csvDelimiter]; ok { - comma, err := util.GetSingleRune(override) - if err != nil { - return pgerror.Wrap(err, pgcode.Syntax, "invalid comma value") - } - format.Csv.Comma = comma - } - - if override, ok := opts[csvComment]; ok { - comment, err := util.GetSingleRune(override) - if err != nil { - return pgerror.Wrap(err, pgcode.Syntax, "invalid comment value") - } - format.Csv.Comment = comment - } - - if override, ok := opts[csvNullIf]; ok { - format.Csv.NullEncoding = &override - } - - if override, ok := opts[csvSkip]; ok { - skip, err := strconv.Atoi(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid %s value", csvSkip) - } - if skip < 0 { - return pgerror.Newf(pgcode.Syntax, "%s must be >= 0", csvSkip) - } - format.Csv.Skip = uint32(skip) - } - if _, ok := opts[csvStrictQuotes]; ok { - format.Csv.StrictQuotes = true - } - if _, ok := opts[importOptionSaveRejected]; ok { - format.SaveRejected = true - } - if override, ok := opts[csvRowLimit]; ok { - rowLimit, err := strconv.Atoi(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) - } - if rowLimit <= 0 { - return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) - } - format.Csv.RowLimit = int64(rowLimit) - } - case "DELIMITED": - if err = validateFormatOptions(importStmt.FileFormat, opts, mysqlOutAllowedOptions); err != nil { - return err - } - format.Format = roachpb.IOFileFormat_MysqlOutfile - format.MysqlOut = roachpb.MySQLOutfileOptions{ - RowSeparator: '\n', - FieldSeparator: '\t', - } - if override, ok := opts[mysqlOutfileRowSep]; ok { - c, err := util.GetSingleRune(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, - "invalid %q value", mysqlOutfileRowSep) - } - format.MysqlOut.RowSeparator = c - } - - if override, ok := opts[mysqlOutfileFieldSep]; ok { - c, err := util.GetSingleRune(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid %q value", mysqlOutfileFieldSep) - } - format.MysqlOut.FieldSeparator = c - } - - if override, ok := opts[mysqlOutfileEnclose]; ok { - c, err := util.GetSingleRune(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid %q value", mysqlOutfileRowSep) - } - format.MysqlOut.Enclose = roachpb.MySQLOutfileOptions_Always - format.MysqlOut.Encloser = c - } - - if override, ok := opts[mysqlOutfileEscape]; ok { - c, err := util.GetSingleRune(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid %q value", mysqlOutfileRowSep) - } - format.MysqlOut.HasEscape = true - format.MysqlOut.Escape = c - } - if override, ok := opts[csvSkip]; ok { - skip, err := strconv.Atoi(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid %s value", csvSkip) - } - if skip < 0 { - return pgerror.Newf(pgcode.Syntax, "%s must be >= 0", csvSkip) - } - format.MysqlOut.Skip = uint32(skip) - } - if override, ok := opts[csvNullIf]; ok { - format.MysqlOut.NullEncoding = &override - } - if _, ok := opts[importOptionSaveRejected]; ok { - format.SaveRejected = true - } - if override, ok := opts[csvRowLimit]; ok { - rowLimit, err := strconv.Atoi(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) - } - if rowLimit <= 0 { - return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) - } - format.MysqlOut.RowLimit = int64(rowLimit) - } - case "MYSQLDUMP": - if err = validateFormatOptions(importStmt.FileFormat, opts, mysqlDumpAllowedOptions); err != nil { - return err - } - format.Format = roachpb.IOFileFormat_Mysqldump - if override, ok := opts[csvRowLimit]; ok { - rowLimit, err := strconv.Atoi(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) - } - if rowLimit <= 0 { - return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) - } - format.MysqlDump.RowLimit = int64(rowLimit) - } - case "PGCOPY": - if err = validateFormatOptions(importStmt.FileFormat, opts, pgCopyAllowedOptions); err != nil { - return err - } - format.Format = roachpb.IOFileFormat_PgCopy - format.PgCopy = roachpb.PgCopyOptions{ - Delimiter: '\t', - Null: `\N`, - } - if override, ok := opts[pgCopyDelimiter]; ok { - c, err := util.GetSingleRune(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid %q value", pgCopyDelimiter) - } - format.PgCopy.Delimiter = c - } - if override, ok := opts[pgCopyNull]; ok { - format.PgCopy.Null = override - } - maxRowSize := int32(defaultScanBuffer) - if override, ok := opts[optMaxRowSize]; ok { - sz, err := humanizeutil.ParseBytes(override) - if err != nil { - return err - } - if sz < 1 || sz > math.MaxInt32 { - return errors.Errorf("%d out of range: %d", maxRowSize, sz) - } - maxRowSize = int32(sz) - } - format.PgCopy.MaxRowSize = maxRowSize - case "PGDUMP": - if err = validateFormatOptions(importStmt.FileFormat, opts, pgDumpAllowedOptions); err != nil { - return err - } - format.Format = roachpb.IOFileFormat_PgDump - maxRowSize := int32(defaultScanBuffer) - if override, ok := opts[optMaxRowSize]; ok { - sz, err := humanizeutil.ParseBytes(override) - if err != nil { - return err - } - if sz < 1 || sz > math.MaxInt32 { - return errors.Errorf("%d out of range: %d", maxRowSize, sz) - } - maxRowSize = int32(sz) - } - format.PgDump.MaxRowSize = maxRowSize - if _, ok := opts[pgDumpIgnoreAllUnsupported]; ok { - format.PgDump.IgnoreUnsupported = true - } - - if dest, ok := opts[pgDumpIgnoreShuntFileDest]; ok { - if !format.PgDump.IgnoreUnsupported { - return errors.New("cannot log unsupported PGDUMP stmts without `ignore_unsupported_statements` option") - } - format.PgDump.IgnoreUnsupportedLog = dest - } - - if override, ok := opts[csvRowLimit]; ok { - rowLimit, err := strconv.Atoi(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) - } - if rowLimit <= 0 { - return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) - } - format.PgDump.RowLimit = int64(rowLimit) - } - case "AVRO": - if err = validateFormatOptions(importStmt.FileFormat, opts, avroAllowedOptions); err != nil { - return err - } - err := parseAvroOptions(ctx, opts, p, &format) - if err != nil { - return err - } - default: - return unimplemented.Newf("import.format", "unsupported import format: %q", importStmt.FileFormat) - } - - // sstSize, if 0, will be set to an appropriate default by the specific - // implementation (local or distributed) since each has different optimal - // settings. - var sstSize int64 - if override, ok := opts[importOptionSSTSize]; ok { - sz, err := humanizeutil.ParseBytes(override) - if err != nil { - return err - } - sstSize = sz - } - var oversample int64 - if override, ok := opts[importOptionOversample]; ok { - os, err := strconv.ParseInt(override, 10, 64) - if err != nil { - return err - } - oversample = os - } - - var skipFKs bool - if _, ok := opts[importOptionSkipFKs]; ok { - skipFKs = true - } - - if override, ok := opts[importOptionDecompress]; ok { - found := false - for name, value := range roachpb.IOFileFormat_Compression_value { - if strings.EqualFold(name, override) { - format.Compression = roachpb.IOFileFormat_Compression(value) - found = true - break - } - } - if !found { - return unimplemented.Newf("import.compression", "unsupported compression value: %q", override) - } - } - - var tableDetails []jobspb.ImportDetails_Table - var tableDescs []*tabledesc.Mutable // parallel with tableDetails - var typeDetails []jobspb.ImportDetails_Type - jobDesc, err := importJobDescription(p, importStmt, nil, filenamePatterns, opts) - if err != nil { - return err - } - - if importStmt.Into { - if _, ok := allowedIntoFormats[importStmt.FileFormat]; !ok { - return errors.Newf( - "%s file format is currently unsupported by IMPORT INTO", - importStmt.FileFormat) - } - _, found, err := p.ResolveMutableTableDescriptor(ctx, table, true, tree.ResolveRequireTableDesc) - if err != nil { - return err - } - - err = ensureRequiredPrivileges(ctx, importIntoRequiredPrivileges, p, found) - if err != nil { - return err - } - - // IMPORT INTO does not currently support interleaved tables. - if found.IsInterleaved() { - // TODO(miretskiy): Handle import into when tables are interleaved. - return pgerror.New(pgcode.FeatureNotSupported, "Cannot use IMPORT INTO with interleaved tables") - } - - // Validate target columns. - var intoCols []string - var isTargetCol = make(map[string]bool) - for _, name := range importStmt.IntoCols { - active, err := tabledesc.FindPublicColumnsWithNames(found, tree.NameList{name}) - if err != nil { - return errors.Wrap(err, "verifying target columns") - } - - isTargetCol[active[0].GetName()] = true - intoCols = append(intoCols, active[0].GetName()) - } - - // Ensure that non-target columns that don't have default - // expressions are nullable. - if len(isTargetCol) != 0 { - for _, col := range found.VisibleColumns() { - if !(isTargetCol[col.GetName()] || col.IsNullable() || col.HasDefault() || col.IsComputed()) { - return errors.Newf( - "all non-target columns in IMPORT INTO must be nullable "+ - "or have default expressions, or have computed expressions"+ - " but violated by column %q", - col.GetName(), - ) - } - if isTargetCol[col.GetName()] && col.IsComputed() { - return schemaexpr.CannotWriteToComputedColError(col.GetName()) - } - } - } - - { - // Resolve the UDTs used by the table being imported into. - typeDescs, err := resolveUDTsUsedByImportInto(ctx, p, found) - if err != nil { - return errors.Wrap(err, "resolving UDTs used by table being imported into") - } - if len(typeDescs) > 0 { - typeDetails = make([]jobspb.ImportDetails_Type, 0, len(typeDescs)) - } - for _, typeDesc := range typeDescs { - typeDetails = append(typeDetails, jobspb.ImportDetails_Type{Desc: typeDesc.TypeDesc()}) - } - } - - tableDetails = []jobspb.ImportDetails_Table{{Desc: &found.TableDescriptor, IsNew: false, TargetCols: intoCols}} - } else { - seqVals := make(map[descpb.ID]int64) - - if importStmt.Bundle { - // If we target a single table, populate details with one entry of tableName. - if table != nil { - tableDetails = make([]jobspb.ImportDetails_Table, 1) - tableName := table.ObjectName.String() - // PGDUMP supports importing tables from non-public schemas, thus we - // must prepend the target table name with the target schema name. - if format.Format == roachpb.IOFileFormat_PgDump { - if table.Schema() == "" { - return errors.Newf("expected schema for target table %s to be resolved", - tableName) - } - tableName = fmt.Sprintf("%s.%s", table.SchemaName.String(), - table.ObjectName.String()) - } - tableDetails[0] = jobspb.ImportDetails_Table{ - Name: tableName, - IsNew: true, - } - } - } else { - if table == nil { - return errors.Errorf("non-bundle format %q should always have a table name", importStmt.FileFormat) - } - var create *tree.CreateTable - if importStmt.CreateDefs != nil { - create = &tree.CreateTable{ - Table: *importStmt.Table, - Defs: importStmt.CreateDefs, - } - } else { - filename, err := createFileFn() - if err != nil { - return err - } - create, err = readCreateTableFromStore(ctx, filename, - p.ExecCfg().DistSQLSrv.ExternalStorageFromURI, p.User()) - if err != nil { - return err - } - - if table.ObjectName != create.Table.ObjectName { - return errors.Errorf( - "importing table %s, but file specifies a schema for table %s", - table.ObjectName, create.Table.ObjectName, - ) - } - } - if create.Locality != nil && - create.Locality.LocalityLevel == tree.LocalityLevelRow { - return unimplemented.NewWithIssueDetailf( - 61133, - "import.regional-by-row", - "IMPORT to REGIONAL BY ROW table not supported", - ) - } - // IMPORT TABLE do not support user defined types, and so we nil out the - // type resolver to protect against unexpected behavior on UDT - // resolution. - semaCtxPtr := makeSemaCtxWithoutTypeResolver(p.SemaCtx()) - tbl, err := MakeSimpleTableDescriptor( - ctx, semaCtxPtr, p.ExecCfg().Settings, create, db, sc, defaultCSVTableID, NoFKs, walltime) - if err != nil { - return err - } - descStr, err := importJobDescription(p, importStmt, create.Defs, filenamePatterns, opts) - if err != nil { - return err - } - jobDesc = descStr - - tableDescs = []*tabledesc.Mutable{tbl} - for _, tbl := range tableDescs { - // For reasons relating to #37691, we disallow user defined types in - // the standard IMPORT case. - for _, col := range tbl.Columns { - if col.Type.UserDefined() { - return errors.Newf("IMPORT cannot be used with user defined types; use IMPORT INTO instead") - } - } - } - - tableDetails = make([]jobspb.ImportDetails_Table, len(tableDescs)) - for i := range tableDescs { - tableDetails[i] = jobspb.ImportDetails_Table{ - Desc: tableDescs[i].TableDesc(), - SeqVal: seqVals[tableDescs[i].ID], - IsNew: true, - } - } - } - - // Due to how we generate and rewrite descriptor ID's for import, we run - // into problems when using user defined schemas. - if sc.GetID() != keys.PublicSchemaID { - err := errors.New("cannot use IMPORT with a user defined schema") - hint := errors.WithHint(err, "create the table with CREATE TABLE and use IMPORT INTO instead") - return hint - } - } - - // Store the primary region of the database being imported into. This is - // used during job execution to evaluate certain default expressions and - // computed columns such as `gateway_region`. - var databasePrimaryRegion descpb.RegionName - if db.IsMultiRegion() { - if err := sql.DescsTxn(ctx, p.ExecCfg(), func(ctx context.Context, txn *kv.Txn, - descsCol *descs.Collection) error { - regionConfig, err := sql.SynthesizeRegionConfig(ctx, txn, db.GetID(), descsCol) - if err != nil { - return err - } - databasePrimaryRegion = regionConfig.PrimaryRegion() - return nil - }); err != nil { - return errors.Wrap(err, "failed to resolve region config for multi region database") - } - } - - telemetry.CountBucketed("import.files", int64(len(files))) - - // Record telemetry for userfile being used as the import target. - for _, file := range files { - uri, err := url.Parse(file) - // This should never be true as we have parsed these file names in an - // earlier step of import. - if err != nil { - log.Warningf(ctx, "failed to collect file specific import telemetry for %s", uri) - continue - } - - if uri.Scheme == "userfile" { - telemetry.Count("import.storage.userfile") - break - } - } - if importStmt.Into { - telemetry.Count("import.into") - } - - // Here we create the job in a side transaction and then kick off the job. - // This is awful. Rather we should be disallowing this statement in an - // explicit transaction and then we should create the job in the user's - // transaction here and then in a post-commit hook we should kick of the - // StartableJob which we attached to the connExecutor somehow. - - importDetails := jobspb.ImportDetails{ - URIs: files, - Format: format, - ParentID: db.GetID(), - Tables: tableDetails, - Types: typeDetails, - SSTSize: sstSize, - Oversample: oversample, - SkipFKs: skipFKs, - ParseBundleSchema: importStmt.Bundle, - DefaultIntSize: p.SessionData().DefaultIntSize, - DatabasePrimaryRegion: databasePrimaryRegion, - } - - jr := jobs.Record{ - Description: jobDesc, - Username: p.User(), - Details: importDetails, - Progress: jobspb.ImportProgress{}, - } - - if isDetached { - // When running inside an explicit transaction, we simply create the job - // record. We do not wait for the job to finish. - jobID := p.ExecCfg().JobRegistry.MakeJobID() - _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( - ctx, jr, jobID, p.ExtendedEvalContext().Txn) - if err != nil { - return err - } - - addToFileFormatTelemetry(format.Format.String(), "started") - resultsCh <- tree.Datums{tree.NewDInt(tree.DInt(jobID))} - return nil - } - - // We create the job record in the planner's transaction to ensure that - // the job record creation happens transactionally. - plannerTxn := p.ExtendedEvalContext().Txn - - // Construct the job and commit the transaction. Perform this work in a - // closure to ensure that the job is cleaned up if an error occurs. - var sj *jobs.StartableJob - if err := func() (err error) { - defer func() { - if err == nil || sj == nil { - return - } - if cleanupErr := sj.CleanupOnRollback(ctx); cleanupErr != nil { - log.Errorf(ctx, "failed to cleanup job: %v", cleanupErr) - } - }() - jobID := p.ExecCfg().JobRegistry.MakeJobID() - if err := p.ExecCfg().JobRegistry.CreateStartableJobWithTxn(ctx, &sj, jobID, plannerTxn, jr); err != nil { - return err - } - - // We commit the transaction here so that the job can be started. This - // is safe because we're in an implicit transaction. If we were in an - // explicit transaction the job would have to be run with the detached - // option and would have been handled above. - return plannerTxn.Commit(ctx) - }(); err != nil { - return err - } - - if err := sj.Start(ctx); err != nil { - return err - } - addToFileFormatTelemetry(format.Format.String(), "started") - if err := sj.AwaitCompletion(ctx); err != nil { - return err - } - return sj.ReportExecutionResults(ctx, resultsCh) - } - - if isDetached { - return fn, utilccl.DetachedJobExecutionResultHeader, nil, false, nil - } - return fn, utilccl.BulkJobExecutionResultHeader, nil, false, nil -} - -func parseAvroOptions( - ctx context.Context, opts map[string]string, p sql.PlanHookState, format *roachpb.IOFileFormat, -) error { - format.Format = roachpb.IOFileFormat_Avro - // Default input format is OCF. - format.Avro.Format = roachpb.AvroOptions_OCF - _, format.Avro.StrictMode = opts[avroStrict] - - _, haveBinRecs := opts[avroBinRecords] - _, haveJSONRecs := opts[avroJSONRecords] - - if haveBinRecs && haveJSONRecs { - return errors.Errorf("only one of the %s or %s options can be set", avroBinRecords, avroJSONRecords) - } - - if override, ok := opts[csvRowLimit]; ok { - rowLimit, err := strconv.Atoi(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) - } - if rowLimit <= 0 { - return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) - } - format.Avro.RowLimit = int64(rowLimit) - } - - if haveBinRecs || haveJSONRecs { - // Input is a "records" format. - if haveBinRecs { - format.Avro.Format = roachpb.AvroOptions_BIN_RECORDS - } else { - format.Avro.Format = roachpb.AvroOptions_JSON_RECORDS - } - - // Set record separator. - format.Avro.RecordSeparator = '\n' - if override, ok := opts[avroRecordsSeparatedBy]; ok { - c, err := util.GetSingleRune(override) - if err != nil { - return pgerror.Wrapf(err, pgcode.Syntax, - "invalid %q value", avroRecordsSeparatedBy) - } - format.Avro.RecordSeparator = c - } - - // See if inline schema is specified. - format.Avro.SchemaJSON = opts[avroSchema] - - if len(format.Avro.SchemaJSON) == 0 { - // Inline schema not set; We must have external schema. - uri, ok := opts[avroSchemaURI] - if !ok { - return errors.Errorf( - "either %s or %s option must be set when importing avro record files", avroSchema, avroSchemaURI) - } - - store, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, uri, p.User()) - if err != nil { - return err - } - defer store.Close() - - raw, err := store.ReadFile(ctx, "") - if err != nil { - return err - } - defer raw.Close() - schemaBytes, err := ioutil.ReadAll(raw) - if err != nil { - return err - } - format.Avro.SchemaJSON = string(schemaBytes) - } - - if override, ok := opts[optMaxRowSize]; ok { - sz, err := humanizeutil.ParseBytes(override) - if err != nil { - return err - } - if sz < 1 || sz > math.MaxInt32 { - return errors.Errorf("%s out of range: %d", override, sz) - } - format.Avro.MaxRecordSize = int32(sz) - } - } - return nil -} - -type importResumer struct { - job *jobs.Job - settings *cluster.Settings - res backupccl.RowCount - - testingKnobs struct { - afterImport func(summary backupccl.RowCount) error - alwaysFlushJobProgress bool - } -} - -func (r *importResumer) ForceRealSpan() bool { - return true -} - -var _ jobs.TraceableJob = &importResumer{} - -// Prepares descriptors for newly created tables being imported into. -func prepareNewTableDescsForIngestion( - ctx context.Context, - txn *kv.Txn, - descsCol *descs.Collection, - p sql.JobExecContext, - importTables []jobspb.ImportDetails_Table, - parentID descpb.ID, - schemaRewrites backupccl.DescRewriteMap, -) ([]*descpb.TableDescriptor, error) { - newMutableTableDescriptors := make([]*tabledesc.Mutable, len(importTables)) - for i := range importTables { - newMutableTableDescriptors[i] = tabledesc.NewBuilder(importTables[i].Desc).BuildCreatedMutableTable() - } - - // Verification steps have passed, generate a new table ID if we're - // restoring. We do this last because we want to avoid calling - // GenerateUniqueDescID if there's any kind of error above. - // Reserving a table ID now means we can avoid the rekey work during restore. - // - // schemaRewrites may contain information which is used in RewriteTableDescs - // to rewrite the parent schema ID in the table desc to point to the correct - // schema ID. - tableRewrites := schemaRewrites - if tableRewrites == nil { - tableRewrites = make(backupccl.DescRewriteMap) - } - seqVals := make(map[descpb.ID]int64, len(importTables)) - for _, tableDesc := range importTables { - id, err := catalogkv.GenerateUniqueDescID(ctx, p.ExecCfg().DB, p.ExecCfg().Codec) - if err != nil { - return nil, err - } - tableRewrites[tableDesc.Desc.ID] = &jobspb.RestoreDetails_DescriptorRewrite{ - ID: id, - ParentID: parentID, - } - seqVals[id] = tableDesc.SeqVal - } - if err := backupccl.RewriteTableDescs( - newMutableTableDescriptors, tableRewrites, "", - ); err != nil { - return nil, err - } - - // After all of the ID's have been remapped, ensure that there aren't any name - // collisions with any importing tables. - for i := range newMutableTableDescriptors { - tbl := newMutableTableDescriptors[i] - err := catalogkv.CheckObjectCollision( - ctx, - txn, - p.ExecCfg().Codec, - tbl.GetParentID(), - tbl.GetParentSchemaID(), - tree.NewUnqualifiedTableName(tree.Name(tbl.GetName())), - ) - if err != nil { - return nil, err - } - } - - // tableDescs contains the same slice as newMutableTableDescriptors but - // as tabledesc.TableDescriptor. - tableDescs := make([]catalog.TableDescriptor, len(newMutableTableDescriptors)) - for i := range tableDescs { - newMutableTableDescriptors[i].SetOffline("importing") - tableDescs[i] = newMutableTableDescriptors[i] - } - - var seqValKVs []roachpb.KeyValue - for _, desc := range newMutableTableDescriptors { - if v, ok := seqVals[desc.GetID()]; ok && v != 0 { - key, val, err := sql.MakeSequenceKeyVal(p.ExecCfg().Codec, desc, v, false) - if err != nil { - return nil, 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.WriteDescriptors(ctx, p.ExecCfg().Codec, txn, p.User(), descsCol, - nil /* databases */, nil, /* schemas */ - tableDescs, nil, tree.RequestedDescriptors, seqValKVs); err != nil { - return nil, errors.Wrapf(err, "creating importTables") - } - - newPreparedTableDescs := make([]*descpb.TableDescriptor, len(newMutableTableDescriptors)) - for i := range newMutableTableDescriptors { - newPreparedTableDescs[i] = newMutableTableDescriptors[i].TableDesc() - } - - return newPreparedTableDescs, nil -} - -// Prepares descriptors for existing tables being imported into. -func prepareExistingTableDescForIngestion( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, desc *descpb.TableDescriptor, -) (*descpb.TableDescriptor, error) { - if len(desc.Mutations) > 0 { - return nil, errors.Errorf("cannot IMPORT INTO a table with schema changes in progress -- try again later (pending mutation %s)", desc.Mutations[0].String()) - } - - // Note that desc is just used to verify that the version matches. - importing, err := descsCol.GetMutableTableVersionByID(ctx, desc.ID, txn) - if err != nil { - return nil, err - } - // Ensure that the version of the table has not been modified since this - // job was created. - if got, exp := importing.Version, desc.Version; got != exp { - return nil, errors.Errorf("another operation is currently operating on the table") - } - - // Take the table offline for import. - // TODO(dt): audit everywhere we get table descs (leases or otherwise) to - // ensure that filtering by state handles IMPORTING correctly. - importing.SetOffline("importing") - - // TODO(dt): de-validate all the FKs. - if err := descsCol.WriteDesc( - ctx, false /* kvTrace */, importing, txn, - ); err != nil { - return nil, err - } - - return importing.TableDesc(), nil -} - -func createNonDropDatabaseChangeJob( - user security.SQLUsername, - databaseID descpb.ID, - jobDesc string, - p sql.JobExecContext, - txn *kv.Txn, -) (*jobs.Job, error) { - jobRecord := jobs.Record{ - Description: jobDesc, - Username: user, - Details: jobspb.SchemaChangeDetails{ - DescID: databaseID, - FormatVersion: jobspb.DatabaseJobFormatVersion, - }, - Progress: jobspb.SchemaChangeProgress{}, - NonCancelable: true, - } - - jobID := p.ExecCfg().JobRegistry.MakeJobID() - return p.ExecCfg().JobRegistry.CreateJobWithTxn( - p.ExtendedEvalContext().Context, - jobRecord, - jobID, - txn, - ) -} - -func writeNonDropDatabaseChange( - ctx context.Context, - desc *dbdesc.Mutable, - txn *kv.Txn, - descsCol *descs.Collection, - p sql.JobExecContext, - jobDesc string, -) ([]jobspb.JobID, error) { - var job *jobs.Job - var err error - if job, err = createNonDropDatabaseChangeJob(p.User(), desc.ID, jobDesc, p, txn); err != nil { - return nil, err - } - - queuedJob := []jobspb.JobID{job.ID()} - b := txn.NewBatch() - err = descsCol.WriteDescToBatch( - ctx, - p.ExtendedEvalContext().Tracing.KVTracingEnabled(), - desc, - b, - ) - if err != nil { - return nil, err - } - return queuedJob, txn.Run(ctx, b) -} - -func createSchemaDescriptorWithID( - ctx context.Context, - idKey roachpb.Key, - id descpb.ID, - descriptor catalog.Descriptor, - p sql.JobExecContext, - descsCol *descs.Collection, - txn *kv.Txn, -) error { - if descriptor.GetID() == descpb.InvalidID { - return errors.AssertionFailedf("cannot create descriptor with an empty ID: %v", descriptor) - } - if descriptor.GetID() != id { - return errors.AssertionFailedf("cannot create descriptor with an ID %v; expected ID %v; descriptor %v", - id, descriptor.GetID(), descriptor) - } - b := &kv.Batch{} - descID := descriptor.GetID() - if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { - log.VEventf(ctx, 2, "CPut %s -> %d", idKey, descID) - } - b.CPut(idKey, descID, nil) - if err := catalogkv.WriteNewDescToBatch( - ctx, - p.ExtendedEvalContext().Tracing.KVTracingEnabled(), - p.ExecCfg().Settings, - b, - p.ExecCfg().Codec, - descID, - descriptor, - ); err != nil { - return err - } - - mutDesc, ok := descriptor.(catalog.MutableDescriptor) - if !ok { - return errors.Newf("unexpected type %T when creating descriptor", descriptor) - } - switch mutDesc.(type) { - case *schemadesc.Mutable: - if err := descsCol.AddUncommittedDescriptor(mutDesc); err != nil { - return err - } - default: - return errors.Newf("unexpected type %T when creating descriptor", mutDesc) - } - - return txn.Run(ctx, b) -} - -// prepareSchemasForIngestion is responsible for assigning the created schema -// descriptors actual IDs, updating the parent DB with references to the new -// schemas and writing the schema descriptors to disk. -func (r *importResumer) prepareSchemasForIngestion( - ctx context.Context, - p sql.JobExecContext, - details jobspb.ImportDetails, - txn *kv.Txn, - descsCol *descs.Collection, -) (*preparedSchemaMetadata, error) { - schemaMetadata := &preparedSchemaMetadata{ - schemaPreparedDetails: details, - newSchemaIDToName: make(map[descpb.ID]string), - oldSchemaIDToName: make(map[descpb.ID]string), - } - - schemaMetadata.schemaPreparedDetails.Schemas = make([]jobspb.ImportDetails_Schema, - len(details.Schemas)) - - desc, err := descsCol.GetMutableDescriptorByID(ctx, details.ParentID, txn) - if err != nil { - return nil, err - } - - dbDesc, ok := desc.(*dbdesc.Mutable) - if !ok { - return nil, errors.Newf("expected ID %d to refer to the database being imported into", - details.ParentID) - } - - if dbDesc.Schemas == nil { - dbDesc.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) - } - - schemaMetadata.schemaRewrites = make(backupccl.DescRewriteMap) - mutableSchemaDescs := make([]*schemadesc.Mutable, 0) - for _, desc := range details.Schemas { - schemaMetadata.oldSchemaIDToName[desc.Desc.GetID()] = desc.Desc.GetName() - newMutableSchemaDescriptor := schemadesc.NewBuilder(desc.Desc).BuildCreatedMutable().(*schemadesc.Mutable) - - // Verification steps have passed, generate a new schema ID. We do this - // last because we want to avoid calling GenerateUniqueDescID if there's - // any kind of error in the prior stages of import. - id, err := catalogkv.GenerateUniqueDescID(ctx, p.ExecCfg().DB, p.ExecCfg().Codec) - if err != nil { - return nil, err - } - newMutableSchemaDescriptor.Version = 1 - newMutableSchemaDescriptor.ID = id - mutableSchemaDescs = append(mutableSchemaDescs, newMutableSchemaDescriptor) - - schemaMetadata.newSchemaIDToName[id] = newMutableSchemaDescriptor.GetName() - - // Update the parent database with this schema information. - dbDesc.Schemas[newMutableSchemaDescriptor.Name] = - descpb.DatabaseDescriptor_SchemaInfo{ID: newMutableSchemaDescriptor.ID, Dropped: false} - - schemaMetadata.schemaRewrites[desc.Desc.ID] = &jobspb.RestoreDetails_DescriptorRewrite{ - ID: id, - } - } - - // Queue a job to write the updated database descriptor. - schemaMetadata.queuedSchemaJobs, err = writeNonDropDatabaseChange(ctx, dbDesc, txn, descsCol, p, - fmt.Sprintf("updating parent database %s when importing new schemas", dbDesc.GetName())) - if err != nil { - return nil, err - } - - // Finally create the schemas on disk. - for i, mutDesc := range mutableSchemaDescs { - nameKey := catalogkeys.MakeSchemaNameKey(p.ExecCfg().Codec, dbDesc.ID, mutDesc.GetName()) - err = createSchemaDescriptorWithID(ctx, nameKey, mutDesc.ID, mutDesc, p, descsCol, txn) - if err != nil { - return nil, err - } - schemaMetadata.schemaPreparedDetails.Schemas[i] = jobspb.ImportDetails_Schema{ - Desc: mutDesc.SchemaDesc(), - } + sec := int64(timeutil.Since(timeutil.FromUnixMicros(r.job.Payload().StartedMicros)).Seconds()) + var mbps int64 + if sec > 0 { + mbps = mb / sec } - - return schemaMetadata, err -} - -func constructSchemaAndTableKey( - tableDesc *descpb.TableDescriptor, schemaIDToName map[descpb.ID]string, -) (schemaAndTableName, error) { - schemaName, ok := schemaIDToName[tableDesc.GetUnexposedParentSchemaID()] - if !ok && tableDesc.UnexposedParentSchemaID != keys.PublicSchemaID { - return schemaAndTableName{}, errors.Newf("invalid parent schema ID %d for table %s", - tableDesc.UnexposedParentSchemaID, tableDesc.GetName()) + telemetry.CountBucketed("import.duration-sec.succeeded", sec) + telemetry.CountBucketed("import.speed-mbps", mbps) + // Tiny imports may skew throughput numbers due to overhead. + if sizeMb > 10 { + telemetry.CountBucketed("import.speed-mbps.over10mb", mbps) } - return schemaAndTableName{schema: schemaName, table: tableDesc.GetName()}, nil + return nil } -// prepareTableDescsForIngestion prepares table descriptors for the ingestion +// prepareTablesForIngestion prepares table descriptors for the ingestion // step of import. The descriptors are in an IMPORTING state (offline) on // successful completion of this method. -func (r *importResumer) prepareTableDescsForIngestion( +func (r *importResumer) prepareTablesForIngestion( ctx context.Context, p sql.JobExecContext, details jobspb.ImportDetails, @@ -1564,7 +338,7 @@ func (r *importResumer) prepareTableDescsForIngestion( var desc *descpb.TableDescriptor for i, table := range details.Tables { if !table.IsNew { - desc, err = prepareExistingTableDescForIngestion(ctx, txn, descsCol, table.Desc) + desc, err = prepareExistingTablesForIngestion(ctx, txn, descsCol, table.Desc) if err != nil { return importDetails, err } @@ -1598,10 +372,10 @@ func (r *importResumer) prepareTableDescsForIngestion( // TODO(adityamaru): This is still unnecessarily complicated. If we can get // the new table desc preparation to work on a per desc basis, rather than // requiring all the newly created descriptors, then this can look like the - // call to prepareExistingTableDescForIngestion. Currently, FK references + // call to prepareExistingTablesForIngestion. Currently, FK references // misbehave when I tried to write the desc one at a time. if len(newTableDescs) != 0 { - res, err := prepareNewTableDescsForIngestion( + res, err := prepareNewTablesForIngestion( ctx, txn, descsCol, p, newTableDescs, importDetails.ParentID, schemaMetadata.schemaRewrites) if err != nil { return importDetails, err @@ -1636,220 +410,281 @@ func (r *importResumer) prepareTableDescsForIngestion( return importDetails, nil } -// ReportResults implements JobResultsReporter interface. -func (r *importResumer) ReportResults(ctx context.Context, resultsCh chan<- tree.Datums) error { - select { - case resultsCh <- tree.Datums{ - tree.NewDInt(tree.DInt(r.job.ID())), - tree.NewDString(string(jobs.StatusSucceeded)), - tree.NewDFloat(tree.DFloat(1.0)), - tree.NewDInt(tree.DInt(r.res.Rows)), - tree.NewDInt(tree.DInt(r.res.IndexEntries)), - tree.NewDInt(tree.DInt(r.res.DataSize)), - }: - return nil - case <-ctx.Done(): - return ctx.Err() +// prepareExistingTablesForIngestion prepares descriptors for existing tables +// being imported into. +func prepareExistingTablesForIngestion( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, desc *descpb.TableDescriptor, +) (*descpb.TableDescriptor, error) { + if len(desc.Mutations) > 0 { + return nil, errors.Errorf("cannot IMPORT INTO a table with schema changes in progress -- try again later (pending mutation %s)", desc.Mutations[0].String()) } -} - -type loggerKind int - -const ( - schemaParsing loggerKind = iota - dataIngestion -) - -// unsupportedStmtLogger is responsible for handling unsupported PGDUMP SQL -// statements seen during the import. -type unsupportedStmtLogger struct { - ctx context.Context - user security.SQLUsername - jobID int64 - // Values are initialized based on the options specified in the IMPORT PGDUMP - // stmt. - ignoreUnsupported bool - ignoreUnsupportedLogDest string - externalStorage cloud.ExternalStorageFactory + // Note that desc is just used to verify that the version matches. + importing, err := descsCol.GetMutableTableVersionByID(ctx, desc.ID, txn) + if err != nil { + return nil, err + } + // Ensure that the version of the table has not been modified since this + // job was created. + if got, exp := importing.Version, desc.Version; got != exp { + return nil, errors.Errorf("another operation is currently operating on the table") + } - // logBuffer holds the string to be flushed to the ignoreUnsupportedLogDest. - logBuffer *bytes.Buffer - numIgnoredStmts int + // Take the table offline for import. + // TODO(dt): audit everywhere we get table descs (leases or otherwise) to + // ensure that filtering by state handles IMPORTING correctly. + importing.SetOffline("importing") - // Incremented every time the logger flushes. It is used as the suffix of the - // log file written to external storage. - flushCount int + // TODO(dt): de-validate all the FKs. + if err := descsCol.WriteDesc( + ctx, false /* kvTrace */, importing, txn, + ); err != nil { + return nil, err + } - loggerType loggerKind + return importing.TableDesc(), nil } -func makeUnsupportedStmtLogger( +// prepareNewTablesForIngestion prepares descriptors for newly created +// tables being imported into. +func prepareNewTablesForIngestion( ctx context.Context, - user security.SQLUsername, - jobID int64, - ignoreUnsupported bool, - unsupportedLogDest string, - loggerType loggerKind, - externalStorage cloud.ExternalStorageFactory, -) *unsupportedStmtLogger { - return &unsupportedStmtLogger{ - ctx: ctx, - user: user, - jobID: jobID, - ignoreUnsupported: ignoreUnsupported, - ignoreUnsupportedLogDest: unsupportedLogDest, - loggerType: loggerType, - logBuffer: new(bytes.Buffer), - externalStorage: externalStorage, + txn *kv.Txn, + descsCol *descs.Collection, + p sql.JobExecContext, + importTables []jobspb.ImportDetails_Table, + parentID descpb.ID, + schemaRewrites backupccl.DescRewriteMap, +) ([]*descpb.TableDescriptor, error) { + newMutableTableDescriptors := make([]*tabledesc.Mutable, len(importTables)) + for i := range importTables { + newMutableTableDescriptors[i] = tabledesc.NewBuilder(importTables[i].Desc).BuildCreatedMutableTable() } -} -func (u *unsupportedStmtLogger) log(logLine string, isParseError bool) error { - // We have already logged parse errors during the schema ingestion phase, so - // skip them to avoid duplicate entries. - skipLoggingParseErr := isParseError && u.loggerType == dataIngestion - if u.ignoreUnsupportedLogDest == "" || skipLoggingParseErr { - return nil + // Verification steps have passed, generate a new table ID if we're + // restoring. We do this last because we want to avoid calling + // GenerateUniqueDescID if there's any kind of error above. + // Reserving a table ID now means we can avoid the rekey work during restore. + // + // schemaRewrites may contain information which is used in RewriteTableDescs + // to rewrite the parent schema ID in the table desc to point to the correct + // schema ID. + tableRewrites := schemaRewrites + if tableRewrites == nil { + tableRewrites = make(backupccl.DescRewriteMap) } - - // Flush to a file if we have hit the max size of our buffer. - if u.numIgnoredStmts >= pgDumpMaxLoggedStmts { - err := u.flush() + seqVals := make(map[descpb.ID]int64, len(importTables)) + for _, tableDesc := range importTables { + id, err := catalogkv.GenerateUniqueDescID(ctx, p.ExecCfg().DB, p.ExecCfg().Codec) if err != nil { - return err + return nil, err + } + tableRewrites[tableDesc.Desc.ID] = &jobspb.RestoreDetails_DescriptorRewrite{ + ID: id, + ParentID: parentID, } + seqVals[id] = tableDesc.SeqVal } - - if isParseError { - logLine = fmt.Sprintf("%s: could not be parsed\n", logLine) - } else { - logLine = fmt.Sprintf("%s: unsupported by IMPORT\n", logLine) + if err := backupccl.RewriteTableDescs( + newMutableTableDescriptors, tableRewrites, "", + ); err != nil { + return nil, err } - u.logBuffer.Write([]byte(logLine)) - u.numIgnoredStmts++ - return nil -} -func (u *unsupportedStmtLogger) flush() error { - if u.ignoreUnsupportedLogDest == "" { - return nil + // After all of the ID's have been remapped, ensure that there aren't any name + // collisions with any importing tables. + for i := range newMutableTableDescriptors { + tbl := newMutableTableDescriptors[i] + err := catalogkv.CheckObjectCollision( + ctx, + txn, + p.ExecCfg().Codec, + tbl.GetParentID(), + tbl.GetParentSchemaID(), + tree.NewUnqualifiedTableName(tree.Name(tbl.GetName())), + ) + if err != nil { + return nil, err + } } - conf, err := cloud.ExternalStorageConfFromURI(u.ignoreUnsupportedLogDest, u.user) - if err != nil { - return errors.Wrap(err, "failed to log unsupported stmts during IMPORT PGDUMP") + // tableDescs contains the same slice as newMutableTableDescriptors but + // as tabledesc.TableDescriptor. + tableDescs := make([]catalog.TableDescriptor, len(newMutableTableDescriptors)) + for i := range tableDescs { + newMutableTableDescriptors[i].SetOffline("importing") + tableDescs[i] = newMutableTableDescriptors[i] } - var s cloud.ExternalStorage - if s, err = u.externalStorage(u.ctx, conf); err != nil { - return errors.New("failed to log unsupported stmts during IMPORT PGDUMP") + + var seqValKVs []roachpb.KeyValue + for _, desc := range newMutableTableDescriptors { + if v, ok := seqVals[desc.GetID()]; ok && v != 0 { + key, val, err := sql.MakeSequenceKeyVal(p.ExecCfg().Codec, desc, v, false) + if err != nil { + return nil, err + } + kv := roachpb.KeyValue{Key: key} + kv.Value.SetInt(val) + seqValKVs = append(seqValKVs, kv) + } } - defer s.Close() - logFileName := fmt.Sprintf("import%d", u.jobID) - if u.loggerType == dataIngestion { - logFileName = path.Join(logFileName, pgDumpUnsupportedDataStmtLog, fmt.Sprintf("%d.log", u.flushCount)) - } else { - logFileName = path.Join(logFileName, pgDumpUnsupportedSchemaStmtLog, fmt.Sprintf("%d.log", u.flushCount)) + // 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.WriteDescriptors(ctx, p.ExecCfg().Codec, txn, p.User(), descsCol, + nil /* databases */, nil, /* schemas */ + tableDescs, nil, tree.RequestedDescriptors, seqValKVs); err != nil { + return nil, errors.Wrapf(err, "creating importTables") } - err = cloud.WriteFile(u.ctx, s, logFileName, bytes.NewReader(u.logBuffer.Bytes())) - if err != nil { - return errors.Wrap(err, "failed to log unsupported stmts to log during IMPORT PGDUMP") + + newPreparedTableDescs := make([]*descpb.TableDescriptor, len(newMutableTableDescriptors)) + for i := range newMutableTableDescriptors { + newPreparedTableDescs[i] = newMutableTableDescriptors[i].TableDesc() } - u.flushCount++ - u.numIgnoredStmts = 0 - u.logBuffer.Truncate(0) - return nil + + return newPreparedTableDescs, nil } -// parseAndCreateBundleTableDescs parses and creates the table -// descriptors for bundle formats. -func parseAndCreateBundleTableDescs( +// prepareSchemasForIngestion is responsible for assigning the created schema +// descriptors actual IDs, updating the parent DB with references to the new +// schemas and writing the schema descriptors to disk. +func (r *importResumer) prepareSchemasForIngestion( ctx context.Context, p sql.JobExecContext, details jobspb.ImportDetails, - seqVals map[descpb.ID]int64, - skipFKs bool, - parentDB catalog.DatabaseDescriptor, - files []string, - format roachpb.IOFileFormat, - walltime int64, - owner security.SQLUsername, - jobID jobspb.JobID, -) ([]*tabledesc.Mutable, []*schemadesc.Mutable, error) { - - var schemaDescs []*schemadesc.Mutable - var tableDescs []*tabledesc.Mutable - var tableName string - - // A single table entry in the import job details when importing a bundle format - // indicates that we are performing a single table import. - // This info is populated during the planning phase. - if len(details.Tables) > 0 { - tableName = details.Tables[0].Name + txn *kv.Txn, + descsCol *descs.Collection, +) (*preparedSchemaMetadata, error) { + schemaMetadata := &preparedSchemaMetadata{ + schemaPreparedDetails: details, + newSchemaIDToName: make(map[descpb.ID]string), + oldSchemaIDToName: make(map[descpb.ID]string), } - store, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, files[0], p.User()) + schemaMetadata.schemaPreparedDetails.Schemas = make([]jobspb.ImportDetails_Schema, + len(details.Schemas)) + + desc, err := descsCol.GetMutableDescriptorByID(ctx, details.ParentID, txn) if err != nil { - return tableDescs, schemaDescs, err + return nil, err } - defer store.Close() - raw, err := store.ReadFile(ctx, "") - if err != nil { - return tableDescs, schemaDescs, err + dbDesc, ok := desc.(*dbdesc.Mutable) + if !ok { + return nil, errors.Newf("expected ID %d to refer to the database being imported into", + details.ParentID) } - defer raw.Close() - reader, err := decompressingReader(raw, files[0], format.Compression) - if err != nil { - return tableDescs, schemaDescs, err + + if dbDesc.Schemas == nil { + dbDesc.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) } - defer reader.Close() - fks := fkHandler{skip: skipFKs, allowed: true, resolver: fkResolver{ - tableNameToDesc: make(map[string]*tabledesc.Mutable), - }} - switch format.Format { - case roachpb.IOFileFormat_Mysqldump: - fks.resolver.format.Format = roachpb.IOFileFormat_Mysqldump - evalCtx := &p.ExtendedEvalContext().EvalContext - tableDescs, err = readMysqlCreateTable( - ctx, reader, evalCtx, p, defaultCSVTableID, parentDB, tableName, fks, - seqVals, owner, walltime, - ) - case roachpb.IOFileFormat_PgDump: - fks.resolver.format.Format = roachpb.IOFileFormat_PgDump - evalCtx := &p.ExtendedEvalContext().EvalContext + schemaMetadata.schemaRewrites = make(backupccl.DescRewriteMap) + mutableSchemaDescs := make([]*schemadesc.Mutable, 0) + for _, desc := range details.Schemas { + schemaMetadata.oldSchemaIDToName[desc.Desc.GetID()] = desc.Desc.GetName() + newMutableSchemaDescriptor := schemadesc.NewBuilder(desc.Desc).BuildCreatedMutable().(*schemadesc.Mutable) - // Setup a logger to handle unsupported DDL statements in the PGDUMP file. - unsupportedStmtLogger := makeUnsupportedStmtLogger(ctx, p.User(), int64(jobID), - format.PgDump.IgnoreUnsupported, format.PgDump.IgnoreUnsupportedLog, schemaParsing, - p.ExecCfg().DistSQLSrv.ExternalStorage) + // Verification steps have passed, generate a new schema ID. We do this + // last because we want to avoid calling GenerateUniqueDescID if there's + // any kind of error in the prior stages of import. + id, err := catalogkv.GenerateUniqueDescID(ctx, p.ExecCfg().DB, p.ExecCfg().Codec) + if err != nil { + return nil, err + } + newMutableSchemaDescriptor.Version = 1 + newMutableSchemaDescriptor.ID = id + mutableSchemaDescs = append(mutableSchemaDescs, newMutableSchemaDescriptor) - tableDescs, schemaDescs, err = readPostgresCreateTable(ctx, reader, evalCtx, p, tableName, - parentDB, walltime, fks, int(format.PgDump.MaxRowSize), owner, unsupportedStmtLogger) + schemaMetadata.newSchemaIDToName[id] = newMutableSchemaDescriptor.GetName() + + // Update the parent database with this schema information. + dbDesc.Schemas[newMutableSchemaDescriptor.Name] = + descpb.DatabaseDescriptor_SchemaInfo{ID: newMutableSchemaDescriptor.ID, Dropped: false} - logErr := unsupportedStmtLogger.flush() - if logErr != nil { - return nil, nil, logErr + schemaMetadata.schemaRewrites[desc.Desc.ID] = &jobspb.RestoreDetails_DescriptorRewrite{ + ID: id, } - - default: - return tableDescs, schemaDescs, errors.Errorf( - "non-bundle format %q does not support reading schemas", format.Format.String()) } + // Queue a job to write the updated database descriptor. + schemaMetadata.queuedSchemaJobs, err = writeNonDropDatabaseChange(ctx, dbDesc, txn, descsCol, p, + fmt.Sprintf("updating parent database %s when importing new schemas", dbDesc.GetName())) if err != nil { - return tableDescs, schemaDescs, err + return nil, err } - if tableDescs == nil && len(details.Tables) > 0 { - return tableDescs, schemaDescs, errors.Errorf("table definition not found for %q", tableName) + // Finally create the schemas on disk. + for i, mutDesc := range mutableSchemaDescs { + nameKey := catalogkeys.MakeSchemaNameKey(p.ExecCfg().Codec, dbDesc.ID, mutDesc.GetName()) + err = createSchemaDescriptorWithID(ctx, nameKey, mutDesc.ID, mutDesc, p, descsCol, txn) + if err != nil { + return nil, err + } + schemaMetadata.schemaPreparedDetails.Schemas[i] = jobspb.ImportDetails_Schema{ + Desc: mutDesc.SchemaDesc(), + } } - return tableDescs, schemaDescs, err + return schemaMetadata, err +} + +// createSchemaDescriptorWithID writes a schema descriptor with `id` to disk. +func createSchemaDescriptorWithID( + ctx context.Context, + idKey roachpb.Key, + id descpb.ID, + descriptor catalog.Descriptor, + p sql.JobExecContext, + descsCol *descs.Collection, + txn *kv.Txn, +) error { + if descriptor.GetID() == descpb.InvalidID { + return errors.AssertionFailedf("cannot create descriptor with an empty ID: %v", descriptor) + } + if descriptor.GetID() != id { + return errors.AssertionFailedf("cannot create descriptor with an ID %v; expected ID %v; descriptor %v", + id, descriptor.GetID(), descriptor) + } + b := &kv.Batch{} + descID := descriptor.GetID() + if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { + log.VEventf(ctx, 2, "CPut %s -> %d", idKey, descID) + } + b.CPut(idKey, descID, nil) + if err := catalogkv.WriteNewDescToBatch( + ctx, + p.ExtendedEvalContext().Tracing.KVTracingEnabled(), + p.ExecCfg().Settings, + b, + p.ExecCfg().Codec, + descID, + descriptor, + ); err != nil { + return err + } + + mutDesc, ok := descriptor.(catalog.MutableDescriptor) + if !ok { + return errors.Newf("unexpected type %T when creating descriptor", descriptor) + } + switch mutDesc.(type) { + case *schemadesc.Mutable: + if err := descsCol.AddUncommittedDescriptor(mutDesc); err != nil { + return err + } + default: + return errors.Newf("unexpected type %T when creating descriptor", mutDesc) + } + + return txn.Run(ctx, b) } +// parseBundleSchemaIfNeeded parses dump files (PGDUMP, MYSQLDUMP) for DDL +// statements and creates the relevant database, schema, table and type +// descriptors. Data from the dump files is ingested into these descriptors in +// the next phase of the import. func (r *importResumer) parseBundleSchemaIfNeeded(ctx context.Context, phs interface{}) error { p := phs.(sql.JobExecContext) seqVals := make(map[descpb.ID]int64) @@ -1936,331 +771,217 @@ func (r *importResumer) parseBundleSchemaIfNeeded(ctx context.Context, phs inter return nil } -type preparedSchemaMetadata struct { - schemaPreparedDetails jobspb.ImportDetails - schemaRewrites backupccl.DescRewriteMap - newSchemaIDToName map[descpb.ID]string - oldSchemaIDToName map[descpb.ID]string - queuedSchemaJobs []jobspb.JobID -} - -func emitImportJobEvent( - ctx context.Context, p sql.JobExecContext, status jobs.Status, job *jobs.Job, -) { - // Emit to the event log now that we have completed the prepare step. - var importEvent eventpb.Import - if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return sql.LogEventForJobs(ctx, p.ExecCfg(), txn, &importEvent, int64(job.ID()), - job.Payload(), p.User(), status) - }); err != nil { - log.Warningf(ctx, "failed to log event: %v", err) - } -} - -// Resume is part of the jobs.Resumer interface. -func (r *importResumer) Resume(ctx context.Context, execCtx interface{}) error { - p := execCtx.(sql.JobExecContext) - if err := r.parseBundleSchemaIfNeeded(ctx, p); err != nil { - return err - } - - details := r.job.Details().(jobspb.ImportDetails) - files := details.URIs - format := details.Format - - tables := make(map[string]*execinfrapb.ReadImportDataSpec_ImportTable, len(details.Tables)) - if details.Tables != nil { - // Skip prepare stage on job resumption, if it has already been completed. - if !details.PrepareComplete { - var schemaMetadata *preparedSchemaMetadata - if err := sql.DescsTxn(ctx, p.ExecCfg(), func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, - ) error { - var preparedDetails jobspb.ImportDetails - schemaMetadata = &preparedSchemaMetadata{ - newSchemaIDToName: make(map[descpb.ID]string), - oldSchemaIDToName: make(map[descpb.ID]string), - } - var err error - curDetails := details - if len(details.Schemas) != 0 { - schemaMetadata, err = r.prepareSchemasForIngestion(ctx, p, curDetails, txn, descsCol) - if err != nil { - return err - } - curDetails = schemaMetadata.schemaPreparedDetails - } - - preparedDetails, err = r.prepareTableDescsForIngestion(ctx, p, curDetails, txn, descsCol, - schemaMetadata) - if err != nil { - return err - } - - // Telemetry for multi-region. - for _, table := range preparedDetails.Tables { - _, dbDesc, err := descsCol.GetImmutableDatabaseByID( - ctx, txn, table.Desc.GetParentID(), tree.DatabaseLookupFlags{Required: true}) - if err != nil { - return err - } - if dbDesc.IsMultiRegion() { - telemetry.Inc(sqltelemetry.ImportIntoMultiRegionDatabaseCounter) - } - } - - // Update the job details now that the schemas and table descs have - // been "prepared". - return r.job.Update(ctx, txn, func( - txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, - ) error { - pl := md.Payload - *pl.GetImport() = preparedDetails - - // Update the set of descriptors for later observability. - // TODO(ajwerner): Do we need this idempotence test? - prev := md.Payload.DescriptorIDs - if prev == nil { - var descriptorIDs []descpb.ID - for _, schema := range preparedDetails.Schemas { - descriptorIDs = append(descriptorIDs, schema.Desc.GetID()) - } - for _, table := range preparedDetails.Tables { - descriptorIDs = append(descriptorIDs, table.Desc.GetID()) - } - pl.DescriptorIDs = descriptorIDs - } - ju.UpdatePayload(pl) - return nil - }) - }); err != nil { - return err - } - - // Run the queued job which updates the database descriptor to contain the - // newly created schemas. - // NB: Seems like the registry eventually adopts the job anyways but this - // is in keeping with the semantics we use when creating a schema during - // sql execution. Namely, queue job in the txn which creates the schema - // desc and run once the txn has committed. - if err := p.ExecCfg().JobRegistry.Run(ctx, p.ExecCfg().InternalExecutor, - schemaMetadata.queuedSchemaJobs); err != nil { - return err - } - - // Re-initialize details after prepare step. - details = r.job.Details().(jobspb.ImportDetails) - emitImportJobEvent(ctx, p, jobs.StatusRunning, r.job) - } - - // Create a mapping from schemaID to schemaName. - schemaIDToName := make(map[descpb.ID]string) - for _, i := range details.Schemas { - schemaIDToName[i.Desc.GetID()] = i.Desc.GetName() - } - - for _, i := range details.Tables { - var tableName string - if i.Name != "" { - tableName = i.Name - } else if i.Desc != nil { - tableName = i.Desc.Name - } else { - return errors.New("invalid table specification") - } - - // If we are importing from PGDUMP, qualify the table name with the schema - // name since we support non-public schemas. - if details.Format.Format == roachpb.IOFileFormat_PgDump { - schemaName := tree.PublicSchema - if schema, ok := schemaIDToName[i.Desc.GetUnexposedParentSchemaID()]; ok { - schemaName = schema - } - tableName = fmt.Sprintf("%s.%s", schemaName, tableName) - } - tables[tableName] = &execinfrapb.ReadImportDataSpec_ImportTable{ - Desc: i.Desc, - TargetCols: i.TargetCols, - } - } - } - - typeDescs := make([]*descpb.TypeDescriptor, len(details.Types)) - for i, t := range details.Types { - typeDescs[i] = t.Desc - } - - // If details.Walltime is still 0, then it was not set during - // `prepareTableDescsForIngestion`. This indicates that we are in an IMPORT INTO, - // and that the walltime was not set in a previous run of IMPORT. - // - // In the case of importing into existing tables we must wait for all nodes - // to see the same version of the updated table descriptor, after which we - // shall chose a ts to import from. - if details.Walltime == 0 { - // Now that we know all the tables are offline, pick a walltime at which we - // will write. - details.Walltime = p.ExecCfg().Clock.Now().WallTime +// parseAndCreateBundleTableDescs parses and creates the table +// descriptors for bundle formats. +func parseAndCreateBundleTableDescs( + ctx context.Context, + p sql.JobExecContext, + details jobspb.ImportDetails, + seqVals map[descpb.ID]int64, + skipFKs bool, + parentDB catalog.DatabaseDescriptor, + files []string, + format roachpb.IOFileFormat, + walltime int64, + owner security.SQLUsername, + jobID jobspb.JobID, +) ([]*tabledesc.Mutable, []*schemadesc.Mutable, error) { - // Check if the tables being imported into are starting empty, in which - // case we can cheaply clear-range instead of revert-range to cleanup. - for i := range details.Tables { - if !details.Tables[i].IsNew { - tblDesc := tabledesc.NewBuilder(details.Tables[i].Desc).BuildImmutableTable() - tblSpan := tblDesc.TableSpan(p.ExecCfg().Codec) - res, err := p.ExecCfg().DB.Scan(ctx, tblSpan.Key, tblSpan.EndKey, 1 /* maxRows */) - if err != nil { - return errors.Wrap(err, "checking if existing table is empty") - } - details.Tables[i].WasEmpty = len(res) == 0 - } - } + var schemaDescs []*schemadesc.Mutable + var tableDescs []*tabledesc.Mutable + var tableName string - if err := r.job.SetDetails(ctx, nil /* txn */, details); err != nil { - return err - } + // A single table entry in the import job details when importing a bundle format + // indicates that we are performing a single table import. + // This info is populated during the planning phase. + if len(details.Tables) > 0 { + tableName = details.Tables[0].Name } - res, err := ingestWithRetry(ctx, p, r.job, tables, typeDescs, files, format, details.Walltime, - r.testingKnobs.alwaysFlushJobProgress) + store, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, files[0], p.User()) if err != nil { - return err + return tableDescs, schemaDescs, err } + defer store.Close() - pkIDs := make(map[uint64]struct{}, len(details.Tables)) - for _, t := range details.Tables { - pkIDs[roachpb.BulkOpSummaryID(uint64(t.Desc.ID), uint64(t.Desc.PrimaryIndex.ID))] = struct{}{} + raw, err := store.ReadFile(ctx, "") + if err != nil { + return tableDescs, schemaDescs, err } - r.res.DataSize = res.DataSize - for id, count := range res.EntryCounts { - if _, ok := pkIDs[id]; ok { - r.res.Rows += count - } else { - r.res.IndexEntries += count - } + defer raw.Close() + reader, err := decompressingReader(raw, files[0], format.Compression) + if err != nil { + return tableDescs, schemaDescs, err } - if r.testingKnobs.afterImport != nil { - if err := r.testingKnobs.afterImport(r.res); err != nil { - return err + defer reader.Close() + + fks := fkHandler{skip: skipFKs, allowed: true, resolver: fkResolver{ + tableNameToDesc: make(map[string]*tabledesc.Mutable), + }} + switch format.Format { + case roachpb.IOFileFormat_Mysqldump: + fks.resolver.format.Format = roachpb.IOFileFormat_Mysqldump + evalCtx := &p.ExtendedEvalContext().EvalContext + tableDescs, err = readMysqlCreateTable( + ctx, reader, evalCtx, p, defaultCSVTableID, parentDB, tableName, fks, + seqVals, owner, walltime, + ) + case roachpb.IOFileFormat_PgDump: + fks.resolver.format.Format = roachpb.IOFileFormat_PgDump + evalCtx := &p.ExtendedEvalContext().EvalContext + + // Setup a logger to handle unsupported DDL statements in the PGDUMP file. + unsupportedStmtLogger := makeUnsupportedStmtLogger(ctx, p.User(), int64(jobID), + format.PgDump.IgnoreUnsupported, format.PgDump.IgnoreUnsupportedLog, schemaParsing, + p.ExecCfg().DistSQLSrv.ExternalStorage) + + tableDescs, schemaDescs, err = readPostgresCreateTable(ctx, reader, evalCtx, p, tableName, + parentDB, walltime, fks, int(format.PgDump.MaxRowSize), owner, unsupportedStmtLogger) + + logErr := unsupportedStmtLogger.flush() + if logErr != nil { + return nil, nil, logErr } - } - // If the table being imported into referenced UDTs, ensure that a concurrent - // schema change on any of the typeDescs has not modified the type descriptor. If - // it has, it is unsafe to import the data and we fail the import job. - if err := r.checkForUDTModification(ctx, p.ExecCfg()); err != nil { - return err + default: + return tableDescs, schemaDescs, errors.Errorf( + "non-bundle format %q does not support reading schemas", format.Format.String()) } - if err := r.publishSchemas(ctx, p.ExecCfg()); err != nil { - return err + if err != nil { + return tableDescs, schemaDescs, err } - if err := r.publishTables(ctx, p.ExecCfg(), res); err != nil { - return err + if tableDescs == nil && len(details.Tables) > 0 { + return tableDescs, schemaDescs, errors.Errorf("table definition not found for %q", tableName) } - // As of 21.2 we do not write a protected timestamp record during IMPORT INTO. - // In case of a mixed version cluster with 21.1 and 21.2 nodes, it is possible - // that the job was planned on an older node and then resumed on a 21.2 node. - // Thus, we still need to clear the timestamp record that was written when the - // IMPORT INTO was planned on the older node. - // - // TODO(adityamaru): Remove in 22.1. - if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return r.releaseProtectedTimestamp(ctx, txn, p.ExecCfg().ProtectedTimestampProvider) - }); err != nil { - log.Errorf(ctx, "failed to release protected timestamp: %v", err) - } + return tableDescs, schemaDescs, err +} - emitImportJobEvent(ctx, p, jobs.StatusSucceeded, r.job) +// publishTables updates the status of imported tables from OFFLINE to PUBLIC. +func (r *importResumer) publishTables( + ctx context.Context, execCfg *sql.ExecutorConfig, res roachpb.BulkOpSummary, +) error { + details := r.job.Details().(jobspb.ImportDetails) + // Tables should only be published once. + if details.TablesPublished { + return nil + } - addToFileFormatTelemetry(details.Format.Format.String(), "succeeded") - telemetry.CountBucketed("import.rows", r.res.Rows) - const mb = 1 << 20 - sizeMb := r.res.DataSize / mb - telemetry.CountBucketed("import.size-mb", sizeMb) + // Write stub statistics for new tables created during the import. This should + // be sufficient until the CREATE STATISTICS run finishes. + r.writeStubStatisticsForImportedTables(ctx, execCfg, res) - sec := int64(timeutil.Since(timeutil.FromUnixMicros(r.job.Payload().StartedMicros)).Seconds()) - var mbps int64 - if sec > 0 { - mbps = mb / sec - } - telemetry.CountBucketed("import.duration-sec.succeeded", sec) - telemetry.CountBucketed("import.speed-mbps", mbps) - // Tiny imports may skew throughput numbers due to overhead. - if sizeMb > 10 { - telemetry.CountBucketed("import.speed-mbps.over10mb", mbps) - } + log.Event(ctx, "making tables live") - return nil -} + err := sql.DescsTxn(ctx, execCfg, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + ) error { + b := txn.NewBatch() + for _, tbl := range details.Tables { + newTableDesc, err := descsCol.GetMutableTableVersionByID(ctx, tbl.Desc.ID, txn) + if err != nil { + return err + } + newTableDesc.SetPublic() -func ingestWithRetry( - ctx context.Context, - execCtx sql.JobExecContext, - job *jobs.Job, - tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable, - typeDescs []*descpb.TypeDescriptor, - from []string, - format roachpb.IOFileFormat, - walltime int64, - alwaysFlushProgress bool, -) (roachpb.BulkOpSummary, error) { - resumerSpan := tracing.SpanFromContext(ctx) + if !tbl.IsNew { + // NB: This is not using AllNonDropIndexes or directly mutating the + // constraints returned by the other usual helpers because we need to + // replace the `OutboundFKs` and `Checks` slices of newTableDesc with copies + // that we can mutate. We need to do that because newTableDesc is a shallow + // copy of tbl.Desc that we'll be asserting is the current version when we + // CPut below. + // + // Set FK constraints to unvalidated before publishing the table imported + // into. + newTableDesc.OutboundFKs = make([]descpb.ForeignKeyConstraint, len(newTableDesc.OutboundFKs)) + copy(newTableDesc.OutboundFKs, tbl.Desc.OutboundFKs) + for i := range newTableDesc.OutboundFKs { + newTableDesc.OutboundFKs[i].Validity = descpb.ConstraintValidity_Unvalidated + } - // We retry on pretty generic failures -- any rpc error. If a worker node were - // to restart, it would produce this kind of error, but there may be other - // errors that are also rpc errors. Don't retry to aggressively. - retryOpts := retry.Options{ - MaxBackoff: 1 * time.Second, - MaxRetries: 5, - } + // Set CHECK constraints to unvalidated before publishing the table imported into. + for _, c := range newTableDesc.AllActiveAndInactiveChecks() { + c.Validity = descpb.ConstraintValidity_Unvalidated + } + } - // We want to retry an import if there are transient failures (i.e. worker - // nodes dying), so if we receive a retryable error, re-plan and retry the - // import. - var res roachpb.BulkOpSummary - var err error - var retryCount int32 - for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); { - retryCount++ - resumerSpan.RecordStructured(&roachpb.RetryTracingEvent{ - Operation: "importResumer.ingestWithRetry", - AttemptNumber: retryCount, - RetryError: tracing.RedactAndTruncateError(err), - }) - res, err = sql.DistIngest(ctx, execCtx, job, tables, typeDescs, from, format, walltime, - alwaysFlushProgress) - if err == nil { - break + // TODO(dt): re-validate any FKs? + if err := descsCol.WriteDescToBatch( + ctx, false /* kvTrace */, newTableDesc, b, + ); err != nil { + return errors.Wrapf(err, "publishing table %d", newTableDesc.ID) + } + } + if err := txn.Run(ctx, b); err != nil { + return errors.Wrap(err, "publishing tables") } - if utilccl.IsPermanentBulkJobError(err) { - return res, err + // Update job record to mark tables published state as complete. + details.TablesPublished = true + err := r.job.SetDetails(ctx, txn, details) + if err != nil { + return errors.Wrap(err, "updating job details after publishing tables") } + return nil + }) + if err != nil { + return err + } - // Re-load the job in order to update our progress object, which may have - // been updated by the changeFrontier processor since the flow started. - reloadedJob, reloadErr := execCtx.ExecCfg().JobRegistry.LoadJob(ctx, job.ID()) - if reloadErr != nil { - if ctx.Err() != nil { - return res, ctx.Err() + // 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 details.Tables { + desc := tabledesc.NewBuilder(details.Tables[i].Desc).BuildImmutableTable() + execCfg.StatsRefresher.NotifyMutation(desc, math.MaxInt32 /* rowsAffected */) + } + + return nil +} + +// writeStubStatisticsForImportedTables writes "stub" statistics for new tables +// created during an import. +func (r *importResumer) writeStubStatisticsForImportedTables( + ctx context.Context, execCfg *sql.ExecutorConfig, res roachpb.BulkOpSummary, +) { + details := r.job.Details().(jobspb.ImportDetails) + for _, tbl := range details.Tables { + if tbl.IsNew { + desc := tabledesc.NewBuilder(tbl.Desc).BuildImmutableTable() + id := roachpb.BulkOpSummaryID(uint64(desc.GetID()), uint64(desc.GetPrimaryIndexID())) + rowCount := uint64(res.EntryCounts[id]) + // TODO(michae2): collect distinct and null counts during import. + distinctCount := uint64(float64(rowCount) * memo.UnknownDistinctCountRatio) + nullCount := uint64(float64(rowCount) * memo.UnknownNullCountRatio) + // Because we don't yet have real distinct and null counts, only produce + // single-column stats to avoid the appearance of perfectly correlated + // columns. + multiColEnabled := false + statistics, err := sql.StubTableStats(desc, jobspb.ImportStatsName, multiColEnabled) + if err == nil { + for _, statistic := range statistics { + statistic.RowCount = rowCount + statistic.DistinctCount = distinctCount + statistic.NullCount = nullCount + } + // TODO(michae2): parallelize insertion of statistics. + err = stats.InsertNewStats(ctx, execCfg.InternalExecutor, nil /* txn */, statistics) + } + if err != nil { + // Failure to create statistics should not fail the entire import. + log.Warningf( + ctx, "error while creating statistics during import of %q: %v", + desc.GetName(), err, + ) } - log.Warningf(ctx, `IMPORT job %d could not reload job progress when retrying: %+v`, - int64(job.ID()), reloadErr) - } else { - job = reloadedJob } - log.Warningf(ctx, `encountered retryable error: %+v`, err) - } - - if err != nil { - return roachpb.BulkOpSummary{}, errors.Wrap(err, "exhausted retries") } - return res, nil } +// publishSchemas updates the status of imported schemas from OFFLINE to PUBLIC. func (r *importResumer) publishSchemas(ctx context.Context, execCfg *sql.ExecutorConfig) error { details := r.job.Details().(jobspb.ImportDetails) // Schemas should only be published once. @@ -2400,126 +1121,149 @@ func (r *importResumer) checkForUDTModification( return sql.DescsTxn(ctx, execCfg, checkTypesAreEquivalent) } -// writeStubStatisticsForImportedTables writes "stub" statistics for new tables -// created during an import. -func (r *importResumer) writeStubStatisticsForImportedTables( - ctx context.Context, execCfg *sql.ExecutorConfig, res roachpb.BulkOpSummary, -) { - details := r.job.Details().(jobspb.ImportDetails) - for _, tbl := range details.Tables { - if tbl.IsNew { - desc := tabledesc.NewBuilder(tbl.Desc).BuildImmutableTable() - id := roachpb.BulkOpSummaryID(uint64(desc.GetID()), uint64(desc.GetPrimaryIndexID())) - rowCount := uint64(res.EntryCounts[id]) - // TODO(michae2): collect distinct and null counts during import. - distinctCount := uint64(float64(rowCount) * memo.UnknownDistinctCountRatio) - nullCount := uint64(float64(rowCount) * memo.UnknownNullCountRatio) - // Because we don't yet have real distinct and null counts, only produce - // single-column stats to avoid the appearance of perfectly correlated - // columns. - multiColEnabled := false - statistics, err := sql.StubTableStats(desc, jobspb.ImportStatsName, multiColEnabled) - if err == nil { - for _, statistic := range statistics { - statistic.RowCount = rowCount - statistic.DistinctCount = distinctCount - statistic.NullCount = nullCount - } - // TODO(michae2): parallelize insertion of statistics. - err = stats.InsertNewStats(ctx, execCfg.InternalExecutor, nil /* txn */, statistics) - } - if err != nil { - // Failure to create statistics should not fail the entire import. - log.Warningf( - ctx, "error while creating statistics during import of %q: %v", - desc.GetName(), err, - ) - } - } - } -} +func ingestWithRetry( + ctx context.Context, + execCtx sql.JobExecContext, + job *jobs.Job, + tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable, + typeDescs []*descpb.TypeDescriptor, + from []string, + format roachpb.IOFileFormat, + walltime int64, + alwaysFlushProgress bool, +) (roachpb.BulkOpSummary, error) { + resumerSpan := tracing.SpanFromContext(ctx) -// publishTables updates the status of imported tables from OFFLINE to PUBLIC. -func (r *importResumer) publishTables( - ctx context.Context, execCfg *sql.ExecutorConfig, res roachpb.BulkOpSummary, -) error { - details := r.job.Details().(jobspb.ImportDetails) - // Tables should only be published once. - if details.TablesPublished { - return nil + // We retry on pretty generic failures -- any rpc error. If a worker node were + // to restart, it would produce this kind of error, but there may be other + // errors that are also rpc errors. Don't retry to aggressively. + retryOpts := retry.Options{ + MaxBackoff: 1 * time.Second, + MaxRetries: 5, } - // Write stub statistics for new tables created during the import. This should - // be sufficient until the CREATE STATISTICS run finishes. - r.writeStubStatisticsForImportedTables(ctx, execCfg, res) + // We want to retry an import if there are transient failures (i.e. worker + // nodes dying), so if we receive a retryable error, re-plan and retry the + // import. + var res roachpb.BulkOpSummary + var err error + var retryCount int32 + for r := retry.StartWithCtx(ctx, retryOpts); r.Next(); { + retryCount++ + resumerSpan.RecordStructured(&roachpb.RetryTracingEvent{ + Operation: "importResumer.ingestWithRetry", + AttemptNumber: retryCount, + RetryError: tracing.RedactAndTruncateError(err), + }) + res, err = distImport(ctx, execCtx, job, tables, typeDescs, from, format, walltime, + alwaysFlushProgress) + if err == nil { + break + } - log.Event(ctx, "making tables live") + if utilccl.IsPermanentBulkJobError(err) { + return res, err + } - err := sql.DescsTxn(ctx, execCfg, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, - ) error { - b := txn.NewBatch() - for _, tbl := range details.Tables { - newTableDesc, err := descsCol.GetMutableTableVersionByID(ctx, tbl.Desc.ID, txn) - if err != nil { - return err + // Re-load the job in order to update our progress object, which may have + // been updated by the changeFrontier processor since the flow started. + reloadedJob, reloadErr := execCtx.ExecCfg().JobRegistry.LoadJob(ctx, job.ID()) + if reloadErr != nil { + if ctx.Err() != nil { + return res, ctx.Err() } - newTableDesc.SetPublic() + log.Warningf(ctx, `IMPORT job %d could not reload job progress when retrying: %+v`, + int64(job.ID()), reloadErr) + } else { + job = reloadedJob + } + log.Warningf(ctx, `encountered retryable error: %+v`, err) + } - if !tbl.IsNew { - // NB: This is not using AllNonDropIndexes or directly mutating the - // constraints returned by the other usual helpers because we need to - // replace the `OutboundFKs` and `Checks` slices of newTableDesc with copies - // that we can mutate. We need to do that because newTableDesc is a shallow - // copy of tbl.Desc that we'll be asserting is the current version when we - // CPut below. - // - // Set FK constraints to unvalidated before publishing the table imported - // into. - newTableDesc.OutboundFKs = make([]descpb.ForeignKeyConstraint, len(newTableDesc.OutboundFKs)) - copy(newTableDesc.OutboundFKs, tbl.Desc.OutboundFKs) - for i := range newTableDesc.OutboundFKs { - newTableDesc.OutboundFKs[i].Validity = descpb.ConstraintValidity_Unvalidated - } + if err != nil { + return roachpb.BulkOpSummary{}, errors.Wrap(err, "exhausted retries") + } + return res, nil +} - // Set CHECK constraints to unvalidated before publishing the table imported into. - for _, c := range newTableDesc.AllActiveAndInactiveChecks() { - c.Validity = descpb.ConstraintValidity_Unvalidated - } - } +// emitImportJobEvent emits an import job event to the event log. +func emitImportJobEvent( + ctx context.Context, p sql.JobExecContext, status jobs.Status, job *jobs.Job, +) { + var importEvent eventpb.Import + if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return sql.LogEventForJobs(ctx, p.ExecCfg(), txn, &importEvent, int64(job.ID()), + job.Payload(), p.User(), status) + }); err != nil { + log.Warningf(ctx, "failed to log event: %v", err) + } +} - // TODO(dt): re-validate any FKs? - if err := descsCol.WriteDescToBatch( - ctx, false /* kvTrace */, newTableDesc, b, - ); err != nil { - return errors.Wrapf(err, "publishing table %d", newTableDesc.ID) - } - } - if err := txn.Run(ctx, b); err != nil { - return errors.Wrap(err, "publishing tables") - } +func constructSchemaAndTableKey( + tableDesc *descpb.TableDescriptor, schemaIDToName map[descpb.ID]string, +) (schemaAndTableName, error) { + schemaName, ok := schemaIDToName[tableDesc.GetUnexposedParentSchemaID()] + if !ok && tableDesc.UnexposedParentSchemaID != keys.PublicSchemaID { + return schemaAndTableName{}, errors.Newf("invalid parent schema ID %d for table %s", + tableDesc.UnexposedParentSchemaID, tableDesc.GetName()) + } - // Update job record to mark tables published state as complete. - details.TablesPublished = true - err := r.job.SetDetails(ctx, txn, details) - if err != nil { - return errors.Wrap(err, "updating job details after publishing tables") - } - return nil - }) + return schemaAndTableName{schema: schemaName, table: tableDesc.GetName()}, nil +} + +func writeNonDropDatabaseChange( + ctx context.Context, + desc *dbdesc.Mutable, + txn *kv.Txn, + descsCol *descs.Collection, + p sql.JobExecContext, + jobDesc string, +) ([]jobspb.JobID, error) { + var job *jobs.Job + var err error + if job, err = createNonDropDatabaseChangeJob(p.User(), desc.ID, jobDesc, p, txn); err != nil { + return nil, err + } + + queuedJob := []jobspb.JobID{job.ID()} + b := txn.NewBatch() + err = descsCol.WriteDescToBatch( + ctx, + p.ExtendedEvalContext().Tracing.KVTracingEnabled(), + desc, + b, + ) if err != nil { - return err + return nil, err } + return queuedJob, txn.Run(ctx, b) +} - // 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 details.Tables { - desc := tabledesc.NewBuilder(details.Tables[i].Desc).BuildImmutableTable() - execCfg.StatsRefresher.NotifyMutation(desc, math.MaxInt32 /* rowsAffected */) +func createNonDropDatabaseChangeJob( + user security.SQLUsername, + databaseID descpb.ID, + jobDesc string, + p sql.JobExecContext, + txn *kv.Txn, +) (*jobs.Job, error) { + jobRecord := jobs.Record{ + Description: jobDesc, + Username: user, + Details: jobspb.SchemaChangeDetails{ + DescID: databaseID, + FormatVersion: jobspb.DatabaseJobFormatVersion, + }, + Progress: jobspb.SchemaChangeProgress{}, + NonCancelable: true, } - return nil + jobID := p.ExecCfg().JobRegistry.MakeJobID() + return p.ExecCfg().JobRegistry.CreateJobWithTxn( + p.ExtendedEvalContext().Context, + jobRecord, + jobID, + txn, + ) } // OnFailOrCancel is part of the jobs.Resumer interface. Removes data that has @@ -2576,121 +1320,6 @@ func (r *importResumer) OnFailOrCancel(ctx context.Context, execCtx interface{}) return nil } -func (r *importResumer) releaseProtectedTimestamp( - ctx context.Context, txn *kv.Txn, pts protectedts.Storage, -) error { - details := r.job.Details().(jobspb.ImportDetails) - ptsID := details.ProtectedTimestampRecord - // If the job doesn't have a protected timestamp then there's nothing to do. - if ptsID == nil { - return nil - } - err := pts.Release(ctx, txn, *ptsID) - if errors.Is(err, protectedts.ErrNotExists) { - // No reason to return an error which might cause problems if it doesn't - // seem to exist. - log.Warningf(ctx, "failed to release protected which seems not to exist: %v", err) - err = nil - } - return err -} - -func (r *importResumer) dropSchemas( - ctx context.Context, - txn *kv.Txn, - descsCol *descs.Collection, - execCfg *sql.ExecutorConfig, - p sql.JobExecContext, -) ([]jobspb.JobID, error) { - details := r.job.Details().(jobspb.ImportDetails) - - // If the prepare step of the import job was not completed then the - // descriptors do not need to be rolled back as the txn updating them never - // completed. - if !details.PrepareComplete || len(details.Schemas) == 0 { - return nil, nil - } - - // Resolve the database descriptor. - desc, err := descsCol.GetMutableDescriptorByID(ctx, details.ParentID, txn) - if err != nil { - return nil, err - } - - dbDesc, ok := desc.(*dbdesc.Mutable) - if !ok { - return nil, errors.Newf("expected ID %d to refer to the database being imported into", - details.ParentID) - } - - droppedSchemaIDs := make([]descpb.ID, 0) - for _, schema := range details.Schemas { - desc, err := descsCol.GetMutableDescriptorByID(ctx, schema.Desc.ID, txn) - if err != nil { - return nil, err - } - var schemaDesc *schemadesc.Mutable - var ok bool - if schemaDesc, ok = desc.(*schemadesc.Mutable); !ok { - return nil, errors.Newf("unable to resolve schema desc with ID %d", schema.Desc.ID) - } - - schemaDesc.DrainingNames = append(schemaDesc.DrainingNames, - descpb.NameInfo{ParentID: details.ParentID, ParentSchemaID: keys.RootNamespaceID, - Name: schemaDesc.Name}) - - // Update the parent database with information about the dropped schema. - if dbDesc.Schemas == nil { - dbDesc.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) - } - dbDesc.Schemas[schema.Desc.Name] = descpb.DatabaseDescriptor_SchemaInfo{ID: dbDesc.ID, - Dropped: true} - - // Mark the descriptor as dropped and write it to the batch. - schemaDesc.SetDropped() - droppedSchemaIDs = append(droppedSchemaIDs, schemaDesc.GetID()) - - b := txn.NewBatch() - if err := descsCol.WriteDescToBatch(ctx, p.ExtendedEvalContext().Tracing.KVTracingEnabled(), - schemaDesc, b); err != nil { - return nil, err - } - err = txn.Run(ctx, b) - if err != nil { - return nil, err - } - } - - // Write out the change to the database. This only creates a job record to be - // run after the txn commits. - queuedJob, err := writeNonDropDatabaseChange(ctx, dbDesc, txn, descsCol, p, "") - if err != nil { - return nil, err - } - - // Create the job to drop the schema. - dropSchemaJobRecord := jobs.Record{ - Description: "dropping schemas as part of an import job rollback", - Username: p.User(), - DescriptorIDs: droppedSchemaIDs, - Details: jobspb.SchemaChangeDetails{ - DroppedSchemas: droppedSchemaIDs, - DroppedDatabaseID: descpb.InvalidID, - FormatVersion: jobspb.DatabaseJobFormatVersion, - }, - Progress: jobspb.SchemaChangeProgress{}, - NonCancelable: true, - } - jobID := p.ExecCfg().JobRegistry.MakeJobID() - job, err := execCfg.JobRegistry.CreateJobWithTxn(ctx, dropSchemaJobRecord, jobID, txn) - if err != nil { - return nil, err - } - queuedJob = append(queuedJob, job.ID()) - - return queuedJob, nil -} - // dropTables implements the OnFailOrCancel logic. func (r *importResumer) dropTables( ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, execCfg *sql.ExecutorConfig, @@ -2816,10 +1445,139 @@ func (r *importResumer) dropTables( return errors.Wrap(txn.Run(ctx, b), "rolling back tables") } -var _ jobs.Resumer = &importResumer{} +func (r *importResumer) dropSchemas( + ctx context.Context, + txn *kv.Txn, + descsCol *descs.Collection, + execCfg *sql.ExecutorConfig, + p sql.JobExecContext, +) ([]jobspb.JobID, error) { + details := r.job.Details().(jobspb.ImportDetails) + + // If the prepare step of the import job was not completed then the + // descriptors do not need to be rolled back as the txn updating them never + // completed. + if !details.PrepareComplete || len(details.Schemas) == 0 { + return nil, nil + } + + // Resolve the database descriptor. + desc, err := descsCol.GetMutableDescriptorByID(ctx, details.ParentID, txn) + if err != nil { + return nil, err + } + + dbDesc, ok := desc.(*dbdesc.Mutable) + if !ok { + return nil, errors.Newf("expected ID %d to refer to the database being imported into", + details.ParentID) + } + + droppedSchemaIDs := make([]descpb.ID, 0) + for _, schema := range details.Schemas { + desc, err := descsCol.GetMutableDescriptorByID(ctx, schema.Desc.ID, txn) + if err != nil { + return nil, err + } + var schemaDesc *schemadesc.Mutable + var ok bool + if schemaDesc, ok = desc.(*schemadesc.Mutable); !ok { + return nil, errors.Newf("unable to resolve schema desc with ID %d", schema.Desc.ID) + } + + schemaDesc.DrainingNames = append(schemaDesc.DrainingNames, + descpb.NameInfo{ParentID: details.ParentID, ParentSchemaID: keys.RootNamespaceID, + Name: schemaDesc.Name}) + + // Update the parent database with information about the dropped schema. + if dbDesc.Schemas == nil { + dbDesc.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) + } + dbDesc.Schemas[schema.Desc.Name] = descpb.DatabaseDescriptor_SchemaInfo{ID: dbDesc.ID, + Dropped: true} + + // Mark the descriptor as dropped and write it to the batch. + schemaDesc.SetDropped() + droppedSchemaIDs = append(droppedSchemaIDs, schemaDesc.GetID()) + + b := txn.NewBatch() + if err := descsCol.WriteDescToBatch(ctx, p.ExtendedEvalContext().Tracing.KVTracingEnabled(), + schemaDesc, b); err != nil { + return nil, err + } + err = txn.Run(ctx, b) + if err != nil { + return nil, err + } + } + + // Write out the change to the database. This only creates a job record to be + // run after the txn commits. + queuedJob, err := writeNonDropDatabaseChange(ctx, dbDesc, txn, descsCol, p, "") + if err != nil { + return nil, err + } + + // Create the job to drop the schema. + dropSchemaJobRecord := jobs.Record{ + Description: "dropping schemas as part of an import job rollback", + Username: p.User(), + DescriptorIDs: droppedSchemaIDs, + Details: jobspb.SchemaChangeDetails{ + DroppedSchemas: droppedSchemaIDs, + DroppedDatabaseID: descpb.InvalidID, + FormatVersion: jobspb.DatabaseJobFormatVersion, + }, + Progress: jobspb.SchemaChangeProgress{}, + NonCancelable: true, + } + jobID := p.ExecCfg().JobRegistry.MakeJobID() + job, err := execCfg.JobRegistry.CreateJobWithTxn(ctx, dropSchemaJobRecord, jobID, txn) + if err != nil { + return nil, err + } + queuedJob = append(queuedJob, job.ID()) + + return queuedJob, nil +} + +func (r *importResumer) releaseProtectedTimestamp( + ctx context.Context, txn *kv.Txn, pts protectedts.Storage, +) error { + details := r.job.Details().(jobspb.ImportDetails) + ptsID := details.ProtectedTimestampRecord + // If the job doesn't have a protected timestamp then there's nothing to do. + if ptsID == nil { + return nil + } + err := pts.Release(ctx, txn, *ptsID) + if errors.Is(err, protectedts.ErrNotExists) { + // No reason to return an error which might cause problems if it doesn't + // seem to exist. + log.Warningf(ctx, "failed to release protected which seems not to exist: %v", err) + err = nil + } + return err +} + +// ReportResults implements JobResultsReporter interface. +func (r *importResumer) ReportResults(ctx context.Context, resultsCh chan<- tree.Datums) error { + select { + case resultsCh <- tree.Datums{ + tree.NewDInt(tree.DInt(r.job.ID())), + tree.NewDString(string(jobs.StatusSucceeded)), + tree.NewDFloat(tree.DFloat(1.0)), + tree.NewDInt(tree.DInt(r.res.Rows)), + tree.NewDInt(tree.DInt(r.res.IndexEntries)), + tree.NewDInt(tree.DInt(r.res.DataSize)), + }: + return nil + case <-ctx.Done(): + return ctx.Err() + } +} func init() { - sql.AddPlanHook(importPlanHook) jobs.RegisterConstructor( jobspb.TypeImport, func(job *jobs.Job, settings *cluster.Settings) jobs.Resumer { diff --git a/pkg/ccl/importccl/import_planning.go b/pkg/ccl/importccl/import_planning.go new file mode 100644 index 000000000000..2175ef2aa6af --- /dev/null +++ b/pkg/ccl/importccl/import_planning.go @@ -0,0 +1,1288 @@ +// 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 ( + "bytes" + "context" + "fmt" + "io/ioutil" + "math" + "net/url" + "path" + "sort" + "strconv" + "strings" + + "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/featureflag" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" + "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/errors" +) + +const ( + csvDelimiter = "delimiter" + csvComment = "comment" + csvNullIf = "nullif" + csvSkip = "skip" + csvRowLimit = "row_limit" + csvStrictQuotes = "strict_quotes" + + mysqlOutfileRowSep = "rows_terminated_by" + mysqlOutfileFieldSep = "fields_terminated_by" + mysqlOutfileEnclose = "fields_enclosed_by" + mysqlOutfileEscape = "fields_escaped_by" + + importOptionSSTSize = "sstsize" + importOptionDecompress = "decompress" + importOptionOversample = "oversample" + importOptionSkipFKs = "skip_foreign_keys" + importOptionDisableGlobMatch = "disable_glob_matching" + importOptionSaveRejected = "experimental_save_rejected" + importOptionDetached = "detached" + + pgCopyDelimiter = "delimiter" + pgCopyNull = "nullif" + + optMaxRowSize = "max_row_size" + + // Turn on strict validation when importing avro records. + avroStrict = "strict_validation" + // Default input format is assumed to be OCF (object container file). + // This default can be changed by specified either of these options. + avroBinRecords = "data_as_binary_records" + avroJSONRecords = "data_as_json_records" + // Record separator; default "\n" + avroRecordsSeparatedBy = "records_terminated_by" + // If we are importing avro records (binary or JSON), we must specify schema + // as either an inline JSON schema, or an external schema URI. + avroSchema = "schema" + avroSchemaURI = "schema_uri" + + pgDumpIgnoreAllUnsupported = "ignore_unsupported_statements" + pgDumpIgnoreShuntFileDest = "log_ignored_statements" + pgDumpUnsupportedSchemaStmtLog = "unsupported_schema_stmts" + pgDumpUnsupportedDataStmtLog = "unsupported_data_stmts" + + // RunningStatusImportBundleParseSchema indicates to the user that a bundle format + // schema is being parsed + runningStatusImportBundleParseSchema jobs.RunningStatus = "parsing schema on Import Bundle" +) + +var importOptionExpectValues = map[string]sql.KVStringOptValidate{ + csvDelimiter: sql.KVStringOptRequireValue, + csvComment: sql.KVStringOptRequireValue, + csvNullIf: sql.KVStringOptRequireValue, + csvSkip: sql.KVStringOptRequireValue, + csvRowLimit: sql.KVStringOptRequireValue, + csvStrictQuotes: sql.KVStringOptRequireNoValue, + + mysqlOutfileRowSep: sql.KVStringOptRequireValue, + mysqlOutfileFieldSep: sql.KVStringOptRequireValue, + mysqlOutfileEnclose: sql.KVStringOptRequireValue, + mysqlOutfileEscape: sql.KVStringOptRequireValue, + + importOptionSSTSize: sql.KVStringOptRequireValue, + importOptionDecompress: sql.KVStringOptRequireValue, + importOptionOversample: sql.KVStringOptRequireValue, + importOptionSaveRejected: sql.KVStringOptRequireNoValue, + + importOptionSkipFKs: sql.KVStringOptRequireNoValue, + importOptionDisableGlobMatch: sql.KVStringOptRequireNoValue, + importOptionDetached: sql.KVStringOptRequireNoValue, + + optMaxRowSize: sql.KVStringOptRequireValue, + + avroStrict: sql.KVStringOptRequireNoValue, + avroSchema: sql.KVStringOptRequireValue, + avroSchemaURI: sql.KVStringOptRequireValue, + avroRecordsSeparatedBy: sql.KVStringOptRequireValue, + avroBinRecords: sql.KVStringOptRequireNoValue, + avroJSONRecords: sql.KVStringOptRequireNoValue, + + pgDumpIgnoreAllUnsupported: sql.KVStringOptRequireNoValue, + pgDumpIgnoreShuntFileDest: sql.KVStringOptRequireValue, +} + +var pgDumpMaxLoggedStmts = 1024 + +func testingSetMaxLogIgnoredImportStatements(maxLogSize int) (cleanup func()) { + prevLogSize := pgDumpMaxLoggedStmts + pgDumpMaxLoggedStmts = maxLogSize + return func() { + pgDumpMaxLoggedStmts = prevLogSize + } +} + +func makeStringSet(opts ...string) map[string]struct{} { + res := make(map[string]struct{}, len(opts)) + for _, opt := range opts { + res[opt] = struct{}{} + } + return res +} + +// Options common to all formats. +var allowedCommonOptions = makeStringSet( + importOptionSSTSize, importOptionDecompress, importOptionOversample, + importOptionSaveRejected, importOptionDisableGlobMatch, importOptionDetached) + +// Format specific allowed options. +var avroAllowedOptions = makeStringSet( + avroStrict, avroBinRecords, avroJSONRecords, + avroRecordsSeparatedBy, avroSchema, avroSchemaURI, optMaxRowSize, csvRowLimit, +) +var csvAllowedOptions = makeStringSet( + csvDelimiter, csvComment, csvNullIf, csvSkip, csvStrictQuotes, csvRowLimit, +) +var mysqlOutAllowedOptions = makeStringSet( + mysqlOutfileRowSep, mysqlOutfileFieldSep, mysqlOutfileEnclose, + mysqlOutfileEscape, csvNullIf, csvSkip, csvRowLimit, +) +var mysqlDumpAllowedOptions = makeStringSet(importOptionSkipFKs, csvRowLimit) +var pgCopyAllowedOptions = makeStringSet(pgCopyDelimiter, pgCopyNull, optMaxRowSize) +var pgDumpAllowedOptions = makeStringSet(optMaxRowSize, importOptionSkipFKs, csvRowLimit, + pgDumpIgnoreAllUnsupported, pgDumpIgnoreShuntFileDest) + +// DROP is required because the target table needs to be take offline during +// IMPORT INTO. +var importIntoRequiredPrivileges = []privilege.Kind{privilege.INSERT, privilege.DROP} + +// File formats supported for IMPORT INTO +var allowedIntoFormats = map[string]struct{}{ + "CSV": {}, + "AVRO": {}, + "DELIMITED": {}, + "PGCOPY": {}, +} + +// featureImportEnabled is used to enable and disable the IMPORT feature. +var featureImportEnabled = settings.RegisterBoolSetting( + "feature.import.enabled", + "set to true to enable imports, false to disable; default is true", + featureflag.FeatureFlagEnabledDefault, +).WithPublic() + +func validateFormatOptions( + format string, specified map[string]string, formatAllowed map[string]struct{}, +) error { + for opt := range specified { + if _, ok := formatAllowed[opt]; !ok { + if _, ok = allowedCommonOptions[opt]; !ok { + return errors.Errorf( + "invalid option %q specified for %s import format", opt, format) + } + } + } + return nil +} + +func importJobDescription( + p sql.PlanHookState, + orig *tree.Import, + defs tree.TableDefs, + files []string, + opts map[string]string, +) (string, error) { + stmt := *orig + stmt.CreateFile = nil + stmt.CreateDefs = defs + stmt.Files = nil + for _, file := range files { + clean, err := cloud.SanitizeExternalStorageURI(file, nil /* extraParams */) + if err != nil { + return "", err + } + stmt.Files = append(stmt.Files, tree.NewDString(clean)) + } + stmt.Options = nil + for k, v := range opts { + opt := tree.KVOption{Key: tree.Name(k)} + val := importOptionExpectValues[k] == sql.KVStringOptRequireValue + val = val || (importOptionExpectValues[k] == sql.KVStringOptAny && len(v) > 0) + if val { + opt.Value = tree.NewDString(v) + } + stmt.Options = append(stmt.Options, opt) + } + sort.Slice(stmt.Options, func(i, j int) bool { return stmt.Options[i].Key < stmt.Options[j].Key }) + ann := p.ExtendedEvalContext().Annotations + return tree.AsStringWithFQNames(&stmt, ann), nil +} + +func ensureRequiredPrivileges( + ctx context.Context, + requiredPrivileges []privilege.Kind, + p sql.PlanHookState, + desc *tabledesc.Mutable, +) error { + for _, priv := range requiredPrivileges { + err := p.CheckPrivilege(ctx, desc, priv) + if err != nil { + return err + } + } + + return nil +} + +// addToFileFormatTelemetry records the different stages of IMPORT on a per file +// format basis. +// +// The current states being counted are: +// attempted: Counted at the very beginning of the IMPORT. +// started: Counted just before the IMPORT job is started. +// failed: Counted when the IMPORT job is failed or canceled. +// succeeded: Counted when the IMPORT job completes successfully. +func addToFileFormatTelemetry(fileFormat, state string) { + telemetry.Count(fmt.Sprintf("%s.%s.%s", "import", strings.ToLower(fileFormat), state)) +} + +// resolveUDTsUsedByImportInto resolves all the user defined types that are +// referenced by the table being imported into. +func resolveUDTsUsedByImportInto( + ctx context.Context, p sql.PlanHookState, table *tabledesc.Mutable, +) ([]catalog.TypeDescriptor, error) { + typeDescs := make([]catalog.TypeDescriptor, 0) + var dbDesc catalog.DatabaseDescriptor + err := sql.DescsTxn(ctx, p.ExecCfg(), func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) (err error) { + _, dbDesc, err = descriptors.GetImmutableDatabaseByID(ctx, txn, table.GetParentID(), + tree.DatabaseLookupFlags{ + Required: true, + AvoidCached: true, + }) + if err != nil { + return err + } + typeIDs, _, err := table.GetAllReferencedTypeIDs(dbDesc, + func(id descpb.ID) (catalog.TypeDescriptor, error) { + immutDesc, err := descriptors.GetImmutableTypeByID(ctx, txn, id, tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, + AvoidCached: true, + }, + }) + if err != nil { + return nil, err + } + return immutDesc, nil + }) + if err != nil { + return errors.Wrap(err, "resolving type descriptors") + } + + for _, typeID := range typeIDs { + immutDesc, err := descriptors.GetImmutableTypeByID(ctx, txn, typeID, tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, + AvoidCached: true, + }, + }) + if err != nil { + return err + } + typeDescs = append(typeDescs, immutDesc) + } + return err + }) + return typeDescs, err +} + +// importPlanHook implements sql.PlanHookFn. +func importPlanHook( + ctx context.Context, stmt tree.Statement, p sql.PlanHookState, +) (sql.PlanHookRowFn, colinfo.ResultColumns, []sql.PlanNode, bool, error) { + importStmt, ok := stmt.(*tree.Import) + if !ok { + return nil, nil, nil, false, nil + } + + if !importStmt.Bundle && !importStmt.Into { + p.BufferClientNotice(ctx, pgnotice.Newf("IMPORT TABLE has been deprecated in 21.2, and will be removed in a future version."+ + " Instead, use CREATE TABLE with the desired schema, and IMPORT INTO the newly created table.")) + } + + addToFileFormatTelemetry(importStmt.FileFormat, "attempted") + + if err := featureflag.CheckEnabled( + ctx, + p.ExecCfg(), + featureImportEnabled, + "IMPORT", + ); err != nil { + return nil, nil, nil, false, err + } + + filesFn, err := p.TypeAsStringArray(ctx, importStmt.Files, "IMPORT") + if err != nil { + return nil, nil, nil, false, err + } + + var createFileFn func() (string, error) + if !importStmt.Bundle && !importStmt.Into && importStmt.CreateDefs == nil { + createFileFn, err = p.TypeAsString(ctx, importStmt.CreateFile, "IMPORT") + if err != nil { + return nil, nil, nil, false, err + } + } + + optsFn, err := p.TypeAsStringOpts(ctx, importStmt.Options, importOptionExpectValues) + if err != nil { + return nil, nil, nil, false, err + } + + opts, optsErr := optsFn() + + var isDetached bool + if _, ok := opts[importOptionDetached]; ok { + isDetached = true + } + + fn := func(ctx context.Context, _ []sql.PlanNode, resultsCh chan<- tree.Datums) error { + // TODO(dan): Move this span into sql. + ctx, span := tracing.ChildSpan(ctx, importStmt.StatementTag()) + defer span.Finish() + + walltime := p.ExecCfg().Clock.Now().WallTime + + if !(p.ExtendedEvalContext().TxnImplicit || isDetached) { + return errors.Errorf("IMPORT cannot be used inside a transaction without DETACHED option") + } + + if optsErr != nil { + return optsErr + } + + filenamePatterns, err := filesFn() + if err != nil { + return err + } + + // Certain ExternalStorage URIs require super-user access. Check all the + // URIs passed to the IMPORT command. + for _, file := range filenamePatterns { + conf, err := cloud.ExternalStorageConfFromURI(file, p.User()) + if err != nil { + // If it is a workload URI, it won't parse as a storage config, but it + // also doesn't have any auth concerns so just continue. + if _, workloadErr := parseWorkloadConfig(file); workloadErr == nil { + continue + } + return err + } + if !conf.AccessIsWithExplicitAuth() { + err := p.RequireAdminRole(ctx, + fmt.Sprintf("IMPORT from the specified %s URI", conf.Provider.String())) + if err != nil { + return err + } + } + } + + var files []string + if _, ok := opts[importOptionDisableGlobMatch]; ok { + files = filenamePatterns + } else { + for _, file := range filenamePatterns { + uri, err := url.Parse(file) + if err != nil { + return err + } + if strings.Contains(uri.Scheme, "workload") || strings.HasPrefix(uri.Scheme, "http") { + files = append(files, file) + continue + } + prefix := cloud.GetPrefixBeforeWildcard(uri.Path) + if len(prefix) < len(uri.Path) { + pattern := uri.Path[len(prefix):] + uri.Path = prefix + s, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, uri.String(), p.User()) + if err != nil { + return err + } + var expandedFiles []string + if err := s.List(ctx, "", "", func(s string) error { + ok, err := path.Match(pattern, s) + if ok { + uri.Path = prefix + s + expandedFiles = append(expandedFiles, uri.String()) + } + return err + }); err != nil { + return err + } + if len(expandedFiles) < 1 { + return errors.Errorf(`no files matched %q in prefix %q in uri provided: %q`, pattern, prefix, file) + } + files = append(files, expandedFiles...) + } else { + files = append(files, file) + } + } + } + + // Typically the SQL grammar means it is only possible to specifying exactly + // one pgdump/mysqldump URI, but glob-expansion could have changed that. + if importStmt.Bundle && len(files) != 1 { + return pgerror.New(pgcode.FeatureNotSupported, "SQL dump files must be imported individually") + } + + table := importStmt.Table + var db catalog.DatabaseDescriptor + var sc catalog.SchemaDescriptor + if table != nil { + // TODO: As part of work for #34240, we should be operating on + // UnresolvedObjectNames here, rather than TableNames. + // We have a target table, so it might specify a DB in its name. + un := table.ToUnresolvedObjectName() + found, prefix, resPrefix, err := resolver.ResolveTarget(ctx, + un, p, p.SessionData().Database, p.SessionData().SearchPath) + if err != nil { + return pgerror.Wrap(err, pgcode.UndefinedTable, + "resolving target import name") + } + if !found { + // Check if database exists right now. It might not after the import is done, + // but it's better to fail fast than wait until restore. + return pgerror.Newf(pgcode.UndefinedObject, + "database does not exist: %q", table) + } + table.ObjectNamePrefix = prefix + db = resPrefix.Database + sc = resPrefix.Schema + // If this is a non-INTO import that will thus be making a new table, we + // need the CREATE priv in the target DB. + if !importStmt.Into { + if err := p.CheckPrivilege(ctx, db, privilege.CREATE); err != nil { + return err + } + } + + switch sc.SchemaKind() { + case catalog.SchemaVirtual: + return pgerror.Newf(pgcode.InvalidSchemaName, + "cannot import into schema %q", table.SchemaName) + } + } else { + // No target table means we're importing whatever we find into the session + // database, so it must exist. + txn := p.ExtendedEvalContext().Txn + db, err = p.Accessor().GetDatabaseDesc(ctx, txn, p.SessionData().Database, tree.DatabaseLookupFlags{ + AvoidCached: true, + Required: true, + }) + if err != nil { + return pgerror.Wrap(err, pgcode.UndefinedObject, + "could not resolve current database") + } + // If this is a non-INTO import that will thus be making a new table, we + // need the CREATE priv in the target DB. + if !importStmt.Into { + if err := p.CheckPrivilege(ctx, db, privilege.CREATE); err != nil { + return err + } + } + sc = schemadesc.GetPublicSchema() + } + + format := roachpb.IOFileFormat{} + switch importStmt.FileFormat { + case "CSV": + if err = validateFormatOptions(importStmt.FileFormat, opts, csvAllowedOptions); err != nil { + return err + } + format.Format = roachpb.IOFileFormat_CSV + // Set the default CSV separator for the cases when it is not overwritten. + format.Csv.Comma = ',' + if override, ok := opts[csvDelimiter]; ok { + comma, err := util.GetSingleRune(override) + if err != nil { + return pgerror.Wrap(err, pgcode.Syntax, "invalid comma value") + } + format.Csv.Comma = comma + } + + if override, ok := opts[csvComment]; ok { + comment, err := util.GetSingleRune(override) + if err != nil { + return pgerror.Wrap(err, pgcode.Syntax, "invalid comment value") + } + format.Csv.Comment = comment + } + + if override, ok := opts[csvNullIf]; ok { + format.Csv.NullEncoding = &override + } + + if override, ok := opts[csvSkip]; ok { + skip, err := strconv.Atoi(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid %s value", csvSkip) + } + if skip < 0 { + return pgerror.Newf(pgcode.Syntax, "%s must be >= 0", csvSkip) + } + format.Csv.Skip = uint32(skip) + } + if _, ok := opts[csvStrictQuotes]; ok { + format.Csv.StrictQuotes = true + } + if _, ok := opts[importOptionSaveRejected]; ok { + format.SaveRejected = true + } + if override, ok := opts[csvRowLimit]; ok { + rowLimit, err := strconv.Atoi(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) + } + if rowLimit <= 0 { + return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) + } + format.Csv.RowLimit = int64(rowLimit) + } + case "DELIMITED": + if err = validateFormatOptions(importStmt.FileFormat, opts, mysqlOutAllowedOptions); err != nil { + return err + } + format.Format = roachpb.IOFileFormat_MysqlOutfile + format.MysqlOut = roachpb.MySQLOutfileOptions{ + RowSeparator: '\n', + FieldSeparator: '\t', + } + if override, ok := opts[mysqlOutfileRowSep]; ok { + c, err := util.GetSingleRune(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, + "invalid %q value", mysqlOutfileRowSep) + } + format.MysqlOut.RowSeparator = c + } + + if override, ok := opts[mysqlOutfileFieldSep]; ok { + c, err := util.GetSingleRune(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid %q value", mysqlOutfileFieldSep) + } + format.MysqlOut.FieldSeparator = c + } + + if override, ok := opts[mysqlOutfileEnclose]; ok { + c, err := util.GetSingleRune(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid %q value", mysqlOutfileRowSep) + } + format.MysqlOut.Enclose = roachpb.MySQLOutfileOptions_Always + format.MysqlOut.Encloser = c + } + + if override, ok := opts[mysqlOutfileEscape]; ok { + c, err := util.GetSingleRune(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid %q value", mysqlOutfileRowSep) + } + format.MysqlOut.HasEscape = true + format.MysqlOut.Escape = c + } + if override, ok := opts[csvSkip]; ok { + skip, err := strconv.Atoi(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid %s value", csvSkip) + } + if skip < 0 { + return pgerror.Newf(pgcode.Syntax, "%s must be >= 0", csvSkip) + } + format.MysqlOut.Skip = uint32(skip) + } + if override, ok := opts[csvNullIf]; ok { + format.MysqlOut.NullEncoding = &override + } + if _, ok := opts[importOptionSaveRejected]; ok { + format.SaveRejected = true + } + if override, ok := opts[csvRowLimit]; ok { + rowLimit, err := strconv.Atoi(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) + } + if rowLimit <= 0 { + return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) + } + format.MysqlOut.RowLimit = int64(rowLimit) + } + case "MYSQLDUMP": + if err = validateFormatOptions(importStmt.FileFormat, opts, mysqlDumpAllowedOptions); err != nil { + return err + } + format.Format = roachpb.IOFileFormat_Mysqldump + if override, ok := opts[csvRowLimit]; ok { + rowLimit, err := strconv.Atoi(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) + } + if rowLimit <= 0 { + return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) + } + format.MysqlDump.RowLimit = int64(rowLimit) + } + case "PGCOPY": + if err = validateFormatOptions(importStmt.FileFormat, opts, pgCopyAllowedOptions); err != nil { + return err + } + format.Format = roachpb.IOFileFormat_PgCopy + format.PgCopy = roachpb.PgCopyOptions{ + Delimiter: '\t', + Null: `\N`, + } + if override, ok := opts[pgCopyDelimiter]; ok { + c, err := util.GetSingleRune(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid %q value", pgCopyDelimiter) + } + format.PgCopy.Delimiter = c + } + if override, ok := opts[pgCopyNull]; ok { + format.PgCopy.Null = override + } + maxRowSize := int32(defaultScanBuffer) + if override, ok := opts[optMaxRowSize]; ok { + sz, err := humanizeutil.ParseBytes(override) + if err != nil { + return err + } + if sz < 1 || sz > math.MaxInt32 { + return errors.Errorf("%d out of range: %d", maxRowSize, sz) + } + maxRowSize = int32(sz) + } + format.PgCopy.MaxRowSize = maxRowSize + case "PGDUMP": + if err = validateFormatOptions(importStmt.FileFormat, opts, pgDumpAllowedOptions); err != nil { + return err + } + format.Format = roachpb.IOFileFormat_PgDump + maxRowSize := int32(defaultScanBuffer) + if override, ok := opts[optMaxRowSize]; ok { + sz, err := humanizeutil.ParseBytes(override) + if err != nil { + return err + } + if sz < 1 || sz > math.MaxInt32 { + return errors.Errorf("%d out of range: %d", maxRowSize, sz) + } + maxRowSize = int32(sz) + } + format.PgDump.MaxRowSize = maxRowSize + if _, ok := opts[pgDumpIgnoreAllUnsupported]; ok { + format.PgDump.IgnoreUnsupported = true + } + + if dest, ok := opts[pgDumpIgnoreShuntFileDest]; ok { + if !format.PgDump.IgnoreUnsupported { + return errors.New("cannot log unsupported PGDUMP stmts without `ignore_unsupported_statements` option") + } + format.PgDump.IgnoreUnsupportedLog = dest + } + + if override, ok := opts[csvRowLimit]; ok { + rowLimit, err := strconv.Atoi(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) + } + if rowLimit <= 0 { + return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) + } + format.PgDump.RowLimit = int64(rowLimit) + } + case "AVRO": + if err = validateFormatOptions(importStmt.FileFormat, opts, avroAllowedOptions); err != nil { + return err + } + err := parseAvroOptions(ctx, opts, p, &format) + if err != nil { + return err + } + default: + return unimplemented.Newf("import.format", "unsupported import format: %q", importStmt.FileFormat) + } + + // sstSize, if 0, will be set to an appropriate default by the specific + // implementation (local or distributed) since each has different optimal + // settings. + var sstSize int64 + if override, ok := opts[importOptionSSTSize]; ok { + sz, err := humanizeutil.ParseBytes(override) + if err != nil { + return err + } + sstSize = sz + } + var oversample int64 + if override, ok := opts[importOptionOversample]; ok { + os, err := strconv.ParseInt(override, 10, 64) + if err != nil { + return err + } + oversample = os + } + + var skipFKs bool + if _, ok := opts[importOptionSkipFKs]; ok { + skipFKs = true + } + + if override, ok := opts[importOptionDecompress]; ok { + found := false + for name, value := range roachpb.IOFileFormat_Compression_value { + if strings.EqualFold(name, override) { + format.Compression = roachpb.IOFileFormat_Compression(value) + found = true + break + } + } + if !found { + return unimplemented.Newf("import.compression", "unsupported compression value: %q", override) + } + } + + var tableDetails []jobspb.ImportDetails_Table + var tableDescs []*tabledesc.Mutable // parallel with tableDetails + var typeDetails []jobspb.ImportDetails_Type + jobDesc, err := importJobDescription(p, importStmt, nil, filenamePatterns, opts) + if err != nil { + return err + } + + if importStmt.Into { + if _, ok := allowedIntoFormats[importStmt.FileFormat]; !ok { + return errors.Newf( + "%s file format is currently unsupported by IMPORT INTO", + importStmt.FileFormat) + } + _, found, err := p.ResolveMutableTableDescriptor(ctx, table, true, tree.ResolveRequireTableDesc) + if err != nil { + return err + } + + err = ensureRequiredPrivileges(ctx, importIntoRequiredPrivileges, p, found) + if err != nil { + return err + } + + // IMPORT INTO does not currently support interleaved tables. + if found.IsInterleaved() { + // TODO(miretskiy): Handle import into when tables are interleaved. + return pgerror.New(pgcode.FeatureNotSupported, "Cannot use IMPORT INTO with interleaved tables") + } + + // Validate target columns. + var intoCols []string + var isTargetCol = make(map[string]bool) + for _, name := range importStmt.IntoCols { + active, err := tabledesc.FindPublicColumnsWithNames(found, tree.NameList{name}) + if err != nil { + return errors.Wrap(err, "verifying target columns") + } + + isTargetCol[active[0].GetName()] = true + intoCols = append(intoCols, active[0].GetName()) + } + + // Ensure that non-target columns that don't have default + // expressions are nullable. + if len(isTargetCol) != 0 { + for _, col := range found.VisibleColumns() { + if !(isTargetCol[col.GetName()] || col.IsNullable() || col.HasDefault() || col.IsComputed()) { + return errors.Newf( + "all non-target columns in IMPORT INTO must be nullable "+ + "or have default expressions, or have computed expressions"+ + " but violated by column %q", + col.GetName(), + ) + } + if isTargetCol[col.GetName()] && col.IsComputed() { + return schemaexpr.CannotWriteToComputedColError(col.GetName()) + } + } + } + + { + // Resolve the UDTs used by the table being imported into. + typeDescs, err := resolveUDTsUsedByImportInto(ctx, p, found) + if err != nil { + return errors.Wrap(err, "resolving UDTs used by table being imported into") + } + if len(typeDescs) > 0 { + typeDetails = make([]jobspb.ImportDetails_Type, 0, len(typeDescs)) + } + for _, typeDesc := range typeDescs { + typeDetails = append(typeDetails, jobspb.ImportDetails_Type{Desc: typeDesc.TypeDesc()}) + } + } + + tableDetails = []jobspb.ImportDetails_Table{{Desc: &found.TableDescriptor, IsNew: false, TargetCols: intoCols}} + } else { + seqVals := make(map[descpb.ID]int64) + + if importStmt.Bundle { + // If we target a single table, populate details with one entry of tableName. + if table != nil { + tableDetails = make([]jobspb.ImportDetails_Table, 1) + tableName := table.ObjectName.String() + // PGDUMP supports importing tables from non-public schemas, thus we + // must prepend the target table name with the target schema name. + if format.Format == roachpb.IOFileFormat_PgDump { + if table.Schema() == "" { + return errors.Newf("expected schema for target table %s to be resolved", + tableName) + } + tableName = fmt.Sprintf("%s.%s", table.SchemaName.String(), + table.ObjectName.String()) + } + tableDetails[0] = jobspb.ImportDetails_Table{ + Name: tableName, + IsNew: true, + } + } + } else { + if table == nil { + return errors.Errorf("non-bundle format %q should always have a table name", importStmt.FileFormat) + } + var create *tree.CreateTable + if importStmt.CreateDefs != nil { + create = &tree.CreateTable{ + Table: *importStmt.Table, + Defs: importStmt.CreateDefs, + } + } else { + filename, err := createFileFn() + if err != nil { + return err + } + create, err = readCreateTableFromStore(ctx, filename, + p.ExecCfg().DistSQLSrv.ExternalStorageFromURI, p.User()) + if err != nil { + return err + } + + if table.ObjectName != create.Table.ObjectName { + return errors.Errorf( + "importing table %s, but file specifies a schema for table %s", + table.ObjectName, create.Table.ObjectName, + ) + } + } + if create.Locality != nil && + create.Locality.LocalityLevel == tree.LocalityLevelRow { + return unimplemented.NewWithIssueDetailf( + 61133, + "import.regional-by-row", + "IMPORT to REGIONAL BY ROW table not supported", + ) + } + // IMPORT TABLE do not support user defined types, and so we nil out the + // type resolver to protect against unexpected behavior on UDT + // resolution. + semaCtxPtr := makeSemaCtxWithoutTypeResolver(p.SemaCtx()) + tbl, err := MakeSimpleTableDescriptor( + ctx, semaCtxPtr, p.ExecCfg().Settings, create, db, sc, defaultCSVTableID, NoFKs, walltime) + if err != nil { + return err + } + descStr, err := importJobDescription(p, importStmt, create.Defs, filenamePatterns, opts) + if err != nil { + return err + } + jobDesc = descStr + + tableDescs = []*tabledesc.Mutable{tbl} + for _, tbl := range tableDescs { + // For reasons relating to #37691, we disallow user defined types in + // the standard IMPORT case. + for _, col := range tbl.Columns { + if col.Type.UserDefined() { + return errors.Newf("IMPORT cannot be used with user defined types; use IMPORT INTO instead") + } + } + } + + tableDetails = make([]jobspb.ImportDetails_Table, len(tableDescs)) + for i := range tableDescs { + tableDetails[i] = jobspb.ImportDetails_Table{ + Desc: tableDescs[i].TableDesc(), + SeqVal: seqVals[tableDescs[i].ID], + IsNew: true, + } + } + } + + // Due to how we generate and rewrite descriptor ID's for import, we run + // into problems when using user defined schemas. + if sc.GetID() != keys.PublicSchemaID { + err := errors.New("cannot use IMPORT with a user defined schema") + hint := errors.WithHint(err, "create the table with CREATE TABLE and use IMPORT INTO instead") + return hint + } + } + + // Store the primary region of the database being imported into. This is + // used during job execution to evaluate certain default expressions and + // computed columns such as `gateway_region`. + var databasePrimaryRegion descpb.RegionName + if db.IsMultiRegion() { + if err := sql.DescsTxn(ctx, p.ExecCfg(), func(ctx context.Context, txn *kv.Txn, + descsCol *descs.Collection) error { + regionConfig, err := sql.SynthesizeRegionConfig(ctx, txn, db.GetID(), descsCol) + if err != nil { + return err + } + databasePrimaryRegion = regionConfig.PrimaryRegion() + return nil + }); err != nil { + return errors.Wrap(err, "failed to resolve region config for multi region database") + } + } + + telemetry.CountBucketed("import.files", int64(len(files))) + + // Record telemetry for userfile being used as the import target. + for _, file := range files { + uri, err := url.Parse(file) + // This should never be true as we have parsed these file names in an + // earlier step of import. + if err != nil { + log.Warningf(ctx, "failed to collect file specific import telemetry for %s", uri) + continue + } + + if uri.Scheme == "userfile" { + telemetry.Count("import.storage.userfile") + break + } + } + if importStmt.Into { + telemetry.Count("import.into") + } + + // Here we create the job in a side transaction and then kick off the job. + // This is awful. Rather we should be disallowing this statement in an + // explicit transaction and then we should create the job in the user's + // transaction here and then in a post-commit hook we should kick of the + // StartableJob which we attached to the connExecutor somehow. + + importDetails := jobspb.ImportDetails{ + URIs: files, + Format: format, + ParentID: db.GetID(), + Tables: tableDetails, + Types: typeDetails, + SSTSize: sstSize, + Oversample: oversample, + SkipFKs: skipFKs, + ParseBundleSchema: importStmt.Bundle, + DefaultIntSize: p.SessionData().DefaultIntSize, + DatabasePrimaryRegion: databasePrimaryRegion, + } + + jr := jobs.Record{ + Description: jobDesc, + Username: p.User(), + Details: importDetails, + Progress: jobspb.ImportProgress{}, + } + + if isDetached { + // When running inside an explicit transaction, we simply create the job + // record. We do not wait for the job to finish. + jobID := p.ExecCfg().JobRegistry.MakeJobID() + _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( + ctx, jr, jobID, p.ExtendedEvalContext().Txn) + if err != nil { + return err + } + + addToFileFormatTelemetry(format.Format.String(), "started") + resultsCh <- tree.Datums{tree.NewDInt(tree.DInt(jobID))} + return nil + } + + // We create the job record in the planner's transaction to ensure that + // the job record creation happens transactionally. + plannerTxn := p.ExtendedEvalContext().Txn + + // Construct the job and commit the transaction. Perform this work in a + // closure to ensure that the job is cleaned up if an error occurs. + var sj *jobs.StartableJob + if err := func() (err error) { + defer func() { + if err == nil || sj == nil { + return + } + if cleanupErr := sj.CleanupOnRollback(ctx); cleanupErr != nil { + log.Errorf(ctx, "failed to cleanup job: %v", cleanupErr) + } + }() + jobID := p.ExecCfg().JobRegistry.MakeJobID() + if err := p.ExecCfg().JobRegistry.CreateStartableJobWithTxn(ctx, &sj, jobID, plannerTxn, jr); err != nil { + return err + } + + // We commit the transaction here so that the job can be started. This + // is safe because we're in an implicit transaction. If we were in an + // explicit transaction the job would have to be run with the detached + // option and would have been handled above. + return plannerTxn.Commit(ctx) + }(); err != nil { + return err + } + + if err := sj.Start(ctx); err != nil { + return err + } + addToFileFormatTelemetry(format.Format.String(), "started") + if err := sj.AwaitCompletion(ctx); err != nil { + return err + } + return sj.ReportExecutionResults(ctx, resultsCh) + } + + if isDetached { + return fn, utilccl.DetachedJobExecutionResultHeader, nil, false, nil + } + return fn, utilccl.BulkJobExecutionResultHeader, nil, false, nil +} + +func parseAvroOptions( + ctx context.Context, opts map[string]string, p sql.PlanHookState, format *roachpb.IOFileFormat, +) error { + format.Format = roachpb.IOFileFormat_Avro + // Default input format is OCF. + format.Avro.Format = roachpb.AvroOptions_OCF + _, format.Avro.StrictMode = opts[avroStrict] + + _, haveBinRecs := opts[avroBinRecords] + _, haveJSONRecs := opts[avroJSONRecords] + + if haveBinRecs && haveJSONRecs { + return errors.Errorf("only one of the %s or %s options can be set", avroBinRecords, avroJSONRecords) + } + + if override, ok := opts[csvRowLimit]; ok { + rowLimit, err := strconv.Atoi(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, "invalid numeric %s value", csvRowLimit) + } + if rowLimit <= 0 { + return pgerror.Newf(pgcode.Syntax, "%s must be > 0", csvRowLimit) + } + format.Avro.RowLimit = int64(rowLimit) + } + + if haveBinRecs || haveJSONRecs { + // Input is a "records" format. + if haveBinRecs { + format.Avro.Format = roachpb.AvroOptions_BIN_RECORDS + } else { + format.Avro.Format = roachpb.AvroOptions_JSON_RECORDS + } + + // Set record separator. + format.Avro.RecordSeparator = '\n' + if override, ok := opts[avroRecordsSeparatedBy]; ok { + c, err := util.GetSingleRune(override) + if err != nil { + return pgerror.Wrapf(err, pgcode.Syntax, + "invalid %q value", avroRecordsSeparatedBy) + } + format.Avro.RecordSeparator = c + } + + // See if inline schema is specified. + format.Avro.SchemaJSON = opts[avroSchema] + + if len(format.Avro.SchemaJSON) == 0 { + // Inline schema not set; We must have external schema. + uri, ok := opts[avroSchemaURI] + if !ok { + return errors.Errorf( + "either %s or %s option must be set when importing avro record files", avroSchema, avroSchemaURI) + } + + store, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, uri, p.User()) + if err != nil { + return err + } + defer store.Close() + + raw, err := store.ReadFile(ctx, "") + if err != nil { + return err + } + defer raw.Close() + schemaBytes, err := ioutil.ReadAll(raw) + if err != nil { + return err + } + format.Avro.SchemaJSON = string(schemaBytes) + } + + if override, ok := opts[optMaxRowSize]; ok { + sz, err := humanizeutil.ParseBytes(override) + if err != nil { + return err + } + if sz < 1 || sz > math.MaxInt32 { + return errors.Errorf("%s out of range: %d", override, sz) + } + format.Avro.MaxRecordSize = int32(sz) + } + } + return nil +} + +type loggerKind int + +const ( + schemaParsing loggerKind = iota + dataIngestion +) + +// unsupportedStmtLogger is responsible for handling unsupported PGDUMP SQL +// statements seen during the import. +type unsupportedStmtLogger struct { + ctx context.Context + user security.SQLUsername + jobID int64 + + // Values are initialized based on the options specified in the IMPORT PGDUMP + // stmt. + ignoreUnsupported bool + ignoreUnsupportedLogDest string + externalStorage cloud.ExternalStorageFactory + + // logBuffer holds the string to be flushed to the ignoreUnsupportedLogDest. + logBuffer *bytes.Buffer + numIgnoredStmts int + + // Incremented every time the logger flushes. It is used as the suffix of the + // log file written to external storage. + flushCount int + + loggerType loggerKind +} + +func makeUnsupportedStmtLogger( + ctx context.Context, + user security.SQLUsername, + jobID int64, + ignoreUnsupported bool, + unsupportedLogDest string, + loggerType loggerKind, + externalStorage cloud.ExternalStorageFactory, +) *unsupportedStmtLogger { + return &unsupportedStmtLogger{ + ctx: ctx, + user: user, + jobID: jobID, + ignoreUnsupported: ignoreUnsupported, + ignoreUnsupportedLogDest: unsupportedLogDest, + loggerType: loggerType, + logBuffer: new(bytes.Buffer), + externalStorage: externalStorage, + } +} + +func (u *unsupportedStmtLogger) log(logLine string, isParseError bool) error { + // We have already logged parse errors during the schema ingestion phase, so + // skip them to avoid duplicate entries. + skipLoggingParseErr := isParseError && u.loggerType == dataIngestion + if u.ignoreUnsupportedLogDest == "" || skipLoggingParseErr { + return nil + } + + // Flush to a file if we have hit the max size of our buffer. + if u.numIgnoredStmts >= pgDumpMaxLoggedStmts { + err := u.flush() + if err != nil { + return err + } + } + + if isParseError { + logLine = fmt.Sprintf("%s: could not be parsed\n", logLine) + } else { + logLine = fmt.Sprintf("%s: unsupported by IMPORT\n", logLine) + } + u.logBuffer.Write([]byte(logLine)) + u.numIgnoredStmts++ + return nil +} + +func (u *unsupportedStmtLogger) flush() error { + if u.ignoreUnsupportedLogDest == "" { + return nil + } + + conf, err := cloud.ExternalStorageConfFromURI(u.ignoreUnsupportedLogDest, u.user) + if err != nil { + return errors.Wrap(err, "failed to log unsupported stmts during IMPORT PGDUMP") + } + var s cloud.ExternalStorage + if s, err = u.externalStorage(u.ctx, conf); err != nil { + return errors.New("failed to log unsupported stmts during IMPORT PGDUMP") + } + defer s.Close() + + logFileName := fmt.Sprintf("import%d", u.jobID) + if u.loggerType == dataIngestion { + logFileName = path.Join(logFileName, pgDumpUnsupportedDataStmtLog, fmt.Sprintf("%d.log", u.flushCount)) + } else { + logFileName = path.Join(logFileName, pgDumpUnsupportedSchemaStmtLog, fmt.Sprintf("%d.log", u.flushCount)) + } + err = cloud.WriteFile(u.ctx, s, logFileName, bytes.NewReader(u.logBuffer.Bytes())) + if err != nil { + return errors.Wrap(err, "failed to log unsupported stmts to log during IMPORT PGDUMP") + } + u.flushCount++ + u.numIgnoredStmts = 0 + u.logBuffer.Truncate(0) + return nil +} + +func init() { + sql.AddPlanHook(importPlanHook) +} diff --git a/pkg/sql/distsql_plan_csv.go b/pkg/ccl/importccl/import_processor_planning.go similarity index 78% rename from pkg/sql/distsql_plan_csv.go rename to pkg/ccl/importccl/import_processor_planning.go index 4f5bc262adc9..a5568afb93c9 100644 --- a/pkg/sql/distsql_plan_csv.go +++ b/pkg/ccl/importccl/import_processor_planning.go @@ -1,14 +1,12 @@ // Copyright 2017 The Cockroach Authors. // -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. +// 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 // -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package sql +package importccl import ( "context" @@ -21,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -35,168 +34,13 @@ import ( "github.com/cockroachdb/logtags" ) -// RowResultWriter is a thin wrapper around a RowContainer. -type RowResultWriter struct { - rowContainer *rowContainerHelper - rowsAffected int - err error -} - -var _ rowResultWriter = &RowResultWriter{} - -// NewRowResultWriter creates a new RowResultWriter. -func NewRowResultWriter(rowContainer *rowContainerHelper) *RowResultWriter { - return &RowResultWriter{rowContainer: rowContainer} -} - -// IncrementRowsAffected implements the rowResultWriter interface. -func (b *RowResultWriter) IncrementRowsAffected(ctx context.Context, n int) { - b.rowsAffected += n -} - -// AddRow implements the rowResultWriter interface. -func (b *RowResultWriter) AddRow(ctx context.Context, row tree.Datums) error { - if b.rowContainer != nil { - return b.rowContainer.addRow(ctx, row) - } - return nil -} - -// SetError is part of the rowResultWriter interface. -func (b *RowResultWriter) SetError(err error) { - b.err = err -} - -// Err is part of the rowResultWriter interface. -func (b *RowResultWriter) Err() error { - return b.err -} - -// callbackResultWriter is a rowResultWriter that runs a callback function -// on AddRow. -type callbackResultWriter struct { - fn func(ctx context.Context, row tree.Datums) error - rowsAffected int - err error -} - -var _ rowResultWriter = &callbackResultWriter{} - -// newCallbackResultWriter creates a new callbackResultWriter. -func newCallbackResultWriter( - fn func(ctx context.Context, row tree.Datums) error, -) *callbackResultWriter { - return &callbackResultWriter{fn: fn} -} - -func (c *callbackResultWriter) IncrementRowsAffected(ctx context.Context, n int) { - c.rowsAffected += n -} - -func (c *callbackResultWriter) AddRow(ctx context.Context, row tree.Datums) error { - return c.fn(ctx, row) -} - -func (c *callbackResultWriter) SetError(err error) { - c.err = err -} - -func (c *callbackResultWriter) Err() error { - return c.err -} - -func getLastImportSummary(job *jobs.Job) roachpb.BulkOpSummary { - progress := job.Progress() - importProgress := progress.GetImport() - return importProgress.Summary -} - -func makeImportReaderSpecs( - job *jobs.Job, - tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable, - typeDescs []*descpb.TypeDescriptor, - from []string, - format roachpb.IOFileFormat, - nodes []roachpb.NodeID, - walltime int64, - user security.SQLUsername, -) []*execinfrapb.ReadImportDataSpec { - details := job.Details().(jobspb.ImportDetails) - // For each input file, assign it to a node. - inputSpecs := make([]*execinfrapb.ReadImportDataSpec, 0, len(nodes)) - progress := job.Progress() - importProgress := progress.GetImport() - for i, input := range from { - // Round robin assign CSV files to nodes. Files 0 through len(nodes)-1 - // creates the spec. Future files just add themselves to the Uris. - if i < len(nodes) { - spec := &execinfrapb.ReadImportDataSpec{ - Tables: tables, - Types: typeDescs, - Format: format, - Progress: execinfrapb.JobProgress{ - JobID: job.ID(), - Slot: int32(i), - }, - WalltimeNanos: walltime, - Uri: make(map[int32]string), - ResumePos: make(map[int32]int64), - UserProto: user.EncodeProto(), - DatabasePrimaryRegion: details.DatabasePrimaryRegion, - } - inputSpecs = append(inputSpecs, spec) - } - n := i % len(nodes) - inputSpecs[n].Uri[int32(i)] = input - if importProgress.ResumePos != nil { - inputSpecs[n].ResumePos[int32(i)] = importProgress.ResumePos[int32(i)] - } - } - - for i := range inputSpecs { - // TODO(mjibson): using the actual file sizes here would improve progress - // accuracy. - inputSpecs[i].Progress.Contribution = float32(len(inputSpecs[i].Uri)) / float32(len(from)) - } - return inputSpecs -} - -func presplitTableBoundaries( - ctx context.Context, - cfg *ExecutorConfig, - tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable, -) error { - var span *tracing.Span - ctx, span = tracing.ChildSpan(ctx, "import-pre-splitting-table-boundaries") - defer span.Finish() - expirationTime := cfg.DB.Clock().Now().Add(time.Hour.Nanoseconds(), 0) - for _, tbl := range tables { - // TODO(ajwerner): Consider passing in the wrapped descriptors. - tblDesc := tabledesc.NewBuilder(tbl.Desc).BuildImmutableTable() - for _, span := range tblDesc.AllIndexSpans(cfg.Codec) { - if err := cfg.DB.AdminSplit(ctx, span.Key, expirationTime); err != nil { - return err - } - - log.VEventf(ctx, 1, "scattering index range %s", span.Key) - scatterReq := &roachpb.AdminScatterRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(span), - } - if _, pErr := kv.SendWrapped(ctx, cfg.DB.NonTransactionalSender(), scatterReq); pErr != nil { - log.Errorf(ctx, "failed to scatter span %s: %s", span.Key, pErr) - } - } - } - return nil -} - -// DistIngest is used by IMPORT to run a DistSQL flow to ingest data by starting +// distImport is used by IMPORT to run a DistSQL flow to ingest data by starting // reader processes on many nodes that each read and ingest their assigned files // and then send back a summary of what they ingested. The combined summary is // returned. -func DistIngest( +func distImport( ctx context.Context, - execCtx JobExecContext, + execCtx sql.JobExecContext, job *jobs.Job, tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable, typeDescs []*descpb.TypeDescriptor, @@ -318,7 +162,7 @@ func DistIngest( } var res roachpb.BulkOpSummary - rowResultWriter := newCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { + rowResultWriter := sql.NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { var counts roachpb.BulkOpSummary if err := protoutil.Unmarshal([]byte(*row[0].(*tree.DBytes)), &counts); err != nil { return err @@ -333,9 +177,9 @@ func DistIngest( } } - recv := MakeDistSQLReceiver( + recv := sql.MakeDistSQLReceiver( ctx, - &MetadataCallbackWriter{rowResultWriter: rowResultWriter, fn: metaFn}, + sql.NewMetadataCallbackWriter(rowResultWriter, metaFn), tree.Rows, nil, /* rangeCache */ nil, /* txn - the flow does not read or write the database */ @@ -380,3 +224,88 @@ func DistIngest( return res, nil } + +func getLastImportSummary(job *jobs.Job) roachpb.BulkOpSummary { + progress := job.Progress() + importProgress := progress.GetImport() + return importProgress.Summary +} + +func makeImportReaderSpecs( + job *jobs.Job, + tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable, + typeDescs []*descpb.TypeDescriptor, + from []string, + format roachpb.IOFileFormat, + nodes []roachpb.NodeID, + walltime int64, + user security.SQLUsername, +) []*execinfrapb.ReadImportDataSpec { + details := job.Details().(jobspb.ImportDetails) + // For each input file, assign it to a node. + inputSpecs := make([]*execinfrapb.ReadImportDataSpec, 0, len(nodes)) + progress := job.Progress() + importProgress := progress.GetImport() + for i, input := range from { + // Round robin assign CSV files to nodes. Files 0 through len(nodes)-1 + // creates the spec. Future files just add themselves to the Uris. + if i < len(nodes) { + spec := &execinfrapb.ReadImportDataSpec{ + Tables: tables, + Types: typeDescs, + Format: format, + Progress: execinfrapb.JobProgress{ + JobID: job.ID(), + Slot: int32(i), + }, + WalltimeNanos: walltime, + Uri: make(map[int32]string), + ResumePos: make(map[int32]int64), + UserProto: user.EncodeProto(), + DatabasePrimaryRegion: details.DatabasePrimaryRegion, + } + inputSpecs = append(inputSpecs, spec) + } + n := i % len(nodes) + inputSpecs[n].Uri[int32(i)] = input + if importProgress.ResumePos != nil { + inputSpecs[n].ResumePos[int32(i)] = importProgress.ResumePos[int32(i)] + } + } + + for i := range inputSpecs { + // TODO(mjibson): using the actual file sizes here would improve progress + // accuracy. + inputSpecs[i].Progress.Contribution = float32(len(inputSpecs[i].Uri)) / float32(len(from)) + } + return inputSpecs +} + +func presplitTableBoundaries( + ctx context.Context, + cfg *sql.ExecutorConfig, + tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable, +) error { + var span *tracing.Span + ctx, span = tracing.ChildSpan(ctx, "import-pre-splitting-table-boundaries") + defer span.Finish() + expirationTime := cfg.DB.Clock().Now().Add(time.Hour.Nanoseconds(), 0) + for _, tbl := range tables { + // TODO(ajwerner): Consider passing in the wrapped descriptors. + tblDesc := tabledesc.NewBuilder(tbl.Desc).BuildImmutableTable() + for _, span := range tblDesc.AllIndexSpans(cfg.Codec) { + if err := cfg.DB.AdminSplit(ctx, span.Key, expirationTime); err != nil { + return err + } + + log.VEventf(ctx, 1, "scattering index range %s", span.Key) + scatterReq := &roachpb.AdminScatterRequest{ + RequestHeader: roachpb.RequestHeaderFromSpan(span), + } + if _, pErr := kv.SendWrapped(ctx, cfg.DB.NonTransactionalSender(), scatterReq); pErr != nil { + log.Errorf(ctx, "failed to scatter span %s: %s", span.Key, pErr) + } + } + } + return nil +} diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index ffc654974093..55188f8fb0bc 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -70,7 +70,6 @@ go_library( "distsql_physical_planner.go", "distsql_plan_backfill.go", "distsql_plan_bulk.go", - "distsql_plan_csv.go", "distsql_plan_ctas.go", "distsql_plan_join.go", "distsql_plan_scrub_physical.go", diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 2f669b2aba69..9e2c62b73d7f 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -710,6 +710,80 @@ func (w *errOnlyResultWriter) IncrementRowsAffected(ctx context.Context, n int) panic("IncrementRowsAffected not supported by errOnlyResultWriter") } +// RowResultWriter is a thin wrapper around a RowContainer. +type RowResultWriter struct { + rowContainer *rowContainerHelper + rowsAffected int + err error +} + +var _ rowResultWriter = &RowResultWriter{} + +// NewRowResultWriter creates a new RowResultWriter. +func NewRowResultWriter(rowContainer *rowContainerHelper) *RowResultWriter { + return &RowResultWriter{rowContainer: rowContainer} +} + +// IncrementRowsAffected implements the rowResultWriter interface. +func (b *RowResultWriter) IncrementRowsAffected(ctx context.Context, n int) { + b.rowsAffected += n +} + +// AddRow implements the rowResultWriter interface. +func (b *RowResultWriter) AddRow(ctx context.Context, row tree.Datums) error { + if b.rowContainer != nil { + return b.rowContainer.addRow(ctx, row) + } + return nil +} + +// SetError is part of the rowResultWriter interface. +func (b *RowResultWriter) SetError(err error) { + b.err = err +} + +// Err is part of the rowResultWriter interface. +func (b *RowResultWriter) Err() error { + return b.err +} + +// CallbackResultWriter is a rowResultWriter that runs a callback function +// on AddRow. +type CallbackResultWriter struct { + fn func(ctx context.Context, row tree.Datums) error + rowsAffected int + err error +} + +var _ rowResultWriter = &CallbackResultWriter{} + +// NewCallbackResultWriter creates a new CallbackResultWriter. +func NewCallbackResultWriter( + fn func(ctx context.Context, row tree.Datums) error, +) *CallbackResultWriter { + return &CallbackResultWriter{fn: fn} +} + +// IncrementRowsAffected is part of the rowResultWriter interface. +func (c *CallbackResultWriter) IncrementRowsAffected(ctx context.Context, n int) { + c.rowsAffected += n +} + +// AddRow is part of the rowResultWriter interface. +func (c *CallbackResultWriter) AddRow(ctx context.Context, row tree.Datums) error { + return c.fn(ctx, row) +} + +// SetError is part of the rowResultWriter interface. +func (c *CallbackResultWriter) SetError(err error) { + c.err = err +} + +// Err is part of the rowResultWriter interface. +func (c *CallbackResultWriter) Err() error { + return c.err +} + var _ execinfra.RowReceiver = &DistSQLReceiver{} var _ execinfra.BatchReceiver = &DistSQLReceiver{} diff --git a/pkg/sql/distsql_running_test.go b/pkg/sql/distsql_running_test.go index 66d0e98eb8d8..1044a67edea0 100644 --- a/pkg/sql/distsql_running_test.go +++ b/pkg/sql/distsql_running_test.go @@ -148,7 +148,7 @@ func TestDistSQLRunningInAbortedTxn(t *testing.T) { } // Create and run a DistSQL plan. - rw := newCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { + rw := NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { return nil }) recv := MakeDistSQLReceiver( diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index cc04e4187e68..5db817d117ff 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -296,7 +296,7 @@ func (sc *SchemaChanger) backfillQueryIntoTable( defer localPlanner.curPlan.close(ctx) res := roachpb.BulkOpSummary{} - rw := newCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { + rw := NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { // TODO(adityamaru): Use the BulkOpSummary for either telemetry or to // return to user. var counts roachpb.BulkOpSummary diff --git a/pkg/sql/testutils.go b/pkg/sql/testutils.go index 47ba7a3ea041..68b063cee571 100644 --- a/pkg/sql/testutils.go +++ b/pkg/sql/testutils.go @@ -127,7 +127,7 @@ func (dsp *DistSQLPlanner) Exec( if err := p.makeOptimizerPlan(ctx); err != nil { return err } - rw := newCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { + rw := NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error { return nil }) execCfg := p.ExecCfg()