github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/cmd/github-pull-request-make/testdata/27595.diff (about) 1 diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html 2 index 90ebcf30ab1..46f33476c42 100644 3 --- a/docs/generated/settings/settings.html 4 +++ b/docs/generated/settings/settings.html 5 @@ -37,6 +37,8 @@ 6 <tr><td><code>rocksdb.min_wal_sync_interval</code></td><td>duration</td><td><code>0s</code></td><td>minimum duration between syncs of the RocksDB WAL</td></tr> 7 <tr><td><code>server.clock.forward_jump_check_enabled</code></td><td>boolean</td><td><code>false</code></td><td>if enabled, forward clock jumps > max_offset/2 will cause a panic.</td></tr> 8 <tr><td><code>server.clock.persist_upper_bound_interval</code></td><td>duration</td><td><code>0s</code></td><td>the interval between persisting the wall time upper bound of the clock. The clock does not generate a wall time greater than the persisted timestamp and will panic if it sees a wall time greater than this value. When cockroach starts, it waits for the wall time to catch-up till this persisted timestamp. This guarantees monotonic wall time across server restarts. Not setting this or setting a value of 0 disables this feature.</td></tr> 9 +<tr><td><code>server.closed_timestamp.close_fraction</code></td><td>float</td><td><code>0.2</code></td><td>desc</td></tr> 10 +<tr><td><code>server.closed_timestamp.target_duration</code></td><td>duration</td><td><code>5s</code></td><td>if nonzero, attempt to provide closed timestamp notifications for timestamps trailing cluster time by approximately this duration</td></tr> 11 <tr><td><code>server.consistency_check.interval</code></td><td>duration</td><td><code>24h0m0s</code></td><td>the time between range consistency checks; set to 0 to disable consistency checking</td></tr> 12 <tr><td><code>server.declined_reservation_timeout</code></td><td>duration</td><td><code>1s</code></td><td>the amount of time to consider the store throttled for up-replication after a reservation was declined</td></tr> 13 <tr><td><code>server.failed_reservation_timeout</code></td><td>duration</td><td><code>5s</code></td><td>the amount of time to consider the store throttled for up-replication after a failed reservation call</td></tr> 14 diff --git a/pkg/storage/closedts/closedts.go b/pkg/storage/closedts/closedts.go 15 new file mode 100644 16 index 00000000000..5abab028897 17 --- /dev/null 18 +++ b/pkg/storage/closedts/closedts.go 19 @@ -0,0 +1,160 @@ 20 +// Copyright 2018 The Cockroach Authors. 21 +// 22 +// Licensed under the Apache License, Version 2.0 (the "License"); 23 +// you may not use this file except in compliance with the License. 24 +// You may obtain a copy of the License at 25 +// 26 +// http://www.apache.org/licenses/LICENSE-2.0 27 +// 28 +// Unless required by applicable law or agreed to in writing, software 29 +// distributed under the License is distributed on an "AS IS" BASIS, 30 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 31 +// implied. See the License for the specific language governing 32 +// permissions and limitations under the License. 33 + 34 +// Package closedts houses the interfaces and basic definitions used by the 35 +// various components of the closed timestamp subsystems. 36 +// 37 +// The following diagram illustrates how these components fit together. In 38 +// running operation, the components are grouped in a container.Container 39 +// (intended as a pass-around per-instance Singleton). 40 +// Replicas proposing commands talk to the Tracker; replicas trying to serve 41 +// follower reads talk to the Provider, which receives closed timestamp updates 42 +// for the local node and its peers. 43 +// 44 +// Node 1 | Node 2 45 +// | 46 +// +---------+ Close +-----------+ | +-----------+ 47 +// | Tracker |<--------| | | | | 48 +// +-----+---+ | +-------+ | | | +-------+ | CanServe 49 +// ^ | |Storage| | | | |Storage| |<---------+ 50 +// | | --------+ | | | +-------+ | | 51 +// |Track | | | | | +----+----+ 52 +// | | Provider | | | Provider | | Follower| 53 +// | +-----------+ | +-----------+ | Replica | 54 +// | ^ ^ +----+----+ 55 +// | |Subscribe |Notify | 56 +// | | | | 57 +// +---------+ | Request | | 58 +// |Proposing| Refresh +---+----+ <------ +---+-----+ Request | 59 +// | Replica |<--------| Server | | Clients |<----------+ 60 +// +---------+ +--------+ ------> +---------+ EnsureClient 61 +// CT 62 +package closedts 63 + 64 +import ( 65 + "context" 66 + "fmt" 67 + 68 + "github.com/cockroachdb/cockroach/pkg/roachpb" 69 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 70 + "github.com/cockroachdb/cockroach/pkg/util/hlc" 71 +) 72 + 73 +// A Storage holds the closed timestamps and associated MLAIs for each node. It 74 +// additionally provides historical information about past state that it 75 +// "compacts" regularly, and which can be introspected via the VisitAscending 76 +// method. 77 +// 78 +// The data in a Storage is ephemeral, i.e. is lost during process restarts. 79 +// Introducing a persistent storage will require some design work to make 80 +// sure a) that the records in the storage are certifiably up to date (they 81 +// won't be naturally, unless we add a synchronous write to each proposal) 82 +// and b) that the proposal at each MLAI has actually been proposed. It's 83 +// unlikely that we'll ever find it useful to introduce persistence here 84 +// (though we want to persist historical information for recovery after 85 +// permanent loss of quorum, but there we only need some consistent on- 86 +// disk state; we don't need to bootstrap it into a new consistent state 87 +// that can be updated incrementally). 88 +type Storage interface { 89 + fmt.Stringer 90 + // VisitAscending visits the historical states contained within the Storage 91 + // in ascending closed timestamp order. Each state (Entry) is full, i.e. 92 + // non-incremental. The iteration stops when all states have been visited 93 + // or the visitor returns true. 94 + VisitAscending(roachpb.NodeID, func(ctpb.Entry) (done bool)) 95 + // VisitDescending visits the historical states contained within the Storage 96 + // in descending closed timestamp order. Each state (Entry) is full, i.e. 97 + // non-incremental. The iteration stops when all states have been visited 98 + // or the visitor returns true. 99 + VisitDescending(roachpb.NodeID, func(ctpb.Entry) (done bool)) 100 + // Add merges the given Entry into the state for the given NodeID. The first 101 + // Entry passed in for any given Entry.Epoch must have Entry.Full set. 102 + Add(roachpb.NodeID, ctpb.Entry) 103 +} 104 + 105 +// A Notifyee is a sink for closed timestamp updates. 106 +type Notifyee interface { 107 + // Notify returns a channel into which updates are written. 108 + // 109 + // In practice, the Notifyee will be a Provider. 110 + Notify(roachpb.NodeID) chan<- ctpb.Entry 111 +} 112 + 113 +// A Producer is a source of closed timestamp updates about the local node. 114 +type Producer interface { 115 + // The Subscribe method blocks and, until the context cancels, writes a 116 + // stream of updates to the provided channel the aggregate of which is 117 + // guaranteed to represent a valid (i.e. gapless) state. 118 + Subscribe(context.Context, chan<- ctpb.Entry) 119 +} 120 + 121 +// Provider is the central coordinator in the closed timestamp subsystem and the 122 +// gatekeeper for the closed timestamp state for both local and remote nodes, 123 +// which it handles in a symmetric fashion. It has the following tasks: 124 +// 125 +// 1. it accepts subscriptions for closed timestamp updates sourced from the 126 +// local node. Upon accepting a subscription, the subscriber first receives 127 +// the aggregate closed timestamp snapshot of the local node and then periodic 128 +// updates. 129 +// 2. it periodically closes out timestamps on the local node and passes the 130 +// resulting entries to all of its subscribers. 131 +// 3. it accepts notifications from other nodes, passing these updates through 132 +// to its local storage, so that 133 +// 4. the CanServe method determines via the the underlying storage whether a 134 +// given read can be satisfied via follower reads. 135 +// 136 +// Note that a Provider has no duty to immediately persist the local closed 137 +// timestamps to the underlying storage. 138 +type Provider interface { 139 + Producer 140 + Notifyee 141 + Start() 142 + CanServe(roachpb.NodeID, hlc.Timestamp, roachpb.RangeID, ctpb.Epoch, ctpb.LAI) bool 143 +} 144 + 145 +// A ClientRegistry is the client component of the follower reads subsystem. It 146 +// contacts other nodes and requests a continuous stream of closed timestamp 147 +// updates which it relays to the Provider. 148 +type ClientRegistry interface { 149 + // Request asynchronously notifies the given node that an update should be 150 + // emitted for the given range. 151 + Request(roachpb.NodeID, roachpb.RangeID) 152 + // EnsureClient instructs the registry to (asynchronously) request a stream 153 + // of closed timestamp updates from the given node. 154 + EnsureClient(roachpb.NodeID) 155 +} 156 + 157 +// CloseFn is periodically called by Producers to close out new timestamps. 158 +// Outside of tests, it corresponds to (*Tracker).Close. 159 +type CloseFn func(next hlc.Timestamp) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI) 160 + 161 +// LiveClockFn supplies a current HLC timestamp from the local node with the 162 +// extra constraints that the local node is live for the returned timestamp at 163 +// the given epoch. 164 +type LiveClockFn func() (liveNow hlc.Timestamp, liveEpoch ctpb.Epoch, _ error) 165 + 166 +// RefreshFn is called by the Producer when it is asked to manually create (and 167 +// emit) an update for a number of its replicas. The closed timestamp subsystem 168 +// intentionally knows as little about the outside world as possible, and this 169 +// function, injected from the outside, provides the minimal glue. Its job is 170 +// to register a proposal for the current lease applied indexes of the replicas 171 +// with the Tracker, so that updates for them are emitted soon thereafter. 172 +type RefreshFn func(...roachpb.RangeID) 173 + 174 +// A Dialer opens closed timestamp connections to receive updates from remote 175 +// nodes. 176 +type Dialer interface { 177 + Dial(context.Context, roachpb.NodeID) (ctpb.Client, error) 178 + Ready(roachpb.NodeID) bool // if false, Dial is likely to fail 179 +} 180 diff --git a/pkg/storage/closedts/container/container.go b/pkg/storage/closedts/container/container.go 181 new file mode 100644 182 index 00000000000..28f3e818ea3 183 --- /dev/null 184 +++ b/pkg/storage/closedts/container/container.go 185 @@ -0,0 +1,142 @@ 186 +// Copyright 2018 The Cockroach Authors. 187 +// 188 +// Licensed under the Apache License, Version 2.0 (the "License"); 189 +// you may not use this file except in compliance with the License. 190 +// You may obtain a copy of the License at 191 +// 192 +// http://www.apache.org/licenses/LICENSE-2.0 193 +// 194 +// Unless required by applicable law or agreed to in writing, software 195 +// distributed under the License is distributed on an "AS IS" BASIS, 196 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 197 +// implied. See the License for the specific language governing 198 +// permissions and limitations under the License. 199 + 200 +package container 201 + 202 +import ( 203 + "context" 204 + "time" 205 + 206 + "github.com/cockroachdb/cockroach/pkg/roachpb" 207 + "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" 208 + "github.com/cockroachdb/cockroach/pkg/settings/cluster" 209 + "github.com/cockroachdb/cockroach/pkg/storage/closedts" 210 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 211 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/minprop" 212 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/provider" 213 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/storage" 214 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/transport" 215 + "github.com/cockroachdb/cockroach/pkg/util/stop" 216 + 217 + "google.golang.org/grpc" 218 +) 219 + 220 +// Config is a container that holds references to all of the components required 221 +// to set up a full closed timestamp subsystem. 222 +type Config struct { 223 + NodeID roachpb.NodeID 224 + Settings *cluster.Settings 225 + Stopper *stop.Stopper 226 + Clock closedts.LiveClockFn 227 + Refresh closedts.RefreshFn 228 + Dialer closedts.Dialer 229 + 230 + // GRPCServer specifies an (optional) server to register the CT update server 231 + // with. 232 + GRPCServer *grpc.Server 233 +} 234 + 235 +// A Container is a full closed timestamp subsystem along with the Config it was 236 +// created from. 237 +type Container struct { 238 + Config 239 + Tracker *minprop.Tracker 240 + Storage closedts.Storage 241 + Provider closedts.Provider 242 + Server ctpb.Server 243 + Clients closedts.ClientRegistry 244 +} 245 + 246 +const ( 247 + // For each node, keep two historical buckets (i.e. one recent one, and one that 248 + // lagging followers can still satisfy some reads from). 249 + storageBucketNum = 2 250 + // StorageBucketScale determines the (exponential) spacing of storage buckets. 251 + // For example, a scale of 5s means that the second bucket will attempt to hold 252 + // a closed timestamp 5s in the past from the first, and the third 5*5=25s from 253 + // the first, etc. 254 + // 255 + // TODO(tschottdorf): it's straightforward to make this dynamic. It should track 256 + // the interval at which timestamps are closed out, ideally being a little shorter. 257 + // The effect of that would be that the most recent closed timestamp and the previous 258 + // one can be queried against separately. 259 + StorageBucketScale = 10 * time.Second 260 +) 261 + 262 +// NewContainer initializes a Container from the given Config. The Container 263 +// will need to be started separately. 264 +func NewContainer(cfg Config) *Container { 265 + storage := storage.NewMultiStorage(func() storage.SingleStorage { 266 + return storage.NewMemStorage(StorageBucketScale, storageBucketNum) 267 + }) 268 + 269 + tracker := minprop.NewTracker() 270 + 271 + pConf := provider.Config{ 272 + NodeID: cfg.NodeID, 273 + Settings: cfg.Settings, 274 + Stopper: cfg.Stopper, 275 + Storage: storage, 276 + Clock: cfg.Clock, 277 + Close: tracker.CloseFn(), 278 + } 279 + provider := provider.NewProvider(&pConf) 280 + 281 + server := transport.NewServer(cfg.Stopper, provider, cfg.Refresh) 282 + 283 + if cfg.GRPCServer != nil { 284 + ctpb.RegisterClosedTimestampServer(cfg.GRPCServer, ctpb.ServerShim{Server: server}) 285 + } 286 + 287 + rConf := transport.Config{ 288 + Settings: cfg.Settings, 289 + Stopper: cfg.Stopper, 290 + Dialer: cfg.Dialer, 291 + Sink: provider, 292 + } 293 + 294 + return &Container{ 295 + Config: cfg, 296 + Storage: storage, 297 + Provider: provider, 298 + Tracker: tracker, 299 + Server: server, 300 + Clients: transport.NewClients(rConf), 301 + } 302 +} 303 + 304 +// Start starts the Container. The Stopper used to create the Container is in 305 +// charge of stopping it. 306 +func (c *Container) Start() { 307 + c.Provider.Start() 308 +} 309 + 310 +type dialerAdapter nodedialer.Dialer 311 + 312 +func (da *dialerAdapter) Ready(nodeID roachpb.NodeID) bool { 313 + return (*nodedialer.Dialer)(da).GetCircuitBreaker(nodeID).Ready() 314 +} 315 + 316 +func (da *dialerAdapter) Dial(ctx context.Context, nodeID roachpb.NodeID) (ctpb.Client, error) { 317 + c, err := (*nodedialer.Dialer)(da).Dial(ctx, nodeID) 318 + if err != nil { 319 + return nil, err 320 + } 321 + return ctpb.NewClosedTimestampClient(c).Get(ctx) 322 +} 323 + 324 +// DialerAdapter turns a node dialer into a closedts.Dialer. 325 +func DialerAdapter(dialer *nodedialer.Dialer) closedts.Dialer { 326 + return (*dialerAdapter)(dialer) 327 +} 328 diff --git a/pkg/storage/closedts/container/container_test.go b/pkg/storage/closedts/container/container_test.go 329 new file mode 100644 330 index 00000000000..714cb7245c1 331 --- /dev/null 332 +++ b/pkg/storage/closedts/container/container_test.go 333 @@ -0,0 +1,379 @@ 334 +// Copyright 2018 The Cockroach Authors. 335 +// 336 +// Licensed under the Apache License, Version 2.0 (the "License"); 337 +// you may not use this file except in compliance with the License. 338 +// You may obtain a copy of the License at 339 +// 340 +// http://www.apache.org/licenses/LICENSE-2.0 341 +// 342 +// Unless required by applicable law or agreed to in writing, software 343 +// distributed under the License is distributed on an "AS IS" BASIS, 344 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 345 +// implied. See the License for the specific language governing 346 +// permissions and limitations under the License. 347 + 348 +package container_test // intentionally test from external package 349 + 350 +import ( 351 + "context" 352 + "reflect" 353 + "sync" 354 + "testing" 355 + "time" 356 + 357 + "github.com/kr/pretty" 358 + "github.com/pkg/errors" 359 + "github.com/stretchr/testify/require" 360 + 361 + "github.com/cockroachdb/cockroach/pkg/roachpb" 362 + "github.com/cockroachdb/cockroach/pkg/settings/cluster" 363 + "github.com/cockroachdb/cockroach/pkg/storage/closedts" 364 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/container" 365 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 366 + transporttestutils "github.com/cockroachdb/cockroach/pkg/storage/closedts/transport/testutils" 367 + "github.com/cockroachdb/cockroach/pkg/testutils" 368 + "github.com/cockroachdb/cockroach/pkg/util/hlc" 369 + "github.com/cockroachdb/cockroach/pkg/util/leaktest" 370 + "github.com/cockroachdb/cockroach/pkg/util/stop" 371 + "github.com/cockroachdb/cockroach/pkg/util/syncutil" 372 +) 373 + 374 +type Tick struct { 375 + liveNow hlc.Timestamp 376 + liveEpoch ctpb.Epoch 377 + err error 378 +} 379 + 380 +type LateBoundDialer struct { 381 + Wrapped *transporttestutils.ChanDialer 382 +} 383 + 384 +func (d *LateBoundDialer) Dial(ctx context.Context, nodeID roachpb.NodeID) (ctpb.Client, error) { 385 + return d.Wrapped.Dial(ctx, nodeID) 386 +} 387 + 388 +func (d *LateBoundDialer) Ready(nodeID roachpb.NodeID) bool { 389 + return d.Wrapped.Ready(nodeID) 390 +} 391 + 392 +type TestContainer struct { 393 + *container.Container 394 + Refreshed struct { 395 + syncutil.Mutex 396 + RangeIDs []roachpb.RangeID 397 + } 398 + Dialer *LateBoundDialer 399 + TestClock *TestClock 400 +} 401 + 402 +type TestClock struct { 403 + stopper *stop.Stopper 404 + ch chan Tick 405 +} 406 + 407 +func NewTestClock(stopper *stop.Stopper) *TestClock { 408 + t := &TestClock{ 409 + stopper: stopper, 410 + ch: make(chan Tick), 411 + } 412 + return t 413 +} 414 + 415 +func (c *TestClock) Tick(liveNow hlc.Timestamp, liveEpoch ctpb.Epoch, err error) { 416 + c.ch <- Tick{liveNow, liveEpoch, err} 417 +} 418 + 419 +func (c *TestClock) LiveNow() (liveNow hlc.Timestamp, liveEpoch ctpb.Epoch, _ error) { 420 + select { 421 + case r := <-c.ch: 422 + return r.liveNow, r.liveEpoch, r.err 423 + case <-c.stopper.ShouldQuiesce(): 424 + return hlc.Timestamp{}, 0, errors.New("quiescing") 425 + } 426 +} 427 + 428 +func prepareContainer(nodeID roachpb.NodeID) *TestContainer { 429 + stopper := stop.NewStopper() 430 + 431 + tc := &TestContainer{} 432 + 433 + tc.TestClock = NewTestClock(stopper) 434 + 435 + var wg sync.WaitGroup 436 + wg.Add(1) 437 + refresh := func(requested ...roachpb.RangeID) { 438 + tc.Refreshed.Lock() 439 + tc.Refreshed.RangeIDs = append(tc.Refreshed.RangeIDs, requested...) 440 + tc.Refreshed.Unlock() 441 + } 442 + 443 + st := cluster.MakeTestingClusterSettings() 444 + 445 + // Set the target duration to a second and the close fraction so small 446 + // that the Provider will essentially close in a hot loop. In this test 447 + // we'll block in the clock to pace the Provider's closer loop. 448 + closedts.TargetDuration.Override(&st.SV, time.Second) 449 + closedts.CloseFraction.Override(&st.SV, 1E-9) 450 + 451 + // We perform a little dance with the Dialer. It needs to be hooked up to the 452 + // Server, but that's only created in NewContainer. The Dialer isn't used until 453 + // that point, so we just create it a little later. 454 + tc.Dialer = &LateBoundDialer{} 455 + 456 + cfg := container.Config{ 457 + Settings: st, 458 + NodeID: nodeID, 459 + Stopper: stopper, 460 + Clock: tc.TestClock.LiveNow, 461 + Refresh: refresh, 462 + Dialer: tc.Dialer, 463 + } 464 + 465 + tc.Container = container.NewContainer(cfg) 466 + return tc 467 +} 468 + 469 +func setupTwoNodeTest() (_ *TestContainer, _ *TestContainer, shutdown func()) { 470 + c1 := prepareContainer(roachpb.NodeID(1)) 471 + c2 := prepareContainer(roachpb.NodeID(2)) 472 + 473 + c1.Dialer.Wrapped = transporttestutils.NewChanDialer(c1.Stopper, c2.Server) 474 + c2.Dialer.Wrapped = transporttestutils.NewChanDialer(c2.Stopper, c1.Server) 475 + 476 + c1.Start() 477 + c2.Start() 478 + 479 + return c1, c2, func() { 480 + // Oh, the joy of multiple stoppers. 481 + var wg sync.WaitGroup 482 + wg.Add(2) 483 + go func() { 484 + defer wg.Done() 485 + c1.Stopper.Stop(context.Background()) 486 + }() 487 + go func() { 488 + defer wg.Done() 489 + c2.Stopper.Stop(context.Background()) 490 + }() 491 + } 492 +} 493 + 494 +func TestTwoNodes(t *testing.T) { 495 + defer leaktest.AfterTest(t)() 496 + 497 + ctx := context.Background() 498 + 499 + c1, c2, shutdown := setupTwoNodeTest() 500 + defer shutdown() 501 + defer func() { 502 + t.Logf("n1 -> n2: %s", pretty.Sprint(c1.Dialer.Wrapped.Transcript(c2.NodeID))) 503 + t.Logf("n2 -> n1: %s", pretty.Sprint(c2.Dialer.Wrapped.Transcript(c1.NodeID))) 504 + }() 505 + 506 + // Initially, can't serve random things for either n1 or n2. 507 + require.False(t, c1.Container.Provider.CanServe( 508 + c1.NodeID, hlc.Timestamp{}, roachpb.RangeID(5), ctpb.Epoch(0), ctpb.LAI(0)), 509 + ) 510 + require.False(t, c1.Container.Provider.CanServe( 511 + c2.NodeID, hlc.Timestamp{}, roachpb.RangeID(5), ctpb.Epoch(0), ctpb.LAI(0)), 512 + ) 513 + 514 + // Track and release a command. 515 + ts, release := c1.Tracker.Track(ctx) 516 + release(ctx, roachpb.RangeID(17), ctpb.LAI(12)) 517 + 518 + // The command is forced above ts=0.2. This is just an artifact of how the 519 + // Tracker is implemented - it closes out 0.1 first, so it begins by forcing 520 + // commands just above that. 521 + require.Equal(t, hlc.Timestamp{Logical: 2}, ts) 522 + 523 + // The clock gives a timestamp to the Provider, which should close out the 524 + // current timestamp and set up 2E9-1E9=1E9 as the next one it wants to close. 525 + // We do this twice (for the same timestamp) to make sure that the Provider 526 + // not only read the tick, but also processed it. Otherwise, it becomes hard 527 + // to write the remainder of the test because the commands we track below may 528 + // fall into either case, and may be forced above the old or new timestamp. 529 + for i := 0; i < 2; i++ { 530 + c1.TestClock.Tick(hlc.Timestamp{WallTime: 2E9}, ctpb.Epoch(1), nil) 531 + } 532 + 533 + // The Tracker still won't let us serve anything, even though it has closed out 534 + // 0.1 - this is because it has no information about any ranges at that timestamp. 535 + // (Note that the Tracker may not have processed the closing yet, so if there were 536 + // a bug here, this test would fail flakily - that's ok). 537 + require.False(t, c1.Container.Provider.CanServe( 538 + c1.NodeID, hlc.Timestamp{Logical: 1}, roachpb.RangeID(17), ctpb.Epoch(1), ctpb.LAI(12)), 539 + ) 540 + 541 + // Two more commands come in. 542 + ts, release = c1.Tracker.Track(ctx) 543 + release(ctx, roachpb.RangeID(17), ctpb.LAI(16)) 544 + require.Equal(t, hlc.Timestamp{WallTime: 1E9, Logical: 1}, ts) 545 + 546 + ts, release = c1.Tracker.Track(ctx) 547 + release(ctx, roachpb.RangeID(8), ctpb.LAI(88)) 548 + require.Equal(t, hlc.Timestamp{WallTime: 1E9, Logical: 1}, ts) 549 + 550 + // Now another tick. Shortly after it, we should be able to serve below 1E9, and 2E9 should 551 + // be the next planned closed timestamp (though we can only verify the former). 552 + c1.TestClock.Tick(hlc.Timestamp{WallTime: 3E9}, ctpb.Epoch(1), nil) 553 + 554 + testutils.SucceedsSoon(t, func() error { 555 + if !c1.Container.Provider.CanServe( 556 + c1.NodeID, hlc.Timestamp{WallTime: 1E9}, roachpb.RangeID(17), ctpb.Epoch(1), ctpb.LAI(12), 557 + ) { 558 + return errors.New("still can't serve") 559 + } 560 + return nil 561 + }) 562 + 563 + // Shouldn't be able to serve the same thing if we haven't caught up yet. 564 + require.False(t, c1.Container.Provider.CanServe( 565 + c1.NodeID, hlc.Timestamp{WallTime: 1E9}, roachpb.RangeID(17), ctpb.Epoch(1), ctpb.LAI(11), 566 + )) 567 + 568 + // Shouldn't be able to serve at a higher timestamp. 569 + require.False(t, c1.Container.Provider.CanServe( 570 + c1.NodeID, hlc.Timestamp{WallTime: 1E9, Logical: 1}, roachpb.RangeID(17), ctpb.Epoch(1), ctpb.LAI(12), 571 + )) 572 + 573 + // Now things get a little more interesting. Tell node2 to get a stream of 574 + // information from node1. We do this via Request, which as a side effect lets 575 + // us ascertain that this request makes it to n1. 576 + c2.Clients.Request(roachpb.NodeID(1), roachpb.RangeID(18)) 577 + testutils.SucceedsSoon(t, func() error { 578 + exp := []roachpb.RangeID{18} 579 + c1.Refreshed.Lock() 580 + defer c1.Refreshed.Unlock() 581 + if !reflect.DeepEqual(exp, c1.Refreshed.RangeIDs) { 582 + return errors.Errorf("still waiting for %v: currently %v", exp, c1.Refreshed.RangeIDs) 583 + } 584 + return nil 585 + }) 586 + 587 + // And n2 should soon also be able to serve follower reads for a range lead by 588 + // n1 when it has caught up. 589 + testutils.SucceedsSoon(t, func() error { 590 + if !c2.Container.Provider.CanServe( 591 + c1.NodeID, hlc.Timestamp{WallTime: 1E9}, roachpb.RangeID(17), ctpb.Epoch(1), ctpb.LAI(12), 592 + ) { 593 + return errors.New("n2 still can't serve") 594 + } 595 + return nil 596 + }) 597 + 598 + // Remember the other proposals we tracked above on n1: (r17, 16) and (r8, 88). Feeding another 599 + // timestamp to n1, we should see them closed out at t=2E9, and both n1 and n2 should automatically 600 + // be able to serve them soon thereafter. 601 + c1.TestClock.Tick(hlc.Timestamp{WallTime: 4E9}, ctpb.Epoch(1), nil) 602 + 603 + checkEpoch1Reads := func(ts hlc.Timestamp) { 604 + for i, c := range []*TestContainer{c1, c2} { 605 + for _, tuple := range []struct { 606 + roachpb.RangeID 607 + ctpb.LAI 608 + }{ 609 + {17, 16}, 610 + {8, 88}, 611 + } { 612 + testutils.SucceedsSoon(t, func() error { 613 + if !c.Container.Provider.CanServe( 614 + c1.NodeID, ts, tuple.RangeID, ctpb.Epoch(1), tuple.LAI, 615 + ) { 616 + return errors.Errorf("n%d still can't serve (r%d,%d) @ %s", i+1, tuple.RangeID, tuple.LAI, ts) 617 + } 618 + return nil 619 + }) 620 + // Still can't serve when not caught up. 621 + require.False(t, c.Container.Provider.CanServe( 622 + c1.NodeID, ts, tuple.RangeID, ctpb.Epoch(1), tuple.LAI-1, 623 + )) 624 + // Can serve when more than caught up. 625 + require.True(t, c.Container.Provider.CanServe( 626 + c1.NodeID, ts, tuple.RangeID, ctpb.Epoch(1), tuple.LAI+1, 627 + )) 628 + // Can't serve when in different epoch, no matter larger or smaller. 629 + require.False(t, c.Container.Provider.CanServe( 630 + c1.NodeID, ts, tuple.RangeID, ctpb.Epoch(0), tuple.LAI, 631 + )) 632 + require.False(t, c.Container.Provider.CanServe( 633 + c1.NodeID, ts, tuple.RangeID, ctpb.Epoch(2), tuple.LAI, 634 + )) 635 + } 636 + } 637 + } 638 + checkEpoch1Reads(hlc.Timestamp{WallTime: 2E9}) 639 + 640 + // Uh-oh! n1 must've missed a heartbeat. The epoch goes up by one. This means 641 + // that soon (after the next tick) timestamps should be closed out under the 642 + // the epoch. 3E9 gets closed out under the first epoch in this tick. The 643 + // timestamp at which this happens is doctored to make sure the Storage holds 644 + // on to the past information, because we want to end-to-end test that this all 645 + // works out. Consequently we try Tick at the rotation interval plus the target 646 + // duration next (so that the next closed timestamp is the rotation interval). 647 + c1.TestClock.Tick(hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 5E9}, ctpb.Epoch(2), nil) 648 + 649 + // Previously valid reads should remain valid. 650 + checkEpoch1Reads(hlc.Timestamp{WallTime: 2E9}) 651 + checkEpoch1Reads(hlc.Timestamp{WallTime: 3E9}) 652 + 653 + // Commands get forced above next closed timestamp (from the tick above) minus target interval. 654 + ts, release = c1.Tracker.Track(ctx) 655 + release(ctx, roachpb.RangeID(123), ctpb.LAI(456)) 656 + require.Equal(t, hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 4E9, Logical: 1}, ts) 657 + 658 + // With the next tick, epoch two fully goes into effect (as the first epoch two 659 + // timestamp gets closed out). We do this twice to make sure it's processed before 660 + // the test proceeds. 661 + c1.TestClock.Tick(hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 6E9}, ctpb.Epoch(2), nil) 662 + 663 + // Previously valid reads should remain valid. Note that this is because the 664 + // storage keeps historical data, and we've fine tuned the epoch flip so that 665 + // it happens after the epoch 1 information rotates into another bucket and 666 + // thus is preserved. If the epoch changed at a smaller timestamp, that 667 + // would've wiped out the first epoch's information. 668 + // 669 + // TODO(tschottdorf): we could make the storage smarter so that it forces a 670 + // rotation when the epoch changes, at the expense of pushing out historical 671 + // information earlier. Frequent epoch changes could lead to very little 672 + // historical information in the storage. Probably better not to risk that. 673 + checkEpoch1Reads(hlc.Timestamp{WallTime: 2E9}) 674 + checkEpoch1Reads(hlc.Timestamp{WallTime: 3E9}) 675 + 676 + // Another second, another tick. Now the proposal tracked during epoch 2 should 677 + // be readable from followers (as `scale+5E9` gets closed out). 678 + c1.TestClock.Tick(hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 7E9}, ctpb.Epoch(2), nil) 679 + for i, c := range []*TestContainer{c1, c2} { 680 + rangeID := roachpb.RangeID(123) 681 + lai := ctpb.LAI(456) 682 + epoch := ctpb.Epoch(2) 683 + ts := hlc.Timestamp{WallTime: int64(container.StorageBucketScale) + 5E9} 684 + 685 + testutils.SucceedsSoon(t, func() error { 686 + if !c.Container.Provider.CanServe( 687 + c1.NodeID, ts, rangeID, epoch, lai, 688 + ) { 689 + return errors.Errorf("n%d still can't serve (r%d,%d) @ %s", i+1, rangeID, lai, ts) 690 + } 691 + return nil 692 + }) 693 + 694 + // Still can't serve when not caught up. 695 + require.False(t, c.Container.Provider.CanServe( 696 + c1.NodeID, ts, rangeID, epoch, lai-1, 697 + )) 698 + 699 + // Can serve when more than caught up. 700 + require.True(t, c.Container.Provider.CanServe( 701 + c1.NodeID, ts, rangeID, epoch, lai+1, 702 + )) 703 + 704 + // Can't serve when in different epoch, no matter larger or smaller. 705 + require.False(t, c.Container.Provider.CanServe( 706 + c1.NodeID, ts, rangeID, epoch-1, lai, 707 + )) 708 + require.False(t, c.Container.Provider.CanServe( 709 + c1.NodeID, ts, rangeID, epoch+1, lai, 710 + )) 711 + } 712 +} 713 diff --git a/pkg/storage/closedts/ctpb/client.go b/pkg/storage/closedts/ctpb/client.go 714 new file mode 100644 715 index 00000000000..76e4acc24f2 716 --- /dev/null 717 +++ b/pkg/storage/closedts/ctpb/client.go 718 @@ -0,0 +1,27 @@ 719 +// Copyright 2018 The Cockroach Authors. 720 +// 721 +// Licensed under the Apache License, Version 2.0 (the "License"); 722 +// you may not use this file except in compliance with the License. 723 +// You may obtain a copy of the License at 724 +// 725 +// http://www.apache.org/licenses/LICENSE-2.0 726 +// 727 +// Unless required by applicable law or agreed to in writing, software 728 +// distributed under the License is distributed on an "AS IS" BASIS, 729 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 730 +// implied. See the License for the specific language governing 731 +// permissions and limitations under the License. 732 + 733 +package ctpb 734 + 735 +import "context" 736 + 737 +// Client is the interface for closed timestamp update clients. 738 +type Client interface { 739 + Send(*Reaction) error 740 + Recv() (*Entry, error) 741 + CloseSend() error 742 + Context() context.Context 743 +} 744 + 745 +var _ Client = ClosedTimestamp_GetClient(nil) 746 diff --git a/pkg/storage/closedts/ctpb/entry.go b/pkg/storage/closedts/ctpb/entry.go 747 new file mode 100644 748 index 00000000000..5d1a284a6cf 749 --- /dev/null 750 +++ b/pkg/storage/closedts/ctpb/entry.go 751 @@ -0,0 +1,59 @@ 752 +// Copyright 2018 The Cockroach Authors. 753 +// 754 +// Licensed under the Apache License, Version 2.0 (the "License"); 755 +// you may not use this file except in compliance with the License. 756 +// You may obtain a copy of the License at 757 +// 758 +// http://www.apache.org/licenses/LICENSE-2.0 759 +// 760 +// Unless required by applicable law or agreed to in writing, software 761 +// distributed under the License is distributed on an "AS IS" BASIS, 762 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 763 +// implied. See the License for the specific language governing 764 +// permissions and limitations under the License. 765 + 766 +package ctpb 767 + 768 +import ( 769 + "fmt" 770 + "sort" 771 + "strings" 772 + 773 + "github.com/cockroachdb/cockroach/pkg/roachpb" 774 +) 775 + 776 +// Epoch is an int64 with its own type to avoid mix-ups in positional arguments. 777 +type Epoch int64 778 + 779 +// LAI is an int64 denoting a lease applied index with its own type to avoid 780 +// mix-ups in positional arguments. 781 +type LAI int64 782 + 783 +// String formats Entry for human consumption as well as testing (by avoiding 784 +// randomness in the output caused by map iteraton order). 785 +func (e Entry) String() string { 786 + rangeIDs := make([]roachpb.RangeID, 0, len(e.MLAI)) 787 + for k := range e.MLAI { 788 + rangeIDs = append(rangeIDs, k) 789 + } 790 + 791 + sort.Slice(rangeIDs, func(i, j int) bool { 792 + a, b := rangeIDs[i], rangeIDs[j] 793 + if a == b { 794 + return e.MLAI[a] < e.MLAI[b] 795 + } 796 + return a < b 797 + }) 798 + sl := make([]string, 0, len(rangeIDs)) 799 + for _, rangeID := range rangeIDs { 800 + sl = append(sl, fmt.Sprintf("r%d: %d", rangeID, e.MLAI[rangeID])) 801 + } 802 + if len(sl) == 0 { 803 + sl = []string{"(empty)"} 804 + } 805 + return fmt.Sprintf("CT: %s @ Epoch %d\nFull: %t\nMLAI: %s\n", e.ClosedTimestamp, e.Epoch, e.Full, strings.Join(sl, ", ")) 806 +} 807 + 808 +func (r Reaction) String() string { 809 + return fmt.Sprintf("Refresh: %v", r.Requested) 810 +} 811 diff --git a/pkg/storage/closedts/ctpb/entry.pb.go b/pkg/storage/closedts/ctpb/entry.pb.go 812 new file mode 100644 813 index 00000000000..7bbec1fe515 814 --- /dev/null 815 +++ b/pkg/storage/closedts/ctpb/entry.pb.go 816 @@ -0,0 +1,813 @@ 817 +// Code generated by protoc-gen-gogo. DO NOT EDIT. 818 +// source: storage/closedts/ctpb/entry.proto 819 + 820 +/* 821 + Package ctpb is a generated protocol buffer package. 822 + 823 + It is generated from these files: 824 + storage/closedts/ctpb/entry.proto 825 + 826 + It has these top-level messages: 827 + Entry 828 + Reaction 829 +*/ 830 +package ctpb 831 + 832 +import proto "github.com/gogo/protobuf/proto" 833 +import fmt "fmt" 834 +import math "math" 835 +import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" 836 + 837 +import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" 838 + 839 +import context "context" 840 +import grpc "google.golang.org/grpc" 841 + 842 +import sortkeys "github.com/gogo/protobuf/sortkeys" 843 + 844 +import io "io" 845 + 846 +// Reference imports to suppress errors if they are not otherwise used. 847 +var _ = proto.Marshal 848 +var _ = fmt.Errorf 849 +var _ = math.Inf 850 + 851 +// This is a compile-time assertion to ensure that this generated file 852 +// is compatible with the proto package it is being compiled against. 853 +// A compilation error at this line likely means your copy of the 854 +// proto package needs to be updated. 855 +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package 856 + 857 +// An Entry is a closed timestamp update. It consists of a closed timestamp 858 +// (i.e. a timestamp at or below which the origin node guarantees no more new 859 +// writes are going to be permitted), an associated epoch in which the origin 860 +// node promises it was live (for the closed timestamp), a map of minimum lease 861 +// applied indexes (which have to be caught up to before being allowed to use 862 +// the closed timestamp) as well as an indicator of whether this update supplies 863 +// a full initial state or an increment to be merged into a previous state. In 864 +// practice, the first Entry received for each epoch is full, while the remainder 865 +// are incremental. An incremental update represents the implicit promise that 866 +// the state accumulated since the last full Entry is the true full state. 867 +type Entry struct { 868 + Epoch Epoch `protobuf:"varint,1,opt,name=epoch,proto3,casttype=Epoch" json:"epoch,omitempty"` 869 + ClosedTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp" json:"closed_timestamp"` 870 + MLAI map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI `protobuf:"bytes,3,rep,name=mlai,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID,castvalue=LAI" json:"mlai,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` 871 + // Full is true if the emitter promises that any future write to any range 872 + // mentioned in this Entry will be reflected in a subsequent Entry before any 873 + // stale follower reads are possible. For example, if range 1 is assigned an 874 + // MLAI of 12 in this Entry and isn't mentioned in the five subsequent 875 + // entries, the recipient may behave as if the MLAI of 12 were repeated across 876 + // all of these entries. 877 + // 878 + // In practice, a Full message is received when a stream of Entries is first 879 + // established (or the Epoch changes), and all other updates are incremental 880 + // (i.e. not Full). 881 + Full bool `protobuf:"varint,4,opt,name=full,proto3" json:"full,omitempty"` 882 +} 883 + 884 +func (m *Entry) Reset() { *m = Entry{} } 885 +func (*Entry) ProtoMessage() {} 886 +func (*Entry) Descriptor() ([]byte, []int) { return fileDescriptorEntry, []int{0} } 887 + 888 +// Reactions flow in the direction opposite to Entries and request for ranges to 889 +// be included in the next Entry. Under rare circumstances, ranges may be omitted 890 +// from closed timestamp updates, and so serving follower reads from them would 891 +// fail. The Reaction mechanism serves to explicitly request the missing information 892 +// when that happens. 893 +type Reaction struct { 894 + Requested []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,rep,packed,name=Requested,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"Requested,omitempty"` 895 +} 896 + 897 +func (m *Reaction) Reset() { *m = Reaction{} } 898 +func (*Reaction) ProtoMessage() {} 899 +func (*Reaction) Descriptor() ([]byte, []int) { return fileDescriptorEntry, []int{1} } 900 + 901 +func init() { 902 + proto.RegisterType((*Entry)(nil), "cockroach.storage.ctupdate.Entry") 903 + proto.RegisterType((*Reaction)(nil), "cockroach.storage.ctupdate.Reaction") 904 +} 905 + 906 +// Reference imports to suppress errors if they are not otherwise used. 907 +var _ context.Context 908 +var _ grpc.ClientConn 909 + 910 +// This is a compile-time assertion to ensure that this generated file 911 +// is compatible with the grpc package it is being compiled against. 912 +const _ = grpc.SupportPackageIsVersion4 913 + 914 +// Client API for ClosedTimestamp service 915 + 916 +type ClosedTimestampClient interface { 917 + Get(ctx context.Context, opts ...grpc.CallOption) (ClosedTimestamp_GetClient, error) 918 +} 919 + 920 +type closedTimestampClient struct { 921 + cc *grpc.ClientConn 922 +} 923 + 924 +func NewClosedTimestampClient(cc *grpc.ClientConn) ClosedTimestampClient { 925 + return &closedTimestampClient{cc} 926 +} 927 + 928 +func (c *closedTimestampClient) Get(ctx context.Context, opts ...grpc.CallOption) (ClosedTimestamp_GetClient, error) { 929 + stream, err := grpc.NewClientStream(ctx, &_ClosedTimestamp_serviceDesc.Streams[0], c.cc, "/cockroach.storage.ctupdate.ClosedTimestamp/Get", opts...) 930 + if err != nil { 931 + return nil, err 932 + } 933 + x := &closedTimestampGetClient{stream} 934 + return x, nil 935 +} 936 + 937 +type ClosedTimestamp_GetClient interface { 938 + Send(*Reaction) error 939 + Recv() (*Entry, error) 940 + grpc.ClientStream 941 +} 942 + 943 +type closedTimestampGetClient struct { 944 + grpc.ClientStream 945 +} 946 + 947 +func (x *closedTimestampGetClient) Send(m *Reaction) error { 948 + return x.ClientStream.SendMsg(m) 949 +} 950 + 951 +func (x *closedTimestampGetClient) Recv() (*Entry, error) { 952 + m := new(Entry) 953 + if err := x.ClientStream.RecvMsg(m); err != nil { 954 + return nil, err 955 + } 956 + return m, nil 957 +} 958 + 959 +// Server API for ClosedTimestamp service 960 + 961 +type ClosedTimestampServer interface { 962 + Get(ClosedTimestamp_GetServer) error 963 +} 964 + 965 +func RegisterClosedTimestampServer(s *grpc.Server, srv ClosedTimestampServer) { 966 + s.RegisterService(&_ClosedTimestamp_serviceDesc, srv) 967 +} 968 + 969 +func _ClosedTimestamp_Get_Handler(srv interface{}, stream grpc.ServerStream) error { 970 + return srv.(ClosedTimestampServer).Get(&closedTimestampGetServer{stream}) 971 +} 972 + 973 +type ClosedTimestamp_GetServer interface { 974 + Send(*Entry) error 975 + Recv() (*Reaction, error) 976 + grpc.ServerStream 977 +} 978 + 979 +type closedTimestampGetServer struct { 980 + grpc.ServerStream 981 +} 982 + 983 +func (x *closedTimestampGetServer) Send(m *Entry) error { 984 + return x.ServerStream.SendMsg(m) 985 +} 986 + 987 +func (x *closedTimestampGetServer) Recv() (*Reaction, error) { 988 + m := new(Reaction) 989 + if err := x.ServerStream.RecvMsg(m); err != nil { 990 + return nil, err 991 + } 992 + return m, nil 993 +} 994 + 995 +var _ClosedTimestamp_serviceDesc = grpc.ServiceDesc{ 996 + ServiceName: "cockroach.storage.ctupdate.ClosedTimestamp", 997 + HandlerType: (*ClosedTimestampServer)(nil), 998 + Methods: []grpc.MethodDesc{}, 999 + Streams: []grpc.StreamDesc{ 1000 + { 1001 + StreamName: "Get", 1002 + Handler: _ClosedTimestamp_Get_Handler, 1003 + ServerStreams: true, 1004 + ClientStreams: true, 1005 + }, 1006 + }, 1007 + Metadata: "storage/closedts/ctpb/entry.proto", 1008 +} 1009 + 1010 +func (m *Entry) Marshal() (dAtA []byte, err error) { 1011 + size := m.Size() 1012 + dAtA = make([]byte, size) 1013 + n, err := m.MarshalTo(dAtA) 1014 + if err != nil { 1015 + return nil, err 1016 + } 1017 + return dAtA[:n], nil 1018 +} 1019 + 1020 +func (m *Entry) MarshalTo(dAtA []byte) (int, error) { 1021 + var i int 1022 + _ = i 1023 + var l int 1024 + _ = l 1025 + if m.Epoch != 0 { 1026 + dAtA[i] = 0x8 1027 + i++ 1028 + i = encodeVarintEntry(dAtA, i, uint64(m.Epoch)) 1029 + } 1030 + dAtA[i] = 0x12 1031 + i++ 1032 + i = encodeVarintEntry(dAtA, i, uint64(m.ClosedTimestamp.Size())) 1033 + n1, err := m.ClosedTimestamp.MarshalTo(dAtA[i:]) 1034 + if err != nil { 1035 + return 0, err 1036 + } 1037 + i += n1 1038 + if len(m.MLAI) > 0 { 1039 + keysForMLAI := make([]int32, 0, len(m.MLAI)) 1040 + for k := range m.MLAI { 1041 + keysForMLAI = append(keysForMLAI, int32(k)) 1042 + } 1043 + sortkeys.Int32s(keysForMLAI) 1044 + for _, k := range keysForMLAI { 1045 + dAtA[i] = 0x1a 1046 + i++ 1047 + v := m.MLAI[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(k)] 1048 + mapSize := 1 + sovEntry(uint64(k)) + 1 + sovEntry(uint64(v)) 1049 + i = encodeVarintEntry(dAtA, i, uint64(mapSize)) 1050 + dAtA[i] = 0x8 1051 + i++ 1052 + i = encodeVarintEntry(dAtA, i, uint64(k)) 1053 + dAtA[i] = 0x10 1054 + i++ 1055 + i = encodeVarintEntry(dAtA, i, uint64(v)) 1056 + } 1057 + } 1058 + if m.Full { 1059 + dAtA[i] = 0x20 1060 + i++ 1061 + if m.Full { 1062 + dAtA[i] = 1 1063 + } else { 1064 + dAtA[i] = 0 1065 + } 1066 + i++ 1067 + } 1068 + return i, nil 1069 +} 1070 + 1071 +func (m *Reaction) Marshal() (dAtA []byte, err error) { 1072 + size := m.Size() 1073 + dAtA = make([]byte, size) 1074 + n, err := m.MarshalTo(dAtA) 1075 + if err != nil { 1076 + return nil, err 1077 + } 1078 + return dAtA[:n], nil 1079 +} 1080 + 1081 +func (m *Reaction) MarshalTo(dAtA []byte) (int, error) { 1082 + var i int 1083 + _ = i 1084 + var l int 1085 + _ = l 1086 + if len(m.Requested) > 0 { 1087 + dAtA3 := make([]byte, len(m.Requested)*10) 1088 + var j2 int 1089 + for _, num1 := range m.Requested { 1090 + num := uint64(num1) 1091 + for num >= 1<<7 { 1092 + dAtA3[j2] = uint8(uint64(num)&0x7f | 0x80) 1093 + num >>= 7 1094 + j2++ 1095 + } 1096 + dAtA3[j2] = uint8(num) 1097 + j2++ 1098 + } 1099 + dAtA[i] = 0xa 1100 + i++ 1101 + i = encodeVarintEntry(dAtA, i, uint64(j2)) 1102 + i += copy(dAtA[i:], dAtA3[:j2]) 1103 + } 1104 + return i, nil 1105 +} 1106 + 1107 +func encodeVarintEntry(dAtA []byte, offset int, v uint64) int { 1108 + for v >= 1<<7 { 1109 + dAtA[offset] = uint8(v&0x7f | 0x80) 1110 + v >>= 7 1111 + offset++ 1112 + } 1113 + dAtA[offset] = uint8(v) 1114 + return offset + 1 1115 +} 1116 +func (m *Entry) Size() (n int) { 1117 + var l int 1118 + _ = l 1119 + if m.Epoch != 0 { 1120 + n += 1 + sovEntry(uint64(m.Epoch)) 1121 + } 1122 + l = m.ClosedTimestamp.Size() 1123 + n += 1 + l + sovEntry(uint64(l)) 1124 + if len(m.MLAI) > 0 { 1125 + for k, v := range m.MLAI { 1126 + _ = k 1127 + _ = v 1128 + mapEntrySize := 1 + sovEntry(uint64(k)) + 1 + sovEntry(uint64(v)) 1129 + n += mapEntrySize + 1 + sovEntry(uint64(mapEntrySize)) 1130 + } 1131 + } 1132 + if m.Full { 1133 + n += 2 1134 + } 1135 + return n 1136 +} 1137 + 1138 +func (m *Reaction) Size() (n int) { 1139 + var l int 1140 + _ = l 1141 + if len(m.Requested) > 0 { 1142 + l = 0 1143 + for _, e := range m.Requested { 1144 + l += sovEntry(uint64(e)) 1145 + } 1146 + n += 1 + sovEntry(uint64(l)) + l 1147 + } 1148 + return n 1149 +} 1150 + 1151 +func sovEntry(x uint64) (n int) { 1152 + for { 1153 + n++ 1154 + x >>= 7 1155 + if x == 0 { 1156 + break 1157 + } 1158 + } 1159 + return n 1160 +} 1161 +func sozEntry(x uint64) (n int) { 1162 + return sovEntry(uint64((x << 1) ^ uint64((int64(x) >> 63)))) 1163 +} 1164 +func (m *Entry) Unmarshal(dAtA []byte) error { 1165 + l := len(dAtA) 1166 + iNdEx := 0 1167 + for iNdEx < l { 1168 + preIndex := iNdEx 1169 + var wire uint64 1170 + for shift := uint(0); ; shift += 7 { 1171 + if shift >= 64 { 1172 + return ErrIntOverflowEntry 1173 + } 1174 + if iNdEx >= l { 1175 + return io.ErrUnexpectedEOF 1176 + } 1177 + b := dAtA[iNdEx] 1178 + iNdEx++ 1179 + wire |= (uint64(b) & 0x7F) << shift 1180 + if b < 0x80 { 1181 + break 1182 + } 1183 + } 1184 + fieldNum := int32(wire >> 3) 1185 + wireType := int(wire & 0x7) 1186 + if wireType == 4 { 1187 + return fmt.Errorf("proto: Entry: wiretype end group for non-group") 1188 + } 1189 + if fieldNum <= 0 { 1190 + return fmt.Errorf("proto: Entry: illegal tag %d (wire type %d)", fieldNum, wire) 1191 + } 1192 + switch fieldNum { 1193 + case 1: 1194 + if wireType != 0 { 1195 + return fmt.Errorf("proto: wrong wireType = %d for field Epoch", wireType) 1196 + } 1197 + m.Epoch = 0 1198 + for shift := uint(0); ; shift += 7 { 1199 + if shift >= 64 { 1200 + return ErrIntOverflowEntry 1201 + } 1202 + if iNdEx >= l { 1203 + return io.ErrUnexpectedEOF 1204 + } 1205 + b := dAtA[iNdEx] 1206 + iNdEx++ 1207 + m.Epoch |= (Epoch(b) & 0x7F) << shift 1208 + if b < 0x80 { 1209 + break 1210 + } 1211 + } 1212 + case 2: 1213 + if wireType != 2 { 1214 + return fmt.Errorf("proto: wrong wireType = %d for field ClosedTimestamp", wireType) 1215 + } 1216 + var msglen int 1217 + for shift := uint(0); ; shift += 7 { 1218 + if shift >= 64 { 1219 + return ErrIntOverflowEntry 1220 + } 1221 + if iNdEx >= l { 1222 + return io.ErrUnexpectedEOF 1223 + } 1224 + b := dAtA[iNdEx] 1225 + iNdEx++ 1226 + msglen |= (int(b) & 0x7F) << shift 1227 + if b < 0x80 { 1228 + break 1229 + } 1230 + } 1231 + if msglen < 0 { 1232 + return ErrInvalidLengthEntry 1233 + } 1234 + postIndex := iNdEx + msglen 1235 + if postIndex > l { 1236 + return io.ErrUnexpectedEOF 1237 + } 1238 + if err := m.ClosedTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { 1239 + return err 1240 + } 1241 + iNdEx = postIndex 1242 + case 3: 1243 + if wireType != 2 { 1244 + return fmt.Errorf("proto: wrong wireType = %d for field MLAI", wireType) 1245 + } 1246 + var msglen int 1247 + for shift := uint(0); ; shift += 7 { 1248 + if shift >= 64 { 1249 + return ErrIntOverflowEntry 1250 + } 1251 + if iNdEx >= l { 1252 + return io.ErrUnexpectedEOF 1253 + } 1254 + b := dAtA[iNdEx] 1255 + iNdEx++ 1256 + msglen |= (int(b) & 0x7F) << shift 1257 + if b < 0x80 { 1258 + break 1259 + } 1260 + } 1261 + if msglen < 0 { 1262 + return ErrInvalidLengthEntry 1263 + } 1264 + postIndex := iNdEx + msglen 1265 + if postIndex > l { 1266 + return io.ErrUnexpectedEOF 1267 + } 1268 + if m.MLAI == nil { 1269 + m.MLAI = make(map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI) 1270 + } 1271 + var mapkey int32 1272 + var mapvalue int64 1273 + for iNdEx < postIndex { 1274 + entryPreIndex := iNdEx 1275 + var wire uint64 1276 + for shift := uint(0); ; shift += 7 { 1277 + if shift >= 64 { 1278 + return ErrIntOverflowEntry 1279 + } 1280 + if iNdEx >= l { 1281 + return io.ErrUnexpectedEOF 1282 + } 1283 + b := dAtA[iNdEx] 1284 + iNdEx++ 1285 + wire |= (uint64(b) & 0x7F) << shift 1286 + if b < 0x80 { 1287 + break 1288 + } 1289 + } 1290 + fieldNum := int32(wire >> 3) 1291 + if fieldNum == 1 { 1292 + for shift := uint(0); ; shift += 7 { 1293 + if shift >= 64 { 1294 + return ErrIntOverflowEntry 1295 + } 1296 + if iNdEx >= l { 1297 + return io.ErrUnexpectedEOF 1298 + } 1299 + b := dAtA[iNdEx] 1300 + iNdEx++ 1301 + mapkey |= (int32(b) & 0x7F) << shift 1302 + if b < 0x80 { 1303 + break 1304 + } 1305 + } 1306 + } else if fieldNum == 2 { 1307 + for shift := uint(0); ; shift += 7 { 1308 + if shift >= 64 { 1309 + return ErrIntOverflowEntry 1310 + } 1311 + if iNdEx >= l { 1312 + return io.ErrUnexpectedEOF 1313 + } 1314 + b := dAtA[iNdEx] 1315 + iNdEx++ 1316 + mapvalue |= (int64(b) & 0x7F) << shift 1317 + if b < 0x80 { 1318 + break 1319 + } 1320 + } 1321 + } else { 1322 + iNdEx = entryPreIndex 1323 + skippy, err := skipEntry(dAtA[iNdEx:]) 1324 + if err != nil { 1325 + return err 1326 + } 1327 + if skippy < 0 { 1328 + return ErrInvalidLengthEntry 1329 + } 1330 + if (iNdEx + skippy) > postIndex { 1331 + return io.ErrUnexpectedEOF 1332 + } 1333 + iNdEx += skippy 1334 + } 1335 + } 1336 + m.MLAI[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(mapkey)] = ((LAI)(mapvalue)) 1337 + iNdEx = postIndex 1338 + case 4: 1339 + if wireType != 0 { 1340 + return fmt.Errorf("proto: wrong wireType = %d for field Full", wireType) 1341 + } 1342 + var v int 1343 + for shift := uint(0); ; shift += 7 { 1344 + if shift >= 64 { 1345 + return ErrIntOverflowEntry 1346 + } 1347 + if iNdEx >= l { 1348 + return io.ErrUnexpectedEOF 1349 + } 1350 + b := dAtA[iNdEx] 1351 + iNdEx++ 1352 + v |= (int(b) & 0x7F) << shift 1353 + if b < 0x80 { 1354 + break 1355 + } 1356 + } 1357 + m.Full = bool(v != 0) 1358 + default: 1359 + iNdEx = preIndex 1360 + skippy, err := skipEntry(dAtA[iNdEx:]) 1361 + if err != nil { 1362 + return err 1363 + } 1364 + if skippy < 0 { 1365 + return ErrInvalidLengthEntry 1366 + } 1367 + if (iNdEx + skippy) > l { 1368 + return io.ErrUnexpectedEOF 1369 + } 1370 + iNdEx += skippy 1371 + } 1372 + } 1373 + 1374 + if iNdEx > l { 1375 + return io.ErrUnexpectedEOF 1376 + } 1377 + return nil 1378 +} 1379 +func (m *Reaction) Unmarshal(dAtA []byte) error { 1380 + l := len(dAtA) 1381 + iNdEx := 0 1382 + for iNdEx < l { 1383 + preIndex := iNdEx 1384 + var wire uint64 1385 + for shift := uint(0); ; shift += 7 { 1386 + if shift >= 64 { 1387 + return ErrIntOverflowEntry 1388 + } 1389 + if iNdEx >= l { 1390 + return io.ErrUnexpectedEOF 1391 + } 1392 + b := dAtA[iNdEx] 1393 + iNdEx++ 1394 + wire |= (uint64(b) & 0x7F) << shift 1395 + if b < 0x80 { 1396 + break 1397 + } 1398 + } 1399 + fieldNum := int32(wire >> 3) 1400 + wireType := int(wire & 0x7) 1401 + if wireType == 4 { 1402 + return fmt.Errorf("proto: Reaction: wiretype end group for non-group") 1403 + } 1404 + if fieldNum <= 0 { 1405 + return fmt.Errorf("proto: Reaction: illegal tag %d (wire type %d)", fieldNum, wire) 1406 + } 1407 + switch fieldNum { 1408 + case 1: 1409 + if wireType == 0 { 1410 + var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID 1411 + for shift := uint(0); ; shift += 7 { 1412 + if shift >= 64 { 1413 + return ErrIntOverflowEntry 1414 + } 1415 + if iNdEx >= l { 1416 + return io.ErrUnexpectedEOF 1417 + } 1418 + b := dAtA[iNdEx] 1419 + iNdEx++ 1420 + v |= (github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(b) & 0x7F) << shift 1421 + if b < 0x80 { 1422 + break 1423 + } 1424 + } 1425 + m.Requested = append(m.Requested, v) 1426 + } else if wireType == 2 { 1427 + var packedLen int 1428 + for shift := uint(0); ; shift += 7 { 1429 + if shift >= 64 { 1430 + return ErrIntOverflowEntry 1431 + } 1432 + if iNdEx >= l { 1433 + return io.ErrUnexpectedEOF 1434 + } 1435 + b := dAtA[iNdEx] 1436 + iNdEx++ 1437 + packedLen |= (int(b) & 0x7F) << shift 1438 + if b < 0x80 { 1439 + break 1440 + } 1441 + } 1442 + if packedLen < 0 { 1443 + return ErrInvalidLengthEntry 1444 + } 1445 + postIndex := iNdEx + packedLen 1446 + if postIndex > l { 1447 + return io.ErrUnexpectedEOF 1448 + } 1449 + for iNdEx < postIndex { 1450 + var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID 1451 + for shift := uint(0); ; shift += 7 { 1452 + if shift >= 64 { 1453 + return ErrIntOverflowEntry 1454 + } 1455 + if iNdEx >= l { 1456 + return io.ErrUnexpectedEOF 1457 + } 1458 + b := dAtA[iNdEx] 1459 + iNdEx++ 1460 + v |= (github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(b) & 0x7F) << shift 1461 + if b < 0x80 { 1462 + break 1463 + } 1464 + } 1465 + m.Requested = append(m.Requested, v) 1466 + } 1467 + } else { 1468 + return fmt.Errorf("proto: wrong wireType = %d for field Requested", wireType) 1469 + } 1470 + default: 1471 + iNdEx = preIndex 1472 + skippy, err := skipEntry(dAtA[iNdEx:]) 1473 + if err != nil { 1474 + return err 1475 + } 1476 + if skippy < 0 { 1477 + return ErrInvalidLengthEntry 1478 + } 1479 + if (iNdEx + skippy) > l { 1480 + return io.ErrUnexpectedEOF 1481 + } 1482 + iNdEx += skippy 1483 + } 1484 + } 1485 + 1486 + if iNdEx > l { 1487 + return io.ErrUnexpectedEOF 1488 + } 1489 + return nil 1490 +} 1491 +func skipEntry(dAtA []byte) (n int, err error) { 1492 + l := len(dAtA) 1493 + iNdEx := 0 1494 + for iNdEx < l { 1495 + var wire uint64 1496 + for shift := uint(0); ; shift += 7 { 1497 + if shift >= 64 { 1498 + return 0, ErrIntOverflowEntry 1499 + } 1500 + if iNdEx >= l { 1501 + return 0, io.ErrUnexpectedEOF 1502 + } 1503 + b := dAtA[iNdEx] 1504 + iNdEx++ 1505 + wire |= (uint64(b) & 0x7F) << shift 1506 + if b < 0x80 { 1507 + break 1508 + } 1509 + } 1510 + wireType := int(wire & 0x7) 1511 + switch wireType { 1512 + case 0: 1513 + for shift := uint(0); ; shift += 7 { 1514 + if shift >= 64 { 1515 + return 0, ErrIntOverflowEntry 1516 + } 1517 + if iNdEx >= l { 1518 + return 0, io.ErrUnexpectedEOF 1519 + } 1520 + iNdEx++ 1521 + if dAtA[iNdEx-1] < 0x80 { 1522 + break 1523 + } 1524 + } 1525 + return iNdEx, nil 1526 + case 1: 1527 + iNdEx += 8 1528 + return iNdEx, nil 1529 + case 2: 1530 + var length int 1531 + for shift := uint(0); ; shift += 7 { 1532 + if shift >= 64 { 1533 + return 0, ErrIntOverflowEntry 1534 + } 1535 + if iNdEx >= l { 1536 + return 0, io.ErrUnexpectedEOF 1537 + } 1538 + b := dAtA[iNdEx] 1539 + iNdEx++ 1540 + length |= (int(b) & 0x7F) << shift 1541 + if b < 0x80 { 1542 + break 1543 + } 1544 + } 1545 + iNdEx += length 1546 + if length < 0 { 1547 + return 0, ErrInvalidLengthEntry 1548 + } 1549 + return iNdEx, nil 1550 + case 3: 1551 + for { 1552 + var innerWire uint64 1553 + var start int = iNdEx 1554 + for shift := uint(0); ; shift += 7 { 1555 + if shift >= 64 { 1556 + return 0, ErrIntOverflowEntry 1557 + } 1558 + if iNdEx >= l { 1559 + return 0, io.ErrUnexpectedEOF 1560 + } 1561 + b := dAtA[iNdEx] 1562 + iNdEx++ 1563 + innerWire |= (uint64(b) & 0x7F) << shift 1564 + if b < 0x80 { 1565 + break 1566 + } 1567 + } 1568 + innerWireType := int(innerWire & 0x7) 1569 + if innerWireType == 4 { 1570 + break 1571 + } 1572 + next, err := skipEntry(dAtA[start:]) 1573 + if err != nil { 1574 + return 0, err 1575 + } 1576 + iNdEx = start + next 1577 + } 1578 + return iNdEx, nil 1579 + case 4: 1580 + return iNdEx, nil 1581 + case 5: 1582 + iNdEx += 4 1583 + return iNdEx, nil 1584 + default: 1585 + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) 1586 + } 1587 + } 1588 + panic("unreachable") 1589 +} 1590 + 1591 +var ( 1592 + ErrInvalidLengthEntry = fmt.Errorf("proto: negative length found during unmarshaling") 1593 + ErrIntOverflowEntry = fmt.Errorf("proto: integer overflow") 1594 +) 1595 + 1596 +func init() { proto.RegisterFile("storage/closedts/ctpb/entry.proto", fileDescriptorEntry) } 1597 + 1598 +var fileDescriptorEntry = []byte{ 1599 + // 459 bytes of a gzipped FileDescriptorProto 1600 + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x52, 0x4f, 0x6b, 0xd4, 0x40, 1601 + 0x1c, 0xcd, 0x6c, 0x12, 0xe9, 0x4e, 0x0f, 0x2d, 0x43, 0x0f, 0x21, 0x68, 0x92, 0x2e, 0x1e, 0x02, 1602 + 0xc2, 0x8c, 0xac, 0x82, 0xa5, 0xb7, 0x46, 0x4b, 0x59, 0x68, 0x3d, 0x0c, 0xc5, 0x83, 0x17, 0x99, 1603 + 0xcc, 0x8e, 0x49, 0xd8, 0xd9, 0x4c, 0xdc, 0x4c, 0x84, 0x5e, 0x3d, 0x89, 0x27, 0x8f, 0x1e, 0xfd, 1604 + 0x38, 0x7b, 0xf4, 0x24, 0x9e, 0xb6, 0x1a, 0xbf, 0x45, 0x4f, 0x92, 0x49, 0x77, 0x17, 0x04, 0x15, 1605 + 0xbc, 0xbd, 0xfc, 0xfe, 0xbc, 0xf7, 0xf2, 0xe6, 0x07, 0x0f, 0x6b, 0xad, 0x16, 0x2c, 0x13, 0x84, 1606 + 0x4b, 0x55, 0x8b, 0xa9, 0xae, 0x09, 0xd7, 0x55, 0x4a, 0x44, 0xa9, 0x17, 0x57, 0xb8, 0x5a, 0x28, 1607 + 0xad, 0x90, 0xcf, 0x15, 0x9f, 0x2d, 0x14, 0xe3, 0x39, 0xbe, 0x1d, 0xc6, 0x5c, 0x37, 0xd5, 0x94, 1608 + 0x69, 0xe1, 0x1f, 0x64, 0x2a, 0x53, 0x66, 0x8c, 0x74, 0xa8, 0xdf, 0xf0, 0xef, 0x66, 0x4a, 0x65, 1609 + 0x52, 0x10, 0x56, 0x15, 0x84, 0x95, 0xa5, 0xd2, 0x4c, 0x17, 0xaa, 0xac, 0x6f, 0xbb, 0x5e, 0xa3, 1610 + 0x0b, 0x49, 0x72, 0xc9, 0x89, 0x2e, 0xe6, 0xa2, 0xd6, 0x6c, 0x5e, 0xf5, 0x9d, 0xd1, 0xd7, 0x01, 1611 + 0x74, 0x4f, 0x3b, 0x65, 0x14, 0x42, 0x57, 0x54, 0x8a, 0xe7, 0x1e, 0x88, 0x40, 0x6c, 0x27, 0xc3, 1612 + 0x9b, 0x55, 0xe8, 0x9e, 0x76, 0x05, 0xda, 0xd7, 0xd1, 0x73, 0xb8, 0xdf, 0x3b, 0x7e, 0xb5, 0x21, 1613 + 0xf1, 0x06, 0x11, 0x88, 0x77, 0xc7, 0xf7, 0xf0, 0xd6, 0x6f, 0xa7, 0x84, 0x73, 0xc9, 0xf1, 0xe5, 1614 + 0x7a, 0x28, 0x71, 0x96, 0xab, 0xd0, 0xa2, 0x7b, 0xfd, 0xf2, 0xa6, 0x8c, 0xde, 0x03, 0xe8, 0xcc, 1615 + 0x25, 0x2b, 0x3c, 0x3b, 0xb2, 0xe3, 0xdd, 0xf1, 0x03, 0xfc, 0xe7, 0x9f, 0xc6, 0xc6, 0x22, 0xbe, 1616 + 0x90, 0xac, 0x30, 0x28, 0x39, 0x6b, 0x57, 0xa1, 0x73, 0x71, 0x7e, 0x32, 0x79, 0x77, 0x1d, 0x3e, 1617 + 0xce, 0x0a, 0x9d, 0x37, 0x29, 0xe6, 0x6a, 0x4e, 0x36, 0x14, 0xd3, 0x74, 0x8b, 0x49, 0x35, 0xcb, 1618 + 0x88, 0x41, 0x55, 0x8a, 0x29, 0x2b, 0x33, 0x31, 0x79, 0xf6, 0xe1, 0x3a, 0xb4, 0xcf, 0x4f, 0x26, 1619 + 0xd4, 0x38, 0x40, 0x08, 0x3a, 0xaf, 0x1b, 0x29, 0x3d, 0x27, 0x02, 0xf1, 0x0e, 0x35, 0xd8, 0x7f, 1620 + 0x02, 0x87, 0x1b, 0x3d, 0xb4, 0x0f, 0xed, 0x99, 0xb8, 0x32, 0xd1, 0xb8, 0xb4, 0x83, 0xe8, 0x00, 1621 + 0xba, 0x6f, 0x99, 0x6c, 0x84, 0x89, 0xc0, 0xa6, 0xfd, 0xc7, 0xf1, 0xe0, 0x08, 0x1c, 0x3b, 0x9f, 1622 + 0x3e, 0x87, 0xd6, 0x28, 0x87, 0x3b, 0x54, 0x30, 0xde, 0xbd, 0x02, 0x7a, 0x01, 0x87, 0x54, 0xbc, 1623 + 0x69, 0x44, 0xad, 0xc5, 0xd4, 0x03, 0x91, 0x1d, 0xbb, 0xc9, 0xd1, 0xcd, 0xea, 0xff, 0x8c, 0xd3, 1624 + 0x2d, 0x55, 0xaf, 0x34, 0xce, 0xe0, 0xde, 0xd3, 0xdf, 0xa2, 0xbd, 0x84, 0xf6, 0x99, 0xd0, 0xe8, 1625 + 0xfe, 0xdf, 0x22, 0x5d, 0xbb, 0xf3, 0x0f, 0xff, 0x19, 0xfc, 0xc8, 0x8a, 0xc1, 0x43, 0x90, 0x04, 1626 + 0xcb, 0x1f, 0x81, 0xb5, 0x6c, 0x03, 0xf0, 0xa5, 0x0d, 0xc0, 0xb7, 0x36, 0x00, 0xdf, 0xdb, 0x00, 1627 + 0x7c, 0xfc, 0x19, 0x58, 0x2f, 0x9d, 0xee, 0x82, 0xd3, 0x3b, 0xe6, 0xa4, 0x1e, 0xfd, 0x0a, 0x00, 1628 + 0x00, 0xff, 0xff, 0x30, 0xde, 0x98, 0x28, 0xe1, 0x02, 0x00, 0x00, 1629 +} 1630 diff --git a/pkg/storage/closedts/ctpb/entry.proto b/pkg/storage/closedts/ctpb/entry.proto 1631 new file mode 100644 1632 index 00000000000..9bd3ad45f3b 1633 --- /dev/null 1634 +++ b/pkg/storage/closedts/ctpb/entry.proto 1635 @@ -0,0 +1,68 @@ 1636 +// Copyright 2018 The Cockroach Authors. 1637 +// 1638 +// Licensed under the Apache License, Version 2.0 (the "License"); 1639 +// you may not use this file except in compliance with the License. 1640 +// You may obtain a copy of the License at 1641 +// 1642 +// http://www.apache.org/licenses/LICENSE-2.0 1643 +// 1644 +// Unless required by applicable law or agreed to in writing, software 1645 +// distributed under the License is distributed on an "AS IS" BASIS, 1646 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 1647 +// implied. See the License for the specific language governing 1648 +// permissions and limitations under the License. 1649 + 1650 +syntax = "proto3"; 1651 +package cockroach.storage.ctupdate; 1652 +option go_package = "ctpb"; 1653 + 1654 +import "gogoproto/gogo.proto"; 1655 +import "google/api/annotations.proto"; 1656 + 1657 +import "util/hlc/timestamp.proto"; 1658 + 1659 +// An Entry is a closed timestamp update. It consists of a closed timestamp 1660 +// (i.e. a timestamp at or below which the origin node guarantees no more new 1661 +// writes are going to be permitted), an associated epoch in which the origin 1662 +// node promises it was live (for the closed timestamp), a map of minimum lease 1663 +// applied indexes (which have to be caught up to before being allowed to use 1664 +// the closed timestamp) as well as an indicator of whether this update supplies 1665 +// a full initial state or an increment to be merged into a previous state. In 1666 +// practice, the first Entry received for each epoch is full, while the remainder 1667 +// are incremental. An incremental update represents the implicit promise that 1668 +// the state accumulated since the last full Entry is the true full state. 1669 +message Entry { 1670 + option (gogoproto.goproto_stringer) = false; 1671 + 1672 + int64 epoch = 1 [(gogoproto.casttype) = "Epoch"]; 1673 + util.hlc.Timestamp closed_timestamp = 2 [(gogoproto.nullable) = false]; 1674 + map<int32, int64> mlai = 3 [(gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID", 1675 + (gogoproto.castvalue) = "LAI", 1676 + (gogoproto.customname) = "MLAI"]; 1677 + // Full is true if the emitter promises that any future write to any range 1678 + // mentioned in this Entry will be reflected in a subsequent Entry before any 1679 + // stale follower reads are possible. For example, if range 1 is assigned an 1680 + // MLAI of 12 in this Entry and isn't mentioned in the five subsequent 1681 + // entries, the recipient may behave as if the MLAI of 12 were repeated across 1682 + // all of these entries. 1683 + // 1684 + // In practice, a Full message is received when a stream of Entries is first 1685 + // established (or the Epoch changes), and all other updates are incremental 1686 + // (i.e. not Full). 1687 + bool full = 4; 1688 +} 1689 + 1690 +// Reactions flow in the direction opposite to Entries and request for ranges to 1691 +// be included in the next Entry. Under rare circumstances, ranges may be omitted 1692 +// from closed timestamp updates, and so serving follower reads from them would 1693 +// fail. The Reaction mechanism serves to explicitly request the missing information 1694 +// when that happens. 1695 +message Reaction { 1696 + option (gogoproto.goproto_stringer) = false; 1697 + 1698 + repeated int32 Requested = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; 1699 +} 1700 + 1701 +service ClosedTimestamp { 1702 + rpc Get(stream Reaction) returns (stream Entry) { } 1703 +} 1704 diff --git a/pkg/storage/closedts/ctpb/server.go b/pkg/storage/closedts/ctpb/server.go 1705 new file mode 100644 1706 index 00000000000..0b8bd186dc6 1707 --- /dev/null 1708 +++ b/pkg/storage/closedts/ctpb/server.go 1709 @@ -0,0 +1,84 @@ 1710 +// Copyright 2018 The Cockroach Authors. 1711 +// 1712 +// Licensed under the Apache License, Version 2.0 (the "License"); 1713 +// you may not use this file except in compliance with the License. 1714 +// You may obtain a copy of the License at 1715 +// 1716 +// http://www.apache.org/licenses/LICENSE-2.0 1717 +// 1718 +// Unless required by applicable law or agreed to in writing, software 1719 +// distributed under the License is distributed on an "AS IS" BASIS, 1720 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 1721 +// implied. See the License for the specific language governing 1722 +// permissions and limitations under the License. 1723 + 1724 +package ctpb 1725 + 1726 +import ( 1727 + "context" 1728 + "errors" 1729 + 1730 + "google.golang.org/grpc/metadata" 1731 +) 1732 + 1733 +// InboundClient is an interface that narrows ClosedTimestamp_GetServer down to what's 1734 +// actually required. 1735 +type InboundClient interface { 1736 + Send(*Entry) error 1737 + Recv() (*Reaction, error) 1738 + Context() context.Context 1739 +} 1740 + 1741 +// Server is the interface implemented by types that want to serve incoming 1742 +// closed timestamp update streams. 1743 +type Server interface { 1744 + Get(InboundClient) error 1745 +} 1746 + 1747 +// ServerShim is a wrapper around Server that provides the wider interface that 1748 +// gRPC expects. 1749 +type ServerShim struct{ Server } 1750 + 1751 +var _ ClosedTimestampServer = (*ServerShim)(nil) 1752 + 1753 +// Get implements ClosedTimestampServer by passing through to the wrapped Server. 1754 +func (s ServerShim) Get(client ClosedTimestamp_GetServer) error { 1755 + return s.Server.Get(client) 1756 +} 1757 + 1758 +var _ InboundClient = ClosedTimestamp_GetServer(nil) 1759 + 1760 +// InboundClientShim extends a ctpb.InboundClient to a ClosedTimestamp_GetServer 1761 +// by returning errors from all added methods where possible, and no-oping the 1762 +// rest. 1763 +type InboundClientShim struct { 1764 + InboundClient 1765 +} 1766 + 1767 +// SetHeader is a shim implementation for ClosedTimestamp_GetServer 1768 +// that always returns an error. 1769 +func (s InboundClientShim) SetHeader(metadata.MD) error { 1770 + return errors.New("unimplemented") 1771 +} 1772 + 1773 +// SendHeader is a shim implementation for ClosedTimestamp_GetServer 1774 +// that always returns an error. 1775 +func (s InboundClientShim) SendHeader(metadata.MD) error { 1776 + return errors.New("unimplemented") 1777 +} 1778 + 1779 +// SetTrailer is a shim implementation for ClosedTimestamp_GetServer 1780 +// that ignores the argument. 1781 +func (s InboundClientShim) SetTrailer(metadata.MD) {} 1782 + 1783 +// SendMsg is a shim implementation for ClosedTimestamp_GetServer 1784 +// that always returns an error. 1785 +func (s InboundClientShim) SendMsg(m interface{}) error { 1786 + return errors.New("unimplemented") 1787 +} 1788 + 1789 +// RecvMsg is a shim implementation for ClosedTimestamp_GetServer 1790 +// that always returns an error. 1791 +func (s InboundClientShim) RecvMsg(m interface{}) error { 1792 + return errors.New("unimplemented") 1793 +} 1794 diff --git a/pkg/storage/minprop/doc.go b/pkg/storage/closedts/minprop/doc.go 1795 similarity index 100% 1796 rename from pkg/storage/minprop/doc.go 1797 rename to pkg/storage/closedts/minprop/doc.go 1798 diff --git a/pkg/storage/minprop/doc_test.go b/pkg/storage/closedts/minprop/doc_test.go 1799 similarity index 98% 1800 rename from pkg/storage/minprop/doc_test.go 1801 rename to pkg/storage/closedts/minprop/doc_test.go 1802 index 6968328a88d..f2d0b6c9f6b 100644 1803 --- a/pkg/storage/minprop/doc_test.go 1804 +++ b/pkg/storage/closedts/minprop/doc_test.go 1805 @@ -21,6 +21,7 @@ import ( 1806 "strings" 1807 1808 "github.com/cockroachdb/cockroach/pkg/roachpb" 1809 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 1810 "github.com/cockroachdb/cockroach/pkg/util/hlc" 1811 ) 1812 1813 @@ -196,7 +197,7 @@ func Example() { 1814 1815 // mlaiString converts an mlai map into a string. Avoids randomized ordering of 1816 // map elements in string output. 1817 -func mlaiString(mlai map[roachpb.RangeID]int64) string { 1818 +func mlaiString(mlai map[roachpb.RangeID]ctpb.LAI) string { 1819 var rangeIDs []roachpb.RangeID 1820 for rangeID := range mlai { 1821 rangeIDs = append(rangeIDs, rangeID) 1822 diff --git a/pkg/storage/minprop/tracker.go b/pkg/storage/closedts/minprop/tracker.go 1823 similarity index 92% 1824 rename from pkg/storage/minprop/tracker.go 1825 rename to pkg/storage/closedts/minprop/tracker.go 1826 index 78d0b4b3005..6d17eb15c9f 100644 1827 --- a/pkg/storage/minprop/tracker.go 1828 +++ b/pkg/storage/closedts/minprop/tracker.go 1829 @@ -20,6 +20,8 @@ import ( 1830 "sort" 1831 1832 "github.com/cockroachdb/cockroach/pkg/roachpb" 1833 + "github.com/cockroachdb/cockroach/pkg/storage/closedts" 1834 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 1835 "github.com/cockroachdb/cockroach/pkg/util/hlc" 1836 "github.com/cockroachdb/cockroach/pkg/util/log" 1837 "github.com/cockroachdb/cockroach/pkg/util/syncutil" 1838 @@ -72,18 +74,20 @@ type Tracker struct { 1839 // `rightRef`. 1840 1841 next hlc.Timestamp 1842 - leftMLAI, rightMLAI map[roachpb.RangeID]int64 1843 + leftMLAI, rightMLAI map[roachpb.RangeID]ctpb.LAI 1844 leftRef, rightRef int 1845 } 1846 } 1847 1848 +var _ closedts.CloseFn = (&Tracker{}).CloseFn() 1849 + 1850 // NewTracker returns a Tracker initialized to a closed timestamp of zero and 1851 // a next closed timestamp of one logical tick past zero. 1852 func NewTracker() *Tracker { 1853 t := &Tracker{} 1854 t.mu.next = hlc.Timestamp{Logical: 1} 1855 - t.mu.leftMLAI = map[roachpb.RangeID]int64{} 1856 - t.mu.rightMLAI = map[roachpb.RangeID]int64{} 1857 + t.mu.leftMLAI = map[roachpb.RangeID]ctpb.LAI{} 1858 + t.mu.rightMLAI = map[roachpb.RangeID]ctpb.LAI{} 1859 1860 return t 1861 } 1862 @@ -97,7 +101,7 @@ func (t *Tracker) String() string { 1863 1864 type item struct { 1865 rangeID roachpb.RangeID 1866 - mlai int64 1867 + mlai ctpb.LAI 1868 left bool 1869 } 1870 1871 @@ -153,12 +157,12 @@ func (t *Tracker) String() string { 1872 // like a successful call that happens to not return any new information). 1873 // Similarly, failure to provide a timestamp strictly larger than that to be 1874 // closed out next results in the same "idempotent" return values. 1875 -func (t *Tracker) Close(next hlc.Timestamp) (hlc.Timestamp, map[roachpb.RangeID]int64) { 1876 +func (t *Tracker) Close(next hlc.Timestamp) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI) { 1877 t.mu.Lock() 1878 defer t.mu.Unlock() 1879 1880 var closed hlc.Timestamp 1881 - var mlai map[roachpb.RangeID]int64 1882 + var mlai map[roachpb.RangeID]ctpb.LAI 1883 if log.V(3) { 1884 log.Infof(context.TODO(), "close: leftRef=%d rightRef=%d next=%s closed=%s new=%s", t.mu.leftRef, t.mu.rightRef, t.mu.next, t.mu.closed, next) 1885 } 1886 @@ -181,7 +185,7 @@ func (t *Tracker) Close(next hlc.Timestamp) (hlc.Timestamp, map[roachpb.RangeID] 1887 // everything that's in-flight). 1888 t.mu.leftMLAI = t.mu.rightMLAI 1889 t.mu.leftRef = t.mu.rightRef 1890 - t.mu.rightMLAI = map[roachpb.RangeID]int64{} 1891 + t.mu.rightMLAI = map[roachpb.RangeID]ctpb.LAI{} 1892 t.mu.rightRef = 0 1893 1894 t.mu.next = next 1895 @@ -204,7 +208,7 @@ func (t *Tracker) Close(next hlc.Timestamp) (hlc.Timestamp, map[roachpb.RangeID] 1896 // arguments once after a regular call. 1897 func (t *Tracker) Track( 1898 ctx context.Context, 1899 -) (hlc.Timestamp, func(context.Context, roachpb.RangeID, int64)) { 1900 +) (hlc.Timestamp, func(context.Context, roachpb.RangeID, ctpb.LAI)) { 1901 shouldLog := log.V(3) 1902 1903 t.mu.Lock() 1904 @@ -217,7 +221,7 @@ func (t *Tracker) Track( 1905 } 1906 1907 var calls int 1908 - release := func(ctx context.Context, rangeID roachpb.RangeID, lai int64) { 1909 + release := func(ctx context.Context, rangeID roachpb.RangeID, lai ctpb.LAI) { 1910 calls++ 1911 if calls != 1 { 1912 if lai != 0 || rangeID != 0 || calls > 2 { 1913 @@ -265,3 +269,10 @@ func (t *Tracker) Track( 1914 1915 return minProp, release 1916 } 1917 + 1918 +// CloseFn returns this Tracker's Close method as a CloseFn. 1919 +func (t *Tracker) CloseFn() closedts.CloseFn { 1920 + return func(next hlc.Timestamp) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI) { 1921 + return t.Close(next) 1922 + } 1923 +} 1924 diff --git a/pkg/storage/minprop/tracker_test.go b/pkg/storage/closedts/minprop/tracker_test.go 1925 similarity index 92% 1926 rename from pkg/storage/minprop/tracker_test.go 1927 rename to pkg/storage/closedts/minprop/tracker_test.go 1928 index 087cefe8cf4..296200fb85c 100644 1929 --- a/pkg/storage/minprop/tracker_test.go 1930 +++ b/pkg/storage/closedts/minprop/tracker_test.go 1931 @@ -22,6 +22,7 @@ import ( 1932 "testing" 1933 1934 "github.com/cockroachdb/cockroach/pkg/roachpb" 1935 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 1936 "github.com/cockroachdb/cockroach/pkg/util/hlc" 1937 "github.com/cockroachdb/cockroach/pkg/util/log" 1938 "github.com/cockroachdb/cockroach/pkg/util/syncutil" 1939 @@ -100,9 +101,9 @@ type modelClient struct { 1940 lai map[roachpb.RangeID]*int64 // read-only map, values accessed atomically 1941 mu struct { 1942 syncutil.Mutex 1943 - closed []hlc.Timestamp // closed timestamps 1944 - released []map[roachpb.RangeID]int64 // known released LAIs, rotated on Close 1945 - m map[roachpb.RangeID]int64 // max over all maps returned from Close() 1946 + closed []hlc.Timestamp // closed timestamps 1947 + released []map[roachpb.RangeID]ctpb.LAI // known released LAIs, rotated on Close 1948 + m map[roachpb.RangeID]ctpb.LAI // max over all maps returned from Close() 1949 } 1950 } 1951 1952 @@ -120,18 +121,18 @@ func TestTrackerConcurrentUse(t *testing.T) { 1953 ) 1954 1955 var mc modelClient 1956 - mc.mu.m = map[roachpb.RangeID]int64{} 1957 + mc.mu.m = map[roachpb.RangeID]ctpb.LAI{} 1958 mc.mu.closed = make([]hlc.Timestamp, 1) 1959 - mc.mu.released = []map[roachpb.RangeID]int64{{}, {}, {}} 1960 + mc.mu.released = []map[roachpb.RangeID]ctpb.LAI{{}, {}, {}} 1961 1962 mc.lai = map[roachpb.RangeID]*int64{} 1963 for i := roachpb.RangeID(1); i <= numRanges; i++ { 1964 mc.lai[i] = new(int64) 1965 } 1966 1967 - get := func(i int) (roachpb.RangeID, int64) { 1968 + get := func(i int) (roachpb.RangeID, ctpb.LAI) { 1969 rangeID := roachpb.RangeID(1 + (i % numRanges)) 1970 - return rangeID, atomic.AddInt64(mc.lai[rangeID], 1) 1971 + return rangeID, ctpb.LAI(atomic.AddInt64(mc.lai[rangeID], 1)) 1972 } 1973 1974 // It becomes a lot more complicated to collect the released indexes 1975 @@ -182,7 +183,7 @@ func TestTrackerConcurrentUse(t *testing.T) { 1976 // weaken the test overall. 1977 released := mc.mu.released[len(mc.mu.released)-3] 1978 // Rotate released commands bucket. 1979 - mc.mu.released = append(mc.mu.released, map[roachpb.RangeID]int64{}) 1980 + mc.mu.released = append(mc.mu.released, map[roachpb.RangeID]ctpb.LAI{}) 1981 1982 for rangeID, mlai := range m { 1983 // Intuitively you expect mc.mu.m[rangeID] < mlai, but this 1984 @@ -230,7 +231,7 @@ func TestTrackerConcurrentUse(t *testing.T) { 1985 runtime.Gosched() 1986 1987 var rangeID roachpb.RangeID 1988 - var lai int64 1989 + var lai ctpb.LAI 1990 switch i % 3 { 1991 case 0: 1992 // Successful evaluation. 1993 @@ -283,7 +284,7 @@ func TestTrackerConcurrentUse(t *testing.T) { 1994 t.Log(tracker) 1995 1996 for rangeID, addr := range mc.lai { 1997 - assignedMLAI := atomic.LoadInt64(addr) 1998 + assignedMLAI := ctpb.LAI(atomic.LoadInt64(addr)) 1999 mlai := mc.mu.m[rangeID] 2000 2001 if assignedMLAI > mlai { 2002 diff --git a/pkg/storage/closedts/provider/provider.go b/pkg/storage/closedts/provider/provider.go 2003 new file mode 100644 2004 index 00000000000..9cfe2614f13 2005 --- /dev/null 2006 +++ b/pkg/storage/closedts/provider/provider.go 2007 @@ -0,0 +1,283 @@ 2008 +// Copyright 2018 The Cockroach Authors. 2009 +// 2010 +// Licensed under the Apache License, Version 2.0 (the "License"); 2011 +// you may not use this file except in compliance with the License. 2012 +// You may obtain a copy of the License at 2013 +// 2014 +// http://www.apache.org/licenses/LICENSE-2.0 2015 +// 2016 +// Unless required by applicable law or agreed to in writing, software 2017 +// distributed under the License is distributed on an "AS IS" BASIS, 2018 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 2019 +// implied. See the License for the specific language governing 2020 +// permissions and limitations under the License. 2021 + 2022 +package provider 2023 + 2024 +import ( 2025 + "context" 2026 + "sync" 2027 + "time" 2028 + 2029 + "github.com/cockroachdb/cockroach/pkg/roachpb" 2030 + "github.com/cockroachdb/cockroach/pkg/settings/cluster" 2031 + "github.com/cockroachdb/cockroach/pkg/storage/closedts" 2032 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 2033 + "github.com/cockroachdb/cockroach/pkg/util/hlc" 2034 + "github.com/cockroachdb/cockroach/pkg/util/log" 2035 + "github.com/cockroachdb/cockroach/pkg/util/stop" 2036 + "github.com/cockroachdb/cockroach/pkg/util/syncutil" 2037 + "github.com/cockroachdb/cockroach/pkg/util/timeutil" 2038 +) 2039 + 2040 +// Config holds the information necessary to create a Provider. 2041 +type Config struct { 2042 + // NodeID is the ID of the node on which the Provider is housed. 2043 + NodeID roachpb.NodeID 2044 + Settings *cluster.Settings 2045 + Stopper *stop.Stopper 2046 + Storage closedts.Storage 2047 + Clock closedts.LiveClockFn 2048 + Close closedts.CloseFn 2049 +} 2050 + 2051 +type subscriber struct { 2052 + ch chan<- ctpb.Entry 2053 + queue []ctpb.Entry 2054 +} 2055 + 2056 +// Provider implements closedts.Provider. It orchestrates the flow of closed 2057 +// timestamps and lets callers check whether they can serve reads. 2058 +type Provider struct { 2059 + cfg *Config 2060 + 2061 + mu struct { 2062 + syncutil.RWMutex 2063 + subscribers []*subscriber 2064 + draining bool // tell subscribers to terminate 2065 + } 2066 + cond *sync.Cond 2067 + 2068 + everyClockLog log.EveryN 2069 +} 2070 + 2071 +var _ closedts.Provider = (*Provider)(nil) 2072 + 2073 +// NewProvider initializes a Provider, that has yet to be started. 2074 +func NewProvider(cfg *Config) *Provider { 2075 + return &Provider{ 2076 + cfg: cfg, 2077 + cond: sync.NewCond((&syncutil.RWMutex{}).RLocker()), 2078 + everyClockLog: log.Every(time.Minute), 2079 + } 2080 +} 2081 + 2082 +// Start implements closedts.Provider. 2083 +// 2084 +// TODO(tschottdorf): the closer functionality could be extracted into its own 2085 +// component, which would make the interfaces a little cleaner. Decide whether 2086 +// it's worth it during testing. 2087 +func (p *Provider) Start() { 2088 + p.cfg.Stopper.RunWorker(log.WithLogTagStr(context.Background(), "ct-closer", ""), p.runCloser) 2089 +} 2090 + 2091 +func (p *Provider) drain() { 2092 + p.mu.Lock() 2093 + p.mu.draining = true 2094 + p.mu.Unlock() 2095 + for { 2096 + p.cond.Broadcast() 2097 + p.mu.Lock() 2098 + done := true 2099 + for _, sub := range p.mu.subscribers { 2100 + done = done && sub == nil 2101 + } 2102 + p.mu.Unlock() 2103 + 2104 + if done { 2105 + return 2106 + } 2107 + } 2108 +} 2109 + 2110 +func (p *Provider) runCloser(ctx context.Context) { 2111 + // The loop below signals the subscribers, so when it exits it needs to do 2112 + // extra work to help the subscribers terminate. 2113 + defer p.drain() 2114 + 2115 + var t timeutil.Timer 2116 + defer t.Stop() 2117 + var lastEpoch ctpb.Epoch 2118 + for { 2119 + closeFraction := closedts.CloseFraction.Get(&p.cfg.Settings.SV) 2120 + targetDuration := float64(closedts.TargetDuration.Get(&p.cfg.Settings.SV)) 2121 + t.Reset(time.Duration(closeFraction * targetDuration)) 2122 + 2123 + select { 2124 + case <-p.cfg.Stopper.ShouldQuiesce(): 2125 + return 2126 + case <-ctx.Done(): 2127 + return 2128 + case <-t.C: 2129 + t.Read = true 2130 + } 2131 + 2132 + next, epoch, err := p.cfg.Clock() 2133 + 2134 + next.WallTime -= int64(targetDuration) 2135 + if err != nil { 2136 + if p.everyClockLog.ShouldLog() { 2137 + log.Warningf(ctx, "unable to move closed timestamp forward: %s", err) 2138 + } 2139 + } else { 2140 + closed, m := p.cfg.Close(next) 2141 + if log.V(1) { 2142 + log.Infof(ctx, "closed ts=%s with %+v, next closed timestamp should be %s", closed, m, next) 2143 + } 2144 + entry := ctpb.Entry{ 2145 + Epoch: lastEpoch, 2146 + ClosedTimestamp: closed, 2147 + MLAI: m, 2148 + } 2149 + // TODO(tschottdorf): this one-off between the epoch is awkward. Clock() gives us the epoch for `next` 2150 + // but the entry wants the epoch for the current closed timestamp. Better to pass both into Close and 2151 + // to get both back from it as well. 2152 + lastEpoch = epoch 2153 + 2154 + // Simulate a subscription to the local node, so that the new information 2155 + // is added to the storage (and thus becomes available to future subscribers 2156 + // as well, not only to existing ones). 2157 + // 2158 + // TODO(tschottdorf): the transport should ignore connection requests from 2159 + // the node to itself. Those connections would pointlessly loop this around 2160 + // once more. 2161 + p.cfg.Storage.Add(p.cfg.NodeID, entry) 2162 + 2163 + // Notify existing subscribers. 2164 + p.mu.Lock() 2165 + for _, sub := range p.mu.subscribers { 2166 + sub.queue = append(sub.queue, entry) 2167 + } 2168 + p.mu.Unlock() 2169 + } 2170 + 2171 + // Broadcast even if nothing new was queued, so that the subscribers 2172 + // loop to check their client's context. 2173 + p.cond.Broadcast() 2174 + } 2175 +} 2176 + 2177 +// Notify implements closedts.Notifyee. It passes the incoming stream of Entries 2178 +// to the local Storage. 2179 +func (p *Provider) Notify(nodeID roachpb.NodeID) chan<- ctpb.Entry { 2180 + ch := make(chan ctpb.Entry) 2181 + 2182 + p.cfg.Stopper.RunWorker(context.Background(), func(ctx context.Context) { 2183 + for entry := range ch { 2184 + p.cfg.Storage.Add(nodeID, entry) 2185 + } 2186 + }) 2187 + 2188 + return ch 2189 +} 2190 + 2191 +// Subscribe implements closedts.Producer. 2192 +func (p *Provider) Subscribe(ctx context.Context, ch chan<- ctpb.Entry) { 2193 + var i int 2194 + sub := &subscriber{ch, nil} 2195 + p.mu.Lock() 2196 + for i = range p.mu.subscribers { 2197 + if p.mu.subscribers[i] == nil { 2198 + p.mu.subscribers[i] = sub 2199 + } 2200 + } 2201 + if i == len(p.mu.subscribers) { 2202 + p.mu.subscribers = append(p.mu.subscribers, sub) 2203 + } 2204 + draining := p.mu.draining 2205 + p.mu.Unlock() 2206 + 2207 + defer func() { 2208 + p.mu.Lock() 2209 + p.mu.subscribers[i] = nil 2210 + p.mu.Unlock() 2211 + close(ch) 2212 + }() 2213 + 2214 + if draining { 2215 + return 2216 + } 2217 + 2218 + if log.V(1) { 2219 + log.Info(ctx, "new subscriber connected") 2220 + } 2221 + 2222 + // The subscription is already active, so any storage snapshot from now on is 2223 + // going to fully catch up the subscriber without a gap. 2224 + { 2225 + p.cfg.Storage.VisitAscending(p.cfg.NodeID, func(e ctpb.Entry) (done bool) { 2226 + select { 2227 + case ch <- e: 2228 + case <-ctx.Done(): 2229 + return true // done 2230 + } 2231 + return false // want more 2232 + }) 2233 + } 2234 + 2235 + for { 2236 + p.cond.L.Lock() 2237 + p.cond.Wait() 2238 + p.cond.L.Unlock() 2239 + 2240 + if err := ctx.Err(); err != nil { 2241 + if log.V(1) { 2242 + log.Info(ctx, err) 2243 + } 2244 + return 2245 + } 2246 + 2247 + var queue []ctpb.Entry 2248 + p.mu.RLock() 2249 + queue, p.mu.subscribers[i].queue = p.mu.subscribers[i].queue, nil 2250 + draining := p.mu.draining 2251 + p.mu.RUnlock() 2252 + 2253 + if draining { 2254 + return 2255 + } 2256 + 2257 + for _, entry := range queue { 2258 + select { 2259 + case ch <- entry: 2260 + default: 2261 + // Abort the subscription if consumer doesn't keep up. 2262 + log.Warning(ctx, "closed timestamp update subscriber did not catch up; terminating") 2263 + return 2264 + } 2265 + } 2266 + } 2267 +} 2268 + 2269 +// CanServe implements closedts.Provider. 2270 +func (p *Provider) CanServe( 2271 + nodeID roachpb.NodeID, ts hlc.Timestamp, rangeID roachpb.RangeID, epoch ctpb.Epoch, lai ctpb.LAI, 2272 +) bool { 2273 + var ok bool 2274 + p.cfg.Storage.VisitDescending(nodeID, func(entry ctpb.Entry) bool { 2275 + mlai, found := entry.MLAI[rangeID] 2276 + ctOK := !entry.ClosedTimestamp.Less(ts) 2277 + 2278 + ok = found && 2279 + ctOK && 2280 + mlai <= lai && 2281 + entry.Epoch == epoch 2282 + 2283 + // We're done either if we proved that the read is possible, or if we're 2284 + // already done looking at closed timestamps large enough to satisfy it. 2285 + done := ok || !ctOK 2286 + return done 2287 + }) 2288 + 2289 + return ok 2290 +} 2291 diff --git a/pkg/storage/closedts/setting.go b/pkg/storage/closedts/setting.go 2292 new file mode 100644 2293 index 00000000000..f4d20b0cfd4 2294 --- /dev/null 2295 +++ b/pkg/storage/closedts/setting.go 2296 @@ -0,0 +1,42 @@ 2297 +// Copyright 2018 The Cockroach Authors. 2298 +// 2299 +// Licensed under the Apache License, Version 2.0 (the "License"); 2300 +// you may not use this file except in compliance with the License. 2301 +// You may obtain a copy of the License at 2302 +// 2303 +// http://www.apache.org/licenses/LICENSE-2.0 2304 +// 2305 +// Unless required by applicable law or agreed to in writing, software 2306 +// distributed under the License is distributed on an "AS IS" BASIS, 2307 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 2308 +// implied. See the License for the specific language governing 2309 +// permissions and limitations under the License. 2310 + 2311 +package closedts 2312 + 2313 +import ( 2314 + "errors" 2315 + "time" 2316 + 2317 + "github.com/cockroachdb/cockroach/pkg/settings" 2318 +) 2319 + 2320 +// TargetDuration is the follower reads closed timestamp update target duration. 2321 +var TargetDuration = settings.RegisterNonNegativeDurationSetting( 2322 + "server.closed_timestamp.target_duration", 2323 + "if nonzero, attempt to provide closed timestamp notifications for timestamps trailing cluster time by approximately this duration", 2324 + 5*time.Second, 2325 +) 2326 + 2327 +// CloseFraction is the fraction of TargetDuration determining how often closed 2328 +// timestamp updates are to be attempted. 2329 +var CloseFraction = settings.RegisterValidatedFloatSetting( 2330 + "server.closed_timestamp.close_fraction", 2331 + "desc", 2332 + 0.2, 2333 + func(v float64) error { 2334 + if v <= 0 || v > 1 { 2335 + return errors.New("value not between zero and one") 2336 + } 2337 + return nil 2338 + }) 2339 diff --git a/pkg/storage/closedts/storage/storage.go b/pkg/storage/closedts/storage/storage.go 2340 new file mode 100644 2341 index 00000000000..f7f5fc3a73d 2342 --- /dev/null 2343 +++ b/pkg/storage/closedts/storage/storage.go 2344 @@ -0,0 +1,149 @@ 2345 +// Copyright 2018 The Cockroach Authors. 2346 +// 2347 +// Licensed under the Apache License, Version 2.0 (the "License"); 2348 +// you may not use this file except in compliance with the License. 2349 +// You may obtain a copy of the License at 2350 +// 2351 +// http://www.apache.org/licenses/LICENSE-2.0 2352 +// 2353 +// Unless required by applicable law or agreed to in writing, software 2354 +// distributed under the License is distributed on an "AS IS" BASIS, 2355 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 2356 +// implied. See the License for the specific language governing 2357 +// permissions and limitations under the License. 2358 + 2359 +package storage 2360 + 2361 +import ( 2362 + "fmt" 2363 + 2364 + "unsafe" 2365 + 2366 + "bytes" 2367 + "sort" 2368 + 2369 + "github.com/cockroachdb/cockroach/pkg/roachpb" 2370 + "github.com/cockroachdb/cockroach/pkg/storage/closedts" 2371 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 2372 + "github.com/cockroachdb/cockroach/pkg/util/syncutil" 2373 +) 2374 + 2375 +// SingleStorage stores and manages closed timestamp updates originating from a 2376 +// single source (i.e. node). A SingleStorage internally maintains multiple 2377 +// buckets for historical closed timestamp information. The reason for this is 2378 +// twofold: 2379 +// 2380 +// 1. The most recent closed timestamp update is also the hardest to prove a 2381 +// read for, since it comes with larger minimum lease applied indexes. In 2382 +// situations in which followers are lagging behind with their command 2383 +// application, this could lead to a runaway scenario, in which a closed 2384 +// timestamp update can never be used until it is replaced by a new one, which 2385 +// in turn also will never be used, etc. Instead, a SingleStorage keeps some 2386 +// amount of history and upstream systems can try to prove a follower read using 2387 +// an older closed timestamp instead. 2388 +// 2389 +// 2. Follower reads can be used to implement recovery of a consistent 2390 +// cluster-wide snapshot after catastrophic loss of quorum. To do this, the 2391 +// mechanism must locate at least one replica of every range in the cluster, and 2392 +// for each range find the largest possible timestamp at which follower reads 2393 +// are possible among the surviving replicas. Of all these per-range timestamps, 2394 +// the smallest can be used to read from all ranges, resulting in a consistent 2395 +// snapshot. This makes it crucial that every replica can serve at least some 2396 +// follower reads, even when regularly outpaced by the closed timestamp 2397 +// frontier. Emitted MLAIs may never even be proposed to Raft in the event of 2398 +// an ill-timed crash, and so historic information is invaluable. 2399 +// 2400 +// TODO(tschottdorf): revisit whether this shouldn't be a concrete impl instead, 2401 +// with only the buckets abstracted out. 2402 +type SingleStorage interface { 2403 + fmt.Stringer 2404 + // VisitAscending walks through the buckets of the storage in ascending 2405 + // closed timestamp order, until the closure returns true (or all buckets 2406 + // have been visited). 2407 + VisitAscending(func(ctpb.Entry) (done bool)) 2408 + // VisitDescending walks through the buckets of the storage in descending 2409 + // closed timestamp order, until the closure returns true (or all buckets 2410 + // have been visited). 2411 + VisitDescending(func(ctpb.Entry) (done bool)) 2412 + // Add adds a new Entry to this storage. The entry is added to the most 2413 + // recent bucket and remaining buckets are rotated as indicated by their age 2414 + // relative to the newly added Entry. 2415 + Add(ctpb.Entry) 2416 +} 2417 + 2418 +type entry struct { 2419 + SingleStorage 2420 +} 2421 + 2422 +// MultiStorage implements the closedts.Storage interface. 2423 +type MultiStorage struct { 2424 + // constructor creates a SingleStorage whenever one is initialized for a new 2425 + // NodeID. 2426 + constructor func() SingleStorage 2427 + // TODO(tschottdorf): clean up storages that haven't been used for extended 2428 + // periods of time. 2429 + m syncutil.IntMap 2430 +} 2431 + 2432 +var _ closedts.Storage = (*MultiStorage)(nil) 2433 + 2434 +// NewMultiStorage sets up a MultiStorage which uses the given factory method 2435 +// for setting up the SingleStorage used for each individual NodeID for which 2436 +// operations are received. 2437 +func NewMultiStorage(constructor func() SingleStorage) *MultiStorage { 2438 + return &MultiStorage{constructor: constructor} 2439 +} 2440 + 2441 +func (ms *MultiStorage) getOrCreate(nodeID roachpb.NodeID) SingleStorage { 2442 + key := int64(nodeID) 2443 + p, found := ms.m.Load(key) 2444 + if found { 2445 + // Fast path that avoids calling f(). 2446 + return (*entry)(p).SingleStorage 2447 + } 2448 + 2449 + ss := ms.constructor() 2450 + p, _ = ms.m.LoadOrStore(key, unsafe.Pointer(&entry{ss})) 2451 + return (*entry)(p).SingleStorage 2452 +} 2453 + 2454 +// VisitAscending implements closedts.Storage. 2455 +func (ms *MultiStorage) VisitAscending(nodeID roachpb.NodeID, f func(ctpb.Entry) (done bool)) { 2456 + ss := ms.getOrCreate(nodeID) 2457 + ss.VisitAscending(f) 2458 +} 2459 + 2460 +// VisitDescending implements closedts.Storage. 2461 +func (ms *MultiStorage) VisitDescending(nodeID roachpb.NodeID, f func(ctpb.Entry) (done bool)) { 2462 + ss := ms.getOrCreate(nodeID) 2463 + ss.VisitDescending(f) 2464 +} 2465 + 2466 +// Add implements closedts.Storage. 2467 +func (ms *MultiStorage) Add(nodeID roachpb.NodeID, entry ctpb.Entry) { 2468 + ss := ms.getOrCreate(nodeID) 2469 + ss.Add(entry) 2470 +} 2471 + 2472 +// String prints a tabular rundown of the contents of the MultiStorage. 2473 +func (ms *MultiStorage) String() string { 2474 + type tuple struct { 2475 + roachpb.NodeID 2476 + SingleStorage 2477 + } 2478 + 2479 + var sl []tuple 2480 + ms.m.Range(func(k int64, p unsafe.Pointer) bool { 2481 + sl = append(sl, tuple{roachpb.NodeID(k), (*entry)(p).SingleStorage}) 2482 + return true // want more 2483 + }) 2484 + sort.Slice(sl, func(i, j int) bool { 2485 + return sl[i].NodeID < sl[j].NodeID 2486 + }) 2487 + var buf bytes.Buffer 2488 + for i := range sl { 2489 + buf.WriteString(fmt.Sprintf("***** n%d *****\n", sl[i].NodeID)) 2490 + buf.WriteString(sl[i].SingleStorage.String()) 2491 + } 2492 + return buf.String() 2493 +} 2494 diff --git a/pkg/storage/closedts/storage/storage_mem.go b/pkg/storage/closedts/storage/storage_mem.go 2495 new file mode 100644 2496 index 00000000000..ea8aa5be009 2497 --- /dev/null 2498 +++ b/pkg/storage/closedts/storage/storage_mem.go 2499 @@ -0,0 +1,199 @@ 2500 +// Copyright 2018 The Cockroach Authors. 2501 +// 2502 +// Licensed under the Apache License, Version 2.0 (the "License"); 2503 +// you may not use this file except in compliance with the License. 2504 +// You may obtain a copy of the License at 2505 +// 2506 +// http://www.apache.org/licenses/LICENSE-2.0 2507 +// 2508 +// Unless required by applicable law or agreed to in writing, software 2509 +// distributed under the License is distributed on an "AS IS" BASIS, 2510 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 2511 +// implied. See the License for the specific language governing 2512 +// permissions and limitations under the License. 2513 + 2514 +package storage 2515 + 2516 +import ( 2517 + "bytes" 2518 + "fmt" 2519 + "sort" 2520 + "strconv" 2521 + "strings" 2522 + "time" 2523 + 2524 + "github.com/cockroachdb/cockroach/pkg/roachpb" 2525 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 2526 + "github.com/cockroachdb/cockroach/pkg/util/syncutil" 2527 + "github.com/olekukonko/tablewriter" 2528 +) 2529 + 2530 +type memStorage struct { 2531 + mu struct { 2532 + syncutil.RWMutex 2533 + buckets []ctpb.Entry 2534 + scale time.Duration 2535 + } 2536 +} 2537 + 2538 +var _ SingleStorage = (*memStorage)(nil) 2539 + 2540 +// NewMemStorage initializes a SingleStorage backed by an in-memory slice that 2541 +// represents the given number of buckets, where the i-th bucket holds a closed 2542 +// timestamp approximately 2^i*scale in the past. 2543 +func NewMemStorage(scale time.Duration, buckets int) SingleStorage { 2544 + m := &memStorage{} 2545 + m.mu.buckets = make([]ctpb.Entry, buckets) 2546 + m.mu.scale = scale 2547 + return m 2548 +} 2549 + 2550 +func (m *memStorage) String() string { 2551 + m.mu.RLock() 2552 + defer m.mu.RUnlock() 2553 + 2554 + var buf bytes.Buffer 2555 + tw := tablewriter.NewWriter(&buf) 2556 + 2557 + header := make([]string, 1+len(m.mu.buckets)) 2558 + header[0] = "" 2559 + align := make([]int, 1+len(m.mu.buckets)) 2560 + align[0] = tablewriter.ALIGN_LEFT 2561 + 2562 + for i := range m.mu.buckets { 2563 + header[1+i] = m.mu.buckets[i].ClosedTimestamp.String() + "\nage=" + time.Duration( 2564 + m.mu.buckets[0].ClosedTimestamp.WallTime-m.mu.buckets[i].ClosedTimestamp.WallTime, 2565 + ).String() + " (target ≤" + m.bucketMaxAge(i).String() + ")\nepoch=" + fmt.Sprintf("%d", m.mu.buckets[i].Epoch) 2566 + align[1+i] = tablewriter.ALIGN_RIGHT 2567 + } 2568 + tw.SetAutoFormatHeaders(false) 2569 + tw.SetColumnAlignment(align) 2570 + tw.SetHeader(header) 2571 + tw.SetHeaderLine(true) 2572 + tw.SetRowLine(false) 2573 + tw.SetColumnSeparator(" ") 2574 + tw.SetBorder(true) 2575 + 2576 + rangeIDs := make([]roachpb.RangeID, 0, len(m.mu.buckets[0].MLAI)) 2577 + for rangeID := range m.mu.buckets[0].MLAI { 2578 + rangeIDs = append(rangeIDs, rangeID) 2579 + } 2580 + sort.Slice(rangeIDs, func(i, j int) bool { 2581 + return rangeIDs[i] < rangeIDs[j] 2582 + }) 2583 + 2584 + row := make([]string, 1+len(m.mu.buckets)) 2585 + for _, rangeID := range rangeIDs { 2586 + row[0] = "r" + strconv.FormatInt(int64(rangeID), 10) 2587 + for i, entry := range m.mu.buckets { 2588 + lai, ok := entry.MLAI[rangeID] 2589 + if ok { 2590 + row[1+i] = strconv.FormatInt(int64(lai), 10) 2591 + } else { 2592 + row[1+i] = "" 2593 + } 2594 + } 2595 + tw.Append(row) 2596 + } 2597 + 2598 + tw.Render() 2599 + 2600 + // It's apparently impossible to write passing Example tests when 2601 + // intermediate lines have trailing whitespace (💩), so remove all of 2602 + // that. 2603 + // 2604 + // See https://github.com/golang/go/issues/6416. 2605 + s := strings.Split(buf.String(), "\n") 2606 + for i := range s { 2607 + s[i] = strings.TrimRight(s[i], " ") 2608 + } 2609 + return strings.Join(s, "\n") 2610 + 2611 +} 2612 + 2613 +func (m *memStorage) bucketMaxAge(index int) time.Duration { 2614 + if index == 0 { 2615 + return 0 2616 + } 2617 + return (1 << uint(index-1)) * m.mu.scale 2618 +} 2619 + 2620 +func (m *memStorage) Add(e ctpb.Entry) { 2621 + m.mu.Lock() 2622 + defer m.mu.Unlock() 2623 + 2624 + now := e.ClosedTimestamp.WallTime 2625 + 2626 + for i := 0; i < len(m.mu.buckets); i++ { 2627 + if time.Duration(now-m.mu.buckets[i].ClosedTimestamp.WallTime) <= m.bucketMaxAge(i) { 2628 + break 2629 + } 2630 + mergedEntry := merge(m.mu.buckets[i], e) 2631 + e = m.mu.buckets[i] 2632 + m.mu.buckets[i] = mergedEntry 2633 + } 2634 +} 2635 + 2636 +func (m *memStorage) VisitAscending(f func(ctpb.Entry) (done bool)) { 2637 + m.mu.RLock() 2638 + defer m.mu.RUnlock() 2639 + 2640 + for i := len(m.mu.buckets) - 1; i >= 0; i-- { 2641 + entry := m.mu.buckets[i] 2642 + if entry.Epoch == 0 { 2643 + // Skip empty buckets. 2644 + continue 2645 + } 2646 + if f(entry) { 2647 + return 2648 + } 2649 + } 2650 +} 2651 + 2652 +func (m *memStorage) VisitDescending(f func(ctpb.Entry) (done bool)) { 2653 + m.mu.RLock() 2654 + defer m.mu.RUnlock() 2655 + 2656 + for l, i := len(m.mu.buckets), 0; i < l; i++ { 2657 + entry := m.mu.buckets[i] 2658 + // Stop once we hit an empty bucket (which implies that all further buckets 2659 + // are also empty), or once the visitor is satisfied. 2660 + if entry.Epoch == 0 || f(entry) { 2661 + return 2662 + } 2663 + } 2664 +} 2665 + 2666 +func merge(e, ee ctpb.Entry) ctpb.Entry { 2667 + // TODO(tschottdorf): if either of these hit, check that what we're 2668 + // returning has Full set. If we make it past, check that either of 2669 + // them has it set. The first Entry the Storage sees for an epoch must have it 2670 + // set, so the assertions should never fire. 2671 + if e.Epoch < ee.Epoch { 2672 + return ee 2673 + } else if e.Epoch > ee.Epoch { 2674 + return e 2675 + } 2676 + 2677 + // Epochs match, so we can actually update. 2678 + 2679 + // Initialize re as a deep copy of e. 2680 + re := e 2681 + re.MLAI = map[roachpb.RangeID]ctpb.LAI{} 2682 + for rangeID, mlai := range e.MLAI { 2683 + re.MLAI[rangeID] = mlai 2684 + } 2685 + // The result is full if either operand is. 2686 + re.Full = e.Full || ee.Full 2687 + 2688 + // Use the larger of both timestamps with the union of the MLAIs, preferring larger 2689 + // ones on conflict. 2690 + re.ClosedTimestamp.Forward(ee.ClosedTimestamp) 2691 + for rangeID, mlai := range ee.MLAI { 2692 + if re.MLAI[rangeID] < mlai { 2693 + re.MLAI[rangeID] = mlai 2694 + } 2695 + } 2696 + 2697 + return re 2698 +} 2699 diff --git a/pkg/storage/closedts/storage/storage_test.go b/pkg/storage/closedts/storage/storage_test.go 2700 new file mode 100644 2701 index 00000000000..7999ac698a1 2702 --- /dev/null 2703 +++ b/pkg/storage/closedts/storage/storage_test.go 2704 @@ -0,0 +1,428 @@ 2705 +// Copyright 2018 The Cockroach Authors. 2706 +// 2707 +// Licensed under the Apache License, Version 2.0 (the "License"); 2708 +// you may not use this file except in compliance with the License. 2709 +// You may obtain a copy of the License at 2710 +// 2711 +// http://www.apache.org/licenses/LICENSE-2.0 2712 +// 2713 +// Unless required by applicable law or agreed to in writing, software 2714 +// distributed under the License is distributed on an "AS IS" BASIS, 2715 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 2716 +// implied. See the License for the specific language governing 2717 +// permissions and limitations under the License. 2718 + 2719 +package storage 2720 + 2721 +import ( 2722 + "fmt" 2723 + "math/rand" 2724 + "testing" 2725 + "time" 2726 + 2727 + "github.com/pkg/errors" 2728 + 2729 + "golang.org/x/sync/errgroup" 2730 + 2731 + "github.com/cockroachdb/cockroach/pkg/roachpb" 2732 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 2733 + "github.com/cockroachdb/cockroach/pkg/util/hlc" 2734 + "github.com/cockroachdb/cockroach/pkg/util/leaktest" 2735 + "github.com/cockroachdb/cockroach/pkg/util/randutil" 2736 +) 2737 + 2738 +func ExampleSingleStorage() { 2739 + s := NewMemStorage(10*time.Second, 4) 2740 + fmt.Println("The empty storage renders as below:") 2741 + fmt.Println(s) 2742 + 2743 + fmt.Println("After adding the following entry:") 2744 + e1 := ctpb.Entry{ 2745 + Full: true, 2746 + ClosedTimestamp: hlc.Timestamp{WallTime: 123E9}, 2747 + MLAI: map[roachpb.RangeID]ctpb.LAI{ 2748 + 1: 1000, 2749 + 9: 2000, 2750 + }, 2751 + } 2752 + fmt.Println(e1) 2753 + s.Add(e1) 2754 + fmt.Println("the result is:") 2755 + fmt.Println(s) 2756 + fmt.Println("Note how the most recent bucket picked up the update.") 2757 + 2758 + fmt.Println("A new update comes in only two seconds later:") 2759 + e2 := ctpb.Entry{ 2760 + ClosedTimestamp: hlc.Timestamp{WallTime: 125E9}, 2761 + MLAI: map[roachpb.RangeID]ctpb.LAI{ 2762 + 1: 1001, 2763 + 7: 12, 2764 + }, 2765 + } 2766 + fmt.Println(e2) 2767 + s.Add(e2) 2768 + fmt.Println("The first bucket now contains the union of both updates.") 2769 + fmt.Println("The second bucket holds on to the previous value of the first.") 2770 + fmt.Println("The remaining buckets are unchanged. The best we could do is") 2771 + fmt.Println("give them identical copies of the second, but that's nonsense.") 2772 + fmt.Println(s) 2773 + 2774 + fmt.Println("Another update, another eight seconds later:") 2775 + e3 := ctpb.Entry{ 2776 + ClosedTimestamp: hlc.Timestamp{WallTime: 133E9}, 2777 + MLAI: map[roachpb.RangeID]ctpb.LAI{ 2778 + 9: 2020, 2779 + 1: 999, 2780 + }, 2781 + } 2782 + fmt.Println(e3) 2783 + s.Add(e3) 2784 + fmt.Println("Note how the second bucket didn't rotate, for it is not yet") 2785 + fmt.Println("older than 10s. Note also how the first bucket ignores the") 2786 + fmt.Println("downgrade for r1; these can occur in practice.") 2787 + fmt.Println(s) 2788 + 2789 + fmt.Println("Half a second later, with the next update, it will rotate:") 2790 + e4 := ctpb.Entry{ 2791 + ClosedTimestamp: hlc.Timestamp{WallTime: 133E9 + 1E9/2}, 2792 + MLAI: map[roachpb.RangeID]ctpb.LAI{ 2793 + 7: 17, 2794 + 8: 711, 2795 + }, 2796 + } 2797 + fmt.Println(e4) 2798 + s.Add(e4) 2799 + fmt.Println("Consequently we now see the third bucket fill up.") 2800 + fmt.Println(s) 2801 + 2802 + fmt.Println("Next update arrives a whopping 46.5s later (why not).") 2803 + e5 := ctpb.Entry{ 2804 + ClosedTimestamp: hlc.Timestamp{WallTime: 180E9}, 2805 + MLAI: map[roachpb.RangeID]ctpb.LAI{ 2806 + 1: 1004, 2807 + 7: 19, 2808 + 2: 929922, 2809 + }, 2810 + } 2811 + fmt.Println(e5) 2812 + s.Add(e5) 2813 + fmt.Println("The second bucket rotated, but due to the sparseness of updates,") 2814 + fmt.Println("it's still above its target age and will rotate again next time.") 2815 + fmt.Println("The same is true for the remaining buckets.") 2816 + fmt.Println(s) 2817 + 2818 + fmt.Println("Another five seconds later, another update:") 2819 + e6 := ctpb.Entry{ 2820 + ClosedTimestamp: hlc.Timestamp{WallTime: 185E9}, 2821 + MLAI: map[roachpb.RangeID]ctpb.LAI{ 2822 + 3: 1771, 2823 + }, 2824 + } 2825 + fmt.Println(e6) 2826 + s.Add(e6) 2827 + fmt.Println("All buckets rotate, but the third and fourth remain over target age.") 2828 + fmt.Println("This would resolve itself if reasonably spaced updates kept coming in.") 2829 + fmt.Println(s) 2830 + 2831 + // Output: 2832 + // The empty storage renders as below: 2833 + // +--+---------------------+----------------------+----------------------+----------------------+ 2834 + // 0.000000000,0 0.000000000,0 0.000000000,0 0.000000000,0 2835 + // age=0s (target ≤0s) age=0s (target ≤10s) age=0s (target ≤20s) age=0s (target ≤40s) 2836 + // epoch=0 epoch=0 epoch=0 epoch=0 2837 + // +--+---------------------+----------------------+----------------------+----------------------+ 2838 + // +--+---------------------+----------------------+----------------------+----------------------+ 2839 + // 2840 + // After adding the following entry: 2841 + // CT: 123.000000000,0 @ Epoch 0 2842 + // Full: true 2843 + // MLAI: r1: 1000, r9: 2000 2844 + // 2845 + // the result is: 2846 + // +----+---------------------+------------------------+------------------------+------------------------+ 2847 + // 123.000000000,0 0.000000000,0 age=2m3s 0.000000000,0 age=2m3s 0.000000000,0 age=2m3s 2848 + // age=0s (target ≤0s) (target ≤10s) epoch=0 (target ≤20s) epoch=0 (target ≤40s) epoch=0 2849 + // epoch=0 2850 + // +----+---------------------+------------------------+------------------------+------------------------+ 2851 + // r1 1000 2852 + // r9 2000 2853 + // +----+---------------------+------------------------+------------------------+------------------------+ 2854 + // 2855 + // Note how the most recent bucket picked up the update. 2856 + // A new update comes in only two seconds later: 2857 + // CT: 125.000000000,0 @ Epoch 0 2858 + // Full: false 2859 + // MLAI: r1: 1001, r7: 12 2860 + // 2861 + // The first bucket now contains the union of both updates. 2862 + // The second bucket holds on to the previous value of the first. 2863 + // The remaining buckets are unchanged. The best we could do is 2864 + // give them identical copies of the second, but that's nonsense. 2865 + // +----+---------------------+----------------------+------------------------+------------------------+ 2866 + // 125.000000000,0 123.000000000,0 0.000000000,0 age=2m5s 0.000000000,0 age=2m5s 2867 + // age=0s (target ≤0s) age=2s (target ≤10s) (target ≤20s) epoch=0 (target ≤40s) epoch=0 2868 + // epoch=0 epoch=0 2869 + // +----+---------------------+----------------------+------------------------+------------------------+ 2870 + // r1 1001 1000 2871 + // r7 12 2872 + // r9 2000 2000 2873 + // +----+---------------------+----------------------+------------------------+------------------------+ 2874 + // 2875 + // Another update, another eight seconds later: 2876 + // CT: 133.000000000,0 @ Epoch 0 2877 + // Full: false 2878 + // MLAI: r1: 999, r9: 2020 2879 + // 2880 + // Note how the second bucket didn't rotate, for it is not yet 2881 + // older than 10s. Note also how the first bucket ignores the 2882 + // downgrade for r1; these can occur in practice. 2883 + // +----+---------------------+-----------------------+-------------------------+-------------------------+ 2884 + // 133.000000000,0 123.000000000,0 0.000000000,0 age=2m13s 0.000000000,0 age=2m13s 2885 + // age=0s (target ≤0s) age=10s (target ≤10s) (target ≤20s) epoch=0 (target ≤40s) epoch=0 2886 + // epoch=0 epoch=0 2887 + // +----+---------------------+-----------------------+-------------------------+-------------------------+ 2888 + // r1 1001 1000 2889 + // r7 12 2890 + // r9 2020 2000 2891 + // +----+---------------------+-----------------------+-------------------------+-------------------------+ 2892 + // 2893 + // Half a second later, with the next update, it will rotate: 2894 + // CT: 133.500000000,0 @ Epoch 0 2895 + // Full: false 2896 + // MLAI: r7: 17, r8: 711 2897 + // 2898 + // Consequently we now see the third bucket fill up. 2899 + // +----+---------------------+-------------------------+-------------------------+---------------------------+ 2900 + // 133.500000000,0 133.000000000,0 123.000000000,0 0.000000000,0 age=2m13.5s 2901 + // age=0s (target ≤0s) age=500ms (target ≤10s) age=10.5s (target ≤20s) (target ≤40s) epoch=0 2902 + // epoch=0 epoch=0 epoch=0 2903 + // +----+---------------------+-------------------------+-------------------------+---------------------------+ 2904 + // r1 1001 1001 1000 2905 + // r7 17 12 2906 + // r8 711 2907 + // r9 2020 2020 2000 2908 + // +----+---------------------+-------------------------+-------------------------+---------------------------+ 2909 + // 2910 + // Next update arrives a whopping 46.5s later (why not). 2911 + // CT: 180.000000000,0 @ Epoch 0 2912 + // Full: false 2913 + // MLAI: r1: 1004, r2: 929922, r7: 19 2914 + // 2915 + // The second bucket rotated, but due to the sparseness of updates, 2916 + // it's still above its target age and will rotate again next time. 2917 + // The same is true for the remaining buckets. 2918 + // +----+---------------------+-------------------------+-----------------------+-----------------------+ 2919 + // 180.000000000,0 133.500000000,0 133.000000000,0 123.000000000,0 2920 + // age=0s (target ≤0s) age=46.5s (target ≤10s) age=47s (target ≤20s) age=57s (target ≤40s) 2921 + // epoch=0 epoch=0 epoch=0 epoch=0 2922 + // +----+---------------------+-------------------------+-----------------------+-----------------------+ 2923 + // r1 1004 1001 1001 1000 2924 + // r2 929922 2925 + // r7 19 17 12 2926 + // r8 711 711 2927 + // r9 2020 2020 2020 2000 2928 + // +----+---------------------+-------------------------+-----------------------+-----------------------+ 2929 + // 2930 + // Another five seconds later, another update: 2931 + // CT: 185.000000000,0 @ Epoch 0 2932 + // Full: false 2933 + // MLAI: r3: 1771 2934 + // 2935 + // All buckets rotate, but the third and fourth remain over target age. 2936 + // This would resolve itself if reasonably spaced updates kept coming in. 2937 + // +----+---------------------+----------------------+-------------------------+-----------------------+ 2938 + // 185.000000000,0 180.000000000,0 133.500000000,0 133.000000000,0 2939 + // age=0s (target ≤0s) age=5s (target ≤10s) age=51.5s (target ≤20s) age=52s (target ≤40s) 2940 + // epoch=0 epoch=0 epoch=0 epoch=0 2941 + // +----+---------------------+----------------------+-------------------------+-----------------------+ 2942 + // r1 1004 1004 1001 1001 2943 + // r2 929922 929922 2944 + // r3 1771 2945 + // r7 19 19 17 12 2946 + // r8 711 711 711 2947 + // r9 2020 2020 2020 2020 2948 + // +----+---------------------+----------------------+-------------------------+-----------------------+ 2949 +} 2950 + 2951 +func ExampleMultiStorage_epoch() { 2952 + ms := NewMultiStorage(func() SingleStorage { 2953 + return NewMemStorage(time.Millisecond, 2) 2954 + }) 2955 + 2956 + e1 := ctpb.Entry{ 2957 + Epoch: 10, 2958 + ClosedTimestamp: hlc.Timestamp{WallTime: 1E9}, 2959 + MLAI: map[roachpb.RangeID]ctpb.LAI{ 2960 + 9: 17, 2961 + }, 2962 + } 2963 + fmt.Println("First, the following entry is added:") 2964 + fmt.Println(e1) 2965 + ms.Add(1, e1) 2966 + fmt.Println(ms) 2967 + 2968 + fmt.Println("The epoch changes. It can only increase, for we receive Entries in a fixed order.") 2969 + e2 := ctpb.Entry{ 2970 + Epoch: 11, 2971 + ClosedTimestamp: hlc.Timestamp{WallTime: 2E9}, 2972 + MLAI: map[roachpb.RangeID]ctpb.LAI{ 2973 + 9: 18, 2974 + 10: 99, 2975 + }, 2976 + } 2977 + ms.Add(1, e2) 2978 + fmt.Println(e2) 2979 + fmt.Println(ms) 2980 + 2981 + fmt.Println("If it *did* decrease, a higher level component should trigger an assertion.") 2982 + fmt.Println("The storage itself will simply ignore such updates:") 2983 + e3 := ctpb.Entry{ 2984 + Epoch: 8, 2985 + ClosedTimestamp: hlc.Timestamp{WallTime: 3E9}, 2986 + MLAI: map[roachpb.RangeID]ctpb.LAI{ 2987 + 9: 19, 2988 + 10: 199, 2989 + }, 2990 + } 2991 + fmt.Println(e3) 2992 + ms.Add(1, e3) 2993 + fmt.Println(ms) 2994 + 2995 + // Output: 2996 + // First, the following entry is added: 2997 + // CT: 1.000000000,0 @ Epoch 10 2998 + // Full: false 2999 + // MLAI: r9: 17 3000 + // 3001 + // ***** n1 ***** 3002 + // +----+---------------------+----------------------+ 3003 + // 1.000000000,0 0.000000000,0 3004 + // age=0s (target ≤0s) age=1s (target ≤1ms) 3005 + // epoch=10 epoch=0 3006 + // +----+---------------------+----------------------+ 3007 + // r9 17 3008 + // +----+---------------------+----------------------+ 3009 + // 3010 + // The epoch changes. It can only increase, for we receive Entries in a fixed order. 3011 + // CT: 2.000000000,0 @ Epoch 11 3012 + // Full: false 3013 + // MLAI: r9: 18, r10: 99 3014 + // 3015 + // ***** n1 ***** 3016 + // +-----+---------------------+----------------------+ 3017 + // 2.000000000,0 1.000000000,0 3018 + // age=0s (target ≤0s) age=1s (target ≤1ms) 3019 + // epoch=11 epoch=10 3020 + // +-----+---------------------+----------------------+ 3021 + // r9 18 17 3022 + // r10 99 3023 + // +-----+---------------------+----------------------+ 3024 + // 3025 + // If it *did* decrease, a higher level component should trigger an assertion. 3026 + // The storage itself will simply ignore such updates: 3027 + // CT: 3.000000000,0 @ Epoch 8 3028 + // Full: false 3029 + // MLAI: r9: 19, r10: 199 3030 + // 3031 + // ***** n1 ***** 3032 + // +-----+---------------------+----------------------+ 3033 + // 2.000000000,0 2.000000000,0 3034 + // age=0s (target ≤0s) age=0s (target ≤1ms) 3035 + // epoch=11 epoch=11 3036 + // +-----+---------------------+----------------------+ 3037 + // r9 18 18 3038 + // r10 99 99 3039 + // +-----+---------------------+----------------------+ 3040 +} 3041 + 3042 +// TestConcurrent runs a very basic sanity check against a Storage, verifiying 3043 +// that the bucketed Entries don't regress in obvious ways. 3044 +func TestConcurrent(t *testing.T) { 3045 + defer leaktest.AfterTest(t)() 3046 + 3047 + ms := NewMultiStorage(func() SingleStorage { 3048 + return NewMemStorage(time.Millisecond, 10) 3049 + }) 3050 + 3051 + var g errgroup.Group 3052 + 3053 + const ( 3054 + iters = 10 3055 + numNodes = roachpb.NodeID(2) 3056 + numRanges = roachpb.RangeID(3) 3057 + numReadersPerNode = 3 3058 + numWritersPerNode = 3 3059 + ) 3060 + 3061 + // concurrently add and read from storage 3062 + // after add: needs to be visible to future read 3063 + // read ts never regresses 3064 + globalRand, seed := randutil.NewPseudoRand() 3065 + t.Log("seed is", seed) 3066 + 3067 + for i := 0; i < numWritersPerNode; i++ { 3068 + for nodeID := roachpb.NodeID(1); nodeID <= numNodes; nodeID++ { 3069 + for i := 0; i < iters; i++ { 3070 + r := rand.New(rand.NewSource(globalRand.Int63())) 3071 + m := make(map[roachpb.RangeID]ctpb.LAI) 3072 + for rangeID := roachpb.RangeID(1); rangeID < numRanges; rangeID++ { 3073 + if r.Intn(int(numRanges)) == 0 { 3074 + continue 3075 + } 3076 + m[rangeID] = ctpb.LAI(rand.Intn(100)) 3077 + } 3078 + ct := hlc.Timestamp{WallTime: r.Int63n(100), Logical: r.Int31n(10)} 3079 + epo := ctpb.Epoch(r.Int63n(100)) 3080 + g.Go(func() error { 3081 + <-time.After(time.Duration(rand.Intn(1E7))) 3082 + ms.Add(nodeID, ctpb.Entry{ 3083 + Epoch: epo, 3084 + ClosedTimestamp: ct, 3085 + MLAI: m, 3086 + }) 3087 + return nil 3088 + }) 3089 + } 3090 + } 3091 + } 3092 + 3093 + for i := 0; i < numReadersPerNode; i++ { 3094 + for nodeID := roachpb.NodeID(1); nodeID <= numNodes; nodeID++ { 3095 + nodeID := nodeID 3096 + g.Go(func() error { 3097 + epo := ctpb.Epoch(-1) 3098 + var ct hlc.Timestamp 3099 + var mlai map[roachpb.RangeID]ctpb.LAI 3100 + var err error 3101 + var n int 3102 + ms.VisitDescending(nodeID, func(e ctpb.Entry) bool { 3103 + n++ 3104 + if n > 1 && e.Epoch > epo { 3105 + err = errors.Errorf("epoch regressed from %d to %d", epo, e.Epoch) 3106 + return true // done 3107 + } 3108 + if n > 1 && ct.Less(e.ClosedTimestamp) { 3109 + err = errors.Errorf("closed timestamp regressed from %s to %s", ct, e.ClosedTimestamp) 3110 + return true // done 3111 + } 3112 + for rangeID := roachpb.RangeID(1); rangeID <= numRanges; rangeID++ { 3113 + if l := mlai[rangeID]; l < e.MLAI[rangeID] && n > 1 { 3114 + err = errors.Errorf("MLAI for r%d regressed: %+v to %+v", rangeID, mlai, e.MLAI) 3115 + return true // done 3116 + } 3117 + } 3118 + 3119 + epo = e.Epoch 3120 + ct = e.ClosedTimestamp 3121 + mlai = e.MLAI 3122 + return false // not done 3123 + }) 3124 + return err 3125 + }) 3126 + } 3127 + } 3128 + 3129 + if err := g.Wait(); err != nil { 3130 + t.Fatal(err) 3131 + } 3132 +} 3133 diff --git a/pkg/storage/closedts/transport/clients.go b/pkg/storage/closedts/transport/clients.go 3134 new file mode 100644 3135 index 00000000000..23e2118a463 3136 --- /dev/null 3137 +++ b/pkg/storage/closedts/transport/clients.go 3138 @@ -0,0 +1,157 @@ 3139 +// Copyright 2018 The Cockroach Authors. 3140 +// 3141 +// Licensed under the Apache License, Version 2.0 (the "License"); 3142 +// you may not use this file except in compliance with the License. 3143 +// You may obtain a copy of the License at 3144 +// 3145 +// http://www.apache.org/licenses/LICENSE-2.0 3146 +// 3147 +// Unless required by applicable law or agreed to in writing, software 3148 +// distributed under the License is distributed on an "AS IS" BASIS, 3149 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 3150 +// implied. See the License for the specific language governing 3151 +// permissions and limitations under the License. 3152 + 3153 +package transport 3154 + 3155 +import ( 3156 + "context" 3157 + "unsafe" 3158 + 3159 + "github.com/cockroachdb/cockroach/pkg/roachpb" 3160 + "github.com/cockroachdb/cockroach/pkg/settings/cluster" 3161 + "github.com/cockroachdb/cockroach/pkg/storage/closedts" 3162 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 3163 + "github.com/cockroachdb/cockroach/pkg/util/log" 3164 + "github.com/cockroachdb/cockroach/pkg/util/stop" 3165 + "github.com/cockroachdb/cockroach/pkg/util/syncutil" 3166 +) 3167 + 3168 +// Config holds the information necessary to create a client registry. 3169 +type Config struct { 3170 + Settings *cluster.Settings 3171 + Stopper *stop.Stopper 3172 + Dialer closedts.Dialer 3173 + Sink closedts.Notifyee 3174 +} 3175 + 3176 +// Clients manages clients receiving closed timestamp updates from 3177 +// peer nodes, along with facilities to request information about certain 3178 +// ranges. Received updates are relayed to a provided Notifyee. 3179 +type Clients struct { 3180 + cfg Config 3181 + 3182 + // TODO(tschottdorf): remove unused clients. Perhaps expiring them after, 3183 + // say, 24h is enough? There is no interruption when doing so; the only 3184 + // price is that a full update is sent, but that is pretty cheap too. 3185 + clients syncutil.IntMap 3186 +} 3187 + 3188 +var _ closedts.ClientRegistry = (*Clients)(nil) 3189 + 3190 +// NewClients sets up a client registry. 3191 +func NewClients(cfg Config) *Clients { 3192 + return &Clients{cfg: cfg} 3193 +} 3194 + 3195 +type client struct { 3196 + mu struct { 3197 + syncutil.Mutex 3198 + requested map[roachpb.RangeID]struct{} // never nil 3199 + } 3200 +} 3201 + 3202 +// Request is called when serving a follower read has failed due to missing or 3203 +// insufficient information. By calling this method, the caller gives the 3204 +// instruction to connect to the given node (if it hasn't already) and ask it to 3205 +// send (or re-send) up-to-date information about the specified range. Having 3206 +// done so, the information should soon thereafter be available to the Sink and 3207 +// from there, further follower read attempts. Does not block. 3208 +func (pr *Clients) Request(nodeID roachpb.NodeID, rangeID roachpb.RangeID) { 3209 + if cl := pr.getOrCreateClient(nodeID); cl != nil { 3210 + cl.mu.Lock() 3211 + cl.mu.requested[rangeID] = struct{}{} 3212 + cl.mu.Unlock() 3213 + } 3214 +} 3215 + 3216 +// EnsureClient makes sure that updates from the given nodes are pulled in, if 3217 +// they aren't already. This call does not block (and is cheap). 3218 +func (pr *Clients) EnsureClient(nodeID roachpb.NodeID) { 3219 + pr.getOrCreateClient(nodeID) 3220 +} 3221 + 3222 +func (pr *Clients) getOrCreateClient(nodeID roachpb.NodeID) *client { 3223 + ctx := log.WithLogTagStr(context.Background(), "ct-client", "") 3224 + // Fast path to check for existing client without an allocation. 3225 + p, found := pr.clients.Load(int64(nodeID)) 3226 + cl := (*client)(p) 3227 + if found { 3228 + return cl 3229 + } 3230 + if !pr.cfg.Dialer.Ready(nodeID) { 3231 + return nil 3232 + } 3233 + 3234 + // Slow path: create the client. Another inserter might race us to it. 3235 + cl = &client{} 3236 + cl.mu.requested = map[roachpb.RangeID]struct{}{} 3237 + 3238 + if firstClient, loaded := pr.clients.LoadOrStore(int64(nodeID), unsafe.Pointer(cl)); loaded { 3239 + return (*client)(firstClient) 3240 + } 3241 + 3242 + // If our client made it into the map, start it. The point in inserting 3243 + // before starting is to be able to collect RangeIDs immediately while never 3244 + // blocking callers. 3245 + pr.cfg.Stopper.RunWorker(ctx, func(ctx context.Context) { 3246 + defer pr.clients.Delete(int64(nodeID)) 3247 + 3248 + c, err := pr.cfg.Dialer.Dial(ctx, nodeID) 3249 + if err != nil { 3250 + return 3251 + } 3252 + defer func() { 3253 + _ = c.CloseSend() 3254 + }() 3255 + 3256 + ctx = c.Context() 3257 + 3258 + ch := pr.cfg.Sink.Notify(nodeID) 3259 + defer close(ch) 3260 + 3261 + reaction := &ctpb.Reaction{} 3262 + for { 3263 + if err := c.Send(reaction); err != nil { 3264 + return 3265 + } 3266 + entry, err := c.Recv() 3267 + if err != nil { 3268 + return 3269 + } 3270 + 3271 + select { 3272 + case ch <- *entry: 3273 + case <-ctx.Done(): 3274 + return 3275 + case <-pr.cfg.Stopper.ShouldQuiesce(): 3276 + return 3277 + } 3278 + 3279 + var requested map[roachpb.RangeID]struct{} 3280 + cl.mu.Lock() 3281 + requested, cl.mu.requested = cl.mu.requested, map[roachpb.RangeID]struct{}{} 3282 + cl.mu.Unlock() 3283 + 3284 + slice := make([]roachpb.RangeID, 0, len(requested)) 3285 + for rangeID := range requested { 3286 + slice = append(slice, rangeID) 3287 + } 3288 + reaction = &ctpb.Reaction{ 3289 + Requested: slice, 3290 + } 3291 + } 3292 + }) 3293 + 3294 + return cl 3295 +} 3296 diff --git a/pkg/storage/closedts/transport/server.go b/pkg/storage/closedts/transport/server.go 3297 new file mode 100644 3298 index 00000000000..a72fca8358e 3299 --- /dev/null 3300 +++ b/pkg/storage/closedts/transport/server.go 3301 @@ -0,0 +1,97 @@ 3302 +// Copyright 2018 The Cockroach Authors. 3303 +// 3304 +// Licensed under the Apache License, Version 2.0 (the "License"); 3305 +// you may not use this file except in compliance with the License. 3306 +// You may obtain a copy of the License at 3307 +// 3308 +// http://www.apache.org/licenses/LICENSE-2.0 3309 +// 3310 +// Unless required by applicable law or agreed to in writing, software 3311 +// distributed under the License is distributed on an "AS IS" BASIS, 3312 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 3313 +// implied. See the License for the specific language governing 3314 +// permissions and limitations under the License. 3315 + 3316 +package transport 3317 + 3318 +import ( 3319 + "context" 3320 + "errors" 3321 + "time" 3322 + 3323 + "github.com/cockroachdb/cockroach/pkg/storage/closedts" 3324 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 3325 + "github.com/cockroachdb/cockroach/pkg/util/stop" 3326 + "github.com/cockroachdb/cockroach/pkg/util/timeutil" 3327 +) 3328 + 3329 +// Server handles incoming closed timestamp update stream requests. 3330 +type Server struct { 3331 + stopper *stop.Stopper 3332 + p closedts.Producer 3333 + refresh closedts.RefreshFn 3334 +} 3335 + 3336 +// NewServer sets up a Server which relays information from the given producer 3337 +// to incoming clients. 3338 +func NewServer(stopper *stop.Stopper, p closedts.Producer, refresh closedts.RefreshFn) *Server { 3339 + return &Server{ 3340 + stopper: stopper, 3341 + p: p, 3342 + refresh: refresh, 3343 + } 3344 +} 3345 + 3346 +var _ ctpb.Server = (*Server)(nil) 3347 + 3348 +// Get handles incoming client connections. 3349 +func (s *Server) Get(client ctpb.InboundClient) error { 3350 + // TODO(tschottdorf): the InboundClient API isn't great since it 3351 + // is blocking. How can we eagerly terminate these connections when 3352 + // the server shuts down? I think we need to inject a cancellation 3353 + // into the context, but grpc hands that to us. 3354 + // This problem has likely been solved somewhere in our codebase. 3355 + ctx := client.Context() 3356 + ch := make(chan ctpb.Entry, 10) 3357 + 3358 + // TODO: X*closedts.CloseFraction*TargetInterval 3359 + const closedTimestampNoUpdateWarnThreshold = 10 * time.Second 3360 + t := timeutil.NewTimer() 3361 + 3362 + s.stopper.RunWorker(ctx, func(ctx context.Context) { 3363 + s.p.Subscribe(ctx, ch) 3364 + }) 3365 + for { 3366 + reaction, err := client.Recv() 3367 + if err != nil { 3368 + return err 3369 + } 3370 + 3371 + if len(reaction.Requested) != 0 { 3372 + s.refresh(reaction.Requested...) 3373 + } 3374 + 3375 + t.Reset(closedTimestampNoUpdateWarnThreshold) 3376 + var entry ctpb.Entry 3377 + var ok bool 3378 + select { 3379 + case <-ctx.Done(): 3380 + return ctx.Err() 3381 + case <-s.stopper.ShouldQuiesce(): 3382 + return errors.New("node is draining") 3383 + case entry, ok = <-ch: 3384 + if !ok { 3385 + return errors.New("subscription dropped unexpectedly") 3386 + } 3387 + case <-t.C: 3388 + t.Read = true 3389 + // Send an empty entry to the client, which can use that to warn 3390 + // about the absence of heartbeats. We don't log here since it 3391 + // would log a message per incoming stream, which makes little 3392 + // sense. It's the producer's job to warn on this node. 3393 + } 3394 + if err := client.Send(&entry); err != nil { 3395 + return err 3396 + } 3397 + } 3398 +} 3399 diff --git a/pkg/storage/closedts/transport/testutils/chan_dialer.go b/pkg/storage/closedts/transport/testutils/chan_dialer.go 3400 new file mode 100644 3401 index 00000000000..97fac236b60 3402 --- /dev/null 3403 +++ b/pkg/storage/closedts/transport/testutils/chan_dialer.go 3404 @@ -0,0 +1,148 @@ 3405 +// Copyright 2018 The Cockroach Authors. 3406 +// 3407 +// Licensed under the Apache License, Version 2.0 (the "License"); 3408 +// you may not use this file except in compliance with the License. 3409 +// You may obtain a copy of the License at 3410 +// 3411 +// http://www.apache.org/licenses/LICENSE-2.0 3412 +// 3413 +// Unless required by applicable law or agreed to in writing, software 3414 +// distributed under the License is distributed on an "AS IS" BASIS, 3415 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 3416 +// implied. See the License for the specific language governing 3417 +// permissions and limitations under the License. 3418 + 3419 +package testutils 3420 + 3421 +import ( 3422 + "context" 3423 + "io" 3424 + 3425 + "github.com/cockroachdb/cockroach/pkg/roachpb" 3426 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 3427 + "github.com/cockroachdb/cockroach/pkg/util/stop" 3428 + "github.com/cockroachdb/cockroach/pkg/util/syncutil" 3429 +) 3430 + 3431 +// ChanDialer is an implementation of closedts.Dialer that connects clients 3432 +// directly via a channel to a Server. 3433 +type ChanDialer struct { 3434 + stopper *stop.Stopper 3435 + server ctpb.Server 3436 + 3437 + mu struct { 3438 + syncutil.Mutex 3439 + transcripts map[roachpb.NodeID][]interface{} 3440 + } 3441 +} 3442 + 3443 +// NewChanDialer sets up a ChanDialer. 3444 +func NewChanDialer(stopper *stop.Stopper, server ctpb.Server) *ChanDialer { 3445 + d := &ChanDialer{ 3446 + stopper: stopper, 3447 + server: server, 3448 + } 3449 + d.mu.transcripts = make(map[roachpb.NodeID][]interface{}) 3450 + return d 3451 +} 3452 + 3453 +// Transcript returns a slice of messages sent over the "wire". 3454 +func (d *ChanDialer) Transcript(nodeID roachpb.NodeID) []interface{} { 3455 + d.mu.Lock() 3456 + defer d.mu.Unlock() 3457 + return append([]interface{}(nil), d.mu.transcripts[nodeID]...) 3458 +} 3459 + 3460 +// Dial implements closedts.Dialer. 3461 +func (d *ChanDialer) Dial(ctx context.Context, nodeID roachpb.NodeID) (ctpb.Client, error) { 3462 + c := &client{ 3463 + ctx: ctx, 3464 + send: make(chan *ctpb.Reaction), 3465 + recv: make(chan *ctpb.Entry), 3466 + stopper: d.stopper, 3467 + observe: func(msg interface{}) { 3468 + d.mu.Lock() 3469 + if d.mu.transcripts == nil { 3470 + d.mu.transcripts = map[roachpb.NodeID][]interface{}{} 3471 + } 3472 + d.mu.transcripts[nodeID] = append(d.mu.transcripts[nodeID], msg) 3473 + d.mu.Unlock() 3474 + }, 3475 + } 3476 + 3477 + d.stopper.RunWorker(ctx, func(ctx context.Context) { 3478 + _ = d.server.Get((*incomingClient)(c)) 3479 + }) 3480 + return c, nil 3481 + 3482 +} 3483 + 3484 +// Ready implements closedts.Dialer by always returning true. 3485 +func (d *ChanDialer) Ready(nodeID roachpb.NodeID) bool { 3486 + return true 3487 +} 3488 + 3489 +type client struct { 3490 + ctx context.Context 3491 + stopper *stop.Stopper 3492 + send chan *ctpb.Reaction 3493 + recv chan *ctpb.Entry 3494 + 3495 + observe func(interface{}) 3496 +} 3497 + 3498 +func (c *client) Send(msg *ctpb.Reaction) error { 3499 + select { 3500 + case <-c.stopper.ShouldQuiesce(): 3501 + return io.EOF 3502 + case c.send <- msg: 3503 + c.observe(msg) 3504 + return nil 3505 + } 3506 +} 3507 + 3508 +func (c *client) Recv() (*ctpb.Entry, error) { 3509 + select { 3510 + case <-c.stopper.ShouldQuiesce(): 3511 + return nil, io.EOF 3512 + case msg := <-c.recv: 3513 + c.observe(msg) 3514 + return msg, nil 3515 + } 3516 +} 3517 + 3518 +func (c *client) CloseSend() error { 3519 + close(c.send) 3520 + return nil 3521 +} 3522 + 3523 +func (c *client) Context() context.Context { 3524 + return c.ctx 3525 +} 3526 + 3527 +type incomingClient client 3528 + 3529 +func (c *incomingClient) Send(msg *ctpb.Entry) error { 3530 + select { 3531 + case <-c.stopper.ShouldQuiesce(): 3532 + return io.EOF 3533 + case c.recv <- msg: 3534 + return nil 3535 + } 3536 +} 3537 + 3538 +func (c *incomingClient) Recv() (*ctpb.Reaction, error) { 3539 + select { 3540 + case <-c.stopper.ShouldQuiesce(): 3541 + return nil, io.EOF 3542 + case msg, ok := <-c.send: 3543 + if !ok { 3544 + return nil, io.EOF 3545 + } 3546 + return msg, nil 3547 + } 3548 +} 3549 + 3550 +func (c *incomingClient) Context() context.Context { 3551 + return c.ctx 3552 +} 3553 diff --git a/pkg/storage/closedts/transport/transport_test.go b/pkg/storage/closedts/transport/transport_test.go 3554 new file mode 100644 3555 index 00000000000..1b62d3ac5f2 3556 --- /dev/null 3557 +++ b/pkg/storage/closedts/transport/transport_test.go 3558 @@ -0,0 +1,199 @@ 3559 +// Copyright 2018 The Cockroach Authors. 3560 +// 3561 +// Licensed under the Apache License, Version 2.0 (the "License"); 3562 +// you may not use this file except in compliance with the License. 3563 +// You may obtain a copy of the License at 3564 +// 3565 +// http://www.apache.org/licenses/LICENSE-2.0 3566 +// 3567 +// Unless required by applicable law or agreed to in writing, software 3568 +// distributed under the License is distributed on an "AS IS" BASIS, 3569 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 3570 +// implied. See the License for the specific language governing 3571 +// permissions and limitations under the License. 3572 + 3573 +package transport_test 3574 + 3575 +import ( 3576 + "context" 3577 + "fmt" 3578 + "strings" 3579 + "testing" 3580 + 3581 + "github.com/kr/pretty" 3582 + "github.com/pkg/errors" 3583 + 3584 + "github.com/cockroachdb/cockroach/pkg/roachpb" 3585 + "github.com/cockroachdb/cockroach/pkg/settings/cluster" 3586 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 3587 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/transport" 3588 + transporttestutils "github.com/cockroachdb/cockroach/pkg/storage/closedts/transport/testutils" 3589 + "github.com/cockroachdb/cockroach/pkg/testutils" 3590 + "github.com/cockroachdb/cockroach/pkg/util/hlc" 3591 + "github.com/cockroachdb/cockroach/pkg/util/leaktest" 3592 + "github.com/cockroachdb/cockroach/pkg/util/stop" 3593 +) 3594 + 3595 +// NewTestContainer sets up an environment suitable for black box testing the 3596 +// transport subsystem. The returned test container contains most notably a 3597 +// Clients and Server set up to communicate to each other via a Dialer (which 3598 +// keeps a transcript that can be verified). 3599 +func NewTestContainer() *TestContainer { 3600 + stopper := stop.NewStopper() 3601 + 3602 + st := cluster.MakeTestingClusterSettings() 3603 + p := &TestProducer{} 3604 + sink := newTestNotifyee(stopper) 3605 + refreshed := &RefreshTracker{} 3606 + s := transport.NewServer(stopper, p, refreshed.Add) 3607 + dialer := transporttestutils.NewChanDialer(stopper, s) 3608 + c := transport.NewClients(transport.Config{ 3609 + Settings: st, 3610 + Stopper: stopper, 3611 + Dialer: dialer, 3612 + Sink: sink, 3613 + }) 3614 + return &TestContainer{ 3615 + Settings: st, 3616 + Stopper: stopper, 3617 + Producer: p, 3618 + Notifyee: sink, 3619 + Refreshed: refreshed, 3620 + Server: s, 3621 + Dialer: dialer, 3622 + Clients: c, 3623 + } 3624 +} 3625 + 3626 +func assertNumSubscribers(t *testing.T, p *TestProducer, exp int) { 3627 + testutils.SucceedsSoon(t, func() error { 3628 + n := p.numSubscriptions() 3629 + if n > exp { 3630 + t.Fatalf("expected a single subscription, got %d", n) 3631 + } 3632 + if n < exp { 3633 + return errors.New("waiting for subscription") 3634 + } 3635 + return nil 3636 + }) 3637 +} 3638 + 3639 +func TestTransportConnectOnRequest(t *testing.T) { 3640 + defer leaktest.AfterTest(t)() 3641 + 3642 + container := NewTestContainer() 3643 + defer container.Stopper.Stop(context.Background()) 3644 + 3645 + const ( 3646 + nodeID = 1 3647 + rangeID = 13 3648 + ) 3649 + 3650 + // Requesting an update for a Range implies a connection attempt. 3651 + container.Clients.Request(nodeID, rangeID) 3652 + 3653 + // Find the connection (via its subscription to receive new Entries). 3654 + assertNumSubscribers(t, container.Producer, 1) 3655 + 3656 + // Verify that the client soon asks the server for an update for this range. 3657 + testutils.SucceedsSoon(t, func() error { 3658 + act := container.Refreshed.Get() 3659 + exp := []roachpb.RangeID{rangeID} 3660 + 3661 + if diff := pretty.Diff(act, exp); len(diff) != 0 { 3662 + // We have to kick the tires a little bit. The client can only send 3663 + // the request as the reaction to an Entry. 3664 + container.Producer.sendAll(ctpb.Entry{}) 3665 + return errors.Errorf("diff(act, exp): %s", strings.Join(diff, "\n")) 3666 + } 3667 + return nil 3668 + }) 3669 +} 3670 + 3671 +func TestTransportClientReceivesEntries(t *testing.T) { 3672 + defer leaktest.AfterTest(t)() 3673 + 3674 + container := NewTestContainer() 3675 + defer container.Stopper.Stop(context.Background()) 3676 + 3677 + const nodeID = 7 3678 + 3679 + // Manual reconnections don't spawn new clients. 3680 + container.Clients.EnsureClient(nodeID) 3681 + container.Clients.EnsureClient(nodeID) 3682 + container.Clients.EnsureClient(nodeID) 3683 + assertNumSubscribers(t, container.Producer, 1) 3684 + 3685 + // But connecting to other nodes does (only once). 3686 + for i := 0; i < 7; i++ { 3687 + container.Clients.EnsureClient(nodeID + 1) 3688 + container.Clients.EnsureClient(nodeID + 2) 3689 + container.Clients.Request(nodeID+3, roachpb.RangeID(7)) 3690 + } 3691 + assertNumSubscribers(t, container.Producer, 4) 3692 + 3693 + // Our initial client doesn't do anything except say "hello" via 3694 + // a Reaction. 3695 + testutils.SucceedsSoon(t, func() error { 3696 + expectedTranscript := []interface{}{ 3697 + &ctpb.Reaction{}, 3698 + } 3699 + return checkTranscript(t, container.Dialer.Transcript(nodeID), expectedTranscript) 3700 + }) 3701 + 3702 + // Now the producer (to which the server should maintain a subscription for this client, and 3703 + // notifications from which it should relay) emits an Entry. 3704 + e1 := ctpb.Entry{ClosedTimestamp: hlc.Timestamp{WallTime: 1E9}, Epoch: 12, MLAI: map[roachpb.RangeID]ctpb.LAI{12: 7}} 3705 + container.Producer.sendAll(e1) 3706 + 3707 + // The client should see this entry soon thereafter. it responds with an empty 3708 + // Reaction (since we haven't Request()ed anything). 3709 + testutils.SucceedsSoon(t, func() error { 3710 + expectedTranscript := []interface{}{ 3711 + &ctpb.Reaction{}, 3712 + &e1, 3713 + &ctpb.Reaction{}, 3714 + } 3715 + return checkTranscript(t, container.Dialer.Transcript(nodeID), expectedTranscript) 3716 + }) 3717 + 3718 + // And again, but only after Request() is called (which should be reflected in the transcript). 3719 + const rangeID = 7 3720 + container.Clients.Request(nodeID, rangeID) 3721 + e2 := ctpb.Entry{ClosedTimestamp: hlc.Timestamp{WallTime: 2E9}, Epoch: 13, MLAI: map[roachpb.RangeID]ctpb.LAI{13: 8}} 3722 + container.Producer.sendAll(e2) 3723 + testutils.SucceedsSoon(t, func() error { 3724 + expectedTranscript := []interface{}{ 3725 + &ctpb.Reaction{}, 3726 + &e1, 3727 + &ctpb.Reaction{}, 3728 + &e2, 3729 + &ctpb.Reaction{Requested: []roachpb.RangeID{rangeID}}, 3730 + } 3731 + return checkTranscript(t, container.Dialer.Transcript(nodeID), expectedTranscript) 3732 + }) 3733 + 3734 +} 3735 + 3736 +func checkTranscript(t *testing.T, actI, expI []interface{}) error { 3737 + t.Helper() 3738 + var act, exp []string 3739 + for _, i := range actI { 3740 + act = append(act, strings.TrimSpace(fmt.Sprintf("%v", i))) 3741 + } 3742 + for _, i := range expI { 3743 + exp = append(exp, strings.TrimSpace(fmt.Sprintf("%v", i))) 3744 + } 3745 + 3746 + diffErr := errors.Errorf("actual:\n%s\nexpected:\n%s", strings.Join(act, "\n"), strings.Join(exp, "\n")) 3747 + if len(act) > len(exp) { 3748 + t.Fatal(errors.Wrap(diffErr, "actual transcript longer than expected")) 3749 + } 3750 + if len(act) < len(exp) { 3751 + return errors.Wrap(diffErr, "waiting for more") 3752 + } 3753 + if diff := pretty.Diff(actI, expI); len(diff) != 0 { 3754 + t.Fatal(errors.Wrapf(diffErr, "diff:\n%v\n", strings.Join(diff, "\n"))) 3755 + } 3756 + return nil 3757 +} 3758 diff --git a/pkg/storage/closedts/transport/transport_util_test.go b/pkg/storage/closedts/transport/transport_util_test.go 3759 new file mode 100644 3760 index 00000000000..d077d0f947e 3761 --- /dev/null 3762 +++ b/pkg/storage/closedts/transport/transport_util_test.go 3763 @@ -0,0 +1,108 @@ 3764 +// Copyright 2018 The Cockroach Authors. 3765 +// 3766 +// Licensed under the Apache License, Version 2.0 (the "License"); 3767 +// you may not use this file except in compliance with the License. 3768 +// You may obtain a copy of the License at 3769 +// 3770 +// http://www.apache.org/licenses/LICENSE-2.0 3771 +// 3772 +// Unless required by applicable law or agreed to in writing, software 3773 +// distributed under the License is distributed on an "AS IS" BASIS, 3774 +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 3775 +// implied. See the License for the specific language governing 3776 +// permissions and limitations under the License. 3777 + 3778 +package transport_test 3779 + 3780 +import ( 3781 + "context" 3782 + 3783 + "github.com/cockroachdb/cockroach/pkg/roachpb" 3784 + "github.com/cockroachdb/cockroach/pkg/settings/cluster" 3785 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/ctpb" 3786 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/transport" 3787 + "github.com/cockroachdb/cockroach/pkg/storage/closedts/transport/testutils" 3788 + "github.com/cockroachdb/cockroach/pkg/util/stop" 3789 + "github.com/cockroachdb/cockroach/pkg/util/syncutil" 3790 +) 3791 + 3792 +type TestContainer struct { 3793 + Settings *cluster.Settings 3794 + Stopper *stop.Stopper 3795 + Producer *TestProducer 3796 + Notifyee *TestNotifyee 3797 + Refreshed *RefreshTracker 3798 + Server *transport.Server 3799 + Dialer *testutils.ChanDialer 3800 + Clients *transport.Clients 3801 +} 3802 + 3803 +type TestProducer struct { 3804 + syncutil.Mutex 3805 + chs []chan<- ctpb.Entry 3806 +} 3807 + 3808 +func (tp *TestProducer) Subscribe(ctx context.Context, ch chan<- ctpb.Entry) { 3809 + tp.Lock() 3810 + tp.chs = append(tp.chs, ch) 3811 + tp.Unlock() 3812 +} 3813 + 3814 +func (tp *TestProducer) numSubscriptions() int { 3815 + tp.Lock() 3816 + defer tp.Unlock() 3817 + return len(tp.chs) 3818 +} 3819 + 3820 +func (tp *TestProducer) sendAll(entry ctpb.Entry) { 3821 + tp.Lock() 3822 + for _, ch := range tp.chs { 3823 + ch <- entry 3824 + } 3825 + tp.Unlock() 3826 +} 3827 + 3828 +type TestNotifyee struct { 3829 + stopper *stop.Stopper 3830 + mu struct { 3831 + syncutil.Mutex 3832 + entries map[roachpb.NodeID][]ctpb.Entry 3833 + } 3834 +} 3835 + 3836 +func newTestNotifyee(stopper *stop.Stopper) *TestNotifyee { 3837 + tn := &TestNotifyee{ 3838 + stopper: stopper, 3839 + } 3840 + tn.mu.entries = make(map[roachpb.NodeID][]ctpb.Entry) 3841 + return tn 3842 +} 3843 + 3844 +func (tn *TestNotifyee) Notify(nodeID roachpb.NodeID) chan<- ctpb.Entry { 3845 + ch := make(chan ctpb.Entry) 3846 + tn.stopper.RunWorker(context.Background(), func(ctx context.Context) { 3847 + for entry := range ch { 3848 + tn.mu.Lock() 3849 + tn.mu.entries[nodeID] = append(tn.mu.entries[nodeID], entry) 3850 + tn.mu.Unlock() 3851 + } 3852 + }) 3853 + return ch 3854 +} 3855 + 3856 +type RefreshTracker struct { 3857 + syncutil.Mutex 3858 + rangeIDs []roachpb.RangeID 3859 +} 3860 + 3861 +func (r *RefreshTracker) Get() []roachpb.RangeID { 3862 + r.Lock() 3863 + defer r.Unlock() 3864 + return append([]roachpb.RangeID(nil), r.rangeIDs...) 3865 +} 3866 + 3867 +func (r *RefreshTracker) Add(rangeIDs ...roachpb.RangeID) { 3868 + r.Lock() 3869 + r.rangeIDs = append(r.rangeIDs, rangeIDs...) 3870 + r.Unlock() 3871 +} 3872 diff --git a/pkg/testutils/soon.go b/pkg/testutils/soon.go 3873 index 9deb29441d8..f0f5c3e7c3e 100644 3874 --- a/pkg/testutils/soon.go 3875 +++ b/pkg/testutils/soon.go 3876 @@ -15,11 +15,16 @@ 3877 package testutils 3878 3879 import ( 3880 + "context" 3881 "runtime/debug" 3882 "testing" 3883 "time" 3884 3885 + "github.com/pkg/errors" 3886 + 3887 + "github.com/cockroachdb/cockroach/pkg/util/log" 3888 "github.com/cockroachdb/cockroach/pkg/util/retry" 3889 + "github.com/cockroachdb/cockroach/pkg/util/timeutil" 3890 ) 3891 3892 // DefaultSucceedsSoonDuration is the maximum amount of time unittests 3893 @@ -29,11 +34,18 @@ const DefaultSucceedsSoonDuration = 45 * time.Second 3894 // SucceedsSoon fails the test (with t.Fatal) unless the supplied 3895 // function runs without error within a preset maximum duration. The 3896 // function is invoked immediately at first and then successively with 3897 -// an exponential backoff starting at 1ns and ending at the maximum 3898 -// duration (currently 15s). 3899 +// an exponential backoff starting at 1ns and ending at around 1s. 3900 func SucceedsSoon(t testing.TB, fn func() error) { 3901 t.Helper() 3902 - if err := retry.ForDuration(DefaultSucceedsSoonDuration, fn); err != nil { 3903 + tBegin := timeutil.Now() 3904 + wrappedFn := func() error { 3905 + err := fn() 3906 + if timeutil.Since(tBegin) > 3*time.Second && err != nil { 3907 + log.InfoDepth(context.Background(), 3, errors.Wrap(err, "SucceedsSoon")) 3908 + } 3909 + return err 3910 + } 3911 + if err := retry.ForDuration(DefaultSucceedsSoonDuration, wrappedFn); err != nil { 3912 t.Fatalf("condition failed to evaluate within %s: %s\n%s", 3913 DefaultSucceedsSoonDuration, err, string(debug.Stack())) 3914 }