Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
76172: server: enhance comments surrounding draining r=knz a=cameronnunez

This patch improves the comments related to the draining process,
fixing incorrect statements and providing more details to make
the process more easily understood.

Release note: None

76409: sql: block FKs to/from TTL tables r=otan a=otan

This table blocks FKs to and from TTL tables. Note the RFC previously
only said to, but I actually thought of a new case and decided that is
was probably prudent to block this both ways. RFC is reflected
accordingly.

Release note: None

76417: ccl/sqlproxyccl: add connector component and support for session revival token r=JeffSwenson a=jaylim-crl

Informs #76000.

Previously, all the connection establishment logic is coupled with the handler
function within proxy_handler.go. This makes connecting to a new SQL pod during
connection migration difficult. This commit refactors all of those connection
logic out of the proxy handler into a connector component, as described in the
connection migration RFC. At the same time, we also add support for the session
revival token within this connector component.

Note that the overall behavior of the SQL proxy should be unchanged with this
commit.

Release note: None

76534: storageccl,cloud: plumb ctx to external file Readers r=andreimatei a=andreimatei

This patch changes the interface implemented by all the cloud-specific
file readers to take a ctx for their Read() and Close() methods. Before,
some implementations were capturing a ctx at construction time, which is
generally a bad thing to do - operations should generally run in the
caller's ctx, rather than in some ctx captured in the past. Besides
being generally bad, it's also dangeroursd: in two instances,
cloud.ResumingReader was capturing a ctx that was immediately becoming
invalid because the tracing span in it was being closed[1] (so when the
ResumingReader was trying to use the captured ctx, that was a
use-after-finish of the respective tracing span). A stack trace show
such a use-after-Finish is captured in [2].

This patch improves the situation by plumbing ctxs to the Read()s. Now,
ResumingReader no longer captures a ctx. The plumbing has not spread
anywhere because of practical reasons. Unfortunately, I had to make
storageccl.sstReader capture a ctx (the sstReader wraps file readers
such as cloud.ResumingReader). If contexts are to be used at all,
someone needs to capture one since the sstReader is used through the
pebble.sstable.Reader interface which does not plumb contexts.
This lifting up of the context capture from ResumingReader to sstReader
is a good thing since sstReader is not a broadly used type and the
capturing of the context is now more visible; it's no longer hidden many
layers down. Also, the use-after-Finish in the ResumingReader is fixed
because the higher-level ctx captured by the sstReader (a DistSQL
processor ctx) lives long enough.

[1]
https://github.com/cockroachdb/cockroach/blob/3be8121ad9e96416c49c27947facfa18831794b9/pkg/cloud/gcp/gcs_storage.go#L186
and also
https://github.com/cockroachdb/cockroach/blob/0a6b7427603c268b41e39d7acba917c1227859ca/pkg/cloud/amazon/s3_storage.go#L497

[2] https://gist.github.com/andreimatei/7b86d414a93c81cf93ffe68fe2788ec6

Release note: None

76545: cmd/reduce: add -tlp option r=yuzefovich a=yuzefovich

**cmd/reduce: remove stdin option and require -file argument**

We tend to not use the option of passing input SQL via stdin, so this
commit removes it. An additional argument in favor of doing that is that
the follow-up commit will introduce another mode of behavior that
requires `-file` argument to be specified, so it's just cleaner to
always require it now.

Release note: None

**cmd/reduce: add -tlp option**

This commit adds `-tlp` boolean flag that changes the behavior of
`reduce`. It is required that `-file` is specified whenever the `-tlp`
flag is used. The behavior is such that the last two queries (delimited
by empty lines) in the file contain unpartitioned and partitioned queries
that return different results although they are equivalent.

If TLP check is requested, then we remove the last two queries from the
input which we use then to construct a special TLP check query that
results in an error if two removed queries return different results.

We do not just include the TLP check query into the input string because
the reducer would then reduce the check query itself, making the
reduction meaningless.

Release note: None

76598: server: use channel for DisableAutomaticVersionUpgrade r=RaduBerinde a=RaduBerinde

