Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
119493: sql: support OUT parameters in PLpgSQL UDFs r=yuzefovich a=yuzefovich

This commit (mostly) adds the support for INOUT / OUT parameters in PLpgSQL UDFs. OUT parameters are not included into the signature (for overload resolution), but they result in uninitialized variables. Any parameter can be left unnamed but can still be referenced via `$i` notation which is unsupported so far. Unlike for SQL UDFs, in PLpgSQL input and output parameters share the same namespace for duplication check.

Additionally, this commit fixes a few issues:
- for UDFs only input parameters need to be added to the scope of the body
- for procedures, all parameters (including OUT) are used for overload resolution.
- UDF returning VOID with empty body previously would hit an index out of bounds internal error.

Addresses: cockroachdb#100405.
Fixes: cockroachdb#119208.
Epic: CRDB-30611

Fixes: cockroachdb#119841.

Release note: None

119820: sql: fix redaction and visibility of SHOW SESSIONS and SHOW QUERIES r=rafiss a=rafiss

Release note (bug fix): If a user has the VIEWACTIVITYREDACTED
privilege, then they should not be able to see constants inside of
queries that originate from other users in the SHOW SESSIONS result.
This redaction was not occurring, but now this is fixed.

Release note (bug fix): The SHOW QUERIES and SHOW STATEMENTS commands
were incorrectly requiring that the user has the VIEWACTIVITY or
VIEWACTIVITYREDACTED privilege. However, a user always should be able to
view their own queries, even without this privilege. This is fixed now.

Epic: CRDB-35306

119840: restore: limit restore spans to 200 files r=dt a=dt

Previously we would limit restore spans to 384MB, but if a backup contained many small files, a large number of these files could be grouped into such a span before it hit the 384MB target. Unfortunately this caused the restore process to open all of those files to process the restore span, leading to very large numbers of concurrent connections when restoring a backup consisting of tiny files.

This patch limits the restore spans to 384MB or 200 files, whichever is hit first. Restores of backups with large numbers of tiny files may thus be slightly slower but will better limit their concurrent outbound connections.

Fixes cockroachdb#119785.

NB: This breaks the test cases for the experimental/disabled support for memory-monitored restores. Given we have no plans to actually enable these at this time, but do need to fix this bug in the normal restore path and backport such a fix,  to reduce churn in said backports the tests of the disabled mode are simply skipped in this diff rather modified, in anticipation of removing them completely from the development branch, tracked in cockroachdb#119836.

Release note (bug fix): fix a bug where RESTORE on certain backups would open a very large number of concurrent connections to the backup storage provider.

Epic: none.

Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: David Taylor <[email protected]>
  • Loading branch information
