Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
131106: clusterversion: introduce rac2 cluster version gates r=sumeerbhola a=kvoli

Introduce two new cluster version gates:

```
V24_3_UseRACV2WithV1EntryEncoding
V24_3_UseRACV2Full
```

Upon a range leader first encountering
`V24_3_UseRACV2WithV1EntryEncoding` via `handleRaftReadyRaftMuLocked`,
it will begin a new term using the replication flow control v2 protocol,
creating a `RangeController` but continue using the v1 entry encoding
and raft still operating in push mode.

Upon a range leader first encountering `V24_3_UseRACV2Full`, it will continue
using the replication flow control v2 protocol, but will now switch to
using the V2 entry encoding.

Note that the necessary protocol migration at the leader, (base) =>
`V24_3_UseRACV2WithV1EntryEncoding` occurs before any other calls in
`handleRaftReadyRaftMuLocked`.

The two version gates are necessary to ensure there are never v2 encoded
entries in the raft log while there is a possibility of a leader running
v1.

---

Move `EnabledWhenLeaderLevel` from `replica_rac2` to the parent package
`kvflowcontrol` and rename `V2EnabledWhenLeaderLevel` to reflect the
move to a shared v1/v2 package.

Also move the corresponding function `racV2EnabledWhenLeaderLevel` to
`kvflowcontrol`. `GetV2EnabledWhenLeaderLevel` will check if there are
testing knob overrides for the enabled level, and if not continue
returning `V2NotEnabledWhenLeader`. Some commentary and todos are also
left around this function, for when we enable the protocol and
separately, pull mode.

Resolves: #131102
Release note: None

131231: backupccl: fill incremental cluster id on alter schedule r=msbutler a=kev-cao

When altering the recurrence of a singleton full backup schedule created before v23.2, the corresponding new incremental schedule creation will fail due to a missing cluster ID. This patch ensures that the cluster ID is set when creating the incremental.

Fixes: #131127

Release note (bug fix): Fixed a bug introduced in v23.2.0 where creating a new incremental schedule via `ALTER SCHEDULE` on a full backup schedule created on an older version would fail.

Co-authored-by: Austen McClernon <[email protected]>
Co-authored-by: Kevin Cao <[email protected]>
  • Loading branch information
