Skip to content

Commit c9b1c1e

Browse files
committed
tree: remove unused Into and Bundle fields of Import
Also update one spot under assumption that `tree.Import.Table` is now always non-nil. Release note: None
1 parent 367b51a commit c9b1c1e

File tree

8 files changed

+121
-224
lines changed

8 files changed

+121
-224
lines changed

pkg/internal/sqlsmith/bulkio.go

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -258,7 +258,6 @@ func makeImport(s *Smither) (tree.Statement, bool) {
258258

259259
return &tree.Import{
260260
Table: tree.NewUnqualifiedTableName(tab),
261-
Into: true,
262261
FileFormat: "CSV",
263262
Files: files,
264263
Options: tree.KVOptions{

pkg/jobs/jobspb/jobs.proto

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -714,6 +714,7 @@ message ImportDetails {
714714
bool schemas_published = 24;
715715
bool tables_published = 13;
716716

717+
// TODO(yuzefovich): remove this.
717718
bool parse_bundle_schema = 14;
718719

719720
// DefaultIntSize is the integer type that a "naked" int will be resolved

pkg/sql/importer/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,6 @@ go_library(
3838
"//pkg/jobs/joberror",
3939
"//pkg/jobs/jobspb",
4040
"//pkg/jobs/jobsprofiler",
41-
"//pkg/keys",
4241
"//pkg/kv",
4342
"//pkg/kv/kvpb",
4443
"//pkg/kv/kvserver/kvserverbase",

pkg/sql/importer/import_planning.go

Lines changed: 91 additions & 166 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@ import (
2121
"github.com/cockroachdb/cockroach/pkg/featureflag"
2222
"github.com/cockroachdb/cockroach/pkg/jobs"
2323
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
24-
"github.com/cockroachdb/cockroach/pkg/keys"
2524
"github.com/cockroachdb/cockroach/pkg/roachpb"
2625
"github.com/cockroachdb/cockroach/pkg/security/username"
2726
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
@@ -33,7 +32,6 @@ import (
3332
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
3433
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
3534
"github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver"
36-
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc"
3735
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
3836
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
3937
"github.com/cockroachdb/cockroach/pkg/sql/exprutil"
@@ -42,7 +40,6 @@ import (
4240
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
4341
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
4442
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
45-
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
4643
"github.com/cockroachdb/cockroach/pkg/sql/sem/idxtype"
4744
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
4845
"github.com/cockroachdb/cockroach/pkg/util"
@@ -346,10 +343,6 @@ func importPlanHook(
346343
return nil, nil, false, nil
347344
}
348345

349-
if !importStmt.Bundle && !importStmt.Into {
350-
p.BufferClientNotice(ctx, pgnotice.Newf("IMPORT TABLE has been deprecated in 21.2, and will be removed in a future version."+
351-
" Instead, use CREATE TABLE with the desired schema, and IMPORT INTO the newly created table."))
352-
}
353346
switch f := strings.ToUpper(importStmt.FileFormat); f {
354347
case "PGDUMP", "MYSQLDUMP":
355348
p.BufferClientNotice(ctx, pgnotice.Newf(
@@ -456,64 +449,30 @@ func importPlanHook(
456449
}
457450
}
458451

459-
// Typically the SQL grammar means it is only possible to specifying exactly
460-
// one pgdump/mysqldump URI, but glob-expansion could have changed that.
461-
if importStmt.Bundle && len(files) != 1 {
462-
return pgerror.New(pgcode.FeatureNotSupported, "SQL dump files must be imported individually")
463-
}
464-
465452
table := importStmt.Table
466-
var db catalog.DatabaseDescriptor
467-
var sc catalog.SchemaDescriptor
468-
if table != nil {
469-
// TODO: As part of work for #34240, we should be operating on
470-
// UnresolvedObjectNames here, rather than TableNames.
471-
// We have a target table, so it might specify a DB in its name.
472-
un := table.ToUnresolvedObjectName()
473-
found, prefix, resPrefix, err := resolver.ResolveTarget(ctx,
474-
un, p, p.SessionData().Database, p.SessionData().SearchPath)
475-
if err != nil {
476-
return pgerror.Wrap(err, pgcode.UndefinedTable,
477-
"resolving target import name")
478-
}
479-
if !found {
480-
// Check if database exists right now. It might not after the import is done,
481-
// but it's better to fail fast than wait until restore.
482-
return pgerror.Newf(pgcode.UndefinedObject,
483-
"database does not exist: %q", table)
484-
}
485-
table.ObjectNamePrefix = prefix
486-
db = resPrefix.Database
487-
sc = resPrefix.Schema
488-
// If this is a non-INTO import that will thus be making a new table, we
489-
// need the CREATE priv in the target DB.
490-
if !importStmt.Into {
491-
if err := p.CheckPrivilege(ctx, db, privilege.CREATE); err != nil {
492-
return err
493-
}
494-
}
453+
// TODO: As part of work for #34240, we should be operating on
454+
// UnresolvedObjectNames here, rather than TableNames.
455+
// We have a target table, so it might specify a DB in its name.
456+
un := table.ToUnresolvedObjectName()
457+
foundDB, prefix, resPrefix, err := resolver.ResolveTarget(ctx,
458+
un, p, p.SessionData().Database, p.SessionData().SearchPath)
459+
if err != nil {
460+
return pgerror.Wrap(err, pgcode.UndefinedTable,
461+
"resolving target import name")
462+
}
463+
if !foundDB {
464+
// Check if database exists right now. It might not after the import is done,
465+
// but it's better to fail fast than wait until restore.
466+
return pgerror.Newf(pgcode.UndefinedObject,
467+
"database does not exist: %q", table)
468+
}
469+
table.ObjectNamePrefix = prefix
470+
db := resPrefix.Database
495471

496-
switch sc.SchemaKind() {
497-
case catalog.SchemaVirtual:
498-
return pgerror.Newf(pgcode.InvalidSchemaName,
499-
"cannot import into schema %q", table.SchemaName)
500-
}
501-
} else {
502-
// No target table means we're importing whatever we find into the session
503-
// database, so it must exist.
504-
db, err = p.MustGetCurrentSessionDatabase(ctx)
505-
if err != nil {
506-
return pgerror.Wrap(err, pgcode.UndefinedObject,
507-
"could not resolve current database")
508-
}
509-
// If this is a non-INTO import that will thus be making a new table, we
510-
// need the CREATE priv in the target DB.
511-
if !importStmt.Into {
512-
if err := p.CheckPrivilege(ctx, db, privilege.CREATE); err != nil {
513-
return err
514-
}
515-
}
516-
sc = schemadesc.GetPublicSchema()
472+
switch resPrefix.Schema.SchemaKind() {
473+
case catalog.SchemaVirtual:
474+
return pgerror.Newf(pgcode.InvalidSchemaName,
475+
"cannot import into schema %q", table.SchemaName)
517476
}
518477

519478
format := roachpb.IOFileFormat{}
@@ -766,123 +725,92 @@ func importPlanHook(
766725
return err
767726
}
768727

769-
if importStmt.Into {
770-
if _, ok := allowedIntoFormats[importStmt.FileFormat]; !ok {
771-
return errors.Newf(
772-
"%s file format is currently unsupported by IMPORT INTO",
773-
importStmt.FileFormat)
774-
}
775-
_, found, err := p.ResolveMutableTableDescriptor(ctx, table, true, tree.ResolveRequireTableDesc)
776-
if err != nil {
777-
return err
728+
if _, ok := allowedIntoFormats[importStmt.FileFormat]; !ok {
729+
return errors.Newf(
730+
"%s file format is currently unsupported by IMPORT INTO",
731+
importStmt.FileFormat)
732+
}
733+
_, found, err := p.ResolveMutableTableDescriptor(ctx, table, true, tree.ResolveRequireTableDesc)
734+
if err != nil {
735+
return err
736+
}
737+
738+
err = ensureRequiredPrivileges(ctx, importIntoRequiredPrivileges, p, found)
739+
if err != nil {
740+
return err
741+
}
742+
// Check if the table has any vector indexes
743+
for _, idx := range found.NonDropIndexes() {
744+
if idx.GetType() == idxtype.VECTOR {
745+
return unimplemented.NewWithIssueDetail(145227, "import.vector-index",
746+
"IMPORT INTO is not supported for tables with vector indexes")
778747
}
748+
}
749+
750+
if len(found.LDRJobIDs) > 0 {
751+
return errors.Newf("cannot run an import on table %s which is apart of a Logical Data Replication stream", table)
752+
}
779753

780-
err = ensureRequiredPrivileges(ctx, importIntoRequiredPrivileges, p, found)
754+
// Import into an RLS table is blocked, unless this is the admin. It is
755+
// allowed for admins since they are exempt from RLS policies and have
756+
// unrestricted read/write access.
757+
if found.IsRowLevelSecurityEnabled() {
758+
admin, err := p.HasAdminRole(ctx)
781759
if err != nil {
782760
return err
761+
} else if !admin {
762+
return pgerror.New(pgcode.FeatureNotSupported,
763+
"IMPORT INTO not supported with row-level security for non-admin users")
783764
}
784-
// Check if the table has any vector indexes
785-
for _, idx := range found.NonDropIndexes() {
786-
if idx.GetType() == idxtype.VECTOR {
787-
return unimplemented.NewWithIssueDetail(145227, "import.vector-index",
788-
"IMPORT INTO is not supported for tables with vector indexes")
789-
}
790-
}
765+
}
791766

792-
if len(found.LDRJobIDs) > 0 {
793-
return errors.Newf("cannot run an import on table %s which is apart of a Logical Data Replication stream", table)
767+
// Validate target columns.
768+
var intoCols []string
769+
isTargetCol := make(map[string]bool)
770+
for _, name := range importStmt.IntoCols {
771+
active, err := catalog.MustFindPublicColumnsByNameList(found, tree.NameList{name})
772+
if err != nil {
773+
return errors.Wrap(err, "verifying target columns")
794774
}
795775

796-
// Import into an RLS table is blocked, unless this is the admin. It is
797-
// allowed for admins since they are exempt from RLS policies and have
798-
// unrestricted read/write access.
799-
if found.IsRowLevelSecurityEnabled() {
800-
admin, err := p.HasAdminRole(ctx)
801-
if err != nil {
802-
return err
803-
} else if !admin {
804-
return pgerror.New(pgcode.FeatureNotSupported,
805-
"IMPORT INTO not supported with row-level security for non-admin users")
806-
}
807-
}
776+
isTargetCol[active[0].GetName()] = true
777+
intoCols = append(intoCols, active[0].GetName())
778+
}
808779

809-
// Validate target columns.
810-
var intoCols []string
811-
isTargetCol := make(map[string]bool)
812-
for _, name := range importStmt.IntoCols {
813-
active, err := catalog.MustFindPublicColumnsByNameList(found, tree.NameList{name})
814-
if err != nil {
815-
return errors.Wrap(err, "verifying target columns")
780+
// Ensure that non-target columns that don't have default
781+
// expressions are nullable.
782+
if len(isTargetCol) != 0 {
783+
for _, col := range found.VisibleColumns() {
784+
if !(isTargetCol[col.GetName()] || col.IsNullable() || col.HasDefault() || col.IsComputed()) {
785+
return errors.Newf(
786+
"all non-target columns in IMPORT INTO must be nullable "+
787+
"or have default expressions, or have computed expressions"+
788+
" but violated by column %q",
789+
col.GetName(),
790+
)
816791
}
817-
818-
isTargetCol[active[0].GetName()] = true
819-
intoCols = append(intoCols, active[0].GetName())
820-
}
821-
822-
// Ensure that non-target columns that don't have default
823-
// expressions are nullable.
824-
if len(isTargetCol) != 0 {
825-
for _, col := range found.VisibleColumns() {
826-
if !(isTargetCol[col.GetName()] || col.IsNullable() || col.HasDefault() || col.IsComputed()) {
827-
return errors.Newf(
828-
"all non-target columns in IMPORT INTO must be nullable "+
829-
"or have default expressions, or have computed expressions"+
830-
" but violated by column %q",
831-
col.GetName(),
832-
)
833-
}
834-
if isTargetCol[col.GetName()] && col.IsComputed() {
835-
return schemaexpr.CannotWriteToComputedColError(col.GetName())
836-
}
792+
if isTargetCol[col.GetName()] && col.IsComputed() {
793+
return schemaexpr.CannotWriteToComputedColError(col.GetName())
837794
}
838795
}
796+
}
839797

840-
{
841-
// Resolve the UDTs used by the table being imported into.
842-
typeDescs, err := resolveUDTsUsedByImportInto(ctx, p, found)
843-
if err != nil {
844-
return errors.Wrap(err, "resolving UDTs used by table being imported into")
845-
}
846-
if len(typeDescs) > 0 {
847-
typeDetails = make([]jobspb.ImportDetails_Type, 0, len(typeDescs))
848-
}
849-
for _, typeDesc := range typeDescs {
850-
typeDetails = append(typeDetails, jobspb.ImportDetails_Type{Desc: typeDesc.TypeDesc()})
851-
}
798+
{
799+
// Resolve the UDTs used by the table being imported into.
800+
typeDescs, err := resolveUDTsUsedByImportInto(ctx, p, found)
801+
if err != nil {
802+
return errors.Wrap(err, "resolving UDTs used by table being imported into")
852803
}
853-
854-
tableDetails = []jobspb.ImportDetails_Table{{Desc: &found.TableDescriptor, IsNew: false, TargetCols: intoCols}}
855-
} else if importStmt.Bundle {
856-
// If we target a single table, populate details with one entry of tableName.
857-
if table != nil {
858-
tableDetails = make([]jobspb.ImportDetails_Table, 1)
859-
tableName := table.ObjectName.String()
860-
// PGDUMP supports importing tables from non-public schemas, thus we
861-
// must prepend the target table name with the target schema name.
862-
if format.Format == roachpb.IOFileFormat_PgDump {
863-
if table.Schema() == "" {
864-
return errors.Newf("expected schema for target table %s to be resolved",
865-
tableName)
866-
}
867-
tableName = fmt.Sprintf("%s.%s", table.SchemaName.String(),
868-
table.ObjectName.String())
869-
}
870-
tableDetails[0] = jobspb.ImportDetails_Table{
871-
Name: tableName,
872-
IsNew: true,
873-
}
804+
if len(typeDescs) > 0 {
805+
typeDetails = make([]jobspb.ImportDetails_Type, 0, len(typeDescs))
874806
}
875-
876-
// Due to how we generate and rewrite descriptor ID's for import, we run
877-
// into problems when using user defined schemas.
878-
publicSchemaID := db.GetSchemaID(catconstants.PublicSchemaName)
879-
if sc.GetID() != publicSchemaID && sc.GetID() != keys.PublicSchemaID {
880-
err := errors.New("cannot use IMPORT with a user defined schema")
881-
hint := errors.WithHint(err, "create the table with CREATE TABLE and use IMPORT INTO instead")
882-
return hint
807+
for _, typeDesc := range typeDescs {
808+
typeDetails = append(typeDetails, jobspb.ImportDetails_Type{Desc: typeDesc.TypeDesc()})
883809
}
884810
}
885811

812+
tableDetails = []jobspb.ImportDetails_Table{{Desc: &found.TableDescriptor, IsNew: false, TargetCols: intoCols}}
813+
886814
// Store the primary region of the database being imported into. This is
887815
// used during job execution to evaluate certain default expressions and
888816
// computed columns such as `gateway_region`.
@@ -919,9 +847,7 @@ func importPlanHook(
919847
break
920848
}
921849
}
922-
if importStmt.Into {
923-
telemetry.Count("import.into")
924-
}
850+
telemetry.Count("import.into")
925851

926852
// Here we create the job in a side transaction and then kick off the job.
927853
// This is awful. Rather we should be disallowing this statement in an
@@ -936,7 +862,6 @@ func importPlanHook(
936862
Tables: tableDetails,
937863
Types: typeDetails,
938864
SkipFKs: skipFKs,
939-
ParseBundleSchema: importStmt.Bundle,
940865
DefaultIntSize: p.SessionData().DefaultIntSize,
941866
DatabasePrimaryRegion: databasePrimaryRegion,
942867
}

pkg/sql/parser/sql.y

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -4144,12 +4144,12 @@ import_stmt:
41444144
IMPORT INTO table_name '(' insert_column_list ')' import_format DATA '(' string_or_placeholder_list ')' opt_with_options
41454145
{
41464146
name := $3.unresolvedObjectName().ToTableName()
4147-
$$.val = &tree.Import{Table: &name, Into: true, IntoCols: $5.nameList(), FileFormat: $7, Files: $10.exprs(), Options: $12.kvOptions()}
4147+
$$.val = &tree.Import{Table: &name, IntoCols: $5.nameList(), FileFormat: $7, Files: $10.exprs(), Options: $12.kvOptions()}
41484148
}
41494149
| IMPORT INTO table_name import_format DATA '(' string_or_placeholder_list ')' opt_with_options
41504150
{
41514151
name := $3.unresolvedObjectName().ToTableName()
4152-
$$.val = &tree.Import{Table: &name, Into: true, IntoCols: nil, FileFormat: $4, Files: $7.exprs(), Options: $9.kvOptions()}
4152+
$$.val = &tree.Import{Table: &name, IntoCols: nil, FileFormat: $4, Files: $7.exprs(), Options: $9.kvOptions()}
41534153
}
41544154
| IMPORT error // SHOW HELP: IMPORT
41554155

0 commit comments

Comments
 (0)