diff --git a/pkg/ccl/backupccl/alter_backup_schedule.go b/pkg/ccl/backupccl/alter_backup_schedule.go index 4cacaf04ce8b..4ff80bc47645 100644 --- a/pkg/ccl/backupccl/alter_backup_schedule.go +++ b/pkg/ccl/backupccl/alter_backup_schedule.go @@ -232,15 +232,15 @@ func emitAlteredSchedule( ) error { to := make([]string, len(stmt.To)) for i, dest := range stmt.To { - to[i] = tree.AsStringWithFlags(dest, tree.FmtBareStrings) + to[i] = tree.AsStringWithFlags(dest, tree.FmtBareStrings|tree.FmtShowFullURIs) } kmsURIs := make([]string, len(stmt.Options.EncryptionKMSURI)) for i, kmsURI := range stmt.Options.EncryptionKMSURI { - kmsURIs[i] = tree.AsStringWithFlags(kmsURI, tree.FmtBareStrings) + kmsURIs[i] = tree.AsStringWithFlags(kmsURI, tree.FmtBareStrings|tree.FmtShowFullURIs) } incDests := make([]string, len(stmt.Options.IncrementalStorage)) for i, incDest := range stmt.Options.IncrementalStorage { - incDests[i] = tree.AsStringWithFlags(incDest, tree.FmtBareStrings) + incDests[i] = tree.AsStringWithFlags(incDest, tree.FmtBareStrings|tree.FmtShowFullURIs) } if err := emitSchedule(job, stmt, to, nil, /* incrementalFrom */ kmsURIs, incDests, resultsCh); err != nil { diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index bff11b9feb41..53ed3bc204e3 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -181,7 +181,9 @@ func backupJobDescription( } ann := p.ExtendedEvalContext().Annotations - return tree.AsStringWithFQNames(b, ann), nil + return tree.AsStringWithFlags( + b, tree.FmtAlwaysQualifyNames|tree.FmtShowFullURIs, tree.FmtAnnotations(ann), + ), nil } // annotatedBackupStatement is a tree.Backup, optionally diff --git a/pkg/ccl/backupccl/backuppb/backup.go b/pkg/ccl/backupccl/backuppb/backup.go index 95b71caa5fc0..7f9289db03a0 100644 --- a/pkg/ccl/backupccl/backuppb/backup.go +++ b/pkg/ccl/backupccl/backuppb/backup.go @@ -162,7 +162,7 @@ func (m ScheduledBackupExecutionArgs) MarshalJSONPB(marshaller *jsonpb.Marshaler backup.Options.EncryptionPassphrase = tree.NewDString("redacted") } - m.BackupStatement = backup.String() + m.BackupStatement = tree.AsStringWithFlags(backup, tree.FmtShowFullURIs) return json.Marshal(m) } diff --git a/pkg/ccl/backupccl/create_scheduled_backup.go b/pkg/ccl/backupccl/create_scheduled_backup.go index 1e6a9ff6bdcd..a2ca4a3d82e4 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup.go +++ b/pkg/ccl/backupccl/create_scheduled_backup.go @@ -556,7 +556,7 @@ func emitSchedule( tree.NewDString(status), nextRun, tree.NewDString(sj.ScheduleExpr()), - tree.NewDString(tree.AsString(redactedBackupNode)), + tree.NewDString(tree.AsStringWithFlags(redactedBackupNode, tree.FmtShowFullURIs)), } return nil } diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 96092b68feaa..fd4bfe54d09a 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -1157,7 +1157,9 @@ func restoreJobDescription( } ann := p.ExtendedEvalContext().Annotations - return tree.AsStringWithFQNames(r, ann), nil + return tree.AsStringWithFlags( + r, tree.FmtAlwaysQualifyNames|tree.FmtShowFullURIs, tree.FmtAnnotations(ann), + ), nil } func restoreTypeCheck( diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index ec7d0e17c8e5..e788ef03ab95 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -999,7 +999,7 @@ func changefeedJobDescription( return "", err } sort.Slice(c.Options, func(i, j int) bool { return c.Options[i].Key < c.Options[j].Key }) - return tree.AsString(c), nil + return tree.AsStringWithFlags(c, tree.FmtShowFullURIs), nil } func logSanitizedChangefeedDestination(ctx context.Context, destination string) { diff --git a/pkg/ccl/changefeedccl/changefeedpb/marshal.go b/pkg/ccl/changefeedccl/changefeedpb/marshal.go index 9ae702a40505..edc808e09b1f 100644 --- a/pkg/ccl/changefeedccl/changefeedpb/marshal.go +++ b/pkg/ccl/changefeedccl/changefeedpb/marshal.go @@ -40,6 +40,6 @@ func (m ScheduledChangefeedExecutionArgs) MarshalJSONPB(x *jsonpb.Marshaler) ([] } export.SinkURI = tree.NewDString(sinkURI) - m.ChangefeedStatement = export.String() + m.ChangefeedStatement = tree.AsStringWithFlags(export, tree.FmtShowFullURIs) return json.Marshal(m) } diff --git a/pkg/ccl/changefeedccl/testfeed_test.go b/pkg/ccl/changefeedccl/testfeed_test.go index 43099336ef8b..6320d9f8cdee 100644 --- a/pkg/ccl/changefeedccl/testfeed_test.go +++ b/pkg/ccl/changefeedccl/testfeed_test.go @@ -304,7 +304,7 @@ func (e *externalConnectionFeedFactory) Feed( } createStmt.SinkURI = tree.NewStrVal(`external://` + randomExternalConnectionName) - return e.TestFeedFactory.Feed(createStmt.String(), args...) + return e.TestFeedFactory.Feed(tree.AsStringWithFlags(createStmt, tree.FmtShowPasswords), args...) } func setURI( @@ -881,7 +881,7 @@ func (f *tableFeedFactory) Feed( return nil, err } - if err := f.startFeedJob(c.jobFeed, createStmt.String(), args...); err != nil { + if err := f.startFeedJob(c.jobFeed, tree.AsStringWithFlags(createStmt, tree.FmtShowPasswords), args...); err != nil { return nil, err } return c, nil @@ -1124,7 +1124,7 @@ func (f *cloudFeedFactory) Feed( dir: feedDir, isBare: createStmt.Select != nil && !explicitEnvelope, } - if err := f.startFeedJob(c.jobFeed, createStmt.String(), args...); err != nil { + if err := f.startFeedJob(c.jobFeed, tree.AsStringWithFlags(createStmt, tree.FmtShowPasswords), args...); err != nil { return nil, err } return c, nil @@ -1822,7 +1822,7 @@ func (k *kafkaFeedFactory) Feed(create string, args ...interface{}) (cdctest.Tes registry: registry, } - if err := k.startFeedJob(c.jobFeed, createStmt.String(), args...); err != nil { + if err := k.startFeedJob(c.jobFeed, tree.AsStringWithFlags(createStmt, tree.FmtShowPasswords), args...); err != nil { return nil, errors.CombineErrors(err, c.Close()) } return c, nil @@ -2052,7 +2052,7 @@ func (f *webhookFeedFactory) Feed(create string, args ...interface{}) (cdctest.T isBare: createStmt.Select != nil && !explicitEnvelope, mockSink: sinkDest, } - if err := f.startFeedJob(c.jobFeed, createStmt.String(), args...); err != nil { + if err := f.startFeedJob(c.jobFeed, tree.AsStringWithFlags(createStmt, tree.FmtShowPasswords), args...); err != nil { sinkDest.Close() return nil, err } @@ -2448,7 +2448,7 @@ func (p *pubsubFeedFactory) Feed(create string, args ...interface{}) (cdctest.Te deprecatedClient: deprecatedClient, } - if err := p.startFeedJob(c.jobFeed, createStmt.String(), args...); err != nil { + if err := p.startFeedJob(c.jobFeed, tree.AsStringWithFlags(createStmt, tree.FmtShowPasswords), args...); err != nil { _ = mockServer.Close() return nil, err } @@ -2723,7 +2723,7 @@ func (p *pulsarFeedFactory) Feed(create string, args ...interface{}) (cdctest.Te pulsarServer: mockServer, } - if err := p.startFeedJob(c.jobFeed, createStmt.String(), args...); err != nil { + if err := p.startFeedJob(c.jobFeed, tree.AsStringWithFlags(createStmt, tree.FmtShowPasswords), args...); err != nil { return nil, err } return c, nil diff --git a/pkg/ccl/telemetryccl/BUILD.bazel b/pkg/ccl/telemetryccl/BUILD.bazel index e9cde6da3dec..069a4ca7ddd0 100644 --- a/pkg/ccl/telemetryccl/BUILD.bazel +++ b/pkg/ccl/telemetryccl/BUILD.bazel @@ -21,6 +21,7 @@ go_test( "//pkg/security/securitytest", "//pkg/server", "//pkg/sql", + "//pkg/sql/sem/tree", "//pkg/sql/sqltestutils", "//pkg/testutils", "//pkg/testutils/serverutils", diff --git a/pkg/ccl/telemetryccl/telemetry_logging_test.go b/pkg/ccl/telemetryccl/telemetry_logging_test.go index 59b4d6560b60..2aaa099633ac 100644 --- a/pkg/ccl/telemetryccl/telemetry_logging_test.go +++ b/pkg/ccl/telemetryccl/telemetry_logging_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/sql" + "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/sqlutils" @@ -267,7 +268,9 @@ func TestBulkJobTelemetryLogging(t *testing.T) { query: fmt.Sprintf(`IMPORT INTO a CSV DATA ('%s')`, srv.URL), sampleQueryEvent: expectedSampleQueryEvent{ eventType: "import", - stmt: fmt.Sprintf(`IMPORT INTO defaultdb.public.a CSV DATA ('%s')`, srv.URL), + stmt: fmt.Sprintf( + `IMPORT INTO defaultdb.public.a CSV DATA (%s)`, tree.PasswordSubstitution, + ), }, recoveryEvent: expectedRecoveryEvent{ numRows: 3, @@ -279,7 +282,10 @@ func TestBulkJobTelemetryLogging(t *testing.T) { query: fmt.Sprintf(`IMPORT INTO a CSV DATA ('%s') WITH detached`, srv.URL), sampleQueryEvent: expectedSampleQueryEvent{ eventType: "import", - stmt: fmt.Sprintf(`IMPORT INTO defaultdb.public.a CSV DATA ('%s') WITH OPTIONS (detached)`, srv.URL), + stmt: fmt.Sprintf( + `IMPORT INTO defaultdb.public.a CSV DATA (%s) WITH OPTIONS (detached)`, + tree.PasswordSubstitution, + ), }, recoveryEvent: expectedRecoveryEvent{ numRows: 3, @@ -291,7 +297,7 @@ func TestBulkJobTelemetryLogging(t *testing.T) { query: fmt.Sprintf(`BACKUP DATABASE mydb INTO '%s'`, nodelocal.MakeLocalStorageURI("test1")), sampleQueryEvent: expectedSampleQueryEvent{ eventType: "backup", - stmt: fmt.Sprintf(`BACKUP DATABASE mydb INTO '%s'`, nodelocal.MakeLocalStorageURI("test1")), + stmt: fmt.Sprintf(`BACKUP DATABASE mydb INTO %s`, tree.PasswordSubstitution), }, recoveryEvent: expectedRecoveryEvent{ numRows: 3, @@ -303,7 +309,7 @@ func TestBulkJobTelemetryLogging(t *testing.T) { query: fmt.Sprintf(`BACKUP DATABASE mydb INTO '%s' WITH detached`, nodelocal.MakeLocalStorageURI("test1")), sampleQueryEvent: expectedSampleQueryEvent{ eventType: "backup", - stmt: fmt.Sprintf(`BACKUP DATABASE mydb INTO '%s' WITH OPTIONS (detached)`, nodelocal.MakeLocalStorageURI("test1")), + stmt: fmt.Sprintf(`BACKUP DATABASE mydb INTO %s WITH OPTIONS (detached)`, tree.PasswordSubstitution), }, recoveryEvent: expectedRecoveryEvent{ numRows: 3, @@ -315,7 +321,7 @@ func TestBulkJobTelemetryLogging(t *testing.T) { query: fmt.Sprintf(`RESTORE DATABASE mydb FROM LATEST IN '%s'`, nodelocal.MakeLocalStorageURI("test1")), sampleQueryEvent: expectedSampleQueryEvent{ eventType: "restore", - stmt: fmt.Sprintf(`RESTORE DATABASE mydb FROM 'latest' IN '%s'`, nodelocal.MakeLocalStorageURI("test1")), + stmt: fmt.Sprintf(`RESTORE DATABASE mydb FROM 'latest' IN %s`, tree.PasswordSubstitution), }, recoveryEvent: expectedRecoveryEvent{ numRows: 3, @@ -327,7 +333,7 @@ func TestBulkJobTelemetryLogging(t *testing.T) { query: fmt.Sprintf(`RESTORE DATABASE mydb FROM LATEST IN '%s' WITH detached`, nodelocal.MakeLocalStorageURI("test1")), sampleQueryEvent: expectedSampleQueryEvent{ eventType: "restore", - stmt: fmt.Sprintf(`RESTORE DATABASE mydb FROM 'latest' IN '%s' WITH OPTIONS (detached)`, nodelocal.MakeLocalStorageURI("test1")), + stmt: fmt.Sprintf(`RESTORE DATABASE mydb FROM 'latest' IN %s WITH OPTIONS (detached)`, tree.PasswordSubstitution), }, recoveryEvent: expectedRecoveryEvent{ numRows: 3, diff --git a/pkg/cloud/externalconn/record.go b/pkg/cloud/externalconn/record.go index 74d4a421e3f4..d7e8882cea68 100644 --- a/pkg/cloud/externalconn/record.go +++ b/pkg/cloud/externalconn/record.go @@ -178,7 +178,7 @@ func (e *MutableExternalConnection) UnredactedConnectionStatement() string { }, As: tree.NewDString(e.rec.ConnectionDetails.UnredactedURI()), } - return tree.AsString(ecNode) + return tree.AsStringWithFlags(ecNode, tree.FmtShowFullURIs) } // RedactedConnectionURI implements the ExternalConnection interface and diff --git a/pkg/kv/kvserver/below_raft_protos_test.go b/pkg/kv/kvserver/below_raft_protos_test.go index 0214d9d6c82b..de7e9734a32c 100644 --- a/pkg/kv/kvserver/below_raft_protos_test.go +++ b/pkg/kv/kvserver/below_raft_protos_test.go @@ -54,6 +54,7 @@ func TestBelowRaftProtosDontChange(t *testing.T) { Term uint64 Vote raftpb.PeerID Commit uint64 + Lead raftpb.PeerID } // Conversion fails if new fields are added to `HardState`, in which case this method // and the expected sums should be updated. @@ -64,6 +65,7 @@ func TestBelowRaftProtosDontChange(t *testing.T) { Term: n % 3, Vote: raftpb.PeerID(n % 7), Commit: n % 11, + Lead: raftpb.PeerID(n % 13), } }, func(r *rand.Rand) protoutil.Message { diff --git a/pkg/kv/kvserver/logstore/logstore.go b/pkg/kv/kvserver/logstore/logstore.go index 3ad62c8aeedc..4e289ca5cc5f 100644 --- a/pkg/kv/kvserver/logstore/logstore.go +++ b/pkg/kv/kvserver/logstore/logstore.go @@ -202,6 +202,7 @@ func (s *LogStore) storeEntriesAndCommitBatch( Term: m.Term, Vote: m.Vote, Commit: m.Commit, + Lead: m.Lead, } if !raft.IsEmptyHardState(hs) { // NB: Note that without additional safeguards, it's incorrect to write diff --git a/pkg/kv/kvserver/logstore/stateloader.go b/pkg/kv/kvserver/logstore/stateloader.go index 0e907cfe5ec1..e6658ab510ba 100644 --- a/pkg/kv/kvserver/logstore/stateloader.go +++ b/pkg/kv/kvserver/logstore/stateloader.go @@ -174,6 +174,11 @@ func (sl StateLoader) SynthesizeHardState( return errors.Newf("can't decrease HardState.Commit from %d to %d", redact.Safe(oldHS.Commit), redact.Safe(newHS.Commit)) } + + // TODO(arul): This function can be called with an empty OldHS. In all other + // cases, where a term is included, we should be able to assert that the term + // isn't regressing (i.e. oldHS.Term >= newHS.Term). + if oldHS.Term > newHS.Term { // The existing HardState is allowed to be ahead of us, which is // relevant in practice for the split trigger. We already checked above @@ -181,9 +186,11 @@ func (sl StateLoader) SynthesizeHardState( // updated votes yet. newHS.Term = oldHS.Term } - // If the existing HardState voted in this term, remember that. + // If the existing HardState voted in this term and knows who the leader is, + // remember that. if oldHS.Term == newHS.Term { newHS.Vote = oldHS.Vote + newHS.Lead = oldHS.Lead } err := sl.SetHardState(ctx, readWriter, newHS) return errors.Wrapf(err, "writing HardState %+v", &newHS) diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 25f13d015a1c..67a21c87e2c8 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -801,7 +801,6 @@ func (r *Replica) handleRaftReadyRaftMuLocked( } var hasReady bool - var softState *raft.SoftState var outboundMsgs []raftpb.Message var msgStorageAppend, msgStorageApply raftpb.Message r.mu.Lock() @@ -836,7 +835,6 @@ func (r *Replica) handleRaftReadyRaftMuLocked( logRaftReady(ctx, syncRd) asyncRd := makeAsyncReady(syncRd) - softState = asyncRd.SoftState outboundMsgs, msgStorageAppend, msgStorageApply = splitLocalStorageMsgs(asyncRd.Messages) } // We unquiesce if we have a Ready (= there's work to do). We also have @@ -876,24 +874,6 @@ func (r *Replica) handleRaftReadyRaftMuLocked( return stats, nil } - refreshReason := noReason - if softState != nil && leaderID != roachpb.ReplicaID(softState.Lead) { - // Refresh pending commands if the Raft leader has changed. This is usually - // the first indication we have of a new leader on a restarted node. - // - // TODO(peter): Re-proposing commands when SoftState.Lead changes can lead - // to wasteful multiple-reproposals when we later see an empty Raft command - // indicating a newly elected leader or a conf change. Replay protection - // prevents any corruption, so the waste is only a performance issue. - if log.V(3) { - log.Infof(ctx, "raft leader changed: %d -> %d", leaderID, softState.Lead) - } - if !r.store.TestingKnobs().DisableRefreshReasonNewLeader { - refreshReason = reasonNewLeader - } - leaderID = roachpb.ReplicaID(softState.Lead) - } - r.traceMessageSends(outboundMsgs, "sending messages") r.sendRaftMessages(ctx, outboundMsgs, pausedFollowers, true /* willDeliverLocal */) @@ -941,7 +921,28 @@ func (r *Replica) handleRaftReadyRaftMuLocked( } } + refreshReason := noReason if hasMsg(msgStorageAppend) { + // Leadership changes, if any, are communicated through MsgStorageAppends. + // Check if that's the case here. + if msgStorageAppend.Lead != raft.None && leaderID != roachpb.ReplicaID(msgStorageAppend.Lead) { + // Refresh pending commands if the Raft leader has changed. This is + // usually the first indication we have of a new leader on a restarted + // node. + // + // TODO(peter): Re-proposing commands when SoftState.Lead changes can lead + // to wasteful multiple-reproposals when we later see an empty Raft command + // indicating a newly elected leader or a conf change. Replay protection + // prevents any corruption, so the waste is only a performance issue. + if log.V(3) { + log.Infof(ctx, "raft leader changed: %d -> %d", leaderID, msgStorageAppend.Lead) + } + if !r.store.TestingKnobs().DisableRefreshReasonNewLeader { + refreshReason = reasonNewLeader + } + leaderID = roachpb.ReplicaID(msgStorageAppend.Lead) + } + if msgStorageAppend.Snapshot != nil { if inSnap.Desc == nil { // If we didn't expect Raft to have a snapshot but it has one @@ -969,6 +970,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( Term: msgStorageAppend.Term, Vote: msgStorageAppend.Vote, Commit: msgStorageAppend.Commit, + Lead: msgStorageAppend.Lead, } if len(msgStorageAppend.Entries) != 0 { log.Fatalf(ctx, "found Entries in MsgStorageAppend with non-empty Snapshot") @@ -1204,11 +1206,6 @@ func (r *Replica) handleRaftReadyRaftMuLocked( // All fields in asyncReady are read-only. // TODO(nvanbenschoten): move this into go.etcd.io/raft. type asyncReady struct { - // The current volatile state of a Node. - // SoftState will be nil if there is no update. - // It is not required to consume or store SoftState. - *raft.SoftState - // Messages specifies outbound messages to other peers and to local storage // threads. These messages can be sent in any order. // @@ -1220,8 +1217,7 @@ type asyncReady struct { // makeAsyncReady constructs an asyncReady from the provided Ready. func makeAsyncReady(rd raft.Ready) asyncReady { return asyncReady{ - SoftState: rd.SoftState, - Messages: rd.Messages, + Messages: rd.Messages, } } @@ -2023,7 +2019,7 @@ func (r *Replica) hasOutstandingSnapshotInFlightToStore( // HasRaftLeader returns true if the raft group has a raft leader currently. func HasRaftLeader(raftStatus *raft.Status) bool { - return raftStatus != nil && raftStatus.SoftState.Lead != 0 + return raftStatus != nil && raftStatus.HardState.Lead != 0 } // pendingCmdSlice sorts by increasing MaxLeaseIndex. diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 8466ef80c6f2..8dcabb958182 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -134,8 +134,8 @@ func upToDateRaftStatus(repls []roachpb.ReplicaDescriptor) *raft.Status { } return &raft.Status{ BasicStatus: raft.BasicStatus{ - HardState: raftpb.HardState{Commit: 100}, - SoftState: raft.SoftState{Lead: 1, RaftState: raft.StateLeader}, + HardState: raftpb.HardState{Commit: 100, Lead: 1}, + SoftState: raft.SoftState{RaftState: raft.StateLeader}, }, Progress: prs, } @@ -8918,7 +8918,7 @@ func TestReplicaMetrics(t *testing.T) { } else { status.SoftState.RaftState = raft.StateFollower } - status.SoftState.Lead = lead + status.HardState.Lead = lead return status } desc := func(ids ...int) roachpb.RangeDescriptor { @@ -10117,9 +10117,7 @@ func TestFollowerQuiesceOnNotify(t *testing.T) { HardState: raftpb.HardState{ Term: 5, Commit: 10, - }, - SoftState: raft.SoftState{ - Lead: 1, + Lead: 1, }, }, }, @@ -11481,7 +11479,9 @@ func TestReplicaShouldCampaignOnWake(t *testing.T) { raftStatus: raft.BasicStatus{ SoftState: raft.SoftState{ RaftState: raft.StateFollower, - Lead: 2, + }, + HardState: raftpb.HardState{ + Lead: 2, }, }, livenessMap: livenesspb.IsLiveMap{ @@ -11599,7 +11599,9 @@ func TestReplicaShouldCampaignOnLeaseRequestRedirect(t *testing.T) { raftStatus: raft.BasicStatus{ SoftState: raft.SoftState{ RaftState: raft.StateFollower, - Lead: 2, + }, + HardState: raftpb.HardState{ + Lead: 2, }, }, livenessMap: livenesspb.IsLiveMap{ @@ -11710,7 +11712,9 @@ func TestReplicaShouldForgetLeaderOnVoteRequest(t *testing.T) { raftStatus: raft.BasicStatus{ SoftState: raft.SoftState{ RaftState: raft.StateFollower, - Lead: 2, + }, + HardState: raftpb.HardState{ + Lead: 2, }, }, livenessMap: livenesspb.IsLiveMap{ diff --git a/pkg/kv/kvserver/stateloader/initial_test.go b/pkg/kv/kvserver/stateloader/initial_test.go index f3df866773da..ca6331f58b3e 100644 --- a/pkg/kv/kvserver/stateloader/initial_test.go +++ b/pkg/kv/kvserver/stateloader/initial_test.go @@ -32,7 +32,7 @@ func TestSynthesizeHardState(t *testing.T) { eng := storage.NewDefaultInMemForTesting() stopper.AddCloser(eng) - tHS := raftpb.HardState{Term: 2, Vote: 3, Commit: 4} + tHS := raftpb.HardState{Term: 2, Vote: 3, Commit: 4, Lead: 5} testCases := []struct { TruncTerm kvpb.RaftTerm @@ -45,10 +45,10 @@ func TestSynthesizeHardState(t *testing.T) { // Can't wind back the committed index of the new HardState. {OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit - 1), Err: "can't decrease HardState.Commit"}, {OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit), NewHS: tHS}, - {OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit + 1), NewHS: raftpb.HardState{Term: tHS.Term, Vote: 3, Commit: tHS.Commit + 1}}, - // Higher Term is picked up, but vote isn't carried over when the term - // changes. - {OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit), TruncTerm: 11, NewHS: raftpb.HardState{Term: 11, Vote: 0, Commit: tHS.Commit}}, + {OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit + 1), NewHS: raftpb.HardState{Term: tHS.Term, Vote: 3, Commit: tHS.Commit + 1, Lead: 5}}, + // Higher Term is picked up, but Vote and Lead aren't carried over when the + // term changes. + {OldHS: &tHS, RaftAppliedIndex: kvpb.RaftIndex(tHS.Commit), TruncTerm: 11, NewHS: raftpb.HardState{Term: 11, Vote: 0, Commit: tHS.Commit, Lead: 0}}, } for i, test := range testCases { diff --git a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/HardState b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/HardState index a1ea2ff7ced1..f3188ac57117 100644 --- a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/HardState +++ b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/HardState @@ -1,3 +1,3 @@ echo ---- -5484201021249543052 +2027450491469602084 diff --git a/pkg/raft/node.go b/pkg/raft/node.go index ef06e5b9a654..fd7e820bfb9f 100644 --- a/pkg/raft/node.go +++ b/pkg/raft/node.go @@ -41,12 +41,11 @@ var ( // SoftState provides state that is useful for logging and debugging. // The state is volatile and does not need to be persisted to the WAL. type SoftState struct { - Lead pb.PeerID // must use atomic operations to access; keep 64-bit aligned. RaftState StateType } func (a *SoftState) equal(b *SoftState) bool { - return a.Lead == b.Lead && a.RaftState == b.RaftState + return a.RaftState == b.RaftState } // Ready encapsulates the entries and messages that are ready to read, @@ -112,7 +111,7 @@ type Ready struct { } func isHardStateEqual(a, b pb.HardState) bool { - return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit + return a.Term == b.Term && a.Vote == b.Vote && a.Commit == b.Commit && a.Lead == b.Lead } // IsEmptyHardState returns true if the given HardState is empty. diff --git a/pkg/raft/node_test.go b/pkg/raft/node_test.go index 48c43455259f..97bc49e9d37e 100644 --- a/pkg/raft/node_test.go +++ b/pkg/raft/node_test.go @@ -150,7 +150,7 @@ func TestNodePropose(t *testing.T) { rd := <-n.Ready() s.Append(rd.Entries) // change the step function to appendStep until this raft becomes leader - if rd.SoftState.Lead == r.id { + if rd.HardState.Lead == r.id { r.step = appendStep n.Advance() break @@ -215,7 +215,7 @@ func TestNodeProposeConfig(t *testing.T) { rd := <-n.Ready() s.Append(rd.Entries) // change the step function to appendStep until this raft becomes leader - if rd.SoftState.Lead == r.id { + if rd.HardState.Lead == r.id { r.step = appendStep n.Advance() break @@ -366,7 +366,7 @@ func TestNodeProposeWaitDropped(t *testing.T) { rd := <-n.Ready() s.Append(rd.Entries) // change the step function to dropStep until this raft becomes leader - if rd.SoftState.Lead == r.id { + if rd.HardState.Lead == r.id { r.step = dropStep n.Advance() break @@ -442,7 +442,7 @@ func TestNodeStart(t *testing.T) { require.NoError(t, err) wants := []Ready{ { - HardState: raftpb.HardState{Term: 1, Commit: 1, Vote: 0}, + HardState: raftpb.HardState{Term: 1, Commit: 1, Vote: 0, Lead: 0}, Entries: []raftpb.Entry{ {Type: raftpb.EntryConfChange, Term: 1, Index: 1, Data: ccdata}, }, @@ -452,13 +452,13 @@ func TestNodeStart(t *testing.T) { MustSync: true, }, { - HardState: raftpb.HardState{Term: 2, Commit: 2, Vote: 1}, + HardState: raftpb.HardState{Term: 2, Commit: 2, Vote: 1, Lead: 1}, Entries: []raftpb.Entry{{Term: 2, Index: 3, Data: []byte("foo")}}, CommittedEntries: []raftpb.Entry{{Term: 2, Index: 2, Data: nil}}, MustSync: true, }, { - HardState: raftpb.HardState{Term: 2, Commit: 3, Vote: 1}, + HardState: raftpb.HardState{Term: 2, Commit: 3, Vote: 1, Lead: 1}, Entries: nil, CommittedEntries: []raftpb.Entry{{Term: 2, Index: 3, Data: []byte("foo")}}, MustSync: false, @@ -647,7 +647,6 @@ func TestSoftStateEqual(t *testing.T) { we bool }{ {&SoftState{}, true}, - {&SoftState{Lead: 1}, false}, {&SoftState{RaftState: StateLeader}, false}, } for i, tt := range tests { @@ -664,6 +663,7 @@ func TestIsHardStateEqual(t *testing.T) { {raftpb.HardState{Vote: 1}, false}, {raftpb.HardState{Commit: 1}, false}, {raftpb.HardState{Term: 1}, false}, + {raftpb.HardState{Lead: 1}, false}, } for i, tt := range tests { diff --git a/pkg/raft/raft.go b/pkg/raft/raft.go index 954fa6f65271..4928d16317d0 100644 --- a/pkg/raft/raft.go +++ b/pkg/raft/raft.go @@ -443,7 +443,7 @@ func newRaft(c *Config) *raft { if c.Applied > 0 { raftlog.appliedTo(c.Applied, 0 /* size */) } - r.becomeFollower(r.Term, None) + r.becomeFollower(r.Term, r.lead) var nodesStrs []string for _, n := range r.trk.VoterNodes() { @@ -458,13 +458,14 @@ func newRaft(c *Config) *raft { func (r *raft) hasLeader() bool { return r.lead != None } -func (r *raft) softState() SoftState { return SoftState{Lead: r.lead, RaftState: r.state} } +func (r *raft) softState() SoftState { return SoftState{RaftState: r.state} } func (r *raft) hardState() pb.HardState { return pb.HardState{ Term: r.Term, Vote: r.Vote, Commit: r.raftLog.committed, + Lead: r.lead, } } @@ -739,6 +740,8 @@ func (r *raft) reset(term uint64) { r.Term = term r.Vote = None } + + // TODO(arul): we should only reset this if the term has changed. r.lead = None r.electionElapsed = 0 @@ -846,6 +849,11 @@ func (r *raft) tickHeartbeat() { } } +// TODO(arul): Consider removing the lead argument from this function. Instead, +// for all the methods that want to set the leader explicitly (the ones that are +// passing in m.From for this field), we can instead have them use an assignLead +// function instead; in there, we can add safety checks to ensure we're not +// overwriting the leader. func (r *raft) becomeFollower(term uint64, lead pb.PeerID) { r.step = stepFollower r.reset(term) @@ -879,6 +887,10 @@ func (r *raft) becomePreCandidate() { r.step = stepCandidate r.trk.ResetVotes() r.tick = r.tickElection + // TODO(arul): We're forgetting the raft leader here. From the perspective of + // leader leases, this is fine, because we wouldn't be here unless we'd + // revoked StoreLiveness support for the leader's store to begin with. It's + // a bit weird from the perspective of raft though. See if we can avoid this. r.lead = None r.state = StatePreCandidate r.logger.Infof("%x became pre-candidate at term %d", r.id, r.Term) @@ -1226,7 +1238,13 @@ func stepLeader(r *raft, m pb.Message) error { case pb.MsgCheckQuorum: if !r.trk.QuorumActive() { r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id) - r.becomeFollower(r.Term, None) + // NB: Stepping down because of CheckQuorum is a special, in that we know + // the QSE is in the past. This means that the leader can safely call a + // new election or vote for a different peer without regressing the QSE. + // We don't need to/want to give this any special treatment -- instead, we + // handle this like the general step down case by simply remembering the + // term/lead information from our stint as the leader. + r.becomeFollower(r.Term, r.id) } // Mark everyone (but ourselves) as inactive in preparation for the next // CheckQuorum. @@ -1606,7 +1624,7 @@ func stepCandidate(r *raft, m pb.Message) error { case quorum.VoteLost: // pb.MsgPreVoteResp contains future term of pre-candidate // m.Term > r.Term; reuse r.Term - r.becomeFollower(r.Term, None) + r.becomeFollower(r.Term, r.lead) } case pb.MsgTimeoutNow: r.logger.Debugf("%x [term %d state %v] ignored MsgTimeoutNow from %x", r.id, r.Term, r.state, m.From) @@ -1623,11 +1641,19 @@ func stepFollower(r *raft, m pb.Message) error { } else if r.disableProposalForwarding { r.logger.Infof("%x not forwarding to leader %x at term %d; dropping proposal", r.id, r.lead, r.Term) return ErrProposalDropped + } else if r.lead == r.id { + r.logger.Infof("%x not forwarding to itself at term %d; dropping proposal", r.id, r.Term) + return ErrProposalDropped } m.To = r.lead r.send(m) case pb.MsgApp: r.electionElapsed = 0 + // TODO(arul): Once r.lead != None, we shouldn't need to update r.lead + // anymore within the course of a single term (in the context of which this + // function is always called). Instead, if r.lead != None, we should be able + // to assert that the leader hasn't changed within a given term. Maybe at + // the caller itself. r.lead = m.From r.handleAppendEntries(m) case pb.MsgHeartbeat: @@ -1642,6 +1668,9 @@ func stepFollower(r *raft, m pb.Message) error { if r.lead == None { r.logger.Infof("%x no leader at term %d; dropping leader transfer msg", r.id, r.Term) return nil + } else if r.lead == r.id { + r.logger.Infof("%x is itself the leader at term %d; dropping leader transfer msg", r.id, r.Term) + return nil } m.To = r.lead r.send(m) @@ -1944,7 +1973,9 @@ func (r *raft) switchToConfig(cfg tracker.Config, trk tracker.ProgressMap) pb.Co // interruption). This might still drop some proposals but it's better than // nothing. if r.stepDownOnRemoval { - r.becomeFollower(r.Term, None) + // NB: Similar to the CheckQuorum step down case, we must remember our + // prior stint as leader, lest we regress the QSE. + r.becomeFollower(r.Term, r.lead) } return cs } @@ -1978,6 +2009,7 @@ func (r *raft) loadState(state pb.HardState) { r.raftLog.committed = state.Commit r.Term = state.Term r.Vote = state.Vote + r.lead = state.Lead } // pastElectionTimeout returns true if r.electionElapsed is greater diff --git a/pkg/raft/raftpb/raft.proto b/pkg/raft/raftpb/raft.proto index fcdc8115139f..3780a56f07f9 100644 --- a/pkg/raft/raftpb/raft.proto +++ b/pkg/raft/raftpb/raft.proto @@ -88,11 +88,12 @@ message Message { optional uint64 index = 6 [(gogoproto.nullable) = false]; repeated Entry entries = 7 [(gogoproto.nullable) = false]; optional uint64 commit = 8 [(gogoproto.nullable) = false]; + optional uint64 lead = 16 [(gogoproto.nullable) = false, (gogoproto.casttype) = "PeerID"]; // (type=MsgStorageAppend,vote=5,term=10) means the local node is voting for - // peer 5 in term 10. For MsgStorageAppends, the term, vote, and commit fields - // will either all be set (to facilitate the construction of a HardState) if - // any of the fields have changed or will all be unset if none of the fields - // have changed. + // peer 5 in term 10. For MsgStorageAppends, the term, vote, lead, and commit + // fields will either all be set (to facilitate the construction of a + // HardState) if any of the fields have changed or will all be unset if none + // of the fields have changed. optional uint64 vote = 13 [(gogoproto.nullable) = false, (gogoproto.casttype) = "PeerID"]; // snapshot is non-nil and non-empty for MsgSnap messages and nil for all other // message types. However, peer nodes running older binary versions may send a @@ -107,26 +108,27 @@ message Message { // is complete. Populated for MsgStorageAppend and MsgStorageApply messages. repeated Message responses = 14 [(gogoproto.nullable) = false]; - // match is the log index up to which the follower's log must persistently - // match the leader's. If the follower's persistent log is shorter, it means - // the follower has broken its promise and violated safety of Raft. Typically - // this means the environment (Storage) hasn't provided the required - // durability guarantees. - // - // If a follower sees a match index exceeding its log's last index, it must - // cease its membership (stop voting and acking appends) in the raft group, in - // order to limit the damage. Today it simply panics. - // - // match is only populated by the leader when sending messages to a voting - // follower. This can be 0 if the leader hasn't yet established the follower's - // match index, or for backward compatibility. - optional uint64 match = 15 [(gogoproto.nullable) = false]; + // match is the log index up to which the follower's log must persistently + // match the leader's. If the follower's persistent log is shorter, it means + // the follower has broken its promise and violated safety of Raft. Typically + // this means the environment (Storage) hasn't provided the required + // durability guarantees. + // + // If a follower sees a match index exceeding its log's last index, it must + // cease its membership (stop voting and acking appends) in the raft group, in + // order to limit the damage. Today it simply panics. + // + // match is only populated by the leader when sending messages to a voting + // follower. This can be 0 if the leader hasn't yet established the follower's + // match index, or for backward compatibility. + optional uint64 match = 15 [(gogoproto.nullable) = false]; } message HardState { optional uint64 term = 1 [(gogoproto.nullable) = false]; optional uint64 vote = 2 [(gogoproto.nullable) = false, (gogoproto.casttype) = "PeerID"]; optional uint64 commit = 3 [(gogoproto.nullable) = false]; + optional uint64 lead = 4 [(gogoproto.nullable) = false, (gogoproto.casttype) = "PeerID"]; } // ConfChangeTransition specifies the behavior of a configuration change with diff --git a/pkg/raft/raftpb/raft_test.go b/pkg/raft/raftpb/raft_test.go index 47e4b0bd3914..b550350a7758 100644 --- a/pkg/raft/raftpb/raft_test.go +++ b/pkg/raft/raftpb/raft_test.go @@ -48,10 +48,10 @@ func TestProtoMemorySizes(t *testing.T) { assert(unsafe.Sizeof(s), if64Bit(144, 80), "Snapshot") var m Message - assert(unsafe.Sizeof(m), if64Bit(168, 112), "Message") + assert(unsafe.Sizeof(m), if64Bit(176, 112), "Message") var hs HardState - assert(unsafe.Sizeof(hs), 24, "HardState") + assert(unsafe.Sizeof(hs), 32, "HardState") var cs ConfState assert(unsafe.Sizeof(cs), if64Bit(104, 52), "ConfState") diff --git a/pkg/raft/rafttest/interaction_env_handler_process_append_thread.go b/pkg/raft/rafttest/interaction_env_handler_process_append_thread.go index f0ff8d076c29..72f2fb32642d 100644 --- a/pkg/raft/rafttest/interaction_env_handler_process_append_thread.go +++ b/pkg/raft/rafttest/interaction_env_handler_process_append_thread.go @@ -63,6 +63,7 @@ func (env *InteractionEnv) ProcessAppendThread(idx int) error { Term: m.Term, Vote: m.Vote, Commit: m.Commit, + Lead: m.Lead, } var snap raftpb.Snapshot if m.Snapshot != nil { diff --git a/pkg/raft/rafttest/node_test.go b/pkg/raft/rafttest/node_test.go index 9145284950a4..4323b76005b9 100644 --- a/pkg/raft/rafttest/node_test.go +++ b/pkg/raft/rafttest/node_test.go @@ -143,7 +143,7 @@ func waitLeader(ns []*node) int { l = make(map[pb.PeerID]struct{}) for i, n := range ns { - lead := n.Status().SoftState.Lead + lead := n.Status().HardState.Lead if lead != 0 { l[lead] = struct{}{} if n.id == lead { diff --git a/pkg/raft/rawnode.go b/pkg/raft/rawnode.go index d8954a9f7c93..e873a90bb836 100644 --- a/pkg/raft/rawnode.go +++ b/pkg/raft/rawnode.go @@ -180,9 +180,10 @@ func MustSync(st, prevst pb.HardState, entsnum int) bool { // Persistent state on all servers: // (Updated on stable storage before responding to RPCs) // currentTerm + // currentLead // votedFor // log entries[] - return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term + return entsnum != 0 || st.Vote != prevst.Vote || st.Term != prevst.Term || st.Lead != prevst.Lead } func needStorageAppendMsg(r *raft, rd Ready) bool { @@ -226,6 +227,7 @@ func newStorageAppendMsg(r *raft, rd Ready) pb.Message { m.Term = rd.Term m.Vote = rd.Vote m.Commit = rd.Commit + m.Lead = rd.Lead } if !IsEmptySnap(rd.Snapshot) { snap := rd.Snapshot diff --git a/pkg/raft/rawnode_test.go b/pkg/raft/rawnode_test.go index 4f79f020da66..547186894582 100644 --- a/pkg/raft/rawnode_test.go +++ b/pkg/raft/rawnode_test.go @@ -251,7 +251,7 @@ func TestRawNodeProposeAndConfChange(t *testing.T) { } rawNode.Advance(rd) // Once we are the leader, propose a command and a ConfChange. - if !proposed && rd.SoftState.Lead == rawNode.raft.id { + if !proposed && rd.HardState.Lead == rawNode.raft.id { require.NoError(t, rawNode.Propose([]byte("somedata"))) if ccv1, ok := tc.cc.AsV1(); ok { ccdata, err = ccv1.Marshal() @@ -380,7 +380,7 @@ func TestRawNodeJointAutoLeave(t *testing.T) { } rawNode.Advance(rd) // Once we are the leader, propose a command and a ConfChange. - if !proposed && rd.SoftState.Lead == rawNode.raft.id { + if !proposed && rd.HardState.Lead == rawNode.raft.id { require.NoError(t, rawNode.Propose([]byte("somedata"))) ccdata, err = testCc.Marshal() require.NoError(t, err) @@ -456,7 +456,7 @@ func TestRawNodeProposeAddDuplicateNode(t *testing.T) { for { rd = rawNode.Ready() s.Append(rd.Entries) - if rd.SoftState.Lead == rawNode.raft.id { + if rd.HardState.Lead == rawNode.raft.id { rawNode.Advance(rd) break } @@ -520,8 +520,8 @@ func TestRawNodeStart(t *testing.T) { {Term: 1, Index: 3, Data: []byte("foo")}, // non-empty entry } want := Ready{ - SoftState: &SoftState{Lead: 1, RaftState: StateLeader}, - HardState: pb.HardState{Term: 1, Commit: 3, Vote: 1}, + SoftState: &SoftState{RaftState: StateLeader}, + HardState: pb.HardState{Term: 1, Commit: 3, Vote: 1, Lead: 1}, Entries: nil, // emitted & checked in intermediate Ready cycle CommittedEntries: entries, MustSync: false, // since we're only applying, not appending @@ -610,10 +610,10 @@ func TestRawNodeRestart(t *testing.T) { {Term: 1, Index: 1}, {Term: 1, Index: 2, Data: []byte("foo")}, } - st := pb.HardState{Term: 1, Commit: 1} + st := pb.HardState{Term: 1, Commit: 1, Lead: 1} want := Ready{ - HardState: emptyState, + HardState: emptyState, // no HardState is emitted because there was no change // commit up to commit index in st CommittedEntries: entries[:st.Commit], MustSync: false, @@ -628,6 +628,18 @@ func TestRawNodeRestart(t *testing.T) { assert.Equal(t, want, rd) rawNode.Advance(rd) assert.False(t, rawNode.HasReady()) + // Ensure that the HardState was correctly loaded post restart. + assert.Equal(t, uint64(1), rawNode.raft.Term) + assert.Equal(t, uint64(1), rawNode.raft.raftLog.committed) + assert.Equal(t, pb.PeerID(1), rawNode.raft.lead) + assert.True(t, rawNode.raft.state == StateFollower) + + // Ensure we campaign after the election timeout has elapsed. + for i := 0; i < rawNode.raft.randomizedElectionTimeout; i++ { + rawNode.raft.tick() + } + assert.Equal(t, StateCandidate, rawNode.raft.state) + assert.Equal(t, uint64(2), rawNode.raft.Term) // this should in-turn bump the term } func TestRawNodeRestartFromSnapshot(t *testing.T) { diff --git a/pkg/raft/status.go b/pkg/raft/status.go index 5fa4341bce23..41e7946e6bd1 100644 --- a/pkg/raft/status.go +++ b/pkg/raft/status.go @@ -99,6 +99,23 @@ func getBasicStatus(r *raft) BasicStatus { s.HardState = r.hardState() s.SoftState = r.softState() s.Applied = r.raftLog.applied + if s.RaftState == StateFollower && s.Lead == r.id { + // A raft leader's term ends when it is shut down. It'll rejoin its peers as + // a follower when it comes back up, but its Lead and Term field may still + // correspond to its pre-restart leadership term. We expect this to quickly + // be updated when it hears from the new leader, if one was elected in its + // absence, or when it campaigns. + // + // The layers above raft (in particular kvserver) do not handle the case + // where a raft node's state is StateFollower but its lead field points to + // itself. They expect the Lead field to correspond to the current leader, + // which we know we are not. For their benefit, we overwrite the Lead field + // to None. + // + // TODO(arul): the layers above should not conflate Lead with current + // leader. Fix that and get rid of this overwrite. + s.HardState.Lead = None + } return s } diff --git a/pkg/raft/testdata/async_storage_writes.txt b/pkg/raft/testdata/async_storage_writes.txt index 1a281301f6f6..48a0763d6e2e 100644 --- a/pkg/raft/testdata/async_storage_writes.txt +++ b/pkg/raft/testdata/async_storage_writes.txt @@ -24,8 +24,8 @@ stabilize ---- > 1 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:1 Vote:1 Commit:10 + State:StateCandidate + HardState Term:1 Vote:1 Commit:10 Lead:0 Messages: 1->2 MsgVote Term:1 Log:1/10 1->3 MsgVote Term:1 Log:1/10 @@ -49,14 +49,14 @@ stabilize 1->1 MsgVoteResp Term:1 Log:0/0 > 2 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:10 + HardState Term:1 Vote:1 Commit:10 Lead:0 Messages: 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses:[ 2->1 MsgVoteResp Term:1 Log:0/0 ] > 3 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:10 + HardState Term:1 Vote:1 Commit:10 Lead:0 Messages: 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Responses:[ 3->1 MsgVoteResp Term:1 Log:0/0 @@ -83,13 +83,14 @@ stabilize 3->1 MsgVoteResp Term:1 Log:0/0 > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + State:StateLeader + HardState Term:1 Vote:1 Commit:10 Lead:1 Entries: 1/11 EntryNormal "" Messages: 1->2 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] 1->3 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] - 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Entries:[1/11 EntryNormal ""] Responses:[ 1->1 MsgAppResp Term:1 Log:0/11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 ] @@ -99,27 +100,27 @@ stabilize 1->3 MsgApp Term:1 Log:1/10 Commit:10 Entries:[1/11 EntryNormal ""] > 1 processing append thread Processing: - 1->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + 1->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Entries:[1/11 EntryNormal ""] Responses: 1->1 MsgAppResp Term:1 Log:0/11 AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 > 2 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower + HardState Term:1 Vote:1 Commit:10 Lead:1 Entries: 1/11 EntryNormal "" Messages: - 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Entries:[1/11 EntryNormal ""] Responses:[ 2->1 MsgAppResp Term:1 Log:0/11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 ] > 3 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower + HardState Term:1 Vote:1 Commit:10 Lead:1 Entries: 1/11 EntryNormal "" Messages: - 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] Responses:[ + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Entries:[1/11 EntryNormal ""] Responses:[ 3->1 MsgAppResp Term:1 Log:0/11 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11 ] @@ -128,13 +129,13 @@ stabilize AppendThread->1 MsgStorageAppendResp Term:1 Log:1/11 > 2 processing append thread Processing: - 2->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + 2->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Entries:[1/11 EntryNormal ""] Responses: 2->1 MsgAppResp Term:1 Log:0/11 AppendThread->2 MsgStorageAppendResp Term:1 Log:1/11 > 3 processing append thread Processing: - 3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] + 3->AppendThread MsgStorageAppend Term:1 Log:0/0 Commit:10 Vote:1 Entries:[1/11 EntryNormal ""] Responses: 3->1 MsgAppResp Term:1 Log:0/11 AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11 @@ -147,7 +148,7 @@ stabilize AppendThread->3 MsgStorageAppendResp Term:1 Log:1/11 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:11 + HardState Term:1 Vote:1 Commit:11 Lead:1 CommittedEntries: 1/11 EntryNormal "" Messages: @@ -172,7 +173,7 @@ stabilize ApplyThread->1 MsgStorageApplyResp Term:0 Log:0/0 Entries:[1/11 EntryNormal ""] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:11 + HardState Term:1 Vote:1 Commit:11 Lead:1 CommittedEntries: 1/11 EntryNormal "" Messages: @@ -184,7 +185,7 @@ stabilize ] > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:11 + HardState Term:1 Vote:1 Commit:11 Lead:1 CommittedEntries: 1/11 EntryNormal "" Messages: @@ -374,7 +375,7 @@ process-ready 1 2 3 ---- > 1 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:12 + HardState Term:1 Vote:1 Commit:12 Lead:1 Entries: 1/14 EntryNormal "prop_3" CommittedEntries: @@ -409,7 +410,7 @@ process-ready 1 2 3 > 2 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:12 + HardState Term:1 Vote:1 Commit:12 Lead:1 Entries: 1/14 EntryNormal "prop_3" CommittedEntries: @@ -425,7 +426,7 @@ process-ready 1 2 3 ] > 3 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:12 + HardState Term:1 Vote:1 Commit:12 Lead:1 Entries: 1/14 EntryNormal "prop_3" CommittedEntries: @@ -478,7 +479,7 @@ process-ready 1 2 3 ---- > 1 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:13 + HardState Term:1 Vote:1 Commit:13 Lead:1 Entries: 1/15 EntryNormal "prop_4" CommittedEntries: @@ -513,7 +514,7 @@ process-ready 1 2 3 > 2 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:13 + HardState Term:1 Vote:1 Commit:13 Lead:1 Entries: 1/15 EntryNormal "prop_4" CommittedEntries: @@ -529,7 +530,7 @@ process-ready 1 2 3 ] > 3 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:13 + HardState Term:1 Vote:1 Commit:13 Lead:1 Entries: 1/15 EntryNormal "prop_4" CommittedEntries: @@ -603,7 +604,7 @@ process-ready 1 2 3 ---- > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:14 + HardState Term:1 Vote:1 Commit:14 Lead:1 CommittedEntries: 1/14 EntryNormal "prop_3" Messages: @@ -631,7 +632,7 @@ process-ready 1 2 3 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:14 + HardState Term:1 Vote:1 Commit:14 Lead:1 CommittedEntries: 1/14 EntryNormal "prop_3" Messages: @@ -644,7 +645,7 @@ process-ready 1 2 3 ] > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:14 + HardState Term:1 Vote:1 Commit:14 Lead:1 CommittedEntries: 1/14 EntryNormal "prop_3" Messages: @@ -715,7 +716,7 @@ process-ready 1 2 3 ---- > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:15 + HardState Term:1 Vote:1 Commit:15 Lead:1 CommittedEntries: 1/15 EntryNormal "prop_4" Messages: @@ -741,7 +742,7 @@ process-ready 1 2 3 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:15 + HardState Term:1 Vote:1 Commit:15 Lead:1 CommittedEntries: 1/15 EntryNormal "prop_4" Messages: @@ -753,7 +754,7 @@ process-ready 1 2 3 ] > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:15 + HardState Term:1 Vote:1 Commit:15 Lead:1 CommittedEntries: 1/15 EntryNormal "prop_4" Messages: diff --git a/pkg/raft/testdata/async_storage_writes_append_aba_race.txt b/pkg/raft/testdata/async_storage_writes_append_aba_race.txt index 83964fe7d087..fbf371d0d6d4 100644 --- a/pkg/raft/testdata/async_storage_writes_append_aba_race.txt +++ b/pkg/raft/testdata/async_storage_writes_append_aba_race.txt @@ -85,8 +85,8 @@ INFO 3 [logterm: 1, index: 11] sent MsgVote request to 7 at term 2 process-ready 3 ---- Ready MustSync=true: -Lead:0 State:StateCandidate -HardState Term:2 Vote:3 Commit:11 +State:StateCandidate +HardState Term:2 Vote:3 Commit:11 Lead:0 Messages: 3->1 MsgVote Term:2 Log:1/11 3->2 MsgVote Term:2 Log:1/11 @@ -117,24 +117,21 @@ process-ready 4 5 6 ---- > 4 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:3 Commit:11 + HardState Term:2 Vote:3 Commit:11 Lead:0 Messages: 4->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[ 4->3 MsgVoteResp Term:2 Log:0/0 ] > 5 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:3 Commit:11 + HardState Term:2 Vote:3 Commit:11 Lead:0 Messages: 5->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[ 5->3 MsgVoteResp Term:2 Log:0/0 ] > 6 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:3 Commit:11 + HardState Term:2 Vote:3 Commit:11 Lead:0 Messages: 6->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Responses:[ 6->3 MsgVoteResp Term:2 Log:0/0 @@ -187,7 +184,8 @@ INFO 3 became leader at term 2 process-ready 3 ---- Ready MustSync=true: -Lead:3 State:StateLeader +State:StateLeader +HardState Term:2 Vote:3 Commit:11 Lead:3 Entries: 2/12 EntryNormal "" Messages: @@ -197,7 +195,7 @@ Messages: 3->5 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] 3->6 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] 3->7 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] -3->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[2/12 EntryNormal ""] Responses:[ +3->AppendThread MsgStorageAppend Term:2 Log:0/0 Commit:11 Vote:3 Entries:[2/12 EntryNormal ""] Responses:[ 3->3 MsgAppResp Term:2 Log:0/12 AppendThread->3 MsgStorageAppendResp Term:2 Log:2/12 ] @@ -222,8 +220,7 @@ dropped: 3->7 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] process-ready 1 ---- Ready MustSync=true: -Lead:3 State:StateFollower -HardState Term:2 Commit:11 +HardState Term:2 Commit:11 Lead:3 Entries: 2/12 EntryNormal "" Messages: @@ -249,8 +246,8 @@ INFO 4 [logterm: 1, index: 11] sent MsgVote request to 7 at term 3 process-ready 4 ---- Ready MustSync=true: -Lead:0 State:StateCandidate -HardState Term:3 Vote:4 Commit:11 +State:StateCandidate +HardState Term:3 Vote:4 Commit:11 Lead:0 Messages: 4->1 MsgVote Term:3 Log:1/11 4->2 MsgVote Term:3 Log:1/11 @@ -281,22 +278,21 @@ process-ready 5 6 7 ---- > 5 handling Ready Ready MustSync=true: - HardState Term:3 Vote:4 Commit:11 + HardState Term:3 Vote:4 Commit:11 Lead:0 Messages: 5->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[ 5->4 MsgVoteResp Term:3 Log:0/0 ] > 6 handling Ready Ready MustSync=true: - HardState Term:3 Vote:4 Commit:11 + HardState Term:3 Vote:4 Commit:11 Lead:0 Messages: 6->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[ 6->4 MsgVoteResp Term:3 Log:0/0 ] > 7 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:3 Vote:4 Commit:11 + HardState Term:3 Vote:4 Commit:11 Lead:0 Messages: 7->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Responses:[ 7->4 MsgVoteResp Term:3 Log:0/0 @@ -344,7 +340,8 @@ INFO 4 became leader at term 3 process-ready 4 ---- Ready MustSync=true: -Lead:4 State:StateLeader +State:StateLeader +HardState Term:3 Vote:4 Commit:11 Lead:4 Entries: 3/12 EntryNormal "" Messages: @@ -354,7 +351,7 @@ Messages: 4->5 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] 4->6 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] 4->7 MsgApp Term:3 Log:1/11 Commit:11 Entries:[3/12 EntryNormal ""] -4->AppendThread MsgStorageAppend Term:0 Log:0/0 Entries:[3/12 EntryNormal ""] Responses:[ +4->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Vote:4 Entries:[3/12 EntryNormal ""] Responses:[ 4->4 MsgAppResp Term:3 Log:0/12 AppendThread->4 MsgStorageAppendResp Term:3 Log:3/12 ] @@ -394,8 +391,7 @@ INFO 1 became follower at term 3 process-ready 1 ---- Ready MustSync=true: -Lead:4 State:StateFollower -HardState Term:3 Commit:11 +HardState Term:3 Commit:11 Lead:4 Messages: 1->4 MsgHeartbeatResp Term:3 Log:0/0 1->AppendThread MsgStorageAppend Term:3 Log:0/0 Commit:11 Responses:[ diff --git a/pkg/raft/testdata/campaign.txt b/pkg/raft/testdata/campaign.txt index 4eefab36d6bc..1f4b10e65d68 100644 --- a/pkg/raft/testdata/campaign.txt +++ b/pkg/raft/testdata/campaign.txt @@ -25,8 +25,8 @@ stabilize ---- > 1 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:1 Vote:1 Commit:2 + State:StateCandidate + HardState Term:1 Vote:1 Commit:2 Lead:0 Messages: 1->2 MsgVote Term:1 Log:1/2 1->3 MsgVote Term:1 Log:1/2 @@ -44,12 +44,12 @@ stabilize INFO 3 [logterm: 1, index: 2, vote: 0] cast MsgVote for 1 [logterm: 1, index: 2] at term 1 > 2 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:2 + HardState Term:1 Vote:1 Commit:2 Lead:0 Messages: 2->1 MsgVoteResp Term:1 Log:0/0 > 3 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:2 + HardState Term:1 Vote:1 Commit:2 Lead:0 Messages: 3->1 MsgVoteResp Term:1 Log:0/0 > 1 receiving messages @@ -60,7 +60,8 @@ stabilize 3->1 MsgVoteResp Term:1 Log:0/0 > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + State:StateLeader + HardState Term:1 Vote:1 Commit:2 Lead:1 Entries: 1/3 EntryNormal "" Messages: @@ -72,14 +73,14 @@ stabilize 1->3 MsgApp Term:1 Log:1/2 Commit:2 Entries:[1/3 EntryNormal ""] > 2 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower + HardState Term:1 Vote:1 Commit:2 Lead:1 Entries: 1/3 EntryNormal "" Messages: 2->1 MsgAppResp Term:1 Log:0/3 > 3 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower + HardState Term:1 Vote:1 Commit:2 Lead:1 Entries: 1/3 EntryNormal "" Messages: @@ -89,7 +90,7 @@ stabilize 3->1 MsgAppResp Term:1 Log:0/3 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:3 + HardState Term:1 Vote:1 Commit:3 Lead:1 CommittedEntries: 1/3 EntryNormal "" Messages: @@ -101,14 +102,14 @@ stabilize 1->3 MsgApp Term:1 Log:1/3 Commit:3 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:3 + HardState Term:1 Vote:1 Commit:3 Lead:1 CommittedEntries: 1/3 EntryNormal "" Messages: 2->1 MsgAppResp Term:1 Log:0/3 > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:3 + HardState Term:1 Vote:1 Commit:3 Lead:1 CommittedEntries: 1/3 EntryNormal "" Messages: diff --git a/pkg/raft/testdata/campaign_learner_must_vote.txt b/pkg/raft/testdata/campaign_learner_must_vote.txt index d05de374f191..e0510edec63f 100644 --- a/pkg/raft/testdata/campaign_learner_must_vote.txt +++ b/pkg/raft/testdata/campaign_learner_must_vote.txt @@ -61,8 +61,8 @@ INFO 2 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2 process-ready 2 ---- Ready MustSync=true: -Lead:0 State:StateCandidate -HardState Term:2 Vote:2 Commit:4 +State:StateCandidate +HardState Term:2 Vote:2 Commit:4 Lead:0 Messages: 2->1 MsgVote Term:2 Log:1/4 2->3 MsgVote Term:2 Log:1/4 @@ -82,8 +82,7 @@ stabilize 3 INFO 3 [logterm: 1, index: 3, vote: 0] cast MsgVote for 2 [logterm: 1, index: 4] at term 2 > 3 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:2 Commit:3 + HardState Term:2 Vote:2 Commit:3 Lead:0 Messages: 3->2 MsgVoteResp Term:2 Log:0/0 @@ -96,7 +95,8 @@ stabilize 2 3 INFO 2 became leader at term 2 > 2 handling Ready Ready MustSync=true: - Lead:2 State:StateLeader + State:StateLeader + HardState Term:2 Vote:2 Commit:4 Lead:2 Entries: 2/5 EntryNormal "" Messages: @@ -106,8 +106,8 @@ stabilize 2 3 2->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] DEBUG 3 [logterm: 0, index: 4] rejected MsgApp [logterm: 1, index: 4] from 2 > 3 handling Ready - Ready MustSync=false: - Lead:2 State:StateFollower + Ready MustSync=true: + HardState Term:2 Vote:2 Commit:3 Lead:2 Messages: 3->2 MsgAppResp Term:2 Log:1/4 Rejected (Hint: 3) > 2 receiving messages @@ -128,7 +128,7 @@ stabilize 2 3 ] > 3 handling Ready Ready MustSync=true: - HardState Term:2 Vote:2 Commit:4 + HardState Term:2 Vote:2 Commit:4 Lead:2 Entries: 1/4 EntryConfChangeV2 v3 2/5 EntryNormal "" @@ -141,7 +141,7 @@ stabilize 2 3 3->2 MsgAppResp Term:2 Log:0/5 > 2 handling Ready Ready MustSync=false: - HardState Term:2 Vote:2 Commit:5 + HardState Term:2 Vote:2 Commit:5 Lead:2 CommittedEntries: 2/5 EntryNormal "" Messages: @@ -150,7 +150,7 @@ stabilize 2 3 2->3 MsgApp Term:2 Log:2/5 Commit:5 > 3 handling Ready Ready MustSync=false: - HardState Term:2 Vote:2 Commit:5 + HardState Term:2 Vote:2 Commit:5 Lead:2 CommittedEntries: 2/5 EntryNormal "" Messages: diff --git a/pkg/raft/testdata/checkquorum.txt b/pkg/raft/testdata/checkquorum.txt index b25c1e63daec..015e05943678 100644 --- a/pkg/raft/testdata/checkquorum.txt +++ b/pkg/raft/testdata/checkquorum.txt @@ -36,8 +36,8 @@ stabilize ---- > 2 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:2 Vote:2 Commit:11 + State:StateCandidate + HardState Term:2 Vote:2 Commit:11 Lead:0 Messages: 2->1 MsgVote Term:2 Log:1/11 2->3 MsgVote Term:2 Log:1/11 @@ -68,7 +68,7 @@ stabilize ---- > 1 handling Ready Ready MustSync=false: - Lead:0 State:StateFollower + State:StateFollower Messages: 1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11 1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11 @@ -128,7 +128,7 @@ stabilize INFO 1 [term: 2] ignored a MsgHeartbeatResp message with lower term from 3 [term: 1] > 1 handling Ready Ready MustSync=true: - HardState Term:2 Commit:11 + HardState Term:2 Commit:11 Lead:0 # Other nodes can now successfully campaign. Note that we haven't ticked 3, so # it won't grant votes. @@ -142,7 +142,7 @@ INFO 2 [logterm: 1, index: 11] sent MsgVote request to 3 at term 3 process-ready 2 ---- Ready MustSync=true: -HardState Term:3 Vote:2 Commit:11 +HardState Term:3 Vote:2 Commit:11 Lead:0 Messages: 2->1 MsgVote Term:3 Log:1/11 2->3 MsgVote Term:3 Log:1/11 @@ -165,7 +165,7 @@ stabilize ---- > 1 handling Ready Ready MustSync=true: - HardState Term:3 Vote:2 Commit:11 + HardState Term:3 Vote:2 Commit:11 Lead:0 Messages: 1->2 MsgVoteResp Term:3 Log:0/0 > 2 receiving messages @@ -175,7 +175,8 @@ stabilize INFO 2 became leader at term 3 > 2 handling Ready Ready MustSync=true: - Lead:2 State:StateLeader + State:StateLeader + HardState Term:3 Vote:2 Commit:11 Lead:2 Entries: 3/12 EntryNormal "" Messages: @@ -189,15 +190,14 @@ stabilize INFO 3 became follower at term 3 > 1 handling Ready Ready MustSync=true: - Lead:2 State:StateFollower + HardState Term:3 Vote:2 Commit:11 Lead:2 Entries: 3/12 EntryNormal "" Messages: 1->2 MsgAppResp Term:3 Log:0/12 > 3 handling Ready Ready MustSync=true: - Lead:2 State:StateFollower - HardState Term:3 Commit:11 + HardState Term:3 Commit:11 Lead:2 Entries: 3/12 EntryNormal "" Messages: @@ -207,7 +207,7 @@ stabilize 3->2 MsgAppResp Term:3 Log:0/12 > 2 handling Ready Ready MustSync=false: - HardState Term:3 Vote:2 Commit:12 + HardState Term:3 Vote:2 Commit:12 Lead:2 CommittedEntries: 3/12 EntryNormal "" Messages: @@ -219,14 +219,14 @@ stabilize 2->3 MsgApp Term:3 Log:3/12 Commit:12 > 1 handling Ready Ready MustSync=false: - HardState Term:3 Vote:2 Commit:12 + HardState Term:3 Vote:2 Commit:12 Lead:2 CommittedEntries: 3/12 EntryNormal "" Messages: 1->2 MsgAppResp Term:3 Log:0/12 > 3 handling Ready Ready MustSync=false: - HardState Term:3 Commit:12 + HardState Term:3 Commit:12 Lead:2 CommittedEntries: 3/12 EntryNormal "" Messages: diff --git a/pkg/raft/testdata/confchange_disable_validation.txt b/pkg/raft/testdata/confchange_disable_validation.txt index 66c1113505f5..d25a17607e06 100644 --- a/pkg/raft/testdata/confchange_disable_validation.txt +++ b/pkg/raft/testdata/confchange_disable_validation.txt @@ -42,7 +42,7 @@ stabilize 1 1/5 EntryConfChangeV2 l2 l3 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 CommittedEntries: 1/4 EntryNormal "foo" > 1 handling Ready diff --git a/pkg/raft/testdata/confchange_drop_if_unapplied.txt b/pkg/raft/testdata/confchange_drop_if_unapplied.txt index 83f4f97f9125..40988d75200b 100644 --- a/pkg/raft/testdata/confchange_drop_if_unapplied.txt +++ b/pkg/raft/testdata/confchange_drop_if_unapplied.txt @@ -43,7 +43,7 @@ stabilize 1 ---- > 1 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:4 + HardState Term:1 Vote:1 Commit:4 Lead:1 Entries: 1/5 EntryNormal "" CommittedEntries: @@ -51,7 +51,7 @@ stabilize 1 INFO 1 switched to configuration voters=(1)&&(1) learners=(2 3) > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 CommittedEntries: 1/5 EntryNormal "" Messages: diff --git a/pkg/raft/testdata/confchange_v1_add_single.txt b/pkg/raft/testdata/confchange_v1_add_single.txt index f13d30777dc0..5be9a6997086 100644 --- a/pkg/raft/testdata/confchange_v1_add_single.txt +++ b/pkg/raft/testdata/confchange_v1_add_single.txt @@ -15,8 +15,8 @@ INFO 1 became candidate at term 1 process-ready 1 ---- Ready MustSync=true: -Lead:0 State:StateCandidate -HardState Term:1 Vote:1 Commit:2 +State:StateCandidate +HardState Term:1 Vote:1 Commit:2 Lead:0 INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 @@ -41,13 +41,14 @@ stabilize ---- > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + State:StateLeader + HardState Term:1 Vote:1 Commit:2 Lead:1 Entries: 1/3 EntryNormal "" 1/4 EntryConfChange v2 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:4 + HardState Term:1 Vote:1 Commit:4 Lead:1 CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChange v2 @@ -63,8 +64,7 @@ stabilize DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower - HardState Term:1 Commit:0 + HardState Term:1 Commit:0 Lead:1 Messages: 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) > 1 receiving messages @@ -87,7 +87,7 @@ stabilize INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:4 + HardState Term:1 Commit:4 Lead:1 Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false Messages: 2->1 MsgAppResp Term:1 Log:0/4 diff --git a/pkg/raft/testdata/confchange_v1_remove_leader.txt b/pkg/raft/testdata/confchange_v1_remove_leader.txt index cc91508a4578..9a60e0e51cb0 100644 --- a/pkg/raft/testdata/confchange_v1_remove_leader.txt +++ b/pkg/raft/testdata/confchange_v1_remove_leader.txt @@ -96,7 +96,7 @@ stabilize 1 2->1 MsgAppResp Term:1 Log:0/5 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 CommittedEntries: 1/4 EntryConfChange r1 1/5 EntryNormal "foo" @@ -122,7 +122,7 @@ stabilize 2 1->2 MsgApp Term:1 Log:1/6 Commit:5 > 2 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 Entries: 1/6 EntryNormal "bar" CommittedEntries: @@ -153,7 +153,7 @@ stabilize 1->3 MsgApp Term:1 Log:1/6 Commit:5 > 3 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 Entries: 1/4 EntryConfChange r1 1/5 EntryNormal "foo" @@ -176,7 +176,7 @@ stabilize 3->1 MsgAppResp Term:1 Log:0/6 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:6 + HardState Term:1 Vote:1 Commit:6 Lead:1 CommittedEntries: 1/6 EntryNormal "bar" Messages: @@ -188,14 +188,14 @@ stabilize 1->3 MsgApp Term:1 Log:1/6 Commit:6 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:6 + HardState Term:1 Vote:1 Commit:6 Lead:1 CommittedEntries: 1/6 EntryNormal "bar" Messages: 2->1 MsgAppResp Term:1 Log:0/6 > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:6 + HardState Term:1 Vote:1 Commit:6 Lead:1 CommittedEntries: 1/6 EntryNormal "bar" Messages: diff --git a/pkg/raft/testdata/confchange_v1_remove_leader_stepdown.txt b/pkg/raft/testdata/confchange_v1_remove_leader_stepdown.txt index fe397650931a..aa92875967a1 100644 --- a/pkg/raft/testdata/confchange_v1_remove_leader_stepdown.txt +++ b/pkg/raft/testdata/confchange_v1_remove_leader_stepdown.txt @@ -95,7 +95,7 @@ stabilize 1 2->1 MsgAppResp Term:1 Log:0/5 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 CommittedEntries: 1/4 EntryConfChange r1 1/5 EntryNormal "foo" @@ -108,7 +108,7 @@ stabilize 1 INFO 1 became follower at term 1 > 1 handling Ready Ready MustSync=false: - Lead:0 State:StateFollower + State:StateFollower raft-state ---- @@ -125,7 +125,7 @@ stabilize 2 1->2 MsgApp Term:1 Log:1/6 Commit:5 > 2 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 Entries: 1/6 EntryNormal "bar" CommittedEntries: @@ -156,7 +156,7 @@ stabilize 1->3 MsgApp Term:1 Log:1/6 Commit:5 > 3 handling Ready Ready MustSync=true: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 Entries: 1/4 EntryConfChange r1 1/5 EntryNormal "foo" @@ -181,7 +181,7 @@ stabilize # n1 can no longer propose. propose 1 baz ---- -INFO 1 no leader at term 1; dropping proposal +INFO 1 not forwarding to itself at term 1; dropping proposal raft proposal dropped # Nor can it campaign to become leader. diff --git a/pkg/raft/testdata/confchange_v2_add_double_auto.txt b/pkg/raft/testdata/confchange_v2_add_double_auto.txt index f290c980e404..b26e73974016 100644 --- a/pkg/raft/testdata/confchange_v2_add_double_auto.txt +++ b/pkg/raft/testdata/confchange_v2_add_double_auto.txt @@ -17,8 +17,8 @@ INFO 1 became candidate at term 1 process-ready 1 ---- Ready MustSync=true: -Lead:0 State:StateCandidate -HardState Term:1 Vote:1 Commit:2 +State:StateCandidate +HardState Term:1 Vote:1 Commit:2 Lead:0 INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 @@ -42,7 +42,8 @@ INFO newRaft 3 [peers: [], term: 0, commit: 0, applied: 0, lastindex: 0, lastter process-ready 1 ---- Ready MustSync=true: -Lead:1 State:StateLeader +State:StateLeader +HardState Term:1 Vote:1 Commit:2 Lead:1 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 v3 @@ -54,7 +55,7 @@ Entries: process-ready 1 ---- Ready MustSync=false: -HardState Term:1 Vote:1 Commit:4 +HardState Term:1 Vote:1 Commit:4 Lead:1 CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 v3 @@ -86,8 +87,7 @@ stabilize 1 2 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower - HardState Term:1 Commit:0 + HardState Term:1 Commit:0 Lead:1 Messages: 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) > 1 receiving messages @@ -110,7 +110,7 @@ stabilize 1 2 INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:4 + HardState Term:1 Commit:4 Lead:1 Snapshot Index:4 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true Messages: 2->1 MsgAppResp Term:1 Log:0/4 @@ -133,7 +133,7 @@ stabilize 1 2 2->1 MsgAppResp Term:1 Log:0/5 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 CommittedEntries: 1/5 EntryConfChangeV2 Messages: @@ -143,7 +143,7 @@ stabilize 1 2 1->2 MsgApp Term:1 Log:1/5 Commit:5 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:5 + HardState Term:1 Commit:5 Lead:1 CommittedEntries: 1/5 EntryConfChangeV2 Messages: @@ -162,8 +162,7 @@ stabilize 1 3 DEBUG 3 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 3 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower - HardState Term:1 Commit:0 + HardState Term:1 Commit:0 Lead:1 Messages: 3->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) > 1 receiving messages @@ -186,7 +185,7 @@ stabilize 1 3 INFO 3 [commit: 5] restored snapshot [index: 5, term: 1] > 3 handling Ready Ready MustSync=false: - HardState Term:1 Commit:5 + HardState Term:1 Commit:5 Lead:1 Snapshot Index:5 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false Messages: 3->1 MsgAppResp Term:1 Log:0/5 @@ -268,7 +267,7 @@ stabilize 1 3->1 MsgAppResp Term:1 Log:0/6 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:6 + HardState Term:1 Vote:1 Commit:6 Lead:1 CommittedEntries: 1/6 EntryConfChangeV2 r2 r3 Messages: @@ -299,7 +298,7 @@ stabilize 2 3 1->3 MsgApp Term:1 Log:1/8 Commit:6 Entries:[1/9 EntryConfChangeV2] > 2 handling Ready Ready MustSync=true: - HardState Term:1 Commit:6 + HardState Term:1 Commit:6 Lead:1 Entries: 1/7 EntryNormal "foo" 1/8 EntryNormal "bar" @@ -314,7 +313,7 @@ stabilize 2 3 INFO 2 switched to configuration voters=(1)&&(1 2 3) autoleave > 3 handling Ready Ready MustSync=true: - HardState Term:1 Commit:6 + HardState Term:1 Commit:6 Lead:1 Entries: 1/7 EntryNormal "foo" 1/8 EntryNormal "bar" @@ -344,7 +343,7 @@ stabilize 3->1 MsgAppResp Term:1 Log:0/9 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:9 + HardState Term:1 Vote:1 Commit:9 Lead:1 CommittedEntries: 1/7 EntryNormal "foo" 1/8 EntryNormal "bar" @@ -367,7 +366,7 @@ stabilize 1->3 MsgApp Term:1 Log:1/9 Commit:9 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:9 + HardState Term:1 Commit:9 Lead:1 CommittedEntries: 1/7 EntryNormal "foo" 1/8 EntryNormal "bar" @@ -379,7 +378,7 @@ stabilize INFO 2 switched to configuration voters=(1) > 3 handling Ready Ready MustSync=false: - HardState Term:1 Commit:9 + HardState Term:1 Commit:9 Lead:1 CommittedEntries: 1/7 EntryNormal "foo" 1/8 EntryNormal "bar" diff --git a/pkg/raft/testdata/confchange_v2_add_double_implicit.txt b/pkg/raft/testdata/confchange_v2_add_double_implicit.txt index 2db3b7c14c78..a2aac25d92e3 100644 --- a/pkg/raft/testdata/confchange_v2_add_double_implicit.txt +++ b/pkg/raft/testdata/confchange_v2_add_double_implicit.txt @@ -19,8 +19,8 @@ INFO 1 became candidate at term 1 process-ready 1 ---- Ready MustSync=true: -Lead:0 State:StateCandidate -HardState Term:1 Vote:1 Commit:2 +State:StateCandidate +HardState Term:1 Vote:1 Commit:2 Lead:0 INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 @@ -44,13 +44,14 @@ stabilize 1 2 ---- > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + State:StateLeader + HardState Term:1 Vote:1 Commit:2 Lead:1 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:4 + HardState Term:1 Vote:1 Commit:4 Lead:1 CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 @@ -69,8 +70,7 @@ stabilize 1 2 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower - HardState Term:1 Commit:0 + HardState Term:1 Commit:0 Lead:1 Messages: 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) > 1 receiving messages @@ -93,7 +93,7 @@ stabilize 1 2 INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:4 + HardState Term:1 Commit:4 Lead:1 Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:true Messages: 2->1 MsgAppResp Term:1 Log:0/4 @@ -116,7 +116,7 @@ stabilize 1 2 2->1 MsgAppResp Term:1 Log:0/5 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 CommittedEntries: 1/5 EntryConfChangeV2 Messages: @@ -126,7 +126,7 @@ stabilize 1 2 1->2 MsgApp Term:1 Log:1/5 Commit:5 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:5 + HardState Term:1 Commit:5 Lead:1 CommittedEntries: 1/5 EntryConfChangeV2 Messages: diff --git a/pkg/raft/testdata/confchange_v2_add_single_auto.txt b/pkg/raft/testdata/confchange_v2_add_single_auto.txt index 3c2341590fd5..df1a5e23e6d3 100644 --- a/pkg/raft/testdata/confchange_v2_add_single_auto.txt +++ b/pkg/raft/testdata/confchange_v2_add_single_auto.txt @@ -17,8 +17,8 @@ INFO 1 became candidate at term 1 process-ready 1 ---- Ready MustSync=true: -Lead:0 State:StateCandidate -HardState Term:1 Vote:1 Commit:2 +State:StateCandidate +HardState Term:1 Vote:1 Commit:2 Lead:0 INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 @@ -42,13 +42,14 @@ stabilize ---- > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + State:StateLeader + HardState Term:1 Vote:1 Commit:2 Lead:1 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:4 + HardState Term:1 Vote:1 Commit:4 Lead:1 CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 @@ -64,8 +65,7 @@ stabilize DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower - HardState Term:1 Commit:0 + HardState Term:1 Commit:0 Lead:1 Messages: 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) > 1 receiving messages @@ -88,7 +88,7 @@ stabilize INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:4 + HardState Term:1 Commit:4 Lead:1 Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false Messages: 2->1 MsgAppResp Term:1 Log:0/4 diff --git a/pkg/raft/testdata/confchange_v2_add_single_explicit.txt b/pkg/raft/testdata/confchange_v2_add_single_explicit.txt index 1d390e55554c..e0857f027adb 100644 --- a/pkg/raft/testdata/confchange_v2_add_single_explicit.txt +++ b/pkg/raft/testdata/confchange_v2_add_single_explicit.txt @@ -17,8 +17,8 @@ INFO 1 became candidate at term 1 process-ready 1 ---- Ready MustSync=true: -Lead:0 State:StateCandidate -HardState Term:1 Vote:1 Commit:2 +State:StateCandidate +HardState Term:1 Vote:1 Commit:2 Lead:0 INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 @@ -42,13 +42,14 @@ stabilize 1 2 ---- > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + State:StateLeader + HardState Term:1 Vote:1 Commit:2 Lead:1 Entries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:4 + HardState Term:1 Vote:1 Commit:4 Lead:1 CommittedEntries: 1/3 EntryNormal "" 1/4 EntryConfChangeV2 v2 @@ -64,8 +65,7 @@ stabilize 1 2 DEBUG 2 [logterm: 0, index: 3] rejected MsgApp [logterm: 1, index: 3] from 1 > 2 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower - HardState Term:1 Commit:0 + HardState Term:1 Commit:0 Lead:1 Messages: 2->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) > 1 receiving messages @@ -88,7 +88,7 @@ stabilize 1 2 INFO 2 [commit: 4] restored snapshot [index: 4, term: 1] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:4 + HardState Term:1 Commit:4 Lead:1 Snapshot Index:4 Term:1 ConfState:Voters:[1 2] VotersOutgoing:[1] Learners:[] LearnersNext:[] AutoLeave:false Messages: 2->1 MsgAppResp Term:1 Log:0/4 @@ -136,7 +136,7 @@ stabilize 2->1 MsgAppResp Term:1 Log:0/6 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:6 + HardState Term:1 Vote:1 Commit:6 Lead:1 CommittedEntries: 1/5 EntryNormal "" 1/6 EntryConfChangeV2 @@ -149,7 +149,7 @@ stabilize 1->2 MsgApp Term:1 Log:1/6 Commit:6 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:6 + HardState Term:1 Commit:6 Lead:1 CommittedEntries: 1/5 EntryNormal "" 1/6 EntryConfChangeV2 @@ -187,7 +187,7 @@ stabilize 2->1 MsgAppResp Term:1 Log:0/7 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:7 + HardState Term:1 Vote:1 Commit:7 Lead:1 CommittedEntries: 1/7 EntryNormal "" Messages: @@ -196,7 +196,7 @@ stabilize 1->2 MsgApp Term:1 Log:1/7 Commit:7 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Commit:7 + HardState Term:1 Commit:7 Lead:1 CommittedEntries: 1/7 EntryNormal "" Messages: diff --git a/pkg/raft/testdata/confchange_v2_replace_leader.txt b/pkg/raft/testdata/confchange_v2_replace_leader.txt index 628cc447e112..44c16fc83668 100644 --- a/pkg/raft/testdata/confchange_v2_replace_leader.txt +++ b/pkg/raft/testdata/confchange_v2_replace_leader.txt @@ -77,7 +77,7 @@ stabilize 3->1 MsgAppResp Term:1 Log:0/4 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:4 + HardState Term:1 Vote:1 Commit:4 Lead:1 CommittedEntries: 1/4 EntryConfChangeV2 r1 v4 Messages: @@ -94,7 +94,7 @@ stabilize 1->4 MsgApp Term:1 Log:1/3 Commit:4 Entries:[1/4 EntryConfChangeV2 r1 v4] > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:4 + HardState Term:1 Vote:1 Commit:4 Lead:1 CommittedEntries: 1/4 EntryConfChangeV2 r1 v4 Messages: @@ -102,7 +102,7 @@ stabilize INFO 2 switched to configuration voters=(2 3 4)&&(1 2 3) > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:4 + HardState Term:1 Vote:1 Commit:4 Lead:1 CommittedEntries: 1/4 EntryConfChangeV2 r1 v4 Messages: @@ -117,8 +117,7 @@ stabilize INFO 4 became follower at term 1 > 4 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower - HardState Term:1 Commit:0 + HardState Term:1 Commit:0 Lead:1 Messages: 4->1 MsgAppResp Term:1 Log:0/3 Rejected (Hint: 0) > 1 receiving messages @@ -137,7 +136,7 @@ stabilize INFO 4 [commit: 4] restored snapshot [index: 4, term: 1] > 4 handling Ready Ready MustSync=false: - HardState Term:1 Commit:4 + HardState Term:1 Commit:4 Lead:1 Snapshot Index:4 Term:1 ConfState:Voters:[2 3 4] VotersOutgoing:[1 2 3] Learners:[] LearnersNext:[] AutoLeave:false Messages: 4->1 MsgAppResp Term:1 Log:0/4 @@ -176,8 +175,8 @@ stabilize INFO 4 [logterm: 1, index: 4] sent MsgVote request to 3 at term 2 > 4 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:2 Vote:4 Commit:4 + State:StateCandidate + HardState Term:2 Vote:4 Commit:4 Lead:0 Messages: 4->1 MsgVote Term:2 Log:1/4 4->2 MsgVote Term:2 Log:1/4 @@ -201,20 +200,18 @@ stabilize INFO 3 [logterm: 1, index: 4, vote: 0] cast MsgVote for 4 [logterm: 1, index: 4] at term 2 > 1 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:4 Commit:4 + State:StateFollower + HardState Term:2 Vote:4 Commit:4 Lead:0 Messages: 1->4 MsgVoteResp Term:2 Log:0/0 > 2 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:4 Commit:4 + HardState Term:2 Vote:4 Commit:4 Lead:0 Messages: 2->4 MsgVoteResp Term:2 Log:0/0 > 3 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:4 Commit:4 + HardState Term:2 Vote:4 Commit:4 Lead:0 Messages: 3->4 MsgVoteResp Term:2 Log:0/0 > 4 receiving messages @@ -228,7 +225,8 @@ stabilize 3->4 MsgVoteResp Term:2 Log:0/0 > 4 handling Ready Ready MustSync=true: - Lead:4 State:StateLeader + State:StateLeader + HardState Term:2 Vote:4 Commit:4 Lead:4 Entries: 2/5 EntryNormal "" Messages: @@ -243,21 +241,21 @@ stabilize 4->3 MsgApp Term:2 Log:1/4 Commit:4 Entries:[2/5 EntryNormal ""] > 1 handling Ready Ready MustSync=true: - Lead:4 State:StateFollower + HardState Term:2 Vote:4 Commit:4 Lead:4 Entries: 2/5 EntryNormal "" Messages: 1->4 MsgAppResp Term:2 Log:0/5 > 2 handling Ready Ready MustSync=true: - Lead:4 State:StateFollower + HardState Term:2 Vote:4 Commit:4 Lead:4 Entries: 2/5 EntryNormal "" Messages: 2->4 MsgAppResp Term:2 Log:0/5 > 3 handling Ready Ready MustSync=true: - Lead:4 State:StateFollower + HardState Term:2 Vote:4 Commit:4 Lead:4 Entries: 2/5 EntryNormal "" Messages: @@ -268,7 +266,7 @@ stabilize 3->4 MsgAppResp Term:2 Log:0/5 > 4 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:5 + HardState Term:2 Vote:4 Commit:5 Lead:4 CommittedEntries: 2/5 EntryNormal "" Messages: @@ -283,21 +281,21 @@ stabilize 4->3 MsgApp Term:2 Log:2/5 Commit:5 > 1 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:5 + HardState Term:2 Vote:4 Commit:5 Lead:4 CommittedEntries: 2/5 EntryNormal "" Messages: 1->4 MsgAppResp Term:2 Log:0/5 > 2 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:5 + HardState Term:2 Vote:4 Commit:5 Lead:4 CommittedEntries: 2/5 EntryNormal "" Messages: 2->4 MsgAppResp Term:2 Log:0/5 > 3 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:5 + HardState Term:2 Vote:4 Commit:5 Lead:4 CommittedEntries: 2/5 EntryNormal "" Messages: @@ -361,7 +359,7 @@ stabilize 3->4 MsgAppResp Term:2 Log:0/6 > 4 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:6 + HardState Term:2 Vote:4 Commit:6 Lead:4 CommittedEntries: 2/6 EntryConfChangeV2 Messages: @@ -377,7 +375,7 @@ stabilize 4->3 MsgApp Term:2 Log:2/6 Commit:6 > 1 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:6 + HardState Term:2 Vote:4 Commit:6 Lead:4 CommittedEntries: 2/6 EntryConfChangeV2 Messages: @@ -385,7 +383,7 @@ stabilize INFO 1 switched to configuration voters=(2 3 4) > 2 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:6 + HardState Term:2 Vote:4 Commit:6 Lead:4 CommittedEntries: 2/6 EntryConfChangeV2 Messages: @@ -393,7 +391,7 @@ stabilize INFO 2 switched to configuration voters=(2 3 4) > 3 handling Ready Ready MustSync=false: - HardState Term:2 Vote:4 Commit:6 + HardState Term:2 Vote:4 Commit:6 Lead:4 CommittedEntries: 2/6 EntryConfChangeV2 Messages: diff --git a/pkg/raft/testdata/confchange_v2_replace_leader_stepdown.txt b/pkg/raft/testdata/confchange_v2_replace_leader_stepdown.txt index 62d01d23b412..29dd420a8f1d 100644 --- a/pkg/raft/testdata/confchange_v2_replace_leader_stepdown.txt +++ b/pkg/raft/testdata/confchange_v2_replace_leader_stepdown.txt @@ -112,7 +112,7 @@ stabilize 4->1 MsgAppResp Term:1 Log:0/5 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 CommittedEntries: 1/5 EntryConfChangeV2 Messages: @@ -129,10 +129,10 @@ stabilize 1->4 MsgApp Term:1 Log:1/5 Commit:5 > 1 handling Ready Ready MustSync=false: - Lead:0 State:StateFollower + State:StateFollower > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 CommittedEntries: 1/5 EntryConfChangeV2 Messages: @@ -140,7 +140,7 @@ stabilize INFO 2 switched to configuration voters=(2 3 4) > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:5 + HardState Term:1 Vote:1 Commit:5 Lead:1 CommittedEntries: 1/5 EntryConfChangeV2 Messages: @@ -148,7 +148,7 @@ stabilize INFO 3 switched to configuration voters=(2 3 4) > 4 handling Ready Ready MustSync=false: - HardState Term:1 Commit:5 + HardState Term:1 Commit:5 Lead:1 CommittedEntries: 1/5 EntryConfChangeV2 Messages: diff --git a/pkg/raft/testdata/forget_leader.txt b/pkg/raft/testdata/forget_leader.txt index a674b346c606..5b6ae89a5c63 100644 --- a/pkg/raft/testdata/forget_leader.txt +++ b/pkg/raft/testdata/forget_leader.txt @@ -80,11 +80,11 @@ stabilize 1->3 MsgHeartbeat Term:1 Log:0/0 Commit:11 1->4 MsgHeartbeat Term:1 Log:0/0 Commit:11 > 2 handling Ready - Ready MustSync=false: - Lead:0 State:StateFollower + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:11 Lead:0 > 4 handling Ready - Ready MustSync=false: - Lead:0 State:StateFollower + Ready MustSync=true: + HardState Term:1 Commit:11 Lead:0 > 2 receiving messages 1->2 MsgHeartbeat Term:1 Log:0/0 Commit:11 > 3 receiving messages @@ -92,8 +92,8 @@ stabilize > 4 receiving messages 1->4 MsgHeartbeat Term:1 Log:0/0 Commit:11 > 2 handling Ready - Ready MustSync=false: - Lead:1 State:StateFollower + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:11 Lead:1 Messages: 2->1 MsgHeartbeatResp Term:1 Log:0/0 > 3 handling Ready @@ -101,8 +101,8 @@ stabilize Messages: 3->1 MsgHeartbeatResp Term:1 Log:0/0 > 4 handling Ready - Ready MustSync=false: - Lead:1 State:StateFollower + Ready MustSync=true: + HardState Term:1 Commit:11 Lead:1 Messages: 4->1 MsgHeartbeatResp Term:1 Log:0/0 > 1 receiving messages diff --git a/pkg/raft/testdata/forget_leader_prevote_checkquorum.txt b/pkg/raft/testdata/forget_leader_prevote_checkquorum.txt index 9b3b80ffae15..f4a4a3c316fc 100644 --- a/pkg/raft/testdata/forget_leader_prevote_checkquorum.txt +++ b/pkg/raft/testdata/forget_leader_prevote_checkquorum.txt @@ -34,8 +34,9 @@ INFO 3 [logterm: 1, index: 11] sent MsgPreVote request to 2 at term 1 stabilize 3 ---- > 3 handling Ready - Ready MustSync=false: - Lead:0 State:StatePreCandidate + Ready MustSync=true: + State:StatePreCandidate + HardState Term:1 Vote:1 Commit:11 Lead:0 Messages: 3->1 MsgPreVote Term:2 Log:1/11 3->2 MsgPreVote Term:2 Log:1/11 @@ -71,8 +72,9 @@ stabilize Messages: 2->1 MsgHeartbeatResp Term:1 Log:0/0 > 3 handling Ready - Ready MustSync=false: - Lead:1 State:StateFollower + Ready MustSync=true: + State:StateFollower + HardState Term:1 Vote:1 Commit:11 Lead:1 Messages: 3->1 MsgHeartbeatResp Term:1 Log:0/0 > 1 receiving messages @@ -107,8 +109,9 @@ INFO 3 [logterm: 1, index: 11] sent MsgPreVote request to 2 at term 1 stabilize 3 ---- > 3 handling Ready - Ready MustSync=false: - Lead:0 State:StatePreCandidate + Ready MustSync=true: + State:StatePreCandidate + HardState Term:1 Vote:1 Commit:11 Lead:0 Messages: 3->1 MsgPreVote Term:2 Log:1/11 3->2 MsgPreVote Term:2 Log:1/11 @@ -118,8 +121,8 @@ stabilize 3 stabilize 2 ---- > 2 handling Ready - Ready MustSync=false: - Lead:0 State:StateFollower + Ready MustSync=true: + HardState Term:1 Vote:1 Commit:11 Lead:0 > 2 receiving messages 3->2 MsgPreVote Term:2 Log:1/11 INFO 2 [logterm: 1, index: 11, vote: 1] cast MsgPreVote for 3 [logterm: 1, index: 11] at term 1 @@ -139,8 +142,8 @@ stabilize 3 INFO 3 [logterm: 1, index: 11] sent MsgVote request to 2 at term 2 > 3 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:2 Vote:3 Commit:11 + State:StateCandidate + HardState Term:2 Vote:3 Commit:11 Lead:0 Messages: 3->1 MsgVote Term:2 Log:1/11 3->2 MsgVote Term:2 Log:1/11 @@ -203,8 +206,9 @@ INFO 1 [logterm: 2, index: 12] sent MsgPreVote request to 3 at term 2 process-ready 1 ---- -Ready MustSync=false: -Lead:0 State:StatePreCandidate +Ready MustSync=true: +State:StatePreCandidate +HardState Term:2 Commit:12 Lead:0 Messages: 1->2 MsgPreVote Term:3 Log:2/12 1->3 MsgPreVote Term:3 Log:2/12 @@ -214,8 +218,8 @@ INFO 1 has received 1 MsgPreVoteResp votes and 0 vote rejections stabilize 2 ---- > 2 handling Ready - Ready MustSync=false: - Lead:0 State:StateFollower + Ready MustSync=true: + HardState Term:2 Vote:3 Commit:12 Lead:0 > 2 receiving messages 1->2 MsgPreVote Term:3 Log:2/12 INFO 2 [logterm: 2, index: 13, vote: 3] rejected MsgPreVote from 1 [logterm: 2, index: 12] at term 2 diff --git a/pkg/raft/testdata/lagging_commit.txt b/pkg/raft/testdata/lagging_commit.txt index 8f8ba33635b2..82ccd3a25460 100644 --- a/pkg/raft/testdata/lagging_commit.txt +++ b/pkg/raft/testdata/lagging_commit.txt @@ -78,7 +78,7 @@ stabilize 1 2 2->1 MsgAppResp Term:1 Log:0/13 > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:13 + HardState Term:1 Vote:1 Commit:13 Lead:1 CommittedEntries: 1/12 EntryNormal "data1" 1/13 EntryNormal "data2" @@ -92,7 +92,7 @@ stabilize 1 2 1->2 MsgApp Term:1 Log:1/13 Commit:13 > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:13 + HardState Term:1 Vote:1 Commit:13 Lead:1 CommittedEntries: 1/12 EntryNormal "data1" 1/13 EntryNormal "data2" @@ -164,7 +164,7 @@ stabilize 1 3 1->3 MsgApp Term:1 Log:1/13 Commit:13 > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:13 + HardState Term:1 Vote:1 Commit:13 Lead:1 CommittedEntries: 1/12 EntryNormal "data1" 1/13 EntryNormal "data2" diff --git a/pkg/raft/testdata/prevote.txt b/pkg/raft/testdata/prevote.txt index db763d3599c6..0c6ff0e2a56e 100644 --- a/pkg/raft/testdata/prevote.txt +++ b/pkg/raft/testdata/prevote.txt @@ -65,8 +65,9 @@ INFO 3 [logterm: 1, index: 11] sent MsgPreVote request to 2 at term 1 process-ready 3 ---- -Ready MustSync=false: -Lead:0 State:StatePreCandidate +Ready MustSync=true: +State:StatePreCandidate +HardState Term:1 Vote:1 Commit:11 Lead:0 Messages: 3->1 MsgPreVote Term:2 Log:1/11 3->2 MsgPreVote Term:2 Log:1/11 @@ -86,7 +87,7 @@ stabilize ---- > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:12 + HardState Term:1 Vote:1 Commit:12 Lead:1 CommittedEntries: 1/12 EntryNormal "prop_1" Messages: @@ -107,15 +108,15 @@ stabilize 2->3 MsgPreVoteResp Term:1 Log:0/0 Rejected (Hint: 0) > 2 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:12 + HardState Term:1 Vote:1 Commit:12 Lead:1 CommittedEntries: 1/12 EntryNormal "prop_1" Messages: 2->1 MsgAppResp Term:1 Log:0/12 > 3 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower - HardState Term:1 Vote:1 Commit:12 + State:StateFollower + HardState Term:1 Vote:1 Commit:12 Lead:1 Entries: 1/12 EntryNormal "prop_1" CommittedEntries: @@ -139,8 +140,9 @@ INFO 2 [logterm: 1, index: 12] sent MsgPreVote request to 3 at term 1 stabilize ---- > 2 handling Ready - Ready MustSync=false: - Lead:0 State:StatePreCandidate + Ready MustSync=true: + State:StatePreCandidate + HardState Term:1 Vote:1 Commit:12 Lead:0 Messages: 2->1 MsgPreVote Term:2 Log:1/12 2->3 MsgPreVote Term:2 Log:1/12 @@ -170,8 +172,8 @@ stabilize 3->2 MsgPreVoteResp Term:2 Log:0/0 > 2 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:2 Vote:2 Commit:12 + State:StateCandidate + HardState Term:2 Vote:2 Commit:12 Lead:0 Messages: 2->1 MsgVote Term:2 Log:1/12 2->3 MsgVote Term:2 Log:1/12 @@ -189,14 +191,13 @@ stabilize INFO 3 [logterm: 1, index: 12, vote: 0] cast MsgVote for 2 [logterm: 1, index: 12] at term 2 > 1 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:2 Commit:12 + State:StateFollower + HardState Term:2 Vote:2 Commit:12 Lead:0 Messages: 1->2 MsgVoteResp Term:2 Log:0/0 > 3 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:2 Commit:12 + HardState Term:2 Vote:2 Commit:12 Lead:0 Messages: 3->2 MsgVoteResp Term:2 Log:0/0 > 2 receiving messages @@ -207,7 +208,8 @@ stabilize 3->2 MsgVoteResp Term:2 Log:0/0 > 2 handling Ready Ready MustSync=true: - Lead:2 State:StateLeader + State:StateLeader + HardState Term:2 Vote:2 Commit:12 Lead:2 Entries: 2/13 EntryNormal "" Messages: @@ -219,14 +221,14 @@ stabilize 2->3 MsgApp Term:2 Log:1/12 Commit:12 Entries:[2/13 EntryNormal ""] > 1 handling Ready Ready MustSync=true: - Lead:2 State:StateFollower + HardState Term:2 Vote:2 Commit:12 Lead:2 Entries: 2/13 EntryNormal "" Messages: 1->2 MsgAppResp Term:2 Log:0/13 > 3 handling Ready Ready MustSync=true: - Lead:2 State:StateFollower + HardState Term:2 Vote:2 Commit:12 Lead:2 Entries: 2/13 EntryNormal "" Messages: @@ -236,7 +238,7 @@ stabilize 3->2 MsgAppResp Term:2 Log:0/13 > 2 handling Ready Ready MustSync=false: - HardState Term:2 Vote:2 Commit:13 + HardState Term:2 Vote:2 Commit:13 Lead:2 CommittedEntries: 2/13 EntryNormal "" Messages: @@ -248,14 +250,14 @@ stabilize 2->3 MsgApp Term:2 Log:2/13 Commit:13 > 1 handling Ready Ready MustSync=false: - HardState Term:2 Vote:2 Commit:13 + HardState Term:2 Vote:2 Commit:13 Lead:2 CommittedEntries: 2/13 EntryNormal "" Messages: 1->2 MsgAppResp Term:2 Log:0/13 > 3 handling Ready Ready MustSync=false: - HardState Term:2 Vote:2 Commit:13 + HardState Term:2 Vote:2 Commit:13 Lead:2 CommittedEntries: 2/13 EntryNormal "" Messages: diff --git a/pkg/raft/testdata/prevote_checkquorum.txt b/pkg/raft/testdata/prevote_checkquorum.txt index 6db6662b0349..7a5a7c86a339 100644 --- a/pkg/raft/testdata/prevote_checkquorum.txt +++ b/pkg/raft/testdata/prevote_checkquorum.txt @@ -34,8 +34,9 @@ INFO 2 [logterm: 1, index: 11] sent MsgPreVote request to 3 at term 1 stabilize ---- > 2 handling Ready - Ready MustSync=false: - Lead:0 State:StatePreCandidate + Ready MustSync=true: + State:StatePreCandidate + HardState Term:1 Vote:1 Commit:11 Lead:0 Messages: 2->1 MsgPreVote Term:2 Log:1/11 2->3 MsgPreVote Term:2 Log:1/11 @@ -67,8 +68,9 @@ INFO 3 [logterm: 1, index: 11] sent MsgPreVote request to 2 at term 1 process-ready 3 ---- -Ready MustSync=false: -Lead:0 State:StatePreCandidate +Ready MustSync=true: +State:StatePreCandidate +HardState Term:1 Vote:1 Commit:11 Lead:0 Messages: 3->1 MsgPreVote Term:2 Log:1/11 3->2 MsgPreVote Term:2 Log:1/11 @@ -100,8 +102,8 @@ stabilize INFO 3 [logterm: 1, index: 11] sent MsgVote request to 2 at term 2 > 3 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:2 Vote:3 Commit:11 + State:StateCandidate + HardState Term:2 Vote:3 Commit:11 Lead:0 Messages: 3->1 MsgVote Term:2 Log:1/11 3->2 MsgVote Term:2 Log:1/11 @@ -117,8 +119,8 @@ stabilize INFO 2 [logterm: 1, index: 11, vote: 0] cast MsgVote for 3 [logterm: 1, index: 11] at term 2 > 2 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:2 Vote:3 Commit:11 + State:StateFollower + HardState Term:2 Vote:3 Commit:11 Lead:0 Messages: 2->3 MsgVoteResp Term:2 Log:0/0 > 3 receiving messages @@ -128,7 +130,8 @@ stabilize INFO 3 became leader at term 2 > 3 handling Ready Ready MustSync=true: - Lead:3 State:StateLeader + State:StateLeader + HardState Term:2 Vote:3 Commit:11 Lead:3 Entries: 2/12 EntryNormal "" Messages: @@ -142,15 +145,15 @@ stabilize 3->2 MsgApp Term:2 Log:1/11 Commit:11 Entries:[2/12 EntryNormal ""] > 1 handling Ready Ready MustSync=true: - Lead:3 State:StateFollower - HardState Term:2 Commit:11 + State:StateFollower + HardState Term:2 Commit:11 Lead:3 Entries: 2/12 EntryNormal "" Messages: 1->3 MsgAppResp Term:2 Log:0/12 > 2 handling Ready Ready MustSync=true: - Lead:3 State:StateFollower + HardState Term:2 Vote:3 Commit:11 Lead:3 Entries: 2/12 EntryNormal "" Messages: @@ -160,7 +163,7 @@ stabilize 2->3 MsgAppResp Term:2 Log:0/12 > 3 handling Ready Ready MustSync=false: - HardState Term:2 Vote:3 Commit:12 + HardState Term:2 Vote:3 Commit:12 Lead:3 CommittedEntries: 2/12 EntryNormal "" Messages: @@ -172,14 +175,14 @@ stabilize 3->2 MsgApp Term:2 Log:2/12 Commit:12 > 1 handling Ready Ready MustSync=false: - HardState Term:2 Commit:12 + HardState Term:2 Commit:12 Lead:3 CommittedEntries: 2/12 EntryNormal "" Messages: 1->3 MsgAppResp Term:2 Log:0/12 > 2 handling Ready Ready MustSync=false: - HardState Term:2 Vote:3 Commit:12 + HardState Term:2 Vote:3 Commit:12 Lead:3 CommittedEntries: 2/12 EntryNormal "" Messages: @@ -206,8 +209,9 @@ INFO 1 [logterm: 2, index: 12] sent MsgPreVote request to 3 at term 2 stabilize ---- > 1 handling Ready - Ready MustSync=false: - Lead:0 State:StatePreCandidate + Ready MustSync=true: + State:StatePreCandidate + HardState Term:2 Commit:12 Lead:0 Messages: 1->2 MsgPreVote Term:3 Log:2/12 1->3 MsgPreVote Term:3 Log:2/12 @@ -230,8 +234,9 @@ INFO 2 [logterm: 2, index: 12] sent MsgPreVote request to 3 at term 2 stabilize ---- > 2 handling Ready - Ready MustSync=false: - Lead:0 State:StatePreCandidate + Ready MustSync=true: + State:StatePreCandidate + HardState Term:2 Vote:3 Commit:12 Lead:0 Messages: 2->1 MsgPreVote Term:3 Log:2/12 2->3 MsgPreVote Term:3 Log:2/12 @@ -256,8 +261,8 @@ stabilize INFO 2 [logterm: 2, index: 12] sent MsgVote request to 3 at term 3 > 2 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:3 Vote:2 Commit:12 + State:StateCandidate + HardState Term:3 Vote:2 Commit:12 Lead:0 Messages: 2->1 MsgVote Term:3 Log:2/12 2->3 MsgVote Term:3 Log:2/12 @@ -273,8 +278,8 @@ stabilize INFO 3 [logterm: 2, index: 12, vote: 3] ignored MsgVote from 2 [logterm: 2, index: 12] at term 2: lease is not expired (remaining ticks: 3) > 1 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:3 Vote:2 Commit:12 + State:StateFollower + HardState Term:3 Vote:2 Commit:12 Lead:0 Messages: 1->2 MsgVoteResp Term:3 Log:0/0 > 2 receiving messages @@ -284,7 +289,8 @@ stabilize INFO 2 became leader at term 3 > 2 handling Ready Ready MustSync=true: - Lead:2 State:StateLeader + State:StateLeader + HardState Term:3 Vote:2 Commit:12 Lead:2 Entries: 3/13 EntryNormal "" Messages: @@ -298,15 +304,15 @@ stabilize INFO 3 became follower at term 3 > 1 handling Ready Ready MustSync=true: - Lead:2 State:StateFollower + HardState Term:3 Vote:2 Commit:12 Lead:2 Entries: 3/13 EntryNormal "" Messages: 1->2 MsgAppResp Term:3 Log:0/13 > 3 handling Ready Ready MustSync=true: - Lead:2 State:StateFollower - HardState Term:3 Commit:12 + State:StateFollower + HardState Term:3 Commit:12 Lead:2 Entries: 3/13 EntryNormal "" Messages: @@ -316,7 +322,7 @@ stabilize 3->2 MsgAppResp Term:3 Log:0/13 > 2 handling Ready Ready MustSync=false: - HardState Term:3 Vote:2 Commit:13 + HardState Term:3 Vote:2 Commit:13 Lead:2 CommittedEntries: 3/13 EntryNormal "" Messages: @@ -328,14 +334,14 @@ stabilize 2->3 MsgApp Term:3 Log:3/13 Commit:13 > 1 handling Ready Ready MustSync=false: - HardState Term:3 Vote:2 Commit:13 + HardState Term:3 Vote:2 Commit:13 Lead:2 CommittedEntries: 3/13 EntryNormal "" Messages: 1->2 MsgAppResp Term:3 Log:0/13 > 3 handling Ready Ready MustSync=false: - HardState Term:3 Commit:13 + HardState Term:3 Commit:13 Lead:2 CommittedEntries: 3/13 EntryNormal "" Messages: diff --git a/pkg/raft/testdata/probe_and_replicate.txt b/pkg/raft/testdata/probe_and_replicate.txt index d970183c677a..a7c3b2a8265e 100644 --- a/pkg/raft/testdata/probe_and_replicate.txt +++ b/pkg/raft/testdata/probe_and_replicate.txt @@ -369,8 +369,8 @@ stabilize 1 ---- > 1 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:8 Vote:1 Commit:18 + State:StateCandidate + HardState Term:8 Vote:1 Commit:18 Lead:0 Messages: 1->2 MsgVote Term:8 Log:6/20 1->3 MsgVote Term:8 Log:6/20 @@ -415,35 +415,34 @@ stabilize 2 3 4 5 6 7 INFO 7 [logterm: 3, index: 21, vote: 0] cast MsgVote for 1 [logterm: 6, index: 20] at term 8 > 2 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:8 Vote:1 Commit:18 + HardState Term:8 Vote:1 Commit:18 Lead:0 Messages: 2->1 MsgVoteResp Term:8 Log:0/0 > 3 handling Ready Ready MustSync=true: - HardState Term:8 Vote:1 Commit:14 + HardState Term:8 Vote:1 Commit:14 Lead:0 Messages: 3->1 MsgVoteResp Term:8 Log:0/0 > 4 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:8 Commit:18 + State:StateFollower + HardState Term:8 Commit:18 Lead:0 Messages: 4->1 MsgVoteResp Term:8 Log:0/0 Rejected (Hint: 0) > 5 handling Ready Ready MustSync=true: - Lead:0 State:StateFollower - HardState Term:8 Commit:18 + State:StateFollower + HardState Term:8 Commit:18 Lead:0 Messages: 5->1 MsgVoteResp Term:8 Log:0/0 Rejected (Hint: 0) > 6 handling Ready Ready MustSync=true: - HardState Term:8 Vote:1 Commit:15 + HardState Term:8 Vote:1 Commit:15 Lead:0 Messages: 6->1 MsgVoteResp Term:8 Log:0/0 > 7 handling Ready Ready MustSync=true: - HardState Term:8 Vote:1 Commit:13 + HardState Term:8 Vote:1 Commit:13 Lead:0 Messages: 7->1 MsgVoteResp Term:8 Log:0/0 @@ -469,7 +468,8 @@ stabilize 1 7->1 MsgVoteResp Term:8 Log:0/0 > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + State:StateLeader + HardState Term:8 Vote:1 Commit:18 Lead:1 Entries: 8/21 EntryNormal "" Messages: @@ -486,8 +486,8 @@ stabilize 1 2 > 2 receiving messages 1->2 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""] > 2 handling Ready - Ready MustSync=false: - Lead:1 State:StateFollower + Ready MustSync=true: + HardState Term:8 Vote:1 Commit:18 Lead:1 Messages: 2->1 MsgAppResp Term:8 Log:6/20 Rejected (Hint: 19) > 1 receiving messages @@ -519,8 +519,8 @@ stabilize 1 3 > 3 receiving messages 1->3 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""] > 3 handling Ready - Ready MustSync=false: - Lead:1 State:StateFollower + Ready MustSync=true: + HardState Term:8 Vote:1 Commit:14 Lead:1 Messages: 3->1 MsgAppResp Term:8 Log:4/20 Rejected (Hint: 14) > 1 receiving messages @@ -549,7 +549,7 @@ stabilize 1 3 ] > 3 handling Ready Ready MustSync=true: - HardState Term:8 Vote:1 Commit:18 + HardState Term:8 Vote:1 Commit:18 Lead:1 Entries: 4/15 EntryNormal "prop_4_15" 5/16 EntryNormal "" @@ -576,7 +576,7 @@ stabilize 1 4 INFO replace the unstable entries from index 21 > 4 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower + HardState Term:8 Commit:18 Lead:1 Entries: 8/21 EntryNormal "" Messages: @@ -585,7 +585,7 @@ stabilize 1 4 4->1 MsgAppResp Term:8 Log:0/21 > 1 handling Ready Ready MustSync=false: - HardState Term:8 Vote:1 Commit:21 + HardState Term:8 Vote:1 Commit:21 Lead:1 CommittedEntries: 6/19 EntryNormal "prop_6_19" 6/20 EntryNormal "prop_6_20" @@ -598,7 +598,7 @@ stabilize 1 4 1->4 MsgApp Term:8 Log:8/21 Commit:21 > 4 handling Ready Ready MustSync=false: - HardState Term:8 Commit:21 + HardState Term:8 Commit:21 Lead:1 CommittedEntries: 6/19 EntryNormal "prop_6_19" 6/20 EntryNormal "prop_6_20" @@ -613,8 +613,8 @@ stabilize 1 5 > 5 receiving messages 1->5 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""] > 5 handling Ready - Ready MustSync=false: - Lead:1 State:StateFollower + Ready MustSync=true: + HardState Term:8 Commit:18 Lead:1 Messages: 5->1 MsgAppResp Term:8 Log:6/20 Rejected (Hint: 18) > 1 receiving messages @@ -637,7 +637,7 @@ stabilize 1 5 INFO replace the unstable entries from index 19 > 5 handling Ready Ready MustSync=true: - HardState Term:8 Commit:21 + HardState Term:8 Commit:21 Lead:1 Entries: 6/19 EntryNormal "prop_6_19" 6/20 EntryNormal "prop_6_20" @@ -656,8 +656,8 @@ stabilize 1 6 > 6 receiving messages 1->6 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""] > 6 handling Ready - Ready MustSync=false: - Lead:1 State:StateFollower + Ready MustSync=true: + HardState Term:8 Vote:1 Commit:15 Lead:1 Messages: 6->1 MsgAppResp Term:8 Log:4/20 Rejected (Hint: 17) > 1 receiving messages @@ -686,7 +686,7 @@ stabilize 1 6 INFO replace the unstable entries from index 16 > 6 handling Ready Ready MustSync=true: - HardState Term:8 Vote:1 Commit:21 + HardState Term:8 Vote:1 Commit:21 Lead:1 Entries: 5/16 EntryNormal "" 5/17 EntryNormal "prop_5_17" @@ -711,8 +711,8 @@ stabilize 1 7 > 7 receiving messages 1->7 MsgApp Term:8 Log:6/20 Commit:18 Entries:[8/21 EntryNormal ""] > 7 handling Ready - Ready MustSync=false: - Lead:1 State:StateFollower + Ready MustSync=true: + HardState Term:8 Vote:1 Commit:13 Lead:1 Messages: 7->1 MsgAppResp Term:8 Log:3/20 Rejected (Hint: 20) > 1 receiving messages @@ -745,7 +745,7 @@ stabilize 1 7 INFO replace the unstable entries from index 14 > 7 handling Ready Ready MustSync=true: - HardState Term:8 Vote:1 Commit:21 + HardState Term:8 Vote:1 Commit:21 Lead:1 Entries: 4/14 EntryNormal "" 4/15 EntryNormal "prop_4_15" diff --git a/pkg/raft/testdata/single_node.txt b/pkg/raft/testdata/single_node.txt index f6aceb712ff4..8e87dc18127b 100644 --- a/pkg/raft/testdata/single_node.txt +++ b/pkg/raft/testdata/single_node.txt @@ -17,18 +17,19 @@ stabilize ---- > 1 handling Ready Ready MustSync=true: - Lead:0 State:StateCandidate - HardState Term:1 Vote:1 Commit:3 + State:StateCandidate + HardState Term:1 Vote:1 Commit:3 Lead:0 INFO 1 received MsgVoteResp from 1 at term 1 INFO 1 has received 1 MsgVoteResp votes and 0 vote rejections INFO 1 became leader at term 1 > 1 handling Ready Ready MustSync=true: - Lead:1 State:StateLeader + State:StateLeader + HardState Term:1 Vote:1 Commit:3 Lead:1 Entries: 1/4 EntryNormal "" > 1 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:4 + HardState Term:1 Vote:1 Commit:4 Lead:1 CommittedEntries: 1/4 EntryNormal "" diff --git a/pkg/raft/testdata/snapshot_succeed_via_app_resp.txt b/pkg/raft/testdata/snapshot_succeed_via_app_resp.txt index b1fc4144e747..28a969b9226c 100644 --- a/pkg/raft/testdata/snapshot_succeed_via_app_resp.txt +++ b/pkg/raft/testdata/snapshot_succeed_via_app_resp.txt @@ -76,8 +76,7 @@ stabilize 3 INFO 3 became follower at term 1 > 3 handling Ready Ready MustSync=true: - Lead:1 State:StateFollower - HardState Term:1 Commit:0 + HardState Term:1 Commit:0 Lead:1 Messages: 3->1 MsgHeartbeatResp Term:1 Log:0/0 @@ -115,7 +114,7 @@ stabilize 3 INFO 3 [commit: 11] restored snapshot [index: 11, term: 1] > 3 handling Ready Ready MustSync=false: - HardState Term:1 Commit:11 + HardState Term:1 Commit:11 Lead:1 Snapshot Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false Messages: 3->1 MsgAppResp Term:1 Log:0/11 diff --git a/pkg/raft/testdata/snapshot_succeed_via_app_resp_behind.txt b/pkg/raft/testdata/snapshot_succeed_via_app_resp_behind.txt index 3bab832d6009..38ca5791fd68 100644 --- a/pkg/raft/testdata/snapshot_succeed_via_app_resp_behind.txt +++ b/pkg/raft/testdata/snapshot_succeed_via_app_resp_behind.txt @@ -99,8 +99,8 @@ DEBUG 3 [logterm: 0, index: 10] rejected MsgApp [logterm: 1, index: 10] from 1 # PendingSnapshot=lastIndex=12. process-ready 3 ---- -Ready MustSync=false: -Lead:1 State:StateFollower +Ready MustSync=true: +HardState Term:1 Vote:1 Commit:5 Lead:1 Messages: 3->1 MsgAppResp Term:1 Log:1/10 Rejected (Hint: 5) @@ -143,7 +143,7 @@ stabilize 3 ---- > 3 handling Ready Ready MustSync=false: - HardState Term:1 Vote:1 Commit:11 + HardState Term:1 Vote:1 Commit:11 Lead:1 Snapshot Index:11 Term:1 ConfState:Voters:[1 2 3] VotersOutgoing:[] Learners:[] LearnersNext:[] AutoLeave:false Messages: 3->1 MsgAppResp Term:1 Log:0/11 diff --git a/pkg/raft/util.go b/pkg/raft/util.go index c5929cf9067a..488eff012c03 100644 --- a/pkg/raft/util.go +++ b/pkg/raft/util.go @@ -87,11 +87,12 @@ func DescribeHardState(hs pb.HardState) string { fmt.Fprintf(&buf, " Vote:%d", hs.Vote) } fmt.Fprintf(&buf, " Commit:%d", hs.Commit) + fmt.Fprintf(&buf, " Lead:%d", hs.Lead) return buf.String() } func DescribeSoftState(ss SoftState) string { - return fmt.Sprintf("Lead:%d State:%s", ss.Lead, ss.RaftState) + return fmt.Sprintf("State:%s", ss.RaftState) } func DescribeConfState(state pb.ConfState) string { diff --git a/pkg/sql/importer/import_csv_mark_redaction_test.go b/pkg/sql/importer/import_csv_mark_redaction_test.go index 209fbeca251e..d76be94faf0b 100644 --- a/pkg/sql/importer/import_csv_mark_redaction_test.go +++ b/pkg/sql/importer/import_csv_mark_redaction_test.go @@ -31,7 +31,7 @@ func TestMarkRedactionCCLStatement(t *testing.T) { }{ { "IMPORT CSV 'file' WITH delimiter = 'foo'", - "IMPORT CSV ‹'file'› WITH OPTIONS (delimiter = ‹'foo'›)", + "IMPORT CSV ‹'*****'› WITH OPTIONS (delimiter = ‹'foo'›)", }, } diff --git a/pkg/sql/importer/import_planning.go b/pkg/sql/importer/import_planning.go index fe3376f0813f..459b8970622b 100644 --- a/pkg/sql/importer/import_planning.go +++ b/pkg/sql/importer/import_planning.go @@ -254,7 +254,9 @@ func importJobDescription( } sort.Slice(stmt.Options, func(i, j int) bool { return stmt.Options[i].Key < stmt.Options[j].Key }) ann := p.ExtendedEvalContext().Annotations - return tree.AsStringWithFQNames(&stmt, ann), nil + return tree.AsStringWithFlags( + &stmt, tree.FmtAlwaysQualifyNames|tree.FmtShowFullURIs, tree.FmtAnnotations(ann), + ), nil } func logSanitizedImportDestination(ctx context.Context, destination string) { diff --git a/pkg/sql/parser/testdata/alter_backup b/pkg/sql/parser/testdata/alter_backup index 81d9a09db33c..c08c44a10ca9 100644 --- a/pkg/sql/parser/testdata/alter_backup +++ b/pkg/sql/parser/testdata/alter_backup @@ -1,63 +1,71 @@ parse ALTER BACKUP 'foo' ADD NEW_KMS = 'a' WITH OLD_KMS = 'b' ---- -ALTER BACKUP 'foo' ADD NEW_KMS='a' WITH OLD_KMS='b' -- normalized! -ALTER BACKUP ('foo') ADD NEW_KMS=('a') WITH OLD_KMS=('b') -- fully parenthesized +ALTER BACKUP '*****' ADD NEW_KMS='*****' WITH OLD_KMS='*****' -- normalized! +ALTER BACKUP ('*****') ADD NEW_KMS=('*****') WITH OLD_KMS=('*****') -- fully parenthesized ALTER BACKUP '_' ADD NEW_KMS='_' WITH OLD_KMS='_' -- literals removed -ALTER BACKUP 'foo' ADD NEW_KMS='a' WITH OLD_KMS='b' -- identifiers removed +ALTER BACKUP '*****' ADD NEW_KMS='*****' WITH OLD_KMS='*****' -- identifiers removed +ALTER BACKUP 'foo' ADD NEW_KMS='a' WITH OLD_KMS='b' -- passwords exposed parse ALTER BACKUP 'foo' ADD NEW_KMS = ('a', 'b') WITH OLD_KMS = ('c', 'd') ---- -ALTER BACKUP 'foo' ADD NEW_KMS=('a', 'b') WITH OLD_KMS=('c', 'd') -- normalized! -ALTER BACKUP ('foo') ADD NEW_KMS=(('a'), ('b')) WITH OLD_KMS=(('c'), ('d')) -- fully parenthesized +ALTER BACKUP '*****' ADD NEW_KMS=('*****', '*****') WITH OLD_KMS=('*****', '*****') -- normalized! +ALTER BACKUP ('*****') ADD NEW_KMS=(('*****'), ('*****')) WITH OLD_KMS=(('*****'), ('*****')) -- fully parenthesized ALTER BACKUP '_' ADD NEW_KMS=('_', '_') WITH OLD_KMS=('_', '_') -- literals removed -ALTER BACKUP 'foo' ADD NEW_KMS=('a', 'b') WITH OLD_KMS=('c', 'd') -- identifiers removed +ALTER BACKUP '*****' ADD NEW_KMS=('*****', '*****') WITH OLD_KMS=('*****', '*****') -- identifiers removed +ALTER BACKUP 'foo' ADD NEW_KMS=('a', 'b') WITH OLD_KMS=('c', 'd') -- passwords exposed parse ALTER BACKUP 'foo' ADD NEW_KMS = ('a', 'b') WITH OLD_KMS = 'c' ---- -ALTER BACKUP 'foo' ADD NEW_KMS=('a', 'b') WITH OLD_KMS='c' -- normalized! -ALTER BACKUP ('foo') ADD NEW_KMS=(('a'), ('b')) WITH OLD_KMS=('c') -- fully parenthesized +ALTER BACKUP '*****' ADD NEW_KMS=('*****', '*****') WITH OLD_KMS='*****' -- normalized! +ALTER BACKUP ('*****') ADD NEW_KMS=(('*****'), ('*****')) WITH OLD_KMS=('*****') -- fully parenthesized ALTER BACKUP '_' ADD NEW_KMS=('_', '_') WITH OLD_KMS='_' -- literals removed -ALTER BACKUP 'foo' ADD NEW_KMS=('a', 'b') WITH OLD_KMS='c' -- identifiers removed +ALTER BACKUP '*****' ADD NEW_KMS=('*****', '*****') WITH OLD_KMS='*****' -- identifiers removed +ALTER BACKUP 'foo' ADD NEW_KMS=('a', 'b') WITH OLD_KMS='c' -- passwords exposed parse ALTER BACKUP 'foo' ADD NEW_KMS = 'a' WITH OLD_KMS = ('b', 'c') ---- -ALTER BACKUP 'foo' ADD NEW_KMS='a' WITH OLD_KMS=('b', 'c') -- normalized! -ALTER BACKUP ('foo') ADD NEW_KMS=('a') WITH OLD_KMS=(('b'), ('c')) -- fully parenthesized +ALTER BACKUP '*****' ADD NEW_KMS='*****' WITH OLD_KMS=('*****', '*****') -- normalized! +ALTER BACKUP ('*****') ADD NEW_KMS=('*****') WITH OLD_KMS=(('*****'), ('*****')) -- fully parenthesized ALTER BACKUP '_' ADD NEW_KMS='_' WITH OLD_KMS=('_', '_') -- literals removed -ALTER BACKUP 'foo' ADD NEW_KMS='a' WITH OLD_KMS=('b', 'c') -- identifiers removed +ALTER BACKUP '*****' ADD NEW_KMS='*****' WITH OLD_KMS=('*****', '*****') -- identifiers removed +ALTER BACKUP 'foo' ADD NEW_KMS='a' WITH OLD_KMS=('b', 'c') -- passwords exposed parse ALTER BACKUP 'foo' in 'bar' ADD NEW_KMS = 'a' WITH OLD_KMS = 'b' ---- -ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS='a' WITH OLD_KMS='b' -- normalized! -ALTER BACKUP ('foo') IN ('bar') ADD NEW_KMS=('a') WITH OLD_KMS=('b') -- fully parenthesized +ALTER BACKUP 'foo' IN '*****' ADD NEW_KMS='*****' WITH OLD_KMS='*****' -- normalized! +ALTER BACKUP ('foo') IN ('*****') ADD NEW_KMS=('*****') WITH OLD_KMS=('*****') -- fully parenthesized ALTER BACKUP '_' IN '_' ADD NEW_KMS='_' WITH OLD_KMS='_' -- literals removed -ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS='a' WITH OLD_KMS='b' -- identifiers removed +ALTER BACKUP 'foo' IN '*****' ADD NEW_KMS='*****' WITH OLD_KMS='*****' -- identifiers removed +ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS='a' WITH OLD_KMS='b' -- passwords exposed parse ALTER BACKUP 'foo' in 'bar' ADD NEW_KMS = ('a', 'b') WITH OLD_KMS = ('c', 'd') ---- -ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS=('a', 'b') WITH OLD_KMS=('c', 'd') -- normalized! -ALTER BACKUP ('foo') IN ('bar') ADD NEW_KMS=(('a'), ('b')) WITH OLD_KMS=(('c'), ('d')) -- fully parenthesized +ALTER BACKUP 'foo' IN '*****' ADD NEW_KMS=('*****', '*****') WITH OLD_KMS=('*****', '*****') -- normalized! +ALTER BACKUP ('foo') IN ('*****') ADD NEW_KMS=(('*****'), ('*****')) WITH OLD_KMS=(('*****'), ('*****')) -- fully parenthesized ALTER BACKUP '_' IN '_' ADD NEW_KMS=('_', '_') WITH OLD_KMS=('_', '_') -- literals removed -ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS=('a', 'b') WITH OLD_KMS=('c', 'd') -- identifiers removed +ALTER BACKUP 'foo' IN '*****' ADD NEW_KMS=('*****', '*****') WITH OLD_KMS=('*****', '*****') -- identifiers removed +ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS=('a', 'b') WITH OLD_KMS=('c', 'd') -- passwords exposed parse ALTER BACKUP 'foo' in 'bar' ADD NEW_KMS = ('a', 'b') WITH OLD_KMS = 'c' ---- -ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS=('a', 'b') WITH OLD_KMS='c' -- normalized! -ALTER BACKUP ('foo') IN ('bar') ADD NEW_KMS=(('a'), ('b')) WITH OLD_KMS=('c') -- fully parenthesized +ALTER BACKUP 'foo' IN '*****' ADD NEW_KMS=('*****', '*****') WITH OLD_KMS='*****' -- normalized! +ALTER BACKUP ('foo') IN ('*****') ADD NEW_KMS=(('*****'), ('*****')) WITH OLD_KMS=('*****') -- fully parenthesized ALTER BACKUP '_' IN '_' ADD NEW_KMS=('_', '_') WITH OLD_KMS='_' -- literals removed -ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS=('a', 'b') WITH OLD_KMS='c' -- identifiers removed +ALTER BACKUP 'foo' IN '*****' ADD NEW_KMS=('*****', '*****') WITH OLD_KMS='*****' -- identifiers removed +ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS=('a', 'b') WITH OLD_KMS='c' -- passwords exposed parse ALTER BACKUP 'foo' in 'bar' ADD NEW_KMS = 'a' WITH OLD_KMS = ('b', 'c') ---- -ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS='a' WITH OLD_KMS=('b', 'c') -- normalized! -ALTER BACKUP ('foo') IN ('bar') ADD NEW_KMS=('a') WITH OLD_KMS=(('b'), ('c')) -- fully parenthesized +ALTER BACKUP 'foo' IN '*****' ADD NEW_KMS='*****' WITH OLD_KMS=('*****', '*****') -- normalized! +ALTER BACKUP ('foo') IN ('*****') ADD NEW_KMS=('*****') WITH OLD_KMS=(('*****'), ('*****')) -- fully parenthesized ALTER BACKUP '_' IN '_' ADD NEW_KMS='_' WITH OLD_KMS=('_', '_') -- literals removed -ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS='a' WITH OLD_KMS=('b', 'c') -- identifiers removed +ALTER BACKUP 'foo' IN '*****' ADD NEW_KMS='*****' WITH OLD_KMS=('*****', '*****') -- identifiers removed +ALTER BACKUP 'foo' IN 'bar' ADD NEW_KMS='a' WITH OLD_KMS=('b', 'c') -- passwords exposed diff --git a/pkg/sql/parser/testdata/alter_changefeed b/pkg/sql/parser/testdata/alter_changefeed index ba8cebdbd532..51e16fdecc79 100644 --- a/pkg/sql/parser/testdata/alter_changefeed +++ b/pkg/sql/parser/testdata/alter_changefeed @@ -98,6 +98,15 @@ ALTER CHANGEFEED (123) ADD TABLE (foo) SET bar = ('baz'), qux = ('quux') DROP ALTER CHANGEFEED _ ADD TABLE foo SET bar = '_', qux = '_' DROP TABLE corge -- literals removed ALTER CHANGEFEED 123 ADD TABLE _ SET _ = 'baz', _ = 'quux' DROP TABLE _ -- identifiers removed +parse +ALTER CHANGEFEED 123 SET sink = 'bar' +---- +ALTER CHANGEFEED 123 SET sink = '*****' -- normalized! +ALTER CHANGEFEED (123) SET sink = ('*****') -- fully parenthesized +ALTER CHANGEFEED _ SET sink = '_' -- literals removed +ALTER CHANGEFEED 123 SET _ = '*****' -- identifiers removed +ALTER CHANGEFEED 123 SET sink = 'bar' -- passwords exposed + parse ALTER CHANGEFEED 123 UNSET foo ---- diff --git a/pkg/sql/parser/testdata/backup_restore b/pkg/sql/parser/testdata/backup_restore index 4746a5fb1fa0..e544118eeaa7 100644 --- a/pkg/sql/parser/testdata/backup_restore +++ b/pkg/sql/parser/testdata/backup_restore @@ -1,66 +1,74 @@ parse BACKUP TABLE foo TO 'bar' ---- -BACKUP TABLE foo TO 'bar' -BACKUP TABLE (foo) TO ('bar') -- fully parenthesized +BACKUP TABLE foo TO '*****' -- normalized! +BACKUP TABLE (foo) TO ('*****') -- fully parenthesized BACKUP TABLE foo TO '_' -- literals removed -BACKUP TABLE _ TO 'bar' -- identifiers removed +BACKUP TABLE _ TO '*****' -- identifiers removed +BACKUP TABLE foo TO 'bar' -- passwords exposed parse BACKUP foo TO 'bar' ---- -BACKUP TABLE foo TO 'bar' -- normalized! -BACKUP TABLE (foo) TO ('bar') -- fully parenthesized +BACKUP TABLE foo TO '*****' -- normalized! +BACKUP TABLE (foo) TO ('*****') -- fully parenthesized BACKUP TABLE foo TO '_' -- literals removed -BACKUP TABLE _ TO 'bar' -- identifiers removed +BACKUP TABLE _ TO '*****' -- identifiers removed +BACKUP TABLE foo TO 'bar' -- passwords exposed parse BACKUP TO 'bar' ---- -BACKUP TO 'bar' -BACKUP TO ('bar') -- fully parenthesized +BACKUP TO '*****' -- normalized! +BACKUP TO ('*****') -- fully parenthesized BACKUP TO '_' -- literals removed -BACKUP TO 'bar' -- identifiers removed +BACKUP TO '*****' -- identifiers removed +BACKUP TO 'bar' -- passwords exposed parse BACKUP role TO 'bar' ---- -BACKUP TABLE "role" TO 'bar' -- normalized! -BACKUP TABLE ("role") TO ('bar') -- fully parenthesized +BACKUP TABLE "role" TO '*****' -- normalized! +BACKUP TABLE ("role") TO ('*****') -- fully parenthesized BACKUP TABLE "role" TO '_' -- literals removed -BACKUP TABLE _ TO 'bar' -- identifiers removed +BACKUP TABLE _ TO '*****' -- identifiers removed +BACKUP TABLE "role" TO 'bar' -- passwords exposed parse BACKUP TABLE foo INTO 'bar' ---- -BACKUP TABLE foo INTO 'bar' -BACKUP TABLE (foo) INTO ('bar') -- fully parenthesized +BACKUP TABLE foo INTO '*****' -- normalized! +BACKUP TABLE (foo) INTO ('*****') -- fully parenthesized BACKUP TABLE foo INTO '_' -- literals removed -BACKUP TABLE _ INTO 'bar' -- identifiers removed +BACKUP TABLE _ INTO '*****' -- identifiers removed +BACKUP TABLE foo INTO 'bar' -- passwords exposed parse BACKUP TABLE foo INTO LATEST IN 'bar' ---- -BACKUP TABLE foo INTO LATEST IN 'bar' -BACKUP TABLE (foo) INTO LATEST IN ('bar') -- fully parenthesized +BACKUP TABLE foo INTO LATEST IN '*****' -- normalized! +BACKUP TABLE (foo) INTO LATEST IN ('*****') -- fully parenthesized BACKUP TABLE foo INTO LATEST IN '_' -- literals removed -BACKUP TABLE _ INTO LATEST IN 'bar' -- identifiers removed +BACKUP TABLE _ INTO LATEST IN '*****' -- identifiers removed +BACKUP TABLE foo INTO LATEST IN 'bar' -- passwords exposed parse BACKUP TABLE foo INTO LATEST IN 'bar' WITH incremental_location = 'baz' ---- -BACKUP TABLE foo INTO LATEST IN 'bar' WITH OPTIONS (incremental_location = 'baz') -- normalized! -BACKUP TABLE (foo) INTO LATEST IN ('bar') WITH OPTIONS (incremental_location = ('baz')) -- fully parenthesized +BACKUP TABLE foo INTO LATEST IN '*****' WITH OPTIONS (incremental_location = '*****') -- normalized! +BACKUP TABLE (foo) INTO LATEST IN ('*****') WITH OPTIONS (incremental_location = ('*****')) -- fully parenthesized BACKUP TABLE foo INTO LATEST IN '_' WITH OPTIONS (incremental_location = '_') -- literals removed -BACKUP TABLE _ INTO LATEST IN 'bar' WITH OPTIONS (incremental_location = 'baz') -- identifiers removed +BACKUP TABLE _ INTO LATEST IN '*****' WITH OPTIONS (incremental_location = '*****') -- identifiers removed +BACKUP TABLE foo INTO LATEST IN 'bar' WITH OPTIONS (incremental_location = 'baz') -- passwords exposed parse BACKUP TABLE foo INTO 'subdir' IN 'bar' ---- -BACKUP TABLE foo INTO 'subdir' IN 'bar' -BACKUP TABLE (foo) INTO ('subdir') IN ('bar') -- fully parenthesized +BACKUP TABLE foo INTO 'subdir' IN '*****' -- normalized! +BACKUP TABLE (foo) INTO ('subdir') IN ('*****') -- fully parenthesized BACKUP TABLE foo INTO '_' IN '_' -- literals removed -BACKUP TABLE _ INTO 'subdir' IN 'bar' -- identifiers removed +BACKUP TABLE _ INTO 'subdir' IN '*****' -- identifiers removed +BACKUP TABLE foo INTO 'subdir' IN 'bar' -- passwords exposed parse BACKUP TABLE foo INTO $1 IN $2 @@ -73,124 +81,138 @@ BACKUP TABLE _ INTO $1 IN $2 -- identifiers removed parse BACKUP TABLE foo INTO LATEST IN 'bar' WITH updates_cluster_monitoring_metrics ---- -BACKUP TABLE foo INTO LATEST IN 'bar' WITH OPTIONS (updates_cluster_monitoring_metrics = true) -- normalized! -BACKUP TABLE (foo) INTO LATEST IN ('bar') WITH OPTIONS (updates_cluster_monitoring_metrics = (true)) -- fully parenthesized +BACKUP TABLE foo INTO LATEST IN '*****' WITH OPTIONS (updates_cluster_monitoring_metrics = true) -- normalized! +BACKUP TABLE (foo) INTO LATEST IN ('*****') WITH OPTIONS (updates_cluster_monitoring_metrics = (true)) -- fully parenthesized BACKUP TABLE foo INTO LATEST IN '_' WITH OPTIONS (updates_cluster_monitoring_metrics = _) -- literals removed -BACKUP TABLE _ INTO LATEST IN 'bar' WITH OPTIONS (updates_cluster_monitoring_metrics = true) -- identifiers removed +BACKUP TABLE _ INTO LATEST IN '*****' WITH OPTIONS (updates_cluster_monitoring_metrics = true) -- identifiers removed +BACKUP TABLE foo INTO LATEST IN 'bar' WITH OPTIONS (updates_cluster_monitoring_metrics = true) -- passwords exposed parse EXPLAIN BACKUP TABLE foo TO 'bar' ---- -EXPLAIN BACKUP TABLE foo TO 'bar' -EXPLAIN BACKUP TABLE (foo) TO ('bar') -- fully parenthesized +EXPLAIN BACKUP TABLE foo TO '*****' -- normalized! +EXPLAIN BACKUP TABLE (foo) TO ('*****') -- fully parenthesized EXPLAIN BACKUP TABLE foo TO '_' -- literals removed -EXPLAIN BACKUP TABLE _ TO 'bar' -- identifiers removed +EXPLAIN BACKUP TABLE _ TO '*****' -- identifiers removed +EXPLAIN BACKUP TABLE foo TO 'bar' -- passwords exposed parse BACKUP TABLE foo.foo, baz.baz TO 'bar' ---- -BACKUP TABLE foo.foo, baz.baz TO 'bar' -BACKUP TABLE (foo.foo), (baz.baz) TO ('bar') -- fully parenthesized +BACKUP TABLE foo.foo, baz.baz TO '*****' -- normalized! +BACKUP TABLE (foo.foo), (baz.baz) TO ('*****') -- fully parenthesized BACKUP TABLE foo.foo, baz.baz TO '_' -- literals removed -BACKUP TABLE _._, _._ TO 'bar' -- identifiers removed +BACKUP TABLE _._, _._ TO '*****' -- identifiers removed +BACKUP TABLE foo.foo, baz.baz TO 'bar' -- passwords exposed parse BACKUP foo.foo, baz.baz TO 'bar' ---- -BACKUP TABLE foo.foo, baz.baz TO 'bar' -- normalized! -BACKUP TABLE (foo.foo), (baz.baz) TO ('bar') -- fully parenthesized +BACKUP TABLE foo.foo, baz.baz TO '*****' -- normalized! +BACKUP TABLE (foo.foo), (baz.baz) TO ('*****') -- fully parenthesized BACKUP TABLE foo.foo, baz.baz TO '_' -- literals removed -BACKUP TABLE _._, _._ TO 'bar' -- identifiers removed +BACKUP TABLE _._, _._ TO '*****' -- identifiers removed +BACKUP TABLE foo.foo, baz.baz TO 'bar' -- passwords exposed parse SHOW BACKUP 'bar' ---- -SHOW BACKUP 'bar' -SHOW BACKUP ('bar') -- fully parenthesized +SHOW BACKUP '*****' -- normalized! +SHOW BACKUP ('*****') -- fully parenthesized SHOW BACKUP '_' -- literals removed -SHOW BACKUP 'bar' -- identifiers removed +SHOW BACKUP '*****' -- identifiers removed +SHOW BACKUP 'bar' -- passwords exposed parse SHOW BACKUP 'bar' WITH ENCRYPTION_PASSPHRASE = 'secret', CHECK_FILES ---- -SHOW BACKUP 'bar' WITH OPTIONS (check_files, encryption_passphrase = '*****') -- normalized! -SHOW BACKUP ('bar') WITH OPTIONS (check_files, encryption_passphrase = '*****') -- fully parenthesized +SHOW BACKUP '*****' WITH OPTIONS (check_files, encryption_passphrase = '*****') -- normalized! +SHOW BACKUP ('*****') WITH OPTIONS (check_files, encryption_passphrase = '*****') -- fully parenthesized SHOW BACKUP '_' WITH OPTIONS (check_files, encryption_passphrase = '*****') -- literals removed -SHOW BACKUP 'bar' WITH OPTIONS (check_files, encryption_passphrase = '*****') -- identifiers removed +SHOW BACKUP '*****' WITH OPTIONS (check_files, encryption_passphrase = '*****') -- identifiers removed SHOW BACKUP 'bar' WITH OPTIONS (check_files, encryption_passphrase = 'secret') -- passwords exposed parse SHOW BACKUP FROM LATEST IN 'bar' WITH incremental_location = 'baz', skip size ---- -SHOW BACKUP FROM 'latest' IN 'bar' WITH OPTIONS (incremental_location = 'baz', skip size) -- normalized! -SHOW BACKUP FROM ('latest') IN ('bar') WITH OPTIONS (incremental_location = ('baz'), skip size) -- fully parenthesized +SHOW BACKUP FROM 'latest' IN '*****' WITH OPTIONS (incremental_location = '*****', skip size) -- normalized! +SHOW BACKUP FROM ('latest') IN ('*****') WITH OPTIONS (incremental_location = ('*****'), skip size) -- fully parenthesized SHOW BACKUP FROM '_' IN '_' WITH OPTIONS (incremental_location = '_', skip size) -- literals removed -SHOW BACKUP FROM 'latest' IN 'bar' WITH OPTIONS (incremental_location = 'baz', skip size) -- identifiers removed +SHOW BACKUP FROM 'latest' IN '*****' WITH OPTIONS (incremental_location = '*****', skip size) -- identifiers removed +SHOW BACKUP FROM 'latest' IN 'bar' WITH OPTIONS (incremental_location = 'baz', skip size) -- passwords exposed parse SHOW BACKUP FROM LATEST IN ('bar','bar1') WITH KMS = ('foo', 'bar'), incremental_location=('hi','hello') ---- -SHOW BACKUP FROM 'latest' IN ('bar', 'bar1') WITH OPTIONS (incremental_location = ('hi', 'hello'), kms = ('foo', 'bar')) -- normalized! -SHOW BACKUP FROM ('latest') IN (('bar'), ('bar1')) WITH OPTIONS (incremental_location = (('hi'), ('hello')), kms = (('foo'), ('bar'))) -- fully parenthesized +SHOW BACKUP FROM 'latest' IN ('*****', '*****') WITH OPTIONS (incremental_location = ('*****', '*****'), kms = ('*****', '*****')) -- normalized! +SHOW BACKUP FROM ('latest') IN (('*****'), ('*****')) WITH OPTIONS (incremental_location = (('*****'), ('*****')), kms = (('*****'), ('*****'))) -- fully parenthesized SHOW BACKUP FROM '_' IN ('_', '_') WITH OPTIONS (incremental_location = ('_', '_'), kms = ('_', '_')) -- literals removed -SHOW BACKUP FROM 'latest' IN ('bar', 'bar1') WITH OPTIONS (incremental_location = ('hi', 'hello'), kms = ('foo', 'bar')) -- identifiers removed +SHOW BACKUP FROM 'latest' IN ('*****', '*****') WITH OPTIONS (incremental_location = ('*****', '*****'), kms = ('*****', '*****')) -- identifiers removed +SHOW BACKUP FROM 'latest' IN ('bar', 'bar1') WITH OPTIONS (incremental_location = ('hi', 'hello'), kms = ('foo', 'bar')) -- passwords exposed parse EXPLAIN SHOW BACKUP 'bar' ---- -EXPLAIN SHOW BACKUP 'bar' -EXPLAIN SHOW BACKUP ('bar') -- fully parenthesized +EXPLAIN SHOW BACKUP '*****' -- normalized! +EXPLAIN SHOW BACKUP ('*****') -- fully parenthesized EXPLAIN SHOW BACKUP '_' -- literals removed -EXPLAIN SHOW BACKUP 'bar' -- identifiers removed +EXPLAIN SHOW BACKUP '*****' -- identifiers removed +EXPLAIN SHOW BACKUP 'bar' -- passwords exposed parse SHOW BACKUP RANGES 'bar' ---- -SHOW BACKUP RANGES 'bar' -SHOW BACKUP RANGES ('bar') -- fully parenthesized +SHOW BACKUP RANGES '*****' -- normalized! +SHOW BACKUP RANGES ('*****') -- fully parenthesized SHOW BACKUP RANGES '_' -- literals removed -SHOW BACKUP RANGES 'bar' -- identifiers removed +SHOW BACKUP RANGES '*****' -- identifiers removed +SHOW BACKUP RANGES 'bar' -- passwords exposed parse SHOW BACKUP FILES 'bar' ---- -SHOW BACKUP FILES 'bar' -SHOW BACKUP FILES ('bar') -- fully parenthesized +SHOW BACKUP FILES '*****' -- normalized! +SHOW BACKUP FILES ('*****') -- fully parenthesized SHOW BACKUP FILES '_' -- literals removed -SHOW BACKUP FILES 'bar' -- identifiers removed +SHOW BACKUP FILES '*****' -- identifiers removed +SHOW BACKUP FILES 'bar' -- passwords exposed parse SHOW BACKUP CONNECTION 'bar' ---- -SHOW BACKUP CONNECTION 'bar' -SHOW BACKUP CONNECTION ('bar') -- fully parenthesized +SHOW BACKUP CONNECTION '*****' -- normalized! +SHOW BACKUP CONNECTION ('*****') -- fully parenthesized SHOW BACKUP CONNECTION '_' -- literals removed -SHOW BACKUP CONNECTION 'bar' -- identifiers removed +SHOW BACKUP CONNECTION '*****' -- identifiers removed +SHOW BACKUP CONNECTION 'bar' -- passwords exposed parse SHOW BACKUP CONNECTION 'bar' WITH TRANSFER = '1KiB', TIME = '1h', CONCURRENTLY = 3 ---- -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '1KiB', TIME = '1h') -- normalized! -SHOW BACKUP CONNECTION ('bar') WITH OPTIONS (CONCURRENTLY = (3), TRANSFER = ('1KiB'), TIME = ('1h')) -- fully parenthesized +SHOW BACKUP CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '1KiB', TIME = '1h') -- normalized! +SHOW BACKUP CONNECTION ('*****') WITH OPTIONS (CONCURRENTLY = (3), TRANSFER = ('1KiB'), TIME = ('1h')) -- fully parenthesized SHOW BACKUP CONNECTION '_' WITH OPTIONS (CONCURRENTLY = _, TRANSFER = '_', TIME = '_') -- literals removed -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '1KiB', TIME = '1h') -- identifiers removed +SHOW BACKUP CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '1KiB', TIME = '1h') -- identifiers removed +SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (CONCURRENTLY = 3, TRANSFER = '1KiB', TIME = '1h') -- passwords exposed parse SHOW BACKUP CONNECTION 'bar' WITH TRANSFER = $1, CONCURRENTLY = $2, TIME = $3 ---- -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- normalized! -SHOW BACKUP CONNECTION ('bar') WITH OPTIONS (CONCURRENTLY = ($2), TRANSFER = ($1), TIME = ($3)) -- fully parenthesized +SHOW BACKUP CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- normalized! +SHOW BACKUP CONNECTION ('*****') WITH OPTIONS (CONCURRENTLY = ($2), TRANSFER = ($1), TIME = ($3)) -- fully parenthesized SHOW BACKUP CONNECTION '_' WITH OPTIONS (CONCURRENTLY = $1, TRANSFER = $1, TIME = $1) -- literals removed -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- identifiers removed +SHOW BACKUP CONNECTION '*****' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- identifiers removed +SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (CONCURRENTLY = $2, TRANSFER = $1, TIME = $3) -- passwords exposed parse SHOW BACKUPS IN 'bar' ---- -SHOW BACKUPS IN 'bar' -SHOW BACKUPS IN ('bar') -- fully parenthesized +SHOW BACKUPS IN '*****' -- normalized! +SHOW BACKUPS IN ('*****') -- fully parenthesized SHOW BACKUPS IN '_' -- literals removed -SHOW BACKUPS IN 'bar' -- identifiers removed +SHOW BACKUPS IN '*****' -- identifiers removed +SHOW BACKUPS IN 'bar' -- passwords exposed parse SHOW BACKUPS IN $1 @@ -203,10 +225,11 @@ SHOW BACKUPS IN $1 -- identifiers removed parse SHOW BACKUP 'foo' IN 'bar' ---- -SHOW BACKUP 'foo' IN 'bar' -SHOW BACKUP ('foo') IN ('bar') -- fully parenthesized +SHOW BACKUP 'foo' IN '*****' -- normalized! +SHOW BACKUP ('foo') IN ('*****') -- fully parenthesized SHOW BACKUP '_' IN '_' -- literals removed -SHOW BACKUP 'foo' IN 'bar' -- identifiers removed +SHOW BACKUP 'foo' IN '*****' -- identifiers removed +SHOW BACKUP 'foo' IN 'bar' -- passwords exposed parse SHOW BACKUP FROM $1 IN $2 WITH privileges @@ -219,26 +242,29 @@ SHOW BACKUP FROM $1 IN $2 WITH OPTIONS (privileges) -- identifiers removed parse SHOW BACKUP FILES FROM 'foo' IN 'bar' ---- -SHOW BACKUP FILES FROM 'foo' IN 'bar' -SHOW BACKUP FILES FROM ('foo') IN ('bar') -- fully parenthesized +SHOW BACKUP FILES FROM 'foo' IN '*****' -- normalized! +SHOW BACKUP FILES FROM ('foo') IN ('*****') -- fully parenthesized SHOW BACKUP FILES FROM '_' IN '_' -- literals removed -SHOW BACKUP FILES FROM 'foo' IN 'bar' -- identifiers removed +SHOW BACKUP FILES FROM 'foo' IN '*****' -- identifiers removed +SHOW BACKUP FILES FROM 'foo' IN 'bar' -- passwords exposed parse SHOW BACKUP RANGES FROM 'foo' IN 'bar' ---- -SHOW BACKUP RANGES FROM 'foo' IN 'bar' -SHOW BACKUP RANGES FROM ('foo') IN ('bar') -- fully parenthesized +SHOW BACKUP RANGES FROM 'foo' IN '*****' -- normalized! +SHOW BACKUP RANGES FROM ('foo') IN ('*****') -- fully parenthesized SHOW BACKUP RANGES FROM '_' IN '_' -- literals removed -SHOW BACKUP RANGES FROM 'foo' IN 'bar' -- identifiers removed +SHOW BACKUP RANGES FROM 'foo' IN '*****' -- identifiers removed +SHOW BACKUP RANGES FROM 'foo' IN 'bar' -- passwords exposed parse SHOW BACKUP SCHEMAS FROM 'foo' IN 'bar' ---- -SHOW BACKUP SCHEMAS FROM 'foo' IN 'bar' -SHOW BACKUP SCHEMAS FROM ('foo') IN ('bar') -- fully parenthesized +SHOW BACKUP SCHEMAS FROM 'foo' IN '*****' -- normalized! +SHOW BACKUP SCHEMAS FROM ('foo') IN ('*****') -- fully parenthesized SHOW BACKUP SCHEMAS FROM '_' IN '_' -- literals removed -SHOW BACKUP SCHEMAS FROM 'foo' IN 'bar' -- identifiers removed +SHOW BACKUP SCHEMAS FROM 'foo' IN '*****' -- identifiers removed +SHOW BACKUP SCHEMAS FROM 'foo' IN 'bar' -- passwords exposed parse SHOW BACKUP $1 IN $2 WITH ENCRYPTION_PASSPHRASE = 'secret', ENCRYPTION_INFO_DIR = 'long_live_backupper' @@ -252,42 +278,47 @@ SHOW BACKUP $1 IN $2 WITH OPTIONS (encryption_passphrase = 'secret', encryption_ parse BACKUP TABLE foo TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' ---- -BACKUP TABLE foo TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' -BACKUP TABLE (foo) TO ('bar') AS OF SYSTEM TIME ('1') INCREMENTAL FROM ('baz') -- fully parenthesized +BACKUP TABLE foo TO '*****' AS OF SYSTEM TIME '1' INCREMENTAL FROM '*****' -- normalized! +BACKUP TABLE (foo) TO ('*****') AS OF SYSTEM TIME ('1') INCREMENTAL FROM ('*****') -- fully parenthesized BACKUP TABLE foo TO '_' AS OF SYSTEM TIME '_' INCREMENTAL FROM '_' -- literals removed -BACKUP TABLE _ TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' -- identifiers removed +BACKUP TABLE _ TO '*****' AS OF SYSTEM TIME '1' INCREMENTAL FROM '*****' -- identifiers removed +BACKUP TABLE foo TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' -- passwords exposed parse BACKUP foo TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' ---- -BACKUP TABLE foo TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' -- normalized! -BACKUP TABLE (foo) TO ('bar') AS OF SYSTEM TIME ('1') INCREMENTAL FROM ('baz') -- fully parenthesized +BACKUP TABLE foo TO '*****' AS OF SYSTEM TIME '1' INCREMENTAL FROM '*****' -- normalized! +BACKUP TABLE (foo) TO ('*****') AS OF SYSTEM TIME ('1') INCREMENTAL FROM ('*****') -- fully parenthesized BACKUP TABLE foo TO '_' AS OF SYSTEM TIME '_' INCREMENTAL FROM '_' -- literals removed -BACKUP TABLE _ TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' -- identifiers removed +BACKUP TABLE _ TO '*****' AS OF SYSTEM TIME '1' INCREMENTAL FROM '*****' -- identifiers removed +BACKUP TABLE foo TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' -- passwords exposed parse BACKUP TABLE foo TO $1 INCREMENTAL FROM 'bar', $2, 'baz' ---- -BACKUP TABLE foo TO $1 INCREMENTAL FROM 'bar', $2, 'baz' -BACKUP TABLE (foo) TO ($1) INCREMENTAL FROM ('bar'), ($2), ('baz') -- fully parenthesized +BACKUP TABLE foo TO $1 INCREMENTAL FROM '*****', $2, '*****' -- normalized! +BACKUP TABLE (foo) TO ($1) INCREMENTAL FROM ('*****'), ($2), ('*****') -- fully parenthesized BACKUP TABLE foo TO $1 INCREMENTAL FROM '_', $1, '_' -- literals removed -BACKUP TABLE _ TO $1 INCREMENTAL FROM 'bar', $2, 'baz' -- identifiers removed +BACKUP TABLE _ TO $1 INCREMENTAL FROM '*****', $2, '*****' -- identifiers removed +BACKUP TABLE foo TO $1 INCREMENTAL FROM 'bar', $2, 'baz' -- passwords exposed parse BACKUP foo TO $1 INCREMENTAL FROM 'bar', $2, 'baz' ---- -BACKUP TABLE foo TO $1 INCREMENTAL FROM 'bar', $2, 'baz' -- normalized! -BACKUP TABLE (foo) TO ($1) INCREMENTAL FROM ('bar'), ($2), ('baz') -- fully parenthesized +BACKUP TABLE foo TO $1 INCREMENTAL FROM '*****', $2, '*****' -- normalized! +BACKUP TABLE (foo) TO ($1) INCREMENTAL FROM ('*****'), ($2), ('*****') -- fully parenthesized BACKUP TABLE foo TO $1 INCREMENTAL FROM '_', $1, '_' -- literals removed -BACKUP TABLE _ TO $1 INCREMENTAL FROM 'bar', $2, 'baz' -- identifiers removed +BACKUP TABLE _ TO $1 INCREMENTAL FROM '*****', $2, '*****' -- identifiers removed +BACKUP TABLE foo TO $1 INCREMENTAL FROM 'bar', $2, 'baz' -- passwords exposed parse BACKUP DATABASE foo TO 'bar' ---- -BACKUP DATABASE foo TO 'bar' -BACKUP DATABASE foo TO ('bar') -- fully parenthesized +BACKUP DATABASE foo TO '*****' -- normalized! +BACKUP DATABASE foo TO ('*****') -- fully parenthesized BACKUP DATABASE foo TO '_' -- literals removed -BACKUP DATABASE _ TO 'bar' -- identifiers removed +BACKUP DATABASE _ TO '*****' -- identifiers removed +BACKUP DATABASE foo TO 'bar' -- passwords exposed parse BACKUP DATABASE foo TO ($1) @@ -300,44 +331,49 @@ BACKUP DATABASE _ TO $1 -- identifiers removed parse EXPLAIN BACKUP DATABASE foo TO 'bar' ---- -EXPLAIN BACKUP DATABASE foo TO 'bar' -EXPLAIN BACKUP DATABASE foo TO ('bar') -- fully parenthesized +EXPLAIN BACKUP DATABASE foo TO '*****' -- normalized! +EXPLAIN BACKUP DATABASE foo TO ('*****') -- fully parenthesized EXPLAIN BACKUP DATABASE foo TO '_' -- literals removed -EXPLAIN BACKUP DATABASE _ TO 'bar' -- identifiers removed +EXPLAIN BACKUP DATABASE _ TO '*****' -- identifiers removed +EXPLAIN BACKUP DATABASE foo TO 'bar' -- passwords exposed parse BACKUP DATABASE foo TO bar ---- -BACKUP DATABASE foo TO 'bar' -- normalized! -BACKUP DATABASE foo TO ('bar') -- fully parenthesized +BACKUP DATABASE foo TO '*****' -- normalized! +BACKUP DATABASE foo TO ('*****') -- fully parenthesized BACKUP DATABASE foo TO '_' -- literals removed -BACKUP DATABASE _ TO 'bar' -- identifiers removed +BACKUP DATABASE _ TO '*****' -- identifiers removed +BACKUP DATABASE foo TO 'bar' -- passwords exposed parse BACKUP DATABASE foo, baz TO 'bar' ---- -BACKUP DATABASE foo, baz TO 'bar' -BACKUP DATABASE foo, baz TO ('bar') -- fully parenthesized +BACKUP DATABASE foo, baz TO '*****' -- normalized! +BACKUP DATABASE foo, baz TO ('*****') -- fully parenthesized BACKUP DATABASE foo, baz TO '_' -- literals removed -BACKUP DATABASE _, _ TO 'bar' -- identifiers removed +BACKUP DATABASE _, _ TO '*****' -- identifiers removed +BACKUP DATABASE foo, baz TO 'bar' -- passwords exposed parse BACKUP DATABASE foo TO "bar.12" INCREMENTAL FROM "baz.34" ---- -BACKUP DATABASE foo TO 'bar.12' INCREMENTAL FROM 'baz.34' -- normalized! -BACKUP DATABASE foo TO ('bar.12') INCREMENTAL FROM ('baz.34') -- fully parenthesized +BACKUP DATABASE foo TO '*****' INCREMENTAL FROM '*****' -- normalized! +BACKUP DATABASE foo TO ('*****') INCREMENTAL FROM ('*****') -- fully parenthesized BACKUP DATABASE foo TO '_' INCREMENTAL FROM '_' -- literals removed -BACKUP DATABASE _ TO 'bar.12' INCREMENTAL FROM 'baz.34' -- identifiers removed +BACKUP DATABASE _ TO '*****' INCREMENTAL FROM '*****' -- identifiers removed +BACKUP DATABASE foo TO 'bar.12' INCREMENTAL FROM 'baz.34' -- passwords exposed parse BACKUP DATABASE foo TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' ---- -BACKUP DATABASE foo TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' -BACKUP DATABASE foo TO ('bar') AS OF SYSTEM TIME ('1') INCREMENTAL FROM ('baz') -- fully parenthesized +BACKUP DATABASE foo TO '*****' AS OF SYSTEM TIME '1' INCREMENTAL FROM '*****' -- normalized! +BACKUP DATABASE foo TO ('*****') AS OF SYSTEM TIME ('1') INCREMENTAL FROM ('*****') -- fully parenthesized BACKUP DATABASE foo TO '_' AS OF SYSTEM TIME '_' INCREMENTAL FROM '_' -- literals removed -BACKUP DATABASE _ TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' -- identifiers removed +BACKUP DATABASE _ TO '*****' AS OF SYSTEM TIME '1' INCREMENTAL FROM '*****' -- identifiers removed +BACKUP DATABASE foo TO 'bar' AS OF SYSTEM TIME '1' INCREMENTAL FROM 'baz' -- passwords exposed parse BACKUP DATABASE foo TO ($1, $2) @@ -350,94 +386,103 @@ BACKUP DATABASE _ TO ($1, $2) -- identifiers removed parse BACKUP DATABASE foo TO ($1, $2) INCREMENTAL FROM 'baz' ---- -BACKUP DATABASE foo TO ($1, $2) INCREMENTAL FROM 'baz' -BACKUP DATABASE foo TO (($1), ($2)) INCREMENTAL FROM ('baz') -- fully parenthesized +BACKUP DATABASE foo TO ($1, $2) INCREMENTAL FROM '*****' -- normalized! +BACKUP DATABASE foo TO (($1), ($2)) INCREMENTAL FROM ('*****') -- fully parenthesized BACKUP DATABASE foo TO ($1, $1) INCREMENTAL FROM '_' -- literals removed -BACKUP DATABASE _ TO ($1, $2) INCREMENTAL FROM 'baz' -- identifiers removed +BACKUP DATABASE _ TO ($1, $2) INCREMENTAL FROM '*****' -- identifiers removed +BACKUP DATABASE foo TO ($1, $2) INCREMENTAL FROM 'baz' -- passwords exposed parse BACKUP foo TO 'bar' WITH ENCRYPTION_PASSPHRASE = 'secret', revision_history, execution locality = 'a=b' ---- -BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, encryption_passphrase = '*****', execution locality = 'a=b') -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH OPTIONS (revision_history = (true), encryption_passphrase = '*****', execution locality = ('a=b')) -- fully parenthesized +BACKUP TABLE foo TO '*****' WITH OPTIONS (revision_history = true, encryption_passphrase = '*****', execution locality = 'a=b') -- normalized! +BACKUP TABLE (foo) TO ('*****') WITH OPTIONS (revision_history = (true), encryption_passphrase = '*****', execution locality = ('a=b')) -- fully parenthesized BACKUP TABLE foo TO '_' WITH OPTIONS (revision_history = _, encryption_passphrase = '*****', execution locality = '_') -- literals removed -BACKUP TABLE _ TO 'bar' WITH OPTIONS (revision_history = true, encryption_passphrase = '*****', execution locality = 'a=b') -- identifiers removed +BACKUP TABLE _ TO '*****' WITH OPTIONS (revision_history = true, encryption_passphrase = '*****', execution locality = 'a=b') -- identifiers removed BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, encryption_passphrase = 'secret', execution locality = 'a=b') -- passwords exposed parse BACKUP foo TO 'bar' WITH KMS = ('foo', 'bar'), revision_history ---- -BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, kms = ('foo', 'bar')) -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH OPTIONS (revision_history = (true), kms = (('foo'), ('bar'))) -- fully parenthesized +BACKUP TABLE foo TO '*****' WITH OPTIONS (revision_history = true, kms = ('*****', '*****')) -- normalized! +BACKUP TABLE (foo) TO ('*****') WITH OPTIONS (revision_history = (true), kms = (('*****'), ('*****'))) -- fully parenthesized BACKUP TABLE foo TO '_' WITH OPTIONS (revision_history = _, kms = ('_', '_')) -- literals removed -BACKUP TABLE _ TO 'bar' WITH OPTIONS (revision_history = true, kms = ('foo', 'bar')) -- identifiers removed +BACKUP TABLE _ TO '*****' WITH OPTIONS (revision_history = true, kms = ('*****', '*****')) -- identifiers removed +BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, kms = ('foo', 'bar')) -- passwords exposed parse BACKUP foo TO 'bar' WITH OPTIONS (detached, ENCRYPTION_PASSPHRASE = 'secret', revision_history) ---- -BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, encryption_passphrase = '*****', detached) -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH OPTIONS (revision_history = (true), encryption_passphrase = '*****', detached) -- fully parenthesized +BACKUP TABLE foo TO '*****' WITH OPTIONS (revision_history = true, encryption_passphrase = '*****', detached) -- normalized! +BACKUP TABLE (foo) TO ('*****') WITH OPTIONS (revision_history = (true), encryption_passphrase = '*****', detached) -- fully parenthesized BACKUP TABLE foo TO '_' WITH OPTIONS (revision_history = _, encryption_passphrase = '*****', detached) -- literals removed -BACKUP TABLE _ TO 'bar' WITH OPTIONS (revision_history = true, encryption_passphrase = '*****', detached) -- identifiers removed +BACKUP TABLE _ TO '*****' WITH OPTIONS (revision_history = true, encryption_passphrase = '*****', detached) -- identifiers removed BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, encryption_passphrase = 'secret', detached) -- passwords exposed parse BACKUP foo TO 'bar' WITH OPTIONS (detached, KMS = ('foo', 'bar'), revision_history) ---- -BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, detached, kms = ('foo', 'bar')) -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH OPTIONS (revision_history = (true), detached, kms = (('foo'), ('bar'))) -- fully parenthesized +BACKUP TABLE foo TO '*****' WITH OPTIONS (revision_history = true, detached, kms = ('*****', '*****')) -- normalized! +BACKUP TABLE (foo) TO ('*****') WITH OPTIONS (revision_history = (true), detached, kms = (('*****'), ('*****'))) -- fully parenthesized BACKUP TABLE foo TO '_' WITH OPTIONS (revision_history = _, detached, kms = ('_', '_')) -- literals removed -BACKUP TABLE _ TO 'bar' WITH OPTIONS (revision_history = true, detached, kms = ('foo', 'bar')) -- identifiers removed +BACKUP TABLE _ TO '*****' WITH OPTIONS (revision_history = true, detached, kms = ('*****', '*****')) -- identifiers removed +BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, detached, kms = ('foo', 'bar')) -- passwords exposed # Regression test for #95235. parse BACKUP foo TO 'bar' WITH OPTIONS (detached = false) ---- -BACKUP TABLE foo TO 'bar' -- normalized! -BACKUP TABLE (foo) TO ('bar') -- fully parenthesized +BACKUP TABLE foo TO '*****' -- normalized! +BACKUP TABLE (foo) TO ('*****') -- fully parenthesized BACKUP TABLE foo TO '_' -- literals removed -BACKUP TABLE _ TO 'bar' -- identifiers removed +BACKUP TABLE _ TO '*****' -- identifiers removed +BACKUP TABLE foo TO 'bar' -- passwords exposed parse BACKUP VIRTUAL CLUSTER 36 TO 'bar' ---- -BACKUP VIRTUAL CLUSTER 36 TO 'bar' -BACKUP VIRTUAL CLUSTER 36 TO ('bar') -- fully parenthesized +BACKUP VIRTUAL CLUSTER 36 TO '*****' -- normalized! +BACKUP VIRTUAL CLUSTER 36 TO ('*****') -- fully parenthesized BACKUP VIRTUAL CLUSTER _ TO '_' -- literals removed -BACKUP VIRTUAL CLUSTER 36 TO 'bar' -- identifiers removed +BACKUP VIRTUAL CLUSTER 36 TO '*****' -- identifiers removed +BACKUP VIRTUAL CLUSTER 36 TO 'bar' -- passwords exposed parse BACKUP TENANT 36 TO 'bar' ---- -BACKUP VIRTUAL CLUSTER 36 TO 'bar' -- normalized! -BACKUP VIRTUAL CLUSTER 36 TO ('bar') -- fully parenthesized +BACKUP VIRTUAL CLUSTER 36 TO '*****' -- normalized! +BACKUP VIRTUAL CLUSTER 36 TO ('*****') -- fully parenthesized BACKUP VIRTUAL CLUSTER _ TO '_' -- literals removed -BACKUP VIRTUAL CLUSTER 36 TO 'bar' -- identifiers removed +BACKUP VIRTUAL CLUSTER 36 TO '*****' -- identifiers removed +BACKUP VIRTUAL CLUSTER 36 TO 'bar' -- passwords exposed parse RESTORE TABLE foo FROM 'bar' ---- -RESTORE TABLE foo FROM 'bar' -RESTORE TABLE (foo) FROM ('bar') -- fully parenthesized +RESTORE TABLE foo FROM '*****' -- normalized! +RESTORE TABLE (foo) FROM ('*****') -- fully parenthesized RESTORE TABLE foo FROM '_' -- literals removed -RESTORE TABLE _ FROM 'bar' -- identifiers removed +RESTORE TABLE _ FROM '*****' -- identifiers removed +RESTORE TABLE foo FROM 'bar' -- passwords exposed parse EXPLAIN RESTORE TABLE foo FROM 'bar' ---- -EXPLAIN RESTORE TABLE foo FROM 'bar' -EXPLAIN RESTORE TABLE (foo) FROM ('bar') -- fully parenthesized +EXPLAIN RESTORE TABLE foo FROM '*****' -- normalized! +EXPLAIN RESTORE TABLE (foo) FROM ('*****') -- fully parenthesized EXPLAIN RESTORE TABLE foo FROM '_' -- literals removed -EXPLAIN RESTORE TABLE _ FROM 'bar' -- identifiers removed +EXPLAIN RESTORE TABLE _ FROM '*****' -- identifiers removed +EXPLAIN RESTORE TABLE foo FROM 'bar' -- passwords exposed parse RESTORE foo FROM 'bar' ---- -RESTORE TABLE foo FROM 'bar' -- normalized! -RESTORE TABLE (foo) FROM ('bar') -- fully parenthesized +RESTORE TABLE foo FROM '*****' -- normalized! +RESTORE TABLE (foo) FROM ('*****') -- fully parenthesized RESTORE TABLE foo FROM '_' -- literals removed -RESTORE TABLE _ FROM 'bar' -- identifiers removed +RESTORE TABLE _ FROM '*****' -- identifiers removed +RESTORE TABLE foo FROM 'bar' -- passwords exposed parse RESTORE TABLE foo FROM $1 @@ -467,76 +512,85 @@ RESTORE TABLE _ FROM $2 IN $1 -- identifiers removed parse RESTORE TABLE foo FROM $1, $2, 'bar' ---- -RESTORE TABLE foo FROM $1, $2, 'bar' -RESTORE TABLE (foo) FROM ($1), ($2), ('bar') -- fully parenthesized +RESTORE TABLE foo FROM $1, $2, '*****' -- normalized! +RESTORE TABLE (foo) FROM ($1), ($2), ('*****') -- fully parenthesized RESTORE TABLE foo FROM $1, $1, '_' -- literals removed -RESTORE TABLE _ FROM $1, $2, 'bar' -- identifiers removed +RESTORE TABLE _ FROM $1, $2, '*****' -- identifiers removed +RESTORE TABLE foo FROM $1, $2, 'bar' -- passwords exposed parse RESTORE foo FROM $1, $2, 'bar' ---- -RESTORE TABLE foo FROM $1, $2, 'bar' -- normalized! -RESTORE TABLE (foo) FROM ($1), ($2), ('bar') -- fully parenthesized +RESTORE TABLE foo FROM $1, $2, '*****' -- normalized! +RESTORE TABLE (foo) FROM ($1), ($2), ('*****') -- fully parenthesized RESTORE TABLE foo FROM $1, $1, '_' -- literals removed -RESTORE TABLE _ FROM $1, $2, 'bar' -- identifiers removed +RESTORE TABLE _ FROM $1, $2, '*****' -- identifiers removed +RESTORE TABLE foo FROM $1, $2, 'bar' -- passwords exposed parse RESTORE TABLE foo FROM 'abc' IN $1, $2, 'bar' ---- -RESTORE TABLE foo FROM 'abc' IN $1, $2, 'bar' -RESTORE TABLE (foo) FROM ('abc') IN ($1), ($2), ('bar') -- fully parenthesized +RESTORE TABLE foo FROM 'abc' IN $1, $2, '*****' -- normalized! +RESTORE TABLE (foo) FROM ('abc') IN ($1), ($2), ('*****') -- fully parenthesized RESTORE TABLE foo FROM '_' IN $1, $1, '_' -- literals removed -RESTORE TABLE _ FROM 'abc' IN $1, $2, 'bar' -- identifiers removed +RESTORE TABLE _ FROM 'abc' IN $1, $2, '*****' -- identifiers removed +RESTORE TABLE foo FROM 'abc' IN $1, $2, 'bar' -- passwords exposed parse RESTORE TABLE foo FROM $4 IN $1, $2, 'bar' ---- -RESTORE TABLE foo FROM $4 IN $1, $2, 'bar' -RESTORE TABLE (foo) FROM ($4) IN ($1), ($2), ('bar') -- fully parenthesized +RESTORE TABLE foo FROM $4 IN $1, $2, '*****' -- normalized! +RESTORE TABLE (foo) FROM ($4) IN ($1), ($2), ('*****') -- fully parenthesized RESTORE TABLE foo FROM $1 IN $1, $1, '_' -- literals removed -RESTORE TABLE _ FROM $4 IN $1, $2, 'bar' -- identifiers removed +RESTORE TABLE _ FROM $4 IN $1, $2, '*****' -- identifiers removed +RESTORE TABLE foo FROM $4 IN $1, $2, 'bar' -- passwords exposed parse RESTORE TABLE foo, baz FROM 'bar' ---- -RESTORE TABLE foo, baz FROM 'bar' -RESTORE TABLE (foo), (baz) FROM ('bar') -- fully parenthesized +RESTORE TABLE foo, baz FROM '*****' -- normalized! +RESTORE TABLE (foo), (baz) FROM ('*****') -- fully parenthesized RESTORE TABLE foo, baz FROM '_' -- literals removed -RESTORE TABLE _, _ FROM 'bar' -- identifiers removed +RESTORE TABLE _, _ FROM '*****' -- identifiers removed +RESTORE TABLE foo, baz FROM 'bar' -- passwords exposed parse RESTORE foo, baz FROM 'bar' ---- -RESTORE TABLE foo, baz FROM 'bar' -- normalized! -RESTORE TABLE (foo), (baz) FROM ('bar') -- fully parenthesized +RESTORE TABLE foo, baz FROM '*****' -- normalized! +RESTORE TABLE (foo), (baz) FROM ('*****') -- fully parenthesized RESTORE TABLE foo, baz FROM '_' -- literals removed -RESTORE TABLE _, _ FROM 'bar' -- identifiers removed +RESTORE TABLE _, _ FROM '*****' -- identifiers removed +RESTORE TABLE foo, baz FROM 'bar' -- passwords exposed parse RESTORE TABLE foo, baz FROM 'bar' AS OF SYSTEM TIME '1' ---- -RESTORE TABLE foo, baz FROM 'bar' AS OF SYSTEM TIME '1' -RESTORE TABLE (foo), (baz) FROM ('bar') AS OF SYSTEM TIME ('1') -- fully parenthesized +RESTORE TABLE foo, baz FROM '*****' AS OF SYSTEM TIME '1' -- normalized! +RESTORE TABLE (foo), (baz) FROM ('*****') AS OF SYSTEM TIME ('1') -- fully parenthesized RESTORE TABLE foo, baz FROM '_' AS OF SYSTEM TIME '_' -- literals removed -RESTORE TABLE _, _ FROM 'bar' AS OF SYSTEM TIME '1' -- identifiers removed +RESTORE TABLE _, _ FROM '*****' AS OF SYSTEM TIME '1' -- identifiers removed +RESTORE TABLE foo, baz FROM 'bar' AS OF SYSTEM TIME '1' -- passwords exposed parse RESTORE foo, baz FROM 'bar' AS OF SYSTEM TIME '1' ---- -RESTORE TABLE foo, baz FROM 'bar' AS OF SYSTEM TIME '1' -- normalized! -RESTORE TABLE (foo), (baz) FROM ('bar') AS OF SYSTEM TIME ('1') -- fully parenthesized +RESTORE TABLE foo, baz FROM '*****' AS OF SYSTEM TIME '1' -- normalized! +RESTORE TABLE (foo), (baz) FROM ('*****') AS OF SYSTEM TIME ('1') -- fully parenthesized RESTORE TABLE foo, baz FROM '_' AS OF SYSTEM TIME '_' -- literals removed -RESTORE TABLE _, _ FROM 'bar' AS OF SYSTEM TIME '1' -- identifiers removed +RESTORE TABLE _, _ FROM '*****' AS OF SYSTEM TIME '1' -- identifiers removed +RESTORE TABLE foo, baz FROM 'bar' AS OF SYSTEM TIME '1' -- passwords exposed parse RESTORE DATABASE foo FROM 'bar' ---- -RESTORE DATABASE foo FROM 'bar' -RESTORE DATABASE foo FROM ('bar') -- fully parenthesized +RESTORE DATABASE foo FROM '*****' -- normalized! +RESTORE DATABASE foo FROM ('*****') -- fully parenthesized RESTORE DATABASE foo FROM '_' -- literals removed -RESTORE DATABASE _ FROM 'bar' -- identifiers removed +RESTORE DATABASE _ FROM '*****' -- identifiers removed +RESTORE DATABASE foo FROM 'bar' -- passwords exposed parse RESTORE DATABASE foo FROM ($1) @@ -549,59 +603,66 @@ RESTORE DATABASE _ FROM $1 -- identifiers removed parse EXPLAIN RESTORE DATABASE foo FROM 'bar' ---- -EXPLAIN RESTORE DATABASE foo FROM 'bar' -EXPLAIN RESTORE DATABASE foo FROM ('bar') -- fully parenthesized +EXPLAIN RESTORE DATABASE foo FROM '*****' -- normalized! +EXPLAIN RESTORE DATABASE foo FROM ('*****') -- fully parenthesized EXPLAIN RESTORE DATABASE foo FROM '_' -- literals removed -EXPLAIN RESTORE DATABASE _ FROM 'bar' -- identifiers removed +EXPLAIN RESTORE DATABASE _ FROM '*****' -- identifiers removed +EXPLAIN RESTORE DATABASE foo FROM 'bar' -- passwords exposed parse RESTORE DATABASE foo FROM bar ---- -RESTORE DATABASE foo FROM 'bar' -- normalized! -RESTORE DATABASE foo FROM ('bar') -- fully parenthesized +RESTORE DATABASE foo FROM '*****' -- normalized! +RESTORE DATABASE foo FROM ('*****') -- fully parenthesized RESTORE DATABASE foo FROM '_' -- literals removed -RESTORE DATABASE _ FROM 'bar' -- identifiers removed +RESTORE DATABASE _ FROM '*****' -- identifiers removed +RESTORE DATABASE foo FROM 'bar' -- passwords exposed parse RESTORE DATABASE foo, baz FROM 'bar' ---- -RESTORE DATABASE foo, baz FROM 'bar' -RESTORE DATABASE foo, baz FROM ('bar') -- fully parenthesized +RESTORE DATABASE foo, baz FROM '*****' -- normalized! +RESTORE DATABASE foo, baz FROM ('*****') -- fully parenthesized RESTORE DATABASE foo, baz FROM '_' -- literals removed -RESTORE DATABASE _, _ FROM 'bar' -- identifiers removed +RESTORE DATABASE _, _ FROM '*****' -- identifiers removed +RESTORE DATABASE foo, baz FROM 'bar' -- passwords exposed parse RESTORE DATABASE foo FROM 'bar' WITH new_db_name = 'baz' ---- -RESTORE DATABASE foo FROM 'bar' WITH OPTIONS (new_db_name = 'baz') -- normalized! -RESTORE DATABASE foo FROM ('bar') WITH OPTIONS (new_db_name = ('baz')) -- fully parenthesized +RESTORE DATABASE foo FROM '*****' WITH OPTIONS (new_db_name = 'baz') -- normalized! +RESTORE DATABASE foo FROM ('*****') WITH OPTIONS (new_db_name = ('baz')) -- fully parenthesized RESTORE DATABASE foo FROM '_' WITH OPTIONS (new_db_name = '_') -- literals removed -RESTORE DATABASE _ FROM 'bar' WITH OPTIONS (new_db_name = 'baz') -- identifiers removed +RESTORE DATABASE _ FROM '*****' WITH OPTIONS (new_db_name = 'baz') -- identifiers removed +RESTORE DATABASE foo FROM 'bar' WITH OPTIONS (new_db_name = 'baz') -- passwords exposed parse RESTORE DATABASE foo FROM 'bar' WITH schema_only ---- -RESTORE DATABASE foo FROM 'bar' WITH OPTIONS (schema_only) -- normalized! -RESTORE DATABASE foo FROM ('bar') WITH OPTIONS (schema_only) -- fully parenthesized +RESTORE DATABASE foo FROM '*****' WITH OPTIONS (schema_only) -- normalized! +RESTORE DATABASE foo FROM ('*****') WITH OPTIONS (schema_only) -- fully parenthesized RESTORE DATABASE foo FROM '_' WITH OPTIONS (schema_only) -- literals removed -RESTORE DATABASE _ FROM 'bar' WITH OPTIONS (schema_only) -- identifiers removed +RESTORE DATABASE _ FROM '*****' WITH OPTIONS (schema_only) -- identifiers removed +RESTORE DATABASE foo FROM 'bar' WITH OPTIONS (schema_only) -- passwords exposed parse RESTORE DATABASE foo FROM 'bar' IN LATEST WITH incremental_location = 'baz' ---- -RESTORE DATABASE foo FROM 'bar' IN 'latest' WITH OPTIONS (incremental_location = 'baz') -- normalized! -RESTORE DATABASE foo FROM ('bar') IN ('latest') WITH OPTIONS (incremental_location = ('baz')) -- fully parenthesized +RESTORE DATABASE foo FROM 'bar' IN '*****' WITH OPTIONS (incremental_location = '*****') -- normalized! +RESTORE DATABASE foo FROM ('bar') IN ('*****') WITH OPTIONS (incremental_location = ('*****')) -- fully parenthesized RESTORE DATABASE foo FROM '_' IN '_' WITH OPTIONS (incremental_location = '_') -- literals removed -RESTORE DATABASE _ FROM 'bar' IN 'latest' WITH OPTIONS (incremental_location = 'baz') -- identifiers removed +RESTORE DATABASE _ FROM 'bar' IN '*****' WITH OPTIONS (incremental_location = '*****') -- identifiers removed +RESTORE DATABASE foo FROM 'bar' IN 'latest' WITH OPTIONS (incremental_location = 'baz') -- passwords exposed parse RESTORE DATABASE foo, baz FROM 'bar' AS OF SYSTEM TIME '1' ---- -RESTORE DATABASE foo, baz FROM 'bar' AS OF SYSTEM TIME '1' -RESTORE DATABASE foo, baz FROM ('bar') AS OF SYSTEM TIME ('1') -- fully parenthesized +RESTORE DATABASE foo, baz FROM '*****' AS OF SYSTEM TIME '1' -- normalized! +RESTORE DATABASE foo, baz FROM ('*****') AS OF SYSTEM TIME ('1') -- fully parenthesized RESTORE DATABASE foo, baz FROM '_' AS OF SYSTEM TIME '_' -- literals removed -RESTORE DATABASE _, _ FROM 'bar' AS OF SYSTEM TIME '1' -- identifiers removed +RESTORE DATABASE _, _ FROM '*****' AS OF SYSTEM TIME '1' -- identifiers removed +RESTORE DATABASE foo, baz FROM 'bar' AS OF SYSTEM TIME '1' -- passwords exposed parse RESTORE DATABASE foo FROM ($1, $2) @@ -702,61 +763,66 @@ RESTORE FROM ($1, $2), ($3, $4) AS OF SYSTEM TIME '1' -- identifiers removed parse RESTORE FROM $1, $2, 'bar' ---- -RESTORE FROM $1, $2, 'bar' -RESTORE FROM ($1), ($2), ('bar') -- fully parenthesized +RESTORE FROM $1, $2, '*****' -- normalized! +RESTORE FROM ($1), ($2), ('*****') -- fully parenthesized RESTORE FROM $1, $1, '_' -- literals removed -RESTORE FROM $1, $2, 'bar' -- identifiers removed +RESTORE FROM $1, $2, '*****' -- identifiers removed +RESTORE FROM $1, $2, 'bar' -- passwords exposed parse RESTORE FROM $4 IN $1, $2, 'bar' ---- -RESTORE FROM $4 IN $1, $2, 'bar' -RESTORE FROM ($4) IN ($1), ($2), ('bar') -- fully parenthesized +RESTORE FROM $4 IN $1, $2, '*****' -- normalized! +RESTORE FROM ($4) IN ($1), ($2), ('*****') -- fully parenthesized RESTORE FROM $1 IN $1, $1, '_' -- literals removed -RESTORE FROM $4 IN $1, $2, 'bar' -- identifiers removed +RESTORE FROM $4 IN $1, $2, '*****' -- identifiers removed +RESTORE FROM $4 IN $1, $2, 'bar' -- passwords exposed parse RESTORE FROM $4 IN $1, $2, 'bar' AS OF SYSTEM TIME '1' WITH skip_missing_foreign_keys ---- -RESTORE FROM $4 IN $1, $2, 'bar' AS OF SYSTEM TIME '1' WITH OPTIONS (skip_missing_foreign_keys) -- normalized! -RESTORE FROM ($4) IN ($1), ($2), ('bar') AS OF SYSTEM TIME ('1') WITH OPTIONS (skip_missing_foreign_keys) -- fully parenthesized +RESTORE FROM $4 IN $1, $2, '*****' AS OF SYSTEM TIME '1' WITH OPTIONS (skip_missing_foreign_keys) -- normalized! +RESTORE FROM ($4) IN ($1), ($2), ('*****') AS OF SYSTEM TIME ('1') WITH OPTIONS (skip_missing_foreign_keys) -- fully parenthesized RESTORE FROM $1 IN $1, $1, '_' AS OF SYSTEM TIME '_' WITH OPTIONS (skip_missing_foreign_keys) -- literals removed -RESTORE FROM $4 IN $1, $2, 'bar' AS OF SYSTEM TIME '1' WITH OPTIONS (skip_missing_foreign_keys) -- identifiers removed +RESTORE FROM $4 IN $1, $2, '*****' AS OF SYSTEM TIME '1' WITH OPTIONS (skip_missing_foreign_keys) -- identifiers removed +RESTORE FROM $4 IN $1, $2, 'bar' AS OF SYSTEM TIME '1' WITH OPTIONS (skip_missing_foreign_keys) -- passwords exposed parse RESTORE abc.xzy FROM 'a' WITH into_db = 'foo', skip_missing_foreign_keys ---- -RESTORE TABLE abc.xzy FROM 'a' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys) -- normalized! -RESTORE TABLE (abc.xzy) FROM ('a') WITH OPTIONS (into_db = ('foo'), skip_missing_foreign_keys) -- fully parenthesized +RESTORE TABLE abc.xzy FROM '*****' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys) -- normalized! +RESTORE TABLE (abc.xzy) FROM ('*****') WITH OPTIONS (into_db = ('foo'), skip_missing_foreign_keys) -- fully parenthesized RESTORE TABLE abc.xzy FROM '_' WITH OPTIONS (into_db = '_', skip_missing_foreign_keys) -- literals removed -RESTORE TABLE _._ FROM 'a' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys) -- identifiers removed +RESTORE TABLE _._ FROM '*****' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys) -- identifiers removed +RESTORE TABLE abc.xzy FROM 'a' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys) -- passwords exposed parse RESTORE FROM 'a' WITH into_db = 'foo', skip_missing_foreign_keys, skip_localities_check ---- -RESTORE FROM 'a' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys, skip_localities_check) -- normalized! -RESTORE FROM ('a') WITH OPTIONS (into_db = ('foo'), skip_missing_foreign_keys, skip_localities_check) -- fully parenthesized +RESTORE FROM '*****' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys, skip_localities_check) -- normalized! +RESTORE FROM ('*****') WITH OPTIONS (into_db = ('foo'), skip_missing_foreign_keys, skip_localities_check) -- fully parenthesized RESTORE FROM '_' WITH OPTIONS (into_db = '_', skip_missing_foreign_keys, skip_localities_check) -- literals removed -RESTORE FROM 'a' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys, skip_localities_check) -- identifiers removed +RESTORE FROM '*****' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys, skip_localities_check) -- identifiers removed +RESTORE FROM 'a' WITH OPTIONS (into_db = 'foo', skip_missing_foreign_keys, skip_localities_check) -- passwords exposed parse RESTORE foo FROM 'bar' WITH OPTIONS (encryption_passphrase='secret', into_db='baz', skip_missing_foreign_keys, skip_missing_sequences, skip_missing_sequence_owners, skip_missing_views, skip_missing_udfs, detached, skip_localities_check) ---- -RESTORE TABLE foo FROM 'bar' WITH OPTIONS (encryption_passphrase = '*****', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, detached, skip_localities_check) -- normalized! -RESTORE TABLE (foo) FROM ('bar') WITH OPTIONS (encryption_passphrase = '*****', into_db = ('baz'), skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, detached, skip_localities_check) -- fully parenthesized +RESTORE TABLE foo FROM '*****' WITH OPTIONS (encryption_passphrase = '*****', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, detached, skip_localities_check) -- normalized! +RESTORE TABLE (foo) FROM ('*****') WITH OPTIONS (encryption_passphrase = '*****', into_db = ('baz'), skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, detached, skip_localities_check) -- fully parenthesized RESTORE TABLE foo FROM '_' WITH OPTIONS (encryption_passphrase = '*****', into_db = '_', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, detached, skip_localities_check) -- literals removed -RESTORE TABLE _ FROM 'bar' WITH OPTIONS (encryption_passphrase = '*****', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, detached, skip_localities_check) -- identifiers removed +RESTORE TABLE _ FROM '*****' WITH OPTIONS (encryption_passphrase = '*****', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, detached, skip_localities_check) -- identifiers removed RESTORE TABLE foo FROM 'bar' WITH OPTIONS (encryption_passphrase = 'secret', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, detached, skip_localities_check) -- passwords exposed parse RESTORE foo FROM 'bar' WITH ENCRYPTION_PASSPHRASE = 'secret', INTO_DB=baz, SKIP_MISSING_FOREIGN_KEYS, SKIP_MISSING_SEQUENCES, SKIP_MISSING_SEQUENCE_OWNERS, SKIP_MISSING_VIEWS, SKIP_LOCALITIES_CHECK, SKIP_MISSING_UDFS ---- -RESTORE TABLE foo FROM 'bar' WITH OPTIONS (encryption_passphrase = '*****', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, skip_localities_check) -- normalized! -RESTORE TABLE (foo) FROM ('bar') WITH OPTIONS (encryption_passphrase = '*****', into_db = ('baz'), skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, skip_localities_check) -- fully parenthesized +RESTORE TABLE foo FROM '*****' WITH OPTIONS (encryption_passphrase = '*****', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, skip_localities_check) -- normalized! +RESTORE TABLE (foo) FROM ('*****') WITH OPTIONS (encryption_passphrase = '*****', into_db = ('baz'), skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, skip_localities_check) -- fully parenthesized RESTORE TABLE foo FROM '_' WITH OPTIONS (encryption_passphrase = '*****', into_db = '_', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, skip_localities_check) -- literals removed -RESTORE TABLE _ FROM 'bar' WITH OPTIONS (encryption_passphrase = '*****', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, skip_localities_check) -- identifiers removed +RESTORE TABLE _ FROM '*****' WITH OPTIONS (encryption_passphrase = '*****', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, skip_localities_check) -- identifiers removed RESTORE TABLE foo FROM 'bar' WITH OPTIONS (encryption_passphrase = 'secret', into_db = 'baz', skip_missing_foreign_keys, skip_missing_sequence_owners, skip_missing_sequences, skip_missing_views, skip_missing_udfs, skip_localities_check) -- passwords exposed parse @@ -802,74 +868,83 @@ RESTORE VIRTUAL CLUSTER 36 FROM ($1, $2) WITH OPTIONS (virtual_cluster = '5') -- parse BACKUP TABLE foo TO 'bar' WITH revision_history, detached ---- -BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, detached) -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH OPTIONS (revision_history = (true), detached) -- fully parenthesized +BACKUP TABLE foo TO '*****' WITH OPTIONS (revision_history = true, detached) -- normalized! +BACKUP TABLE (foo) TO ('*****') WITH OPTIONS (revision_history = (true), detached) -- fully parenthesized BACKUP TABLE foo TO '_' WITH OPTIONS (revision_history = _, detached) -- literals removed -BACKUP TABLE _ TO 'bar' WITH OPTIONS (revision_history = true, detached) -- identifiers removed +BACKUP TABLE _ TO '*****' WITH OPTIONS (revision_history = true, detached) -- identifiers removed +BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = true, detached) -- passwords exposed parse BACKUP TABLE foo TO 'bar' WITH revision_history = $1, detached, execution locality = $2 ---- -BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = $1, detached, execution locality = $2) -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH OPTIONS (revision_history = ($1), detached, execution locality = ($2)) -- fully parenthesized +BACKUP TABLE foo TO '*****' WITH OPTIONS (revision_history = $1, detached, execution locality = $2) -- normalized! +BACKUP TABLE (foo) TO ('*****') WITH OPTIONS (revision_history = ($1), detached, execution locality = ($2)) -- fully parenthesized BACKUP TABLE foo TO '_' WITH OPTIONS (revision_history = $1, detached, execution locality = $1) -- literals removed -BACKUP TABLE _ TO 'bar' WITH OPTIONS (revision_history = $1, detached, execution locality = $2) -- identifiers removed +BACKUP TABLE _ TO '*****' WITH OPTIONS (revision_history = $1, detached, execution locality = $2) -- identifiers removed +BACKUP TABLE foo TO 'bar' WITH OPTIONS (revision_history = $1, detached, execution locality = $2) -- passwords exposed parse RESTORE TABLE foo FROM 'bar' WITH skip_missing_foreign_keys, skip_missing_sequences, detached ---- -RESTORE TABLE foo FROM 'bar' WITH OPTIONS (skip_missing_foreign_keys, skip_missing_sequences, detached) -- normalized! -RESTORE TABLE (foo) FROM ('bar') WITH OPTIONS (skip_missing_foreign_keys, skip_missing_sequences, detached) -- fully parenthesized +RESTORE TABLE foo FROM '*****' WITH OPTIONS (skip_missing_foreign_keys, skip_missing_sequences, detached) -- normalized! +RESTORE TABLE (foo) FROM ('*****') WITH OPTIONS (skip_missing_foreign_keys, skip_missing_sequences, detached) -- fully parenthesized RESTORE TABLE foo FROM '_' WITH OPTIONS (skip_missing_foreign_keys, skip_missing_sequences, detached) -- literals removed -RESTORE TABLE _ FROM 'bar' WITH OPTIONS (skip_missing_foreign_keys, skip_missing_sequences, detached) -- identifiers removed +RESTORE TABLE _ FROM '*****' WITH OPTIONS (skip_missing_foreign_keys, skip_missing_sequences, detached) -- identifiers removed +RESTORE TABLE foo FROM 'bar' WITH OPTIONS (skip_missing_foreign_keys, skip_missing_sequences, detached) -- passwords exposed parse RESTORE TABLE foo FROM 'bar' WITH remove_regions ---- -RESTORE TABLE foo FROM 'bar' WITH OPTIONS (skip_localities_check, remove_regions) -- normalized! -RESTORE TABLE (foo) FROM ('bar') WITH OPTIONS (skip_localities_check, remove_regions) -- fully parenthesized +RESTORE TABLE foo FROM '*****' WITH OPTIONS (skip_localities_check, remove_regions) -- normalized! +RESTORE TABLE (foo) FROM ('*****') WITH OPTIONS (skip_localities_check, remove_regions) -- fully parenthesized RESTORE TABLE foo FROM '_' WITH OPTIONS (skip_localities_check, remove_regions) -- literals removed -RESTORE TABLE _ FROM 'bar' WITH OPTIONS (skip_localities_check, remove_regions) -- identifiers removed +RESTORE TABLE _ FROM '*****' WITH OPTIONS (skip_localities_check, remove_regions) -- identifiers removed +RESTORE TABLE foo FROM 'bar' WITH OPTIONS (skip_localities_check, remove_regions) -- passwords exposed parse BACKUP INTO 'bar' WITH include_all_virtual_clusters = $1, detached ---- -BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- normalized! -BACKUP INTO ('bar') WITH OPTIONS (detached, include_all_virtual_clusters = ($1)) -- fully parenthesized +BACKUP INTO '*****' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- normalized! +BACKUP INTO ('*****') WITH OPTIONS (detached, include_all_virtual_clusters = ($1)) -- fully parenthesized BACKUP INTO '_' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- literals removed -BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- identifiers removed +BACKUP INTO '*****' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- identifiers removed +BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- passwords exposed parse BACKUP INTO 'bar' WITH include_all_secondary_tenants = $1, detached ---- -BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- normalized! -BACKUP INTO ('bar') WITH OPTIONS (detached, include_all_virtual_clusters = ($1)) -- fully parenthesized +BACKUP INTO '*****' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- normalized! +BACKUP INTO ('*****') WITH OPTIONS (detached, include_all_virtual_clusters = ($1)) -- fully parenthesized BACKUP INTO '_' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- literals removed -BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- identifiers removed +BACKUP INTO '*****' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- identifiers removed +BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = $1) -- passwords exposed parse BACKUP INTO 'bar' WITH include_all_virtual_clusters, detached ---- -BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- normalized! -BACKUP INTO ('bar') WITH OPTIONS (detached, include_all_virtual_clusters = (true)) -- fully parenthesized +BACKUP INTO '*****' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- normalized! +BACKUP INTO ('*****') WITH OPTIONS (detached, include_all_virtual_clusters = (true)) -- fully parenthesized BACKUP INTO '_' WITH OPTIONS (detached, include_all_virtual_clusters = _) -- literals removed -BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- identifiers removed +BACKUP INTO '*****' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- identifiers removed +BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- passwords exposed parse BACKUP INTO 'bar' WITH include_all_secondary_tenants, detached ---- -BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- normalized! -BACKUP INTO ('bar') WITH OPTIONS (detached, include_all_virtual_clusters = (true)) -- fully parenthesized +BACKUP INTO '*****' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- normalized! +BACKUP INTO ('*****') WITH OPTIONS (detached, include_all_virtual_clusters = (true)) -- fully parenthesized BACKUP INTO '_' WITH OPTIONS (detached, include_all_virtual_clusters = _) -- literals removed -BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- identifiers removed +BACKUP INTO '*****' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- identifiers removed +BACKUP INTO 'bar' WITH OPTIONS (detached, include_all_virtual_clusters = true) -- passwords exposed parse RESTORE FROM LATEST IN 'bar' WITH unsafe_restore_incompatible_version, execution locality = 'abc', detached ---- -RESTORE FROM 'latest' IN 'bar' WITH OPTIONS (detached, unsafe_restore_incompatible_version, execution locality = 'abc') -- normalized! -RESTORE FROM ('latest') IN ('bar') WITH OPTIONS (detached, unsafe_restore_incompatible_version, execution locality = ('abc')) -- fully parenthesized +RESTORE FROM 'latest' IN '*****' WITH OPTIONS (detached, unsafe_restore_incompatible_version, execution locality = 'abc') -- normalized! +RESTORE FROM ('latest') IN ('*****') WITH OPTIONS (detached, unsafe_restore_incompatible_version, execution locality = ('abc')) -- fully parenthesized RESTORE FROM '_' IN '_' WITH OPTIONS (detached, unsafe_restore_incompatible_version, execution locality = '_') -- literals removed -RESTORE FROM 'latest' IN 'bar' WITH OPTIONS (detached, unsafe_restore_incompatible_version, execution locality = 'abc') -- identifiers removed +RESTORE FROM 'latest' IN '*****' WITH OPTIONS (detached, unsafe_restore_incompatible_version, execution locality = 'abc') -- identifiers removed +RESTORE FROM 'latest' IN 'bar' WITH OPTIONS (detached, unsafe_restore_incompatible_version, execution locality = 'abc') -- passwords exposed error BACKUP foo TO 'bar' WITH key1, key2 = 'value' @@ -1008,16 +1083,18 @@ HINT: try \h RESTORE parse BACKUP INTO LATEST IN UNLOGGED WITH OPTIONS ( DETACHED = FALSE ) ---- -BACKUP INTO LATEST IN 'unlogged' -- normalized! -BACKUP INTO LATEST IN ('unlogged') -- fully parenthesized +BACKUP INTO LATEST IN '*****' -- normalized! +BACKUP INTO LATEST IN ('*****') -- fully parenthesized BACKUP INTO LATEST IN '_' -- literals removed -BACKUP INTO LATEST IN 'unlogged' -- identifiers removed +BACKUP INTO LATEST IN '*****' -- identifiers removed +BACKUP INTO LATEST IN 'unlogged' -- passwords exposed # Regression test for https://github.com/cockroachdb/cockroach/issues/110411. parse SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (TIME = '1h') ---- -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (TIME = '1h') -SHOW BACKUP CONNECTION ('bar') WITH OPTIONS (TIME = ('1h')) -- fully parenthesized +SHOW BACKUP CONNECTION '*****' WITH OPTIONS (TIME = '1h') -- normalized! +SHOW BACKUP CONNECTION ('*****') WITH OPTIONS (TIME = ('1h')) -- fully parenthesized SHOW BACKUP CONNECTION '_' WITH OPTIONS (TIME = '_') -- literals removed -SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (TIME = '1h') -- identifiers removed +SHOW BACKUP CONNECTION '*****' WITH OPTIONS (TIME = '1h') -- identifiers removed +SHOW BACKUP CONNECTION 'bar' WITH OPTIONS (TIME = '1h') -- passwords exposed diff --git a/pkg/sql/parser/testdata/changefeed b/pkg/sql/parser/testdata/changefeed index 26b4ab854a05..1a8dd1b50805 100644 --- a/pkg/sql/parser/testdata/changefeed +++ b/pkg/sql/parser/testdata/changefeed @@ -17,42 +17,47 @@ EXPERIMENTAL CHANGEFEED FOR TABLE _ FAMILY _ -- identifiers removed parse EXPLAIN CREATE CHANGEFEED FOR TABLE foo INTO 'sink' ---- -EXPLAIN CREATE CHANGEFEED FOR TABLE foo INTO 'sink' -EXPLAIN CREATE CHANGEFEED FOR TABLE (foo) INTO ('sink') -- fully parenthesized +EXPLAIN CREATE CHANGEFEED FOR TABLE foo INTO '*****' -- normalized! +EXPLAIN CREATE CHANGEFEED FOR TABLE (foo) INTO ('*****') -- fully parenthesized EXPLAIN CREATE CHANGEFEED FOR TABLE foo INTO '_' -- literals removed -EXPLAIN CREATE CHANGEFEED FOR TABLE _ INTO 'sink' -- identifiers removed +EXPLAIN CREATE CHANGEFEED FOR TABLE _ INTO '*****' -- identifiers removed +EXPLAIN CREATE CHANGEFEED FOR TABLE foo INTO 'sink' -- passwords exposed parse CREATE CHANGEFEED FOR foo INTO 'sink' ---- -CREATE CHANGEFEED FOR TABLE foo INTO 'sink' -- normalized! -CREATE CHANGEFEED FOR TABLE (foo) INTO ('sink') -- fully parenthesized +CREATE CHANGEFEED FOR TABLE foo INTO '*****' -- normalized! +CREATE CHANGEFEED FOR TABLE (foo) INTO ('*****') -- fully parenthesized CREATE CHANGEFEED FOR TABLE foo INTO '_' -- literals removed -CREATE CHANGEFEED FOR TABLE _ INTO 'sink' -- identifiers removed +CREATE CHANGEFEED FOR TABLE _ INTO '*****' -- identifiers removed +CREATE CHANGEFEED FOR TABLE foo INTO 'sink' -- passwords exposed parse CREATE CHANGEFEED FOR TABLE foo INTO sink ---- -CREATE CHANGEFEED FOR TABLE foo INTO 'sink' -- normalized! -CREATE CHANGEFEED FOR TABLE (foo) INTO ('sink') -- fully parenthesized +CREATE CHANGEFEED FOR TABLE foo INTO '*****' -- normalized! +CREATE CHANGEFEED FOR TABLE (foo) INTO ('*****') -- fully parenthesized CREATE CHANGEFEED FOR TABLE foo INTO '_' -- literals removed -CREATE CHANGEFEED FOR TABLE _ INTO 'sink' -- identifiers removed +CREATE CHANGEFEED FOR TABLE _ INTO '*****' -- identifiers removed +CREATE CHANGEFEED FOR TABLE foo INTO 'sink' -- passwords exposed parse CREATE CHANGEFEED FOR TABLE foo, db.bar, foo FAMILY bar, schema.db.foo INTO 'sink' ---- -CREATE CHANGEFEED FOR TABLE foo, TABLE db.bar, TABLE foo FAMILY bar, TABLE schema.db.foo INTO 'sink' -- normalized! -CREATE CHANGEFEED FOR TABLE (foo), TABLE (db.bar), TABLE (foo) FAMILY bar, TABLE (schema.db.foo) INTO ('sink') -- fully parenthesized +CREATE CHANGEFEED FOR TABLE foo, TABLE db.bar, TABLE foo FAMILY bar, TABLE schema.db.foo INTO '*****' -- normalized! +CREATE CHANGEFEED FOR TABLE (foo), TABLE (db.bar), TABLE (foo) FAMILY bar, TABLE (schema.db.foo) INTO ('*****') -- fully parenthesized CREATE CHANGEFEED FOR TABLE foo, TABLE db.bar, TABLE foo FAMILY bar, TABLE schema.db.foo INTO '_' -- literals removed -CREATE CHANGEFEED FOR TABLE _, TABLE _._, TABLE _ FAMILY _, TABLE _._._ INTO 'sink' -- identifiers removed +CREATE CHANGEFEED FOR TABLE _, TABLE _._, TABLE _ FAMILY _, TABLE _._._ INTO '*****' -- identifiers removed +CREATE CHANGEFEED FOR TABLE foo, TABLE db.bar, TABLE foo FAMILY bar, TABLE schema.db.foo INTO 'sink' -- passwords exposed parse CREATE CHANGEFEED FOR TABLE foo INTO 'sink' ---- -CREATE CHANGEFEED FOR TABLE foo INTO 'sink' -CREATE CHANGEFEED FOR TABLE (foo) INTO ('sink') -- fully parenthesized +CREATE CHANGEFEED FOR TABLE foo INTO '*****' -- normalized! +CREATE CHANGEFEED FOR TABLE (foo) INTO ('*****') -- fully parenthesized CREATE CHANGEFEED FOR TABLE foo INTO '_' -- literals removed -CREATE CHANGEFEED FOR TABLE _ INTO 'sink' -- identifiers removed +CREATE CHANGEFEED FOR TABLE _ INTO '*****' -- identifiers removed +CREATE CHANGEFEED FOR TABLE foo INTO 'sink' -- passwords exposed ## TODO(dan): Implement: ## CREATE CHANGEFEED FOR TABLE foo VALUES FROM (1) TO (2) INTO 'sink' @@ -62,10 +67,11 @@ CREATE CHANGEFEED FOR TABLE _ INTO 'sink' -- identifiers removed parse CREATE CHANGEFEED FOR TABLE foo INTO 'sink' WITH bar = 'baz' ---- -CREATE CHANGEFEED FOR TABLE foo INTO 'sink' WITH OPTIONS (bar = 'baz') -- normalized! -CREATE CHANGEFEED FOR TABLE (foo) INTO ('sink') WITH OPTIONS (bar = ('baz')) -- fully parenthesized +CREATE CHANGEFEED FOR TABLE foo INTO '*****' WITH OPTIONS (bar = 'baz') -- normalized! +CREATE CHANGEFEED FOR TABLE (foo) INTO ('*****') WITH OPTIONS (bar = ('baz')) -- fully parenthesized CREATE CHANGEFEED FOR TABLE foo INTO '_' WITH OPTIONS (bar = '_') -- literals removed -CREATE CHANGEFEED FOR TABLE _ INTO 'sink' WITH OPTIONS (_ = 'baz') -- identifiers removed +CREATE CHANGEFEED FOR TABLE _ INTO '*****' WITH OPTIONS (_ = 'baz') -- identifiers removed +CREATE CHANGEFEED FOR TABLE foo INTO 'sink' WITH OPTIONS (bar = 'baz') -- passwords exposed parse CREATE CHANGEFEED AS SELECT * FROM foo diff --git a/pkg/sql/parser/testdata/copy b/pkg/sql/parser/testdata/copy index 03d0565d1d2c..dd803190f2aa 100644 --- a/pkg/sql/parser/testdata/copy +++ b/pkg/sql/parser/testdata/copy @@ -25,10 +25,11 @@ COPY _ FROM STDIN WITH (QUOTE '"') -- identifiers removed parse COPY crdb_internal.file_upload FROM STDIN WITH destination = 'filename' ---- -COPY crdb_internal.file_upload FROM STDIN WITH (DESTINATION 'filename') -- normalized! -COPY crdb_internal.file_upload FROM STDIN WITH (DESTINATION ('filename')) -- fully parenthesized +COPY crdb_internal.file_upload FROM STDIN WITH (DESTINATION '*****') -- normalized! +COPY crdb_internal.file_upload FROM STDIN WITH (DESTINATION ('*****')) -- fully parenthesized COPY crdb_internal.file_upload FROM STDIN WITH (DESTINATION '_') -- literals removed -COPY _._ FROM STDIN WITH (DESTINATION 'filename') -- identifiers removed +COPY _._ FROM STDIN WITH (DESTINATION '*****') -- identifiers removed +COPY crdb_internal.file_upload FROM STDIN WITH (DESTINATION 'filename') -- passwords exposed parse COPY t (a, b, c) FROM STDIN WITH BINARY @@ -41,10 +42,11 @@ COPY _ (_, _, _) FROM STDIN WITH (FORMAT BINARY) -- identifiers removed parse COPY crdb_internal.file_upload FROM STDIN WITH BINARY destination = 'filename' ---- -COPY crdb_internal.file_upload FROM STDIN WITH (FORMAT BINARY, DESTINATION 'filename') -- normalized! -COPY crdb_internal.file_upload FROM STDIN WITH (FORMAT BINARY, DESTINATION ('filename')) -- fully parenthesized +COPY crdb_internal.file_upload FROM STDIN WITH (FORMAT BINARY, DESTINATION '*****') -- normalized! +COPY crdb_internal.file_upload FROM STDIN WITH (FORMAT BINARY, DESTINATION ('*****')) -- fully parenthesized COPY crdb_internal.file_upload FROM STDIN WITH (FORMAT BINARY, DESTINATION '_') -- literals removed -COPY _._ FROM STDIN WITH (FORMAT BINARY, DESTINATION 'filename') -- identifiers removed +COPY _._ FROM STDIN WITH (FORMAT BINARY, DESTINATION '*****') -- identifiers removed +COPY crdb_internal.file_upload FROM STDIN WITH (FORMAT BINARY, DESTINATION 'filename') -- passwords exposed parse COPY t (a, b, c) FROM STDIN WITH CSV DELIMITER ',' NULL 'NUL' @@ -57,10 +59,11 @@ COPY _ (_, _, _) FROM STDIN WITH (FORMAT CSV, DELIMITER ',', NULL 'NUL') -- iden parse COPY t (a, b, c) FROM STDIN WITH CSV DELIMITER ',' destination = 'filename' ---- -COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ',', DESTINATION 'filename') -- normalized! -COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER (','), DESTINATION ('filename')) -- fully parenthesized +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ',', DESTINATION '*****') -- normalized! +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER (','), DESTINATION ('*****')) -- fully parenthesized COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER '_', DESTINATION '_') -- literals removed -COPY _ (_, _, _) FROM STDIN WITH (FORMAT CSV, DELIMITER ',', DESTINATION 'filename') -- identifiers removed +COPY _ (_, _, _) FROM STDIN WITH (FORMAT CSV, DELIMITER ',', DESTINATION '*****') -- identifiers removed +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ',', DESTINATION 'filename') -- passwords exposed parse COPY t (a, b, c) FROM STDIN BINARY @@ -73,26 +76,29 @@ COPY _ (_, _, _) FROM STDIN WITH (FORMAT BINARY) -- identifiers removed parse COPY t (a, b, c) FROM STDIN destination = 'filename' BINARY ---- -COPY t (a, b, c) FROM STDIN WITH (FORMAT BINARY, DESTINATION 'filename') -- normalized! -COPY t (a, b, c) FROM STDIN WITH (FORMAT BINARY, DESTINATION ('filename')) -- fully parenthesized +COPY t (a, b, c) FROM STDIN WITH (FORMAT BINARY, DESTINATION '*****') -- normalized! +COPY t (a, b, c) FROM STDIN WITH (FORMAT BINARY, DESTINATION ('*****')) -- fully parenthesized COPY t (a, b, c) FROM STDIN WITH (FORMAT BINARY, DESTINATION '_') -- literals removed -COPY _ (_, _, _) FROM STDIN WITH (FORMAT BINARY, DESTINATION 'filename') -- identifiers removed +COPY _ (_, _, _) FROM STDIN WITH (FORMAT BINARY, DESTINATION '*****') -- identifiers removed +COPY t (a, b, c) FROM STDIN WITH (FORMAT BINARY, DESTINATION 'filename') -- passwords exposed parse COPY t (a, b, c) FROM STDIN destination = 'filename' CSV DELIMITER ' ' ---- -COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION 'filename') -- normalized! -COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER (' '), DESTINATION ('filename')) -- fully parenthesized +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION '*****') -- normalized! +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER (' '), DESTINATION ('*****')) -- fully parenthesized COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER '_', DESTINATION '_') -- literals removed -COPY _ (_, _, _) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION 'filename') -- identifiers removed +COPY _ (_, _, _) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION '*****') -- identifiers removed +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION 'filename') -- passwords exposed parse COPY t (a, b, c) FROM STDIN destination = 'filename' CSV DELIMITER ' ' ESCAPE 'x' HEADER ENCODING 'utf8' ---- -COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', ENCODING 'utf8', DESTINATION 'filename', ESCAPE 'x', HEADER true) -- normalized! -COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER (' '), ENCODING ('utf8'), DESTINATION ('filename'), ESCAPE ('x'), HEADER true) -- fully parenthesized +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', ENCODING 'utf8', DESTINATION '*****', ESCAPE 'x', HEADER true) -- normalized! +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER (' '), ENCODING ('utf8'), DESTINATION ('*****'), ESCAPE ('x'), HEADER true) -- fully parenthesized COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER '_', ENCODING '_', DESTINATION '_', ESCAPE '_', HEADER true) -- literals removed -COPY _ (_, _, _) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', ENCODING 'utf8', DESTINATION 'filename', ESCAPE 'x', HEADER true) -- identifiers removed +COPY _ (_, _, _) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', ENCODING 'utf8', DESTINATION '*****', ESCAPE 'x', HEADER true) -- identifiers removed +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', ENCODING 'utf8', DESTINATION 'filename', ESCAPE 'x', HEADER true) -- passwords exposed parse COPY t TO STDOUT @@ -408,10 +414,11 @@ COPY _ FROM STDIN WITH (ENCODING 'utf8', ESCAPE '%', HEADER true) -- identifiers parse COPY t (a, b, c) FROM STDIN destination = 'filename' CSV DELIMITER ' ' ---- -COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION 'filename') -- normalized! -COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER (' '), DESTINATION ('filename')) -- fully parenthesized +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION '*****') -- normalized! +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER (' '), DESTINATION ('*****')) -- fully parenthesized COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER '_', DESTINATION '_') -- literals removed -COPY _ (_, _, _) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION 'filename') -- identifiers removed +COPY _ (_, _, _) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION '*****') -- identifiers removed +COPY t (a, b, c) FROM STDIN WITH (FORMAT CSV, DELIMITER ' ', DESTINATION 'filename') -- passwords exposed error COPY "copytab" FROM STDIN (FORMAT csv, ENCODING 'abc', ENCODING 'def') diff --git a/pkg/sql/parser/testdata/create_external_connection b/pkg/sql/parser/testdata/create_external_connection index 9486b727ef44..525a5fb1137c 100644 --- a/pkg/sql/parser/testdata/create_external_connection +++ b/pkg/sql/parser/testdata/create_external_connection @@ -1,15 +1,17 @@ parse CREATE EXTERNAL CONNECTION 'foo' AS 'bar' ---- -CREATE EXTERNAL CONNECTION 'foo' AS 'bar' -CREATE EXTERNAL CONNECTION ('foo') AS ('bar') -- fully parenthesized +CREATE EXTERNAL CONNECTION 'foo' AS '*****' -- normalized! +CREATE EXTERNAL CONNECTION ('foo') AS ('*****') -- fully parenthesized CREATE EXTERNAL CONNECTION '_' AS '_' -- literals removed -CREATE EXTERNAL CONNECTION 'foo' AS 'bar' -- identifiers removed +CREATE EXTERNAL CONNECTION 'foo' AS '*****' -- identifiers removed +CREATE EXTERNAL CONNECTION 'foo' AS 'bar' -- passwords exposed parse CREATE EXTERNAL CONNECTION IF NOT EXISTS 'foo' AS 'bar' ---- -CREATE EXTERNAL CONNECTION IF NOT EXISTS 'foo' AS 'bar' -CREATE EXTERNAL CONNECTION IF NOT EXISTS ('foo') AS ('bar') -- fully parenthesized +CREATE EXTERNAL CONNECTION IF NOT EXISTS 'foo' AS '*****' -- normalized! +CREATE EXTERNAL CONNECTION IF NOT EXISTS ('foo') AS ('*****') -- fully parenthesized CREATE EXTERNAL CONNECTION IF NOT EXISTS '_' AS '_' -- literals removed -CREATE EXTERNAL CONNECTION IF NOT EXISTS 'foo' AS 'bar' -- identifiers removed +CREATE EXTERNAL CONNECTION IF NOT EXISTS 'foo' AS '*****' -- identifiers removed +CREATE EXTERNAL CONNECTION IF NOT EXISTS 'foo' AS 'bar' -- passwords exposed diff --git a/pkg/sql/parser/testdata/create_schedule b/pkg/sql/parser/testdata/create_schedule index ba8a8bb7a43d..d8887f692ea1 100644 --- a/pkg/sql/parser/testdata/create_schedule +++ b/pkg/sql/parser/testdata/create_schedule @@ -3,66 +3,74 @@ parse CREATE SCHEDULE FOR BACKUP TABLE foo INTO 'bar' RECURRING '@hourly' ---- -CREATE SCHEDULE FOR BACKUP TABLE foo INTO 'bar' RECURRING '@hourly' -CREATE SCHEDULE FOR BACKUP TABLE (foo) INTO ('bar') RECURRING ('@hourly') -- fully parenthesized +CREATE SCHEDULE FOR BACKUP TABLE foo INTO '*****' RECURRING '@hourly' -- normalized! +CREATE SCHEDULE FOR BACKUP TABLE (foo) INTO ('*****') RECURRING ('@hourly') -- fully parenthesized CREATE SCHEDULE FOR BACKUP TABLE foo INTO '_' RECURRING '_' -- literals removed -CREATE SCHEDULE FOR BACKUP TABLE _ INTO 'bar' RECURRING '@hourly' -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE _ INTO '*****' RECURRING '@hourly' -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE foo INTO 'bar' RECURRING '@hourly' -- passwords exposed parse CREATE SCHEDULE 'my schedule' FOR BACKUP TABLE foo INTO 'bar' RECURRING '@daily' ---- -CREATE SCHEDULE 'my schedule' FOR BACKUP TABLE foo INTO 'bar' RECURRING '@daily' -CREATE SCHEDULE ('my schedule') FOR BACKUP TABLE (foo) INTO ('bar') RECURRING ('@daily') -- fully parenthesized +CREATE SCHEDULE 'my schedule' FOR BACKUP TABLE foo INTO '*****' RECURRING '@daily' -- normalized! +CREATE SCHEDULE ('my schedule') FOR BACKUP TABLE (foo) INTO ('*****') RECURRING ('@daily') -- fully parenthesized CREATE SCHEDULE '_' FOR BACKUP TABLE foo INTO '_' RECURRING '_' -- literals removed -CREATE SCHEDULE 'my schedule' FOR BACKUP TABLE _ INTO 'bar' RECURRING '@daily' -- identifiers removed +CREATE SCHEDULE 'my schedule' FOR BACKUP TABLE _ INTO '*****' RECURRING '@daily' -- identifiers removed +CREATE SCHEDULE 'my schedule' FOR BACKUP TABLE foo INTO 'bar' RECURRING '@daily' -- passwords exposed parse CREATE SCHEDULE FOR BACKUP TABLE foo INTO 'bar' RECURRING '@daily' ---- -CREATE SCHEDULE FOR BACKUP TABLE foo INTO 'bar' RECURRING '@daily' -CREATE SCHEDULE FOR BACKUP TABLE (foo) INTO ('bar') RECURRING ('@daily') -- fully parenthesized +CREATE SCHEDULE FOR BACKUP TABLE foo INTO '*****' RECURRING '@daily' -- normalized! +CREATE SCHEDULE FOR BACKUP TABLE (foo) INTO ('*****') RECURRING ('@daily') -- fully parenthesized CREATE SCHEDULE FOR BACKUP TABLE foo INTO '_' RECURRING '_' -- literals removed -CREATE SCHEDULE FOR BACKUP TABLE _ INTO 'bar' RECURRING '@daily' -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE _ INTO '*****' RECURRING '@daily' -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE foo INTO 'bar' RECURRING '@daily' -- passwords exposed parse CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' RECURRING '@daily' FULL BACKUP ALWAYS ---- -CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' RECURRING '@daily' FULL BACKUP ALWAYS -CREATE SCHEDULE FOR BACKUP TABLE (foo), (bar), (buz) INTO ('bar') RECURRING ('@daily') FULL BACKUP ALWAYS -- fully parenthesized +CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO '*****' RECURRING '@daily' FULL BACKUP ALWAYS -- normalized! +CREATE SCHEDULE FOR BACKUP TABLE (foo), (bar), (buz) INTO ('*****') RECURRING ('@daily') FULL BACKUP ALWAYS -- fully parenthesized CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO '_' RECURRING '_' FULL BACKUP ALWAYS -- literals removed -CREATE SCHEDULE FOR BACKUP TABLE _, _, _ INTO 'bar' RECURRING '@daily' FULL BACKUP ALWAYS -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE _, _, _ INTO '*****' RECURRING '@daily' FULL BACKUP ALWAYS -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' RECURRING '@daily' FULL BACKUP ALWAYS -- passwords exposed parse CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' RECURRING '@daily' FULL BACKUP '@weekly' ---- -CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' RECURRING '@daily' FULL BACKUP '@weekly' -CREATE SCHEDULE FOR BACKUP TABLE (foo), (bar), (buz) INTO ('bar') RECURRING ('@daily') FULL BACKUP ('@weekly') -- fully parenthesized +CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO '*****' RECURRING '@daily' FULL BACKUP '@weekly' -- normalized! +CREATE SCHEDULE FOR BACKUP TABLE (foo), (bar), (buz) INTO ('*****') RECURRING ('@daily') FULL BACKUP ('@weekly') -- fully parenthesized CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO '_' RECURRING '_' FULL BACKUP '_' -- literals removed -CREATE SCHEDULE FOR BACKUP TABLE _, _, _ INTO 'bar' RECURRING '@daily' FULL BACKUP '@weekly' -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE _, _, _ INTO '*****' RECURRING '@daily' FULL BACKUP '@weekly' -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' RECURRING '@daily' FULL BACKUP '@weekly' -- passwords exposed parse CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' ---- -CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' -- normalized! -CREATE SCHEDULE FOR BACKUP TABLE (foo), (bar), (buz) INTO ('bar') WITH revision_history = (true) RECURRING ('@daily') FULL BACKUP ('@weekly') -- fully parenthesized +CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO '*****' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' -- normalized! +CREATE SCHEDULE FOR BACKUP TABLE (foo), (bar), (buz) INTO ('*****') WITH revision_history = (true) RECURRING ('@daily') FULL BACKUP ('@weekly') -- fully parenthesized CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO '_' WITH revision_history = _ RECURRING '_' FULL BACKUP '_' -- literals removed -CREATE SCHEDULE FOR BACKUP TABLE _, _, _ INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE _, _, _ INTO '*****' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' -- passwords exposed parse CREATE SCHEDULE FOR BACKUP INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS foo = 'bar' ---- -CREATE SCHEDULE FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS foo = 'bar' -- normalized! -CREATE SCHEDULE FOR BACKUP INTO ('bar') WITH revision_history = (true) RECURRING ('@daily') FULL BACKUP ('@weekly') WITH SCHEDULE OPTIONS foo = ('bar') -- fully parenthesized +CREATE SCHEDULE FOR BACKUP INTO '*****' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS foo = 'bar' -- normalized! +CREATE SCHEDULE FOR BACKUP INTO ('*****') WITH revision_history = (true) RECURRING ('@daily') FULL BACKUP ('@weekly') WITH SCHEDULE OPTIONS foo = ('bar') -- fully parenthesized CREATE SCHEDULE FOR BACKUP INTO '_' WITH revision_history = _ RECURRING '_' FULL BACKUP '_' WITH SCHEDULE OPTIONS foo = '_' -- literals removed -CREATE SCHEDULE FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS _ = 'bar' -- identifiers removed +CREATE SCHEDULE FOR BACKUP INTO '*****' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS _ = 'bar' -- identifiers removed +CREATE SCHEDULE FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS foo = 'bar' -- passwords exposed parse CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS first_run = 'now' ---- -CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS first_run = 'now' -- normalized! -CREATE SCHEDULE IF NOT EXISTS ('baz') FOR BACKUP INTO ('bar') WITH revision_history = (true) RECURRING ('@daily') FULL BACKUP ('@weekly') WITH SCHEDULE OPTIONS first_run = ('now') -- fully parenthesized +CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO '*****' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS first_run = 'now' -- normalized! +CREATE SCHEDULE IF NOT EXISTS ('baz') FOR BACKUP INTO ('*****') WITH revision_history = (true) RECURRING ('@daily') FULL BACKUP ('@weekly') WITH SCHEDULE OPTIONS first_run = ('now') -- fully parenthesized CREATE SCHEDULE IF NOT EXISTS '_' FOR BACKUP INTO '_' WITH revision_history = _ RECURRING '_' FULL BACKUP '_' WITH SCHEDULE OPTIONS first_run = '_' -- literals removed -CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS _ = 'now' -- identifiers removed +CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO '*****' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS _ = 'now' -- identifiers removed +CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS first_run = 'now' -- passwords exposed # Scheduled Changefeed Tests diff --git a/pkg/sql/parser/testdata/import_export b/pkg/sql/parser/testdata/import_export index 22d0e5e65a13..3dcdd91c6c80 100644 --- a/pkg/sql/parser/testdata/import_export +++ b/pkg/sql/parser/testdata/import_export @@ -1,95 +1,107 @@ parse IMPORT TABLE foo FROM PGDUMPCREATE 'nodelocal://0/foo/bar' WITH temp = 'path/to/temp' ---- -IMPORT TABLE foo FROM PGDUMPCREATE 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- normalized! -IMPORT TABLE foo FROM PGDUMPCREATE ('nodelocal://0/foo/bar') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized +IMPORT TABLE foo FROM PGDUMPCREATE '*****' WITH OPTIONS (temp = 'path/to/temp') -- normalized! +IMPORT TABLE foo FROM PGDUMPCREATE ('*****') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized IMPORT TABLE foo FROM PGDUMPCREATE '_' WITH OPTIONS (temp = '_') -- literals removed -IMPORT TABLE _ FROM PGDUMPCREATE 'nodelocal://0/foo/bar' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT TABLE _ FROM PGDUMPCREATE '*****' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT TABLE foo FROM PGDUMPCREATE 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- passwords exposed parse IMPORT TABLE foo FROM PGDUMPCREATE ('nodelocal://0/foo/bar') WITH temp = 'path/to/temp' ---- -IMPORT TABLE foo FROM PGDUMPCREATE 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- normalized! -IMPORT TABLE foo FROM PGDUMPCREATE ('nodelocal://0/foo/bar') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized +IMPORT TABLE foo FROM PGDUMPCREATE '*****' WITH OPTIONS (temp = 'path/to/temp') -- normalized! +IMPORT TABLE foo FROM PGDUMPCREATE ('*****') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized IMPORT TABLE foo FROM PGDUMPCREATE '_' WITH OPTIONS (temp = '_') -- literals removed -IMPORT TABLE _ FROM PGDUMPCREATE 'nodelocal://0/foo/bar' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT TABLE _ FROM PGDUMPCREATE '*****' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT TABLE foo FROM PGDUMPCREATE 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- passwords exposed parse IMPORT INTO foo(id, email) CSV DATA ('path/to/some/file', $1) WITH temp = 'path/to/temp' ---- -IMPORT INTO foo(id, email) CSV DATA ('path/to/some/file', $1) WITH OPTIONS (temp = 'path/to/temp') -- normalized! -IMPORT INTO foo(id, email) CSV DATA (('path/to/some/file'), ($1)) WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized +IMPORT INTO foo(id, email) CSV DATA ('*****', $1) WITH OPTIONS (temp = 'path/to/temp') -- normalized! +IMPORT INTO foo(id, email) CSV DATA (('*****'), ($1)) WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized IMPORT INTO foo(id, email) CSV DATA ('_', $1) WITH OPTIONS (temp = '_') -- literals removed -IMPORT INTO _(_, _) CSV DATA ('path/to/some/file', $1) WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT INTO _(_, _) CSV DATA ('*****', $1) WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT INTO foo(id, email) CSV DATA ('path/to/some/file', $1) WITH OPTIONS (temp = 'path/to/temp') -- passwords exposed parse IMPORT INTO foo CSV DATA ('path/to/some/file', $1) WITH temp = 'path/to/temp' ---- -IMPORT INTO foo CSV DATA ('path/to/some/file', $1) WITH OPTIONS (temp = 'path/to/temp') -- normalized! -IMPORT INTO foo CSV DATA (('path/to/some/file'), ($1)) WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized +IMPORT INTO foo CSV DATA ('*****', $1) WITH OPTIONS (temp = 'path/to/temp') -- normalized! +IMPORT INTO foo CSV DATA (('*****'), ($1)) WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized IMPORT INTO foo CSV DATA ('_', $1) WITH OPTIONS (temp = '_') -- literals removed -IMPORT INTO _ CSV DATA ('path/to/some/file', $1) WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT INTO _ CSV DATA ('*****', $1) WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT INTO foo CSV DATA ('path/to/some/file', $1) WITH OPTIONS (temp = 'path/to/temp') -- passwords exposed parse IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH temp = 'path/to/temp' ---- -IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- normalized! -IMPORT PGDUMP ('nodelocal://0/foo/bar') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized +IMPORT PGDUMP '*****' WITH OPTIONS (temp = 'path/to/temp') -- normalized! +IMPORT PGDUMP ('*****') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized IMPORT PGDUMP '_' WITH OPTIONS (temp = '_') -- literals removed -IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT PGDUMP '*****' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- passwords exposed parse EXPLAIN IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH temp = 'path/to/temp' ---- -EXPLAIN IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- normalized! -EXPLAIN IMPORT PGDUMP ('nodelocal://0/foo/bar') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized +EXPLAIN IMPORT PGDUMP '*****' WITH OPTIONS (temp = 'path/to/temp') -- normalized! +EXPLAIN IMPORT PGDUMP ('*****') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized EXPLAIN IMPORT PGDUMP '_' WITH OPTIONS (temp = '_') -- literals removed -EXPLAIN IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +EXPLAIN IMPORT PGDUMP '*****' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +EXPLAIN IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- passwords exposed parse IMPORT PGDUMP ('nodelocal://0/foo/bar') WITH temp = 'path/to/temp' ---- -IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- normalized! -IMPORT PGDUMP ('nodelocal://0/foo/bar') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized +IMPORT PGDUMP '*****' WITH OPTIONS (temp = 'path/to/temp') -- normalized! +IMPORT PGDUMP ('*****') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized IMPORT PGDUMP '_' WITH OPTIONS (temp = '_') -- literals removed -IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT PGDUMP '*****' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- passwords exposed parse IMPORT PGDUMP ('nodelocal://0/foo/bar') WITH OPTIONS (temp = 'path/to/temp') ---- -IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- normalized! -IMPORT PGDUMP ('nodelocal://0/foo/bar') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized +IMPORT PGDUMP '*****' WITH OPTIONS (temp = 'path/to/temp') -- normalized! +IMPORT PGDUMP ('*****') WITH OPTIONS (temp = ('path/to/temp')) -- fully parenthesized IMPORT PGDUMP '_' WITH OPTIONS (temp = '_') -- literals removed -IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT PGDUMP '*****' WITH OPTIONS (_ = 'path/to/temp') -- identifiers removed +IMPORT PGDUMP 'nodelocal://0/foo/bar' WITH OPTIONS (temp = 'path/to/temp') -- passwords exposed parse EXPORT INTO CSV 'a' FROM TABLE a ---- -EXPORT INTO CSV 'a' FROM TABLE a -EXPORT INTO CSV ('a') FROM TABLE a -- fully parenthesized +EXPORT INTO CSV '*****' FROM TABLE a -- normalized! +EXPORT INTO CSV ('*****') FROM TABLE a -- fully parenthesized EXPORT INTO CSV '_' FROM TABLE a -- literals removed -EXPORT INTO CSV 'a' FROM TABLE _ -- identifiers removed +EXPORT INTO CSV '*****' FROM TABLE _ -- identifiers removed +EXPORT INTO CSV 'a' FROM TABLE a -- passwords exposed parse EXPORT INTO CSV 'a' FROM SELECT * FROM a ---- -EXPORT INTO CSV 'a' FROM SELECT * FROM a -EXPORT INTO CSV ('a') FROM SELECT (*) FROM a -- fully parenthesized +EXPORT INTO CSV '*****' FROM SELECT * FROM a -- normalized! +EXPORT INTO CSV ('*****') FROM SELECT (*) FROM a -- fully parenthesized EXPORT INTO CSV '_' FROM SELECT * FROM a -- literals removed -EXPORT INTO CSV 'a' FROM SELECT * FROM _ -- identifiers removed +EXPORT INTO CSV '*****' FROM SELECT * FROM _ -- identifiers removed +EXPORT INTO CSV 'a' FROM SELECT * FROM a -- passwords exposed parse EXPORT INTO CSV 's3://my/path/%part%.csv' WITH delimiter = '|' FROM TABLE a ---- -EXPORT INTO CSV 's3://my/path/%part%.csv' WITH OPTIONS(delimiter = '|') FROM TABLE a -- normalized! -EXPORT INTO CSV ('s3://my/path/%part%.csv') WITH OPTIONS(delimiter = ('|')) FROM TABLE a -- fully parenthesized +EXPORT INTO CSV '*****' WITH OPTIONS(delimiter = '|') FROM TABLE a -- normalized! +EXPORT INTO CSV ('*****') WITH OPTIONS(delimiter = ('|')) FROM TABLE a -- fully parenthesized EXPORT INTO CSV '_' WITH OPTIONS(delimiter = '_') FROM TABLE a -- literals removed -EXPORT INTO CSV 's3://my/path/%part%.csv' WITH OPTIONS(_ = '|') FROM TABLE _ -- identifiers removed +EXPORT INTO CSV '*****' WITH OPTIONS(_ = '|') FROM TABLE _ -- identifiers removed +EXPORT INTO CSV 's3://my/path/%part%.csv' WITH OPTIONS(delimiter = '|') FROM TABLE a -- passwords exposed parse EXPORT INTO CSV 's3://my/path/%part%.csv' WITH delimiter = '|' FROM SELECT a, sum(b) FROM c WHERE d = 1 ORDER BY sum(b) DESC LIMIT 10 ---- -EXPORT INTO CSV 's3://my/path/%part%.csv' WITH OPTIONS(delimiter = '|') FROM SELECT a, sum(b) FROM c WHERE d = 1 ORDER BY sum(b) DESC LIMIT 10 -- normalized! -EXPORT INTO CSV ('s3://my/path/%part%.csv') WITH OPTIONS(delimiter = ('|')) FROM SELECT (a), (sum((b))) FROM c WHERE ((d) = (1)) ORDER BY (sum((b))) DESC LIMIT (10) -- fully parenthesized +EXPORT INTO CSV '*****' WITH OPTIONS(delimiter = '|') FROM SELECT a, sum(b) FROM c WHERE d = 1 ORDER BY sum(b) DESC LIMIT 10 -- normalized! +EXPORT INTO CSV ('*****') WITH OPTIONS(delimiter = ('|')) FROM SELECT (a), (sum((b))) FROM c WHERE ((d) = (1)) ORDER BY (sum((b))) DESC LIMIT (10) -- fully parenthesized EXPORT INTO CSV '_' WITH OPTIONS(delimiter = '_') FROM SELECT a, sum(b) FROM c WHERE d = _ ORDER BY sum(b) DESC LIMIT _ -- literals removed -EXPORT INTO CSV 's3://my/path/%part%.csv' WITH OPTIONS(_ = '|') FROM SELECT _, _(_) FROM _ WHERE _ = 1 ORDER BY _(_) DESC LIMIT 10 -- identifiers removed +EXPORT INTO CSV '*****' WITH OPTIONS(_ = '|') FROM SELECT _, _(_) FROM _ WHERE _ = 1 ORDER BY _(_) DESC LIMIT 10 -- identifiers removed +EXPORT INTO CSV 's3://my/path/%part%.csv' WITH OPTIONS(delimiter = '|') FROM SELECT a, sum(b) FROM c WHERE d = 1 ORDER BY sum(b) DESC LIMIT 10 -- passwords exposed diff --git a/pkg/sql/parser/testdata/prepared_stmts b/pkg/sql/parser/testdata/prepared_stmts index 8c62e696fc2e..a5f53ccaa31c 100644 --- a/pkg/sql/parser/testdata/prepared_stmts +++ b/pkg/sql/parser/testdata/prepared_stmts @@ -97,10 +97,11 @@ PREPARE _ (INT8) AS DELETE FROM _ WHERE _ = $1 -- identifiers removed parse PREPARE a AS BACKUP DATABASE a TO 'b' ---- -PREPARE a AS BACKUP DATABASE a TO 'b' -PREPARE a AS BACKUP DATABASE a TO ('b') -- fully parenthesized +PREPARE a AS BACKUP DATABASE a TO '*****' -- normalized! +PREPARE a AS BACKUP DATABASE a TO ('*****') -- fully parenthesized PREPARE a AS BACKUP DATABASE a TO '_' -- literals removed -PREPARE _ AS BACKUP DATABASE _ TO 'b' -- identifiers removed +PREPARE _ AS BACKUP DATABASE _ TO '*****' -- identifiers removed +PREPARE a AS BACKUP DATABASE a TO 'b' -- passwords exposed parse PREPARE a (STRING) AS BACKUP DATABASE a TO $1 @@ -113,10 +114,11 @@ PREPARE _ (STRING) AS BACKUP DATABASE _ TO $1 -- identifiers removed parse PREPARE a AS RESTORE DATABASE a FROM 'b' ---- -PREPARE a AS RESTORE DATABASE a FROM 'b' -PREPARE a AS RESTORE DATABASE a FROM ('b') -- fully parenthesized +PREPARE a AS RESTORE DATABASE a FROM '*****' -- normalized! +PREPARE a AS RESTORE DATABASE a FROM ('*****') -- fully parenthesized PREPARE a AS RESTORE DATABASE a FROM '_' -- literals removed -PREPARE _ AS RESTORE DATABASE _ FROM 'b' -- identifiers removed +PREPARE _ AS RESTORE DATABASE _ FROM '*****' -- identifiers removed +PREPARE a AS RESTORE DATABASE a FROM 'b' -- passwords exposed parse PREPARE a (STRING) AS RESTORE DATABASE a FROM $1 @@ -265,10 +267,11 @@ PREPARE _ (INT8) AS RESUME JOBS SELECT $1 -- identifiers removed parse PREPARE a AS IMPORT INTO a CSV DATA ('c') WITH temp = 'd' ---- -PREPARE a AS IMPORT INTO a CSV DATA ('c') WITH OPTIONS (temp = 'd') -- normalized! -PREPARE a AS IMPORT INTO a CSV DATA (('c')) WITH OPTIONS (temp = ('d')) -- fully parenthesized +PREPARE a AS IMPORT INTO a CSV DATA ('*****') WITH OPTIONS (temp = 'd') -- normalized! +PREPARE a AS IMPORT INTO a CSV DATA (('*****')) WITH OPTIONS (temp = ('d')) -- fully parenthesized PREPARE a AS IMPORT INTO a CSV DATA ('_') WITH OPTIONS (temp = '_') -- literals removed -PREPARE _ AS IMPORT INTO _ CSV DATA ('c') WITH OPTIONS (_ = 'd') -- identifiers removed +PREPARE _ AS IMPORT INTO _ CSV DATA ('*****') WITH OPTIONS (_ = 'd') -- identifiers removed +PREPARE a AS IMPORT INTO a CSV DATA ('c') WITH OPTIONS (temp = 'd') -- passwords exposed parse PREPARE a (STRING, STRING, STRING) AS IMPORT INTO a CSV DATA ($2) WITH temp = $3 diff --git a/pkg/sql/parser/testdata/show b/pkg/sql/parser/testdata/show index 059291a15176..6937bcc1b05a 100644 --- a/pkg/sql/parser/testdata/show +++ b/pkg/sql/parser/testdata/show @@ -2087,26 +2087,29 @@ SHOW VIRTUAL CLUSTER _ WITH REPLICATION STATUS, PRIOR REPLICATION DETAILS, CAPAB parse SHOW BACKUP 'family' IN ('string', 'placeholder', 'placeholder', 'placeholder', 'string', 'placeholder', 'string', 'placeholder') WITH incremental_location = 'nullif', privileges, debug_dump_metadata_sst ---- -SHOW BACKUP 'family' IN ('string', 'placeholder', 'placeholder', 'placeholder', 'string', 'placeholder', 'string', 'placeholder') WITH OPTIONS (incremental_location = 'nullif', privileges, debug_dump_metadata_sst) -- normalized! -SHOW BACKUP ('family') IN (('string'), ('placeholder'), ('placeholder'), ('placeholder'), ('string'), ('placeholder'), ('string'), ('placeholder')) WITH OPTIONS (incremental_location = ('nullif'), privileges, debug_dump_metadata_sst) -- fully parenthesized +SHOW BACKUP 'family' IN ('*****', '*****', '*****', '*****', '*****', '*****', '*****', '*****') WITH OPTIONS (incremental_location = '*****', privileges, debug_dump_metadata_sst) -- normalized! +SHOW BACKUP ('family') IN (('*****'), ('*****'), ('*****'), ('*****'), ('*****'), ('*****'), ('*****'), ('*****')) WITH OPTIONS (incremental_location = ('*****'), privileges, debug_dump_metadata_sst) -- fully parenthesized SHOW BACKUP '_' IN ('_', '_', '_', '_', '_', '_', '_', '_') WITH OPTIONS (incremental_location = '_', privileges, debug_dump_metadata_sst) -- literals removed -SHOW BACKUP 'family' IN ('string', 'placeholder', 'placeholder', 'placeholder', 'string', 'placeholder', 'string', 'placeholder') WITH OPTIONS (incremental_location = 'nullif', privileges, debug_dump_metadata_sst) -- identifiers removed +SHOW BACKUP 'family' IN ('*****', '*****', '*****', '*****', '*****', '*****', '*****', '*****') WITH OPTIONS (incremental_location = '*****', privileges, debug_dump_metadata_sst) -- identifiers removed +SHOW BACKUP 'family' IN ('string', 'placeholder', 'placeholder', 'placeholder', 'string', 'placeholder', 'string', 'placeholder') WITH OPTIONS (incremental_location = 'nullif', privileges, debug_dump_metadata_sst) -- passwords exposed parse SHOW BACKUP 'abc' WITH SKIP SIZE ---- -SHOW BACKUP 'abc' WITH OPTIONS (skip size) -- normalized! -SHOW BACKUP ('abc') WITH OPTIONS (skip size) -- fully parenthesized +SHOW BACKUP '*****' WITH OPTIONS (skip size) -- normalized! +SHOW BACKUP ('*****') WITH OPTIONS (skip size) -- fully parenthesized SHOW BACKUP '_' WITH OPTIONS (skip size) -- literals removed -SHOW BACKUP 'abc' WITH OPTIONS (skip size) -- identifiers removed +SHOW BACKUP '*****' WITH OPTIONS (skip size) -- identifiers removed +SHOW BACKUP 'abc' WITH OPTIONS (skip size) -- passwords exposed parse SHOW BACKUP 'abc' WITH NOWAIT ---- -SHOW BACKUP 'abc' WITH OPTIONS (skip size) -- normalized! -SHOW BACKUP ('abc') WITH OPTIONS (skip size) -- fully parenthesized +SHOW BACKUP '*****' WITH OPTIONS (skip size) -- normalized! +SHOW BACKUP ('*****') WITH OPTIONS (skip size) -- fully parenthesized SHOW BACKUP '_' WITH OPTIONS (skip size) -- literals removed -SHOW BACKUP 'abc' WITH OPTIONS (skip size) -- identifiers removed +SHOW BACKUP '*****' WITH OPTIONS (skip size) -- identifiers removed +SHOW BACKUP 'abc' WITH OPTIONS (skip size) -- passwords exposed parse SHOW DEFAULT SESSION VARIABLES FOR ROLE foo diff --git a/pkg/sql/plpgsql/parser/testdata/stmt_exec_sql b/pkg/sql/plpgsql/parser/testdata/stmt_exec_sql index 0524b8221fb1..5cadf819151a 100644 --- a/pkg/sql/plpgsql/parser/testdata/stmt_exec_sql +++ b/pkg/sql/plpgsql/parser/testdata/stmt_exec_sql @@ -8,12 +8,12 @@ END ---- DECLARE BEGIN -IMPORT TABLE foo FROM PGDUMP 'userfile://defaultdb.public.userfiles_root/db.sql' WITH OPTIONS (max_row_size = '524288'); +IMPORT TABLE foo FROM PGDUMP '*****' WITH OPTIONS (max_row_size = '524288'); END; -- normalized! DECLARE BEGIN -IMPORT TABLE foo FROM PGDUMP ('userfile://defaultdb.public.userfiles_root/db.sql') WITH OPTIONS (max_row_size = ('524288')); +IMPORT TABLE foo FROM PGDUMP ('*****') WITH OPTIONS (max_row_size = ('524288')); END; -- fully parenthesized DECLARE @@ -23,9 +23,14 @@ END; -- literals removed DECLARE BEGIN -IMPORT TABLE _ FROM PGDUMP 'userfile://defaultdb.public.userfiles_root/db.sql' WITH OPTIONS (_ = '524288'); +IMPORT TABLE _ FROM PGDUMP '*****' WITH OPTIONS (_ = '524288'); END; -- identifiers removed +DECLARE +BEGIN +IMPORT TABLE foo FROM PGDUMP 'userfile://defaultdb.public.userfiles_root/db.sql' WITH OPTIONS (max_row_size = '524288'); +END; + -- passwords exposed parse DECLARE diff --git a/pkg/sql/sem/tree/alter_backup.go b/pkg/sql/sem/tree/alter_backup.go index 51a9a0ad48b8..b7291bb7d929 100644 --- a/pkg/sql/sem/tree/alter_backup.go +++ b/pkg/sql/sem/tree/alter_backup.go @@ -29,7 +29,7 @@ func (node *AlterBackup) Format(ctx *FmtCtx) { ctx.WriteString(" IN ") } - ctx.FormatNode(node.Backup) + ctx.FormatURI(node.Backup) ctx.FormatNode(&node.Cmds) } @@ -64,10 +64,10 @@ type AlterBackupKMS struct { // Format implements the NodeFormatter interface. func (node *AlterBackupKMS) Format(ctx *FmtCtx) { ctx.WriteString(" ADD NEW_KMS=") - ctx.FormatNode(&node.KMSInfo.NewKMSURI) + ctx.FormatURIs(node.KMSInfo.NewKMSURI) ctx.WriteString(" WITH OLD_KMS=") - ctx.FormatNode(&node.KMSInfo.OldKMSURI) + ctx.FormatURIs(node.KMSInfo.OldKMSURI) } // BackupKMS represents possible options used when altering a backup KMS diff --git a/pkg/sql/sem/tree/alter_backup_schedule.go b/pkg/sql/sem/tree/alter_backup_schedule.go index a58be8ab02d6..e64191591e7c 100644 --- a/pkg/sql/sem/tree/alter_backup_schedule.go +++ b/pkg/sql/sem/tree/alter_backup_schedule.go @@ -88,7 +88,7 @@ type AlterBackupScheduleSetInto struct { // Format implements the NodeFormatter interface. func (node *AlterBackupScheduleSetInto) Format(ctx *FmtCtx) { ctx.WriteString("SET INTO ") - ctx.FormatNode(&node.Into) + ctx.FormatURIs(node.Into) } // AlterBackupScheduleSetWith represents an SET command diff --git a/pkg/sql/sem/tree/alter_changefeed.go b/pkg/sql/sem/tree/alter_changefeed.go index a7c5945bd490..6d72db4ed462 100644 --- a/pkg/sql/sem/tree/alter_changefeed.go +++ b/pkg/sql/sem/tree/alter_changefeed.go @@ -91,7 +91,15 @@ type AlterChangefeedSetOptions struct { // Format implements the NodeFormatter interface. func (node *AlterChangefeedSetOptions) Format(ctx *FmtCtx) { ctx.WriteString(" SET ") - ctx.FormatNode(&node.Options) + node.Options.formatEach(ctx, func(n *KVOption, ctx *FmtCtx) { + // The "sink" option is a URL. (Use a literal here to avoid pulling in + // changefeedbase as a dependency.) + if string(n.Key) == "sink" { + ctx.FormatURI(n.Value) + } else { + ctx.FormatNode(n.Value) + } + }) } // AlterChangefeedUnsetOptions represents an UNSET command diff --git a/pkg/sql/sem/tree/backup.go b/pkg/sql/sem/tree/backup.go index 971d53085c4a..e98eb77c42b4 100644 --- a/pkg/sql/sem/tree/backup.go +++ b/pkg/sql/sem/tree/backup.go @@ -101,14 +101,19 @@ func (node *Backup) Format(ctx *FmtCtx) { } else { ctx.WriteString("TO ") } - ctx.FormatNode(&node.To) + ctx.FormatURIs(node.To) if node.AsOf.Expr != nil { ctx.WriteString(" ") ctx.FormatNode(&node.AsOf) } if node.IncrementalFrom != nil { ctx.WriteString(" INCREMENTAL FROM ") - ctx.FormatNode(&node.IncrementalFrom) + for i, from := range node.IncrementalFrom { + if i > 0 { + ctx.WriteString(", ") + } + ctx.FormatURI(from) + } } if !node.Options.IsDefault() { @@ -192,7 +197,7 @@ func (node *Restore) Format(ctx *FmtCtx) { if i > 0 { ctx.WriteString(", ") } - ctx.FormatNode(&node.From[i]) + ctx.FormatURIs(node.From[i]) } if node.AsOf.Expr != nil { ctx.WriteString(" ") @@ -226,6 +231,13 @@ func (o *KVOptions) HasKey(key Name) bool { // Format implements the NodeFormatter interface. func (o *KVOptions) Format(ctx *FmtCtx) { + o.formatEach(ctx, func(n *KVOption, ctx *FmtCtx) { + ctx.FormatNode(n.Value) + }) +} + +// formatEach is like Format but allows custom formatting of the value part. +func (o *KVOptions) formatEach(ctx *FmtCtx, formatValue func(*KVOption, *FmtCtx)) { for i := range *o { n := &(*o)[i] if i > 0 { @@ -238,7 +250,7 @@ func (o *KVOptions) Format(ctx *FmtCtx) { }) if n.Value != nil { ctx.WriteString(` = `) - ctx.FormatNode(n.Value) + formatValue(n, ctx) } } } @@ -293,13 +305,13 @@ func (o *BackupOptions) Format(ctx *FmtCtx) { if o.EncryptionKMSURI != nil { maybeAddSep() ctx.WriteString("kms = ") - ctx.FormatNode(&o.EncryptionKMSURI) + ctx.FormatURIs(o.EncryptionKMSURI) } if o.IncrementalStorage != nil { maybeAddSep() ctx.WriteString("incremental_location = ") - ctx.FormatNode(&o.IncrementalStorage) + ctx.FormatURIs(o.IncrementalStorage) } if o.ExecutionLocality != nil { @@ -417,7 +429,7 @@ func (o *RestoreOptions) Format(ctx *FmtCtx) { if o.DecryptionKMSURI != nil { maybeAddSep() ctx.WriteString("kms = ") - ctx.FormatNode(&o.DecryptionKMSURI) + ctx.FormatURIs(o.DecryptionKMSURI) } if o.IntoDB != nil { @@ -470,7 +482,7 @@ func (o *RestoreOptions) Format(ctx *FmtCtx) { if o.IncrementalStorage != nil { maybeAddSep() ctx.WriteString("incremental_location = ") - ctx.FormatNode(&o.IncrementalStorage) + ctx.FormatURIs(o.IncrementalStorage) } if o.AsTenant != nil { diff --git a/pkg/sql/sem/tree/changefeed.go b/pkg/sql/sem/tree/changefeed.go index 85f21f8f607b..20dbc59f1744 100644 --- a/pkg/sql/sem/tree/changefeed.go +++ b/pkg/sql/sem/tree/changefeed.go @@ -44,7 +44,7 @@ func (node *CreateChangefeed) Format(ctx *FmtCtx) { ctx.FormatNode(&node.Targets) if node.SinkURI != nil { ctx.WriteString(" INTO ") - ctx.FormatNode(node.SinkURI) + ctx.FormatURI(node.SinkURI) } if node.Options != nil { ctx.WriteString(" WITH OPTIONS (") diff --git a/pkg/sql/sem/tree/copy.go b/pkg/sql/sem/tree/copy.go index b801894e5b83..6647d8d40e5d 100644 --- a/pkg/sql/sem/tree/copy.go +++ b/pkg/sql/sem/tree/copy.go @@ -136,7 +136,7 @@ func (o *CopyOptions) Format(ctx *FmtCtx) { // by copy_file_upload.go, so this will provide backward // compatibility with older servers. ctx.WriteString("DESTINATION ") - ctx.FormatNode(o.Destination) + ctx.FormatURI(o.Destination) addSep = true } if o.Escape != nil { diff --git a/pkg/sql/sem/tree/create.go b/pkg/sql/sem/tree/create.go index 3ef281b91337..00862a601173 100644 --- a/pkg/sql/sem/tree/create.go +++ b/pkg/sql/sem/tree/create.go @@ -2177,7 +2177,7 @@ func (node *CreateExternalConnection) Format(ctx *FmtCtx) { ctx.WriteString("CREATE EXTERNAL CONNECTION") ctx.FormatNode(&node.ConnectionLabelSpec) ctx.WriteString(" AS ") - ctx.FormatNode(node.As) + ctx.FormatURI(node.As) } // CreateTenant represents a CREATE VIRTUAL CLUSTER statement. diff --git a/pkg/sql/sem/tree/export.go b/pkg/sql/sem/tree/export.go index 9b592ff65ad6..f1c7181c5ad3 100644 --- a/pkg/sql/sem/tree/export.go +++ b/pkg/sql/sem/tree/export.go @@ -25,7 +25,7 @@ func (node *Export) Format(ctx *FmtCtx) { ctx.WriteString("EXPORT INTO ") ctx.WriteString(node.FileFormat) ctx.WriteString(" ") - ctx.FormatNode(node.File) + ctx.FormatURI(node.File) if node.Options != nil { ctx.WriteString(" WITH OPTIONS(") ctx.FormatNode(&node.Options) diff --git a/pkg/sql/sem/tree/format.go b/pkg/sql/sem/tree/format.go index 437275f4422a..91e1b141e6dc 100644 --- a/pkg/sql/sem/tree/format.go +++ b/pkg/sql/sem/tree/format.go @@ -34,7 +34,7 @@ func (f FmtFlags) HasFlags(subset FmtFlags) bool { return f&subset == subset } -// HasAnyFlags tests whether any of the given flags are all set. +// HasAnyFlags tests whether any of the given flags are set. func (f FmtFlags) HasAnyFlags(subset FmtFlags) bool { return f&subset != 0 } @@ -192,6 +192,10 @@ const ( // FmtAlwaysQualifyUserDefinedTypeNames instructs the pretty-printer to include // the name of user-defined types as a three-part name. FmtAlwaysQualifyUserDefinedTypeNames + + // FmtShowFullURIs instructs the pretty-printer to not sanitize URIs. If not + // set, URIs are sanitized to prevent leaking secrets. + FmtShowFullURIs ) const genericArityIndicator = "__more__" @@ -277,8 +281,8 @@ const ( // other things (eg. fixing #33429). FmtExport = FmtBareStrings | fmtRawStrings - // fmtAlwaysQualifyNames will fully qualify various types of names. - fmtAlwaysQualifyNames = FmtAlwaysQualifyTableNames | FmtAlwaysQualifyUserDefinedTypeNames + // FmtAlwaysQualifyNames will fully qualify various types of names. + FmtAlwaysQualifyNames = FmtAlwaysQualifyTableNames | FmtAlwaysQualifyUserDefinedTypeNames ) const flagsRequiringAnnotations = FmtAlwaysQualifyTableNames @@ -433,11 +437,18 @@ func (ctx *FmtCtx) WithFlags(flags FmtFlags, fn func()) { fn() } -// HasFlags returns true iff the given flags are set in the formatter context. +// HasFlags returns true iff all of the given flags are set in the formatter +// context. func (ctx *FmtCtx) HasFlags(f FmtFlags) bool { return ctx.flags.HasFlags(f) } +// HasAnyFlags returns true iff any of the given flags are set in the formatter +// context. +func (ctx *FmtCtx) HasAnyFlags(f FmtFlags) bool { + return ctx.flags.HasAnyFlags(f) +} + // Printf calls fmt.Fprintf on the linked bytes.Buffer. It is provided // for convenience, to avoid having to call fmt.Fprintf(&ctx.Buffer, ...). // @@ -469,6 +480,59 @@ func (ctx *FmtCtx) FormatNameP(s *string) { ctx.FormatNode((*Name)(s)) } +// FormatURIs formats a list of string literals or placeholders containing URIs. +func (ctx *FmtCtx) FormatURIs(uris []Expr) { + if len(uris) > 1 { + ctx.WriteString("(") + } + for i, uri := range uris { + if i > 0 { + ctx.WriteString(", ") + } + ctx.FormatURI(uri) + } + if len(uris) > 1 { + ctx.WriteString(")") + } +} + +// FormatURI formats a string literal or placeholder containing a URI. If the +// node is a string literal, we redact the contents to avoid leaking secrets. +func (ctx *FmtCtx) FormatURI(uri Expr) { + switch n := uri.(type) { + case *StrVal, *DString: + if ctx.HasAnyFlags( + FmtShowPasswords | FmtShowFullURIs | FmtHideConstants | FmtConstantsAsUnderscores, + ) { + ctx.FormatNode(n) + return + } + var raw, elided string + if str, ok := n.(*StrVal); ok { + raw = str.RawString() + } else { + raw = string(MustBeDString(uri)) + } + if raw == "" || raw == "_" { + // Some commands treat empty URIs as special. And if we've re-parsed a URI + // formatted with FmtHideConstants, we should not try to interpret it as a + // URL but should leave it as-is. + ctx.FormatNode(n) + return + } + // TODO(michae2): Call SanitizeExternalStorageURI for fine-grained + // sanitization. + elided = strings.Trim(PasswordSubstitution, "'") + ctx.FormatNode(NewStrVal(elided)) + case *Placeholder: + ctx.FormatNode(n) + default: + // We don't want to fail to sanitize other literals, so disallow other types + // of expressions (which should already be disallowed by the parser anyway). + panic(errors.AssertionFailedf("expected *StrVal, *DString, or *Placeholder, found %T", n)) + } +} + // FormatNode recurses into a node for pretty-printing. // Flag-driven special cases can hook into this. func (ctx *FmtCtx) FormatNode(n NodeFormatter) { @@ -658,7 +722,7 @@ func AsStringWithFlags(n NodeFormatter, fl FmtFlags, opts ...FmtCtxOption) strin // AsStringWithFQNames pretty prints a node to a string with the // FmtAlwaysQualifyTableNames flag (which requires annotations). func AsStringWithFQNames(n NodeFormatter, ann *Annotations) string { - ctx := NewFmtCtx(fmtAlwaysQualifyNames, FmtAnnotations(ann)) + ctx := NewFmtCtx(FmtAlwaysQualifyNames, FmtAnnotations(ann)) ctx.FormatNode(n) return ctx.CloseAndGetString() } diff --git a/pkg/sql/sem/tree/import.go b/pkg/sql/sem/tree/import.go index a1510c9de11f..a777f87e7b15 100644 --- a/pkg/sql/sem/tree/import.go +++ b/pkg/sql/sem/tree/import.go @@ -35,7 +35,7 @@ func (node *Import) Format(ctx *FmtCtx) { } ctx.WriteString(node.FileFormat) ctx.WriteByte(' ') - ctx.FormatNode(&node.Files) + ctx.FormatURIs(node.Files) } else { if node.Into { ctx.WriteString("INTO ") @@ -52,9 +52,14 @@ func (node *Import) Format(ctx *FmtCtx) { ctx.FormatNode(node.Table) } ctx.WriteString(node.FileFormat) - ctx.WriteString(" DATA (") - ctx.FormatNode(&node.Files) - ctx.WriteString(")") + ctx.WriteString(" DATA ") + if len(node.Files) == 1 { + ctx.WriteString("(") + } + ctx.FormatURIs(node.Files) + if len(node.Files) == 1 { + ctx.WriteString(")") + } } if node.Options != nil { diff --git a/pkg/sql/sem/tree/schedule.go b/pkg/sql/sem/tree/schedule.go index 48a19e5b3f79..1e4ad167921f 100644 --- a/pkg/sql/sem/tree/schedule.go +++ b/pkg/sql/sem/tree/schedule.go @@ -60,7 +60,7 @@ func (node *ScheduledBackup) Format(ctx *FmtCtx) { } ctx.WriteString(" INTO ") - ctx.FormatNode(&node.To) + ctx.FormatURIs(node.To) if !node.BackupOptions.IsDefault() { ctx.WriteString(" WITH ") diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index 0d80a99da146..556c8e487854 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -113,7 +113,7 @@ type ShowBackup struct { func (node *ShowBackup) Format(ctx *FmtCtx) { if node.InCollection != nil && node.Path == nil { ctx.WriteString("SHOW BACKUPS IN ") - ctx.FormatNode(&node.InCollection) + ctx.FormatURIs(node.InCollection) return } ctx.WriteString("SHOW BACKUP ") @@ -133,10 +133,12 @@ func (node *ShowBackup) Format(ctx *FmtCtx) { ctx.WriteString("FROM ") } - ctx.FormatNode(node.Path) if node.InCollection != nil { + ctx.FormatNode(node.Path) ctx.WriteString(" IN ") - ctx.FormatNode(&node.InCollection) + ctx.FormatURIs(node.InCollection) + } else { + ctx.FormatURI(node.Path) } if !node.Options.IsDefault() { ctx.WriteString(" WITH OPTIONS (") @@ -205,7 +207,7 @@ func (o *ShowBackupOptions) Format(ctx *FmtCtx) { if o.IncrementalStorage != nil { maybeAddSep() ctx.WriteString("incremental_location = ") - ctx.FormatNode(&o.IncrementalStorage) + ctx.FormatURIs(o.IncrementalStorage) } if o.Privileges { @@ -221,7 +223,7 @@ func (o *ShowBackupOptions) Format(ctx *FmtCtx) { if o.DecryptionKMSURI != nil { maybeAddSep() ctx.WriteString("kms = ") - ctx.FormatNode(&o.DecryptionKMSURI) + ctx.FormatURIs(o.DecryptionKMSURI) } if o.SkipSize { maybeAddSep()