Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
83527: sqlproxyccl: support pgwire cancel protocol r=jeffswenson,jaylim-crl a=rafiss

See individual commits.

This implements the RFC here: https://github.com/cockroachdb/cockroach/blob/969bcf747a9f87f5231a043c9833c51d9979a549/docs/RFCS/20220202_pgwire_compatible_cancel.md

84728: ttl: improve row-level TTL performance using DistSQL r=rafiss a=ecwall

fixes #76914

Release note (performance improvement): The row-level TTL job has been modified
to distribute work using DistSQL. This usually results in the leaseholder nodes
managing deleting of the spans they own.

85080: cluster-ui: add sql api request wrapper and clusterLocks request r=xinhaoz a=xinhaoz

This commit allows DB Console to use the SQL over HTTP API from
`/api/v2/sql/`. A new fetch wrapper providing the custom header
necessary for the API and using content type JSON has been added.

The clusterLocksApi components added in this commit use the above
SQL api functions to query from the `crdb_internal.cluster_locks`
table.

Release note: None

85684: colexec: reorder some fields to reduce memory footprint r=yuzefovich a=yuzefovich

This commit applies some of the fixes found by the `fieldalignment` tool
on the colexec packages which reorder some fields in the structs so that
the memory footprint is reduced.

Release note: None

85697: kvcoord: optimize usages of BatchRequest.IsReverse r=yuzefovich a=yuzefovich

This commit audits all places where `BatchRequest.IsReverse` is called
and ensures that we call that only when necessary. In particular, it
avoids the call altogether when a request needs to be resumed as well as
when verifying the batch initially when key and / or bytes limits are
present.

Release note: None

85708: storage: don't synthesize MVCC point tombstones below point keys r=jbowens a=erikgrinaker

This patch changes `pointSynthesizingIter` (and by extension MVCC scans
and gets) to not synthesize MVCC point tombstones below existing point
keys, only above them. Point tombstones are still synthesized at the
start bound of all MVCC range tombstones regardless.

This patch only focuses on the behavioral change, and is not concerned
with performance. A later patch will address performance optimizations.
Even so, this can significantly improve `MVCCScan` performance with many
range keys:

```
MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=0-24      2.76ms ± 1%    2.78ms ± 2%      ~     (p=0.274 n=8+10)
MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=1-24      6.34ms ± 1%    5.72ms ± 1%    -9.80%  (p=0.000 n=10+10)
MVCCScan_Pebble/rows=10000/versions=1/valueSize=64/numRangeKeys=100-24    60.1ms ± 7%    23.6ms ± 7%   -60.72%  (p=0.000 n=10+10)
MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=0-24        2.73µs ± 1%    2.72µs ± 1%      ~     (p=0.268 n=9+10)
MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=1-24        5.40µs ± 1%    5.46µs ± 1%    +1.18%  (p=0.001 n=10+10)
MVCCGet_Pebble/batch=true/versions=1/valueSize=8/numRangeKeys=100-24       171µs ± 1%     170µs ± 1%      ~     (p=0.247 n=10+10)
MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=0-24       3.87µs ± 1%    3.85µs ± 0%    -0.58%  (p=0.030 n=10+9)
MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=1-24       7.11µs ± 1%    7.24µs ± 1%    +1.83%  (p=0.000 n=9+10)
MVCCGet_Pebble/batch=true/versions=10/valueSize=8/numRangeKeys=100-24      179µs ± 1%     178µs ± 1%      ~     (p=0.063 n=10+10)
MVCCGet_Pebble/batch=true/versions=100/valueSize=8/numRangeKeys=0-24      10.4µs ± 5%    10.0µs ± 3%    -3.96%  (p=0.013 n=10+9)
MVCCGet_Pebble/batch=true/versions=100/valueSize=8/numRangeKeys=1-24      15.9µs ± 3%    16.2µs ± 3%    +2.11%  (p=0.007 n=10+10)
MVCCGet_Pebble/batch=true/versions=100/valueSize=8/numRangeKeys=100-24     222µs ± 1%     220µs ± 2%      ~     (p=0.063 n=10+10)
```

Resolves #83899.

Release note: None

85713: storage: add `MVCCRangeKeyStack.CloneInto()` r=jbowens a=erikgrinaker

This patch adds a `CloneInto()` method for MVCC range key stacks, which
will reuse the allocations of the given stack.

An alternative approach was also considered that would use a single
allocation for all byte slices, but in the vast majority of cases range
keys will have no values, and so we'll typically only make two
allocations anyway. This can be reconsidered later.