DisableAutomaticVersionUpgrade is an atomic integer which is rechecked
in a retry loop. This is not a very clean mechanism, and can lead to
issues where you're unknowingly dealing with a copy of the knobs and
setting the wrong atomic. The retry loop can also add unnecessary
delays in tests.

This commit changes DisableAutomaticVersionUpgrade from an atomic
integer to a channel. If the channel is set, auto-upgrade waits until
the channel is closed.

Release note: None

Co-authored-by: Cameron Nunez <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
Co-authored-by: Jay <[email protected]>
Co-authored-by: Andrei Matei <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
7 people committed Feb 15, 2022
7 parents 86db5d0 + b7dfd62 + 3dafe1b + 21ee3bd + 6a879f7 + 4d41f10 + b2e112b commit 1452123
Show file tree
Hide file tree
Showing 103 changed files with 1,748 additions and 479 deletions.
14 changes: 10 additions & 4 deletions docs/RFCS/20220120_row_level_ttl.md
Original file line number Diff line number Diff line change
Expand Up @@ -320,10 +320,16 @@ Rows that have expired their TTL can be optionally removed from all SQL query
results. Work for this is required at the optimizer layer. However, this is not
planned to be implemented for the first iteration of TTL.

## Foreign Keys to a TTL Table
To avoid additional complexity in the initial implementation, foreign keys to
TTL tables will not be permitted. More thought has to be put on ON
DELETE/ON UPDATE CASCADEs before we can look at allowing this functionality.
## Foreign Keys to/from TTL Tables
To avoid additional complexity in the initial implementation, foreign keys (FK)
to and from TTL tables will not be permitted due to complexities with the
implementation which are complex to handle, for example:
* When having a non-TTL table with a FK dependent on a TTL table with an
`ON UPDATE/DELETE CASCADE`, the non-TTL table need to hide any rows which
are linked to an expired TTL row.
* When having a TTL table with an FK dependent on a non-TTL table,
`ON DELETE RESTRICT` should only block a delete on the non-TTL table
if the row has expired.

