Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
80474: gcp: make per-chunk retry upload timeout configurable r=dt a=adityamaru

This change adds a cluster setting `cloudstorage.gs.chunking.retry_timeout`
that can be used to change the default per-chunk retry timeout
that GCS imposes when chunking of file upload is enabled. The default
value is set to 60 seconds, which is double of the default google sdk
value of 30s.

This change was motivated by sporadic occurrences of a 503 service unavailable
error during backups. On its own this change is not expected to solve the
resiliency issues of backup when the upload service is unavailable, but it
is nice to have configurable setting nonetheless.

Release note (sql change): `cloudstorage.gs.chunking.retry_timeout`
is a cluster setting that can be used to configure the per-chunk retry
timeout of files to Google Cloud Storage. The default value is 60 seconds.

86696: sql/gcjob: make index GC robust to descriptors being deleted r=ajwerner a=ajwerner

First commit is #86690

If the descriptor was deleted, the GC job should exit gracefully.

Fixes #86340

Release justification: bug fix for backport

Release note (bug fix): In some scenarios, when a DROP INDEX was
run around the same time as a DROP TABLE or DROP DATABASE covering the same
data, the `DROP INDEX` gc job could get caught retrying indefinitely. This
has been fixed.

87378: builtins: stream consistency checker output r=pavelkalinnikov a=tbg

Also makes it resilient to per-Range errors, which now no longer
tank the entire operation.

```sql
-- avoid buffering in cli
\set display_format=csv;
-- avoid rows getting buffered at server
set avoid_buffering=true;
-- compute as fast as possible
SET CLUSTER SETTING server.consistency_check.max_rate = '1tb';

SELECT * FROM crdb_internal.check_consistency(false, '', '');
```

Release justification: improvement for a debugging-related feature
Release note: None


87657: Makefile: always use `cockroach-short` for file generation r=knz a=rickystewart

This defaults to the full `cockroach` executable which requires pulling in all the UI stuff. Use `cockroach-short` to make generation require fewer dependencies.

Release note: None

87701: testccl/workload/schemachange: skip random schema test r=ajwerner a=ajwerner

This is very flakey. Some of it is due to #87672. Some of it was due to #85677. There are some issues with inserts which need to be fixed. Until this stabilizes, it's causing problems.

Along the way, I'm marking #78478 as a GA blocker so we do actually fix it.

Release note: None

Co-authored-by: Aditya Maru <adityamaru@gmail.com>
Co-authored-by: Andrew Werner <awerner32@gmail.com>
Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
Co-authored-by: Ricky Stewart <ricky@cockroachlabs.com>
  • Loading branch information
5 people committed Sep 9, 2022
6 parents 2b5d87f + ac2387e + ed2e090 + 19c9524 + b050bab + 5c363f7 commit aea3285
Show file tree
Hide file tree
Showing 12 changed files with 289 additions and 40 deletions.
10 changes: 4 additions & 6 deletions Makefile
Expand Up @@ -997,13 +997,11 @@ $(go-targets): override LINKFLAGS += \
$(COCKROACH) $(COCKROACHOSS) go-install: override LINKFLAGS += \
-X "github.com/cockroachdb/cockroach/pkg/build.utcTime=$(shell date -u '+%Y/%m/%d %H:%M:%S')"

settings-doc-gen = $(if $(filter buildshort,$(MAKECMDGOALS)),$(COCKROACHSHORT),$(COCKROACH))
docs/generated/settings/settings.html: $(COCKROACHSHORT)
@$(COCKROACHSHORT) gen settings-list --format=rawhtml > $@

docs/generated/settings/settings.html: $(settings-doc-gen)
@$(settings-doc-gen) gen settings-list --format=rawhtml > $@

docs/generated/settings/settings-for-tenants.txt: $(settings-doc-gen)
@$(settings-doc-gen) gen settings-list --without-system-only > $@
docs/generated/settings/settings-for-tenants.txt: $(COCKROACHSHORT)
@$(COCKROACHSHORT) gen settings-list --without-system-only > $@

