From 42f76d603849a753a20649c220f47f63063ecae2 Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Tue, 2 Feb 2021 19:00:30 -0500 Subject: [PATCH] importccl: add option to log unsupported stmts This is the last commit, that adds an `ignored_stmt_log` options to IMPORT PGDUMP. This option specifies the destination we will log the statements that we skip over during an import. This option can only be used in conjunction with `ignore_unsupported`, else the IMPORT will fail. Currently, we will write to two files during the import. One during the schema parsing phase, and another during the data ingestion phase. The files will be called: `unsupported-data-stmts`: Contains unparseable stmts, and unsupported DML stmts. `unsupported-schema-stmts`: Contains unparseable stmts, and unsupported DDL stmts. Release note (sql change): New IMPORT PGDUMP option `ignored_stmt_log` that allows users to specify where they would like to log stmts that have been skipped during an import, by virtue of being unsupported. --- pkg/ccl/importccl/import_processor.go | 3 +- pkg/ccl/importccl/import_processor_test.go | 7 +- pkg/ccl/importccl/import_stmt.go | 126 +++++++++++- pkg/ccl/importccl/import_stmt_test.go | 95 ++++++++- pkg/ccl/importccl/read_import_pgdump.go | 193 ++++++++++-------- pkg/ccl/importccl/read_import_pgdump_test.go | 6 +- pkg/ccl/importccl/testdata/pgdump/simple.sql | 14 ++ pkg/roachpb/io-formats.pb.go | 195 +++++++++++-------- pkg/roachpb/io-formats.proto | 5 + pkg/sql/sem/tree/BUILD.bazel | 2 +- 10 files changed, 463 insertions(+), 183 deletions(-) diff --git a/pkg/ccl/importccl/import_processor.go b/pkg/ccl/importccl/import_processor.go index f00696c3ee32..7aae80e09781 100644 --- a/pkg/ccl/importccl/import_processor.go +++ b/pkg/ccl/importccl/import_processor.go @@ -238,8 +238,7 @@ func makeInputConverter( return newPgCopyReader(spec.Format.PgCopy, kvCh, spec.WalltimeNanos, int(spec.ReaderParallelism), singleTable, singleTableTargetCols, evalCtx) case roachpb.IOFileFormat_PgDump: - return newPgDumpReader(ctx, kvCh, spec.Format.PgDump, spec.WalltimeNanos, spec.Tables, - spec.Format.PgDump.IgnoreUnsupported, evalCtx) + return newPgDumpReader(ctx, kvCh, spec.Format.PgDump, spec.WalltimeNanos, spec.Tables, evalCtx) case roachpb.IOFileFormat_Avro: return newAvroInputReader( kvCh, singleTable, spec.Format.Avro, spec.WalltimeNanos, diff --git a/pkg/ccl/importccl/import_processor_test.go b/pkg/ccl/importccl/import_processor_test.go index b79cd8f303f6..7281deec0cfb 100644 --- a/pkg/ccl/importccl/import_processor_test.go +++ b/pkg/ccl/importccl/import_processor_test.go @@ -116,8 +116,7 @@ func TestConverterFlushesBatches(t *testing.T) { } kvCh := make(chan row.KVBatch, batchSize) - conv, err := makeInputConverter(ctx, converterSpec, &evalCtx, kvCh, - nil /* seqChunkProvider */) + conv, err := makeInputConverter(ctx, converterSpec, &evalCtx, kvCh, nil /* seqChunkProvider */) if err != nil { t.Fatalf("makeInputConverter() error = %v", err) } @@ -953,8 +952,8 @@ func pgDumpFormat() roachpb.IOFileFormat { return roachpb.IOFileFormat{ Format: roachpb.IOFileFormat_PgDump, PgDump: roachpb.PgDumpOptions{ - MaxRowSize: 64 * 1024, - //IgnoreUnsupported: true, + MaxRowSize: 64 * 1024, + IgnoreUnsupported: true, }, } } diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index 2d1de513764f..90e921578262 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -9,6 +9,7 @@ package importccl import ( + "bytes" "context" "fmt" "io/ioutil" @@ -99,7 +100,11 @@ const ( avroSchema = "schema" avroSchemaURI = "schema_uri" - pgDumpIgnoreAllUnsupported = "ignore_unsupported" + pgDumpIgnoreAllUnsupported = "ignore_unsupported" + pgDumpIgnoreShuntFileDest = "ignored_stmt_log" + pgDumpUnsupportedSchemaStmtLog = "unsupported_schema_stmts" + pgDumpUnsupportedDataStmtLog = "unsupported_data-_stmts" + pgDumpMaxLoggedStmts = 10 // RunningStatusImportBundleParseSchema indicates to the user that a bundle format // schema is being parsed @@ -137,6 +142,7 @@ var importOptionExpectValues = map[string]sql.KVStringOptValidate{ avroJSONRecords: sql.KVStringOptRequireNoValue, pgDumpIgnoreAllUnsupported: sql.KVStringOptRequireNoValue, + pgDumpIgnoreShuntFileDest: sql.KVStringOptRequireValue, } func makeStringSet(opts ...string) map[string]struct{} { @@ -167,7 +173,7 @@ var mysqlOutAllowedOptions = makeStringSet( var mysqlDumpAllowedOptions = makeStringSet(importOptionSkipFKs, csvRowLimit) var pgCopyAllowedOptions = makeStringSet(pgCopyDelimiter, pgCopyNull, optMaxRowSize) var pgDumpAllowedOptions = makeStringSet(optMaxRowSize, importOptionSkipFKs, csvRowLimit, - pgDumpIgnoreAllUnsupported) + pgDumpIgnoreAllUnsupported, pgDumpIgnoreShuntFileDest) // DROP is required because the target table needs to be take offline during // IMPORT INTO. @@ -616,6 +622,13 @@ func importPlanHook( 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` option") + } + format.PgDump.IgnoreUnsupportedLog = dest + } + if override, ok := opts[csvRowLimit]; ok { rowLimit, err := strconv.Atoi(override) if err != nil { @@ -1257,6 +1270,102 @@ func (r *importResumer) ReportResults(ctx context.Context, resultsCh chan<- tree } } +type loggerKind int + +const ( + schemaParsing loggerKind = iota + dataIngestion +) + +// unsupportedStmtLogger is responsible for handling unsupported PGDUMP SQL +// statements seen during the import. +type unsupportedStmtLogger struct { + // 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 + + loggerType loggerKind +} + +func makeUnsupportedStmtLogger( + ignoreUnsupported bool, + unsupportedLogDest string, + loggerType loggerKind, + externalStorage cloud.ExternalStorageFactory, +) *unsupportedStmtLogger { + l := &unsupportedStmtLogger{ + ignoreUnsupported: ignoreUnsupported, + ignoreUnsupportedLogDest: unsupportedLogDest, + loggerType: loggerType, + logBuffer: new(bytes.Buffer), + externalStorage: externalStorage, + } + header := "Unsupported statements during schema parse phase:\n\n" + if loggerType == dataIngestion { + header = "Unsupported statements during data ingestion phase:\n\n" + } + l.logBuffer.WriteString(header) + return l +} + +func (u *unsupportedStmtLogger) log(logLine string, isParseError bool) { + // 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 + } + + if u.numIgnoredStmts < pgDumpMaxLoggedStmts { + 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++ +} + +func (u *unsupportedStmtLogger) flush(ctx context.Context, user security.SQLUsername) error { + if u.ignoreUnsupportedLogDest == "" { + return nil + } + + numLoggedStmts := pgDumpMaxLoggedStmts + if u.numIgnoredStmts < pgDumpMaxLoggedStmts { + numLoggedStmts = u.numIgnoredStmts + } + u.logBuffer.WriteString(fmt.Sprintf("\nLogging %d out of %d ignored statements.\n", + numLoggedStmts, u.numIgnoredStmts)) + + conf, err := cloudimpl.ExternalStorageConfFromURI(u.ignoreUnsupportedLogDest, 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(ctx, conf); err != nil { + return errors.New("failed to log unsupported stmts during IMPORT PGDUMP") + } + defer s.Close() + + logFileName := pgDumpUnsupportedSchemaStmtLog + if u.loggerType == dataIngestion { + logFileName = pgDumpUnsupportedDataStmtLog + } + err = s.WriteFile(ctx, logFileName, bytes.NewReader(u.logBuffer.Bytes())) + if err != nil { + return errors.Wrap(err, "failed to log unsupported stmts to log during IMPORT PGDUMP") + } + return nil +} + // parseAndCreateBundleTableDescs parses and creates the table // descriptors for bundle formats. func parseAndCreateBundleTableDescs( @@ -1306,8 +1415,19 @@ func parseAndCreateBundleTableDescs( tableDescs, err = readMysqlCreateTable(ctx, reader, evalCtx, p, defaultCSVTableID, parentID, tableName, fks, seqVals, owner, walltime) case roachpb.IOFileFormat_PgDump: evalCtx := &p.ExtendedEvalContext().EvalContext + + // Setup a logger to handle unsupported DDL statements in the PGDUMP file. + unsupportedStmtLogger := makeUnsupportedStmtLogger(format.PgDump.IgnoreUnsupported, + format.PgDump.IgnoreUnsupportedLog, schemaParsing, p.ExecCfg().DistSQLSrv.ExternalStorage) + tableDescs, err = readPostgresCreateTable(ctx, reader, evalCtx, p, tableName, parentID, - walltime, fks, int(format.PgDump.MaxRowSize), owner, format.PgDump.IgnoreUnsupported) + walltime, fks, int(format.PgDump.MaxRowSize), owner, unsupportedStmtLogger) + + logErr := unsupportedStmtLogger.flush(ctx, p.User()) + if logErr != nil { + return nil, logErr + } + default: return tableDescs, errors.Errorf("non-bundle format %q does not support reading schemas", format.Format.String()) } diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go index c1363141a08e..b6c43de4eaed 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -818,6 +818,7 @@ END; name: "fk", typ: "PGDUMP", data: testPgdumpFk, + with: "WITH ignore_unsupported", query: map[string][][]string{ getTablesQuery: { {"public", "cities", "table"}, @@ -896,7 +897,7 @@ END; name: "fk-skip", typ: "PGDUMP", data: testPgdumpFk, - with: `WITH skip_foreign_keys`, + with: `WITH skip_foreign_keys, ignore_unsupported`, query: map[string][][]string{ getTablesQuery: { {"public", "cities", "table"}, @@ -911,13 +912,14 @@ END; name: "fk unreferenced", typ: "TABLE weather FROM PGDUMP", data: testPgdumpFk, + with: "WITH ignore_unsupported", err: `table "cities" not found`, }, { name: "fk unreferenced skipped", typ: "TABLE weather FROM PGDUMP", data: testPgdumpFk, - with: `WITH skip_foreign_keys`, + with: `WITH skip_foreign_keys, ignore_unsupported`, query: map[string][][]string{ getTablesQuery: {{"public", "weather", "table"}}, }, @@ -5671,13 +5673,20 @@ func TestImportPgDumpIgnoredStmts(t *testing.T) { sqlDB := sqlutils.MakeSQLRunner(conn) data := ` + -- Statements that CRDB cannot parse. CREATE TRIGGER conditions_set_updated_at BEFORE UPDATE ON conditions FOR EACH ROW EXECUTE PROCEDURE set_updated_at(); + REVOKE ALL ON SEQUENCE knex_migrations_id_seq FROM PUBLIC; REVOKE ALL ON SEQUENCE knex_migrations_id_seq FROM database; + GRANT ALL ON SEQUENCE knex_migrations_id_seq TO database; GRANT SELECT ON SEQUENCE knex_migrations_id_seq TO opentrials_readonly; - CREATE TABLE foo (id INT); + COMMENT ON EXTENSION plpgsql IS 'PL/pgSQL procedural language'; + CREATE EXTENSION IF NOT EXISTS plpgsql WITH SCHEMA pg_catalog; + + -- Valid statement. + CREATE TABLE foo (id INT); CREATE FUNCTION public.isnumeric(text) RETURNS boolean LANGUAGE sql @@ -5686,15 +5695,21 @@ func TestImportPgDumpIgnoredStmts(t *testing.T) { $_$; ALTER FUNCTION public.isnumeric(text) OWNER TO roland; - INSERT INTO foo VALUES (1), (2), (3); - + -- Valid statements. + INSERT INTO foo VALUES (1), (2), (3); CREATE TABLE t (i INT8); + + -- Statements that CRDB can parse, but IMPORT does not support. + -- These are processed during the schema pass of IMPORT. COMMENT ON TABLE t IS 'This should be skipped'; COMMENT ON DATABASE t IS 'This should be skipped'; COMMENT ON COLUMN t IS 'This should be skipped'; - COMMENT ON EXTENSION; - COMMENT ON EXTENSION plpgsql IS 'PL/pgSQL procedural language'; - CREATE EXTENSION IF NOT EXISTS plpgsql WITH SCHEMA pg_catalog; + + + -- Statements that CRDB can parse, but IMPORT does not support. + -- These are processed during the data ingestion pass of IMPORT. + SELECT pg_catalog.set_config('search_path', '', false); + DELETE FROM geometry_columns WHERE f_table_name = 'nyc_census_blocks' AND f_table_schema = 'public'; ` srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { @@ -5704,16 +5719,78 @@ func TestImportPgDumpIgnoredStmts(t *testing.T) { })) defer srv.Close() t.Run("ignore-unsupported", func(t *testing.T) { + sqlDB.Exec(t, "CREATE DATABASE foo; USE foo;") sqlDB.Exec(t, "IMPORT PGDUMP ($1) WITH ignore_unsupported", srv.URL) // Check that statements which are not expected to be ignored, are still // processed. sqlDB.CheckQueryResults(t, "SELECT * FROM foo", [][]string{{"1"}, {"2"}, {"3"}}) - sqlDB.Exec(t, "DROP TABLE foo") }) t.Run("dont-ignore-unsupported", func(t *testing.T) { + sqlDB.Exec(t, "CREATE DATABASE foo1; USE foo1;") sqlDB.ExpectErr(t, "syntax error", "IMPORT PGDUMP ($1)", srv.URL) }) + + t.Run("require-both-unsupported-options", func(t *testing.T) { + sqlDB.Exec(t, "CREATE DATABASE foo2; USE foo2;") + ignoredLog := `userfile:///ignore.log` + sqlDB.ExpectErr(t, "cannot log unsupported PGDUMP stmts without `ignore_unsupported` option", + "IMPORT PGDUMP ($1) WITH ignored_stmt_log=$2", srv.URL, ignoredLog) + }) + + t.Run("log-unsupported-stmts", func(t *testing.T) { + sqlDB.Exec(t, "CREATE DATABASE foo3; USE foo3;") + ignoredLog := `userfile:///ignore.log` + sqlDB.Exec(t, "IMPORT PGDUMP ($1) WITH ignore_unsupported, ignored_stmt_log=$2", + srv.URL, ignoredLog) + // Check that statements which are not expected to be ignored, are still + // processed. + sqlDB.CheckQueryResults(t, "SELECT * FROM foo", [][]string{{"1"}, {"2"}, {"3"}}) + + // Read the unsupported log and verify its contents. + store, err := cloudimpl.ExternalStorageFromURI(ctx, ignoredLog, + base.ExternalIODirConfig{}, + tc.Servers[0].ClusterSettings(), + blobs.TestEmptyBlobClientFactory, + security.RootUserName(), + tc.Servers[0].InternalExecutor().(*sql.InternalExecutor), tc.Servers[0].DB()) + require.NoError(t, err) + defer store.Close() + content, err := store.ReadFile(ctx, pgDumpUnsupportedSchemaStmtLog) + require.NoError(t, err) + descBytes, err := ioutil.ReadAll(content) + require.NoError(t, err) + expectedSchemaLog := `Unsupported statements during schema parse phase: + +create trigger: could not be parsed +revoke privileges on sequence: could not be parsed +revoke privileges on sequence: could not be parsed +grant privileges on sequence: could not be parsed +grant privileges on sequence: could not be parsed +comment on extension: could not be parsed +create extension if not exists with: could not be parsed +create function: could not be parsed +alter function: could not be parsed +COMMENT ON TABLE t IS 'This should be skipped': unsupported by IMPORT + +Logging 10 out of 13 ignored statements. +` + require.Equal(t, []byte(expectedSchemaLog), descBytes) + + expectedDataLog := `Unsupported statements during data ingestion phase: + +unsupported 3 fn args in select: ['search_path' '' false]: unsupported by IMPORT +unsupported *tree.Delete statement: DELETE FROM geometry_columns WHERE (f_table_name = 'nyc_census_blocks') AND (f_table_schema = 'public'): unsupported by IMPORT + +Logging 2 out of 2 ignored statements. +` + + content, err = store.ReadFile(ctx, pgDumpUnsupportedDataStmtLog) + require.NoError(t, err) + descBytes, err = ioutil.ReadAll(content) + require.NoError(t, err) + require.Equal(t, []byte(expectedDataLog), descBytes) + }) } // TestImportPgDumpGeo tests that a file with SQLFn classes can be diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go index 728a35f1048b..07af37520c87 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -11,6 +11,7 @@ package importccl import ( "bufio" "context" + "fmt" "io" "regexp" "strings" @@ -42,17 +43,20 @@ import ( ) type postgreStream struct { - s *bufio.Scanner - copy *postgreStreamCopy - ignoreUnsupportedStmts bool + ctx context.Context + s *bufio.Scanner + copy *postgreStreamCopy + unsupportedStmtLogger *unsupportedStmtLogger } // newPostgreStream returns a struct that can stream statements from an // io.Reader. -func newPostgreStream(r io.Reader, max int, ignoreUnsupportedStmts bool) *postgreStream { +func newPostgreStream( + ctx context.Context, r io.Reader, max int, unsupportedStmtLogger *unsupportedStmtLogger, +) *postgreStream { s := bufio.NewScanner(r) s.Buffer(nil, max) - p := &postgreStream{s: s, ignoreUnsupportedStmts: ignoreUnsupportedStmts} + p := &postgreStream{ctx: ctx, s: s, unsupportedStmtLogger: unsupportedStmtLogger} s.Split(p.split) return p } @@ -100,11 +104,13 @@ func (p *postgreStream) Next() (interface{}, error) { stmts, err := parser.Parse(t) if err != nil { - // There are some statements which CRDB is unable to parse but we have - // explicitly marked as "to be skipped" during a PGDUMP import. - // TODO(adityamaru): Write these to a shunt file to see what has been - // skipped. - if p.ignoreUnsupportedStmts && errors.HasType(err, (*tree.UnsupportedError)(nil)) { + // There are some statements that CRDB is unable to parse. If the user has + // indicated that they want to skip these stmts during the IMPORT, then do + // so here. + if p.unsupportedStmtLogger.ignoreUnsupported && errors.HasType(err, (*tree.UnsupportedError)(nil)) { + if unsupportedErr := (*tree.UnsupportedError)(nil); errors.As(err, &unsupportedErr) { + p.unsupportedStmtLogger.log(unsupportedErr.FeatureName, true /* isParseError */) + } continue } return nil, err @@ -216,7 +222,7 @@ func readPostgresCreateTable( fks fkHandler, max int, owner security.SQLUsername, - ignoreUnsupportedStmts bool, + unsupportedStmtLogger *unsupportedStmtLogger, ) ([]*tabledesc.Mutable, error) { // Modify the CreateTable stmt with the various index additions. We do this // instead of creating a full table descriptor first and adding indexes @@ -227,7 +233,7 @@ func readPostgresCreateTable( createTbl := make(map[string]*tree.CreateTable) createSeq := make(map[string]*tree.CreateSequence) tableFKs := make(map[string][]*tree.ForeignKeyConstraintTableDef) - ps := newPostgreStream(input, max, ignoreUnsupportedStmts) + ps := newPostgreStream(ctx, input, max, unsupportedStmtLogger) for { stmt, err := ps.Next() if err == io.EOF { @@ -299,7 +305,7 @@ func readPostgresCreateTable( return nil, errors.Wrap(err, "postgres parse error") } if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, tableFKs, stmt, p, - parentID, ignoreUnsupportedStmts); err != nil { + parentID, unsupportedStmtLogger); err != nil { return nil, err } } @@ -316,8 +322,9 @@ func readPostgresStmt( stmt interface{}, p sql.JobExecContext, parentID descpb.ID, - ignoreUnsupportedStmts bool, + unsupportedStmtLogger *unsupportedStmtLogger, ) error { + ignoreUnsupportedStmts := unsupportedStmtLogger.ignoreUnsupported switch stmt := stmt.(type) { case *tree.CreateTable: name, err := getTableName(&stmt.Table) @@ -393,7 +400,7 @@ func readPostgresStmt( case *tree.AlterTableAddColumn: if cmd.IfNotExists { if ignoreUnsupportedStmts { - // Write to shunt. + unsupportedStmtLogger.log(stmt.String(), false /* isParseError */) continue } return errors.Errorf("unsupported statement: %s", stmt) @@ -418,14 +425,18 @@ func readPostgresStmt( } default: if ignoreUnsupportedStmts { - // Write to shunt. + unsupportedStmtLogger.log(stmt.String(), false /* isParseError */) continue } return errors.Errorf("unsupported statement: %s", stmt) } } case *tree.AlterTableOwner: - // ignore + if ignoreUnsupportedStmts { + unsupportedStmtLogger.log(stmt.String(), false /* isParseError */) + return nil + } + return errors.Errorf("unsupported statement: %s", stmt) case *tree.CreateSequence: name, err := getTableName(&stmt.Name) if err != nil { @@ -436,7 +447,7 @@ func readPostgresStmt( } case *tree.AlterSequence: if ignoreUnsupportedStmts { - // Write to shunt file. + unsupportedStmtLogger.log(stmt.String(), false /* isParseError */) return nil } return errors.Errorf("unsupported %T statement: %s", stmt, stmt) @@ -463,11 +474,13 @@ func readPostgresStmt( // Search for a SQLFn, which returns a SQL string to execute. fn := ov.SQLFn if fn == nil { + err := errors.Errorf("unsupported function call: %s in stmt: %s", + expr.Func.String(), stmt.String()) if ignoreUnsupportedStmts { - // Write to shunt file. + unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported function call: %s", expr.Func.String()) + return err } // Attempt to convert all func exprs to datums. datums := make(tree.Datums, len(expr.Exprs)) @@ -495,7 +508,7 @@ func readPostgresStmt( switch ast := fnStmt.AST.(type) { case *tree.AlterTable: if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, - tableFKs, ast, p, parentID, ignoreUnsupportedStmts); err != nil { + tableFKs, ast, p, parentID, unsupportedStmtLogger); err != nil { return err } default: @@ -504,19 +517,21 @@ func readPostgresStmt( } } default: + err := errors.Errorf("unsupported %T SELECT expr: %s", expr, expr) if ignoreUnsupportedStmts { - // Write unsupported select expressions to the SHUNT file. + unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported %T SELECT expr: %s", expr, expr) + return err } } default: + err := errors.Errorf("unsupported %T SELECT %s", sel, sel) if ignoreUnsupportedStmts { - // Write to shunt file. + unsupportedStmtLogger.log(err.Error(), false /* isParseError */) return nil } - return errors.Errorf("unsupported %T SELECT: %s", sel, sel) + return err } case *tree.DropTable: names := stmt.Names @@ -559,7 +574,7 @@ func readPostgresStmt( // - ANALYZE is syntactic sugar for CreateStatistics. It can be ignored // because the auto stats stuff will pick up the changes and run if needed. if ignoreUnsupportedStmts { - // Write to shunt if the user has asked us to skip. + unsupportedStmtLogger.log(fmt.Sprintf("%s", stmt), false /* isParseError */) return nil } return errors.Errorf("unsupported %T statement: %s", stmt, stmt) @@ -569,7 +584,7 @@ func readPostgresStmt( } default: if ignoreUnsupportedStmts { - // Write to shunt file. + unsupportedStmtLogger.log(fmt.Sprintf("%s", stmt), false /* isParseError */) return nil } return errors.Errorf("unsupported %T statement: %s", stmt, stmt) @@ -601,15 +616,15 @@ func getTableName2(u *tree.UnresolvedObjectName) (string, error) { } type pgDumpReader struct { - tableDescs map[string]catalog.TableDescriptor - tables map[string]*row.DatumRowConverter - descs map[string]*execinfrapb.ReadImportDataSpec_ImportTable - kvCh chan row.KVBatch - opts roachpb.PgDumpOptions - walltime int64 - colMap map[*row.DatumRowConverter](map[string]int) - ignoreUnsupportedStmts bool - evalCtx *tree.EvalContext + tableDescs map[string]catalog.TableDescriptor + tables map[string]*row.DatumRowConverter + descs map[string]*execinfrapb.ReadImportDataSpec_ImportTable + kvCh chan row.KVBatch + opts roachpb.PgDumpOptions + walltime int64 + colMap map[*row.DatumRowConverter](map[string]int) + unsupportedStmtLogger *unsupportedStmtLogger + evalCtx *tree.EvalContext } var _ inputConverter = &pgDumpReader{} @@ -621,7 +636,6 @@ func newPgDumpReader( opts roachpb.PgDumpOptions, walltime int64, descs map[string]*execinfrapb.ReadImportDataSpec_ImportTable, - ignoreUnsupportedStmts bool, evalCtx *tree.EvalContext, ) (*pgDumpReader, error) { tableDescs := make(map[string]catalog.TableDescriptor, len(descs)) @@ -652,15 +666,14 @@ func newPgDumpReader( } } return &pgDumpReader{ - kvCh: kvCh, - tableDescs: tableDescs, - tables: converters, - descs: descs, - opts: opts, - walltime: walltime, - colMap: colMap, - ignoreUnsupportedStmts: ignoreUnsupportedStmts, - evalCtx: evalCtx, + kvCh: kvCh, + tableDescs: tableDescs, + tables: converters, + descs: descs, + opts: opts, + walltime: walltime, + colMap: colMap, + evalCtx: evalCtx, }, nil } @@ -675,7 +688,16 @@ func (m *pgDumpReader) readFiles( makeExternalStorage cloud.ExternalStorageFactory, user security.SQLUsername, ) error { - return readInputFiles(ctx, dataFiles, resumePos, format, m.readFile, makeExternalStorage, user) + // Setup logger to handle unsupported DML statements seen in the PGDUMP file. + m.unsupportedStmtLogger = makeUnsupportedStmtLogger(format.PgDump.IgnoreUnsupported, + format.PgDump.IgnoreUnsupportedLog, dataIngestion, makeExternalStorage) + + err := readInputFiles(ctx, dataFiles, resumePos, format, m.readFile, makeExternalStorage, user) + if err != nil { + return err + } + + return m.unsupportedStmtLogger.flush(ctx, user) } func (m *pgDumpReader) readFile( @@ -684,7 +706,7 @@ func (m *pgDumpReader) readFile( tableNameToRowsProcessed := make(map[string]int64) var inserts, count int64 rowLimit := m.opts.RowLimit - ps := newPostgreStream(input, int(m.opts.MaxRowSize), m.opts.IgnoreUnsupported) + ps := newPostgreStream(ctx, input, int(m.opts.MaxRowSize), m.unsupportedStmtLogger) semaCtx := tree.MakeSemaContext() for _, conv := range m.tables { conv.KvBatch.Source = inputIdx @@ -729,8 +751,10 @@ func (m *pgDumpReader) readFile( timestamp := timestampAfterEpoch(m.walltime) values, ok := i.Rows.Select.(*tree.ValuesClause) if !ok { - if m.ignoreUnsupportedStmts { - // Write to shunt. + if m.unsupportedStmtLogger.ignoreUnsupported { + logLine := fmt.Sprintf("%s: unsupported by IMPORT\n", + i.Rows.Select.String()) + m.unsupportedStmtLogger.log(logLine, false /* isParseError */) continue } return errors.Errorf("unsupported: %s", i.Rows.Select) @@ -879,42 +903,47 @@ func (m *pgDumpReader) readFile( // by pg_dump, and thus if it isn't, we don't try to figure out what to do. sc, ok := i.Select.(*tree.SelectClause) if !ok { - if m.ignoreUnsupportedStmts { - // Write to shunt file. + err := errors.Errorf("unsupported %T Select: %v", i.Select, i.Select) + if m.unsupportedStmtLogger.ignoreUnsupported { + m.unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported %T Select: %v", i.Select, i.Select) + return err } if len(sc.Exprs) != 1 { - if m.ignoreUnsupportedStmts { - // Write to shunt file. + err := errors.Errorf("unsupported %d select args: %v", len(sc.Exprs), sc.Exprs) + if m.unsupportedStmtLogger.ignoreUnsupported { + m.unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported %d select args: %v", len(sc.Exprs), sc.Exprs) + return err } fn, ok := sc.Exprs[0].Expr.(*tree.FuncExpr) if !ok { - if m.ignoreUnsupportedStmts { - // Write to shunt file. + err := errors.Errorf("unsupported select arg %T: %v", sc.Exprs[0].Expr, sc.Exprs[0].Expr) + if m.unsupportedStmtLogger.ignoreUnsupported { + m.unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported select arg %T: %v", sc.Exprs[0].Expr, sc.Exprs[0].Expr) + return err } switch funcName := strings.ToLower(fn.Func.String()); funcName { case "search_path", "pg_catalog.set_config": - if m.ignoreUnsupportedStmts { - // Write to shunt file. + err := errors.Errorf("unsupported %d fn args in select: %v", len(fn.Exprs), fn.Exprs) + if m.unsupportedStmtLogger.ignoreUnsupported { + m.unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported %d fn args: %v", len(fn.Exprs), fn.Exprs) + return err case "setval", "pg_catalog.setval": if args := len(fn.Exprs); args < 2 || args > 3 { - if m.ignoreUnsupportedStmts { - // Write to shunt file. + err := errors.Errorf("unsupported %d fn args in select: %v", len(fn.Exprs), fn.Exprs) + if m.unsupportedStmtLogger.ignoreUnsupported { + m.unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported %d fn args: %v", len(fn.Exprs), fn.Exprs) + return err } seqname, ok := fn.Exprs[0].(*tree.StrVal) if !ok { @@ -930,11 +959,12 @@ func (m *pgDumpReader) readFile( } seqval, ok := fn.Exprs[1].(*tree.NumVal) if !ok { - if m.ignoreUnsupportedStmts { - // Write to shunt file. + err := errors.Errorf("unsupported setval %T arg: %v", fn.Exprs[1], fn.Exprs[1]) + if m.unsupportedStmtLogger.ignoreUnsupported { + m.unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported setval %T arg: %v", fn.Exprs[1], fn.Exprs[1]) + return err } val, err := seqval.AsInt64() if err != nil { @@ -944,11 +974,12 @@ func (m *pgDumpReader) readFile( if len(fn.Exprs) == 3 { called, ok := fn.Exprs[2].(*tree.DBool) if !ok { - if m.ignoreUnsupportedStmts { - // Write to shunt file. + err := errors.Errorf("unsupported setval %T arg: %v", fn.Exprs[2], fn.Exprs[2]) + if m.unsupportedStmtLogger.ignoreUnsupported { + m.unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported setval %T arg: %v", fn.Exprs[2], fn.Exprs[2]) + return err } isCalled = bool(*called) } @@ -972,11 +1003,12 @@ func (m *pgDumpReader) readFile( case "addgeometrycolumn": // handled during schema extraction. default: - if m.ignoreUnsupportedStmts { - // Write to shunt file. + err := errors.Errorf("unsupported function %s in stmt %s", funcName, i.Select.String()) + if m.unsupportedStmtLogger.ignoreUnsupported { + m.unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported function: %s", funcName) + return err } case *tree.CreateExtension, *tree.CommentOnDatabase, *tree.CommentOnTable, *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.AlterSequence: @@ -986,18 +1018,13 @@ func (m *pgDumpReader) readFile( case *tree.CreateTable, *tree.AlterTable, *tree.AlterTableOwner, *tree.CreateIndex, *tree.CreateSequence, *tree.DropTable: // handled during schema extraction. - case *tree.Delete: - if m.ignoreUnsupportedStmts { - // Write to shunt file. - continue - } - return errors.Errorf("unsupported DELETE FROM %T statement: %s", stmt, stmt) default: - if m.ignoreUnsupportedStmts { - // Write to shunt file. + err := errors.Errorf("unsupported %T statement: %v", i, i) + if m.unsupportedStmtLogger.ignoreUnsupported { + m.unsupportedStmtLogger.log(err.Error(), false /* isParseError */) continue } - return errors.Errorf("unsupported %T statement: %v", i, i) + return err } } for _, conv := range m.tables { diff --git a/pkg/ccl/importccl/read_import_pgdump_test.go b/pkg/ccl/importccl/read_import_pgdump_test.go index 16d57be0cb6f..0e42cd9b23bf 100644 --- a/pkg/ccl/importccl/read_import_pgdump_test.go +++ b/pkg/ccl/importccl/read_import_pgdump_test.go @@ -39,8 +39,7 @@ select '123456789012345678901234567890123456789012345678901234567890123456789012 -- ` - p := newPostgreStream(strings.NewReader(sql), defaultScanBuffer, - false /* ignoreUnsupportedStmts */) + p := newPostgreStream(context.Background(), strings.NewReader(sql), defaultScanBuffer, nil /* unsupportedStmtLogger */) var sb strings.Builder for { s, err := p.Next() @@ -122,8 +121,7 @@ COPY public.t (s) FROM stdin; -- ` - p := newPostgreStream(strings.NewReader(sql), defaultScanBuffer, - false /* ignoreUnsupportedStmts */) + p := newPostgreStream(context.Background(), strings.NewReader(sql), defaultScanBuffer, nil /* unsupportedStmtLogger */) var sb strings.Builder for { s, err := p.Next() diff --git a/pkg/ccl/importccl/testdata/pgdump/simple.sql b/pkg/ccl/importccl/testdata/pgdump/simple.sql index e117fea0017e..1d49cfa0336a 100644 --- a/pkg/ccl/importccl/testdata/pgdump/simple.sql +++ b/pkg/ccl/importccl/testdata/pgdump/simple.sql @@ -19,6 +19,20 @@ SET default_tablespace = ''; SET default_with_oids = false; + +-- +-- Name: simple; Type: EXTENSION; Schema: -; Owner: +-- + +CREATE EXTENSION IF NOT EXISTS simple WITH SCHEMA pg_catalog; + + +-- +-- Name: EXTENSION plpgsql; Type: COMMENT; Schema: -; Owner: +-- + +COMMENT ON EXTENSION simple IS 'simple extension'; + -- -- Name: simple; Type: TABLE; Schema: public; Owner: postgres -- diff --git a/pkg/roachpb/io-formats.pb.go b/pkg/roachpb/io-formats.pb.go index 94091beae32b..a8a9295af5da 100644 --- a/pkg/roachpb/io-formats.pb.go +++ b/pkg/roachpb/io-formats.pb.go @@ -68,7 +68,7 @@ func (x *IOFileFormat_FileFormat) UnmarshalJSON(data []byte) error { return nil } func (IOFileFormat_FileFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{0, 0} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{0, 0} } type IOFileFormat_Compression int32 @@ -110,7 +110,7 @@ func (x *IOFileFormat_Compression) UnmarshalJSON(data []byte) error { return nil } func (IOFileFormat_Compression) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{0, 1} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{0, 1} } type MySQLOutfileOptions_Enclose int32 @@ -149,7 +149,7 @@ func (x *MySQLOutfileOptions_Enclose) UnmarshalJSON(data []byte) error { return nil } func (MySQLOutfileOptions_Enclose) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{2, 0} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{2, 0} } type AvroOptions_Format int32 @@ -191,7 +191,7 @@ func (x *AvroOptions_Format) UnmarshalJSON(data []byte) error { return nil } func (AvroOptions_Format) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{6, 0} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{6, 0} } type IOFileFormat struct { @@ -211,7 +211,7 @@ func (m *IOFileFormat) Reset() { *m = IOFileFormat{} } func (m *IOFileFormat) String() string { return proto.CompactTextString(m) } func (*IOFileFormat) ProtoMessage() {} func (*IOFileFormat) Descriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{0} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{0} } func (m *IOFileFormat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -258,7 +258,7 @@ func (m *CSVOptions) Reset() { *m = CSVOptions{} } func (m *CSVOptions) String() string { return proto.CompactTextString(m) } func (*CSVOptions) ProtoMessage() {} func (*CSVOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{1} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{1} } func (m *CSVOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -309,7 +309,7 @@ func (m *MySQLOutfileOptions) Reset() { *m = MySQLOutfileOptions{} } func (m *MySQLOutfileOptions) String() string { return proto.CompactTextString(m) } func (*MySQLOutfileOptions) ProtoMessage() {} func (*MySQLOutfileOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{2} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{2} } func (m *MySQLOutfileOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -348,7 +348,7 @@ func (m *PgCopyOptions) Reset() { *m = PgCopyOptions{} } func (m *PgCopyOptions) String() string { return proto.CompactTextString(m) } func (*PgCopyOptions) ProtoMessage() {} func (*PgCopyOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{3} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{3} } func (m *PgCopyOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -383,13 +383,18 @@ type PgDumpOptions struct { // Indicates if all unparseable and parseable, but unimplemented PGDUMP stmts // should be ignored during IMPORT. IgnoreUnsupported bool `protobuf:"varint,3,opt,name=ignore_unsupported,json=ignoreUnsupported" json:"ignore_unsupported"` + // Points to the destination where unsupported statements during a PGDUMP + // import should be logged. This can only be used when ignore_unsupported is + // specified, otherwise the IMPORT errors out on encountering an unsupported + // stmt. + IgnoreUnsupportedLog string `protobuf:"bytes,4,opt,name=ignore_unsupported_log,json=ignoreUnsupportedLog" json:"ignore_unsupported_log"` } func (m *PgDumpOptions) Reset() { *m = PgDumpOptions{} } func (m *PgDumpOptions) String() string { return proto.CompactTextString(m) } func (*PgDumpOptions) ProtoMessage() {} func (*PgDumpOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{4} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{4} } func (m *PgDumpOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -424,7 +429,7 @@ func (m *MysqldumpOptions) Reset() { *m = MysqldumpOptions{} } func (m *MysqldumpOptions) String() string { return proto.CompactTextString(m) } func (*MysqldumpOptions) ProtoMessage() {} func (*MysqldumpOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{5} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{5} } func (m *MysqldumpOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -467,7 +472,7 @@ func (m *AvroOptions) Reset() { *m = AvroOptions{} } func (m *AvroOptions) String() string { return proto.CompactTextString(m) } func (*AvroOptions) ProtoMessage() {} func (*AvroOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_io_formats_488bfbb473a8192c, []int{6} + return fileDescriptor_io_formats_b77b03f938e9e2ae, []int{6} } func (m *AvroOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -739,6 +744,10 @@ func (m *PgDumpOptions) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + dAtA[i] = 0x22 + i++ + i = encodeVarintIoFormats(dAtA, i, uint64(len(m.IgnoreUnsupportedLog))) + i += copy(dAtA[i:], m.IgnoreUnsupportedLog) return i, nil } @@ -899,6 +908,8 @@ func (m *PgDumpOptions) Size() (n int) { n += 1 + sovIoFormats(uint64(m.MaxRowSize)) n += 1 + sovIoFormats(uint64(m.RowLimit)) n += 2 + l = len(m.IgnoreUnsupportedLog) + n += 1 + l + sovIoFormats(uint64(l)) return n } @@ -1842,6 +1853,35 @@ func (m *PgDumpOptions) Unmarshal(dAtA []byte) error { } } m.IgnoreUnsupported = bool(v != 0) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IgnoreUnsupportedLog", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIoFormats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthIoFormats + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IgnoreUnsupportedLog = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipIoFormats(dAtA[iNdEx:]) @@ -2213,70 +2253,71 @@ var ( ) func init() { - proto.RegisterFile("roachpb/io-formats.proto", fileDescriptor_io_formats_488bfbb473a8192c) -} - -var fileDescriptor_io_formats_488bfbb473a8192c = []byte{ - // 966 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xcd, 0x6e, 0xdb, 0x46, - 0x10, 0x16, 0xf5, 0x47, 0x71, 0x24, 0xd9, 0x9b, 0x6d, 0x0f, 0x44, 0xd0, 0xb2, 0xaa, 0xd2, 0x14, - 0x4e, 0xdb, 0xc8, 0x80, 0x5b, 0x03, 0xbd, 0x15, 0xb1, 0x62, 0x37, 0x0e, 0x62, 0xa9, 0x91, 0x90, - 0x1c, 0x7a, 0x21, 0x58, 0x6a, 0x2d, 0xb3, 0x26, 0xb9, 0xf4, 0x2e, 0x29, 0x45, 0x79, 0x87, 0x02, - 0x7d, 0xa6, 0x9e, 0x7c, 0xcc, 0xad, 0x01, 0x0a, 0x14, 0xad, 0xfd, 0x1e, 0x45, 0xb1, 0xcb, 0xa5, - 0x44, 0xda, 0x6c, 0x9a, 0xdb, 0x60, 0xbe, 0x99, 0xd9, 0x99, 0xef, 0x1b, 0x0e, 0xc1, 0x64, 0xd4, - 0x71, 0xcf, 0xa2, 0x9f, 0x76, 0x3d, 0xfa, 0xf0, 0x94, 0xb2, 0xc0, 0x89, 0xf9, 0x20, 0x62, 0x34, - 0xa6, 0xf8, 0x8e, 0x4b, 0xdd, 0x73, 0x89, 0x0e, 0x54, 0xcc, 0xdd, 0x0f, 0xe7, 0x74, 0x4e, 0x25, - 0xba, 0x2b, 0xac, 0x34, 0xb0, 0xff, 0x4f, 0x03, 0x3a, 0xc7, 0xe3, 0x23, 0xcf, 0x27, 0x47, 0xb2, - 0x00, 0x7e, 0x02, 0xcd, 0xb4, 0x94, 0xa9, 0xf5, 0xb4, 0x9d, 0xad, 0xbd, 0x2f, 0x06, 0xb7, 0x4a, - 0x0d, 0xf2, 0x09, 0x83, 0x8d, 0x79, 0x50, 0xbf, 0xfc, 0xf3, 0x93, 0xca, 0x44, 0xe5, 0xe3, 0x7d, - 0xa8, 0xb9, 0x7c, 0x61, 0x56, 0x7b, 0xda, 0x4e, 0x7b, 0xef, 0xe3, 0x92, 0x32, 0xc3, 0xe9, 0xcb, - 0x71, 0x14, 0x7b, 0x34, 0xe4, 0x2a, 0x53, 0xc4, 0xe3, 0x63, 0x30, 0x82, 0x15, 0xbf, 0xf0, 0x6d, - 0x9a, 0xc4, 0x66, 0x4d, 0x26, 0x7f, 0x5e, 0x92, 0x7c, 0xb2, 0x9a, 0x3e, 0x7f, 0x36, 0x4e, 0xe2, - 0x53, 0xcf, 0x27, 0xc5, 0x2a, 0x2d, 0x99, 0x3e, 0x4e, 0x62, 0xfc, 0x1d, 0xe8, 0xd1, 0xdc, 0x76, - 0x69, 0xb4, 0x32, 0xeb, 0xb2, 0x50, 0xaf, 0xa4, 0xd0, 0x0f, 0xf3, 0x21, 0x8d, 0x56, 0xc5, 0x12, - 0xcd, 0x48, 0x3a, 0xf1, 0x14, 0xda, 0x2e, 0x0d, 0x22, 0x46, 0x38, 0xf7, 0x68, 0x68, 0x36, 0x24, - 0x23, 0x5f, 0xfe, 0x1f, 0x23, 0xc3, 0x4d, 0x8a, 0xaa, 0x97, 0xaf, 0xa2, 0xba, 0x9a, 0x25, 0x41, - 0x64, 0x36, 0xdf, 0xd1, 0xd5, 0xe3, 0x24, 0x88, 0x6e, 0x75, 0x25, 0x9c, 0xf8, 0x01, 0x74, 0xb9, - 0xb3, 0x20, 0x36, 0x23, 0x3f, 0x13, 0x37, 0x26, 0x33, 0x53, 0xef, 0x69, 0x3b, 0x2d, 0x15, 0xd4, - 0x11, 0xd0, 0x44, 0x21, 0xf8, 0x5b, 0xa8, 0x3b, 0x0b, 0x46, 0xcd, 0x96, 0x7c, 0xc8, 0x2a, 0x79, - 0xe8, 0xd1, 0x82, 0xd1, 0xe2, 0x33, 0x32, 0x03, 0x3f, 0x01, 0x48, 0x65, 0x90, 0x8d, 0x1a, 0x32, - 0xff, 0x5e, 0xa9, 0x0e, 0xfc, 0xc2, 0x9f, 0xdd, 0xea, 0x35, 0xd5, 0x50, 0xb4, 0xdb, 0x27, 0x00, - 0xb9, 0xfd, 0x6a, 0x83, 0xfe, 0x22, 0x3c, 0x0f, 0xe9, 0x32, 0x44, 0x15, 0xac, 0x43, 0x6d, 0x38, - 0x7d, 0x89, 0x34, 0x8c, 0xa0, 0x73, 0xa2, 0x54, 0x13, 0x82, 0xa2, 0x2a, 0xee, 0x82, 0xb1, 0x2e, - 0x8d, 0x6a, 0x18, 0xa0, 0x99, 0x0a, 0x85, 0xea, 0xa9, 0x2d, 0x4a, 0xa3, 0x06, 0x6e, 0x41, 0x5d, - 0x4c, 0x80, 0x9a, 0xfd, 0x7d, 0x68, 0xe7, 0x88, 0x97, 0x40, 0x12, 0x53, 0x54, 0x11, 0xd6, 0x88, - 0x86, 0x04, 0x69, 0xc2, 0xfa, 0xfe, 0xb5, 0x17, 0xa1, 0xaa, 0xb0, 0x0e, 0x84, 0x55, 0xeb, 0xff, - 0xa1, 0x01, 0x6c, 0x16, 0x11, 0xdf, 0x85, 0x86, 0x4b, 0x83, 0xc0, 0x91, 0xdb, 0xdf, 0x50, 0xc3, - 0xa4, 0x2e, 0x6c, 0x81, 0x2e, 0x0c, 0x12, 0xc6, 0x72, 0xa9, 0x33, 0x34, 0x73, 0x0a, 0x5d, 0xc2, - 0xc4, 0xf7, 0x6d, 0x12, 0xba, 0x74, 0xe6, 0x85, 0x73, 0xb9, 0xbd, 0x86, 0x8c, 0xd2, 0x26, 0x1d, - 0x01, 0x1d, 0x2a, 0x04, 0x9b, 0x50, 0xe7, 0xe7, 0x5e, 0x24, 0xd7, 0xb2, 0x9b, 0xf1, 0x2e, 0x3c, - 0x52, 0xdc, 0x98, 0x79, 0x6e, 0x6c, 0x5f, 0x24, 0x34, 0x26, 0x5c, 0x2e, 0xdd, 0x46, 0x5c, 0x09, - 0x3d, 0x97, 0x08, 0xfe, 0x14, 0x0c, 0x46, 0x97, 0xb6, 0xef, 0x05, 0x5e, 0x2c, 0x57, 0xa9, 0x96, - 0x7d, 0x01, 0x8c, 0x2e, 0x9f, 0x09, 0x6f, 0xff, 0xb7, 0x1a, 0x7c, 0x50, 0xf2, 0xa5, 0x88, 0x57, - 0x44, 0x2a, 0x27, 0x91, 0xc3, 0x9c, 0x98, 0xb2, 0xc2, 0xb8, 0x1d, 0x46, 0x97, 0xd3, 0x0c, 0xc1, - 0x0f, 0x61, 0xfb, 0xd4, 0x23, 0xfe, 0x2c, 0x17, 0x9c, 0x9f, 0x7e, 0x4b, 0x82, 0x9b, 0xf0, 0x11, - 0xe8, 0x24, 0x74, 0x7d, 0xca, 0x89, 0x1c, 0x7f, 0x6b, 0x6f, 0xf0, 0x7e, 0x1f, 0xef, 0xe0, 0x30, - 0xcd, 0xca, 0x48, 0x55, 0x45, 0x70, 0x0f, 0x5a, 0xca, 0x64, 0x92, 0xad, 0xec, 0xdd, 0xb5, 0x17, - 0xdf, 0x03, 0x38, 0x73, 0xb8, 0x4d, 0xb8, 0xeb, 0x44, 0xa4, 0x40, 0x97, 0x71, 0xe6, 0xf0, 0x43, - 0xe9, 0xc6, 0x1f, 0x41, 0x53, 0x05, 0x34, 0x73, 0x45, 0x94, 0x6f, 0x2d, 0x87, 0x5e, 0x26, 0x47, - 0x51, 0xd3, 0xd6, 0x7f, 0x6a, 0x5a, 0x90, 0x03, 0x4a, 0xe5, 0x18, 0x80, 0xae, 0xc6, 0xc4, 0x06, - 0x34, 0x46, 0x64, 0x41, 0x18, 0xaa, 0x88, 0x7d, 0x7e, 0xe4, 0x2f, 0x9d, 0x15, 0x47, 0x1a, 0xee, - 0x40, 0x2b, 0x25, 0xc4, 0xf1, 0x51, 0xf5, 0x69, 0xbd, 0x65, 0x20, 0xe8, 0x73, 0xe8, 0x16, 0x8e, - 0x14, 0xee, 0x83, 0x31, 0x23, 0xf2, 0x1d, 0x52, 0x54, 0x6e, 0xe3, 0x16, 0x23, 0x89, 0xee, 0xa4, - 0x56, 0x46, 0x36, 0x92, 0xf0, 0xe0, 0xcf, 0x00, 0x02, 0xe7, 0xd5, 0x84, 0x2e, 0xa7, 0xde, 0xeb, - 0x54, 0xa4, 0x2c, 0x3d, 0xe7, 0xef, 0xff, 0xa2, 0x89, 0x57, 0x73, 0x47, 0xe8, 0x46, 0x9e, 0x56, - 0x9e, 0x57, 0x64, 0xa1, 0x5a, 0xc6, 0x02, 0xfe, 0x1a, 0xb0, 0x37, 0x0f, 0x29, 0x23, 0x76, 0x12, - 0xf2, 0x24, 0x8a, 0x28, 0x13, 0x47, 0xac, 0x96, 0x13, 0xee, 0x4e, 0x8a, 0xbf, 0xd8, 0xc0, 0xfd, - 0x7d, 0x40, 0x37, 0x4f, 0x4d, 0xf1, 0x2d, 0xad, 0x94, 0xf1, 0xdf, 0xab, 0xd0, 0xce, 0x9d, 0x38, - 0x3c, 0xbc, 0xf1, 0x7b, 0xbb, 0xff, 0xee, 0x93, 0x38, 0x28, 0xfd, 0xb3, 0xdd, 0x87, 0xb6, 0xfa, - 0x46, 0x03, 0x3a, 0x23, 0x72, 0xca, 0xac, 0x73, 0x48, 0x81, 0x13, 0x3a, 0x23, 0x82, 0x30, 0xee, - 0x9e, 0x91, 0xc0, 0x79, 0x3a, 0x1d, 0x8f, 0x72, 0xc7, 0x40, 0x44, 0xad, 0xfd, 0xf8, 0x2b, 0xd8, - 0x0e, 0x9c, 0x57, 0x36, 0x23, 0x2e, 0x65, 0x33, 0x9b, 0x0b, 0x6e, 0xf3, 0x7b, 0xde, 0x15, 0xdc, - 0x4a, 0x4c, 0xd2, 0xbb, 0x0b, 0x28, 0x8b, 0x5c, 0x7f, 0x8e, 0x8d, 0x5c, 0xf8, 0x76, 0x8a, 0x6e, - 0xbe, 0xc7, 0xf7, 0x38, 0x12, 0xdf, 0x40, 0x53, 0x1d, 0x67, 0x1d, 0x6a, 0xe3, 0xe1, 0x11, 0xaa, - 0xe0, 0x6d, 0x68, 0x1f, 0x1c, 0x8f, 0xec, 0xc9, 0xe1, 0x70, 0x3c, 0x79, 0x3c, 0x4d, 0x0f, 0xb4, - 0xe8, 0x76, 0xed, 0xa9, 0x1e, 0x3c, 0xb8, 0xfc, 0xdb, 0xaa, 0x5c, 0x5e, 0x59, 0xda, 0x9b, 0x2b, - 0x4b, 0x7b, 0x7b, 0x65, 0x69, 0x7f, 0x5d, 0x59, 0xda, 0xaf, 0xd7, 0x56, 0xe5, 0xcd, 0xb5, 0x55, - 0x79, 0x7b, 0x6d, 0x55, 0x7e, 0xd4, 0x15, 0xa1, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x26, 0x62, - 0x48, 0xfc, 0xa8, 0x08, 0x00, 0x00, + proto.RegisterFile("roachpb/io-formats.proto", fileDescriptor_io_formats_b77b03f938e9e2ae) +} + +var fileDescriptor_io_formats_b77b03f938e9e2ae = []byte{ + // 986 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xcf, 0x6e, 0xdb, 0xc6, + 0x13, 0x16, 0xf5, 0x8f, 0xe2, 0x48, 0xb2, 0x99, 0xfd, 0x05, 0x3f, 0x10, 0x41, 0xcb, 0xaa, 0x4a, + 0x53, 0x38, 0x6d, 0x23, 0x03, 0x6e, 0x0d, 0x14, 0xbd, 0x14, 0xb1, 0x62, 0x37, 0x0e, 0x6c, 0xa9, + 0x91, 0x90, 0x1c, 0x7a, 0x21, 0x58, 0x72, 0x2d, 0xb3, 0x26, 0xb9, 0xf4, 0x2e, 0x29, 0x45, 0x79, + 0x8a, 0x3e, 0x53, 0x4f, 0xee, 0x2d, 0xb7, 0x06, 0x28, 0x50, 0xb4, 0xf6, 0x7b, 0x14, 0xc5, 0x2e, + 0x97, 0x12, 0x69, 0xb1, 0x69, 0x6e, 0x83, 0xf9, 0xe6, 0x1b, 0xce, 0xcc, 0x37, 0x3b, 0x04, 0x83, + 0x12, 0xdb, 0x39, 0x8f, 0x7e, 0xdc, 0xf5, 0xc8, 0xa3, 0x33, 0x42, 0x03, 0x3b, 0x66, 0x83, 0x88, + 0x92, 0x98, 0xa0, 0x3b, 0x0e, 0x71, 0x2e, 0x04, 0x3a, 0x90, 0x31, 0xf7, 0xee, 0xce, 0xc8, 0x8c, + 0x08, 0x74, 0x97, 0x5b, 0x69, 0x60, 0xff, 0xef, 0x06, 0x74, 0x8e, 0xc7, 0x47, 0x9e, 0x8f, 0x8f, + 0x44, 0x02, 0xf4, 0x14, 0x9a, 0x69, 0x2a, 0x43, 0xe9, 0x29, 0x3b, 0x5b, 0x7b, 0x9f, 0x0d, 0x36, + 0x52, 0x0d, 0xf2, 0x84, 0xc1, 0xda, 0x3c, 0xa8, 0x5f, 0xfd, 0xf1, 0x51, 0x65, 0x22, 0xf9, 0x68, + 0x1f, 0x6a, 0x0e, 0x9b, 0x1b, 0xd5, 0x9e, 0xb2, 0xd3, 0xde, 0xfb, 0xb0, 0x24, 0xcd, 0x70, 0xfa, + 0x72, 0x1c, 0xc5, 0x1e, 0x09, 0x99, 0x64, 0xf2, 0x78, 0x74, 0x0c, 0x5a, 0xb0, 0x64, 0x97, 0xbe, + 0x45, 0x92, 0xd8, 0xa8, 0x09, 0xf2, 0xa7, 0x25, 0xe4, 0xd3, 0xe5, 0xf4, 0xf9, 0xc9, 0x38, 0x89, + 0xcf, 0x3c, 0x1f, 0x17, 0xb3, 0xb4, 0x04, 0x7d, 0x9c, 0xc4, 0xe8, 0x5b, 0x50, 0xa3, 0x99, 0xe5, + 0x90, 0x68, 0x69, 0xd4, 0x45, 0xa2, 0x5e, 0x49, 0xa2, 0xef, 0x67, 0x43, 0x12, 0x2d, 0x8b, 0x29, + 0x9a, 0x91, 0x70, 0xa2, 0x29, 0xb4, 0x1d, 0x12, 0x44, 0x14, 0x33, 0xe6, 0x91, 0xd0, 0x68, 0x88, + 0x89, 0x7c, 0xfe, 0x5f, 0x13, 0x19, 0xae, 0x29, 0x32, 0x5f, 0x3e, 0x8b, 0xac, 0xca, 0x4d, 0x82, + 0xc8, 0x68, 0xbe, 0xa3, 0xaa, 0x27, 0x49, 0x10, 0x6d, 0x54, 0xc5, 0x9d, 0xe8, 0x21, 0x74, 0x99, + 0x3d, 0xc7, 0x16, 0xc5, 0x3f, 0x61, 0x27, 0xc6, 0xae, 0xa1, 0xf6, 0x94, 0x9d, 0x96, 0x0c, 0xea, + 0x70, 0x68, 0x22, 0x11, 0xf4, 0x35, 0xd4, 0xed, 0x39, 0x25, 0x46, 0x4b, 0x7c, 0xc8, 0x2c, 0xf9, + 0xd0, 0xe3, 0x39, 0x25, 0xc5, 0xcf, 0x08, 0x06, 0x7a, 0x0a, 0x90, 0xca, 0x20, 0x0a, 0xd5, 0x04, + 0xff, 0x7e, 0xa9, 0x0e, 0xec, 0xd2, 0x77, 0x37, 0x6a, 0x4d, 0x35, 0xe4, 0xe5, 0xf6, 0x31, 0x40, + 0x6e, 0xbf, 0xda, 0xa0, 0xbe, 0x08, 0x2f, 0x42, 0xb2, 0x08, 0xf5, 0x0a, 0x52, 0xa1, 0x36, 0x9c, + 0xbe, 0xd4, 0x15, 0xa4, 0x43, 0xe7, 0x54, 0xaa, 0xc6, 0x05, 0xd5, 0xab, 0xa8, 0x0b, 0xda, 0x2a, + 0xb5, 0x5e, 0x43, 0x00, 0xcd, 0x54, 0x28, 0xbd, 0x9e, 0xda, 0x3c, 0xb5, 0xde, 0x40, 0x2d, 0xa8, + 0xf3, 0x0e, 0xf4, 0x66, 0x7f, 0x1f, 0xda, 0xb9, 0xc1, 0x0b, 0x20, 0x89, 0x89, 0x5e, 0xe1, 0xd6, + 0x88, 0x84, 0x58, 0x57, 0xb8, 0xf5, 0xdd, 0x6b, 0x2f, 0xd2, 0xab, 0xdc, 0x3a, 0xe0, 0x56, 0xad, + 0xff, 0xbb, 0x02, 0xb0, 0x5e, 0x44, 0x74, 0x0f, 0x1a, 0x0e, 0x09, 0x02, 0x5b, 0x6c, 0x7f, 0x43, + 0x36, 0x93, 0xba, 0x90, 0x09, 0x2a, 0x37, 0x70, 0x18, 0x8b, 0xa5, 0xce, 0xd0, 0xcc, 0xc9, 0x75, + 0x09, 0x13, 0xdf, 0xb7, 0x70, 0xe8, 0x10, 0xd7, 0x0b, 0x67, 0x62, 0x7b, 0x35, 0x11, 0xa5, 0x4c, + 0x3a, 0x1c, 0x3a, 0x94, 0x08, 0x32, 0xa0, 0xce, 0x2e, 0xbc, 0x48, 0xac, 0x65, 0x37, 0x9b, 0x3b, + 0xf7, 0x08, 0x71, 0x63, 0xea, 0x39, 0xb1, 0x75, 0x99, 0x90, 0x18, 0x33, 0xb1, 0x74, 0x6b, 0x71, + 0x05, 0xf4, 0x5c, 0x20, 0xe8, 0x63, 0xd0, 0x28, 0x59, 0x58, 0xbe, 0x17, 0x78, 0xb1, 0x58, 0xa5, + 0x5a, 0xf6, 0x02, 0x28, 0x59, 0x9c, 0x70, 0x6f, 0xff, 0x97, 0x1a, 0xfc, 0xaf, 0xe4, 0xa5, 0xf0, + 0xaf, 0x70, 0x2a, 0xc3, 0x91, 0x4d, 0xed, 0x98, 0xd0, 0x42, 0xbb, 0x1d, 0x4a, 0x16, 0xd3, 0x0c, + 0x41, 0x8f, 0x60, 0xfb, 0xcc, 0xc3, 0xbe, 0x9b, 0x0b, 0xce, 0x77, 0xbf, 0x25, 0xc0, 0x75, 0xf8, + 0x08, 0x54, 0x1c, 0x3a, 0x3e, 0x61, 0x58, 0xb4, 0xbf, 0xb5, 0x37, 0x78, 0xbf, 0xc7, 0x3b, 0x38, + 0x4c, 0x59, 0xd9, 0x50, 0x65, 0x12, 0xd4, 0x83, 0x96, 0x34, 0xa9, 0x98, 0x56, 0xf6, 0xdd, 0x95, + 0x17, 0xdd, 0x07, 0x38, 0xb7, 0x99, 0x85, 0x99, 0x63, 0x47, 0xb8, 0x30, 0x2e, 0xed, 0xdc, 0x66, + 0x87, 0xc2, 0x8d, 0x3e, 0x80, 0xa6, 0x0c, 0x68, 0xe6, 0x92, 0x48, 0xdf, 0x4a, 0x0e, 0xb5, 0x4c, + 0x8e, 0xa2, 0xa6, 0xad, 0x7f, 0xd5, 0xb4, 0x20, 0x07, 0x94, 0xca, 0x31, 0x00, 0x55, 0xb6, 0x89, + 0x34, 0x68, 0x8c, 0xf0, 0x1c, 0x53, 0xbd, 0xc2, 0xf7, 0xf9, 0xb1, 0xbf, 0xb0, 0x97, 0x4c, 0x57, + 0x50, 0x07, 0x5a, 0xe9, 0x40, 0x6c, 0x5f, 0xaf, 0x3e, 0xab, 0xb7, 0x34, 0x1d, 0xfa, 0x0c, 0xba, + 0x85, 0x23, 0x85, 0xfa, 0xa0, 0xb9, 0x58, 0x7c, 0x07, 0x17, 0x95, 0x5b, 0xbb, 0x79, 0x4b, 0xbc, + 0x3a, 0xa1, 0x95, 0x96, 0xb5, 0xc4, 0x3d, 0xe8, 0x13, 0x80, 0xc0, 0x7e, 0x35, 0x21, 0x8b, 0xa9, + 0xf7, 0x3a, 0x15, 0x29, 0xa3, 0xe7, 0xfc, 0xfd, 0x5f, 0x15, 0xfe, 0xd5, 0xdc, 0x11, 0xba, 0xc5, + 0x53, 0xca, 0x79, 0xc5, 0x29, 0x54, 0xcb, 0xa6, 0x80, 0xbe, 0x04, 0xe4, 0xcd, 0x42, 0x42, 0xb1, + 0x95, 0x84, 0x2c, 0x89, 0x22, 0x42, 0xf9, 0x11, 0xab, 0xe5, 0x84, 0xbb, 0x93, 0xe2, 0x2f, 0xd6, + 0x30, 0xfa, 0x06, 0xfe, 0xbf, 0x49, 0xb2, 0x7c, 0x32, 0x13, 0x5b, 0x91, 0x75, 0x78, 0x77, 0x83, + 0x78, 0x42, 0x66, 0xfd, 0x7d, 0xd0, 0x6f, 0x9f, 0xa9, 0x62, 0x9d, 0x4a, 0xa9, 0x5a, 0xbf, 0x55, + 0xa1, 0x9d, 0x3b, 0x8f, 0x68, 0x78, 0xeb, 0xd7, 0xf8, 0xe0, 0xdd, 0xe7, 0x74, 0x50, 0xfa, 0x57, + 0x7c, 0x00, 0x6d, 0xf9, 0xbe, 0x03, 0xe2, 0x62, 0x31, 0xa1, 0xac, 0x6b, 0x48, 0x81, 0x53, 0xe2, + 0x62, 0x3e, 0x6c, 0xe6, 0x9c, 0xe3, 0xc0, 0x7e, 0x36, 0x1d, 0x8f, 0x72, 0x87, 0x84, 0x47, 0xad, + 0xfc, 0xe8, 0x0b, 0xd8, 0x0e, 0xec, 0x57, 0x16, 0xc5, 0x0e, 0xa1, 0xae, 0xc5, 0xb8, 0x2e, 0xf9, + 0x37, 0xd2, 0xe5, 0xba, 0x08, 0x4c, 0x48, 0xb3, 0x0b, 0x7a, 0x16, 0xb9, 0x7a, 0xca, 0x8d, 0x5c, + 0xf8, 0x76, 0x8a, 0xae, 0xdf, 0xf2, 0x7b, 0x1c, 0x98, 0xaf, 0xa0, 0x29, 0x0f, 0xbb, 0x0a, 0xb5, + 0xf1, 0xf0, 0x48, 0xaf, 0xa0, 0x6d, 0x68, 0x1f, 0x1c, 0x8f, 0xac, 0xc9, 0xe1, 0x70, 0x3c, 0x79, + 0x32, 0x4d, 0x8f, 0x3b, 0xaf, 0x76, 0xe5, 0xa9, 0x1e, 0x3c, 0xbc, 0xfa, 0xcb, 0xac, 0x5c, 0x5d, + 0x9b, 0xca, 0x9b, 0x6b, 0x53, 0x79, 0x7b, 0x6d, 0x2a, 0x7f, 0x5e, 0x9b, 0xca, 0xcf, 0x37, 0x66, + 0xe5, 0xcd, 0x8d, 0x59, 0x79, 0x7b, 0x63, 0x56, 0x7e, 0x50, 0xe5, 0x40, 0xff, 0x09, 0x00, 0x00, + 0xff, 0xff, 0x17, 0x19, 0x54, 0x15, 0xe4, 0x08, 0x00, 0x00, } diff --git a/pkg/roachpb/io-formats.proto b/pkg/roachpb/io-formats.proto index d47d4994844b..e7291619c67b 100644 --- a/pkg/roachpb/io-formats.proto +++ b/pkg/roachpb/io-formats.proto @@ -113,6 +113,11 @@ message PgDumpOptions { // Indicates if all unparseable and parseable, but unimplemented PGDUMP stmts // should be ignored during IMPORT. optional bool ignore_unsupported = 3 [(gogoproto.nullable) = false]; + // Points to the destination where unsupported statements during a PGDUMP + // import should be logged. This can only be used when ignore_unsupported is + // specified, otherwise the IMPORT errors out on encountering an unsupported + // stmt. + optional string ignore_unsupported_log = 4 [(gogoproto.nullable) = false]; } message MysqldumpOptions { diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 0059fc748225..bf56e8d61a46 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -87,7 +87,7 @@ go_library( "type_check.go", "type_name.go", "union.go", - "unsupported.go", + "unsupported_error.go", "update.go", "values.go", "var_name.go",