Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
…85329

84975: storage: add `MVCCRangeKeyStack` for range keys r=nicktrav,jbowens a=erikgrinaker

**storage: add `MVCCRangeKeyStack` for range keys**

This patch adds `MVCCRangeKeyStack` and `MVCCRangeKeyVersion`, a new
range key representation that will be returned by `SimpleMVCCIterator`.
It is more compact, for efficiency, and comes with a set of convenience
methods to simplify common range key processing.

Resolves #83895.

Release note: None
  
**storage: return `MVCCRangeKeyStack` from `SimpleMVCCIterator`**

This patch changes `SimpleMVCCIterator.RangeKeys()` to return
`MVCCRangeKeyStack` instead of `[]MVCCRangeKeyValue`. Callers have not
been migrated to properly make use of this -- instead, they call
`AsRangeKeyValues()` and construct and use the old data structure.

The MVCC range tombstones tech note is also updated to reflect this.

Release note: None
  
**storage: migrate MVCC code to `MVCCRangeKeyStack`**

Release note: None
  
***: migrate higher-level code to `MVCCRangeKeyStack`**

Release note: None
  
**kvserver/gc: partially migrate to `MVCCRangeKeyStack`**

Some parts require invasive changes to MVCC stats helpers. These will
shortly be consolidated with other MVCC stats logic elsewhere, so the
existing logic is retained for now by using `AsRangeKeyValues()`.

Release note: None
  
**storage: remove `FirstRangeKeyAbove()` and `HasRangeKeyBetween()`**

Release note: None

85017: Revert "sql: Add database ID to sampled query log" r=THardy98 a=THardy98

Reverts: #84195
This reverts commit 307817e.

Removes the DatabaseID field from the
`SampledQuery` telemetry log due to the potential of indefinite blocking
in the case of a lease acquisition failure. Protobuf field not reserved as 
no official build was released with these changes yet.

Release note (sql change): Removes the DatabaseID field from the
`SampledQuery` telemetry log due to the potential of indefinite blocking
in the case of a lease acquisition failure.

85024: cloud/gcp: add custom retryer for gcs storage, retry on stream INTERNAL_ERROR r=rhu713 a=rhu713

Currently, errors like
`stream error: stream ID <x>; INTERNAL_ERROR; received from peer`
are not being retried. Create a custom retryer to retry these errors as
suggested by:

googleapis/google-cloud-go#3735
googleapis/google-cloud-go#784

Fixes: #85217, #85216, #85204, #84162

Release note: None


85069: optbuilder: handle unnest returning a tuple r=DrewKimball a=DrewKimball

Currently, the return types of SRFs that return multiple columns are
represented as tuples with labels. The tuple labels are used to decide
whether or not to create a single output column for the SRF, or multiple.
The `unnest` function can return a single column if it has a single argument,
and the type of that column can be a tuple with labels. This could cause the
old logic to mistakenly create multiple output columns for `unnest`, which
could lead to panics down the line and incorrect behavior otherwise.

This commit adds a special case for `unnest` in the `optbuilder` to only expand
tuple return types if there is more than one argument (implying more than one
output column). Other SRFs do not have the same problem because they either
always return the same number of columns, cannot return tuples, or both.

Fixes #58438

Release note (bug fix): Fixed a bug existing since release 20.1 that could
cause a panic in rare cases when the unnest function was used with a
tuple return type.

85100: opt: perf improvements for large queries r=DrewKimball a=DrewKimball

**opt: add bench test for slow queries**

This commit adds two slow-planning queries pulled from #64793 to be used
in benchmarking the optimizer. In addition, the `ReorderJoinsLimit` has been
set to the default 8 for benchmarking tests.

**opt: add struct for tracking column equivalence sets**

Previously, the `JoinOrderBuilder` would construct a `FuncDepSet` from
scratch on each call to `addJoins` in order to eliminate redundant join
filters. This led to unnecessary large allocations because `addJoins` is
called an exponential number of times in query size.

This commit adds a struct `EquivSet` that efficiently stores equivalence
relations as `ColSets` in a slice. Rather than being constructed on each
call to `addJoins`, a `Reset` method is called that maintains slice memory.

In the future, `EquivSet` can be used to handle equivalencies within `FuncDepSet`
structs as well. This well avoid a significant number of allocations in cases with
many equivalent columns, as outlined in #83963.

**opt: avoid usage of FastIntMap in optimizer hot paths**