4 people committed Mar 4, 2024
4 parents c52d468 + dbb16fb + 3cd24e8 + 1bc152e commit 1d79dd8
Show file tree
Hide file tree
Showing 30 changed files with 1,622 additions and 600 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/bench_covering_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ func BenchmarkCoverageChecks(b *testing.B) {
b.Run(fmt.Sprintf("numFiles=%d", baseFiles), func(b *testing.B) {
for _, hasExternalFilesList := range []bool{true, false} {
b.Run(fmt.Sprintf("slim=%t", hasExternalFilesList), func(b *testing.B) {
backups, err := MockBackupChain(ctx, numBackups, numSpans, baseFiles, r, hasExternalFilesList, execCfg)
backups, err := MockBackupChain(ctx, numBackups, numSpans, baseFiles, 1<<20, r, hasExternalFilesList, execCfg)
require.NoError(b, err)
b.ResetTimer()

Expand Down Expand Up @@ -75,7 +75,7 @@ func BenchmarkRestoreEntryCover(b *testing.B) {
for _, hasExternalFilesList := range []bool{true, false} {
b.Run(fmt.Sprintf("hasExternalFilesList=%t", hasExternalFilesList),
func(b *testing.B) {
backups, err := MockBackupChain(ctx, numBackups, numSpans, baseFiles, r, hasExternalFilesList, execCfg)
backups, err := MockBackupChain(ctx, numBackups, numSpans, baseFiles, 1<<20, r, hasExternalFilesList, execCfg)
require.NoError(b, err)
b.ResetTimer()
for i := 0; i < b.N; i++ {
Expand Down
20 changes: 17 additions & 3 deletions pkg/ccl/backupccl/restore_span_covering.go
Original file line number Diff line number Diff line change
Expand Up @@ -307,6 +307,7 @@ func generateAndSendImportSpans(
// lastCovSpanSize is the size of files added to the right-most span of
// the cover so far.
var lastCovSpanSize int64
var lastCovSpanCount int
var lastCovSpan roachpb.Span
var covFilesByLayer [][]*backuppb.BackupManifest_File
var firstInSpan bool
Expand Down Expand Up @@ -390,8 +391,8 @@ func generateAndSendImportSpans(
}

var filesByLayer [][]*backuppb.BackupManifest_File
var covSize int64
var newCovFilesSize int64
var covSize, newCovFilesSize int64
var covCount, newCovFilesCount int

for layer := range newFilesByLayer {
for _, file := range newFilesByLayer[layer] {
Expand All @@ -401,6 +402,7 @@ func generateAndSendImportSpans(
}
newCovFilesSize += sz
}
newCovFilesCount += len(newFilesByLayer[layer])
filesByLayer = append(filesByLayer, newFilesByLayer[layer])
}

Expand All @@ -413,6 +415,7 @@ func generateAndSendImportSpans(

if fsc.overlaps(coverSpan, file.Span) {
covSize += sz
covCount++
filesByLayer[layer] = append(filesByLayer[layer], file)
}
}
Expand All @@ -422,8 +425,17 @@ func generateAndSendImportSpans(
covFilesByLayer = newFilesByLayer
lastCovSpan = coverSpan
lastCovSpanSize = newCovFilesSize
lastCovSpanCount = newCovFilesCount
} else {
if (newCovFilesSize == 0 || lastCovSpanSize+newCovFilesSize <= filter.targetSize) && !firstInSpan {
// We have room to add to the last span if doing so would remain below
// both the target byte size and 200 total files. We limit the number
// of files since we default to running multiple concurrent workers so
// we want to bound sum total open files across all of them to <= 1k.
// We bound the span byte size to improve work distribution and make
// the progress more granular.
fits := lastCovSpanSize+newCovFilesSize <= filter.targetSize && lastCovSpanCount+newCovFilesCount <= 200

if (newCovFilesCount == 0 || fits) && !firstInSpan {
// If there are no new files that cover this span or if we can add the
// files in the new span's cover to the last span's cover and still stay
// below targetSize, then we should merge the two spans.
Expand All @@ -432,13 +444,15 @@ func generateAndSendImportSpans(
}
lastCovSpan.EndKey = coverSpan.EndKey
lastCovSpanSize = lastCovSpanSize + newCovFilesSize
lastCovSpanCount = lastCovSpanCount + newCovFilesCount
} else {
if err := flush(ctx); err != nil {
return err
}
lastCovSpan = coverSpan
covFilesByLayer = filesByLayer
lastCovSpanSize = covSize
lastCovSpanCount = covCount
}
}
firstInSpan = false
Expand Down
26 changes: 21 additions & 5 deletions pkg/ccl/backupccl/restore_span_covering_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ import (
// Files spans are ordered by start key but may overlap.
func MockBackupChain(
ctx context.Context,
length, spans, baseFiles int,
length, spans, baseFiles, fileSize int,
r *rand.Rand,
hasExternalFilesList bool,
execCfg sql.ExecutorConfig,
Expand Down Expand Up @@ -108,7 +108,7 @@ func MockBackupChain(
backups[i].Files[f].Span.Key = encoding.EncodeVarintAscending(k, int64(start))
backups[i].Files[f].Span.EndKey = encoding.EncodeVarintAscending(k, int64(end))
backups[i].Files[f].Path = fmt.Sprintf("12345-b%d-f%d.sst", i, f)
backups[i].Files[f].EntryCounts.DataSize = 1 << 20
backups[i].Files[f].EntryCounts.DataSize = int64(fileSize)
}

es, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx,
Expand Down Expand Up @@ -225,6 +225,9 @@ func checkRestoreCovering(
}
var spanIdx int
for _, c := range cov {
if len(c.Files) > 500 {
return errors.Errorf("%d files in span %v", len(c.Files), c.Span)
}
for _, f := range c.Files {
if requireSpan, ok := required[f.Path]; ok {
requireSpan.Sub(c.Span)
Expand Down Expand Up @@ -721,8 +724,21 @@ func sanityCheckFileIterator(
}
}

func TestRestoreEntryCoverTinyFiles(t *testing.T) {
defer leaktest.AfterTest(t)()
runTestRestoreEntryCoverForSpanAndFileCounts(t, 5, 5<<10, []int{5}, []int{1000, 5000})
}

//lint:ignore U1000 unused
func runTestRestoreEntryCover(t *testing.T, numBackups int) {
spans := []int{1, 2, 3, 5, 9, 11, 12}
files := []int{0, 1, 2, 3, 4, 10, 12, 50}
runTestRestoreEntryCoverForSpanAndFileCounts(t, numBackups, 1<<20, spans, files)
}

func runTestRestoreEntryCoverForSpanAndFileCounts(
t *testing.T, numBackups, fileSize int, spanCounts, fileCounts []int,
) {
r, _ := randutil.NewTestRand()
ctx := context.Background()
tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication)
Expand Down Expand Up @@ -752,10 +768,10 @@ func runTestRestoreEntryCover(t *testing.T, numBackups int) {
return merged
}

for _, spans := range []int{1, 2, 3, 5, 9, 11, 12} {
for _, files := range []int{0, 1, 2, 3, 4, 10, 12, 50} {
for _, spans := range spanCounts {
for _, files := range fileCounts {
for _, hasExternalFilesList := range []bool{true, false} {
backups, err := MockBackupChain(ctx, numBackups, spans, files, r, hasExternalFilesList, execCfg)
backups, err := MockBackupChain(ctx, numBackups, spans, files, fileSize, r, hasExternalFilesList, execCfg)
require.NoError(t, err)
layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx,
execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil)
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/backupccl/utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -577,6 +578,7 @@ func requireRecoveryEvent(
//
//lint:ignore U1000 unused
func runTestRestoreMemoryMonitoring(t *testing.T, numSplits, numInc, restoreProcessorMaxFiles int) {
skip.WithIssue(t, 119836, "this functionality was never enabled and will likely be removed rather than enabled")
const splitSize = 10
numAccounts := numSplits * splitSize
var expectedNumFiles int
Expand Down
Loading

0 comments on commit 1d79dd8

Please sign in to comment.