Callers haven't been updated to make use of this yet. A pass will be
made later, which will also apply various other MVCC range key
optimizations.

```
name                                                                          time/op
MVCCRangeKeyStack_Clone/keySize=16/numVersions=1/withValues=0/Clone-24         123ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=1/withValues=0/CloneInto-24    16.7ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=1/withValues=1/Clone-24         149ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=1/withValues=1/CloneInto-24    17.2ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=3/withValues=0/Clone-24         169ns ± 1%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=3/withValues=0/CloneInto-24    27.5ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=3/withValues=1/Clone-24         202ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=3/withValues=1/CloneInto-24    27.9ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=10/withValues=0/Clone-24        291ns ± 1%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=10/withValues=0/CloneInto-24   64.8ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=10/withValues=1/Clone-24        327ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=10/withValues=1/CloneInto-24   64.5ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=100/withValues=0/Clone-24      1.94µs ± 1%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=100/withValues=0/CloneInto-24   544ns ± 0%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=100/withValues=1/Clone-24      2.01µs ± 1%
MVCCRangeKeyStack_Clone/keySize=16/numVersions=100/withValues=1/CloneInto-24   551ns ± 0%
```

Resolves #85381.

Release note: None

85724: scripts: update benchmarking script to use `dev` r=irfansharif a=irfansharif

Fixes #80407. Usage:

    BENCHES=BenchmarkTracing/1node/scan/trace=off \
      PKG=./pkg/bench \
      scripts/bench HEAD HEAD~1

Invokes the following underneath the hood:

    dev bench ./pkg/bench --timeout=5m \
      --filter=BenchmarkTracing/1node/scan/trace=off --count=10 \
      --bench-mem -v --ignore-cache

Release note: None

85742: sql/randgen: avoid invalid inverted indexes in CREATE TABLE r=fqazi a=fqazi

Previously, the create table made via the randgen component
could create invalid CREATE TABLE statements where the last
columns of an inverted index were descending. This could
lead to unexpected failures in certain workloads, which
expected valid statements from this components. To address
this, this patch stops randgen from generating CREATE table
statements with invalid inverted indexes.

Release note: None

Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
Co-authored-by: Evan Wall <wall@cockroachlabs.com>
Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
Co-authored-by: Faizan Qazi <faizan@cockroachlabs.com>
  • Loading branch information