Previously, `computeHashJoinCost` would use a `FastIntMap` to represent join
equality filters to pass to `computeFiltersCost`. In addition,
`GenerateMergeJoins` used a `FastIntMap` to look up columns among its join
equality columns. This lead to unnecessary allocations since column IDs are
often large enough to exceed the small field of `FastIntMap`.

This commit modifies `computeFiltersCost` to take an anonymous function
that is used to decide whether to skip an equality condition, removing the
need for a mapping between columns.

This commit also refactors `GenerateMergeJoins` to simply perform a linear
scan of its equality columns; this avoids the allocation issue, and should be
fast in practice because the number of equalities will not generally be large.

Release note: None

85146: [backupccl] Use Expr for backup's Detached and Revision History options r=benbardin a=benbardin

This will allow us to set them to null, which will be helpful for ALTER commands.

Release note: None

85234: dev: add rewritable paths for norm tests r=mgartner a=mgartner

Tests in `pkg/sql/opt/norm` are similar to tests in `pkg/sql/opt/xform`
and `pkg/sql/opt/memo` in that they rely on fixtures in
`pkg/sql/opt/testutils/opttester/testfixtures`. This commit adds these
fixtures as rewritable paths for norm tests so that
`./dev test pkg/sql/opt/xform --rewrite` does not fail with errors like:

    open pkg/sql/opt/testutils/opttester/testfixtures/tpcc_schema: operation not permitted

Release note: None

85325: sql: fix explain gist output to show number of scan span constraints r=cucaroach a=cucaroach

If there were span constraints we would always print 1, need to actually
append them to get the count right.

Fixes: #85324

Release note: None


85327: sql: fix udf logic test r=chengxiong-ruan a=chengxiong-ruan

Fixes: #85303

Release note: None

85329: colexec: fix recent concat fix r=yuzefovich a=yuzefovich

The recent fix of the Concat operator in the vectorized engine doesn't
handle the array concatenation correctly and this is now fixed.