SETTINGS_DOC_PAGES := docs/generated/settings/settings.html docs/generated/settings/settings-for-tenants.txt

Expand Down
2 changes: 1 addition & 1 deletion docs/generated/sql/functions.md
Expand Up @@ -3018,7 +3018,7 @@ may increase either contention or retry errors, or both.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="crdb_internal.assignment_cast"></a><code>crdb_internal.assignment_cast(val: anyelement, type: anyelement) &rarr; anyelement</code></td><td><span class="funcdesc"><p>This function is used internally to perform assignment casts during mutations.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="crdb_internal.check_consistency"></a><code>crdb_internal.check_consistency(stats_only: <a href="bool.html">bool</a>, start_key: <a href="bytes.html">bytes</a>, end_key: <a href="bytes.html">bytes</a>) &rarr; tuple{int AS range_id, bytes AS start_key, string AS start_key_pretty, string AS status, string AS detail}</code></td><td><span class="funcdesc"><p>Runs a consistency check on ranges touching the specified key range. an empty start or end key is treated as the minimum and maximum possible, respectively. stats_only should only be set to false when targeting a small number of ranges to avoid overloading the cluster. Each returned row contains the range ID, the status (a roachpb.CheckConsistencyResponse_Status), and verbose detail.</p>
<tr><td><a name="crdb_internal.check_consistency"></a><code>crdb_internal.check_consistency(stats_only: <a href="bool.html">bool</a>, start_key: <a href="bytes.html">bytes</a>, end_key: <a href="bytes.html">bytes</a>) &rarr; tuple{int AS range_id, bytes AS start_key, string AS start_key_pretty, string AS status, string AS detail, interval AS duration}</code></td><td><span class="funcdesc"><p>Runs a consistency check on ranges touching the specified key range. an empty start or end key is treated as the minimum and maximum possible, respectively. stats_only should only be set to false when targeting a small number of ranges to avoid overloading the cluster. Each returned row contains the range ID, the status (a roachpb.CheckConsistencyResponse_Status), and verbose detail.</p>
<p>Example usage:
SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</p>
</span></td><td>Volatile</td></tr>
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/testccl/workload/schemachange/BUILD.bazel
Expand Up @@ -20,6 +20,7 @@ go_test(
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
Expand Down
Expand Up @@ -21,6 +21,7 @@ import (
_ "github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl/multiregionccltestutils"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/workload"
Expand All @@ -33,6 +34,7 @@ import (
func TestWorkload(t *testing.T) {
defer leaktest.AfterTest(t)()
defer utilccl.TestingEnableEnterprise()()
skip.WithIssue(t, 78478)

dir := t.TempDir()
ctx := context.Background()
Expand Down
10 changes: 10 additions & 0 deletions pkg/cloud/gcp/gcs_storage.go
Expand Up @@ -66,6 +66,15 @@ var gcsChunkingEnabled = settings.RegisterBoolSetting(
true, /* default */
)

// gcsChunkRetryTimeout is used to configure the per-chunk retry deadline when
// uploading chunks to Google Cloud Storage.
var gcsChunkRetryTimeout = settings.RegisterDurationSetting(
settings.TenantWritable,
"cloudstorage.gs.chunking.retry_timeout",
"per-chunk retry deadline when chunking of file upload to Google Cloud Storage",
60,
)

func parseGSURL(_ cloud.ExternalStorageURIContext, uri *url.URL) (cloudpb.ExternalStorage, error) {
gsURL := cloud.ConsumeURL{URL: uri}
conf := cloudpb.ExternalStorage{}
Expand Down Expand Up @@ -253,6 +262,7 @@ func (g *gcsStorage) Writer(ctx context.Context, basename string) (io.WriteClose
if !gcsChunkingEnabled.Get(&g.settings.SV) {
w.ChunkSize = 0
}
w.ChunkRetryDeadline = gcsChunkRetryTimeout.Get(&g.settings.SV)
return w, nil
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/gcjob/BUILD.bazel
Expand Up @@ -38,6 +38,7 @@ go_library(
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/tree",
"//pkg/sql/sqlerrors",
"//pkg/util/admission/admissionpb",
"//pkg/util/hlc",
"//pkg/util/log",
Expand Down
38 changes: 35 additions & 3 deletions pkg/sql/gcjob/index_garbage_collection.go
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)
Expand All @@ -44,6 +45,10 @@ func deleteIndexData(
// are no longer in use. This is necessary in the case of truncate, where we
// schedule a GC Job in the transaction that commits the truncation.
parentDesc, err := sql.WaitToUpdateLeases(ctx, execCfg.LeaseManager, parentID)
if errors.Is(err, catalog.ErrDescriptorNotFound) {
handleTableDescriptorDeleted(ctx, parentID, progress)
return nil
}
if err != nil {
return err
}
Expand Down Expand Up @@ -89,6 +94,10 @@ func gcIndexes(
// are no longer in use. This is necessary in the case of truncate, where we
// schedule a GC Job in the transaction that commits the truncation.
parentDesc, err := sql.WaitToUpdateLeases(ctx, execCfg.LeaseManager, parentID)
if errors.Is(err, catalog.ErrDescriptorNotFound) {
handleTableDescriptorDeleted(ctx, parentID, progress)
return nil
}
if err != nil {
return err
}
Expand Down Expand Up @@ -129,10 +138,15 @@ func gcIndexes(
ctx, txn, execCfg, descriptors, freshParentTableDesc, []uint32{uint32(index.IndexID)},
)
}
if err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs); err != nil {
err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs)
if errors.Is(err, catalog.ErrDescriptorNotFound) ||
sqlerrors.IsUndefinedRelationError(err) {
handleTableDescriptorDeleted(ctx, parentID, progress)
return nil
}
if err != nil {
return errors.Wrapf(err, "removing index %d zone configs", index.IndexID)
}

markIndexGCed(
ctx, index.IndexID, progress, jobspb.SchemaChangeGCProgress_CLEARED,
)
Expand Down Expand Up @@ -214,7 +228,8 @@ func deleteIndexZoneConfigsAfterGC(
}
err := sql.DescsTxn(ctx, execCfg, removeIndexZoneConfigs)
switch {
case errors.Is(err, catalog.ErrDescriptorNotFound):
case errors.Is(err, catalog.ErrDescriptorNotFound),
sqlerrors.IsUndefinedRelationError(err):
log.Infof(ctx, "removing index %d zone config from table %d failed: %v",
index.IndexID, parentID, err)
case err != nil:
Expand All @@ -226,3 +241,20 @@ func deleteIndexZoneConfigsAfterGC(
}
return nil
}

// handleTableDescriptorDeleted should be called when logic detects that
// a table descriptor has been deleted while attempting to GC an index.
// The function marks in progress that all indexes have been cleared.
func handleTableDescriptorDeleted(
ctx context.Context, parentID descpb.ID, progress *jobspb.SchemaChangeGCProgress,
) {
droppedIndexes := progress.Indexes
// If the descriptor has been removed, then we need to assume that the relevant
// zone configs and data have been cleaned up by another process.
log.Infof(ctx, "descriptor %d dropped, assuming another process has handled GC", parentID)
for _, index := range droppedIndexes {
markIndexGCed(
ctx, index.IndexID, progress, jobspb.SchemaChangeGCProgress_CLEARED,
)
}
}
2 changes: 2 additions & 0 deletions pkg/sql/gcjob_test/BUILD.bazel
Expand Up @@ -33,6 +33,8 @@ go_test(
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/gcjob",
"//pkg/sql/gcjob/gcjobnotifier",
"//pkg/sql/sem/catid",
"//pkg/sql/sqlutil",
"//pkg/testutils",
"//pkg/testutils/jobutils",
"//pkg/testutils/serverutils",
Expand Down
118 changes: 118 additions & 0 deletions pkg/sql/gcjob_test/gc_job_test.go
Expand Up @@ -39,6 +39,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/gcjob"
"github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -506,6 +508,122 @@ func TestGCTenant(t *testing.T) {
})
}

// This test exercises code whereby an index GC job is running, and, in the
// meantime, the descriptor is removed. We want to ensure that the GC job
// finishes without an error.
func TestDropIndexWithDroppedDescriptor(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

// The way the GC job works is that it initially clears the index
// data, then it waits for the background MVCC GC to run and remove
// the underlying tombstone, and then finally it removes any relevant
// zone configurations for the index from system.zones. In the first
// and final phases, the job resolves the descriptor. This test ensures
// that the code is robust to the descriptor being removed both before
// the initial DelRange, and after, when going to remove the zone config.
testutils.RunTrueAndFalse(t, "before DelRange", func(
t *testing.T, beforeDelRange bool,
) {
ctx, cancel := context.WithCancel(context.Background())
gcJobID := make(chan jobspb.JobID)
knobs := base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
GCJob: &sql.GCJobTestingKnobs{
RunBeforeResume: func(jobID jobspb.JobID) error {
select {
case <-ctx.Done():
return ctx.Err()
case gcJobID <- jobID:
return nil
}
},
SkipWaitingForMVCCGC: true,
},
}
delRangeChan := make(chan chan struct{})
var tablePrefix atomic.Value
tablePrefix.Store(roachpb.Key{})
// If not running beforeDelRange, we want to delete the descriptor during
// the DeleteRange operation. To do this, we install the below testing knob.
if !beforeDelRange {
knobs.Store = &kvserver.StoreTestingKnobs{
TestingRequestFilter: func(
ctx context.Context, request roachpb.BatchRequest,
) *roachpb.Error {
req, ok := request.GetArg(roachpb.DeleteRange)
if !ok {
return nil
}
dr := req.(*roachpb.DeleteRangeRequest)
if !dr.UseRangeTombstone {
return nil
}
k := tablePrefix.Load().(roachpb.Key)
if len(k) == 0 {
return nil
}
ch := make(chan struct{})
select {
case delRangeChan <- ch:
case <-ctx.Done():
}
select {
case <-ch:
case <-ctx.Done():
}
return nil
},
}
}
s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{
Knobs: knobs,
})
defer s.Stopper().Stop(ctx)
defer cancel()
tdb := sqlutils.MakeSQLRunner(sqlDB)

// Create the table and index to be dropped.
tdb.Exec(t, "CREATE TABLE foo (i INT PRIMARY KEY, j INT, INDEX(j, i))")
// Store the relevant IDs to make it easy to intercept the DelRange.
var tableID catid.DescID
var indexID catid.IndexID
tdb.QueryRow(t, `
SELECT descriptor_id, index_id
FROM crdb_internal.table_indexes
WHERE descriptor_name = 'foo'
AND index_name = 'foo_j_i_idx';`).Scan(&tableID, &indexID)
// Drop the index.
tdb.Exec(t, "DROP INDEX foo@foo_j_i_idx")
codec := s.ExecutorConfig().(sql.ExecutorConfig).Codec
tablePrefix.Store(codec.TablePrefix(uint32(tableID)))

deleteDescriptor := func(t *testing.T) {
t.Helper()
k := catalogkeys.MakeDescMetadataKey(codec, tableID)
_, err := kvDB.Del(ctx, k)
require.NoError(t, err)
}

// Delete the descriptor either before the initial job run, or after
// the job has started, but during the sending of DeleteRange requests.
var jobID jobspb.JobID
if beforeDelRange {
deleteDescriptor(t)
jobID = <-gcJobID
} else {
jobID = <-gcJobID
ch := <-delRangeChan
deleteDescriptor(t)
close(ch)
}
// Ensure that the job completes successfully in either case.
require.NoError(t, s.JobRegistry().(*jobs.Registry).WaitForJobs(
ctx, s.InternalExecutor().(sqlutil.InternalExecutor), []jobspb.JobID{jobID},
))
})
}

// TestGCJobNoSystemConfig tests that the GC job is robust to running with
// no system config provided by the SystemConfigProvider. It is a regression
// test for a panic which could occur due to a slow systemconfigwatcher
Expand Down
22 changes: 13 additions & 9 deletions pkg/sql/logictest/testdata/logic_test/builtin_function_notenant
Expand Up @@ -4,20 +4,20 @@ subtest check_consistency

# Sanity-check crdb_internal.check_consistency.

statement error start key must be >= "\\x02"
SELECT crdb_internal.check_consistency(true, '\x01', '\xffff')
statement error start key must be > "\\x02"
SELECT crdb_internal.check_consistency(true, '\x02', '\xffff')

statement error end key must be < "\\xff\\xff"
SELECT crdb_internal.check_consistency(true, '\x02', '\xffff00')
SELECT crdb_internal.check_consistency(true, '\x0200', '\xffff00')

statement error start key must be less than end key
SELECT crdb_internal.check_consistency(true, '\x02', '\x02')
SELECT crdb_internal.check_consistency(true, '\x03', '\x03')

statement error start key must be less than end key
SELECT crdb_internal.check_consistency(true, '\x03', '\x02')
SELECT crdb_internal.check_consistency(true, '\x04', '\x03')

query ITT
SELECT range_id, status, regexp_replace(detail, '[0-9]+', '', 'g') FROM crdb_internal.check_consistency(true, '\x02', '\xffff') WHERE range_id = 1
SELECT range_id, status, regexp_replace(detail, '[0-9]+', '', 'g') FROM crdb_internal.check_consistency(true, '\x03', '\xffff') WHERE range_id = 1
----
1 RANGE_CONSISTENT stats: {ContainsEstimates: LastUpdateNanos: IntentAge: GCBytesAge: LiveBytes: LiveCount: KeyBytes: KeyCount: ValBytes: ValCount: IntentBytes: IntentCount: SeparatedIntentCount: RangeKeyCount: RangeKeyBytes: RangeValCount: RangeValBytes: SysBytes: SysCount: AbortSpanBytes:}

Expand All @@ -32,14 +32,18 @@ SELECT count(*) > 5 FROM crdb_internal.check_consistency(true, '', '')
true

# Query that should touch only a single range.
#
# NB: the use of ScanMetaKVs causes issues here. Bounds [`k`, k.Next()]` don't work,
# with errors such as (here k=\xff):
# pq: failed to verify keys for Scan: end key /Meta2/"\xff\x00" must be greater than start /Meta2/"\xff\x00"
query B
SELECT count(*) = 1 FROM crdb_internal.check_consistency(true, '\x03', '\x0300')
SELECT count(*) = 1 FROM crdb_internal.check_consistency(true, '\xff', '\xffff')
----
true

# Ditto, but implicit start key \x02
# Ditto, but implicit start key \x03
query B
SELECT count(*) = 1 FROM crdb_internal.check_consistency(true, '', '\x0200')
SELECT count(*) = 1 FROM crdb_internal.check_consistency(true, '', '\x04')
----
true

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/sem/builtins/fixed_oids.go
Expand Up @@ -370,7 +370,7 @@ var builtinOidsBySignature = map[string]oid.Oid{
`crdb_internal.active_version() -> jsonb`: 1296,
`crdb_internal.approximate_timestamp(timestamp: decimal) -> timestamp`: 1298,
`crdb_internal.assignment_cast(val: anyelement, type: anyelement) -> anyelement`: 1341,
`crdb_internal.check_consistency(stats_only: bool, start_key: bytes, end_key: bytes) -> tuple{int AS range_id, bytes AS start_key, string AS start_key_pretty, string AS status, string AS detail}`: 347,
`crdb_internal.check_consistency(stats_only: bool, start_key: bytes, end_key: bytes) -> tuple{int AS range_id, bytes AS start_key, string AS start_key_pretty, string AS status, string AS detail, interval AS duration}`: 347,
`crdb_internal.check_password_hash_format(password: bytes) -> string`: 1376,
`crdb_internal.cluster_id() -> uuid`: 1299,
`crdb_internal.cluster_name() -> string`: 1301,
Expand Down

0 comments on commit aea3285

Please sign in to comment.