3 people committed Sep 24, 2024
3 parents 7d241ed + 7cf9492 + f943f81 commit 311c0d5
Show file tree
Hide file tree
Showing 14 changed files with 150 additions and 50 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -399,4 +399,4 @@ trace.snapshot.rate duration 0s if non-zero, interval at which background trace
trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https://<ui>/#/debug/tracez application
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used. application
ui.display_timezone enumeration etc/utc the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1] application
version version 1000024.2-upgrading-to-1000024.3-step-016 set the active cluster version in the format '<major>.<minor>' application
version version 1000024.2-upgrading-to-1000024.3-step-020 set the active cluster version in the format '<major>.<minor>' application
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -356,6 +356,6 @@
<tr><td><div id="setting-trace-span-registry-enabled" class="anchored"><code>trace.span_registry.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>if set, ongoing traces can be seen at https://&lt;ui&gt;/#/debug/tracez</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-trace-zipkin-collector" class="anchored"><code>trace.zipkin.collector</code></div></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as &lt;host&gt;:&lt;port&gt;. If no port is specified, 9411 will be used.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-ui-display-timezone" class="anchored"><code>ui.display_timezone</code></div></td><td>enumeration</td><td><code>etc/utc</code></td><td>the timezone used to format timestamps in the ui [etc/utc = 0, america/new_york = 1]</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000024.2-upgrading-to-1000024.3-step-016</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-version" class="anchored"><code>version</code></div></td><td>version</td><td><code>1000024.2-upgrading-to-1000024.3-step-020</code></td><td>set the active cluster version in the format &#39;&lt;major&gt;.&lt;minor&gt;&#39;</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
</tbody>
</table>
10 changes: 9 additions & 1 deletion pkg/ccl/backupccl/alter_backup_schedule.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
pbtypes "github.com/gogo/protobuf/types"
)
Expand Down Expand Up @@ -464,6 +465,13 @@ func processFullBackupRecurrence(
s.incStmt = &tree.Backup{}
*s.incStmt = *s.fullStmt
s.incStmt.AppendToLatest = true
// Pre 23.2 schedules did not have a cluster ID, so if we are altering a
// schedule that was created before 23.2, we need to set the cluster ID on
// the newly created incremental manually.
schedDetails := *s.fullJob.ScheduleDetails()
if schedDetails.ClusterID.Equal(uuid.Nil) {
schedDetails.ClusterID = p.ExtendedEvalContext().ClusterID
}

rec := s.fullJob.ScheduleExpr()
incRecurrence, err := schedulebase.ComputeScheduleRecurrence(env.Now(), &rec)
Expand All @@ -476,7 +484,7 @@ func processFullBackupRecurrence(
p.User(),
s.fullJob.ScheduleLabel(),
incRecurrence,
*s.fullJob.ScheduleDetails(),
schedDetails,
jobspb.InvalidScheduleID,
s.fullArgs.UpdatesLastBackupMetric,
s.incStmt,
Expand Down
37 changes: 37 additions & 0 deletions pkg/ccl/backupccl/alter_backup_schedule_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,5 +158,42 @@ INSERT INTO t1 values (1), (10), (100);

rows = th.sqlDB.QueryStr(t, fmt.Sprintf(`ALTER BACKUP SCHEDULE %d EXECUTE IMMEDIATELY;`, scheduleID))
require.Equal(t, trim(th.env.Now().String()), trim(rows[0][3]))
}

func TestAlterBackupScheduleSetsIncrementalClusterID(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

th, cleanup := newAlterSchedulesTestHelper(t, nil)
defer cleanup()

rows := th.sqlDB.QueryStr(
t,
`CREATE SCHEDULE FOR BACKUP INTO 'nodelocal://1/backup/alter-schedule' RECURRING '@daily' FULL BACKUP ALWAYS;`,
)
require.Len(t, rows, 1)
scheduleID, err := strconv.Atoi(rows[0][0])
require.NoError(t, err)

// Artificially remove cluster ID from full backup to simulate pre-23.2 schedule.
th.sqlDB.QueryStr(
t,
fmt.Sprintf(`UPDATE system.scheduled_jobs
SET
schedule_details = crdb_internal.json_to_pb(
'cockroach.jobs.jobspb.ScheduleDetails',
json_remove_path(
crdb_internal.pb_to_json('cockroach.jobs.jobspb.ScheduleDetails', schedule_details),
ARRAY['clusterId']
)
)
WHERE schedule_id=%d;`, scheduleID),
)

// Ensure creating incremental from a full backup schedule without a cluster ID passes
rows = th.sqlDB.QueryStr(t, fmt.Sprintf(
`ALTER BACKUP SCHEDULE %d SET RECURRING '@hourly', SET FULL BACKUP '@daily'`,
scheduleID),
)
require.Len(t, rows, 2)
}
11 changes: 11 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,15 @@ const (
// policies.
V24_3_MaybePreventUpgradeForCoreLicenseDeprecation

// V24_3_UseRACV2WithV1EntryEncoding is the earliest version which supports
// ranges using replication flow control v2, still with v1 entry encoding.
V24_3_UseRACV2WithV1EntryEncoding

// V24_3_UseRACV2Full is the earliest version which supports ranges using
// replication flow control v2, with v2 entry encoding. Replication flow
// control v1 is unsupported at this version.
V24_3_UseRACV2Full

// *************************************************
// Step (1) Add new versions above this comment.
// Do not add new versions to a patch release.
Expand Down Expand Up @@ -302,6 +311,8 @@ var versionTable = [numKeys]roachpb.Version{
V24_3_AdvanceCommitIndexViaMsgApps: {Major: 24, Minor: 2, Internal: 12},
V24_3_SQLInstancesAddDraining: {Major: 24, Minor: 2, Internal: 14},
V24_3_MaybePreventUpgradeForCoreLicenseDeprecation: {Major: 24, Minor: 2, Internal: 16},
V24_3_UseRACV2WithV1EntryEncoding: {Major: 24, Minor: 2, Internal: 18},
V24_3_UseRACV2Full: {Major: 24, Minor: 2, Internal: 20},

// *************************************************
// Step (2): Add new versions above this comment.
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/flow_control_stores.go
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ func (sh *storeForFlowControl) LookupReplicationAdmissionHandle(
}
// NB: Admit is called soon after this lookup.
level := repl.flowControlV2.GetEnabledWhenLeader()
useV1 := level == replica_rac2.NotEnabledWhenLeader
useV1 := level == kvflowcontrol.V2NotEnabledWhenLeader
var v1Handle kvflowcontrol.ReplicationAdmissionHandle
if useV1 {
repl.mu.Lock()
Expand Down Expand Up @@ -453,7 +453,7 @@ func (h admissionDemuxHandle) Admit(
// can cause either value of admitted. See the comment in
// ReplicationAdmissionHandle.
level := h.r.flowControlV2.GetEnabledWhenLeader()
if level == replica_rac2.NotEnabledWhenLeader {
if level == kvflowcontrol.V2NotEnabledWhenLeader {
return admitted, err
}
// Transition from v1 => v2 happened while waiting. Fall through to wait
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/kvflowcontrol/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_library(
"//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb",
"//pkg/roachpb",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/util/admission/admissionpb",
"//pkg/util/metamorphic",
"@com_github_cockroachdb_redact//:redact",
Expand Down
53 changes: 53 additions & 0 deletions pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb"
"github.com/cockroachdb/cockroach/pkg/util/metamorphic"
"github.com/cockroachdb/redact"
Expand Down Expand Up @@ -118,6 +119,58 @@ var validateTokenRange = settings.WithValidateInt(func(b int64) error {
return nil
})

// V2EnabledWhenLeaderLevel captures the level at which RACv2 is enabled when
// this replica is the leader.
//
// State transitions are V2NotEnabledWhenLeader =>
// V2EnabledWhenLeaderV1Encoding => V2EnabledWhenLeaderV2Encoding, i.e., the
// level will never regress.
type V2EnabledWhenLeaderLevel = uint32

const (
V2NotEnabledWhenLeader V2EnabledWhenLeaderLevel = iota
V2EnabledWhenLeaderV1Encoding
V2EnabledWhenLeaderV2Encoding
)

// GetV2EnabledWhenLeaderLevel returns the level at which RACV2 is enabled when
// this replica is the leader.
//
// The level is determined by the cluster version, and is ratcheted up as the
// cluster version advances. The level is used to determine:
//
// 1. Whether the leader should use the RACv2 protocol.
// 2. Whether the leader should use the V1 or V2 entry encoding iff (1) is
// true.
//
// Upon the leader first seeing V24_3_UseRACV2WithV1EntryEncoding, it will
// create a RangeController and use the V1 entry encoding, operating in Push
// mode. Upon the leader first seeing V24_3_UseRACV2Full, it will continue
// using the RACV2 protocol, but will switch to the V2 entry encoding. Note the
// necessary migration for V2NotEnabledWhenLeader =>
// V2EnabledWhenLeaderV1Encoding occurs before anything else in
// kvserver.handleRaftReadyRaftMuLocked.
//
// TODO(kvoli,sumeerbhola,pav-kv): When we introduce pull mode (and associated
// cluster setting), update this comment to mention that the cluster setting is
// only relevant when at V2EnabledWhenLeaderV2Encoding level.
func GetV2EnabledWhenLeaderLevel(
ctx context.Context, st *cluster.Settings, knobs *TestingKnobs,
) V2EnabledWhenLeaderLevel {
if knobs != nil && knobs.OverrideV2EnabledWhenLeaderLevel != nil {
return knobs.OverrideV2EnabledWhenLeaderLevel()
}
// TODO(kvoli): Enable once #130619 merges and tests affected by enabling v2
// are addressed:
// if st.Version.IsActive(ctx, clusterversion.V24_3_UseRACV2Full) {
// return V2EnabledWhenLeaderV2Encoding
// }
// if st.Version.IsActive(ctx, clusterversion.V24_3_UseRACV2WithV1EntryEncoding) {
// return V2EnabledWhenLeaderV1Encoding
// }
return V2NotEnabledWhenLeader
}

// Stream models the stream over which we replicate data traffic, the
// transmission for which we regulate using flow control. It's segmented by the
// specific store the traffic is bound for and the tenant driving it. Despite
Expand Down
30 changes: 9 additions & 21 deletions pkg/kv/kvserver/kvflowcontrol/replica_rac2/processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -183,19 +183,6 @@ type RangeControllerFactory interface {
New(ctx context.Context, state rangeControllerInitState) rac2.RangeController
}

// EnabledWhenLeaderLevel captures the level at which RACv2 is enabled when
// this replica is the leader.
//
// State transitions are NotEnabledWhenLeader => EnabledWhenLeaderV1Encoding
// => EnabledWhenLeaderV2Encoding, i.e., the level will never regress.
type EnabledWhenLeaderLevel = uint32

const (
NotEnabledWhenLeader EnabledWhenLeaderLevel = iota
EnabledWhenLeaderV1Encoding
EnabledWhenLeaderV2Encoding
)

// ProcessorOptions are specified when creating a new Processor.
type ProcessorOptions struct {
// Various constant fields that are duplicated from Replica, since we
Expand All @@ -216,7 +203,7 @@ type ProcessorOptions struct {
Settings *cluster.Settings
EvalWaitMetrics *rac2.EvalWaitMetrics

EnabledWhenLeaderLevel EnabledWhenLeaderLevel
EnabledWhenLeaderLevel kvflowcontrol.V2EnabledWhenLeaderLevel
Knobs *kvflowcontrol.TestingKnobs
}

Expand Down Expand Up @@ -289,14 +276,14 @@ type Processor interface {
// This may be a noop if the level has already been reached.
//
// raftMu is held.
SetEnabledWhenLeaderRaftMuLocked(ctx context.Context, level EnabledWhenLeaderLevel)
SetEnabledWhenLeaderRaftMuLocked(ctx context.Context, level kvflowcontrol.V2EnabledWhenLeaderLevel)
// GetEnabledWhenLeader returns the current level. It may be used in
// highly concurrent settings at the leaseholder, when waiting for eval,
// and when encoding a proposal. Note that if the leaseholder is not the
// leader and the leader has switched to a higher level, there is no harm
// done, since the leaseholder can continue waiting for v1 tokens and use
// the v1 entry encoding.
GetEnabledWhenLeader() EnabledWhenLeaderLevel
GetEnabledWhenLeader() kvflowcontrol.V2EnabledWhenLeaderLevel

// OnDescChangedLocked provides a possibly updated RangeDescriptor. The
// tenantID passed in all calls must be the same.
Expand Down Expand Up @@ -502,7 +489,7 @@ type processorImpl struct {
// enabledWhenLeader indicates the RACv2 mode of operation when this replica
// is the leader. Atomic value, for serving GetEnabledWhenLeader. Updated only
// while holding raftMu. Can be read non-atomically if raftMu is held.
enabledWhenLeader EnabledWhenLeaderLevel
enabledWhenLeader kvflowcontrol.V2EnabledWhenLeaderLevel

v1EncodingPriorityMismatch log.EveryN
}
Expand Down Expand Up @@ -547,14 +534,15 @@ func (p *processorImpl) OnDestroyRaftMuLocked(ctx context.Context) {

// SetEnabledWhenLeaderRaftMuLocked implements Processor.
func (p *processorImpl) SetEnabledWhenLeaderRaftMuLocked(
ctx context.Context, level EnabledWhenLeaderLevel,
ctx context.Context, level kvflowcontrol.V2EnabledWhenLeaderLevel,
) {
p.opts.Replica.RaftMuAssertHeld()
if p.destroyed || p.enabledWhenLeader >= level {
return
}
atomic.StoreUint32(&p.enabledWhenLeader, level)
if level != EnabledWhenLeaderV1Encoding || p.desc.replicas == nil {
if level != kvflowcontrol.V2EnabledWhenLeaderV1Encoding ||
p.desc.replicas == nil {
return
}
// May need to create RangeController.
Expand All @@ -576,7 +564,7 @@ func (p *processorImpl) SetEnabledWhenLeaderRaftMuLocked(
}

// GetEnabledWhenLeader implements Processor.
func (p *processorImpl) GetEnabledWhenLeader() EnabledWhenLeaderLevel {
func (p *processorImpl) GetEnabledWhenLeader() kvflowcontrol.V2EnabledWhenLeaderLevel {
return atomic.LoadUint32(&p.enabledWhenLeader)
}

Expand Down Expand Up @@ -693,7 +681,7 @@ func (p *processorImpl) makeStateConsistentRaftMuLocked(
return
}
// Is the leader.
if p.enabledWhenLeader == NotEnabledWhenLeader {
if p.enabledWhenLeader == kvflowcontrol.V2NotEnabledWhenLeader {
return
}
if p.leader.rc != nil && termChanged {
Expand Down
22 changes: 12 additions & 10 deletions pkg/kv/kvserver/kvflowcontrol/replica_rac2/processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -281,7 +281,7 @@ func TestProcessorBasic(t *testing.T) {
var st *cluster.Settings
var p *processorImpl
tenantID := roachpb.MustMakeTenantID(4)
reset := func(enabled EnabledWhenLeaderLevel) {
reset := func(enabled kvflowcontrol.V2EnabledWhenLeaderLevel) {
b.Reset()
r = newTestReplica(&b)
sched = testRaftScheduler{b: &b}
Expand Down Expand Up @@ -533,31 +533,33 @@ func parseAdmissionPriority(t *testing.T, td *datadriven.TestData) admissionpb.W
return admissionpb.NormalPri
}

func parseEnabledLevel(t *testing.T, td *datadriven.TestData) EnabledWhenLeaderLevel {
func parseEnabledLevel(
t *testing.T, td *datadriven.TestData,
) kvflowcontrol.V2EnabledWhenLeaderLevel {
if td.HasArg("enabled-level") {
var str string
td.ScanArgs(t, "enabled-level", &str)
switch str {
case "not-enabled":
return NotEnabledWhenLeader
return kvflowcontrol.V2NotEnabledWhenLeader
case "v1-encoding":
return EnabledWhenLeaderV1Encoding
return kvflowcontrol.V2EnabledWhenLeaderV1Encoding
case "v2-encoding":
return EnabledWhenLeaderV2Encoding
return kvflowcontrol.V2EnabledWhenLeaderV2Encoding
default:
t.Fatalf("unrecoginized level %s", str)
}
}
return NotEnabledWhenLeader
return kvflowcontrol.V2NotEnabledWhenLeader
}

func enabledLevelString(enabledLevel EnabledWhenLeaderLevel) string {
func enabledLevelString(enabledLevel kvflowcontrol.V2EnabledWhenLeaderLevel) string {
switch enabledLevel {
case NotEnabledWhenLeader:
case kvflowcontrol.V2NotEnabledWhenLeader:
return "not-enabled"
case EnabledWhenLeaderV1Encoding:
case kvflowcontrol.V2EnabledWhenLeaderV1Encoding:
return "v1-encoding"
case EnabledWhenLeaderV2Encoding:
case kvflowcontrol.V2EnabledWhenLeaderV2Encoding:
return "v2-encoding"
}
return "unknown-level"
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/kvflowcontrol/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@ type TestingKnobs struct {
// OverrideTokenDeduction is used to override how many tokens are deducted
// post-evaluation.
OverrideTokenDeduction func() Tokens
// OverrideV2EnabledWhenLeaderLevel is used to override the level at which
// RACv2 is enabled when a replica is the leader.
OverrideV2EnabledWhenLeaderLevel func() V2EnabledWhenLeaderLevel
}

// TestingKnobsV1 are the testing knobs that appply to replication flow control
Expand Down
10 changes: 2 additions & 8 deletions pkg/kv/kvserver/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/replica_rac2"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
Expand Down Expand Up @@ -327,7 +328,7 @@ type Replica struct {
// being applied to the state machine.
bytesAccount logstore.BytesAccount

flowControlLevel replica_rac2.EnabledWhenLeaderLevel
flowControlLevel kvflowcontrol.V2EnabledWhenLeaderLevel

// Scratch for populating RaftEvent for flowControlV2.
msgAppScratchForFlowControl map[roachpb.ReplicaID][]raftpb.Message
Expand Down Expand Up @@ -2526,13 +2527,6 @@ func (r *Replica) GetMutexForTesting() *ReplicaMutex {
return &r.mu.ReplicaMutex
}

func racV2EnabledWhenLeaderLevel(
ctx context.Context, st *cluster.Settings,
) replica_rac2.EnabledWhenLeaderLevel {
// TODO(sumeer): implement fully, once all the dependencies are implemented.
return replica_rac2.NotEnabledWhenLeader
}

// maybeEnqueueProblemRange will enqueue the replica for processing into the
// replicate queue iff:
//
Expand Down
Loading

0 comments on commit 311c0d5

Please sign in to comment.