8 people committed Aug 8, 2022
10 parents c548f4a + 99b629c + 081cce5 + b75a704 + e9c7cae + 3ffbe4d + 92b5eaf + 6c95e87 + 973c5a8 + 16e5d55 commit d58473e
Show file tree
Hide file tree
Showing 60 changed files with 2,376 additions and 692 deletions.
2 changes: 2 additions & 0 deletions pkg/ccl/sqlproxyccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ go_library(
"metrics.go",
"proxy.go",
"proxy_handler.go",
"query_cancel.go",
"server.go",
":gen-errorcode-stringer", # keep
],
Expand All @@ -39,6 +40,7 @@ go_library(
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/netutil/addr",
"//pkg/util/randutil",
"//pkg/util/retry",
"//pkg/util/stop",
"//pkg/util/syncutil",
Expand Down
67 changes: 41 additions & 26 deletions pkg/ccl/sqlproxyccl/authentication.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,11 @@ import (
// authenticate handles the startup of the pgwire protocol to the point where
// the connections is considered authenticated. If that doesn't happen, it
// returns an error.
var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttler.AttemptStatus) error) error {
var authenticate = func(
clientConn, crdbConn net.Conn,
proxyBackendKeyData *pgproto3.BackendKeyData,
throttleHook func(throttler.AttemptStatus) error,
) (crdbBackendKeyData *pgproto3.BackendKeyData, _ error) {
fe := pgproto3.NewBackend(pgproto3.NewChunkReader(clientConn), clientConn)
be := pgproto3.NewFrontend(pgproto3.NewChunkReader(crdbConn), crdbConn)

Expand All @@ -39,7 +43,7 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl
// TODO(spaskob): in verbose mode, log these messages.
backendMsg, err := be.Receive()
if err != nil {
return newErrorf(codeBackendReadFailed, "unable to receive message from backend: %v", err)
return nil, newErrorf(codeBackendReadFailed, "unable to receive message from backend: %v", err)
}

// The cases in this switch are roughly sorted in the order the server will send them.
Expand All @@ -54,7 +58,7 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl
*pgproto3.AuthenticationSASLFinal,
*pgproto3.AuthenticationSASL:
if err = feSend(backendMsg); err != nil {
return err
return nil, err
}
switch backendMsg.(type) {
case *pgproto3.AuthenticationCleartextPassword:
Expand All @@ -73,11 +77,11 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl
}
fntMsg, err := fe.Receive()
if err != nil {
return newErrorf(codeClientReadFailed, "unable to receive message from client: %v", err)
return nil, newErrorf(codeClientReadFailed, "unable to receive message from client: %v", err)
}
err = be.Send(fntMsg)
if err != nil {
return newErrorf(
return nil, newErrorf(
codeBackendWriteFailed, "unable to send message %v to backend: %v", fntMsg, err,
)
}
Expand All @@ -89,12 +93,12 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl
throttleError := throttleHook(throttler.AttemptOK)
if throttleError != nil {
if err = feSend(toPgError(throttleError)); err != nil {
return err
return nil, err
}
return throttleError
return nil, throttleError
}
if err = feSend(backendMsg); err != nil {
return err
return nil, err
}

// Server has rejected the authentication response from the client and
Expand All @@ -103,36 +107,44 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl
throttleError := throttleHook(throttler.AttemptInvalidCredentials)
if throttleError != nil {
if err = feSend(toPgError(throttleError)); err != nil {
return err
return nil, err
}
return throttleError
return nil, throttleError
}
if err = feSend(backendMsg); err != nil {
return err
return nil, err
}
return newErrorf(codeAuthFailed, "authentication failed: %s", tp.Message)
return nil, newErrorf(codeAuthFailed, "authentication failed: %s", tp.Message)

// Information provided by the server to the client before the connection is ready
// to accept queries. These are typically returned after AuthenticationOk and before
// ReadyForQuery.
case *pgproto3.ParameterStatus, *pgproto3.BackendKeyData:
case *pgproto3.ParameterStatus:
if err = feSend(backendMsg); err != nil {
return err
return nil, err
}

// BackendKeyData is part of the Postgres query cancellation protocol.
// sqlproxy saves it and returns a different one to the client.
case *pgproto3.BackendKeyData:
crdbBackendKeyData = tp
if err = feSend(proxyBackendKeyData); err != nil {
return nil, err
}

// Server has authenticated the connection successfully and is ready to
// serve queries.
case *pgproto3.ReadyForQuery:
if err = feSend(backendMsg); err != nil {
return err
return nil, err
}
return nil
return crdbBackendKeyData, nil

default:
return newErrorf(codeBackendDisconnected, "received unexpected backend message type: %v", tp)
return nil, newErrorf(codeBackendDisconnected, "received unexpected backend message type: %v", tp)
}
}
return newErrorf(codeBackendDisconnected, "authentication took more than %d iterations", i)
return nil, newErrorf(codeBackendDisconnected, "authentication took more than %d iterations", i)
}