## Introspection
The TTL definition for the table will appear in `SHOW CREATE TABLE`. The options
Expand Down
6 changes: 3 additions & 3 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -60,9 +60,9 @@ server.oidc_authentication.provider_url string sets OIDC provider URL ({provide
server.oidc_authentication.redirect_url string https://localhost:8080/oidc/v1/callback sets OIDC redirect URL via a URL string or a JSON string containing a required `redirect_urls` key with an object that maps from region keys to URL strings (URLs should point to your load balancer and must route to the path /oidc/v1/callback)
server.oidc_authentication.scopes string openid sets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`)
server.rangelog.ttl duration 720h0m0s if nonzero, range log entries older than this duration are deleted every 10m0s. Should not be lowered below 24 hours.
server.shutdown.drain_wait duration 0s the amount of time a server waits in an unready state before proceeding with the rest of the shutdown process (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)
server.shutdown.lease_transfer_wait duration 5s the amount of time a server waits to transfer range leases before proceeding with the rest of the shutdown process (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)
server.shutdown.query_wait duration 10s the server will wait for at least this amount of time for active queries to finish (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)
server.shutdown.drain_wait duration 0s the amount of time a server waits in an unready state before proceeding with a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)
server.shutdown.lease_transfer_wait duration 5s the timeout for a single iteration of the range lease transfer phase of draining (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)
server.shutdown.query_wait duration 10s the timeout for waiting for active queries to finish during a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)
server.time_until_store_dead duration 5m0s the time after which if there is no new gossiped information about a store, it is considered dead
server.user_login.cert_password_method.auto_scram_promotion.enabled boolean true whether to automatically promote cert-password authentication to use SCRAM
server.user_login.min_password_length integer 1 the minimum length accepted for passwords set in cleartext via SQL. Note that a value lower than 1 is ignored: passwords cannot be empty in any case.
Expand Down
6 changes: 3 additions & 3 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -71,9 +71,9 @@
<tr><td><code>server.oidc_authentication.redirect_url</code></td><td>string</td><td><code>https://localhost:8080/oidc/v1/callback</code></td><td>sets OIDC redirect URL via a URL string or a JSON string containing a required `redirect_urls` key with an object that maps from region keys to URL strings (URLs should point to your load balancer and must route to the path /oidc/v1/callback) </td></tr>
<tr><td><code>server.oidc_authentication.scopes</code></td><td>string</td><td><code>openid</code></td><td>sets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`)</td></tr>
<tr><td><code>server.rangelog.ttl</code></td><td>duration</td><td><code>720h0m0s</code></td><td>if nonzero, range log entries older than this duration are deleted every 10m0s. Should not be lowered below 24 hours.</td></tr>
<tr><td><code>server.shutdown.drain_wait</code></td><td>duration</td><td><code>0s</code></td><td>the amount of time a server waits in an unready state before proceeding with the rest of the shutdown process (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)</td></tr>
<tr><td><code>server.shutdown.lease_transfer_wait</code></td><td>duration</td><td><code>5s</code></td><td>the amount of time a server waits to transfer range leases before proceeding with the rest of the shutdown process (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)</td></tr>
<tr><td><code>server.shutdown.query_wait</code></td><td>duration</td><td><code>10s</code></td><td>the server will wait for at least this amount of time for active queries to finish (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)</td></tr>
<tr><td><code>server.shutdown.drain_wait</code></td><td>duration</td><td><code>0s</code></td><td>the amount of time a server waits in an unready state before proceeding with a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)</td></tr>
<tr><td><code>server.shutdown.lease_transfer_wait</code></td><td>duration</td><td><code>5s</code></td><td>the timeout for a single iteration of the range lease transfer phase of draining (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)</td></tr>
<tr><td><code>server.shutdown.query_wait</code></td><td>duration</td><td><code>10s</code></td><td>the timeout for waiting for active queries to finish during a drain (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting)</td></tr>
<tr><td><code>server.time_until_store_dead</code></td><td>duration</td><td><code>5m0s</code></td><td>the time after which if there is no new gossiped information about a store, it is considered dead</td></tr>
<tr><td><code>server.user_login.cert_password_method.auto_scram_promotion.enabled</code></td><td>boolean</td><td><code>true</code></td><td>whether to automatically promote cert-password authentication to use SCRAM</td></tr>
<tr><td><code>server.user_login.min_password_length</code></td><td>integer</td><td><code>1</code></td><td>the minimum length accepted for passwords set in cleartext via SQL. Note that a value lower than 1 is ignored: passwords cannot be empty in any case.</td></tr>
Expand Down
2 changes: 2 additions & 0 deletions pkg/blobs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_library(
"//pkg/rpc",
"//pkg/rpc/nodedialer",
"//pkg/util/fileutil",
"//pkg/util/ioctx",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//oserror",
"@org_golang_google_grpc//codes",
Expand Down Expand Up @@ -45,6 +46,7 @@ go_test(
"//pkg/testutils",
"//pkg/util",
"//pkg/util/hlc",
"//pkg/util/ioctx",
"//pkg/util/leaktest",
"//pkg/util/netutil",
"//pkg/util/stop",
Expand Down
3 changes: 2 additions & 1 deletion pkg/blobs/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -96,7 +97,7 @@ func benchmarkStreamingReadFile(b *testing.B, tc *benchmarkTestCase) {
if err != nil {
b.Fatal(err)
}
if _, err := io.Copy(w, reader); err != nil {
if _, err := io.Copy(w, ioctx.ReaderCtxAdapter(ctx, reader)); err != nil {
b.Fatal(errors.CombineErrors(err, w.Close()))
}
if err := w.Close(); err != nil {
Expand Down
7 changes: 4 additions & 3 deletions pkg/blobs/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/errors"
"google.golang.org/grpc/metadata"
)
Expand All @@ -30,7 +31,7 @@ type BlobClient interface {
// ReadFile fetches the named payload from the requested node,
// and stores it in memory. It then returns an io.ReadCloser to
// read the contents.
ReadFile(ctx context.Context, file string, offset int64) (io.ReadCloser, int64, error)
ReadFile(ctx context.Context, file string, offset int64) (ioctx.ReadCloserCtx, int64, error)

// Writer opens the named payload on the requested node for writing.
Writer(ctx context.Context, file string) (io.WriteCloser, error)
Expand Down Expand Up @@ -61,7 +62,7 @@ func newRemoteClient(blobClient blobspb.BlobClient) BlobClient {

func (c *remoteClient) ReadFile(
ctx context.Context, file string, offset int64,
) (io.ReadCloser, int64, error) {
) (ioctx.ReadCloserCtx, int64, error) {
// Check that file exists before reading from it and get size to return.
st, err := c.Stat(ctx, file)
if err != nil {
Expand Down Expand Up @@ -156,7 +157,7 @@ func NewLocalClient(externalIODir string) (BlobClient, error) {

func (c *localClient) ReadFile(
ctx context.Context, file string, offset int64,
) (io.ReadCloser, int64, error) {
) (ioctx.ReadCloserCtx, int64, error) {
return c.localStorage.ReadFile(file, offset)
}

Expand Down
3 changes: 2 additions & 1 deletion pkg/blobs/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/netutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
Expand Down Expand Up @@ -191,7 +192,7 @@ func TestBlobClientReadFile(t *testing.T) {
t.Fatal(err)
}
// Check that fetched file content is correct
content, err := ioutil.ReadAll(reader)
content, err := ioctx.ReadAll(ctx, reader)
if err != nil {
t.Fatal(err)
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/blobs/local_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/blobs/blobspb"
"github.com/cockroachdb/cockroach/pkg/util/fileutil"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -133,7 +134,7 @@ func (l *LocalStorage) Writer(ctx context.Context, filename string) (io.WriteClo
// ReadFile prepends IO dir to filename and reads the content of that local file.
func (l *LocalStorage) ReadFile(
filename string, offset int64,
) (res io.ReadCloser, size int64, err error) {
) (res ioctx.ReadCloserCtx, size int64, err error) {
fullPath, err := l.prependExternalIODir(filename)
if err != nil {
return nil, 0, err
Expand Down Expand Up @@ -161,7 +162,7 @@ func (l *LocalStorage) ReadFile(
return nil, 0, errors.Errorf("seek to offset %d returned %d", offset, ret)
}
}
return f, fi.Size(), nil
return ioctx.ReadCloserAdapter(f), fi.Size(), nil
}

// List prepends IO dir to pattern and glob matches all local files against that pattern.
Expand Down
9 changes: 5 additions & 4 deletions pkg/blobs/service.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"io"

"github.com/cockroachdb/cockroach/pkg/blobs/blobspb"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/errors/oserror"
"google.golang.org/grpc/codes"
Expand All @@ -56,8 +57,8 @@ func (s *Service) GetStream(req *blobspb.GetRequest, stream blobspb.Blob_GetStre
if err != nil {
return err
}
defer content.Close()
return streamContent(stream, content)
defer content.Close(stream.Context())
return streamContent(stream.Context(), stream, content)
}

// PutStream implements the gRPC service.
Expand All @@ -71,7 +72,7 @@ func (s *Service) PutStream(stream blobspb.Blob_PutStreamServer) error {
return errors.New("no filename in metadata")
}
reader := newPutStreamReader(stream)
defer reader.Close()
defer reader.Close(stream.Context())
ctx, cancel := context.WithCancel(stream.Context())
defer cancel()

Expand All @@ -80,7 +81,7 @@ func (s *Service) PutStream(stream blobspb.Blob_PutStreamServer) error {
cancel()
return err
}
if _, err := io.Copy(w, reader); err != nil {
if _, err := io.Copy(w, ioctx.ReaderCtxAdapter(stream.Context(), reader)); err != nil {
cancel()
return errors.CombineErrors(w.Close(), err)
}
Expand Down
16 changes: 9 additions & 7 deletions pkg/blobs/stream.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,11 @@
package blobs

import (
"context"
"io"

"github.com/cockroachdb/cockroach/pkg/blobs/blobspb"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
)

// Within the blob service, streaming is used in two functions:
Expand All @@ -35,7 +37,7 @@ var chunkSize = 128 * 1 << 10

// blobStreamReader implements a ReadCloser which receives
// gRPC streaming messages.
var _ io.ReadCloser = &blobStreamReader{}
var _ ioctx.ReadCloserCtx = &blobStreamReader{}

type streamReceiver interface {
SendAndClose(*blobspb.StreamResponse) error
Expand All @@ -55,15 +57,15 @@ func (*nopSendAndClose) SendAndClose(*blobspb.StreamResponse) error {

// newGetStreamReader creates an io.ReadCloser that uses gRPC's streaming API
// to read chunks of data.
func newGetStreamReader(client blobspb.Blob_GetStreamClient) io.ReadCloser {
func newGetStreamReader(client blobspb.Blob_GetStreamClient) ioctx.ReadCloserCtx {
return &blobStreamReader{
stream: &nopSendAndClose{client},
}
}

// newPutStreamReader creates an io.ReadCloser that uses gRPC's streaming API
// to read chunks of data.
func newPutStreamReader(client blobspb.Blob_PutStreamServer) io.ReadCloser {
func newPutStreamReader(client blobspb.Blob_PutStreamServer) ioctx.ReadCloserCtx {
return &blobStreamReader{stream: client}
}

Expand All @@ -74,7 +76,7 @@ type blobStreamReader struct {
EOFReached bool
}

func (r *blobStreamReader) Read(out []byte) (int, error) {
func (r *blobStreamReader) Read(ctx context.Context, out []byte) (int, error) {
if r.EOFReached {
return 0, io.EOF
}
Expand Down Expand Up @@ -115,7 +117,7 @@ func (r *blobStreamReader) Read(out []byte) (int, error) {
return offset, nil
}

func (r *blobStreamReader) Close() error {
func (r *blobStreamReader) Close(ctx context.Context) error {
return r.stream.SendAndClose(&blobspb.StreamResponse{})
}

Expand All @@ -126,11 +128,11 @@ type streamSender interface {
// streamContent splits the content into chunks, of size `chunkSize`,
// and streams those chunks to sender.
// Note: This does not close the stream.
func streamContent(sender streamSender, content io.Reader) error {
func streamContent(ctx context.Context, sender streamSender, content ioctx.ReaderCtx) error {
payload := make([]byte, chunkSize)
var chunk blobspb.StreamChunk
for {
n, err := content.Read(payload)
n, err := content.Read(ctx, payload)
if n > 0 {
chunk.Payload = payload[:n]
err = sender.Send(&chunk)
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,7 @@ go_library(
"//pkg/util/errorutil/unimplemented",
"//pkg/util/hlc",
"//pkg/util/interval",
"//pkg/util/ioctx",
"//pkg/util/log",
"//pkg/util/log/eventpb",
"//pkg/util/metric",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backup_destination.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,6 @@ package backupccl

import (
"context"
"io/ioutil"
"net/url"
"path"

Expand All @@ -22,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -316,7 +316,7 @@ func readLatestFile(
}
return "", pgerror.WithCandidateCode(err, pgcode.Io)
}
latest, err := ioutil.ReadAll(latestFile)
latest, err := ioctx.ReadAll(ctx, latestFile)
if err != nil {
return "", err
}
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 @@ -564,7 +564,7 @@ func checkForExistingBackupsInCollection(
r, err := defaultStore.ReadFile(ctx, latestFileName)
if err == nil {
// A full backup has already been taken to this location.
r.Close()
r.Close(ctx)
return errors.Newf("backups already created in %s; to ignore existing backups, "+
"the schedule can be created with the 'ignore_existing_backups' option",
collectionURI)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ func TestInsertMissingPublicSchemaNamespaceEntry(t *testing.T) {
ExternalIODir: dir,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: 1,
DisableAutomaticVersionUpgrade: make(chan struct{}),
BinaryVersionOverride: clusterversion.ByKey(clusterversion.InsertPublicSchemaNamespaceEntryOnRestore - 1),
},
},
Expand Down
Loading

0 comments on commit 1452123

Please sign in to comment.