diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go index 5545eadf97d1..ccde27f3e070 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -2605,6 +2605,13 @@ func TestImportIntoCSV(t *testing.T) { fmt.Sprintf(`non-constant default expression .* for non-targeted column "b" is not supported by IMPORT INTO`), fmt.Sprintf(`IMPORT INTO t (a) CSV DATA ("%s")`, srv.URL)) }) + t.Run("pgdump", func(t *testing.T) { + data = "INSERT INTO t VALUES (1, 2), (3, 4)" + sqlDB.Exec(t, `CREATE TABLE t (a INT, b INT DEFAULT 42, c INT)`) + sqlDB.Exec(t, "IMPORT INTO t (c, a) PGDUMP DATA ($1)", srv.URL) + defer sqlDB.Exec(t, `DROP TABLE t`) + sqlDB.CheckQueryResults(t, `SELECT * from t`, [][]string{{"2", "42", "1"}, {"4", "42", "3"}}) + }) }) t.Run("import-not-targeted-not-null", func(t *testing.T) { @@ -4029,6 +4036,21 @@ func TestImportPgDump(t *testing.T) { } }) } + t.Run("target-cols-reordered", func(t *testing.T) { + data := ` + CREATE TABLE "t" ("a" INT, "b" INT DEFAULT 42, "c" INT); + INSERT INTO "t" ("c", "a") VALUES ('1', '2'), ('3', '4'); + ` + srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if r.Method == "GET" { + _, _ = w.Write([]byte(data)) + } + })) + defer srv.Close() + defer sqlDB.Exec(t, "DROP TABLE t") + sqlDB.Exec(t, "IMPORT PGDUMP ($1)", srv.URL) + sqlDB.CheckQueryResults(t, `SELECT * from t`, [][]string{{"2", "42", "1"}, {"4", "42", "3"}}) + }) } // 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 de11a19e38f6..12fe406cd597 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -483,6 +483,7 @@ type pgDumpReader struct { descs map[string]*execinfrapb.ReadImportDataSpec_ImportTable kvCh chan row.KVBatch opts roachpb.PgDumpOptions + colMap map[*row.DatumRowConverter](map[string]int) } var _ inputConverter = &pgDumpReader{} @@ -496,13 +497,23 @@ func newPgDumpReader( evalCtx *tree.EvalContext, ) (*pgDumpReader, error) { converters := make(map[string]*row.DatumRowConverter, len(descs)) + colMap := make(map[*row.DatumRowConverter](map[string]int)) for name, table := range descs { if table.Desc.IsTable() { - conv, err := row.NewDatumRowConverter(ctx, table.Desc, nil /* targetColNames */, evalCtx, kvCh) + colSubMap := make(map[string]int, len(table.TargetCols)) + targetCols := make(tree.NameList, len(table.TargetCols)) + for i, colName := range table.TargetCols { + targetCols[i] = tree.Name(colName) + } + for i, col := range table.Desc.VisibleColumns() { + colSubMap[col.Name] = i + } + conv, err := row.NewDatumRowConverter(ctx, table.Desc, targetCols, evalCtx, kvCh) if err != nil { return nil, err } converters[name] = conv + colMap[conv] = colSubMap } } return &pgDumpReader{ @@ -510,6 +521,7 @@ func newPgDumpReader( tables: converters, descs: descs, opts: opts, + colMap: colMap, }, nil } @@ -567,22 +579,46 @@ func (m *pgDumpReader) readFile( if ok && conv == nil { return errors.Errorf("missing schema info for requested table %q", name) } + expectedColLen := len(i.Columns) + if expectedColLen == 0 { + // Case where the targeted columns are not specified in the PGDUMP file, but in + // the command "IMPORT INTO table (targetCols) PGDUMP DATA (filename)" + expectedColLen = len(conv.VisibleCols) + } values, ok := i.Rows.Select.(*tree.ValuesClause) if !ok { return errors.Errorf("unsupported: %s", i.Rows.Select) } inserts++ startingCount := count + var targetColMapInd []int + if len(i.Columns) != 0 { + targetColMapInd = make([]int, len(i.Columns)) + conv.IsTargetCol = make(map[int]struct{}, len(i.Columns)) + for j := range i.Columns { + colName := i.Columns[j].String() + ind, ok := m.colMap[conv][colName] + if !ok { + return errors.Newf("targeted column %q not found", colName) + } + conv.IsTargetCol[ind] = struct{}{} + targetColMapInd[j] = ind + } + } for _, tuple := range values.Rows { count++ if count <= resumePos { continue } - if expected, got := len(conv.VisibleCols), len(tuple); expected != got { - return errors.Errorf("expected %d values, got %d: %v", expected, got, tuple) + if got := len(tuple); expectedColLen != got { + return errors.Errorf("expected %d values, got %d: %v", expectedColLen, got, tuple) } - for i, expr := range tuple { - typed, err := expr.TypeCheck(ctx, &semaCtx, conv.VisibleColTypes[i]) + for j, expr := range tuple { + ind := j + if len(i.Columns) != 0 { + ind = targetColMapInd[j] + } + typed, err := expr.TypeCheck(ctx, &semaCtx, conv.VisibleColTypes[ind]) if err != nil { return errors.Wrapf(err, "reading row %d (%d in insert statement %d)", count, count-startingCount, inserts) @@ -592,7 +628,7 @@ func (m *pgDumpReader) readFile( return errors.Wrapf(err, "reading row %d (%d in insert statement %d)", count, count-startingCount, inserts) } - conv.Datums[i] = converted + conv.Datums[ind] = converted } if err := conv.Row(ctx, inputIdx, count); err != nil { return err diff --git a/pkg/cli/cli.go b/pkg/cli/cli.go index 88d85bbe1e4c..e5213b1efc10 100644 --- a/pkg/cli/cli.go +++ b/pkg/cli/cli.go @@ -195,6 +195,7 @@ func init() { quitCmd, sqlShellCmd, + stmtDiagCmd, authCmd, nodeCmd, dumpCmd, diff --git a/pkg/cli/cli_test.go b/pkg/cli/cli_test.go index 96a00da1e4c5..deda6da3f561 100644 --- a/pkg/cli/cli_test.go +++ b/pkg/cli/cli_test.go @@ -297,7 +297,7 @@ func isSQLCommand(args []string) bool { return false } switch args[0] { - case "sql", "dump", "workload", "nodelocal": + case "sql", "dump", "workload", "nodelocal", "statement-diag": return true case "node": if len(args) == 0 { @@ -1405,6 +1405,7 @@ Available Commands: init initialize a cluster cert create ca, node, and client certs sql open a sql shell + statement-diag commands for managing statement diagnostics bundles auth-session log in and out of HTTP sessions node list, inspect, drain or remove nodes dump dump sql tables diff --git a/pkg/cli/cliflags/flags.go b/pkg/cli/cliflags/flags.go index 6a9be35611ba..409e43e07802 100644 --- a/pkg/cli/cliflags/flags.go +++ b/pkg/cli/cliflags/flags.go @@ -1201,4 +1201,14 @@ other items retrieved by the zip command may still consider confidential data or PII. `, } + + StmtDiagDeleteAll = FlagInfo{ + Name: "all", + Description: `Delete all bundles.`, + } + + StmtDiagCancelAll = FlagInfo{ + Name: "all", + Description: `Cancel all outstanding requests.`, + } ) diff --git a/pkg/cli/context.go b/pkg/cli/context.go index a6560f3e464a..420d351705e8 100644 --- a/pkg/cli/context.go +++ b/pkg/cli/context.go @@ -49,6 +49,7 @@ func initCLIDefaults() { setNetworkBenchContextDefaults() setSqlfmtContextDefaults() setDemoContextDefaults() + setStmtDiagContextDefaults() setAuthContextDefaults() initPreFlagsDefaults() @@ -506,6 +507,16 @@ func setDemoContextDefaults() { demoCtx.insecure = false } +// stmtDiagCtx captures the command-line parameters of the 'statement-diag' +// command. +var stmtDiagCtx struct { + all bool +} + +func setStmtDiagContextDefaults() { + stmtDiagCtx.all = false +} + // GetServerCfgStores provides direct public access to the StoreSpecList inside // serverCfg. This is used by CCL code to populate some fields. // diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go index 2bf9868649b6..ee51b6661f2e 100644 --- a/pkg/cli/flags.go +++ b/pkg/cli/flags.go @@ -539,6 +539,7 @@ func init() { clientCmds = append(clientCmds, nodeCmds...) clientCmds = append(clientCmds, systemBenchCmds...) clientCmds = append(clientCmds, nodeLocalCmds...) + clientCmds = append(clientCmds, stmtDiagCmds...) for _, cmd := range clientCmds { f := cmd.PersistentFlags() varFlag(f, addrSetter{&cliCtx.clientConnHost, &cliCtx.clientConnPort}, cliflags.ClientHost) @@ -649,6 +650,7 @@ func init() { sqlCmds := []*cobra.Command{sqlShellCmd, dumpCmd, demoCmd} sqlCmds = append(sqlCmds, authCmds...) sqlCmds = append(sqlCmds, demoCmd.Commands()...) + sqlCmds = append(sqlCmds, stmtDiagCmds...) sqlCmds = append(sqlCmds, nodeLocalCmds...) for _, cmd := range sqlCmds { f := cmd.Flags() @@ -736,6 +738,12 @@ func init() { stringFlag(f, &startCtx.geoLibsDir, cliflags.GeoLibsDir) } + // statement-diag command. + { + boolFlag(stmtDiagDeleteCmd.Flags(), &stmtDiagCtx.all, cliflags.StmtDiagDeleteAll) + boolFlag(stmtDiagCancelCmd.Flags(), &stmtDiagCtx.all, cliflags.StmtDiagCancelAll) + } + // sqlfmt command. { f := sqlfmtCmd.Flags() diff --git a/pkg/cli/main_test.go b/pkg/cli/main_test.go index e860d1f0d64f..a233ca2e699b 100644 --- a/pkg/cli/main_test.go +++ b/pkg/cli/main_test.go @@ -13,6 +13,7 @@ package cli_test import ( "os" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/security" @@ -30,6 +31,10 @@ func TestMain(m *testing.M) { // a version injected. Pretend to be a very up-to-date version. defer build.TestingOverrideTag("v999.0.0")() + // The times for Example_statement_diag are reported in the local timezone. + // Fix it to UTC so the output is always the same. + time.Local = time.UTC + serverutils.InitTestServerFactory(server.TestServerFactory) os.Exit(m.Run()) } diff --git a/pkg/cli/statement_diag.go b/pkg/cli/statement_diag.go new file mode 100644 index 000000000000..518d0eecbdaa --- /dev/null +++ b/pkg/cli/statement_diag.go @@ -0,0 +1,358 @@ +// 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 cli + +import ( + "bytes" + "database/sql/driver" + "fmt" + "io" + "os" + "strconv" + "text/tabwriter" + "time" + + "github.com/cockroachdb/errors" + "github.com/spf13/cobra" +) + +var stmtDiagCmd = &cobra.Command{ + Use: "statement-diag [command]", + Short: "commands for managing statement diagnostics bundles", + Long: `This set of commands can be used to manage and download statement diagnostic +bundles, and to cancel outstanding diagnostics activation requests. Statement +diagnostics can be activated from the UI or using EXPLAIN ANALYZE (DEBUG).`, + RunE: usageAndErr, +} + +var stmtDiagListCmd = &cobra.Command{ + Use: "list [options]", + Short: "list available bundles and outstanding activation requests", + Long: `List statements diagnostics that are available for download and outstanding +diagnostics activation requests.`, + Args: cobra.NoArgs, + RunE: MaybeDecorateGRPCError(runStmtDiagList), +} + +func runStmtDiagList(cmd *cobra.Command, args []string) error { + const timeFmt = "2006-01-02 15:04:05 MST" + + conn, err := makeSQLClient("cockroach statement-diag", useSystemDb) + if err != nil { + return err + } + defer conn.Close() + + // -- List bundles -- + + rows, err := conn.Query( + `SELECT id, statement_fingerprint, collected_at + FROM system.statement_diagnostics + WHERE error IS NULL + ORDER BY collected_at DESC`, + nil, /* args */ + ) + if err != nil { + return err + } + vals := make([]driver.Value, 3) + var buf bytes.Buffer + w := tabwriter.NewWriter(&buf, 4, 0, 2, ' ', 0) + fmt.Fprint(w, " ID\tCollection time\tStatement\n") + num := 0 + for { + if err := rows.Next(vals); err == io.EOF { + break + } else if err != nil { + return err + } + id := vals[0].(int64) + stmt := vals[1].(string) + t := vals[2].(time.Time) + fmt.Fprintf(w, " %d\t%s\t%s\n", id, t.Local().Format(timeFmt), stmt) + num++ + } + if err := rows.Close(); err != nil { + return err + } + if num == 0 { + fmt.Printf("No statement diagnostics bundles available.\n") + } else { + fmt.Printf("Statement diagnostics bundles:\n") + _ = w.Flush() + // When we show a list of bundles, we want an extra blank line. + fmt.Println(buf.String()) + } + + // -- List outstanding activation requests -- + + rows, err = conn.Query( + `SELECT id, statement_fingerprint, requested_at + FROM system.statement_diagnostics_requests + WHERE NOT completed + ORDER BY requested_at DESC`, + nil, /* args */ + ) + if err != nil { + return err + } + + buf.Reset() + w = tabwriter.NewWriter(&buf, 4, 0, 2, ' ', 0) + fmt.Fprint(w, " ID\tActivation time\tStatement\n") + num = 0 + for { + if err := rows.Next(vals); err == io.EOF { + break + } else if err != nil { + return err + } + id := vals[0].(int64) + stmt := vals[1].(string) + t := vals[2].(time.Time) + fmt.Fprintf(w, " %d\t%s\t%s\n", id, t.Local().Format(timeFmt), stmt) + num++ + } + if err := rows.Close(); err != nil { + return err + } + if num == 0 { + fmt.Printf("No outstanding activation requests.\n") + } else { + fmt.Printf("Outstanding activation requests:\n") + _ = w.Flush() + fmt.Print(buf.String()) + } + + return nil +} + +var stmtDiagDownloadCmd = &cobra.Command{ + Use: "download [options]", + Short: "download statement diagnostics bundle into a zip file", + Long: `Download statements diagnostics bundle into a zip file, using an ID returned by +the list command.`, + Args: cobra.ExactArgs(2), + RunE: MaybeDecorateGRPCError(runStmtDiagDownload), +} + +func runStmtDiagDownload(cmd *cobra.Command, args []string) error { + id, err := strconv.ParseInt(args[0], 10, 64) + if err != nil || id < 0 { + return errors.New("invalid bundle id") + } + filename := args[1] + + conn, err := makeSQLClient("cockroach statement-diag", useSystemDb) + if err != nil { + return err + } + defer conn.Close() + + // Retrieve the chunk IDs; these are stored in an INT ARRAY column. + rows, err := conn.Query( + "SELECT unnest(bundle_chunks) FROM system.statement_diagnostics WHERE id = $1", + []driver.Value{id}, + ) + if err != nil { + return err + } + var chunkIDs []int64 + vals := make([]driver.Value, 1) + for { + if err := rows.Next(vals); err == io.EOF { + break + } else if err != nil { + return err + } + chunkIDs = append(chunkIDs, vals[0].(int64)) + } + if err := rows.Close(); err != nil { + return err + } + + if len(chunkIDs) == 0 { + return errors.Newf("no statement diagnostics bundle with ID %d", id) + } + + // Create the file and write out the chunks. + out, err := os.Create(filename) + if err != nil { + return err + } + + for _, chunkID := range chunkIDs { + data, err := conn.QueryRow( + "SELECT data FROM system.statement_bundle_chunks WHERE id = $1", + []driver.Value{chunkID}, + ) + if err != nil { + _ = out.Close() + return err + } + if _, err := out.Write(data[0].([]byte)); err != nil { + _ = out.Close() + return err + } + } + + return out.Close() +} + +var stmtDiagDeleteCmd = &cobra.Command{ + Use: "delete { --all | }", + Short: "delete statement diagnostics bundles", + Long: `Delete a statement diagnostics bundle using an ID returned by the list +command, or delete all bundles.`, + Args: cobra.MaximumNArgs(1), + RunE: MaybeDecorateGRPCError(runStmtDiagDelete), +} + +func runStmtDiagDelete(cmd *cobra.Command, args []string) error { + conn, err := makeSQLClient("cockroach statement-diag", useSystemDb) + if err != nil { + return err + } + defer conn.Close() + + if stmtDiagCtx.all { + if len(args) > 0 { + return errors.New("extra arguments with --all") + } + return runStmtDiagDeleteAll(conn) + } + if len(args) != 1 { + return fmt.Errorf("accepts 1 arg, received %d", len(args)) + } + + id, err := strconv.ParseInt(args[0], 10, 64) + if err != nil || id < 0 { + return errors.New("invalid id") + } + + _, err = conn.QueryRow( + "SELECT 1 FROM system.statement_diagnostics WHERE id = $1", + []driver.Value{id}, + ) + if err != nil { + if err == io.EOF { + return errors.Newf("no statement diagnostics bundle with ID %d", id) + } + return err + } + + return conn.ExecTxn(func(conn *sqlConn) error { + // Delete the request metadata. + if err := conn.Exec( + "DELETE FROM system.statement_diagnostics_requests WHERE statement_diagnostics_id = $1", + []driver.Value{id}, + ); err != nil { + return err + } + // Delete the bundle chunks. + if err := conn.Exec( + `DELETE FROM system.statement_bundle_chunks + WHERE id IN ( + SELECT unnest(bundle_chunks) FROM system.statement_diagnostics WHERE id = $1 + )`, + []driver.Value{id}, + ); err != nil { + return err + } + // Finally, delete the diagnostics entry. + return conn.Exec( + "DELETE FROM system.statement_diagnostics WHERE id = $1", + []driver.Value{id}, + ) + }) +} + +func runStmtDiagDeleteAll(conn *sqlConn) error { + return conn.ExecTxn(func(conn *sqlConn) error { + // Delete the request metadata. + if err := conn.Exec( + "DELETE FROM system.statement_diagnostics_requests WHERE completed", + nil, + ); err != nil { + return err + } + // Delete all bundle chunks. + if err := conn.Exec( + `DELETE FROM system.statement_bundle_chunks WHERE true`, + nil, + ); err != nil { + return err + } + // Finally, delete the diagnostics entry. + return conn.Exec( + "DELETE FROM system.statement_diagnostics WHERE true", + nil, + ) + }) +} + +var stmtDiagCancelCmd = &cobra.Command{ + Use: "cancel { -all | }", + Short: "cancel outstanding activation requests", + Long: `Cancel an outstanding activation request, using an ID returned by the +list command, or cancel all outstanding requests.`, + Args: cobra.MaximumNArgs(1), + RunE: MaybeDecorateGRPCError(runStmtDiagCancel), +} + +func runStmtDiagCancel(cmd *cobra.Command, args []string) error { + conn, err := makeSQLClient("cockroach statement-diag", useSystemDb) + if err != nil { + return err + } + defer conn.Close() + + if stmtDiagCtx.all { + if len(args) > 0 { + return errors.New("extra arguments with --all") + } + return conn.Exec( + "DELETE FROM system.statement_diagnostics_requests WHERE NOT completed", + nil, + ) + } + if len(args) != 1 { + return fmt.Errorf("accepts 1 arg, received %d", len(args)) + } + + id, err := strconv.ParseInt(args[0], 10, 64) + if err != nil || id < 0 { + return errors.New("invalid id") + } + + _, err = conn.QueryRow( + "DELETE FROM system.statement_diagnostics_requests WHERE id = $1 RETURNING id", + []driver.Value{id}, + ) + if err != nil { + if err == io.EOF { + return errors.Newf("no outstanding activation requests with ID %d", id) + } + return err + } + return nil +} + +var stmtDiagCmds = []*cobra.Command{ + stmtDiagListCmd, + stmtDiagDownloadCmd, + stmtDiagDeleteCmd, + stmtDiagCancelCmd, +} + +func init() { + stmtDiagCmd.AddCommand(stmtDiagCmds...) +} diff --git a/pkg/cli/statement_diag_test.go b/pkg/cli/statement_diag_test.go new file mode 100644 index 000000000000..ce65bc8ced25 --- /dev/null +++ b/pkg/cli/statement_diag_test.go @@ -0,0 +1,152 @@ +// 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 cli + +import ( + "context" + "fmt" + "io/ioutil" + "os" + + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func Example_statement_diag() { + c := newCLITest(cliTestParams{}) + defer c.cleanup() + + // First, set up some diagnostics state. + commands := []string{ + `INSERT INTO system.statement_bundle_chunks(id, data) VALUES (1001, 'chunk1'), (1002, 'chunk2'), (1003, 'chunk3')`, + + `INSERT INTO system.statement_diagnostics(id, statement_fingerprint, statement, collected_at, bundle_chunks) + VALUES (10, 'SELECT _ FROM _', 'SELECT a FROM t', '2010-01-02 03:04:05', ARRAY[1001]), + (20, 'SELECT _ FROM _ WHERE _ > _', 'SELECT a FROM t WHERE b > 1', '2010-01-02 03:04:06', ARRAY[1001,1002,1003]), + (30, 'SELECT _ FROM _ WHERE _ > _', 'SELECT a FROM t WHERE b > 1', '2010-01-02 03:04:07', ARRAY[1001])`, + + `INSERT INTO system.statement_diagnostics_requests(id, completed, statement_fingerprint, statement_diagnostics_id, requested_at) + VALUES (1, TRUE, 'SELECT _ FROM _', 10, '2010-01-02 03:04:00'), + (2, TRUE, 'SELECT _ FROM _ WHERE _ > _', 20, '2010-01-02 03:04:02'), + (3, TRUE, 'SELECT _ FROM _ WHERE _ > _', 30, '2010-01-02 03:04:05'), + (4, FALSE, 'SELECT _ + _', NULL, '2010-01-02 03:04:10'), + (5, FALSE, 'SELECT _ - _', NULL, '2010-01-02 03:04:11'), + (6, FALSE, 'SELECT _ / _', NULL, '2010-01-02 03:04:12')`, + } + + for _, cmd := range commands { + _, err := c.RunWithCaptureArgs([]string{"sql", "-e", cmd}) + if err != nil { + log.Fatalf(context.Background(), "Couldn't execute sql: %s", err) + } + } + c.RunWithArgs([]string{"statement-diag", "list"}) + c.RunWithArgs([]string{"statement-diag", "download", "13", "foo.zip"}) + tmpfile, err := ioutil.TempFile("", "bundle-*.zip") + if err != nil { + log.Fatalf(context.Background(), "Couldn't execute sql: %s", err) + } + bundleFile := tmpfile.Name() + _ = tmpfile.Close() + defer func() { _ = os.Remove(bundleFile) }() + + fmt.Printf("statement-diag download 20 tempfile.zip\n") + _, err = c.RunWithCaptureArgs([]string{"statement-diag", "download", "20", bundleFile}) + if err != nil { + log.Fatalf(context.Background(), "Error downloading bundle: %s", err) + } + data, err := ioutil.ReadFile(bundleFile) + if err != nil { + log.Fatalf(context.Background(), "Error reading bundle: %s", err) + } + fmt.Printf("bundle data: %s\n", data) + + c.RunWithArgs([]string{"statement-diag", "download", "xx"}) + c.RunWithArgs([]string{"statement-diag", "delete", "--all", "20"}) + c.RunWithArgs([]string{"statement-diag", "delete", "20", "30"}) + c.RunWithArgs([]string{"statement-diag", "delete", "xx"}) + c.RunWithArgs([]string{"statement-diag", "delete", "13"}) + c.RunWithArgs([]string{"statement-diag", "delete", "10"}) + c.RunWithArgs([]string{"statement-diag", "list"}) + c.RunWithArgs([]string{"statement-diag", "delete", "--all"}) + c.RunWithArgs([]string{"statement-diag", "list"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "xx"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "5", "6"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "--all", "5"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "4"}) + c.RunWithArgs([]string{"statement-diag", "list"}) + c.RunWithArgs([]string{"statement-diag", "cancel", "--all"}) + c.RunWithArgs([]string{"statement-diag", "list"}) + + // Output: + // statement-diag list + // Statement diagnostics bundles: + // ID Collection time Statement + // 30 2010-01-02 03:04:07 UTC SELECT _ FROM _ WHERE _ > _ + // 20 2010-01-02 03:04:06 UTC SELECT _ FROM _ WHERE _ > _ + // 10 2010-01-02 03:04:05 UTC SELECT _ FROM _ + // + // Outstanding activation requests: + // ID Activation time Statement + // 6 2010-01-02 03:04:12 UTC SELECT _ / _ + // 5 2010-01-02 03:04:11 UTC SELECT _ - _ + // 4 2010-01-02 03:04:10 UTC SELECT _ + _ + // statement-diag download 13 foo.zip + // ERROR: no statement diagnostics bundle with ID 13 + // statement-diag download 20 tempfile.zip + // bundle data: chunk1chunk2chunk3 + // statement-diag download xx + // ERROR: accepts 2 arg(s), received 1 + // statement-diag delete --all 20 + // ERROR: extra arguments with --all + // statement-diag delete 20 30 + // ERROR: accepts at most 1 arg(s), received 2 + // statement-diag delete xx + // ERROR: invalid id + // statement-diag delete 13 + // ERROR: no statement diagnostics bundle with ID 13 + // statement-diag delete 10 + // statement-diag list + // Statement diagnostics bundles: + // ID Collection time Statement + // 30 2010-01-02 03:04:07 UTC SELECT _ FROM _ WHERE _ > _ + // 20 2010-01-02 03:04:06 UTC SELECT _ FROM _ WHERE _ > _ + // + // Outstanding activation requests: + // ID Activation time Statement + // 6 2010-01-02 03:04:12 UTC SELECT _ / _ + // 5 2010-01-02 03:04:11 UTC SELECT _ - _ + // 4 2010-01-02 03:04:10 UTC SELECT _ + _ + // statement-diag delete --all + // statement-diag list + // No statement diagnostics bundles available. + // Outstanding activation requests: + // ID Activation time Statement + // 6 2010-01-02 03:04:12 UTC SELECT _ / _ + // 5 2010-01-02 03:04:11 UTC SELECT _ - _ + // 4 2010-01-02 03:04:10 UTC SELECT _ + _ + // statement-diag cancel xx + // ERROR: invalid id + // statement-diag cancel 5 6 + // ERROR: accepts at most 1 arg(s), received 2 + // statement-diag cancel --all 5 + // ERROR: extra arguments with --all + // statement-diag cancel 4 + // statement-diag list + // No statement diagnostics bundles available. + // Outstanding activation requests: + // ID Activation time Statement + // 6 2010-01-02 03:04:12 UTC SELECT _ / _ + // 5 2010-01-02 03:04:11 UTC SELECT _ - _ + // statement-diag cancel --all + // statement-diag list + // No statement diagnostics bundles available. + // No outstanding activation requests. +} diff --git a/pkg/jobs/job_scheduler.go b/pkg/jobs/job_scheduler.go index c0b3a8ce755d..273f25a53d8f 100644 --- a/pkg/jobs/job_scheduler.go +++ b/pkg/jobs/job_scheduler.go @@ -101,13 +101,15 @@ SELECT FROM %s J WHERE J.created_by_type = '%s' AND J.created_by_id = S.schedule_id AND - J.status NOT IN ('failed', 'succeeded', 'cancelled') + J.status NOT IN ('%s', '%s', '%s') ) AS num_running, S.* FROM %s S WHERE next_run < %s ORDER BY next_run %s -`, env.SystemJobsTableName(), createdByName, env.ScheduledJobsTableName(), env.NowExpr(), limitClause) +`, env.SystemJobsTableName(), createdByName, + StatusSucceeded, StatusCanceled, StatusFailed, + env.ScheduledJobsTableName(), env.NowExpr(), limitClause) } // unmarshalScheduledJob is a helper to deserialize a row returned by diff --git a/pkg/jobs/job_scheduler_test.go b/pkg/jobs/job_scheduler_test.go index 55f91db0a63f..7e56a56bf843 100644 --- a/pkg/jobs/job_scheduler_test.go +++ b/pkg/jobs/job_scheduler_test.go @@ -106,6 +106,56 @@ func TestJobSchedulerReschedulesRunning(t *testing.T) { } } +func TestJobSchedulerExecutesAfterTerminal(t *testing.T) { + defer leaktest.AfterTest(t)() + h, cleanup := newTestHelper(t) + defer cleanup() + + ctx := context.Background() + + // If all of the previous runs are in a terminal state, the waiting policy + // should not matter, so ensure that the behavior is the same for them all. + for _, wait := range []jobspb.ScheduleDetails_WaitBehavior{ + jobspb.ScheduleDetails_WAIT, + jobspb.ScheduleDetails_SKIP, + } { + t.Run(wait.String(), func(t *testing.T) { + // Create job that waits for the previous runs to finish. + j := h.newScheduledJob(t, "j", "SELECT 42 AS meaning_of_life;") + j.SetScheduleDetails(jobspb.ScheduleDetails{Wait: wait}) + require.NoError(t, j.SetSchedule("@hourly")) + + require.NoError(t, + h.kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + require.NoError(t, j.Create(ctx, h.ex, txn)) + + // Let's add few fake runs for this schedule which are in every + // terminal state. + for _, status := range []Status{StatusFailed, StatusCanceled, StatusSucceeded} { + addFakeJob(t, h, j.ScheduleID(), status, txn) + } + return nil + })) + + // Verify the job has expected nextRun time. + expectedRunTime := cronexpr.MustParse("@hourly").Next(h.env.Now()) + loaded := h.loadJob(t, j.ScheduleID()) + require.Equal(t, expectedRunTime, loaded.NextRun()) + + // Advance time past the expected start time. + h.env.SetTime(expectedRunTime.Add(time.Second)) + + // Execute the job and verify it has the next run scheduled. + s := newJobScheduler(h.env, h.ex) + require.NoError(t, s.executeSchedules(ctx, allSchedules, nil)) + + expectedRunTime = cronexpr.MustParse("@hourly").Next(h.env.Now()) + loaded = h.loadJob(t, j.ScheduleID()) + require.Equal(t, expectedRunTime, loaded.NextRun()) + }) + } +} + func TestJobSchedulerExecutesAndSchedulesNextRun(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/kv/kvclient/kvcoord/txn_metrics.go b/pkg/kv/kvclient/kvcoord/txn_metrics.go index 979bc6b905bb..e99703344d19 100644 --- a/pkg/kv/kvclient/kvcoord/txn_metrics.go +++ b/pkg/kv/kvclient/kvcoord/txn_metrics.go @@ -60,7 +60,7 @@ var ( } metaCommits1PCRates = metric.Metadata{ Name: "txn.commits1PC", - Help: "Number of KV transaction on-phase commit attempts", + Help: "Number of KV transaction one-phase commit attempts", Measurement: "KV Transactions", Unit: metric.Unit_COUNT, } diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index 6a00961d4717..66aceda31a77 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -70,6 +70,7 @@ var retiredSettings = map[string]struct{}{ "sql.distsql.merge_joins.enabled": {}, "sql.defaults.optimizer_foreign_keys.enabled": {}, "sql.defaults.experimental_optimizer_foreign_key_cascades.enabled": {}, + "sql.parallel_scans.enabled": {}, } // register adds a setting to the registry. diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index ebdb80c78616..d1366635cda0 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -39,14 +39,12 @@ import ( // from kv, presenting it as coldata.Batches via the exec.Operator interface. type colBatchScan struct { colexecbase.ZeroInputNode - spans roachpb.Spans - flowCtx *execinfra.FlowCtx - rf *cFetcher - limitHint int64 - ctx context.Context - // maxResults is non-zero if there is a limit on the total number of rows - // that the colBatchScan will read. - maxResults uint64 + spans roachpb.Spans + flowCtx *execinfra.FlowCtx + rf *cFetcher + limitHint int64 + parallelize bool + ctx context.Context // init is true after Init() has been called. init bool } @@ -57,8 +55,7 @@ func (s *colBatchScan) Init() { s.ctx = context.Background() s.init = true - limitBatches := execinfra.ScanShouldLimitBatches(s.maxResults, s.limitHint, s.flowCtx) - + limitBatches := !s.parallelize if err := s.rf.StartScan( s.ctx, s.flowCtx.Txn, s.spans, limitBatches, s.limitHint, s.flowCtx.TraceKV, @@ -153,11 +150,13 @@ func NewColBatchScan( spans[i] = spec.Spans[i].Span } return &colBatchScan{ - spans: spans, - flowCtx: flowCtx, - rf: &fetcher, - limitHint: limitHint, - maxResults: spec.MaxResults, + spans: spans, + flowCtx: flowCtx, + rf: &fetcher, + limitHint: limitHint, + // Parallelize shouldn't be set when there's a limit hint, but double-check + // just in case. + parallelize: spec.Parallelize && limitHint == 0, }, nil } diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 64d21d5770e0..1ab892627792 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1122,7 +1122,7 @@ func (dsp *DistSQLPlanner) createTableReaders( spans: n.spans, reverse: n.reverse, scanVisibility: n.colCfg.visibility, - maxResults: n.maxResults, + parallelize: n.parallelize, estimatedRowCount: n.estimatedRowCount, reqOrdering: n.reqOrdering, cols: n.cols, @@ -1142,7 +1142,7 @@ type tableReaderPlanningInfo struct { spans []roachpb.Span reverse bool scanVisibility execinfrapb.ScanVisibility - maxResults uint64 + parallelize bool estimatedRowCount uint64 reqOrdering ReqOrdering cols []*sqlbase.ColumnDescriptor @@ -1198,7 +1198,7 @@ func (dsp *DistSQLPlanner) planTableReaders( tr.Spans = append(tr.Spans, execinfrapb.TableReaderSpan{Span: sp.Spans[j]}) } - tr.MaxResults = info.maxResults + tr.Parallelize = info.parallelize p.TotalEstimatedScannedRows += info.estimatedRowCount if info.estimatedRowCount > p.MaxEstimatedRowCount { p.MaxEstimatedRowCount = info.estimatedRowCount diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index 52bc2955af8c..55b3a7091c1c 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -164,7 +164,7 @@ func (e *distSQLSpecExecFactory) ConstructScan( hardLimit int64, softLimit int64, reverse bool, - maxResults uint64, + parallelize bool, reqOrdering exec.OutputOrdering, rowCount float64, locking *tree.LockingItem, @@ -204,9 +204,6 @@ func (e *distSQLSpecExecFactory) ConstructScan( return e.ConstructValues([][]tree.TypedExpr{} /* rows */, p.ResultColumns) } - // TODO(yuzefovich): scanNode adds "parallel" attribute in walk.go when - // scanNode.canParallelize() returns true. We should plumb that info from - // here somehow as well. var spans roachpb.Spans if invertedConstraint != nil { spans, err = GenerateInvertedSpans(invertedConstraint, sb) @@ -277,7 +274,7 @@ func (e *distSQLSpecExecFactory) ConstructScan( spans: spans, reverse: reverse, scanVisibility: colCfg.visibility, - maxResults: maxResults, + parallelize: parallelize, estimatedRowCount: uint64(rowCount), reqOrdering: ReqOrdering(reqOrdering), cols: cols, diff --git a/pkg/sql/execinfra/scanbase.go b/pkg/sql/execinfra/scanbase.go index 676c1af22741..50bbb5eb6ce2 100644 --- a/pkg/sql/execinfra/scanbase.go +++ b/pkg/sql/execinfra/scanbase.go @@ -10,31 +10,7 @@ package execinfra -import ( - "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -) - -// ParallelScanResultThreshold is the number of results up to which, if the -// maximum number of results returned by a scan is known, the table reader -// disables batch limits in the dist sender. This results in the parallelization -// of these scans. -const ParallelScanResultThreshold = 10000 - -// ScanShouldLimitBatches returns whether the scan should pace itself. -func ScanShouldLimitBatches(maxResults uint64, limitHint int64, flowCtx *FlowCtx) bool { - // We don't limit batches if the scan doesn't have a limit, and if the - // spans scanned will return less than the ParallelScanResultThreshold. - // This enables distsender parallelism - if we limit batches, distsender - // does *not* parallelize multi-range scan requests. - if maxResults != 0 && - maxResults < ParallelScanResultThreshold && - limitHint == 0 && - sqlbase.ParallelScans.Get(&flowCtx.Cfg.Settings.SV) { - return false - } - return true -} +import "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" // Prettier aliases for execinfrapb.ScanVisibility values. const ( diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index d7287205b311..45544860f8e8 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -62,7 +62,7 @@ import ( // // ATTENTION: When updating these fields, add to version_history.txt explaining // what changed. -const Version execinfrapb.DistSQLVersion = 30 +const Version execinfrapb.DistSQLVersion = 31 // MinAcceptedVersion is the oldest version that the server is // compatible with; see above. diff --git a/pkg/sql/execinfrapb/processors_sql.pb.go b/pkg/sql/execinfrapb/processors_sql.pb.go index e7cbcd20262c..fbc2298fcccd 100644 --- a/pkg/sql/execinfrapb/processors_sql.pb.go +++ b/pkg/sql/execinfrapb/processors_sql.pb.go @@ -64,7 +64,7 @@ func (x *ScanVisibility) UnmarshalJSON(data []byte) error { return nil } func (ScanVisibility) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{0} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{0} } // These mirror the aggregate functions supported by sql/parser. See @@ -181,7 +181,7 @@ func (x *AggregatorSpec_Func) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Func) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{12, 0} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{12, 0} } type AggregatorSpec_Type int32 @@ -227,7 +227,7 @@ func (x *AggregatorSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{12, 1} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{12, 1} } type WindowerSpec_WindowFunc int32 @@ -291,7 +291,7 @@ func (x *WindowerSpec_WindowFunc) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_WindowFunc) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15, 0} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15, 0} } // Mode indicates which mode of framing is used. @@ -335,7 +335,7 @@ func (x *WindowerSpec_Frame_Mode) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Mode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15, 1, 0} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15, 1, 0} } // BoundType indicates which type of boundary is used. @@ -382,7 +382,7 @@ func (x *WindowerSpec_Frame_BoundType) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_BoundType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15, 1, 1} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15, 1, 1} } // Exclusion specifies the type of frame exclusion. @@ -425,7 +425,7 @@ func (x *WindowerSpec_Frame_Exclusion) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Exclusion) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15, 1, 2} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15, 1, 2} } // ValuesCoreSpec is the core of a processor that has no inputs and generates @@ -445,7 +445,7 @@ func (m *ValuesCoreSpec) Reset() { *m = ValuesCoreSpec{} } func (m *ValuesCoreSpec) String() string { return proto.CompactTextString(m) } func (*ValuesCoreSpec) ProtoMessage() {} func (*ValuesCoreSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{0} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{0} } func (m *ValuesCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -504,9 +504,11 @@ type TableReaderSpec struct { // the consumer of this TableReader expects to be able to see in-progress // schema changes. Visibility ScanVisibility `protobuf:"varint,7,opt,name=visibility,enum=cockroach.sql.distsqlrun.ScanVisibility" json:"visibility"` - // If non-zero, this is a guarantee for the upper bound of rows a TableReader - // will read. If 0, the number of results is unbounded. - MaxResults uint64 `protobuf:"varint,8,opt,name=max_results,json=maxResults" json:"max_results"` + // If set, the TableReader can read all the spans in parallel, without any + // batch limits. This should only be the case when there is a known upper + // bound on the number of rows we can read, and when there is no limit or + // limit hint. + Parallelize bool `protobuf:"varint,12,opt,name=parallelize" json:"parallelize"` // If non-zero, this enables inconsistent historical scanning where different // batches can be read with different timestamps. This is used for // long-running table statistics which may outlive the TTL. Using this setting @@ -550,7 +552,7 @@ func (m *TableReaderSpec) Reset() { *m = TableReaderSpec{} } func (m *TableReaderSpec) String() string { return proto.CompactTextString(m) } func (*TableReaderSpec) ProtoMessage() {} func (*TableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{1} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{1} } func (m *TableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -608,7 +610,7 @@ func (m *IndexSkipTableReaderSpec) Reset() { *m = IndexSkipTableReaderSp func (m *IndexSkipTableReaderSpec) String() string { return proto.CompactTextString(m) } func (*IndexSkipTableReaderSpec) ProtoMessage() {} func (*IndexSkipTableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{2} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{2} } func (m *IndexSkipTableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -710,7 +712,7 @@ func (m *JoinReaderSpec) Reset() { *m = JoinReaderSpec{} } func (m *JoinReaderSpec) String() string { return proto.CompactTextString(m) } func (*JoinReaderSpec) ProtoMessage() {} func (*JoinReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{3} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{3} } func (m *JoinReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +756,7 @@ func (m *SorterSpec) Reset() { *m = SorterSpec{} } func (m *SorterSpec) String() string { return proto.CompactTextString(m) } func (*SorterSpec) ProtoMessage() {} func (*SorterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{4} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{4} } func (m *SorterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -816,7 +818,7 @@ func (m *DistinctSpec) Reset() { *m = DistinctSpec{} } func (m *DistinctSpec) String() string { return proto.CompactTextString(m) } func (*DistinctSpec) ProtoMessage() {} func (*DistinctSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{5} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{5} } func (m *DistinctSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -851,7 +853,7 @@ func (m *OrdinalitySpec) Reset() { *m = OrdinalitySpec{} } func (m *OrdinalitySpec) String() string { return proto.CompactTextString(m) } func (*OrdinalitySpec) ProtoMessage() {} func (*OrdinalitySpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{6} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{6} } func (m *OrdinalitySpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -909,7 +911,7 @@ func (m *ZigzagJoinerSpec) Reset() { *m = ZigzagJoinerSpec{} } func (m *ZigzagJoinerSpec) String() string { return proto.CompactTextString(m) } func (*ZigzagJoinerSpec) ProtoMessage() {} func (*ZigzagJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{7} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{7} } func (m *ZigzagJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -985,7 +987,7 @@ func (m *MergeJoinerSpec) Reset() { *m = MergeJoinerSpec{} } func (m *MergeJoinerSpec) String() string { return proto.CompactTextString(m) } func (*MergeJoinerSpec) ProtoMessage() {} func (*MergeJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{8} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{8} } func (m *MergeJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1072,7 +1074,7 @@ func (m *HashJoinerSpec) Reset() { *m = HashJoinerSpec{} } func (m *HashJoinerSpec) String() string { return proto.CompactTextString(m) } func (*HashJoinerSpec) ProtoMessage() {} func (*HashJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{9} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{9} } func (m *HashJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1169,7 +1171,7 @@ func (m *InvertedJoinerSpec) Reset() { *m = InvertedJoinerSpec{} } func (m *InvertedJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InvertedJoinerSpec) ProtoMessage() {} func (*InvertedJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{10} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{10} } func (m *InvertedJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1222,7 +1224,7 @@ func (m *InvertedFiltererSpec) Reset() { *m = InvertedFiltererSpec{} } func (m *InvertedFiltererSpec) String() string { return proto.CompactTextString(m) } func (*InvertedFiltererSpec) ProtoMessage() {} func (*InvertedFiltererSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{11} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{11} } func (m *InvertedFiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1268,7 +1270,7 @@ func (m *AggregatorSpec) Reset() { *m = AggregatorSpec{} } func (m *AggregatorSpec) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec) ProtoMessage() {} func (*AggregatorSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{12} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{12} } func (m *AggregatorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1319,7 +1321,7 @@ func (m *AggregatorSpec_Aggregation) Reset() { *m = AggregatorSpec_Aggre func (m *AggregatorSpec_Aggregation) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec_Aggregation) ProtoMessage() {} func (*AggregatorSpec_Aggregation) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{12, 0} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{12, 0} } func (m *AggregatorSpec_Aggregation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1394,7 +1396,7 @@ func (m *InterleavedReaderJoinerSpec) Reset() { *m = InterleavedReaderJo func (m *InterleavedReaderJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec) ProtoMessage() {} func (*InterleavedReaderJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{13} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{13} } func (m *InterleavedReaderJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1450,7 +1452,7 @@ func (m *InterleavedReaderJoinerSpec_Table) Reset() { *m = InterleavedRe func (m *InterleavedReaderJoinerSpec_Table) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec_Table) ProtoMessage() {} func (*InterleavedReaderJoinerSpec_Table) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{13, 0} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{13, 0} } func (m *InterleavedReaderJoinerSpec_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1490,7 +1492,7 @@ func (m *ProjectSetSpec) Reset() { *m = ProjectSetSpec{} } func (m *ProjectSetSpec) String() string { return proto.CompactTextString(m) } func (*ProjectSetSpec) ProtoMessage() {} func (*ProjectSetSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{14} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{14} } func (m *ProjectSetSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1532,7 +1534,7 @@ func (m *WindowerSpec) Reset() { *m = WindowerSpec{} } func (m *WindowerSpec) String() string { return proto.CompactTextString(m) } func (*WindowerSpec) ProtoMessage() {} func (*WindowerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15} } func (m *WindowerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1568,7 +1570,7 @@ func (m *WindowerSpec_Func) Reset() { *m = WindowerSpec_Func{} } func (m *WindowerSpec_Func) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Func) ProtoMessage() {} func (*WindowerSpec_Func) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15, 0} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15, 0} } func (m *WindowerSpec_Func) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1604,7 +1606,7 @@ func (m *WindowerSpec_Frame) Reset() { *m = WindowerSpec_Frame{} } func (m *WindowerSpec_Frame) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame) ProtoMessage() {} func (*WindowerSpec_Frame) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15, 1} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15, 1} } func (m *WindowerSpec_Frame) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1645,7 +1647,7 @@ func (m *WindowerSpec_Frame_Bound) Reset() { *m = WindowerSpec_Frame_Bou func (m *WindowerSpec_Frame_Bound) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bound) ProtoMessage() {} func (*WindowerSpec_Frame_Bound) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15, 1, 0} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15, 1, 0} } func (m *WindowerSpec_Frame_Bound) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1681,7 +1683,7 @@ func (m *WindowerSpec_Frame_Bounds) Reset() { *m = WindowerSpec_Frame_Bo func (m *WindowerSpec_Frame_Bounds) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bounds) ProtoMessage() {} func (*WindowerSpec_Frame_Bounds) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15, 1, 1} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15, 1, 1} } func (m *WindowerSpec_Frame_Bounds) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1731,7 +1733,7 @@ func (m *WindowerSpec_WindowFn) Reset() { *m = WindowerSpec_WindowFn{} } func (m *WindowerSpec_WindowFn) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_WindowFn) ProtoMessage() {} func (*WindowerSpec_WindowFn) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_b868445ee9565a25, []int{15, 2} + return fileDescriptor_processors_sql_d2d2fa74141fd705, []int{15, 2} } func (m *WindowerSpec_WindowFn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1889,9 +1891,6 @@ func (m *TableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x38 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Visibility)) - dAtA[i] = 0x40 - i++ - i = encodeVarintProcessorsSql(dAtA, i, uint64(m.MaxResults)) dAtA[i] = 0x48 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.MaxTimestampAgeNanos)) @@ -1901,6 +1900,14 @@ func (m *TableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x58 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingWaitPolicy)) + dAtA[i] = 0x60 + i++ + if m.Parallelize { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ return i, nil } @@ -3015,10 +3022,10 @@ func (m *TableReaderSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(m.LimitHint)) n += 2 n += 1 + sovProcessorsSql(uint64(m.Visibility)) - n += 1 + sovProcessorsSql(uint64(m.MaxResults)) n += 1 + sovProcessorsSql(uint64(m.MaxTimestampAgeNanos)) n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) + n += 2 return n } @@ -3808,11 +3815,11 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { break } } - case 8: + case 9: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxResults", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field MaxTimestampAgeNanos", wireType) } - m.MaxResults = 0 + m.MaxTimestampAgeNanos = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowProcessorsSql @@ -3822,16 +3829,16 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MaxResults |= (uint64(b) & 0x7F) << shift + m.MaxTimestampAgeNanos |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - case 9: + case 10: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxTimestampAgeNanos", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LockingStrength", wireType) } - m.MaxTimestampAgeNanos = 0 + m.LockingStrength = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowProcessorsSql @@ -3841,16 +3848,16 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MaxTimestampAgeNanos |= (uint64(b) & 0x7F) << shift + m.LockingStrength |= (sqlbase.ScanLockingStrength(b) & 0x7F) << shift if b < 0x80 { break } } - case 10: + case 11: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LockingStrength", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field LockingWaitPolicy", wireType) } - m.LockingStrength = 0 + m.LockingWaitPolicy = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowProcessorsSql @@ -3860,16 +3867,16 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.LockingStrength |= (sqlbase.ScanLockingStrength(b) & 0x7F) << shift + m.LockingWaitPolicy |= (sqlbase.ScanLockingWaitPolicy(b) & 0x7F) << shift if b < 0x80 { break } } - case 11: + case 12: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LockingWaitPolicy", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Parallelize", wireType) } - m.LockingWaitPolicy = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowProcessorsSql @@ -3879,11 +3886,12 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.LockingWaitPolicy |= (sqlbase.ScanLockingWaitPolicy(b) & 0x7F) << shift + v |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + m.Parallelize = bool(v != 0) default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -7934,175 +7942,175 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_b868445ee9565a25) + proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_d2d2fa74141fd705) } -var fileDescriptor_processors_sql_b868445ee9565a25 = []byte{ - // 2645 bytes of a gzipped FileDescriptorProto +var fileDescriptor_processors_sql_d2d2fa74141fd705 = []byte{ + // 2649 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5a, 0x4b, 0x73, 0x1b, 0xc7, - 0x11, 0xe6, 0xe2, 0x41, 0x02, 0x8d, 0x07, 0x57, 0x23, 0xda, 0x82, 0x21, 0x17, 0x45, 0xc1, 0x2f, - 0x4a, 0x96, 0xc9, 0x98, 0x49, 0x39, 0x65, 0x3b, 0x07, 0xe3, 0xb1, 0xa0, 0x40, 0x81, 0xbb, 0xd4, - 0x02, 0x90, 0xfc, 0xa8, 0xf2, 0xd6, 0x12, 0x3b, 0x04, 0xd7, 0x5a, 0xec, 0x80, 0xbb, 0x0b, 0x91, - 0xf4, 0x1f, 0xc8, 0x35, 0xa9, 0x5c, 0x52, 0x39, 0xa4, 0x7c, 0x49, 0xfc, 0x07, 0x72, 0x76, 0x2e, - 0x39, 0xe8, 0xe8, 0x53, 0xe2, 0x53, 0x2a, 0x96, 0xfe, 0x40, 0x6e, 0xae, 0xdc, 0x52, 0xf3, 0xd8, - 0xe5, 0x02, 0x45, 0x28, 0x82, 0x24, 0x5b, 0x95, 0x0b, 0x0b, 0xd3, 0x3d, 0xfd, 0x75, 0x4f, 0x77, - 0x4f, 0x4f, 0xcf, 0x2c, 0xe1, 0x75, 0xff, 0xc8, 0xd9, 0xc4, 0x27, 0xb8, 0x6f, 0xbb, 0x07, 0x9e, - 0x39, 0xda, 0xdf, 0x1c, 0x79, 0xa4, 0x8f, 0x7d, 0x9f, 0x78, 0xbe, 0xe1, 0x1f, 0x39, 0x1b, 0x23, - 0x8f, 0x04, 0x04, 0x95, 0xfa, 0xa4, 0x7f, 0xcf, 0x23, 0x66, 0xff, 0x70, 0x83, 0x12, 0x2d, 0xdb, - 0x0f, 0xfc, 0x23, 0xc7, 0x1b, 0xbb, 0xe5, 0x57, 0xa9, 0xbc, 0x7f, 0xe4, 0xec, 0x9b, 0x3e, 0xde, - 0xf4, 0x03, 0x6f, 0xdc, 0x0f, 0xc6, 0x1e, 0xb6, 0xb8, 0x5c, 0xf9, 0x72, 0x9c, 0xfb, 0x05, 0xb1, - 0x5d, 0x23, 0x38, 0x1d, 0x61, 0xc1, 0x7c, 0x25, 0xce, 0x74, 0x48, 0xff, 0x9e, 0xed, 0x0e, 0x04, - 0xeb, 0x25, 0xca, 0xa2, 0x53, 0x7d, 0xfe, 0x57, 0x90, 0xcb, 0xd3, 0xc6, 0x5a, 0x66, 0x60, 0x0a, - 0xde, 0x1b, 0x8f, 0x59, 0x08, 0xd5, 0x20, 0xa6, 0x5d, 0xa7, 0xd3, 0xc8, 0x28, 0xd8, 0xb4, 0xdd, - 0xfb, 0xd8, 0x0b, 0xb0, 0x85, 0x4f, 0x46, 0xde, 0xa6, 0x3f, 0x32, 0x5d, 0x83, 0xfe, 0xc2, 0xbe, - 0x6f, 0x13, 0x57, 0xcc, 0x5d, 0x19, 0x90, 0x01, 0x61, 0x3f, 0x37, 0xe9, 0x2f, 0x4e, 0xad, 0xfc, - 0x4e, 0x82, 0xe2, 0x1d, 0xd3, 0x19, 0x63, 0xbf, 0x4e, 0x3c, 0xdc, 0x19, 0xe1, 0x3e, 0xaa, 0xc3, - 0x52, 0x9f, 0x38, 0xe3, 0xa1, 0xeb, 0x97, 0xa4, 0xb5, 0xe4, 0x7a, 0x6e, 0xeb, 0xb5, 0x8d, 0x59, - 0x0e, 0xdb, 0x68, 0x98, 0xc1, 0x78, 0xd8, 0x72, 0x0f, 0x48, 0x2d, 0xf5, 0xe0, 0x9f, 0x57, 0x16, - 0xf4, 0x50, 0x12, 0x5d, 0x86, 0xac, 0x67, 0x1e, 0x1b, 0xfb, 0xa7, 0x01, 0xf6, 0x4b, 0x89, 0xb5, - 0xe4, 0x7a, 0x5e, 0xcf, 0x78, 0xe6, 0x71, 0x8d, 0x8e, 0xd1, 0x15, 0xc8, 0xb8, 0xe3, 0xa1, 0xe1, - 0x91, 0x63, 0xbf, 0x94, 0x5c, 0x93, 0xd6, 0x53, 0xa1, 0xb4, 0x3b, 0x1e, 0xea, 0xe4, 0xd8, 0xaf, - 0xfc, 0x21, 0x0d, 0xcb, 0x5d, 0x73, 0xdf, 0xc1, 0x3a, 0x36, 0x2d, 0xec, 0x31, 0xb3, 0x6a, 0x90, - 0x0e, 0x28, 0xa9, 0x24, 0xad, 0x49, 0xeb, 0xb9, 0xad, 0x37, 0xa7, 0x8c, 0x12, 0xae, 0xdf, 0x60, - 0x62, 0x0d, 0xec, 0xf7, 0x3d, 0x7b, 0x14, 0x10, 0x4f, 0x20, 0x73, 0x51, 0x74, 0x15, 0xb2, 0xb6, - 0x6b, 0xe1, 0x13, 0xc3, 0xb6, 0x4e, 0x4a, 0x89, 0x35, 0x69, 0xbd, 0x20, 0xf8, 0x19, 0x46, 0x6e, - 0x59, 0x27, 0x68, 0x15, 0x96, 0x3c, 0x7c, 0x1f, 0x7b, 0x3e, 0x66, 0xa6, 0x65, 0x42, 0xd3, 0x04, - 0x11, 0x29, 0x90, 0xa6, 0xfe, 0xf5, 0x4b, 0x29, 0xe6, 0x9b, 0x6b, 0xb3, 0x7d, 0x33, 0xb1, 0x00, - 0xd3, 0x0d, 0x2d, 0x61, 0xd2, 0xe8, 0x35, 0x00, 0xc7, 0x1e, 0xda, 0x81, 0x71, 0x68, 0xbb, 0x41, - 0x29, 0xbd, 0x26, 0xad, 0x27, 0xc5, 0x84, 0x2c, 0xa3, 0xdf, 0xb4, 0xdd, 0x80, 0xfa, 0xc9, 0xf6, - 0x8d, 0xfe, 0x21, 0xee, 0xdf, 0x2b, 0x2d, 0xc6, 0x8d, 0xb1, 0xfd, 0x3a, 0x25, 0x22, 0x15, 0xe0, - 0xbe, 0xed, 0xdb, 0xfb, 0xb6, 0x63, 0x07, 0xa7, 0xa5, 0xa5, 0x35, 0x69, 0xbd, 0xb8, 0xb5, 0x3e, - 0xdb, 0xa2, 0x4e, 0xdf, 0x74, 0xef, 0x44, 0xf3, 0x05, 0x58, 0x0c, 0x01, 0xbd, 0x01, 0xb9, 0xa1, - 0x79, 0x62, 0x78, 0xd8, 0x1f, 0x3b, 0x81, 0x5f, 0xca, 0xc4, 0x62, 0x03, 0x43, 0xf3, 0x44, 0xe7, - 0x74, 0xf4, 0x21, 0x5c, 0xa2, 0xd3, 0x02, 0x7b, 0x88, 0xfd, 0xc0, 0x1c, 0x8e, 0x0c, 0x73, 0x80, - 0x0d, 0xd7, 0x74, 0x89, 0x5f, 0xca, 0xc6, 0x44, 0x56, 0x86, 0xe6, 0x49, 0x37, 0x9c, 0x53, 0x1d, - 0x60, 0x95, 0xce, 0x40, 0x9f, 0x81, 0x2c, 0xb6, 0x87, 0xe1, 0x07, 0x1e, 0x76, 0x07, 0xc1, 0x61, - 0x09, 0x98, 0xe5, 0xd7, 0x67, 0x84, 0x94, 0x9a, 0xdd, 0xe6, 0x22, 0x1d, 0x21, 0x21, 0x34, 0x2c, - 0x3b, 0x93, 0x64, 0xb4, 0x0f, 0x17, 0x43, 0xf0, 0x63, 0xd3, 0x0e, 0x8c, 0x11, 0x71, 0xec, 0xfe, - 0x69, 0x29, 0xc7, 0xf0, 0x6f, 0xfc, 0x6f, 0xfc, 0xbb, 0xa6, 0x1d, 0xec, 0x31, 0x19, 0xa1, 0xe1, - 0x82, 0x33, 0xcd, 0xa8, 0xfc, 0x27, 0x09, 0xa5, 0x16, 0x4d, 0x97, 0xce, 0x3d, 0x7b, 0xf4, 0x82, - 0xb2, 0x34, 0xca, 0xc2, 0xe4, 0x33, 0x65, 0xe1, 0x64, 0xfe, 0xa4, 0x9e, 0x39, 0x7f, 0x62, 0x9b, - 0x27, 0x7d, 0xde, 0xe6, 0x39, 0x2f, 0xf6, 0x8b, 0x3f, 0x72, 0xec, 0x97, 0x9e, 0x67, 0xec, 0xbf, - 0x4e, 0x43, 0x71, 0x87, 0xd8, 0xee, 0x4f, 0x1f, 0xf1, 0x6b, 0x50, 0x74, 0x08, 0xb9, 0x37, 0x1e, - 0x19, 0x61, 0x71, 0xa6, 0xa1, 0x2f, 0xd4, 0x12, 0xb2, 0xa4, 0x17, 0x38, 0xa7, 0x2e, 0x6a, 0x6f, - 0x1d, 0x96, 0x08, 0x3f, 0x00, 0x58, 0x48, 0x73, 0x5b, 0xaf, 0xcf, 0x0e, 0xa9, 0x12, 0x1d, 0x13, - 0x42, 0xe3, 0x22, 0x71, 0x29, 0x0d, 0xbd, 0x0f, 0x29, 0x7a, 0x58, 0x89, 0xf0, 0x5c, 0x99, 0xb1, - 0x2a, 0xea, 0x8b, 0xee, 0xe9, 0x08, 0x0b, 0x61, 0x26, 0xf2, 0xdc, 0xab, 0xd2, 0xfb, 0xf0, 0xf2, - 0xe4, 0xd2, 0x0d, 0xd3, 0xc3, 0xc6, 0x3d, 0x7c, 0xca, 0x0a, 0x54, 0x98, 0x64, 0x17, 0x27, 0x9c, - 0x50, 0xf5, 0xf0, 0x2d, 0x7c, 0x7a, 0x6e, 0xc2, 0x65, 0x7f, 0xe4, 0x84, 0x83, 0xe7, 0x98, 0x70, - 0xe8, 0x5d, 0xb8, 0x30, 0x34, 0x6d, 0x37, 0x30, 0x6d, 0xd7, 0x20, 0x9e, 0x85, 0x3d, 0xdb, 0x1d, - 0xb0, 0x72, 0x16, 0x2e, 0x5b, 0x0e, 0xd9, 0x9a, 0xe0, 0xee, 0xa4, 0x32, 0x69, 0x79, 0x91, 0x1e, - 0xec, 0xd0, 0x21, 0x5e, 0x20, 0xb2, 0xf4, 0x36, 0x2c, 0x93, 0x71, 0x30, 0x1a, 0x07, 0x67, 0x28, - 0x3c, 0x5f, 0x2b, 0xb3, 0x03, 0x13, 0x22, 0x0a, 0x4d, 0x45, 0x0e, 0x10, 0x52, 0xd1, 0x16, 0xa0, - 0x10, 0xcb, 0x18, 0x9a, 0x41, 0xff, 0xd0, 0x70, 0xb0, 0x3b, 0x91, 0xbd, 0x72, 0xc8, 0xdf, 0xa5, - 0xec, 0x36, 0x76, 0x2b, 0xdf, 0x48, 0x90, 0x6f, 0xd8, 0x7e, 0x60, 0xbb, 0xfd, 0x80, 0xd9, 0xf5, - 0x16, 0x2c, 0xb3, 0x49, 0xd8, 0x32, 0xe2, 0x4d, 0x47, 0x41, 0x2f, 0x0a, 0x72, 0x98, 0xd4, 0xd7, - 0x40, 0xb6, 0x84, 0x60, 0x34, 0x33, 0xc1, 0x66, 0x2e, 0x87, 0xf4, 0x70, 0xea, 0x16, 0x20, 0x77, - 0xec, 0x38, 0x3c, 0x4d, 0x42, 0xe6, 0xc4, 0x69, 0x2e, 0x33, 0x7e, 0xd5, 0xc3, 0xa1, 0x2d, 0xe8, - 0x4d, 0xc8, 0x63, 0xcf, 0x23, 0x9e, 0x41, 0x5c, 0xc3, 0x1a, 0x8f, 0xd8, 0xc6, 0xc9, 0x86, 0xb9, - 0xc8, 0x38, 0x9a, 0xdb, 0x18, 0x8f, 0x2a, 0x32, 0x14, 0x35, 0xcf, 0xb2, 0x5d, 0x93, 0x66, 0x26, - 0x5d, 0x41, 0xe5, 0xf7, 0x49, 0x90, 0x3f, 0xb5, 0x07, 0x5f, 0x9a, 0x03, 0xba, 0x19, 0x84, 0xbb, - 0x1b, 0xb0, 0xc8, 0x76, 0x76, 0xd8, 0x42, 0xcd, 0x57, 0x15, 0x84, 0x2c, 0x6a, 0x02, 0xe0, 0xa3, - 0x89, 0xd5, 0xe6, 0xb6, 0xae, 0xce, 0x8e, 0x97, 0x58, 0x7f, 0xd8, 0x47, 0xe0, 0xa3, 0x33, 0xdf, - 0x15, 0x79, 0x79, 0x21, 0xdc, 0xf4, 0x89, 0xda, 0xc1, 0x38, 0x62, 0x4d, 0xcf, 0xa9, 0x76, 0xdc, - 0x82, 0xfc, 0x81, 0x7d, 0x82, 0x2d, 0xe3, 0x3e, 0xeb, 0x2c, 0x4b, 0x69, 0x66, 0xf9, 0x63, 0x4a, - 0xc0, 0x64, 0x07, 0xaa, 0xe7, 0x98, 0x34, 0x27, 0x3e, 0x43, 0x21, 0xaa, 0xfc, 0x3d, 0x09, 0xcb, - 0xbb, 0xd8, 0x1b, 0xe0, 0x58, 0x64, 0x76, 0xa1, 0xe0, 0xe0, 0x83, 0x67, 0xd8, 0x06, 0x79, 0x2a, - 0x1e, 0x6d, 0x02, 0x0d, 0x8a, 0x9e, 0x3d, 0x38, 0x8c, 0xe1, 0x25, 0xe6, 0xc4, 0x2b, 0x30, 0xf9, - 0x08, 0x30, 0x16, 0x80, 0xf4, 0x8b, 0x28, 0xde, 0xd7, 0xa0, 0x40, 0x37, 0x87, 0x81, 0x8f, 0xc6, - 0x66, 0x54, 0xbf, 0xc3, 0x7d, 0x93, 0xa7, 0x2c, 0x45, 0x70, 0xd0, 0x07, 0x70, 0x89, 0xb9, 0xf2, - 0x2c, 0x47, 0x67, 0x14, 0x66, 0x7c, 0x10, 0x28, 0x47, 0x93, 0x85, 0xf9, 0x57, 0x50, 0xe2, 0x7e, - 0x3b, 0x47, 0x38, 0x1b, 0x13, 0x5e, 0x61, 0xb3, 0xa6, 0xa4, 0x2b, 0xbf, 0x4d, 0x42, 0xf1, 0xa6, - 0xe9, 0x1f, 0xc6, 0xe2, 0x7a, 0x1d, 0x96, 0xa7, 0x8c, 0xe1, 0x85, 0x44, 0x1c, 0x90, 0x71, 0x13, - 0xd0, 0x0d, 0x90, 0xa7, 0x95, 0xf3, 0x5a, 0xc2, 0x26, 0x17, 0x27, 0x55, 0xbe, 0xf0, 0x88, 0xbc, - 0x0d, 0xc5, 0x21, 0x4d, 0xe2, 0xb3, 0x0a, 0x19, 0x0f, 0x49, 0x81, 0xf3, 0x42, 0x63, 0x5f, 0x5c, - 0x4c, 0x7e, 0x48, 0x00, 0x6a, 0x89, 0x6b, 0x68, 0x2c, 0x2e, 0x3f, 0x59, 0x7b, 0x54, 0x98, 0xe8, - 0x11, 0x58, 0xb9, 0x2f, 0x44, 0x5b, 0x36, 0xd6, 0x1a, 0x20, 0x0d, 0x0a, 0xe1, 0x75, 0xf9, 0x69, - 0x0b, 0x5d, 0x3e, 0x04, 0x60, 0xb1, 0x7d, 0xc1, 0x09, 0x52, 0xf9, 0xb3, 0x04, 0x2b, 0xa1, 0xe7, - 0x9b, 0xb6, 0x13, 0x60, 0x4f, 0xf8, 0x7e, 0x03, 0xe4, 0x68, 0xa5, 0x7d, 0xe2, 0x30, 0xf7, 0x49, - 0x31, 0xbf, 0x14, 0x43, 0x6e, 0x9d, 0x38, 0xd4, 0x89, 0x9f, 0x4f, 0x7b, 0x86, 0xd7, 0xb2, 0x9f, - 0x4f, 0x19, 0x43, 0x46, 0xc1, 0x46, 0xfc, 0xc1, 0x61, 0x83, 0x5e, 0x2c, 0xce, 0x16, 0xb6, 0xe7, - 0x91, 0x80, 0x9c, 0xe7, 0xa8, 0xca, 0xbf, 0x97, 0xa0, 0x58, 0x1d, 0x0c, 0x3c, 0x3c, 0x30, 0x03, - 0xc2, 0x4d, 0xbc, 0x0a, 0x30, 0xf0, 0x08, 0x0f, 0x5b, 0x7c, 0x13, 0x66, 0x19, 0xb5, 0x4e, 0x1c, - 0x1f, 0x7d, 0x0e, 0x79, 0x53, 0x08, 0xd9, 0x24, 0xba, 0xf2, 0xfc, 0x62, 0xb6, 0x8f, 0x27, 0x55, - 0x44, 0xc3, 0x58, 0xf8, 0xe2, 0x78, 0xe8, 0x67, 0xa2, 0x8f, 0xc1, 0x96, 0x11, 0x33, 0x25, 0x15, - 0x99, 0x22, 0x0b, 0xee, 0x76, 0x64, 0xd1, 0xb6, 0x88, 0x55, 0x9a, 0xc5, 0xea, 0x9d, 0x27, 0xb6, - 0x64, 0x3a, 0x72, 0xe5, 0x5f, 0x27, 0x20, 0x17, 0x33, 0x8f, 0x02, 0x1f, 0x8c, 0xdd, 0x3e, 0x0b, - 0xd2, 0x3c, 0xc0, 0xcd, 0xb1, 0xdb, 0x0f, 0x81, 0x29, 0x00, 0x5a, 0x83, 0x4c, 0xd4, 0xf8, 0x24, - 0x62, 0x5b, 0x37, 0xa2, 0xa2, 0xd7, 0xa1, 0x78, 0xc0, 0x72, 0x25, 0xca, 0x0c, 0xba, 0x0d, 0x0a, - 0x7a, 0x9e, 0x53, 0x45, 0x46, 0x5c, 0x62, 0x6f, 0x41, 0x8c, 0x9d, 0x66, 0xcd, 0xd6, 0x62, 0x9f, - 0x33, 0x6e, 0x42, 0xd6, 0xf4, 0x06, 0xe3, 0x21, 0x76, 0x03, 0xbf, 0xb4, 0xc8, 0x22, 0x32, 0x4f, - 0xd6, 0x9f, 0x09, 0xef, 0xa4, 0x32, 0x49, 0x39, 0x55, 0xf9, 0x3a, 0x09, 0x29, 0xba, 0x0a, 0x24, - 0x43, 0xbe, 0xaa, 0x7e, 0x62, 0xa8, 0x5a, 0xd7, 0x50, 0x7b, 0xed, 0xb6, 0xbc, 0x80, 0x96, 0x20, - 0x59, 0xbd, 0xb3, 0x2d, 0x4b, 0x28, 0x0f, 0x99, 0x9a, 0xa6, 0xb5, 0x8d, 0xaa, 0xda, 0x90, 0x13, - 0x28, 0x07, 0x4b, 0x6c, 0xa4, 0xe9, 0x72, 0x12, 0x15, 0x01, 0xea, 0x9a, 0x5a, 0xaf, 0x76, 0x8d, - 0xea, 0xf6, 0xb6, 0x9c, 0x42, 0x59, 0x48, 0xd7, 0xb5, 0x9e, 0xda, 0x95, 0xd3, 0x54, 0x7c, 0xb7, - 0xfa, 0xb1, 0xbc, 0xc4, 0x7e, 0xb4, 0x54, 0x39, 0x83, 0x00, 0x16, 0x3b, 0xdd, 0x46, 0x43, 0xb9, - 0x23, 0x67, 0x29, 0xb1, 0xd3, 0xdb, 0x95, 0x81, 0xc2, 0x75, 0x7a, 0xbb, 0x46, 0x4b, 0xed, 0xca, - 0x39, 0xaa, 0xe9, 0x4e, 0x55, 0x6f, 0x55, 0xd5, 0xba, 0x22, 0xe7, 0x29, 0xeb, 0x63, 0x4d, 0x67, - 0xc8, 0x05, 0xae, 0xa9, 0xa7, 0x76, 0x0d, 0x5d, 0xbb, 0xdb, 0x91, 0x8b, 0x4c, 0xee, 0xb6, 0xde, - 0x68, 0x35, 0x9b, 0xf2, 0x32, 0x42, 0x50, 0x6c, 0xb6, 0xd4, 0x6a, 0xdb, 0x88, 0xa4, 0x65, 0xba, - 0x20, 0x4e, 0x13, 0x3a, 0x2f, 0xa0, 0x02, 0x64, 0xab, 0xba, 0x5e, 0xfd, 0x84, 0x21, 0x22, 0xaa, - 0x6c, 0xa7, 0xa3, 0xa9, 0x6c, 0x74, 0x91, 0x32, 0xe9, 0xa8, 0xc6, 0x86, 0x2b, 0x54, 0x5d, 0xa7, - 0xab, 0xb7, 0xd4, 0x6d, 0x36, 0x7e, 0x89, 0xad, 0xba, 0xd5, 0x65, 0x2e, 0x78, 0x99, 0x2e, 0x84, - 0x0e, 0x34, 0x5d, 0xbe, 0x84, 0x32, 0x90, 0xaa, 0x6b, 0xba, 0x2e, 0x97, 0x50, 0x09, 0x56, 0xf6, - 0x14, 0xbd, 0xae, 0xa8, 0xdd, 0x56, 0x5b, 0x31, 0x1a, 0xad, 0x4e, 0xdd, 0x68, 0xed, 0xee, 0xb5, - 0xe5, 0x57, 0xa6, 0x38, 0x75, 0x4d, 0xed, 0x72, 0x4e, 0x19, 0x5d, 0x84, 0x65, 0x66, 0x83, 0x56, - 0xdb, 0x51, 0xea, 0xdc, 0x89, 0x97, 0xd1, 0x0a, 0xc8, 0xdc, 0x94, 0x18, 0xf5, 0xd5, 0xca, 0x0d, - 0x48, 0xd1, 0x3c, 0xa6, 0x0a, 0xab, 0xbd, 0xae, 0x26, 0x2f, 0x30, 0x7f, 0xd6, 0xab, 0xed, 0xaa, - 0x2e, 0x4b, 0xd4, 0x5e, 0x55, 0x53, 0x0d, 0x31, 0x4e, 0x54, 0xfe, 0xba, 0x08, 0x97, 0x5b, 0x6e, - 0x80, 0x3d, 0x07, 0x9b, 0xf7, 0xb1, 0xc5, 0xef, 0xcd, 0xb1, 0xe3, 0xe1, 0x93, 0xa9, 0x46, 0xf9, - 0xc3, 0xd9, 0x49, 0xf4, 0x18, 0x18, 0x7e, 0x7c, 0x4c, 0x75, 0xcf, 0xb1, 0xc7, 0x88, 0xc4, 0x79, - 0x8f, 0x11, 0x93, 0x4f, 0x70, 0xc9, 0xf3, 0x9f, 0xe0, 0x9e, 0xeb, 0x5d, 0x3a, 0x3d, 0xff, 0xe1, - 0xff, 0xff, 0xfe, 0x62, 0x52, 0xfe, 0x26, 0x01, 0x69, 0x16, 0x1d, 0xf4, 0x11, 0xa4, 0x2c, 0xec, - 0xf7, 0x9f, 0xaa, 0x11, 0x60, 0x92, 0x4f, 0xd2, 0x07, 0xd4, 0x21, 0x35, 0x22, 0x3e, 0x0f, 0xe7, - 0x63, 0xdf, 0xc5, 0xf6, 0x88, 0x1f, 0xec, 0xf1, 0x47, 0x75, 0x9a, 0x41, 0xa1, 0x1e, 0x2a, 0x8c, - 0x1a, 0x90, 0x89, 0xda, 0xf9, 0xd4, 0x9c, 0xed, 0x7c, 0x24, 0x79, 0xf6, 0x46, 0x97, 0x7e, 0x96, - 0x37, 0xba, 0xca, 0xdf, 0x24, 0x28, 0xee, 0x79, 0xe4, 0x0b, 0xdc, 0x0f, 0x3a, 0x98, 0x5f, 0x9a, - 0x3f, 0x82, 0x34, 0xcd, 0xc8, 0x70, 0xcf, 0xcc, 0x93, 0x92, 0x5c, 0x10, 0x6d, 0xc3, 0x85, 0x01, - 0x76, 0xb1, 0x67, 0x06, 0xb1, 0xb6, 0x92, 0x5f, 0x30, 0xcb, 0xd3, 0x81, 0xc1, 0xc3, 0x0d, 0xfe, - 0xd9, 0xa2, 0xab, 0xcb, 0x91, 0x50, 0xd8, 0x6f, 0xbe, 0x05, 0xb2, 0x3b, 0x1e, 0xb2, 0x23, 0xd3, - 0x18, 0x61, 0xcf, 0x18, 0x60, 0x97, 0x5f, 0x2e, 0xf5, 0x82, 0x3b, 0x1e, 0xd2, 0xd3, 0x72, 0x0f, - 0x7b, 0xdb, 0xd8, 0xad, 0x7c, 0x5f, 0x80, 0xfc, 0x5d, 0xdb, 0xb5, 0xc8, 0xb1, 0xd8, 0xf9, 0x6b, - 0x90, 0x1b, 0x99, 0x5e, 0x60, 0xb3, 0x73, 0xf9, 0x54, 0xdc, 0xfa, 0xe3, 0x24, 0xd4, 0x81, 0xec, - 0x31, 0x93, 0x68, 0x46, 0xc6, 0x6d, 0xce, 0x5e, 0x6a, 0x1c, 0x5c, 0x0c, 0x9a, 0xd1, 0x71, 0x13, - 0xe1, 0x94, 0xff, 0x22, 0x89, 0x83, 0xa6, 0x03, 0x85, 0xb0, 0x0d, 0xc0, 0xcd, 0xa7, 0x3d, 0x74, - 0xf5, 0x49, 0x0c, 0x74, 0x1b, 0x40, 0xa8, 0xa2, 0x88, 0x09, 0x86, 0xf8, 0xee, 0x7c, 0x36, 0x53, - 0xd4, 0x18, 0xc8, 0x07, 0xa9, 0x07, 0x5f, 0x5d, 0x91, 0xca, 0x5f, 0x2d, 0x41, 0xba, 0xe9, 0x99, - 0x43, 0x8c, 0x6e, 0x41, 0x6a, 0x48, 0x2c, 0x2c, 0xcc, 0x7d, 0x52, 0x70, 0x26, 0xbb, 0xb1, 0x4b, - 0xac, 0xa8, 0xbc, 0x50, 0x10, 0x74, 0x1b, 0x16, 0xf7, 0xc9, 0xd8, 0xb5, 0xfc, 0x19, 0xad, 0xde, - 0xe3, 0xe1, 0x6a, 0x4c, 0x34, 0x2c, 0x76, 0x1c, 0x08, 0x7d, 0x0a, 0x59, 0x7c, 0xd2, 0x77, 0xc6, - 0x34, 0xe9, 0xd8, 0x36, 0x2c, 0x6e, 0xbd, 0x37, 0x17, 0xaa, 0x12, 0x4a, 0x47, 0x0f, 0x19, 0x21, - 0xa1, 0xfc, 0x83, 0x04, 0x69, 0xa6, 0x94, 0x6a, 0x61, 0xfa, 0x68, 0xc1, 0x14, 0xae, 0x78, 0x6f, - 0x7e, 0xdb, 0x63, 0xe5, 0xf6, 0x0c, 0x8e, 0x1e, 0x0c, 0xb6, 0x1b, 0x18, 0xe4, 0xe0, 0xc0, 0xc7, - 0xbc, 0x7d, 0x0a, 0xbf, 0x68, 0x64, 0x6d, 0x37, 0xd0, 0x18, 0x19, 0x5d, 0x85, 0x3c, 0xdd, 0x15, - 0x56, 0x38, 0x8d, 0xae, 0x34, 0xaf, 0xe7, 0x18, 0x4d, 0x4c, 0xd9, 0x81, 0x1c, 0x67, 0xb2, 0xef, - 0x84, 0xa2, 0x92, 0xcc, 0xf1, 0x31, 0x0d, 0xb8, 0x34, 0xb5, 0xa9, 0xfc, 0x47, 0x09, 0x16, 0xb9, - 0xbb, 0x91, 0x0a, 0x69, 0x3f, 0x30, 0xbd, 0x40, 0x14, 0xd2, 0xad, 0xf9, 0x97, 0x1d, 0x15, 0x18, - 0x0a, 0x83, 0x1a, 0x90, 0xc4, 0xae, 0x25, 0x12, 0xe0, 0x29, 0xd0, 0x74, 0x2a, 0x5e, 0x79, 0x0b, - 0x52, 0x34, 0xbb, 0x68, 0xe7, 0xa5, 0x57, 0xd5, 0x6d, 0x45, 0x5e, 0xa0, 0x1d, 0x02, 0x6b, 0x92, - 0x24, 0xda, 0x21, 0x6c, 0xeb, 0x5a, 0x6f, 0xaf, 0x23, 0x27, 0x2a, 0x5f, 0x42, 0x36, 0xf2, 0x3d, - 0xba, 0x04, 0x17, 0x7b, 0x6a, 0x4d, 0xeb, 0xa9, 0x0d, 0xa5, 0x61, 0xec, 0xe9, 0x4a, 0x5d, 0x69, - 0xb4, 0xd4, 0x6d, 0x79, 0x61, 0x92, 0xd1, 0xd4, 0xda, 0x6d, 0xed, 0x2e, 0x65, 0x48, 0xb4, 0x29, - 0xd1, 0x9a, 0xcd, 0x8e, 0xd2, 0x8d, 0x4d, 0x4f, 0xc4, 0xa8, 0x67, 0x73, 0x93, 0x68, 0x19, 0x72, - 0xf5, 0x9e, 0xae, 0x2b, 0xbc, 0x5b, 0x93, 0x53, 0x95, 0xcf, 0x20, 0x1b, 0x65, 0x17, 0x6d, 0xcc, - 0x54, 0xcd, 0x50, 0x3e, 0xae, 0xb7, 0x7b, 0x9d, 0x96, 0xa6, 0x72, 0xa5, 0x6c, 0xd8, 0x50, 0x8c, - 0xb8, 0x9c, 0x84, 0x2e, 0x40, 0x21, 0x64, 0xb0, 0x75, 0xc8, 0x09, 0x2a, 0x1d, 0x92, 0xba, 0x2d, - 0xa5, 0x23, 0x27, 0xcb, 0xff, 0x48, 0x40, 0x26, 0xac, 0x3b, 0x48, 0x89, 0x75, 0xf2, 0xb9, 0xad, - 0xb7, 0x9f, 0xd4, 0xab, 0xd3, 0x7d, 0xfc, 0xf3, 0x39, 0x89, 0x6a, 0x90, 0x3e, 0xa0, 0xf1, 0x12, - 0xd7, 0xd3, 0x1b, 0xf3, 0xc4, 0x58, 0xe7, 0xa2, 0x68, 0x1d, 0x26, 0x6e, 0x06, 0xac, 0x09, 0x49, - 0x87, 0xf7, 0xa9, 0x89, 0x3b, 0x43, 0x19, 0x32, 0xa6, 0x37, 0xf0, 0x5b, 0xd6, 0x89, 0x5f, 0x5a, - 0x62, 0x55, 0x3d, 0x1a, 0x53, 0x14, 0xfe, 0x8a, 0x2c, 0x50, 0x32, 0xf1, 0x5b, 0x7a, 0x9c, 0xb3, - 0x93, 0xca, 0x24, 0xe4, 0xa4, 0xb8, 0x1c, 0xfc, 0x49, 0x02, 0x38, 0xab, 0x8e, 0xb4, 0xc7, 0xd4, - 0xb5, 0xbb, 0x86, 0xda, 0xdb, 0xad, 0x29, 0xba, 0xc8, 0xb3, 0xaa, 0x7a, 0x8b, 0x77, 0x9f, 0x0d, - 0x45, 0xed, 0x28, 0x06, 0x1b, 0xb3, 0x20, 0x89, 0x86, 0x97, 0x53, 0x92, 0xb4, 0xbd, 0xae, 0xf7, - 0x76, 0x59, 0x5b, 0xdc, 0xe5, 0xf7, 0x04, 0xd6, 0x0c, 0xf3, 0x7b, 0x42, 0xbb, 0xba, 0x2d, 0x2f, - 0x52, 0xb8, 0xb6, 0x52, 0x6d, 0xc8, 0x4b, 0x34, 0x7f, 0x9a, 0x2d, 0xbd, 0xd3, 0x35, 0xee, 0x54, - 0xdb, 0x3d, 0x45, 0xce, 0x50, 0xfc, 0x76, 0x35, 0x1a, 0x67, 0x29, 0x9a, 0xda, 0xbd, 0x29, 0x86, - 0x70, 0xfd, 0x97, 0x50, 0x9c, 0xfc, 0x98, 0x41, 0x13, 0x7f, 0xaf, 0x57, 0x6b, 0xb7, 0xea, 0xf2, - 0x02, 0x7a, 0x05, 0x5e, 0xe2, 0xbf, 0x69, 0xf7, 0xce, 0x2e, 0x38, 0x82, 0x25, 0xd5, 0xde, 0x79, - 0xf0, 0xfd, 0xea, 0xc2, 0x83, 0x87, 0xab, 0xd2, 0xb7, 0x0f, 0x57, 0xa5, 0xef, 0x1e, 0xae, 0x4a, - 0xff, 0x7a, 0xb8, 0x2a, 0xfd, 0xe6, 0xd1, 0xea, 0xc2, 0xb7, 0x8f, 0x56, 0x17, 0xbe, 0x7b, 0xb4, - 0xba, 0xf0, 0x69, 0x2e, 0xf6, 0xcf, 0x00, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff, 0xc0, 0x8c, 0x4b, - 0x98, 0xe7, 0x20, 0x00, 0x00, + 0xf1, 0xe7, 0xe2, 0x41, 0x02, 0x8d, 0x07, 0x47, 0x23, 0xda, 0x82, 0x21, 0x17, 0x45, 0xc1, 0xfe, + 0xdb, 0x94, 0x2c, 0x93, 0x7f, 0x33, 0x29, 0xa7, 0x6c, 0xe7, 0x60, 0x3c, 0x16, 0x14, 0x28, 0x70, + 0x97, 0x5a, 0x00, 0x92, 0x1f, 0x55, 0xde, 0x5a, 0x62, 0x87, 0xe0, 0x5a, 0x8b, 0x5d, 0x70, 0x77, + 0x21, 0x92, 0xfe, 0x02, 0xb9, 0x26, 0x95, 0x4b, 0x4e, 0x29, 0xe7, 0x90, 0xf8, 0x0b, 0xe4, 0xec, + 0x5c, 0x72, 0xd0, 0xd1, 0xa7, 0xc4, 0xa7, 0x54, 0x2c, 0x7f, 0x81, 0xdc, 0x5c, 0xb9, 0xa5, 0xe6, + 0xb1, 0xcb, 0x05, 0x8a, 0x50, 0x04, 0x89, 0xb6, 0x2a, 0x17, 0x16, 0xa6, 0x7b, 0xfa, 0xd7, 0x3d, + 0xdd, 0x3d, 0x3d, 0x3d, 0xb3, 0x84, 0xd7, 0xfd, 0x23, 0x7b, 0x93, 0x9c, 0x90, 0xbe, 0xe5, 0x1c, + 0x78, 0xc6, 0x68, 0x7f, 0x73, 0xe4, 0xb9, 0x7d, 0xe2, 0xfb, 0xae, 0xe7, 0xeb, 0xfe, 0x91, 0xbd, + 0x31, 0xf2, 0xdc, 0xc0, 0xc5, 0xa5, 0xbe, 0xdb, 0x7f, 0xe0, 0xb9, 0x46, 0xff, 0x70, 0x83, 0x12, + 0x4d, 0xcb, 0x0f, 0xfc, 0x23, 0xdb, 0x1b, 0x3b, 0xe5, 0x57, 0xa9, 0xbc, 0x7f, 0x64, 0xef, 0x1b, + 0x3e, 0xd9, 0xf4, 0x03, 0x6f, 0xdc, 0x0f, 0xc6, 0x1e, 0x31, 0xb9, 0x5c, 0xf9, 0x6a, 0x9c, 0xfb, + 0xb9, 0x6b, 0x39, 0x7a, 0x70, 0x3a, 0x22, 0x82, 0xf9, 0x4a, 0x9c, 0x69, 0xbb, 0xfd, 0x07, 0x96, + 0x33, 0x10, 0xac, 0x97, 0x28, 0x8b, 0x4e, 0xf5, 0xf9, 0x5f, 0x41, 0x2e, 0x4f, 0x1b, 0x6b, 0x1a, + 0x81, 0x21, 0x78, 0xff, 0xf7, 0x84, 0x85, 0x50, 0x0d, 0x62, 0xda, 0x4d, 0x3a, 0xcd, 0x1d, 0x05, + 0x9b, 0x96, 0xf3, 0x90, 0x78, 0x01, 0x31, 0xc9, 0xc9, 0xc8, 0xdb, 0xf4, 0x47, 0x86, 0xa3, 0xd3, + 0x5f, 0xc4, 0xf7, 0x2d, 0xd7, 0x11, 0x73, 0x57, 0x06, 0xee, 0xc0, 0x65, 0x3f, 0x37, 0xe9, 0x2f, + 0x4e, 0xad, 0xfc, 0x56, 0x82, 0xe2, 0x3d, 0xc3, 0x1e, 0x13, 0xbf, 0xee, 0x7a, 0xa4, 0x33, 0x22, + 0x7d, 0x5c, 0x87, 0xa5, 0xbe, 0x6b, 0x8f, 0x87, 0x8e, 0x5f, 0x92, 0xd6, 0x92, 0xeb, 0xb9, 0xad, + 0xd7, 0x36, 0x66, 0x39, 0x6c, 0xa3, 0x61, 0x04, 0xe3, 0x61, 0xcb, 0x39, 0x70, 0x6b, 0xa9, 0x47, + 0xff, 0xb8, 0xb6, 0xa0, 0x85, 0x92, 0xf8, 0x2a, 0x64, 0x3d, 0xe3, 0x58, 0xdf, 0x3f, 0x0d, 0x88, + 0x5f, 0x4a, 0xac, 0x25, 0xd7, 0xf3, 0x5a, 0xc6, 0x33, 0x8e, 0x6b, 0x74, 0x8c, 0xaf, 0x41, 0xc6, + 0x19, 0x0f, 0x75, 0xcf, 0x3d, 0xf6, 0x4b, 0xc9, 0x35, 0x69, 0x3d, 0x15, 0x4a, 0x3b, 0xe3, 0xa1, + 0xe6, 0x1e, 0xfb, 0x95, 0x3f, 0xa4, 0x61, 0xb9, 0x6b, 0xec, 0xdb, 0x44, 0x23, 0x86, 0x49, 0x3c, + 0x66, 0x56, 0x0d, 0xd2, 0x01, 0x25, 0x95, 0xa4, 0x35, 0x69, 0x3d, 0xb7, 0xf5, 0xc6, 0x94, 0x51, + 0xc2, 0xf5, 0x1b, 0x4c, 0xac, 0x41, 0xfc, 0xbe, 0x67, 0x8d, 0x02, 0xd7, 0x13, 0xc8, 0x5c, 0x14, + 0x5f, 0x87, 0xac, 0xe5, 0x98, 0xe4, 0x44, 0xb7, 0xcc, 0x93, 0x52, 0x62, 0x4d, 0x5a, 0x2f, 0x08, + 0x7e, 0x86, 0x91, 0x5b, 0xe6, 0x09, 0x5e, 0x85, 0x25, 0x8f, 0x3c, 0x24, 0x9e, 0x4f, 0x98, 0x69, + 0x99, 0xd0, 0x34, 0x41, 0xc4, 0x32, 0xa4, 0xa9, 0x7f, 0xfd, 0x52, 0x8a, 0xf9, 0xe6, 0xc6, 0x6c, + 0xdf, 0x4c, 0x2c, 0xc0, 0x70, 0x42, 0x4b, 0x98, 0x34, 0x7e, 0x0d, 0xc0, 0xb6, 0x86, 0x56, 0xa0, + 0x1f, 0x5a, 0x4e, 0x50, 0x4a, 0xaf, 0x49, 0xeb, 0x49, 0x31, 0x21, 0xcb, 0xe8, 0xb7, 0x2d, 0x27, + 0xa0, 0x7e, 0xb2, 0x7c, 0xbd, 0x7f, 0x48, 0xfa, 0x0f, 0x4a, 0x8b, 0x71, 0x63, 0x2c, 0xbf, 0x4e, + 0x89, 0x58, 0x01, 0x78, 0x68, 0xf9, 0xd6, 0xbe, 0x65, 0x5b, 0xc1, 0x69, 0x69, 0x69, 0x4d, 0x5a, + 0x2f, 0x6e, 0xad, 0xcf, 0xb6, 0xa8, 0xd3, 0x37, 0x9c, 0x7b, 0xd1, 0x7c, 0x01, 0x16, 0x43, 0xc0, + 0x1f, 0xc0, 0x95, 0xa1, 0x71, 0xa2, 0x07, 0xd6, 0x90, 0xf8, 0x81, 0x31, 0x1c, 0xe9, 0xc6, 0x80, + 0xe8, 0x8e, 0xe1, 0xb8, 0x7e, 0x29, 0x1b, 0x8b, 0xd3, 0xca, 0xd0, 0x38, 0xe9, 0x86, 0x73, 0xaa, + 0x03, 0xa2, 0xd0, 0x19, 0xf8, 0x53, 0x40, 0x22, 0xef, 0x75, 0x3f, 0xf0, 0x88, 0x33, 0x08, 0x0e, + 0x4b, 0xc0, 0x4c, 0xba, 0x39, 0x23, 0x56, 0xd4, 0x9e, 0x36, 0x17, 0xe9, 0x08, 0x09, 0xa1, 0x61, + 0xd9, 0x9e, 0x24, 0xe3, 0x7d, 0xb8, 0x1c, 0x82, 0x1f, 0x1b, 0x56, 0xa0, 0x8f, 0x5c, 0xdb, 0xea, + 0x9f, 0x96, 0x72, 0x0c, 0xff, 0xd6, 0x7f, 0xc7, 0xbf, 0x6f, 0x58, 0xc1, 0x1e, 0x93, 0x11, 0x1a, + 0x2e, 0xd9, 0xd3, 0x0c, 0xfc, 0x06, 0xe4, 0x46, 0x86, 0x67, 0xd8, 0x36, 0xb1, 0xad, 0x2f, 0x48, + 0x29, 0x1f, 0xf3, 0x78, 0x9c, 0xb1, 0x93, 0xca, 0x64, 0x50, 0xb6, 0xf2, 0xef, 0x24, 0x94, 0x5a, + 0x34, 0x6b, 0x3a, 0x0f, 0xac, 0xd1, 0x0b, 0x4a, 0xd6, 0x28, 0x19, 0x93, 0xcf, 0x95, 0x8c, 0x93, + 0x69, 0x94, 0x7a, 0xee, 0x34, 0x8a, 0xed, 0xa1, 0xf4, 0x79, 0x7b, 0xe8, 0xbc, 0x4c, 0x59, 0xfc, + 0x91, 0x33, 0x65, 0xe9, 0x02, 0x33, 0xa5, 0xf2, 0x55, 0x1a, 0x8a, 0x3b, 0xae, 0xe5, 0xfc, 0xf4, + 0x11, 0xbf, 0x01, 0x45, 0xdb, 0x75, 0x1f, 0x8c, 0x47, 0x7a, 0x58, 0xa3, 0x69, 0xe8, 0x0b, 0xb5, + 0x04, 0x92, 0xb4, 0x02, 0xe7, 0xd4, 0x45, 0x09, 0xae, 0xc3, 0x92, 0xcb, 0xcf, 0x01, 0x16, 0xd2, + 0xdc, 0xd6, 0xeb, 0xb3, 0x43, 0x2a, 0x47, 0xa7, 0x85, 0xd0, 0xb8, 0xe8, 0x3a, 0x94, 0x86, 0xdf, + 0x83, 0x14, 0x3d, 0xb3, 0x44, 0x78, 0xae, 0xcd, 0x58, 0x15, 0xf5, 0x45, 0xf7, 0x74, 0x44, 0x84, + 0x30, 0x13, 0xb9, 0xf0, 0xe2, 0xf4, 0x1e, 0xbc, 0x3c, 0xb9, 0x74, 0xdd, 0xf0, 0x88, 0xfe, 0x80, + 0x9c, 0x96, 0x32, 0xb1, 0x24, 0xbb, 0x3c, 0xe1, 0x84, 0xaa, 0x47, 0xee, 0x90, 0xd3, 0x73, 0x13, + 0x2e, 0xfb, 0x23, 0x27, 0x1c, 0x5c, 0x64, 0x69, 0x7a, 0x07, 0x2e, 0x0d, 0x0d, 0xcb, 0x09, 0x0c, + 0xcb, 0xd1, 0x5d, 0xcf, 0x24, 0x9e, 0xe5, 0x0c, 0x58, 0xf1, 0x0b, 0x97, 0x8d, 0x42, 0xb6, 0x2a, + 0xb8, 0x3b, 0xa9, 0x4c, 0x1a, 0x2d, 0xd2, 0xf3, 0x1d, 0x3a, 0xae, 0x17, 0x88, 0x2c, 0xbd, 0x0b, + 0xcb, 0xee, 0x38, 0x18, 0x8d, 0x83, 0x33, 0x14, 0x9e, 0xaf, 0x95, 0xd9, 0x81, 0x09, 0x11, 0x85, + 0xa6, 0x22, 0x07, 0x08, 0xa9, 0x78, 0x0b, 0x70, 0x88, 0xa5, 0x0f, 0x8d, 0xa0, 0x7f, 0xa8, 0xdb, + 0xc4, 0x99, 0xc8, 0x5e, 0x14, 0xf2, 0x77, 0x29, 0xbb, 0x4d, 0x9c, 0xca, 0xd7, 0x12, 0xe4, 0x1b, + 0x96, 0x1f, 0x58, 0x4e, 0x3f, 0x60, 0x76, 0xbd, 0x09, 0xcb, 0x6c, 0x12, 0x31, 0xf5, 0x78, 0xef, + 0x51, 0xd0, 0x8a, 0x82, 0x1c, 0x26, 0xf5, 0x0d, 0x40, 0xa6, 0x10, 0x8c, 0x66, 0x26, 0xd8, 0xcc, + 0xe5, 0x90, 0x1e, 0x4e, 0xdd, 0x02, 0xec, 0x8c, 0x6d, 0x9b, 0xa7, 0x49, 0xc8, 0x9c, 0x38, 0xd4, + 0x11, 0xe3, 0x57, 0x3d, 0x12, 0xda, 0x82, 0xdf, 0x80, 0x3c, 0xf1, 0x3c, 0xd7, 0xd3, 0x5d, 0x47, + 0x37, 0xc7, 0x23, 0xb6, 0x71, 0xb2, 0x61, 0x2e, 0x32, 0x8e, 0xea, 0x34, 0xc6, 0xa3, 0x0a, 0x82, + 0xa2, 0xea, 0x99, 0x96, 0x63, 0xd0, 0xcc, 0xa4, 0x2b, 0xa8, 0xfc, 0x2e, 0x09, 0xe8, 0x13, 0x6b, + 0xf0, 0x85, 0x31, 0xa0, 0x9b, 0x41, 0xb8, 0xbb, 0x01, 0x8b, 0x6c, 0x67, 0x87, 0x9d, 0xd4, 0x7c, + 0x55, 0x41, 0xc8, 0xe2, 0x26, 0x00, 0x39, 0x9a, 0x58, 0x6d, 0x6e, 0xeb, 0xfa, 0xec, 0x78, 0x89, + 0xf5, 0x87, 0xed, 0x04, 0x39, 0x3a, 0xf3, 0x5d, 0x91, 0x97, 0x17, 0x97, 0x9b, 0x3e, 0x51, 0x3b, + 0x18, 0x47, 0xac, 0xe9, 0x82, 0x6a, 0xc7, 0x1d, 0xc8, 0x1f, 0x58, 0x27, 0xc4, 0xd4, 0x1f, 0xb2, + 0x06, 0xb3, 0x94, 0x66, 0x96, 0x3f, 0xa1, 0x04, 0x4c, 0x36, 0xa2, 0x5a, 0x8e, 0x49, 0x73, 0xe2, + 0x73, 0x14, 0xa2, 0xca, 0xdf, 0x92, 0xb0, 0xbc, 0x4b, 0xbc, 0x01, 0x89, 0x45, 0x66, 0x17, 0x0a, + 0x36, 0x39, 0x78, 0x8e, 0x6d, 0x90, 0xa7, 0xe2, 0xd1, 0x26, 0x50, 0xa1, 0xe8, 0x59, 0x83, 0xc3, + 0x18, 0x5e, 0x62, 0x4e, 0xbc, 0x02, 0x93, 0x8f, 0x00, 0x63, 0x01, 0x48, 0xbf, 0x88, 0xe2, 0x7d, + 0x03, 0x0a, 0x74, 0x73, 0xe8, 0xe4, 0x68, 0x6c, 0x44, 0xf5, 0x3b, 0xdc, 0x37, 0x79, 0xca, 0x92, + 0x05, 0x07, 0xbf, 0x0f, 0x57, 0x98, 0x2b, 0xcf, 0x72, 0x74, 0x46, 0x61, 0x26, 0x07, 0x81, 0x7c, + 0x34, 0x59, 0x98, 0x7f, 0x09, 0x25, 0xee, 0xb7, 0x73, 0x84, 0xb3, 0x31, 0xe1, 0x15, 0x36, 0x6b, + 0x4a, 0xba, 0xf2, 0x9b, 0x24, 0x14, 0x6f, 0x1b, 0xfe, 0x61, 0x2c, 0xae, 0x37, 0x61, 0x79, 0xca, + 0x18, 0x5e, 0x48, 0xc4, 0x01, 0x19, 0x37, 0x01, 0xdf, 0x02, 0x34, 0xad, 0x9c, 0xd7, 0x12, 0x36, + 0xb9, 0x38, 0xa9, 0xf2, 0x85, 0x47, 0xe4, 0x2d, 0x28, 0x0e, 0x69, 0x12, 0x9f, 0x55, 0xc8, 0x78, + 0x48, 0x0a, 0x9c, 0x17, 0x1a, 0xfb, 0xe2, 0x62, 0xf2, 0x43, 0x02, 0x70, 0x4b, 0xdc, 0x46, 0x63, + 0x71, 0xf9, 0xc9, 0xda, 0xa3, 0xc2, 0x44, 0x8f, 0xc0, 0xca, 0x7d, 0x21, 0xda, 0xb2, 0xb1, 0xd6, + 0x00, 0xab, 0x50, 0x08, 0x6f, 0xcd, 0xcf, 0x5a, 0xe8, 0xf2, 0x21, 0x00, 0x8b, 0xed, 0x0b, 0x4e, + 0x90, 0xca, 0x9f, 0x24, 0x58, 0x09, 0x3d, 0xdf, 0xb4, 0xec, 0x80, 0x78, 0xc2, 0xf7, 0x1b, 0x80, + 0xa2, 0x95, 0xf6, 0x5d, 0x9b, 0xb9, 0x4f, 0x8a, 0xf9, 0xa5, 0x18, 0x72, 0xeb, 0xae, 0x4d, 0x9d, + 0xf8, 0xd9, 0xb4, 0x67, 0x78, 0x2d, 0xfb, 0xd9, 0x94, 0x31, 0xee, 0x28, 0xd8, 0x88, 0xbf, 0x3b, + 0x6c, 0xd0, 0x8b, 0xc5, 0xd9, 0xc2, 0xf6, 0x3c, 0x37, 0x70, 0xcf, 0x73, 0x54, 0xe5, 0x5f, 0x4b, + 0x50, 0xac, 0x0e, 0x06, 0x1e, 0x19, 0x18, 0x81, 0xcb, 0x4d, 0xbc, 0x0e, 0x30, 0xf0, 0x5c, 0x1e, + 0xb6, 0xf8, 0x26, 0xcc, 0x32, 0x6a, 0xdd, 0xb5, 0x7d, 0xfc, 0x19, 0xe4, 0x0d, 0x21, 0x64, 0xb9, + 0xd1, 0x95, 0xe7, 0xe7, 0xb3, 0x7d, 0x3c, 0xa9, 0x22, 0x1a, 0xc6, 0xc2, 0x17, 0xc7, 0xc3, 0xff, + 0x2f, 0xfa, 0x18, 0x62, 0xea, 0x31, 0x53, 0x52, 0x91, 0x29, 0x48, 0x70, 0xb7, 0x23, 0x8b, 0xb6, + 0x45, 0xac, 0xd2, 0x2c, 0x56, 0x6f, 0x3f, 0xb5, 0x25, 0xd3, 0x91, 0x2b, 0xff, 0x2a, 0x01, 0xb9, + 0x98, 0x79, 0x14, 0xf8, 0x60, 0xec, 0xf4, 0x59, 0x90, 0xe6, 0x01, 0x6e, 0x8e, 0x9d, 0x7e, 0x08, + 0x4c, 0x01, 0xf0, 0x1a, 0x64, 0xa2, 0xc6, 0x27, 0x11, 0xdb, 0xba, 0x11, 0x15, 0xbf, 0x0e, 0xc5, + 0x03, 0x96, 0x2b, 0x51, 0x66, 0xd0, 0x6d, 0x50, 0xd0, 0xf2, 0x9c, 0x2a, 0x32, 0xe2, 0x0a, 0x7b, + 0x12, 0x62, 0xec, 0x34, 0x6b, 0xb6, 0x16, 0xfb, 0x9c, 0x71, 0x1b, 0xb2, 0x86, 0x37, 0x18, 0x0f, + 0x89, 0x13, 0xf8, 0xa5, 0x45, 0x16, 0x91, 0x79, 0xb2, 0xfe, 0x4c, 0x78, 0x27, 0x95, 0x49, 0xa2, + 0x54, 0xe5, 0xab, 0x24, 0xa4, 0xe8, 0x2a, 0x30, 0x82, 0x7c, 0x55, 0xf9, 0x58, 0x57, 0xd4, 0xae, + 0xae, 0xf4, 0xda, 0x6d, 0xb4, 0x80, 0x97, 0x20, 0x59, 0xbd, 0xb7, 0x8d, 0x24, 0x9c, 0x87, 0x4c, + 0x4d, 0x55, 0xdb, 0x7a, 0x55, 0x69, 0xa0, 0x04, 0xce, 0xc1, 0x12, 0x1b, 0xa9, 0x1a, 0x4a, 0xe2, + 0x22, 0x40, 0x5d, 0x55, 0xea, 0xd5, 0xae, 0x5e, 0xdd, 0xde, 0x46, 0x29, 0x9c, 0x85, 0x74, 0x5d, + 0xed, 0x29, 0x5d, 0x94, 0xa6, 0xe2, 0xbb, 0xd5, 0x8f, 0xd0, 0x12, 0xfb, 0xd1, 0x52, 0x50, 0x06, + 0x03, 0x2c, 0x76, 0xba, 0x8d, 0x86, 0x7c, 0x0f, 0x65, 0x29, 0xb1, 0xd3, 0xdb, 0x45, 0x40, 0xe1, + 0x3a, 0xbd, 0x5d, 0xbd, 0xa5, 0x74, 0x51, 0x8e, 0x6a, 0xba, 0x57, 0xd5, 0x5a, 0x55, 0xa5, 0x2e, + 0xa3, 0x3c, 0x65, 0x7d, 0xa4, 0x6a, 0x0c, 0xb9, 0xc0, 0x35, 0xf5, 0x94, 0xae, 0xae, 0xa9, 0xf7, + 0x3b, 0xa8, 0xc8, 0xe4, 0xee, 0x6a, 0x8d, 0x56, 0xb3, 0x89, 0x96, 0x31, 0x86, 0x62, 0xb3, 0xa5, + 0x54, 0xdb, 0x7a, 0x24, 0x8d, 0xe8, 0x82, 0x38, 0x4d, 0xe8, 0xbc, 0x84, 0x0b, 0x90, 0xad, 0x6a, + 0x5a, 0xf5, 0x63, 0x86, 0x88, 0xa9, 0xb2, 0x9d, 0x8e, 0xaa, 0xb0, 0xd1, 0x65, 0xca, 0xa4, 0xa3, + 0x1a, 0x1b, 0xae, 0x50, 0x75, 0x9d, 0xae, 0xd6, 0x52, 0xb6, 0xd9, 0xf8, 0x25, 0xb6, 0xea, 0x56, + 0x97, 0xb9, 0xe0, 0x65, 0xba, 0x10, 0x3a, 0x50, 0x35, 0x74, 0x05, 0x67, 0x20, 0x55, 0x57, 0x35, + 0x0d, 0x95, 0x70, 0x09, 0x56, 0xf6, 0x64, 0xad, 0x2e, 0x2b, 0xdd, 0x56, 0x5b, 0xd6, 0x1b, 0xad, + 0x4e, 0x5d, 0x6f, 0xed, 0xee, 0xb5, 0xd1, 0x2b, 0x53, 0x9c, 0xba, 0xaa, 0x74, 0x39, 0xa7, 0x8c, + 0x2f, 0xc3, 0x32, 0xb3, 0x41, 0xad, 0xed, 0xc8, 0x75, 0xee, 0xc4, 0xab, 0x78, 0x05, 0x10, 0x37, + 0x25, 0x46, 0x7d, 0xb5, 0x72, 0x0b, 0x52, 0x34, 0x8f, 0xa9, 0xc2, 0x6a, 0xaf, 0xab, 0xa2, 0x05, + 0xe6, 0xcf, 0x7a, 0xb5, 0x5d, 0xd5, 0x90, 0x44, 0xed, 0x55, 0x54, 0x45, 0x17, 0xe3, 0x44, 0xe5, + 0x2f, 0x8b, 0x70, 0xb5, 0xe5, 0x04, 0xc4, 0xb3, 0x89, 0xf1, 0x90, 0x98, 0xfc, 0xde, 0x1c, 0x3b, + 0x1e, 0x3e, 0x9e, 0x6a, 0x94, 0x3f, 0x98, 0x9d, 0x44, 0x4f, 0x80, 0xe1, 0xc7, 0xc7, 0x54, 0xf7, + 0x1c, 0x7b, 0x8c, 0x48, 0x9c, 0xf7, 0x18, 0x31, 0xf9, 0x12, 0x97, 0x3c, 0xff, 0x25, 0xee, 0x42, + 0xef, 0xd2, 0xe9, 0xf9, 0x0f, 0xff, 0xff, 0xf5, 0x17, 0x93, 0xf2, 0xd7, 0x09, 0x48, 0xb3, 0xe8, + 0xe0, 0x0f, 0x21, 0x65, 0x12, 0xbf, 0xff, 0x4c, 0x8d, 0x00, 0x93, 0x7c, 0x9a, 0x3e, 0xa0, 0x0e, + 0xa9, 0x91, 0xeb, 0xf3, 0x70, 0x3e, 0xf1, 0x5d, 0x6c, 0xcf, 0xf5, 0x83, 0x3d, 0xfe, 0xb6, 0x4e, + 0x33, 0x28, 0xd4, 0x43, 0x85, 0x71, 0x03, 0x32, 0x51, 0x3b, 0x9f, 0x9a, 0xb3, 0x9d, 0x8f, 0x24, + 0xcf, 0xde, 0xe8, 0xd2, 0xcf, 0xf3, 0x46, 0x57, 0xf9, 0xab, 0x04, 0xc5, 0x3d, 0xcf, 0xfd, 0x9c, + 0xf4, 0x83, 0x0e, 0xe1, 0x97, 0xe6, 0x0f, 0x21, 0x4d, 0x33, 0x32, 0xdc, 0x33, 0xf3, 0xa4, 0x24, + 0x17, 0xc4, 0xdb, 0x70, 0x69, 0x40, 0x1c, 0xe2, 0x19, 0x41, 0xac, 0xad, 0xe4, 0x17, 0xcc, 0xf2, + 0x74, 0x60, 0xc8, 0x70, 0x83, 0x7f, 0xbd, 0xe8, 0x6a, 0x28, 0x12, 0x0a, 0xfb, 0xcd, 0x37, 0x01, + 0x39, 0xe3, 0x21, 0x3b, 0x32, 0xf5, 0x11, 0xf1, 0xf4, 0x01, 0x71, 0xf8, 0xe5, 0x52, 0x2b, 0x38, + 0xe3, 0x21, 0x3d, 0x2d, 0xf7, 0x88, 0xb7, 0x4d, 0x9c, 0xca, 0x77, 0x05, 0xc8, 0xdf, 0xb7, 0x1c, + 0xd3, 0x3d, 0x16, 0x3b, 0x7f, 0x8d, 0x3d, 0xba, 0x06, 0x16, 0x3b, 0x97, 0x4f, 0xc5, 0xad, 0x3f, + 0x4e, 0xc2, 0x1d, 0xc8, 0x1e, 0x33, 0x89, 0x66, 0x64, 0xdc, 0xe6, 0xec, 0xa5, 0xc6, 0xc1, 0xc5, + 0xa0, 0x19, 0x1d, 0x37, 0x11, 0x4e, 0xf9, 0xcf, 0x92, 0x38, 0x68, 0x3a, 0x50, 0x08, 0xdb, 0x00, + 0xd2, 0x7c, 0xd6, 0x43, 0x57, 0x9b, 0xc4, 0xc0, 0x77, 0x01, 0x84, 0x2a, 0x8a, 0x98, 0x60, 0x88, + 0xef, 0xcc, 0x67, 0x33, 0x45, 0x8d, 0x81, 0xbc, 0x9f, 0x7a, 0xf4, 0xe5, 0x35, 0xa9, 0xfc, 0xe5, + 0x12, 0xa4, 0x9b, 0x9e, 0x31, 0x24, 0xf8, 0x0e, 0xa4, 0x86, 0xae, 0x49, 0x84, 0xb9, 0x4f, 0x0b, + 0xce, 0x64, 0x37, 0x76, 0x5d, 0x33, 0x2a, 0x2f, 0x14, 0x04, 0xdf, 0x85, 0xc5, 0x7d, 0x77, 0xec, + 0x98, 0xfe, 0x8c, 0x56, 0xef, 0xc9, 0x70, 0x35, 0x26, 0x1a, 0x16, 0x3b, 0x0e, 0x84, 0x3f, 0x81, + 0x2c, 0x39, 0xe9, 0xdb, 0x63, 0x9a, 0x74, 0x6c, 0x1b, 0x16, 0xb7, 0xde, 0x9d, 0x0b, 0x55, 0x0e, + 0xa5, 0xa3, 0x87, 0x8c, 0x90, 0x50, 0xfe, 0x41, 0x82, 0x34, 0x53, 0x4a, 0xb5, 0x30, 0x7d, 0xb4, + 0x60, 0x0a, 0x57, 0xbc, 0x3b, 0xbf, 0xed, 0xb1, 0x72, 0x7b, 0x06, 0x47, 0x0f, 0x06, 0xcb, 0x09, + 0x74, 0xf7, 0xe0, 0xc0, 0x27, 0xbc, 0x7d, 0x0a, 0xbf, 0x7f, 0x64, 0x2d, 0x27, 0x50, 0x19, 0x19, + 0x5f, 0x87, 0x3c, 0xdd, 0x15, 0x66, 0x38, 0x8d, 0xae, 0x34, 0xaf, 0xe5, 0x18, 0x4d, 0x4c, 0xd9, + 0x81, 0x1c, 0x67, 0xb2, 0xcf, 0x85, 0xa2, 0x92, 0xcc, 0xf1, 0x4d, 0x0d, 0xb8, 0x34, 0xb5, 0xa9, + 0xfc, 0x7b, 0x09, 0x16, 0xb9, 0xbb, 0xb1, 0x02, 0x69, 0x3f, 0x30, 0xbc, 0x40, 0x14, 0xd2, 0xad, + 0xf9, 0x97, 0x1d, 0x15, 0x18, 0x0a, 0x83, 0x1b, 0x90, 0x24, 0x8e, 0x29, 0x12, 0xe0, 0x19, 0xd0, + 0x34, 0x2a, 0x5e, 0x79, 0x13, 0x52, 0x34, 0xbb, 0x68, 0xe7, 0xa5, 0x55, 0x95, 0x6d, 0x19, 0x2d, + 0xd0, 0x0e, 0x81, 0x35, 0x49, 0x12, 0xed, 0x10, 0xb6, 0x35, 0xb5, 0xb7, 0xd7, 0x41, 0x89, 0xca, + 0x17, 0x90, 0x8d, 0x7c, 0x8f, 0xaf, 0xc0, 0xe5, 0x9e, 0x52, 0x53, 0x7b, 0x4a, 0x43, 0x6e, 0xe8, + 0x7b, 0x9a, 0x5c, 0x97, 0x1b, 0x2d, 0x65, 0x1b, 0x2d, 0x4c, 0x32, 0x9a, 0x6a, 0xbb, 0xad, 0xde, + 0xa7, 0x0c, 0x89, 0x36, 0x25, 0x6a, 0xb3, 0xd9, 0x91, 0xbb, 0xb1, 0xe9, 0x89, 0x18, 0xf5, 0x6c, + 0x6e, 0x12, 0x2f, 0x43, 0xae, 0xde, 0xd3, 0x34, 0x99, 0x77, 0x6b, 0x28, 0x55, 0xf9, 0x14, 0xb2, + 0x51, 0x76, 0xd1, 0xc6, 0x4c, 0x51, 0x75, 0xf9, 0xa3, 0x7a, 0xbb, 0xd7, 0x69, 0xa9, 0x0a, 0x57, + 0xca, 0x86, 0x0d, 0x59, 0x8f, 0xcb, 0x49, 0xf8, 0x12, 0x14, 0x42, 0x06, 0x5b, 0x07, 0x4a, 0x50, + 0xe9, 0x90, 0xd4, 0x6d, 0xc9, 0x1d, 0x94, 0x2c, 0xff, 0x3d, 0x01, 0x99, 0xb0, 0xee, 0x60, 0x39, + 0xd6, 0xc9, 0xe7, 0xb6, 0xde, 0x7a, 0x5a, 0xaf, 0x4e, 0xf7, 0xf1, 0x17, 0x73, 0x12, 0xd5, 0x20, + 0x7d, 0x40, 0xe3, 0x25, 0xae, 0xa7, 0xb7, 0xe6, 0x89, 0xb1, 0xc6, 0x45, 0xf1, 0x3a, 0x4c, 0xdc, + 0x0c, 0x58, 0x13, 0x92, 0x0e, 0xef, 0x53, 0x13, 0x77, 0x86, 0x32, 0x64, 0x0c, 0x6f, 0xe0, 0xb7, + 0xcc, 0x13, 0xbf, 0xb4, 0xc4, 0xaa, 0x7a, 0x34, 0xa6, 0x28, 0xfc, 0x15, 0x59, 0xa0, 0x64, 0xe2, + 0xb7, 0xf4, 0x38, 0x67, 0x27, 0x95, 0x49, 0xa0, 0xa4, 0xb8, 0x1c, 0xfc, 0x51, 0x02, 0x38, 0xab, + 0x8e, 0xb4, 0xc7, 0xd4, 0xd4, 0xfb, 0xba, 0xd2, 0xdb, 0xad, 0xc9, 0x9a, 0xc8, 0xb3, 0xaa, 0x72, + 0x87, 0x77, 0x9f, 0x0d, 0x59, 0xe9, 0xc8, 0x3a, 0x1b, 0xb3, 0x20, 0x89, 0x86, 0x97, 0x53, 0x92, + 0xb4, 0xbd, 0xae, 0xf7, 0x76, 0x59, 0x5b, 0xdc, 0xe5, 0xf7, 0x04, 0xd6, 0x0c, 0xf3, 0x7b, 0x42, + 0xbb, 0xba, 0x8d, 0x16, 0x29, 0x5c, 0x5b, 0xae, 0x36, 0xd0, 0x12, 0xcd, 0x9f, 0x66, 0x4b, 0xeb, + 0x74, 0xf5, 0x7b, 0xd5, 0x76, 0x4f, 0x46, 0x19, 0x8a, 0xdf, 0xae, 0x46, 0xe3, 0x2c, 0x45, 0x53, + 0xba, 0xb7, 0xc5, 0x10, 0x6e, 0xfe, 0x02, 0x8a, 0x93, 0x1f, 0x33, 0x68, 0xe2, 0xef, 0xf5, 0x6a, + 0xed, 0x56, 0x1d, 0x2d, 0xe0, 0x57, 0xe0, 0x25, 0xfe, 0x9b, 0x76, 0xef, 0xec, 0x82, 0x23, 0x58, + 0x52, 0xed, 0xed, 0x47, 0xdf, 0xad, 0x2e, 0x3c, 0x7a, 0xbc, 0x2a, 0x7d, 0xf3, 0x78, 0x55, 0xfa, + 0xf6, 0xf1, 0xaa, 0xf4, 0xcf, 0xc7, 0xab, 0xd2, 0xaf, 0xbf, 0x5f, 0x5d, 0xf8, 0xe6, 0xfb, 0xd5, + 0x85, 0x6f, 0xbf, 0x5f, 0x5d, 0xf8, 0x24, 0x17, 0xfb, 0x9f, 0x80, 0xff, 0x04, 0x00, 0x00, 0xff, + 0xff, 0x8b, 0x00, 0x06, 0xaf, 0xee, 0x20, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_sql.proto b/pkg/sql/execinfrapb/processors_sql.proto index 0b9703ba17c3..57de66ada668 100644 --- a/pkg/sql/execinfrapb/processors_sql.proto +++ b/pkg/sql/execinfrapb/processors_sql.proto @@ -88,9 +88,15 @@ message TableReaderSpec { // schema changes. optional ScanVisibility visibility = 7 [(gogoproto.nullable) = false]; - // If non-zero, this is a guarantee for the upper bound of rows a TableReader - // will read. If 0, the number of results is unbounded. - optional uint64 max_results = 8 [(gogoproto.nullable) = false]; + // This field used to be an upper bound for the number of rows we will read; + // replaced by the parallelize field. + reserved 8; + + // If set, the TableReader can read all the spans in parallel, without any + // batch limits. This should only be the case when there is a known upper + // bound on the number of rows we can read, and when there is no limit or + // limit hint. + optional bool parallelize = 12 [(gogoproto.nullable) = false]; // If non-zero, this enables inconsistent historical scanning where different // batches can be read with different timestamps. This is used for diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index 88447b5fce70..636693252517 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -74,9 +74,11 @@ func setExplainBundleResult( text = []string{ "Statement diagnostics bundle generated. Download from the Admin UI (Advanced", - "Debug -> Statement Diagnostics History) or use the direct link below.", + "Debug -> Statement Diagnostics History), via the direct link below, or using", + "the command line.", fmt.Sprintf("Admin UI: %s", execCfg.AdminURL()), fmt.Sprintf("Direct link: %s/_admin/v1/stmtbundle/%d", execCfg.AdminURL(), diagID), + "Command line: cockroach statement-diag list / download", } }() diff --git a/pkg/sql/opt/bench/stub_factory.go b/pkg/sql/opt/bench/stub_factory.go index 41dea8469ba2..28cfcbf63920 100644 --- a/pkg/sql/opt/bench/stub_factory.go +++ b/pkg/sql/opt/bench/stub_factory.go @@ -40,7 +40,7 @@ func (f *stubFactory) ConstructScan( hardLimit int64, softLimit int64, reverse bool, - maxResults uint64, + parallelize bool, reqOrdering exec.OutputOrdering, rowCount float64, locking *tree.LockingItem, diff --git a/pkg/sql/opt/exec/execbuilder/builder.go b/pkg/sql/opt/exec/execbuilder/builder.go index 4fa6b5390962..a56f8b9ee054 100644 --- a/pkg/sql/opt/exec/execbuilder/builder.go +++ b/pkg/sql/opt/exec/execbuilder/builder.go @@ -21,6 +21,12 @@ import ( "github.com/cockroachdb/errors" ) +// ParallelScanResultThreshold is the number of results up to which, if the +// maximum number of results returned by a scan is known, the scan disables +// batch limits in the dist sender. This results in the parallelization of these +// scans. +const ParallelScanResultThreshold = 10000 + // Builder constructs a tree of execution nodes (exec.Node) from an optimized // expression tree (opt.Expr). type Builder struct { diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go index 464bd5f3e27b..cb0012564e51 100644 --- a/pkg/sql/opt/exec/execbuilder/relational.go +++ b/pkg/sql/opt/exec/execbuilder/relational.go @@ -492,6 +492,20 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { locking = forUpdateLocking } + parallelize := false + if hardLimit == 0 && softLimit == 0 { + maxResults := b.indexConstraintMaxResults(scan) + if maxResults != 0 && maxResults < ParallelScanResultThreshold { + // Don't set the flag when we have a single span which returns a single + // row: it does nothing in this case except litter EXPLAINs. + // There are still cases where the flag doesn't do anything when the spans + // cover a single range, but there is nothing we can do about that. + if !(maxResults == 1 && scan.Constraint.Spans.Count() == 1) { + parallelize = true + } + } + } + root, err := b.factory.ConstructScan( tab, tab.Index(scan.Index), @@ -502,7 +516,7 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) { softLimit, // HardLimit.Reverse() is taken into account by ScanIsReverse. ordering.ScanIsReverse(scan, &scan.RequiredPhysical().Ordering), - b.indexConstraintMaxResults(scan), + parallelize, res.reqOrdering(scan), rowCount, locking, diff --git a/pkg/sql/opt/exec/execbuilder/testdata/check_constraints b/pkg/sql/opt/exec/execbuilder/testdata/check_constraints index cef2e2c530e1..1feed4b6b558 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/check_constraints +++ b/pkg/sql/opt/exec/execbuilder/testdata/check_constraints @@ -43,7 +43,6 @@ count · · () └── scan · · (a, b, d) · · table t9@primary · · · spans /5/0-/5/1/2 /5/3/1-/5/3/2 · · -· parallel · · · query TTTTT EXPLAIN (VERBOSE) UPDATE t9 SET a = 2 WHERE a = 5 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/select b/pkg/sql/opt/exec/execbuilder/testdata/select index 4af72b20fd4b..fca053f6fba1 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/select +++ b/pkg/sql/opt/exec/execbuilder/testdata/select @@ -1336,21 +1336,6 @@ render · · · table a@p · spans /5.000000000000001-/9.999999999999998/PrefixEnd /20.000000000000004-/39.99999999999999/PrefixEnd -statement ok -SET CLUSTER SETTING sql.parallel_scans.enabled = false - -query TTT -EXPLAIN SELECT * FROM a WHERE a IN (10, 20) ----- -· distribution local -· vectorized true -scan · · -· table a@primary -· spans /10-/10/# /20-/20/# - -statement ok -SET CLUSTER SETTING sql.parallel_scans.enabled = true - query TTT EXPLAIN SELECT * FROM b WHERE (a = 10 AND b = 10) OR (a = 20 AND b = 20) ---- diff --git a/pkg/sql/opt/exec/execbuilder/testdata/select_index b/pkg/sql/opt/exec/execbuilder/testdata/select_index index 655b6c8b5cc2..2462f749d0cf 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/select_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/select_index @@ -1485,7 +1485,6 @@ render · · └── scan · · · table t4@primary · spans /10/20/0-/10/20/1 /10/20/2/1-/10/20/2/2 -· parallel · statement ok SET tracing = on,kv,results; SELECT d FROM t4 WHERE a = 10 and b = 20; SET tracing = off @@ -1510,7 +1509,6 @@ render · · └── scan · · · table t4@primary · spans /10/20/0-/10/20/1 /10/20/2/1-/10/20/3/2 -· parallel · # Optimization should also be applied for updates. query TTT diff --git a/pkg/sql/opt/exec/execbuilder/testdata/sql_fn b/pkg/sql/opt/exec/execbuilder/testdata/sql_fn index a303661672aa..5bed65bee121 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/sql_fn +++ b/pkg/sql/opt/exec/execbuilder/testdata/sql_fn @@ -248,3 +248,8 @@ query T SELECT addgeometrycolumn('a', 'b', 3, NULL, 2); ---- NULL + +query T +SELECT addgeometrycolumn('a', 'b', NULL::string, 'c', 9223372036854775807:::INT8, 'd', NULL::int) +---- +NULL diff --git a/pkg/sql/opt/exec/factory.go b/pkg/sql/opt/exec/factory.go index c4ea4c1e9c33..f4ee182cc11d 100644 --- a/pkg/sql/opt/exec/factory.go +++ b/pkg/sql/opt/exec/factory.go @@ -63,8 +63,10 @@ type Factory interface { // - If softLimit > 0, then the scan may be required to return up to all // of its rows (or up to the hardLimit if it is set), but can be optimized // under the assumption that only softLimit rows will be needed. - // - If maxResults > 0, the scan is guaranteed to return at most maxResults - // rows. + // - If parallelize is true, the scan will scan all spans in parallel. It + // should only be set to true if there is a known upper bound on the + // number of rows that will be scanned. It should not be set if there is + // a hard or soft limit. // - If locking is provided, the scan should use the specified row-level // locking mode. ConstructScan( @@ -76,7 +78,7 @@ type Factory interface { hardLimit int64, softLimit int64, reverse bool, - maxResults uint64, + parallelize bool, reqOrdering OutputOrdering, rowCount float64, locking *tree.LockingItem, diff --git a/pkg/sql/opt/optbuilder/sql_fn.go b/pkg/sql/opt/optbuilder/sql_fn.go index 1027e56c610b..d6981534e0ce 100644 --- a/pkg/sql/opt/optbuilder/sql_fn.go +++ b/pkg/sql/opt/optbuilder/sql_fn.go @@ -62,6 +62,9 @@ func (b *Builder) buildSQLFn( )) } exprs[i] = memo.ExtractConstDatum(info.args[i]) + if exprs[i] == tree.DNull && !info.def.Properties.NullableArgs { + return b.factory.ConstructNull(info.ResolvedType()) + } } // Get the SQL statement and parse it. diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index eee954cbe2c5..1d56b74e4795 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -79,14 +79,14 @@ func (ef *execFactory) ConstructScan( hardLimit int64, softLimit int64, reverse bool, - maxResults uint64, + parallelize bool, reqOrdering exec.OutputOrdering, rowCount float64, locking *tree.LockingItem, ) (exec.Node, error) { if table.IsVirtualTable() { return ef.constructVirtualScan( - table, index, needed, indexConstraint, hardLimit, softLimit, reverse, maxResults, + table, index, needed, indexConstraint, hardLimit, softLimit, reverse, reqOrdering, rowCount, locking, ) } @@ -119,8 +119,7 @@ func (ef *execFactory) ConstructScan( scan.softLimit = softLimit scan.reverse = reverse - scan.maxResults = maxResults - scan.parallelScansEnabled = sqlbase.ParallelScans.Get(&ef.planner.extendedEvalCtx.Settings.SV) + scan.parallelize = parallelize var err error if invertedConstraint != nil { scan.spans, err = GenerateInvertedSpans(invertedConstraint, sb) @@ -154,7 +153,6 @@ func (ef *execFactory) constructVirtualScan( hardLimit int64, softLimit int64, reverse bool, - maxResults uint64, reqOrdering exec.OutputOrdering, rowCount float64, locking *tree.LockingItem, diff --git a/pkg/sql/pgwire/conn.go b/pkg/sql/pgwire/conn.go index eace1e5f3454..879366e22a0f 100644 --- a/pkg/sql/pgwire/conn.go +++ b/pkg/sql/pgwire/conn.go @@ -305,7 +305,6 @@ func (c *conn) serveImpl( var err error var terminateSeen bool - var doingExtendedQueryMessage bool // We need an intSizer, which we're ultimately going to get from the // authenticator once authentication succeeds (because it will actually be a @@ -352,6 +351,12 @@ Loop: } } + // TODO(jordan): there's one big missing piece of implementation here. + // In Postgres, if an error is encountered during extended protocol mode, + // the protocol skips all messages until a Sync is received to "regain + // protocol synchronization". We don't do this. If this becomes a problem, + // we should copy their behavior. + switch typ { case pgwirebase.ClientMsgPassword: // This messages are only acceptable during the auth phase, handled above. @@ -361,17 +366,6 @@ Loop: &c.msgBuilder, &c.writerState.buf) break Loop case pgwirebase.ClientMsgSimpleQuery: - if doingExtendedQueryMessage { - if err = c.stmtBuf.Push( - ctx, - sql.SendError{ - Err: pgwirebase.NewProtocolViolationErrorf( - "SimpleQuery not allowed while in extended protocol mode"), - }, - ); err != nil { - break - } - } if err = c.handleSimpleQuery( ctx, &c.readBuf, timeReceived, intSizer.GetUnqualifiedIntSize(), ); err != nil { @@ -380,23 +374,18 @@ Loop: err = c.stmtBuf.Push(ctx, sql.Sync{}) case pgwirebase.ClientMsgExecute: - doingExtendedQueryMessage = true err = c.handleExecute(ctx, &c.readBuf, timeReceived) case pgwirebase.ClientMsgParse: - doingExtendedQueryMessage = true err = c.handleParse(ctx, &c.readBuf, intSizer.GetUnqualifiedIntSize()) case pgwirebase.ClientMsgDescribe: - doingExtendedQueryMessage = true err = c.handleDescribe(ctx, &c.readBuf) case pgwirebase.ClientMsgBind: - doingExtendedQueryMessage = true err = c.handleBind(ctx, &c.readBuf) case pgwirebase.ClientMsgClose: - doingExtendedQueryMessage = true err = c.handleClose(ctx, &c.readBuf) case pgwirebase.ClientMsgTerminate: @@ -404,7 +393,6 @@ Loop: break Loop case pgwirebase.ClientMsgSync: - doingExtendedQueryMessage = false // We're starting a batch here. If the client continues using the extended // protocol and encounters an error, everything until the next sync // message has to be skipped. See: @@ -413,7 +401,6 @@ Loop: err = c.stmtBuf.Push(ctx, sql.Sync{}) case pgwirebase.ClientMsgFlush: - doingExtendedQueryMessage = true err = c.handleFlush(ctx) case pgwirebase.ClientMsgCopyData, pgwirebase.ClientMsgCopyDone, pgwirebase.ClientMsgCopyFail: diff --git a/pkg/sql/pgwire/testdata/pgtest/pgjdbc b/pkg/sql/pgwire/testdata/pgtest/pgjdbc new file mode 100644 index 000000000000..864572d153e6 --- /dev/null +++ b/pkg/sql/pgwire/testdata/pgtest/pgjdbc @@ -0,0 +1,18 @@ +# Send a simple query in the middle of extended protocol, which is apparently +# allowed. (See #41511, #33693) +send +Parse {"Name": "S_3", "Query": "BEGIN"} +Bind {"PreparedStatement": "S_3"} +Execute +Query {"String": "SAVEPOINT PGJDBC_AUTOSAVE"} +---- + +until +CommandComplete +ReadyForQuery +---- +{"Type":"ParseComplete"} +{"Type":"BindComplete"} +{"Type":"CommandComplete","CommandTag":"BEGIN"} +{"Type":"CommandComplete","CommandTag":"SAVEPOINT"} +{"Type":"ReadyForQuery","TxStatus":"T"} diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index cb2f54b455c2..8e7dbda012c9 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -313,17 +313,6 @@ func NewDatumRowConverter( c.Datums = append(c.Datums, nil) } else { if !isTargetCol(col) && col.DefaultExpr != nil { - // Check if the default expression is a constant expression as we do not - // support non-constant default expressions for non-target columns in IMPORT INTO. - // - // TODO (anzoteh96): add support to non-constant default expressions. Perhaps - // we can start with those with Stable volatility, like now(). - if !tree.IsConst(evalCtx, defaultExprs[i]) { - return nil, errors.Newf( - "non-constant default expression %s for non-targeted column %q is not supported by IMPORT INTO", - defaultExprs[i].String(), - col.Name) - } // Placeholder for columns with default values that will be evaluated when // each import row is being created. c.Datums = append(c.Datums, nil) @@ -385,6 +374,17 @@ func (c *DatumRowConverter) Row(ctx context.Context, sourceID int32, rowIndex in for i := range c.cols { col := &c.cols[i] if _, ok := c.IsTargetCol[i]; !ok && !col.Hidden && col.DefaultExpr != nil { + if !tree.IsConst(c.EvalCtx, c.defaultExprs[i]) { + // Check if the default expression is a constant expression as we do not + // support non-constant default expressions for non-target columns in IMPORT INTO. + // + // TODO (anzoteh96): add support to non-constant default expressions. Perhaps + // we can start with those with Stable volatility, like now(). + return errors.Newf( + "non-constant default expression %s for non-targeted column %q is not supported by IMPORT INTO", + c.defaultExprs[i].String(), + col.Name) + } datum, err := c.defaultExprs[i].Eval(c.EvalCtx) if err != nil { return errors.Wrapf(err, "error evaluating default expression for IMPORT INTO") diff --git a/pkg/sql/rowexec/tablereader.go b/pkg/sql/rowexec/tablereader.go index cf0b4e6b98e6..aa9d23d21264 100644 --- a/pkg/sql/rowexec/tablereader.go +++ b/pkg/sql/rowexec/tablereader.go @@ -35,12 +35,9 @@ import ( type tableReader struct { execinfra.ProcessorBase - spans roachpb.Spans - limitHint int64 - - // maxResults is non-zero if there is a limit on the total number of rows - // that the tableReader will read. - maxResults uint64 + spans roachpb.Spans + limitHint int64 + parallelize bool // See TableReaderSpec.MaxTimestampAgeNanos. maxTimestampAge time.Duration @@ -86,7 +83,9 @@ func newTableReader( tr := trPool.Get().(*tableReader) tr.limitHint = execinfra.LimitHint(spec.LimitHint, post) - tr.maxResults = spec.MaxResults + // Parallelize shouldn't be set when there's a limit hint, but double-check + // just in case. + tr.parallelize = spec.Parallelize && tr.limitHint == 0 tr.maxTimestampAge = time.Duration(spec.MaxTimestampAgeNanos) returnMutations := spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic @@ -157,7 +156,7 @@ func (tr *tableReader) Start(ctx context.Context) context.Context { ctx = tr.StartInternal(ctx, tableReaderProcName) - limitBatches := execinfra.ScanShouldLimitBatches(tr.maxResults, tr.limitHint, tr.FlowCtx) + limitBatches := !tr.parallelize log.VEventf(ctx, 1, "starting scan with limitBatches %t", limitBatches) var err error if tr.maxTimestampAge == 0 { diff --git a/pkg/sql/rowexec/version_history.txt b/pkg/sql/rowexec/version_history.txt index 357ab2bc8999..6ae9619b7dc1 100644 --- a/pkg/sql/rowexec/version_history.txt +++ b/pkg/sql/rowexec/version_history.txt @@ -116,3 +116,7 @@ - The Sampler and SampleAggregator specs for optimizer statistics have added fields for inverted sketches. Two new row types are produced by Samplers, which are not backward compatible with the previous version. +- Version: 31 (MinAcceptedVersion: 30) + - The TableReader field MaxResults was retired in favor of the new field + Parallelize. The change is backwards compatible (mixed versions will + prevent parallelization). diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index a89d103bd01f..4beeb7bbb316 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -85,8 +85,8 @@ type scanNode struct { disableBatchLimits bool - // Should be set to true if sqlbase.ParallelScans is true. - parallelScansEnabled bool + // See exec.Factory.ConstructScan. + parallelize bool // Is this a full scan of an index? isFull bool @@ -95,10 +95,6 @@ type scanNode struct { // only true when running SCRUB commands. isCheck bool - // maxResults, if greater than 0, is the maximum number of results that a - // scan is guaranteed to return. - maxResults uint64 - // estimatedRowCount is the estimated number of rows that this scanNode will // output. When there are no statistics to make the estimation, it will be // set to zero. @@ -181,20 +177,6 @@ func (n *scanNode) disableBatchLimit() { n.softLimit = 0 } -// canParallelize returns true if this scanNode can be parallelized at the -// distSender level safely. -func (n *scanNode) canParallelize() bool { - // We choose only to parallelize if we are certain that no more than - // ParallelScanResultThreshold results will be returned, to prevent potential - // memory blowup. - // We can't parallelize if we have a non-zero limit hint, since DistSender - // is limited to running limited batches serially. - return n.maxResults != 0 && - n.maxResults < execinfra.ParallelScanResultThreshold && - n.limitHint() == 0 && - n.parallelScansEnabled -} - func (n *scanNode) limitHint() int64 { var limitHint int64 if n.hardLimit != 0 { diff --git a/pkg/sql/sqlbase/settings.go b/pkg/sql/sqlbase/settings.go deleted file mode 100644 index 223a79dd9aca..000000000000 --- a/pkg/sql/sqlbase/settings.go +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2018 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 sqlbase - -import "github.com/cockroachdb/cockroach/pkg/settings" - -// ParallelScans controls parallelizing multi-range scans when the maximum size -// of the result set is known. -var ParallelScans = settings.RegisterBoolSetting( - "sql.parallel_scans.enabled", - "parallelizes scanning different ranges when the maximum result size can be deduced", - true, -) diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index 030f32257a18..08e647ef720a 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -181,7 +181,7 @@ func (v *planVisitor) visitInternal(plan planNode, name string) { // we know we will get only one result from the scan. There are cases // in which "parallel" will be printed out even though the spans cover // a single range, but there is nothing we can do about that. - if n.canParallelize() && (len(n.spans) > 1 || n.maxResults > 1) { + if n.parallelize { v.observer.attr(name, "parallel", "") } if n.index.IsPartial() {