// readTokenAuthResult reads the result for the token-based authentication, and
Expand All @@ -153,37 +165,40 @@ var authenticate = func(clientConn, crdbConn net.Conn, throttleHook func(throttl
// we should merge them back in the future. Instead of having the writer as the
// other end, the writer should be the same connection. That way, a
// sqlproxyccl.Conn can be used to read-from, or write-to the same component.
var readTokenAuthResult = func(conn net.Conn) error {
var readTokenAuthResult = func(conn net.Conn) (*pgproto3.BackendKeyData, error) {
// This interceptor is discarded once this function returns. Just like
// pgproto3.NewFrontend, this serverConn object has an internal buffer.
// Discarding the buffer is fine since there won't be any other messages
// from the server once we receive the ReadyForQuery message because the
// caller (i.e. proxy) does not forward client messages until then.
serverConn := interceptor.NewFrontendConn(conn)

var backendKeyData *pgproto3.BackendKeyData
// The auth step should require only a few back and forths so 20 iterations
// should be enough.
var i int
for ; i < 20; i++ {
backendMsg, err := serverConn.ReadMsg()
if err != nil {
return newErrorf(codeBackendReadFailed, "unable to receive message from backend: %v", err)
return nil, newErrorf(codeBackendReadFailed, "unable to receive message from backend: %v", err)
}

switch tp := backendMsg.(type) {
case *pgproto3.AuthenticationOk, *pgproto3.ParameterStatus, *pgproto3.BackendKeyData:
// Do nothing.
case *pgproto3.AuthenticationOk, *pgproto3.ParameterStatus:
// Do nothing.
case *pgproto3.BackendKeyData:
backendKeyData = tp

case *pgproto3.ErrorResponse:
return newErrorf(codeAuthFailed, "authentication failed: %s", tp.Message)
return nil, newErrorf(codeAuthFailed, "authentication failed: %s", tp.Message)

case *pgproto3.ReadyForQuery:
return nil
return backendKeyData, nil

default:
return newErrorf(codeBackendDisconnected, "received unexpected backend message type: %v", tp)
return nil, newErrorf(codeBackendDisconnected, "received unexpected backend message type: %v", tp)
}
}

return newErrorf(codeBackendDisconnected, "authentication took more than %d iterations", i)
return nil, newErrorf(codeBackendDisconnected, "authentication took more than %d iterations", i)
}
49 changes: 33 additions & 16 deletions pkg/ccl/sqlproxyccl/authentication_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/errors"
"github.com/jackc/pgproto3/v2"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)

Expand All @@ -30,15 +31,26 @@ func TestAuthenticateOK(t *testing.T) {
be := pgproto3.NewBackend(pgproto3.NewChunkReader(srv), srv)
fe := pgproto3.NewFrontend(pgproto3.NewChunkReader(cli), cli)

proxyBackendKeyData := &pgproto3.BackendKeyData{ProcessID: 1, SecretKey: 1}
crdbBackendKeyData := &pgproto3.BackendKeyData{ProcessID: 2, SecretKey: 2}
go func() {
err := be.Send(&pgproto3.ReadyForQuery{})
require.NoError(t, err)
// First the frontend gets back the proxy's BackendKeyData.
err := be.Send(crdbBackendKeyData)
assert.NoError(t, err)
beMsg, err := fe.Receive()
require.NoError(t, err)
require.Equal(t, beMsg, &pgproto3.ReadyForQuery{})
assert.NoError(t, err)
assert.Equal(t, beMsg, proxyBackendKeyData)
// Then the frontend gets ReadyForQuery.
err = be.Send(&pgproto3.ReadyForQuery{})
assert.NoError(t, err)
beMsg, err = fe.Receive()
assert.NoError(t, err)
assert.Equal(t, beMsg, &pgproto3.ReadyForQuery{})
}()

require.NoError(t, authenticate(srv, cli, nilThrottleHook))
receivedCrdbBackendKeyData, err := authenticate(srv, cli, proxyBackendKeyData, nilThrottleHook)
require.NoError(t, err)
require.Equal(t, crdbBackendKeyData, receivedCrdbBackendKeyData)
}

func TestAuthenticateClearText(t *testing.T) {
Expand Down Expand Up @@ -80,7 +92,8 @@ func TestAuthenticateClearText(t *testing.T) {
require.Equal(t, beMsg, &pgproto3.ReadyForQuery{})
}()

require.NoError(t, authenticate(srv, cli, nilThrottleHook))
_, err := authenticate(srv, cli, nil /* proxyBackendKeyData */, nilThrottleHook)
require.NoError(t, err)
}

func TestAuthenticateThrottled(t *testing.T) {
Expand Down Expand Up @@ -144,10 +157,11 @@ func TestAuthenticateThrottled(t *testing.T) {
go server(t, sqlServer, &pgproto3.AuthenticationOk{})
go client(t, sqlClient)

err := authenticate(proxyToClient, proxyToServer, func(status throttler.AttemptStatus) error {
require.Equal(t, throttler.AttemptOK, status)
return throttledError
})
_, err := authenticate(proxyToClient, proxyToServer, nil, /* proxyBackendKeyData */
func(status throttler.AttemptStatus) error {
require.Equal(t, throttler.AttemptOK, status)
return throttledError
})
require.Error(t, err)
require.Contains(t, err.Error(), "connection attempt throttled")

Expand All @@ -172,7 +186,7 @@ func TestAuthenticateError(t *testing.T) {
require.Equal(t, beMsg, &pgproto3.ErrorResponse{Severity: "FATAL", Code: "foo"})
}()

err := authenticate(srv, cli, nilThrottleHook)
_, err := authenticate(srv, cli, nil /* proxyBackendKeyData */, nilThrottleHook)
require.Error(t, err)
codeErr := (*codeError)(nil)
require.True(t, errors.As(err, &codeErr))
Expand All @@ -193,7 +207,7 @@ func TestAuthenticateUnexpectedMessage(t *testing.T) {
require.Error(t, err)
}()

err := authenticate(srv, cli, nilThrottleHook)
_, err := authenticate(srv, cli, nil /* proxyBackendKeyData */, nilThrottleHook)

srv.Close()

Expand All @@ -214,7 +228,7 @@ func TestReadTokenAuthResult(t *testing.T) {
require.NoError(t, err)
}()

err := readTokenAuthResult(cli)
_, err := readTokenAuthResult(cli)
require.Error(t, err)
codeErr := (*codeError)(nil)
require.True(t, errors.As(err, &codeErr))
Expand All @@ -229,7 +243,7 @@ func TestReadTokenAuthResult(t *testing.T) {
require.NoError(t, err)
}()

err := readTokenAuthResult(cli)
_, err := readTokenAuthResult(cli)
require.Error(t, err)
codeErr := (*codeError)(nil)
require.True(t, errors.As(err, &codeErr))
Expand All @@ -238,6 +252,7 @@ func TestReadTokenAuthResult(t *testing.T) {

t.Run("successful", func(t *testing.T) {
cli, srv := net.Pipe()
crdbBackendKeyData := &pgproto3.BackendKeyData{ProcessID: 42, SecretKey: 99}

go func() {
_, err := srv.Write((&pgproto3.AuthenticationOk{}).Encode(nil))
Expand All @@ -246,13 +261,15 @@ func TestReadTokenAuthResult(t *testing.T) {
_, err = srv.Write((&pgproto3.ParameterStatus{Name: "Server Version", Value: "1.3"}).Encode(nil))
require.NoError(t, err)

_, err = srv.Write((&pgproto3.BackendKeyData{ProcessID: uint32(42)}).Encode(nil))
_, err = srv.Write(crdbBackendKeyData.Encode(nil))
require.NoError(t, err)

_, err = srv.Write((&pgproto3.ReadyForQuery{}).Encode(nil))
require.NoError(t, err)
}()

require.NoError(t, readTokenAuthResult(cli))
receivedCrdbBackendKeyData, err := readTokenAuthResult(cli)
require.NoError(t, err)
require.Equal(t, crdbBackendKeyData, receivedCrdbBackendKeyData)
})
}
16 changes: 11 additions & 5 deletions pkg/ccl/sqlproxyccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,10 @@ type connector struct {
// DialTenantRetries counts how often dialing a tenant is retried.
DialTenantRetries *metric.Counter

// CancelInfo contains the data used to implement pgwire query cancellation.
// It is only populated after authenticating the connection.
CancelInfo *cancelInfo

// Testing knobs for internal connector calls. If specified, these will
// be called instead of the actual logic.
testingKnobs struct {
Expand Down Expand Up @@ -116,11 +120,11 @@ func (c *connector) OpenTenantConnWithToken(
// Since this method is only used during connection migration (i.e. proxy
// is connecting to the SQL pod), we'll discard all of the messages, and
// only return once we've seen a ReadyForQuery message.
//
// NOTE: This will need to be updated when we implement query cancellation.
if err := readTokenAuthResult(serverConn); err != nil {
newBackendKeyData, err := readTokenAuthResult(serverConn)
if err != nil {
return nil, err
}
c.CancelInfo.setNewBackend(newBackendKeyData, serverConn.RemoteAddr().(*net.TCPAddr))
log.Infof(ctx, "connected to %s through token-based auth", serverConn.RemoteAddr())
return serverConn, nil
}
Expand All @@ -139,7 +143,7 @@ func (c *connector) OpenTenantConnWithAuth(
requester balancer.ConnectionHandle,
clientConn net.Conn,
throttleHook func(throttler.AttemptStatus) error,
) (retServerConn net.Conn, sentToClient bool, retErr error) {
) (retServerConnection net.Conn, sentToClient bool, retErr error) {
// Just a safety check, but this shouldn't happen since we will block the
// startup param in the frontend admitter. The only case where we actually
// need to delete this param is if OpenTenantConnWithToken was called
Expand All @@ -158,10 +162,12 @@ func (c *connector) OpenTenantConnWithAuth(

// Perform user authentication for non-token-based auth methods. This will
// block until the server has authenticated the client.
if err := authenticate(clientConn, serverConn, throttleHook); err != nil {
crdbBackendKeyData, err := authenticate(clientConn, serverConn, c.CancelInfo.proxyBackendKeyData, throttleHook)
if err != nil {
return nil, true, err
}
log.Infof(ctx, "connected to %s through normal auth", serverConn.RemoteAddr())
c.CancelInfo.setNewBackend(crdbBackendKeyData, serverConn.RemoteAddr().(*net.TCPAddr))
return serverConn, false, nil
}

Expand Down

0 comments on commit d58473e

Please sign in to comment.