Fixes: #85295.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Thomas Hardy <[email protected]>
Co-authored-by: Rui Hu <[email protected]>
Co-authored-by: DrewKimball <[email protected]>
Co-authored-by: Andrew Kimball <[email protected]>
Co-authored-by: Ben Bardin <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Tommy Reilly <[email protected]>
Co-authored-by: Chengxiong Ruan <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
11 people committed Jul 29, 2022
11 parents 4c8e32a + 4f78a76 + 53d2cd6 + cb92673 + 008d5ab + cf1534d + d28b4d1 + 872e31f + 2a174ce + 1fb2c34 + 719e155 commit 7e2df69
Show file tree
Hide file tree
Showing 77 changed files with 1,630 additions and 559 deletions.
3 changes: 2 additions & 1 deletion build/bazelutil/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,8 @@
"cockroach/pkg/sql/pgwire/pgerror/pgcode\\.go$": "invalid direct cast on error object",
"cockroach/pkg/testutils/lint/lint_test\\.go$": "invalid direct cast on error object",
"cockroach/pkg/util/contextutil/timeout_error\\.go$": "invalid direct cast on error object",
"cockroach/pkg/util/sysutil/sysutil_.*": "type can change by system"
"cockroach/pkg/util/sysutil/sysutil_.*": "type can change by system",
"cockroach/pkg/cloud/gcp/gcs_retry\\.go$": "invalid direct cast on error object"
},
"only_files": {
"cockroach/pkg/.*$": "first-party code"
Expand Down
1 change: 0 additions & 1 deletion docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -2463,7 +2463,6 @@ contains common SQL event/execution details.
| `Database` | Name of the database that initiated the query. | no |
| `StatementID` | Statement ID of the query. | no |
| `TransactionID` | Transaction ID of the query. | no |
| `DatabaseID` | Database ID of the query. | no |
| `StatementFingerprintID` | Statement fingerprint ID of the query. | no |
| `MaxFullScanRowsEstimate` | Maximum number of rows scanned by a full scan, as estimated by the optimizer. | no |
| `TotalScanRowsEstimate` | Total number of rows read by all scans in the query, as estimated by the optimizer. | no |
Expand Down
3 changes: 3 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -2160,7 +2160,10 @@ array_expr ::=
backup_options ::=
'ENCRYPTION_PASSPHRASE' '=' string_or_placeholder
| 'REVISION_HISTORY'
| 'REVISION_HISTORY' '=' a_expr
| 'DETACHED'
| 'DETACHED' '=' 'TRUE'
| 'DETACHED' '=' 'FALSE'
| 'KMS' '=' string_or_placeholder_opt_list
| 'INCREMENTAL_LOCATION' '=' string_or_placeholder_opt_list

Expand Down
37 changes: 28 additions & 9 deletions docs/tech-notes/mvcc-range-tombstones.md
Original file line number Diff line number Diff line change
Expand Up @@ -75,8 +75,8 @@ type MVCCRangeKey struct {
}
```

A range key stores an encoded `MVCCValue`, similarly to `MVCCKey`. They are
often paired as an `MVCCRangeKeyValue`:
A range key stores an encoded `MVCCValue`, similarly to `MVCCKey`. They can be
paired as an `MVCCRangeKeyValue`:

```go
type MVCCRangeKeyValue struct {
Expand Down Expand Up @@ -122,12 +122,31 @@ exist between the bounds `[a-d)`, and `c` is within those bounds. The same is
true for `a@5`, even though it is above both MVCC range tombstones. It is up to
the iterator caller to interpret the range keys as appropriate relative to the
point key. It follows that all range keys overlapping a key will be pulled into
memory at once, but we assume that overlapping range keys will be few. More on
MVCC iteration later.
memory at once, but we assume that overlapping range keys will be few.

In the KV API, however, this distinction doesn't really matter: `Get(c)` at
timestamp >= 5 would return nothing, while `Get(b)` would return `b5`. Again,
more on this later.
This is represented as a specialized compact data structure,
`MVCCRangeKeyStack`, where all range keys have the same bounds due to
fragmentation (described below):

```go
type MVCCRangeKeyStack struct {
Bounds roachpb.Span
Versions MVCCRangeKeyVersions
}

type MVCCRangeKeyVersions []MVCCRangeKeyVersion

type MVCCRangeKeyVersion struct {
Timestamp hlc.Timestamp
Value []byte // encoded MVCCValue
}
```

In the KV API, however, the relationship between point keys and range keys
doesn't really matter: `Get(c)` at timestamp >= 5 would simply return nothing,
while `Get(b)` would return `b5`. More on this later.

### Fragmentation

Range keys do not have a stable, discrete identity, and should be considered a
continuum: they may be partially removed or replaced, merged or fragmented by
Expand Down Expand Up @@ -161,7 +180,7 @@ Pebble: `[a-b)@1`, `[b-c)@2`, `[b-c)@1`, and `[c-d)@2`. Similarly, clearing
`[b-d)@2` would merge the remaining keys back into `[a-c)@1`.

This implies that all range keys exposed for a specific key position all have
the same key bounds.
the same key bounds, as shown in `MVCCRangeKeyStack`.

Fragmentation is beneficial because it makes all range key properties local,
which avoids incurring unnecessary access costs across SSTs and CRDB ranges when
Expand Down Expand Up @@ -268,7 +287,7 @@ The properties of point and range keys are accessed via:
* `RangeBounds()`: start and end bounds of range keys overlapping the current
position, if any.
* `RangeKeys()`: all range keys at the current key position (i.e. at all
timestamps), as `[]MVCCRangeKeyValue`.
timestamps), as `MVCCRangeKeyStack`.

During iteration with `IterKeyTypePointsAndRanges`, range keys are emitted at
their start key and at every overlapping point key. Consider a modified
Expand Down
26 changes: 20 additions & 6 deletions pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -426,6 +426,18 @@ func backupPlanHook(
return nil, nil, nil, false, err
}

detached := false
if backupStmt.Options.Detached == tree.DBoolTrue {
detached = true
}
revisionHistoryFn := func() (bool, error) { return false, nil } // Defaults to false.
if backupStmt.Options.CaptureRevisionHistory != nil {
revisionHistoryFn, err = p.TypeAsBool(ctx, backupStmt.Options.CaptureRevisionHistory, "BACKUP")
if err != nil {
return nil, nil, nil, false, err
}
}

encryptionParams := jobspb.BackupEncryptionOptions{Mode: jobspb.EncryptionMode_None}

var pwFn func() (string, error)
Expand Down Expand Up @@ -464,7 +476,7 @@ func backupPlanHook(
ctx, span := tracing.ChildSpan(ctx, stmt.StatementTag())
defer span.Finish()

if !(p.ExtendedEvalContext().TxnIsSingleStmt || backupStmt.Options.Detached) {
if !(p.ExtendedEvalContext().TxnIsSingleStmt || detached) {
return errors.Errorf("BACKUP cannot be used inside a multi-statement transaction without DETACHED option")
}

Expand Down Expand Up @@ -529,12 +541,14 @@ func backupPlanHook(
}
}

var revisionHistory bool
if backupStmt.Options.CaptureRevisionHistory {
revisionHistory, err := revisionHistoryFn()
if err != nil {
return err
}
if revisionHistory {
if err := requireEnterprise(p.ExecCfg(), "revision_history"); err != nil {
return err
}
revisionHistory = true
}

var targetDescs []catalog.Descriptor
Expand Down Expand Up @@ -633,7 +647,7 @@ func backupPlanHook(
}
plannerTxn := p.Txn()

if backupStmt.Options.Detached {
if detached {
// When running inside an explicit transaction, we simply create the job
// record. We do not wait for the job to finish.
_, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn(
Expand Down Expand Up @@ -674,7 +688,7 @@ func backupPlanHook(
return sj.ReportExecutionResults(ctx, resultsCh)
}

if backupStmt.Options.Detached {
if detached {
return fn, jobs.DetachedJobExecutionResultHeader, nil, false, nil
}
return fn, jobs.BulkJobExecutionResultHeader, nil, false, nil
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5583,7 +5583,7 @@ func TestBackupRestoreShowJob(t *testing.T) {
sqlDB.CheckQueryResults(
t, "SELECT description FROM [SHOW JOBS] WHERE description != 'updating privileges' ORDER BY description",
[][]string{
{"BACKUP DATABASE data TO 'nodelocal://0/foo' WITH revision_history"},
{"BACKUP DATABASE data TO 'nodelocal://0/foo' WITH revision_history = true"},
{"RESTORE TABLE data.bank FROM 'nodelocal://0/foo' WITH into_db = 'data 2', skip_missing_foreign_keys"},
},
)
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,7 @@ func doCreateBackupSchedules(
backupNode := &tree.Backup{
Options: tree.BackupOptions{
CaptureRevisionHistory: eval.BackupOptions.CaptureRevisionHistory,
Detached: true,
Detached: tree.DBoolTrue,
},
Nested: true,
AppendToLatest: false,
Expand Down
28 changes: 14 additions & 14 deletions pkg/ccl/backupccl/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -422,7 +422,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
expectedSchedules: []expectedSchedule{
{
nameRe: "BACKUP .+",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH revision_history, detached",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH revision_history = true, detached",
period: time.Hour,
},
},
Expand Down Expand Up @@ -459,15 +459,15 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
nameRe: "BACKUP .*",
backupStmt: "BACKUP TABLE system.public.jobs, " +
"system.public.scheduled_jobs INTO LATEST IN 'nodelocal://0/backup' WITH" +
" revision_history, detached",
" revision_history = true, detached",
period: time.Hour,
paused: true,
chainProtectedTimestampRecord: true,
},
{
nameRe: "BACKUP .+",
backupStmt: "BACKUP TABLE system.public.jobs, " +
"system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH revision_history, detached",
"system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH revision_history = true, detached",
period: 24 * time.Hour,
runsNow: true,
chainProtectedTimestampRecord: true,
Expand All @@ -483,14 +483,14 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
expectedSchedules: []expectedSchedule{
{
nameRe: "BACKUP .*",
backupStmt: "BACKUP DATABASE system INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached",
backupStmt: "BACKUP DATABASE system INTO LATEST IN 'nodelocal://0/backup' WITH revision_history = true, detached",
period: time.Hour,
paused: true,
chainProtectedTimestampRecord: true,
},
{
nameRe: "BACKUP .+",
backupStmt: "BACKUP DATABASE system INTO 'nodelocal://0/backup' WITH revision_history, detached",
backupStmt: "BACKUP DATABASE system INTO 'nodelocal://0/backup' WITH revision_history = true, detached",
period: 24 * time.Hour,
runsNow: true,
chainProtectedTimestampRecord: true,
Expand All @@ -506,14 +506,14 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
expectedSchedules: []expectedSchedule{
{
nameRe: "BACKUP .*",
backupStmt: "BACKUP TABLE system.public.* INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached",
backupStmt: "BACKUP TABLE system.public.* INTO LATEST IN 'nodelocal://0/backup' WITH revision_history = true, detached",
period: time.Hour,
paused: true,
chainProtectedTimestampRecord: true,
},
{
nameRe: "BACKUP .+",
backupStmt: "BACKUP TABLE system.public.* INTO 'nodelocal://0/backup' WITH revision_history, detached",
backupStmt: "BACKUP TABLE system.public.* INTO 'nodelocal://0/backup' WITH revision_history = true, detached",
period: 24 * time.Hour,
runsNow: true,
chainProtectedTimestampRecord: true,
Expand Down Expand Up @@ -546,14 +546,14 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
expectedSchedules: []expectedSchedule{
{
nameRe: "my_backup_name",
backupStmt: "BACKUP INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached",
backupStmt: "BACKUP INTO LATEST IN 'nodelocal://0/backup' WITH revision_history = true, detached",
period: time.Hour,
paused: true,
chainProtectedTimestampRecord: true,
},
{
nameRe: "my_backup_name",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH revision_history, detached",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH revision_history = true, detached",
period: 24 * time.Hour,
runsNow: true,
chainProtectedTimestampRecord: true,
Expand All @@ -571,10 +571,10 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
nameRe: "BACKUP .*",
backupStmt: "BACKUP TABLE system.public.jobs, " +
"system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH" +
" revision_history, encryption_passphrase = 'secret', detached",
" revision_history = true, encryption_passphrase = 'secret', detached",
shownStmt: "BACKUP TABLE system.public.jobs, " +
"system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH" +
" revision_history, encryption_passphrase = '*****', detached",
" revision_history = true, encryption_passphrase = '*****', detached",
period: 7 * 24 * time.Hour,
},
},
Expand All @@ -596,7 +596,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
nameRe: "BACKUP .+",
backupStmt: "BACKUP DATABASE system INTO " +
"('nodelocal://0/backup?COCKROACH_LOCALITY=x%3Dy', 'nodelocal://0/backup2?COCKROACH_LOCALITY=default') " +
"WITH revision_history, detached",
"WITH revision_history = true, detached",
period: 24 * time.Hour,
},
},
Expand All @@ -614,7 +614,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
},
}

for _, tc := range testCases {
for i, tc := range testCases {
t.Run(fmt.Sprintf("%s-%s", tc.name, tc.user), func(t *testing.T) {
defer th.clearSchedules(t)

Expand Down Expand Up @@ -652,7 +652,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
for _, s := range schedules {
stmt := getScheduledBackupStatement(t, s.ExecutionArgs())
expectedSchedule, ok := expectedByName[stmt]
require.True(t, ok, "could not find matching name for %q", stmt)
require.True(t, ok, "in test case %d, could not find matching name for %q", i, stmt)
require.Regexp(t, regexp.MustCompile(expectedSchedule.nameRe), s.ScheduleLabel())

expectedShown := fmt.Sprintf("%q", expectedSchedule.backupStmt)
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/backupccl/file_sst_sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -363,8 +363,9 @@ func (s *fileSSTSink) copyRangeKeys(dataSST []byte) error {
} else if !ok {
break
}
for _, rkv := range iter.RangeKeys() {
if err := s.sst.PutRawMVCCRangeKey(rkv.RangeKey, rkv.Value); err != nil {
rangeKeys := iter.RangeKeys()
for _, v := range rangeKeys.Versions {
if err := s.sst.PutRawMVCCRangeKey(rangeKeys.AsRangeKey(v), v.Value); err != nil {
return err
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/schedule_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,8 +70,8 @@ func (e *scheduledBackupExecutor) executeBackup(
}

// Sanity check: backup should be detached.
if !backupStmt.Options.Detached {
backupStmt.Options.Detached = true
if backupStmt.Options.Detached != tree.DBoolTrue {
backupStmt.Options.Detached = tree.DBoolTrue
log.Warningf(ctx, "force setting detached option for backup schedule %d",
sj.ScheduleID())
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/cloud/gcp/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
name = "gcp",
srcs = [
"gcs_kms.go",
"gcs_retry.go",
"gcs_storage.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/cloud/gcp",
Expand All @@ -23,11 +24,15 @@ go_library(
"@com_github_gogo_protobuf//types",
"@com_google_cloud_go_kms//apiv1",
"@com_google_cloud_go_storage//:storage",
"@org_golang_google_api//googleapi",
"@org_golang_google_api//impersonate",
"@org_golang_google_api//iterator",
"@org_golang_google_api//option",
"@org_golang_google_genproto//googleapis/cloud/kms/v1:kms",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//status",
"@org_golang_google_protobuf//types/known/wrapperspb",
"@org_golang_x_net//http2",
"@org_golang_x_oauth2//:oauth2",
],
)
Expand Down
Loading

0 comments on commit 7e2df69

Please sign in to comment.