Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
68282: cli,log: allow use of `debug merge-logs` on older logs r=knz a=ajwerner

Fixes [#68278](#68278).

Log parsers require the flag `--format` when parsing older logs (because 
they do not contain format specification). With this patch, this is no longer 
a requirement as the log format is now inferred based on the structure of 
the log if no log format specification exists.

Release justification: bug fix

Release note (bug fix): The debug merge-logs command no longer returns an error 
when the log decoder attempts to parse older logs.

69903: importccl: add support for IMPORT INTO RBR table r=arulajmani,ajstorm,dt a=adityamaru

This change overrides the `default_to_database_primary_region`
and `gateway_region` to always return the primary region of the
database of the table being imported into. This allows for
IMPORT INTO an RBR table.

To ensure that the import is idempotent across resumptions, we cache
the primary region of the database being imported into, during planning.
This information is store in the job details and flow spec to be used
when evaluating the relevant default expr/computed column.

Since IMPORT is a job, it does not have an associated session data
and so it cannot rely on the planners' implementation of the regional
operator. This change also implements the relevant methods in the
`importRegionOperator` to allow resolution of the primary region
of the database being imported into.

Fixes: #69616

Release note (sql change): IMPORT INTO regional by row tables
is supported.

Release justification: fixes for high-priority or high-severity bugs in existing functionality

70150: server: fix TestAdminAPIJobs failure r=knz a=adityamaru

This change sorts the expected job IDs before ensuring
that they are equal.

Fixes: #69401

Release note: None

70226: changefeedccl: updated retryable error warning message r=wongio123 a=wongio123

Retryable error warning message contained the word "error"
Confusing to users because warning message had the word "error" in it
Prefaced warning message with "WARNING"

Release note (enterprise change): updated retyable error warning message to begin with "WARNING"

Closes #69677 

70229: [CRDB-9016] ui: fix drag to zoom on custom charts r=Santamaura a=Santamaura

This PR addresses the issue where a user creates a custom chart and selects an area to zoom into which leaves the grey highlight after the graph zooms in. This was due to the history prop not being passed into the linegraph component and caused an error to throw when updating the url params. This was resolved by passing in the history to propagate to the
linegraph component.

Release note (ui change): fix drag to zoom on custom charts

https://user-images.githubusercontent.com/17861665/133342585-d7b37e9b-7eb8-4a48-b2c5-814fed62556a.mp4



70262: ui: add column selector to transation page r=maryliag a=maryliag

Add column selector to Transaction Page

Fixes #70148

<img width="414" alt="Screen Shot 2021-09-15 at 11 28 56 AM" src="https://user-images.githubusercontent.com/1017486/133463202-7ed7ac3a-9614-4101-ad76-8f431defe688.png">


Release justification: Category 4
Release note (ui change): Add column selector to transaction page

70268: clusterversion: remove Start21_1 (no longer applicable) r=irfansharif a=irfansharif

Fixes #65200. The last remaining 21.1 version (V21_1) can be removed as
part of #69828.

Release note: None

Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Alex Wong <[email protected]>
Co-authored-by: Santamaura <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
Co-authored-by: irfan sharif <[email protected]>
  • Loading branch information
7 people committed Sep 15, 2021
8 parents e4974b1 + 68ef2d0 + fb180a9 + ca68a2e + 68bb9a9 + bd029e1 + 662c466 + 887e84a commit f278e64
Show file tree
Hide file tree
Showing 34 changed files with 1,204 additions and 704 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -734,7 +734,7 @@ func (b *changefeedResumer) resumeWithRetries(
return err
}

log.Warningf(ctx, `CHANGEFEED job %d encountered retryable error: %v`, jobID, err)
log.Warningf(ctx, `WARNING: CHANGEFEED job %d encountered retryable error: %v`, jobID, err)
b.setJobRunningStatus(ctx, "retryable error: %s", err)
if metrics, ok := execCfg.JobRegistry.MetricsStruct().Changefeed.(*Metrics); ok {
metrics.ErrorRetries.Inc(1)
Expand Down
42 changes: 29 additions & 13 deletions pkg/ccl/importccl/import_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -974,6 +974,24 @@ func importPlanHook(
}
}

// Store the primary region of the database being imported into. This is
// used during job execution to evaluate certain default expressions and
// computed columns such as `gateway_region`.
var databasePrimaryRegion descpb.RegionName
if db.IsMultiRegion() {
if err := sql.DescsTxn(ctx, p.ExecCfg(), func(ctx context.Context, txn *kv.Txn,
descsCol *descs.Collection) error {
regionConfig, err := sql.SynthesizeRegionConfig(ctx, txn, db.GetID(), descsCol)
if err != nil {
return err
}
databasePrimaryRegion = regionConfig.PrimaryRegion()
return nil
}); err != nil {
return errors.Wrap(err, "failed to resolve region config for multi region database")
}
}

telemetry.CountBucketed("import.files", int64(len(files)))

// Record telemetry for userfile being used as the import target.
Expand Down Expand Up @@ -1002,16 +1020,17 @@ func importPlanHook(
// StartableJob which we attached to the connExecutor somehow.

importDetails := jobspb.ImportDetails{
URIs: files,
Format: format,
ParentID: db.GetID(),
Tables: tableDetails,
Types: typeDetails,
SSTSize: sstSize,
Oversample: oversample,
SkipFKs: skipFKs,
ParseBundleSchema: importStmt.Bundle,
DefaultIntSize: p.SessionData().DefaultIntSize,
URIs: files,
Format: format,
ParentID: db.GetID(),
Tables: tableDetails,
Types: typeDetails,
SSTSize: sstSize,
Oversample: oversample,
SkipFKs: skipFKs,
ParseBundleSchema: importStmt.Bundle,
DefaultIntSize: p.SessionData().DefaultIntSize,
DatabasePrimaryRegion: databasePrimaryRegion,
}

jr := jobs.Record{
Expand Down Expand Up @@ -1295,9 +1314,6 @@ func prepareExistingTableDescForIngestion(
if len(desc.Mutations) > 0 {
return nil, errors.Errorf("cannot IMPORT INTO a table with schema changes in progress -- try again later (pending mutation %s)", desc.Mutations[0].String())
}
if desc.LocalityConfig != nil && desc.LocalityConfig.GetRegionalByRow() != nil {
return nil, unimplemented.NewWithIssueDetailf(61133, "import.regional-by-row", "IMPORT into REGIONAL BY ROW table not supported")
}

// Note that desc is just used to verify that the version matches.
importing, err := descsCol.GetMutableTableVersionByID(ctx, desc.ID, txn)
Expand Down
43 changes: 37 additions & 6 deletions pkg/ccl/importccl/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6429,6 +6429,14 @@ func TestImportMultiRegion(t *testing.T) {

simpleOcf := fmt.Sprintf("nodelocal://0/avro/%s", "simple.ocf")

var data string
srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
if r.Method == "GET" {
_, _ = w.Write([]byte(data))
}
}))
defer srv.Close()

// Table schemas for USING
tableSchemaMR := fmt.Sprintf("nodelocal://0/avro/%s", "simple-schema-multi-region.sql")
tableSchemaMRRegionalByRow := fmt.Sprintf("nodelocal://0/avro/%s",
Expand Down Expand Up @@ -6498,6 +6506,7 @@ DROP VIEW IF EXISTS v`,
create string
args []interface{}
errString string
data string
}{
{
name: "import-create-using-multi-region-to-non-multi-region-database",
Expand All @@ -6523,16 +6532,34 @@ DROP VIEW IF EXISTS v`,
errString: "IMPORT to REGIONAL BY ROW table not supported",
},
{
name: "import-into-multi-region-regional-by-row-to-multi-region-database",
name: "import-into-multi-region-regional-by-row-default-col-to-multi-region-database",
db: "multi_region",
table: "mr_regional_by_row",
create: "CREATE TABLE mr_regional_by_row (i INT8 PRIMARY KEY, s text, b bytea) LOCALITY REGIONAL BY ROW",
sql: "IMPORT INTO mr_regional_by_row AVRO DATA ($1)",
args: []interface{}{simpleOcf},
},
{
name: "import-into-multi-region-regional-by-row-to-multi-region-database",
db: "multi_region",
table: "mr_regional_by_row",
create: "CREATE TABLE mr_regional_by_row (i INT8 PRIMARY KEY, s text, b bytea) LOCALITY REGIONAL BY ROW",
sql: "IMPORT INTO mr_regional_by_row (i, s, b, crdb_region) CSV DATA ($1)",
args: []interface{}{srv.URL},
data: "1,\"foo\",NULL,us-east1\n",
},
{
name: "import-into-multi-region-regional-by-row-to-multi-region-database-wrong-value",
db: "multi_region",
table: "mr_regional_by_row",
create: "CREATE TABLE mr_regional_by_row (i INT8 PRIMARY KEY, s text, b bytea) LOCALITY REGIONAL BY ROW",
sql: "IMPORT INTO mr_regional_by_row AVRO DATA ($1)",
args: []interface{}{simpleOcf},
errString: "IMPORT into REGIONAL BY ROW table not supported",
sql: "IMPORT INTO mr_regional_by_row (i, s, b, crdb_region) CSV DATA ($1)",
args: []interface{}{srv.URL},
data: "1,\"foo\",NULL,us-west1\n",
errString: "invalid input value for enum crdb_internal_region",
},
{
name: "import-into-using-multi-region-global-to-multi-region-database",
name: "import-into-multi-region-global-to-multi-region-database",
db: "multi_region",
table: "mr_global",
create: "CREATE TABLE mr_global (i INT8 PRIMARY KEY, s text, b bytea) LOCALITY GLOBAL",
Expand All @@ -6549,14 +6576,18 @@ DROP VIEW IF EXISTS v`,
_, err = sqlDB.Exec(fmt.Sprintf("DROP TABLE IF EXISTS %q CASCADE", test.table))
require.NoError(t, err)

if test.data != "" {
data = test.data
}

if test.create != "" {
_, err = sqlDB.Exec(test.create)
require.NoError(t, err)
}

_, err = sqlDB.ExecContext(context.Background(), test.sql, test.args...)
if test.errString != "" {
testutils.IsError(err, test.errString)
require.True(t, testutils.IsError(err, test.errString))
} else {
require.NoError(t, err)
res := sqlDB.QueryRow(fmt.Sprintf("SELECT count(*) FROM %q", test.table))
Expand Down
31 changes: 28 additions & 3 deletions pkg/ccl/importccl/import_table_creation.go
Original file line number Diff line number Diff line change
Expand Up @@ -205,7 +205,7 @@ func MakeSimpleTableDescriptor(
evalCtx := tree.EvalContext{
Context: ctx,
Sequence: &importSequenceOperators{},
Regions: &importRegionOperator{},
Regions: makeImportRegionOperator(""),
SessionDataStack: sessiondata.NewStack(&sessiondata.SessionData{}),
ClientNoticeSender: &faketreeeval.DummyClientNoticeSender{},
Settings: st,
Expand Down Expand Up @@ -264,13 +264,38 @@ var (
)

// Implements the tree.RegionOperator interface.
type importRegionOperator struct{}
type importRegionOperator struct {
primaryRegion descpb.RegionName
}

func makeImportRegionOperator(primaryRegion descpb.RegionName) *importRegionOperator {
return &importRegionOperator{primaryRegion: primaryRegion}
}

// importDatabaseRegionConfig is a stripped down version of
// multiregion.RegionConfig that is used by import.
type importDatabaseRegionConfig struct {
primaryRegion descpb.RegionName
}

// IsValidRegionNameString implements the tree.DatabaseRegionConfig interface.
func (i importDatabaseRegionConfig) IsValidRegionNameString(_ string) bool {
// Unimplemented.
return false
}

// PrimaryRegionString implements the tree.DatabaseRegionConfig interface.
func (i importDatabaseRegionConfig) PrimaryRegionString() string {
return string(i.primaryRegion)
}

var _ tree.DatabaseRegionConfig = &importDatabaseRegionConfig{}

// CurrentDatabaseRegionConfig is part of the tree.EvalDatabase interface.
func (so *importRegionOperator) CurrentDatabaseRegionConfig(
_ context.Context,
) (tree.DatabaseRegionConfig, error) {
return nil, errors.WithStack(errRegionOperator)
return importDatabaseRegionConfig{primaryRegion: so.primaryRegion}, nil
}

// ValidateAllMultiRegionZoneConfigsInCurrentDatabase is part of the tree.EvalDatabase interface.
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/importccl/read_import_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ func runImport(
// TODO(adityamaru): Should we just plumb the flowCtx instead of this
// assignment.
evalCtx.DB = flowCtx.Cfg.DB
evalCtx.Regions = makeImportRegionOperator(spec.DatabasePrimaryRegion)
semaCtx := tree.MakeSemaContext()
semaCtx.TypeResolver = importResolver
conv, err := makeInputConverter(ctx, &semaCtx, spec, evalCtx, kvCh, seqChunkProvider)
Expand Down
8 changes: 8 additions & 0 deletions pkg/cli/debug_merge_logs.go
Original file line number Diff line number Diff line change
Expand Up @@ -502,6 +502,14 @@ func (s *fileLogStream) open() bool {
if s.f, s.err = os.Open(s.fi.path); s.err != nil {
return false
}
if s.format == "" {
if _, s.format, s.err = log.ReadFormatFromLogFile(s.f); s.err != nil {
return false
}
if _, s.err = s.f.Seek(0, io.SeekStart); s.err != nil {
return false
}
}
if s.err = seekToFirstAfterFrom(s.f, s.from, s.editMode, s.format); s.err != nil {
return false
}
Expand Down
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
I210801 21:05:59.364923 1 util/log/sync_buffer.go:70 [config] binary: CockroachDB CCL v20.1.17 (x86_64-apple-darwin14, built 2021/05/17 16:30:22,
I210801 21:05:59.364923 1 util/log/sync_buffer.go:70 [config] arguments: [./cockroach start]
I210801 21:05:59.364923 1 util/log/sync_buffer.go:70 line format: [IWEF]yymmdd hh:mm:ss.uuuuuu goid file:line msg utf8=✓
{"header":1,"timestamp":"1631568932.530915000","goroutine":1,"file":"util/log/file_sync_buffer.go","line":238,"redactable":1,"tags":{"config":""},"message":"file created at: 2021/09/13 21:35:32"}
{"header":1,"timestamp":"1631568932.530923000","goroutine":1,"file":"util/log/file_sync_buffer.go","line":238,"redactable":1,"tags":{"config":""},"message":"running on machine: "}
{"header":1,"timestamp":"1631568932.530929000","goroutine":1,"file":"util/log/file_sync_buffer.go","line":238,"redactable":1,"tags":{"config":""},"message":"binary: CockroachDB CCL v21.2.0-alpha.00000000-4019-g6d1becda18-dirty (x86_64-apple-darwin20.6.0, built , go1.16.6)"}
{"header":1,"timestamp":"1631568932.530934000","goroutine":1,"file":"util/log/file_sync_buffer.go","line":238,"redactable":1,"tags":{"config":""},"message":"arguments: []"}
{"header":1,"timestamp":"1631568932.530945000","goroutine":1,"file":"util/log/file_sync_buffer.go","line":238,"redactable":1,"tags":{"config":""},"message":"log format (utf8=✓): json"}
17 changes: 7 additions & 10 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,10 +155,8 @@ const (
_ Key = iota - 1 // want first named one to start at zero

// v21.1 versions.
//
// Start21_1 demarcates work towards CockroachDB v21.1.
Start21_1
// V21_1 is CockroachDB v21.1. It's used for all v21.1.x patch releases.
// TODO(irfansharif): This can be removed as part of once #69828 lands (bumping the min cluster version.
V21_1

// v21.1PLUS release. This is a special v21.1.x release with extra changes,
Expand Down Expand Up @@ -251,6 +249,7 @@ const (
// PebbleFormatVersioned ratchets Pebble's format major version to
// the version FormatVersioned.
PebbleFormatVersioned

// *************************************************
// Step (1): Add new versions here.
// Do not add new versions to a patch release.
Expand All @@ -275,18 +274,15 @@ const (
// minor version until we are absolutely sure that no new migrations will need
// to be added (i.e., when cutting the final release candidate).
var versionsSingleton = keyedVersions{

// v21.1 versions. Internal versions defined here-on-forth must be even.
{
Key: Start21_1,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 2},
},
// v21.1 versions.
{
// V21_1 is CockroachDB v21.1. It's used for all v21.1.x patch releases.
Key: V21_1,
Version: roachpb.Version{Major: 21, Minor: 1},
},

// Internal versions must be even.

// v21.1PLUS version. This is a special v21.1.x release with extra changes,
// used internally for the 2021 Serverless offering.
//
Expand Down Expand Up @@ -436,7 +432,8 @@ var (
// binaryMinSupportedVersion is the earliest version of data supported by
// this binary. If this binary is started using a store marked with an older
// version than binaryMinSupportedVersion, then the binary will exit with
// an error.
// an error. This typically trails the current release by one (see top-level
// comment).
binaryMinSupportedVersion = ByKey(V21_1)

// binaryVersion is the version of this binary.
Expand Down
71 changes: 35 additions & 36 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Loading

0 comments on commit f278e64

Please sign in to comment.