From dc7075440c6484337cb739993fe661bbb57b360a Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Fri, 11 Dec 2020 11:10:05 -0500 Subject: [PATCH] importccl: cleanup how we ignore stmts in IMPORT PGDUMP There are two kinds of unsupported statements when processing an IMPORT PGDUMP: - Unparseable - Parseable, but unsupported Previously, we had a set of regex statements to skip over statements which fell into the first category. Those which fell into the second category were skipped when we saw an AST Node of that kind. This commit is responsible for the following: - Removes regex and adds/modifies yacc rules to encompass these regex representations. - Introduces a special UnsupportedNode which backs all yacc rules which previously returned an unimplemented error. - Introduces an IMPORT PGDUMP option to `ignore_unsupported` statements. - All statements which are not parseable will now be ignored if the option is set. Note: the eventual goal is to have the `ignore_unsupported` flag also allow users to ignore parseable, but unsupported statements. This will be done in the next commit. Release note (sql change): New IMPORT PGDUMP option `ignore_unsupported` to skip over all the unsupported PGDUMP stmts. The collection of these statements will be appropriately documented. --- pkg/ccl/importccl/import_stmt.go | 13 +- pkg/ccl/importccl/import_stmt_test.go | 90 ++++++--- pkg/ccl/importccl/read_import_pgdump.go | 73 +++---- pkg/ccl/importccl/read_import_pgdump_test.go | 6 +- pkg/roachpb/io-formats.pb.go | 198 +++++++++++-------- pkg/roachpb/io-formats.proto | 3 + pkg/sql/parser/lexer.go | 44 +++-- pkg/sql/parser/sql.y | 24 ++- pkg/sql/sem/tree/BUILD.bazel | 1 + pkg/sql/sem/tree/unsupported.go | 31 +++ 10 files changed, 320 insertions(+), 163 deletions(-) create mode 100644 pkg/sql/sem/tree/unsupported.go diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index b69e61ff2f14..7e73ef1bcfee 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -99,6 +99,8 @@ const ( avroSchema = "schema" avroSchemaURI = "schema_uri" + pgDumpIgnoreAllUnsupported = "ignore_unsupported" + // RunningStatusImportBundleParseSchema indicates to the user that a bundle format // schema is being parsed runningStatusImportBundleParseSchema jobs.RunningStatus = "parsing schema on Import Bundle" @@ -133,6 +135,8 @@ var importOptionExpectValues = map[string]sql.KVStringOptValidate{ avroRecordsSeparatedBy: sql.KVStringOptRequireValue, avroBinRecords: sql.KVStringOptRequireNoValue, avroJSONRecords: sql.KVStringOptRequireNoValue, + + pgDumpIgnoreAllUnsupported: sql.KVStringOptRequireNoValue, } func makeStringSet(opts ...string) map[string]struct{} { @@ -162,7 +166,8 @@ var mysqlOutAllowedOptions = makeStringSet( ) var mysqlDumpAllowedOptions = makeStringSet(importOptionSkipFKs, csvRowLimit) var pgCopyAllowedOptions = makeStringSet(pgCopyDelimiter, pgCopyNull, optMaxRowSize) -var pgDumpAllowedOptions = makeStringSet(optMaxRowSize, importOptionSkipFKs, csvRowLimit) +var pgDumpAllowedOptions = makeStringSet(optMaxRowSize, importOptionSkipFKs, csvRowLimit, + pgDumpIgnoreAllUnsupported) // DROP is required because the target table needs to be take offline during // IMPORT INTO. @@ -607,6 +612,9 @@ func importPlanHook( maxRowSize = int32(sz) } format.PgDump.MaxRowSize = maxRowSize + if _, ok := opts[pgDumpIgnoreAllUnsupported]; ok { + format.PgDump.IgnoreUnsupported = true + } if override, ok := opts[csvRowLimit]; ok { rowLimit, err := strconv.Atoi(override) @@ -1298,7 +1306,8 @@ func parseAndCreateBundleTableDescs( tableDescs, err = readMysqlCreateTable(ctx, reader, evalCtx, p, defaultCSVTableID, parentID, tableName, fks, seqVals, owner, walltime) case roachpb.IOFileFormat_PgDump: evalCtx := &p.ExtendedEvalContext().EvalContext - tableDescs, err = readPostgresCreateTable(ctx, reader, evalCtx, p, tableName, parentID, walltime, fks, int(format.PgDump.MaxRowSize), owner) + tableDescs, err = readPostgresCreateTable(ctx, reader, evalCtx, p, tableName, parentID, + walltime, fks, int(format.PgDump.MaxRowSize), owner, format.PgDump.IgnoreUnsupported) 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 eb9f171e0ee1..378b823cdc9a 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -1020,29 +1020,6 @@ END; data: "create table s.t (i INT8)", err: `non-public schemas unsupported: s`, }, - { - name: "various create ignores", - typ: "PGDUMP", - data: ` - 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 FUNCTION public.isnumeric(text) RETURNS boolean - LANGUAGE sql - AS $_$ - SELECT $1 ~ '^[0-9]+$' - $_$; - ALTER FUNCTION public.isnumeric(text) OWNER TO roland; - - CREATE TABLE t (i INT8); - `, - query: map[string][][]string{ - getTablesQuery: {{"public", "t", "table"}}, - }, - }, { name: "many tables", typ: "PGDUMP", @@ -1597,7 +1574,8 @@ func TestImportRowLimit(t *testing.T) { expectedRowLimit := 4 // Import a single table `second` and verify number of rows imported. - importQuery := fmt.Sprintf(`IMPORT TABLE second FROM PGDUMP ($1) WITH row_limit="%d"`, expectedRowLimit) + importQuery := fmt.Sprintf(`IMPORT TABLE second FROM PGDUMP ($1) WITH row_limit="%d",ignore_unsupported`, + expectedRowLimit) sqlDB.Exec(t, importQuery, second...) var numRows int @@ -1608,7 +1586,7 @@ func TestImportRowLimit(t *testing.T) { // Import multiple tables including `simple` and `second`. expectedRowLimit = 3 - importQuery = fmt.Sprintf(`IMPORT PGDUMP ($1) WITH row_limit="%d"`, expectedRowLimit) + importQuery = fmt.Sprintf(`IMPORT PGDUMP ($1) WITH row_limit="%d",ignore_unsupported`, expectedRowLimit) sqlDB.Exec(t, importQuery, multitable...) sqlDB.QueryRow(t, "SELECT count(*) FROM second").Scan(&numRows) require.Equal(t, expectedRowLimit, numRows) @@ -5536,9 +5514,9 @@ func TestImportPgDump(t *testing.T) { }, {`single table dump`, expectSimple, `IMPORT TABLE simple FROM PGDUMP ($1)`, simple}, {`second table dump`, expectSecond, `IMPORT TABLE second FROM PGDUMP ($1)`, second}, - {`simple from multi`, expectSimple, `IMPORT TABLE simple FROM PGDUMP ($1)`, multitable}, - {`second from multi`, expectSecond, `IMPORT TABLE second FROM PGDUMP ($1)`, multitable}, - {`all from multi`, expectAll, `IMPORT PGDUMP ($1)`, multitable}, + {`simple from multi`, expectSimple, `IMPORT TABLE simple FROM PGDUMP ($1) WITH ignore_unsupported`, multitable}, + {`second from multi`, expectSecond, `IMPORT TABLE second FROM PGDUMP ($1) WITH ignore_unsupported`, multitable}, + {`all from multi`, expectAll, `IMPORT PGDUMP ($1) WITH ignore_unsupported`, multitable}, } { t.Run(c.name, func(t *testing.T) { sqlDB.Exec(t, `DROP TABLE IF EXISTS simple, second`) @@ -5676,6 +5654,62 @@ func TestImportPgDump(t *testing.T) { }) } +func TestImportPgDumpIgnoredStmts(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1 /* nodes */, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + conn := tc.Conns[0] + sqlDB := sqlutils.MakeSQLRunner(conn) + + data := ` + 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); + + CREATE FUNCTION public.isnumeric(text) RETURNS boolean + LANGUAGE sql + AS $_$ + SELECT $1 ~ '^[0-9]+$' + $_$; + ALTER FUNCTION public.isnumeric(text) OWNER TO roland; + + INSERT INTO foo VALUES (1), (2), (3); + + CREATE TABLE t (i INT8); + 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; + ` + + srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if r.Method == "GET" { + _, _ = w.Write([]byte(data)) + } + })) + defer srv.Close() + t.Run("ignore-unsupported", func(t *testing.T) { + 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.ExpectErr(t, "syntax error", "IMPORT PGDUMP ($1)", srv.URL) + }) +} + // TestImportPgDumpGeo tests that a file with SQLFn classes can be // imported. These are functions like AddGeometryColumn which create and // execute SQL when called (!). They are, for example, used by shp2pgsql diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go index fe17f28b4d8e..e6c59e343bee 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -42,16 +42,17 @@ import ( ) type postgreStream struct { - s *bufio.Scanner - copy *postgreStreamCopy + s *bufio.Scanner + copy *postgreStreamCopy + ignoreUnsupportedStmts bool } // newPostgreStream returns a struct that can stream statements from an // io.Reader. -func newPostgreStream(r io.Reader, max int) *postgreStream { +func newPostgreStream(r io.Reader, max int, ignoreUnsupportedStmts bool) *postgreStream { s := bufio.NewScanner(r) s.Buffer(nil, max) - p := &postgreStream{s: s} + p := &postgreStream{s: s, ignoreUnsupportedStmts: ignoreUnsupportedStmts} s.Split(p.split) return p } @@ -95,14 +96,19 @@ func (p *postgreStream) Next() (interface{}, error) { for p.s.Scan() { t := p.s.Text() - // Regardless if we can parse the statement, check that it's not something - // we want to ignore. - if isIgnoredStatement(t) { - continue - } + skipOverComments(t) 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 errors.HasType(err, (*tree.Unsupported)(nil)) { + if p.ignoreUnsupportedStmts { + continue + } + } return nil, err } switch len(stmts) { @@ -145,20 +151,10 @@ func (p *postgreStream) Next() (interface{}, error) { } var ( - ignoreComments = regexp.MustCompile(`^\s*(--.*)`) - ignoreStatements = []*regexp.Regexp{ - regexp.MustCompile("(?i)^alter function"), - regexp.MustCompile("(?i)^alter sequence .* owned by"), - regexp.MustCompile("(?i)^comment on"), - regexp.MustCompile("(?i)^create extension"), - regexp.MustCompile("(?i)^create function"), - regexp.MustCompile("(?i)^create trigger"), - regexp.MustCompile("(?i)^grant .* on sequence"), - regexp.MustCompile("(?i)^revoke .* on sequence"), - } + ignoreComments = regexp.MustCompile(`^\s*(--.*)`) ) -func isIgnoredStatement(s string) bool { +func skipOverComments(s string) { // Look for the first line with no whitespace or comments. for { m := ignoreComments.FindStringIndex(s) @@ -167,13 +163,6 @@ func isIgnoredStatement(s string) bool { } s = s[m[1]:] } - s = strings.TrimSpace(s) - for _, re := range ignoreStatements { - if re.MatchString(s) { - return true - } - } - return false } type regclassRewriter struct{} @@ -229,6 +218,7 @@ func readPostgresCreateTable( fks fkHandler, max int, owner security.SQLUsername, + ignoreUnsupportedStmts bool, ) ([]*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 @@ -239,7 +229,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) + ps := newPostgreStream(input, max, ignoreUnsupportedStmts) for { stmt, err := ps.Next() if err == io.EOF { @@ -310,7 +300,8 @@ func readPostgresCreateTable( if err != nil { return nil, errors.Wrap(err, "postgres parse error") } - if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, tableFKs, stmt, p, parentID); err != nil { + if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, tableFKs, stmt, p, + parentID, ignoreUnsupportedStmts); err != nil { return nil, err } } @@ -327,6 +318,7 @@ func readPostgresStmt( stmt interface{}, p sql.JobExecContext, parentID descpb.ID, + ignoreUnsupportedStmts bool, ) error { switch stmt := stmt.(type) { case *tree.CreateTable: @@ -438,6 +430,11 @@ func readPostgresStmt( if match == "" || match == name { createSeq[name] = stmt } + case *tree.AlterSequence: + if len(stmt.Options) != 1 || stmt.Options[0].Name != tree.SeqOptOwnedBy || + !ignoreUnsupportedStmts { + return errors.Errorf("unsupported %T statement: %s", stmt, stmt) + } // Some SELECT statements mutate schema. Search for those here. case *tree.Select: switch sel := stmt.Select.(type) { @@ -493,7 +490,8 @@ func readPostgresStmt( for _, fnStmt := range fnStmts { switch ast := fnStmt.AST.(type) { case *tree.AlterTable: - if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, tableFKs, ast, p, parentID); err != nil { + if err := readPostgresStmt(ctx, evalCtx, match, fks, createTbl, createSeq, + tableFKs, ast, p, parentID, ignoreUnsupportedStmts); err != nil { return err } default: @@ -534,13 +532,16 @@ func readPostgresStmt( return err } } + case *tree.CreateExtension, *tree.CommentOnDatabase, *tree.CommentOnTable, + *tree.CommentOnIndex, *tree.CommentOnColumn: + // ignore parsed by unsupported stmts. case *tree.BeginTransaction, *tree.CommitTransaction: // ignore txns. case *tree.SetVar, *tree.Insert, *tree.CopyFrom, copyData, *tree.Delete: // ignore SETs and DMLs. case *tree.Analyze: - // ANALYZE is syntatictic sugar for CreateStatistics. It can be ignored because - // the auto stats stuff will pick up the changes and run if needed. + // ANALYZE is syntactic sugar for CreateStatistics. It can be ignored because + // the auto stats stuff will pick up the changes and run if needed. case error: if !errors.Is(stmt, errCopyDone) { return stmt @@ -655,7 +656,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)) + ps := newPostgreStream(input, int(m.opts.MaxRowSize), m.opts.IgnoreUnsupported) semaCtx := tree.MakeSemaContext() for _, conv := range m.tables { conv.KvBatch.Source = inputIdx @@ -858,6 +859,7 @@ func (m *pgDumpReader) readFile( switch funcName := strings.ToLower(fn.Func.String()); funcName { case "search_path", "pg_catalog.set_config": + continue case "setval", "pg_catalog.setval": if args := len(fn.Exprs); args < 2 || args > 3 { @@ -913,6 +915,9 @@ func (m *pgDumpReader) readFile( default: return errors.Errorf("unsupported function: %s", funcName) } + case *tree.CreateExtension, *tree.CommentOnDatabase, *tree.CommentOnTable, + *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.AlterSequence: + // parseable but ignored during schema extraction. case *tree.SetVar, *tree.BeginTransaction, *tree.CommitTransaction, *tree.Analyze: // ignored. case *tree.CreateTable, *tree.AlterTable, *tree.AlterTableOwner, *tree.CreateIndex, *tree.CreateSequence, *tree.DropTable: diff --git a/pkg/ccl/importccl/read_import_pgdump_test.go b/pkg/ccl/importccl/read_import_pgdump_test.go index b883810426ae..16d57be0cb6f 100644 --- a/pkg/ccl/importccl/read_import_pgdump_test.go +++ b/pkg/ccl/importccl/read_import_pgdump_test.go @@ -39,7 +39,8 @@ select '123456789012345678901234567890123456789012345678901234567890123456789012 -- ` - p := newPostgreStream(strings.NewReader(sql), defaultScanBuffer) + p := newPostgreStream(strings.NewReader(sql), defaultScanBuffer, + false /* ignoreUnsupportedStmts */) var sb strings.Builder for { s, err := p.Next() @@ -121,7 +122,8 @@ COPY public.t (s) FROM stdin; -- ` - p := newPostgreStream(strings.NewReader(sql), defaultScanBuffer) + p := newPostgreStream(strings.NewReader(sql), defaultScanBuffer, + false /* ignoreUnsupportedStmts */) var sb strings.Builder for { s, err := p.Next() diff --git a/pkg/roachpb/io-formats.pb.go b/pkg/roachpb/io-formats.pb.go index 45b572a6e788..a48fe7148c30 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_cf12087409a41794, []int{0, 0} + return fileDescriptor_io_formats_802d999645186915, []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_cf12087409a41794, []int{0, 1} + return fileDescriptor_io_formats_802d999645186915, []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_cf12087409a41794, []int{2, 0} + return fileDescriptor_io_formats_802d999645186915, []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_cf12087409a41794, []int{6, 0} + return fileDescriptor_io_formats_802d999645186915, []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_cf12087409a41794, []int{0} + return fileDescriptor_io_formats_802d999645186915, []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_cf12087409a41794, []int{1} + return fileDescriptor_io_formats_802d999645186915, []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_cf12087409a41794, []int{2} + return fileDescriptor_io_formats_802d999645186915, []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_cf12087409a41794, []int{3} + return fileDescriptor_io_formats_802d999645186915, []int{3} } func (m *PgCopyOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -380,13 +380,16 @@ type PgDumpOptions struct { // Indicates the number of rows to import per table. // Must be a non-zero positive number. RowLimit int64 `protobuf:"varint,2,opt,name=row_limit,json=rowLimit" json:"row_limit"` + // 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"` } 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_cf12087409a41794, []int{4} + return fileDescriptor_io_formats_802d999645186915, []int{4} } func (m *PgDumpOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -421,7 +424,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_cf12087409a41794, []int{5} + return fileDescriptor_io_formats_802d999645186915, []int{5} } func (m *MysqldumpOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -464,7 +467,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_cf12087409a41794, []int{6} + return fileDescriptor_io_formats_802d999645186915, []int{6} } func (m *AvroOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -728,6 +731,14 @@ func (m *PgDumpOptions) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x10 i++ i = encodeVarintIoFormats(dAtA, i, uint64(m.RowLimit)) + dAtA[i] = 0x18 + i++ + if m.IgnoreUnsupported { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ return i, nil } @@ -887,6 +898,7 @@ func (m *PgDumpOptions) Size() (n int) { _ = l n += 1 + sovIoFormats(uint64(m.MaxRowSize)) n += 1 + sovIoFormats(uint64(m.RowLimit)) + n += 2 return n } @@ -1202,7 +1214,7 @@ func (m *IOFileFormat) Unmarshal(dAtA []byte) error { if err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + if skippy < 0 { return ErrInvalidLengthIoFormats } if (iNdEx + skippy) > l { @@ -1378,7 +1390,7 @@ func (m *CSVOptions) Unmarshal(dAtA []byte) error { if err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + if skippy < 0 { return ErrInvalidLengthIoFormats } if (iNdEx + skippy) > l { @@ -1611,7 +1623,7 @@ func (m *MySQLOutfileOptions) Unmarshal(dAtA []byte) error { if err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + if skippy < 0 { return ErrInvalidLengthIoFormats } if (iNdEx + skippy) > l { @@ -1728,7 +1740,7 @@ func (m *PgCopyOptions) Unmarshal(dAtA []byte) error { if err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + if skippy < 0 { return ErrInvalidLengthIoFormats } if (iNdEx + skippy) > l { @@ -1810,13 +1822,33 @@ func (m *PgDumpOptions) Unmarshal(dAtA []byte) error { break } } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IgnoreUnsupported", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIoFormats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.IgnoreUnsupported = bool(v != 0) default: iNdEx = preIndex skippy, err := skipIoFormats(dAtA[iNdEx:]) if err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + if skippy < 0 { return ErrInvalidLengthIoFormats } if (iNdEx + skippy) > l { @@ -1885,7 +1917,7 @@ func (m *MysqldumpOptions) Unmarshal(dAtA []byte) error { if err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + if skippy < 0 { return ErrInvalidLengthIoFormats } if (iNdEx + skippy) > l { @@ -2060,7 +2092,7 @@ func (m *AvroOptions) Unmarshal(dAtA []byte) error { if err != nil { return err } - if (skippy < 0) || (iNdEx+skippy) < 0 { + if skippy < 0 { return ErrInvalidLengthIoFormats } if (iNdEx + skippy) > l { @@ -2181,68 +2213,70 @@ var ( ) func init() { - proto.RegisterFile("roachpb/io-formats.proto", fileDescriptor_io_formats_cf12087409a41794) -} - -var fileDescriptor_io_formats_cf12087409a41794 = []byte{ - // 938 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0x5f, 0x6f, 0xe3, 0x44, - 0x10, 0xb7, 0xf3, 0xcf, 0xf1, 0x24, 0x69, 0x97, 0x85, 0x07, 0xeb, 0x04, 0x26, 0xe4, 0x38, 0xd4, - 0x03, 0xce, 0x95, 0x2a, 0x2a, 0xf1, 0x86, 0xae, 0xb9, 0x96, 0xeb, 0xe9, 0x9a, 0x70, 0x89, 0x38, - 0x21, 0x5e, 0x2c, 0x63, 0x6f, 0x53, 0x53, 0xdb, 0xeb, 0x7a, 0x9d, 0xe4, 0x72, 0x9f, 0x82, 0xcf, - 0xc4, 0x53, 0x1f, 0xef, 0x8d, 0x93, 0x90, 0x10, 0xb4, 0xdf, 0x03, 0xa1, 0x5d, 0xaf, 0x13, 0xbb, - 0x35, 0x77, 0x7d, 0x1b, 0xcd, 0x6f, 0x7e, 0xb3, 0x33, 0xf3, 0x9b, 0xdd, 0x05, 0x23, 0xa1, 0x8e, - 0x7b, 0x16, 0xff, 0xb2, 0xeb, 0xd3, 0x47, 0xa7, 0x34, 0x09, 0x9d, 0x94, 0x59, 0x71, 0x42, 0x53, - 0x8a, 0x3f, 0x70, 0xa9, 0x7b, 0x2e, 0x50, 0x4b, 0xc6, 0xdc, 0xfb, 0x68, 0x46, 0x67, 0x54, 0xa0, - 0xbb, 0xdc, 0xca, 0x02, 0x07, 0xff, 0x36, 0xa1, 0x7b, 0x3c, 0x3e, 0xf2, 0x03, 0x72, 0x24, 0x12, - 0xe0, 0xa7, 0xd0, 0xca, 0x52, 0x19, 0x6a, 0x5f, 0xdd, 0xd9, 0xda, 0xfb, 0xd2, 0xba, 0x95, 0xca, - 0x2a, 0x12, 0xac, 0x8d, 0x79, 0xd0, 0xb8, 0xfc, 0xeb, 0x53, 0x65, 0x22, 0xf9, 0x78, 0x1f, 0xea, - 0x2e, 0x5b, 0x18, 0xb5, 0xbe, 0xba, 0xd3, 0xd9, 0xfb, 0xa4, 0x22, 0xcd, 0x70, 0xfa, 0x72, 0x1c, - 0xa7, 0x3e, 0x8d, 0x98, 0x64, 0xf2, 0x78, 0x7c, 0x0c, 0x7a, 0xb8, 0x62, 0x17, 0x81, 0x4d, 0xe7, - 0xa9, 0x51, 0x17, 0xe4, 0x2f, 0x2a, 0xc8, 0x27, 0xab, 0xe9, 0x8b, 0xe7, 0xe3, 0x79, 0x7a, 0xea, - 0x07, 0xa4, 0x9c, 0xa5, 0x2d, 0xe8, 0xe3, 0x79, 0x8a, 0xbf, 0x03, 0x2d, 0x9e, 0xd9, 0x2e, 0x8d, - 0x57, 0x46, 0x43, 0x24, 0xea, 0x57, 0x24, 0xfa, 0x61, 0x36, 0xa4, 0xf1, 0xaa, 0x9c, 0xa2, 0x15, - 0x0b, 0x27, 0x9e, 0x42, 0xc7, 0xa5, 0x61, 0x9c, 0x10, 0xc6, 0x7c, 0x1a, 0x19, 0x4d, 0x31, 0x91, - 0xaf, 0xde, 0x37, 0x91, 0xe1, 0x86, 0x22, 0xf3, 0x15, 0xb3, 0xc8, 0xaa, 0xbc, 0x79, 0x18, 0x1b, - 0xad, 0x77, 0x54, 0xf5, 0x64, 0x1e, 0xc6, 0xb7, 0xaa, 0xe2, 0x4e, 0xfc, 0x10, 0x7a, 0xcc, 0x59, - 0x10, 0x3b, 0x21, 0xbf, 0x12, 0x37, 0x25, 0x9e, 0xa1, 0xf5, 0xd5, 0x9d, 0xb6, 0x0c, 0xea, 0x72, - 0x68, 0x22, 0x11, 0xfc, 0x2d, 0x34, 0x9c, 0x45, 0x42, 0x8d, 0xb6, 0x38, 0xc8, 0xac, 0x38, 0xe8, - 0xf1, 0x22, 0xa1, 0xe5, 0x63, 0x04, 0x03, 0x3f, 0x05, 0xc8, 0x64, 0x10, 0x85, 0xea, 0x82, 0x7f, - 0xbf, 0x52, 0x07, 0x76, 0x11, 0x78, 0xb7, 0x6a, 0xcd, 0x34, 0xe4, 0xe5, 0x0e, 0x08, 0x40, 0x61, - 0xbf, 0x3a, 0xa0, 0xfd, 0x18, 0x9d, 0x47, 0x74, 0x19, 0x21, 0x05, 0x6b, 0x50, 0x1f, 0x4e, 0x5f, - 0x22, 0x15, 0x23, 0xe8, 0x9e, 0x48, 0xd5, 0xb8, 0xa0, 0xa8, 0x86, 0x7b, 0xa0, 0xaf, 0x53, 0xa3, - 0x3a, 0x06, 0x68, 0x65, 0x42, 0xa1, 0x46, 0x66, 0xf3, 0xd4, 0xa8, 0x89, 0xdb, 0xd0, 0xe0, 0x1d, - 0xa0, 0xd6, 0x60, 0x1f, 0x3a, 0x85, 0xc1, 0x0b, 0x60, 0x9e, 0x52, 0xa4, 0x70, 0x6b, 0x44, 0x23, - 0x82, 0x54, 0x6e, 0x7d, 0xff, 0xda, 0x8f, 0x51, 0x8d, 0x5b, 0x07, 0xdc, 0xaa, 0x0f, 0xfe, 0x54, - 0x01, 0x36, 0x8b, 0x88, 0xef, 0x41, 0xd3, 0xa5, 0x61, 0xe8, 0x88, 0xed, 0x6f, 0xca, 0x66, 0x32, - 0x17, 0x36, 0x41, 0xe3, 0x06, 0x89, 0x52, 0xb1, 0xd4, 0x39, 0x9a, 0x3b, 0xb9, 0x2e, 0xd1, 0x3c, - 0x08, 0x6c, 0x12, 0xb9, 0xd4, 0xf3, 0xa3, 0x99, 0xd8, 0x5e, 0x5d, 0x44, 0xa9, 0x93, 0x2e, 0x87, - 0x0e, 0x25, 0x82, 0x0d, 0x68, 0xb0, 0x73, 0x3f, 0x16, 0x6b, 0xd9, 0xcb, 0xe7, 0xce, 0x3d, 0x42, - 0xdc, 0x34, 0xf1, 0xdd, 0xd4, 0xbe, 0x98, 0xd3, 0x94, 0x30, 0xb1, 0x74, 0x1b, 0x71, 0x05, 0xf4, - 0x42, 0x20, 0xf8, 0x33, 0xd0, 0x13, 0xba, 0xb4, 0x03, 0x3f, 0xf4, 0x53, 0xb1, 0x4a, 0xf5, 0xfc, - 0x06, 0x24, 0x74, 0xf9, 0x9c, 0x7b, 0x07, 0xbf, 0xd7, 0xe1, 0xc3, 0x8a, 0x9b, 0xc2, 0x4f, 0xe1, - 0x54, 0x46, 0x62, 0x27, 0x71, 0x52, 0x9a, 0x94, 0xda, 0xed, 0x26, 0x74, 0x39, 0xcd, 0x11, 0xfc, - 0x08, 0xb6, 0x4f, 0x7d, 0x12, 0x78, 0x85, 0xe0, 0x62, 0xf7, 0x5b, 0x02, 0xdc, 0x84, 0x8f, 0x40, - 0x23, 0x91, 0x1b, 0x50, 0x46, 0x44, 0xfb, 0x5b, 0x7b, 0xd6, 0xdd, 0x2e, 0xaf, 0x75, 0x98, 0xb1, - 0xf2, 0xa1, 0xca, 0x24, 0xb8, 0x0f, 0x6d, 0x69, 0x26, 0x62, 0x5a, 0xf9, 0xb9, 0x6b, 0x2f, 0xbe, - 0x0f, 0x70, 0xe6, 0x30, 0x9b, 0x30, 0xd7, 0x89, 0x49, 0x69, 0x5c, 0xfa, 0x99, 0xc3, 0x0e, 0x85, - 0x1b, 0x7f, 0x0c, 0x2d, 0x19, 0xd0, 0x2a, 0x24, 0x91, 0xbe, 0xb5, 0x1c, 0x5a, 0x95, 0x1c, 0x65, - 0x4d, 0xdb, 0xff, 0xab, 0x69, 0x49, 0x0e, 0xa8, 0x94, 0xc3, 0x02, 0x4d, 0xb6, 0x89, 0x75, 0x68, - 0x8e, 0xc8, 0x82, 0x24, 0x48, 0xe1, 0xfb, 0xfc, 0x38, 0x58, 0x3a, 0x2b, 0x86, 0x54, 0xdc, 0x85, - 0x76, 0x36, 0x10, 0x27, 0x40, 0xb5, 0x67, 0x8d, 0xb6, 0x8e, 0x60, 0xc0, 0xa0, 0x57, 0x7a, 0xa4, - 0xf0, 0x00, 0x74, 0x8f, 0x88, 0x73, 0x48, 0x59, 0xb9, 0x8d, 0x9b, 0xb7, 0xc4, 0xab, 0x13, 0x5a, - 0xe9, 0x79, 0x4b, 0xdc, 0x83, 0x3f, 0x07, 0x08, 0x9d, 0x57, 0x13, 0xba, 0x9c, 0xfa, 0xaf, 0x33, - 0x91, 0x72, 0x7a, 0xc1, 0x3f, 0xf8, 0x89, 0x1f, 0x5a, 0x78, 0x83, 0x6e, 0xd0, 0xd4, 0x6a, 0x5a, - 0x79, 0x08, 0xb5, 0xca, 0x21, 0xec, 0x03, 0xba, 0xf9, 0x68, 0x94, 0x69, 0x6a, 0x25, 0xed, 0x8f, - 0x1a, 0x74, 0x0a, 0x8f, 0x15, 0x1e, 0xde, 0xf8, 0xa8, 0x1e, 0xbc, 0xfb, 0x71, 0xb3, 0x2a, 0xff, - 0xa8, 0x07, 0xd0, 0x91, 0xb7, 0x2d, 0xa4, 0x1e, 0x11, 0x05, 0xe7, 0xcb, 0x03, 0x19, 0x70, 0x42, - 0x3d, 0xc2, 0x7b, 0x67, 0xee, 0x19, 0x09, 0x9d, 0x67, 0xd3, 0xf1, 0xa8, 0x70, 0xad, 0x79, 0xd4, - 0xda, 0x8f, 0xbf, 0x86, 0xed, 0xd0, 0x79, 0x65, 0x27, 0xc4, 0xa5, 0x89, 0x67, 0x33, 0x3e, 0xa6, - 0xe2, 0xc6, 0xf6, 0xf8, 0x98, 0x04, 0x26, 0x26, 0xb5, 0x0b, 0x28, 0x8f, 0x5c, 0x5f, 0xac, 0x66, - 0x21, 0x7c, 0x3b, 0x43, 0x37, 0x37, 0xeb, 0x0e, 0xd7, 0xfd, 0x1b, 0x68, 0xc9, 0x67, 0x56, 0x83, - 0xfa, 0x78, 0x78, 0x84, 0x14, 0xbc, 0x0d, 0x9d, 0x83, 0xe3, 0x91, 0x3d, 0x39, 0x1c, 0x8e, 0x27, - 0x4f, 0xa6, 0xd9, 0x53, 0xcb, 0xab, 0x5d, 0x7b, 0x6a, 0x07, 0x0f, 0x2f, 0xff, 0x31, 0x95, 0xcb, - 0x2b, 0x53, 0x7d, 0x73, 0x65, 0xaa, 0x6f, 0xaf, 0x4c, 0xf5, 0xef, 0x2b, 0x53, 0xfd, 0xed, 0xda, - 0x54, 0xde, 0x5c, 0x9b, 0xca, 0xdb, 0x6b, 0x53, 0xf9, 0x59, 0x93, 0x03, 0xfd, 0x2f, 0x00, 0x00, - 0xff, 0xff, 0xf9, 0x03, 0xbb, 0x68, 0x72, 0x08, 0x00, 0x00, + proto.RegisterFile("roachpb/io-formats.proto", fileDescriptor_io_formats_802d999645186915) +} + +var fileDescriptor_io_formats_802d999645186915 = []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, } diff --git a/pkg/roachpb/io-formats.proto b/pkg/roachpb/io-formats.proto index b61cf85c9123..d47d4994844b 100644 --- a/pkg/roachpb/io-formats.proto +++ b/pkg/roachpb/io-formats.proto @@ -110,6 +110,9 @@ message PgDumpOptions { // Indicates the number of rows to import per table. // Must be a non-zero positive number. optional int64 row_limit = 2 [(gogoproto.nullable) = false]; + // Indicates if all unparseable and parseable, but unimplemented PGDUMP stmts + // should be ignored during IMPORT. + optional bool ignore_unsupported = 3 [(gogoproto.nullable) = false]; } message MysqldumpOptions { diff --git a/pkg/sql/parser/lexer.go b/pkg/sql/parser/lexer.go index 62f55940d1f2..142b5cba03e3 100644 --- a/pkg/sql/parser/lexer.go +++ b/pkg/sql/parser/lexer.go @@ -153,36 +153,52 @@ func (l *lexer) UpdateNumPlaceholders(p *tree.Placeholder) { } } -// Unimplemented wraps Error, setting lastUnimplementedError. -func (l *lexer) Unimplemented(feature string) { - l.lastError = unimp.New(feature, "this syntax") +// PurposelyUnimplemented wraps Error, setting lastUnimplementedError. +func (l *lexer) PurposelyUnimplemented(feature string, reason string) { + // We purposely do not use unimp here, as it appends hints to suggest that + // the error may be actively tracked as a bug. + l.lastError = errors.WithHint( + errors.WithTelemetry( + pgerror.Newf(pgcode.Syntax, "unimplemented: this syntax"), + fmt.Sprintf("sql.purposely_unimplemented.%s", feature), + ), + reason, + ) l.populateErrorDetails() + l.lastError = &tree.Unsupported{ + Err: l.lastError, + FeatureName: feature, + } } // UnimplementedWithIssue wraps Error, setting lastUnimplementedError. func (l *lexer) UnimplementedWithIssue(issue int) { l.lastError = unimp.NewWithIssue(issue, "this syntax") l.populateErrorDetails() + l.lastError = &tree.Unsupported{ + Err: l.lastError, + FeatureName: fmt.Sprintf("https://github.com/cockroachdb/cockroach/issues/%d", issue), + } } // UnimplementedWithIssueDetail wraps Error, setting lastUnimplementedError. func (l *lexer) UnimplementedWithIssueDetail(issue int, detail string) { l.lastError = unimp.NewWithIssueDetail(issue, detail, "this syntax") l.populateErrorDetails() + l.lastError = &tree.Unsupported{ + Err: l.lastError, + FeatureName: detail, + } } -// PurposelyUnimplemented wraps Error, setting lastUnimplementedError. -func (l *lexer) PurposelyUnimplemented(feature string, reason string) { - // We purposely do not use unimp here, as it appends hints to suggest that - // the error may be actively tracked as a bug. - l.lastError = errors.WithHint( - errors.WithTelemetry( - pgerror.Newf(pgcode.Syntax, "unimplemented: this syntax"), - fmt.Sprintf("sql.purposely_unimplemented.%s", feature), - ), - reason, - ) +// Unimplemented wraps Error, setting lastUnimplementedError. +func (l *lexer) Unimplemented(feature string) { + l.lastError = unimp.New(feature, "this syntax") l.populateErrorDetails() + l.lastError = &tree.Unsupported{ + Err: l.lastError, + FeatureName: feature, + } } // setErr is called from parsing action rules to register an error observed diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index f363c8803755..8629605c73ae 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -70,6 +70,8 @@ func unimplementedWithIssueDetail(sqllex sqlLexer, issue int, detail string) int sqllex.(*lexer).UnimplementedWithIssueDetail(issue, detail) return 1 } + + %} %{ @@ -726,6 +728,7 @@ func (u *sqlSymUnion) objectNamePrefixList() tree.ObjectNamePrefixList { %type stmt_block %type stmt + %type alter_stmt %type alter_ddl_stmt %type alter_table_stmt @@ -738,6 +741,7 @@ func (u *sqlSymUnion) objectNamePrefixList() tree.ObjectNamePrefixList { %type alter_role_stmt %type alter_type_stmt %type alter_schema_stmt +%type alter_unsupported_stmt // ALTER RANGE %type alter_zone_range_stmt @@ -1328,6 +1332,7 @@ stmt: alter_stmt: alter_ddl_stmt // help texts in sub-rule | alter_role_stmt // EXTEND WITH HELP: ALTER ROLE +| alter_unsupported_stmt | ALTER error // SHOW HELP: ALTER alter_ddl_stmt: @@ -2710,6 +2715,13 @@ import_format: $$ = strings.ToUpper($1) } +alter_unsupported_stmt: + ALTER FUNCTION error + { + return unimplemented(sqllex, "ALTER FUNCTION") + } + + // %Help: IMPORT - load data from file in a distributed manner // %Category: CCL // %Text: @@ -3065,6 +3077,7 @@ comment_stmt: { $$.val = &tree.CommentOnIndex{Index: $4.tableIndexName(), Comment: $6.strPtr()} } +| COMMENT ON EXTENSION error { return unimplemented(sqllex, "comment on extension") } comment_text: SCONST @@ -3104,6 +3117,7 @@ create_extension_stmt: | CREATE EXTENSION name { $$.val = &tree.CreateExtension{Name: $3} } +| CREATE EXTENSION IF NOT EXISTS name WITH error { return unimplemented(sqllex, "create extension if not exists with") } | CREATE EXTENSION error // SHOW HELP: CREATE EXTENSION create_unsupported: @@ -3125,7 +3139,7 @@ create_unsupported: | CREATE SUBSCRIPTION error { return unimplemented(sqllex, "create subscription") } | CREATE TABLESPACE error { return unimplementedWithIssueDetail(sqllex, 54113, "create tablespace") } | CREATE TEXT error { return unimplementedWithIssueDetail(sqllex, 7821, "create text") } -| CREATE TRIGGER error { return unimplementedWithIssueDetail(sqllex, 28296, "create") } +| CREATE TRIGGER error { return unimplementedWithIssueDetail(sqllex, 28296, "create trigger") } opt_or_replace: OR REPLACE {} @@ -3863,6 +3877,10 @@ grant_stmt: Grantees: $7.nameList(), } } +| GRANT privileges ON SEQUENCE error + { + return unimplemented(sqllex, "GRANT ON SEQUENCE") + } | GRANT error // SHOW HELP: GRANT // %Help: REVOKE - remove access privileges and role memberships @@ -3910,6 +3928,10 @@ revoke_stmt: Grantees: $7.nameList(), } } +| REVOKE privileges ON SEQUENCE error + { + return unimplemented(sqllex, "REVOKE ON SEQUENCE") + } | REVOKE error // SHOW HELP: REVOKE // ALL can either be by itself, or with the optional PRIVILEGES keyword (which no-ops) diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 0900fe3bf1bb..920c7a73d8a1 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -87,6 +87,7 @@ go_library( "type_check.go", "type_name.go", "union.go", + "unsupported.go", "update.go", "values.go", "var_name.go", diff --git a/pkg/sql/sem/tree/unsupported.go b/pkg/sql/sem/tree/unsupported.go new file mode 100644 index 000000000000..2316892e9dae --- /dev/null +++ b/pkg/sql/sem/tree/unsupported.go @@ -0,0 +1,31 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tree + +var _ error = &Unsupported{} + +// Unsupported is an error object which is returned by some unimplemented SQL +// statements. It is currently only used to skip over PGDUMP statements during +// an import. +type Unsupported struct { + Err error + FeatureName string +} + +func (u *Unsupported) Error() string { + return u.Err.Error() +} + +// Cause implements causer. +func (u *Unsupported) Cause() error { return u.Err } + +// Unwrap implements wrapper. +func (u *Unsupported) Unwrap() error { return u.Err }