github.com/dolthub/dolt/go@v0.40.5-0.20240520175717-68db7794bea6/store/nbs/store.go (about) 1 // Copyright 2019 Dolthub, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 // 15 // This file incorporates work covered by the following copyright and 16 // permission notice: 17 // 18 // Copyright 2016 Attic Labs, Inc. All rights reserved. 19 // Licensed under the Apache License, version 2.0: 20 // http://www.apache.org/licenses/LICENSE-2.0 21 22 package nbs 23 24 import ( 25 "context" 26 "fmt" 27 "io" 28 "os" 29 "path/filepath" 30 "sort" 31 "sync" 32 "sync/atomic" 33 "time" 34 35 "cloud.google.com/go/storage" 36 "github.com/aws/aws-sdk-go/service/s3/s3iface" 37 "github.com/dustin/go-humanize" 38 lru "github.com/hashicorp/golang-lru/v2" 39 "github.com/oracle/oci-go-sdk/v65/common" 40 "github.com/oracle/oci-go-sdk/v65/objectstorage" 41 "github.com/pkg/errors" 42 "go.opentelemetry.io/otel" 43 "go.opentelemetry.io/otel/attribute" 44 "go.opentelemetry.io/otel/trace" 45 "golang.org/x/sync/errgroup" 46 47 "github.com/dolthub/dolt/go/store/blobstore" 48 "github.com/dolthub/dolt/go/store/chunks" 49 "github.com/dolthub/dolt/go/store/hash" 50 ) 51 52 var ( 53 ErrFetchFailure = errors.New("fetch failed") 54 ErrSpecWithoutChunkSource = errors.New("manifest referenced table file for which there is no chunkSource.") 55 ErrConcurrentManifestWriteDuringOverwrite = errors.New("concurrent manifest write during manifest overwrite") 56 ) 57 58 // The root of a Noms Chunk Store is stored in a 'manifest', along with the 59 // names of the tables that hold all the chunks in the store. The number of 60 // chunks in each table is also stored in the manifest. 61 62 const ( 63 // StorageVersion is the version of the on-disk Noms Chunks Store data format. 64 StorageVersion = "5" 65 66 defaultMemTableSize uint64 = (1 << 20) * 128 // 128MB 67 defaultMaxTables = 256 68 69 defaultManifestCacheSize = 1 << 23 // 8MB 70 preflushChunkCount = 8 71 ) 72 73 var ( 74 cacheOnce = sync.Once{} 75 makeManifestManager func(manifest) manifestManager 76 ) 77 78 var tracer = otel.Tracer("github.com/dolthub/dolt/go/store/nbs") 79 80 func makeGlobalCaches() { 81 manifestCache := newManifestCache(defaultManifestCacheSize) 82 manifestLocks := newManifestLocks() 83 makeManifestManager = func(m manifest) manifestManager { return manifestManager{m, manifestCache, manifestLocks} } 84 } 85 86 type NBSCompressedChunkStore interface { 87 chunks.ChunkStore 88 GetManyCompressed(context.Context, hash.HashSet, func(context.Context, CompressedChunk)) error 89 } 90 91 type NomsBlockStore struct { 92 mm manifestManager 93 p tablePersister 94 c conjoinStrategy 95 96 mu sync.RWMutex // protects the following state 97 mt *memTable 98 tables tableSet 99 upstream manifestContents 100 101 cond *sync.Cond 102 gcInProgress bool 103 // keeperFunc is set when |gcInProgress| and appends to the GC sweep queue 104 // or blocks on GC finalize 105 keeperFunc func(hash.Hash) bool 106 107 mtSize uint64 108 putCount uint64 109 110 hasCache *lru.TwoQueueCache[hash.Hash, struct{}] 111 112 stats *Stats 113 } 114 115 func (nbs *NomsBlockStore) PersistGhostHashes(ctx context.Context, refs hash.HashSet) error { 116 return fmt.Errorf("runtime error: PersistGhostHashes should never be called on the NomsBlockStore") 117 } 118 119 var _ chunks.TableFileStore = &NomsBlockStore{} 120 var _ chunks.ChunkStoreGarbageCollector = &NomsBlockStore{} 121 122 // 20-byte keys, ~2MB of key data. 123 // 124 // Likely big enough to keep common top of DAG references in the scan resistant 125 // portion for most databases. 126 const hasCacheSize = 100000 127 128 type Range struct { 129 Offset uint64 130 Length uint32 131 } 132 133 // ChunkJournal returns the ChunkJournal in use by this NomsBlockStore, or nil if no ChunkJournal is being used. 134 func (nbs *NomsBlockStore) ChunkJournal() *ChunkJournal { 135 if cj, ok := nbs.p.(*ChunkJournal); ok { 136 return cj 137 } 138 return nil 139 } 140 141 func (nbs *NomsBlockStore) GetChunkLocationsWithPaths(hashes hash.HashSet) (map[string]map[hash.Hash]Range, error) { 142 locs, err := nbs.GetChunkLocations(hashes) 143 if err != nil { 144 return nil, err 145 } 146 toret := make(map[string]map[hash.Hash]Range, len(locs)) 147 for k, v := range locs { 148 toret[k.String()] = v 149 } 150 return toret, nil 151 } 152 153 func (nbs *NomsBlockStore) GetChunkLocations(hashes hash.HashSet) (map[hash.Hash]map[hash.Hash]Range, error) { 154 gr := toGetRecords(hashes) 155 ranges := make(map[hash.Hash]map[hash.Hash]Range) 156 157 fn := func(css chunkSourceSet) error { 158 for _, cs := range css { 159 rng, err := cs.getRecordRanges(gr) 160 if err != nil { 161 return err 162 } 163 164 h := hash.Hash(cs.hash()) 165 if m, ok := ranges[h]; ok { 166 for k, v := range rng { 167 m[k] = v 168 } 169 } else { 170 ranges[h] = rng 171 } 172 } 173 return nil 174 } 175 176 tables := func() tableSet { 177 nbs.mu.RLock() 178 defer nbs.mu.RUnlock() 179 return nbs.tables 180 }() 181 182 if err := fn(tables.upstream); err != nil { 183 return nil, err 184 } 185 if err := fn(tables.novel); err != nil { 186 return nil, err 187 } 188 return ranges, nil 189 } 190 191 func (nbs *NomsBlockStore) conjoinIfRequired(ctx context.Context) (bool, error) { 192 if nbs.c.conjoinRequired(nbs.tables) { 193 newUpstream, cleanup, err := conjoin(ctx, nbs.c, nbs.upstream, nbs.mm, nbs.p, nbs.stats) 194 if err != nil { 195 return false, err 196 } 197 198 newTables, err := nbs.tables.rebase(ctx, newUpstream.specs, nbs.stats) 199 if err != nil { 200 return false, err 201 } 202 203 nbs.upstream = newUpstream 204 oldTables := nbs.tables 205 nbs.tables = newTables 206 err = oldTables.close() 207 if err != nil { 208 return true, err 209 } 210 cleanup() 211 return true, nil 212 } else { 213 return false, nil 214 } 215 } 216 217 func (nbs *NomsBlockStore) UpdateManifest(ctx context.Context, updates map[hash.Hash]uint32) (mi ManifestInfo, err error) { 218 nbs.mu.Lock() 219 defer nbs.mu.Unlock() 220 err = nbs.waitForGC(ctx) 221 if err != nil { 222 return 223 } 224 225 err = nbs.checkAllManifestUpdatesExist(ctx, updates) 226 if err != nil { 227 return 228 } 229 230 nbs.mm.LockForUpdate() 231 defer func() { 232 unlockErr := nbs.mm.UnlockForUpdate() 233 234 if err == nil { 235 err = unlockErr 236 } 237 }() 238 239 _, err = nbs.conjoinIfRequired(ctx) 240 if err != nil { 241 return manifestContents{}, err 242 } 243 244 var updatedContents manifestContents 245 for { 246 ok, contents, _, ferr := nbs.mm.Fetch(ctx, nbs.stats) 247 if ferr != nil { 248 return manifestContents{}, ferr 249 } else if !ok { 250 contents = manifestContents{nbfVers: nbs.upstream.nbfVers} 251 } 252 253 originalLock := contents.lock 254 255 currSpecs := contents.getSpecSet() 256 257 var addCount int 258 for h, count := range updates { 259 if _, ok := currSpecs[h]; !ok { 260 addCount++ 261 contents.specs = append(contents.specs, tableSpec{h, count}) 262 } 263 } 264 265 if addCount == 0 { 266 return contents, nil 267 } 268 269 contents.lock = generateLockHash(contents.root, contents.specs, contents.appendix) 270 271 // ensure we don't drop existing appendices 272 if contents.appendix != nil && len(contents.appendix) > 0 { 273 contents, err = fromManifestAppendixOptionNewContents(contents, contents.appendix, ManifestAppendixOption_Set) 274 if err != nil { 275 return manifestContents{}, err 276 } 277 } 278 279 updatedContents, err = nbs.mm.Update(ctx, originalLock, contents, nbs.stats, nil) 280 if err != nil { 281 return manifestContents{}, err 282 } 283 284 if updatedContents.lock == contents.lock { 285 break 286 } 287 } 288 289 newTables, err := nbs.tables.rebase(ctx, updatedContents.specs, nbs.stats) 290 if err != nil { 291 return manifestContents{}, err 292 } 293 294 nbs.upstream = updatedContents 295 oldTables := nbs.tables 296 nbs.tables = newTables 297 err = oldTables.close() 298 if err != nil { 299 return manifestContents{}, err 300 } 301 302 return updatedContents, nil 303 } 304 305 func (nbs *NomsBlockStore) UpdateManifestWithAppendix(ctx context.Context, updates map[hash.Hash]uint32, option ManifestAppendixOption) (mi ManifestInfo, err error) { 306 nbs.mu.Lock() 307 defer nbs.mu.Unlock() 308 err = nbs.waitForGC(ctx) 309 if err != nil { 310 return 311 } 312 313 err = nbs.checkAllManifestUpdatesExist(ctx, updates) 314 if err != nil { 315 return 316 } 317 318 nbs.mm.LockForUpdate() 319 defer func() { 320 unlockErr := nbs.mm.UnlockForUpdate() 321 322 if err == nil { 323 err = unlockErr 324 } 325 }() 326 327 _, err = nbs.conjoinIfRequired(ctx) 328 if err != nil { 329 return manifestContents{}, err 330 } 331 332 var updatedContents manifestContents 333 for { 334 ok, contents, _, ferr := nbs.mm.Fetch(ctx, nbs.stats) 335 336 if ferr != nil { 337 return manifestContents{}, ferr 338 } else if !ok { 339 contents = manifestContents{nbfVers: nbs.upstream.nbfVers} 340 } 341 342 originalLock := contents.lock 343 344 currAppendixSpecs := contents.getAppendixSet() 345 346 appendixSpecs := make([]tableSpec, 0) 347 var addCount int 348 for h, count := range updates { 349 if option == ManifestAppendixOption_Set { 350 appendixSpecs = append(appendixSpecs, tableSpec{h, count}) 351 } else { 352 if _, ok := currAppendixSpecs[h]; !ok { 353 addCount++ 354 appendixSpecs = append(appendixSpecs, tableSpec{h, count}) 355 } 356 } 357 } 358 359 if addCount == 0 && option != ManifestAppendixOption_Set { 360 return contents, nil 361 } 362 363 contents, err = fromManifestAppendixOptionNewContents(contents, appendixSpecs, option) 364 if err != nil { 365 return manifestContents{}, err 366 } 367 368 updatedContents, err = nbs.mm.Update(ctx, originalLock, contents, nbs.stats, nil) 369 if err != nil { 370 return manifestContents{}, err 371 } 372 373 if updatedContents.lock == contents.lock { 374 break 375 } 376 } 377 378 newTables, err := nbs.tables.rebase(ctx, updatedContents.specs, nbs.stats) 379 if err != nil { 380 return manifestContents{}, err 381 } 382 383 nbs.upstream = updatedContents 384 oldTables := nbs.tables 385 nbs.tables = newTables 386 err = oldTables.close() 387 if err != nil { 388 return manifestContents{}, err 389 } 390 return updatedContents, nil 391 } 392 393 func (nbs *NomsBlockStore) checkAllManifestUpdatesExist(ctx context.Context, updates map[hash.Hash]uint32) error { 394 eg, ctx := errgroup.WithContext(ctx) 395 eg.SetLimit(128) 396 for h, c := range updates { 397 h := h 398 c := c 399 eg.Go(func() error { 400 ok, err := nbs.p.Exists(ctx, h, c, nbs.stats) 401 if err != nil { 402 return err 403 } 404 if !ok { 405 return fmt.Errorf("missing table file referenced in UpdateManifest call: %v", h) 406 } 407 return nil 408 }) 409 } 410 return eg.Wait() 411 } 412 413 func fromManifestAppendixOptionNewContents(upstream manifestContents, appendixSpecs []tableSpec, option ManifestAppendixOption) (manifestContents, error) { 414 contents, upstreamAppendixSpecs := upstream.removeAppendixSpecs() 415 switch option { 416 case ManifestAppendixOption_Append: 417 // append all appendix specs to contents.specs 418 specs := append([]tableSpec{}, appendixSpecs...) 419 specs = append(specs, upstreamAppendixSpecs...) 420 contents.specs = append(specs, contents.specs...) 421 422 // append all appendix specs to contents.appendix 423 newAppendixSpecs := append([]tableSpec{}, upstreamAppendixSpecs...) 424 contents.appendix = append(newAppendixSpecs, appendixSpecs...) 425 426 contents.lock = generateLockHash(contents.root, contents.specs, contents.appendix) 427 return contents, nil 428 case ManifestAppendixOption_Set: 429 if len(appendixSpecs) < 1 { 430 return contents, nil 431 } 432 433 // append new appendix specs to contents.specs 434 // dropping all upstream appendix specs 435 specs := append([]tableSpec{}, appendixSpecs...) 436 contents.specs = append(specs, contents.specs...) 437 438 // append new appendix specs to contents.appendix 439 contents.appendix = append([]tableSpec{}, appendixSpecs...) 440 441 contents.lock = generateLockHash(contents.root, contents.specs, contents.appendix) 442 return contents, nil 443 default: 444 return manifestContents{}, ErrUnsupportedManifestAppendixOption 445 } 446 } 447 448 // OverwriteStoreManifest is a low level interface to completely replace the manifest contents 449 // of |store| with the supplied |root|, |tableFiles| and |appendixTableFiles|. It performs concurrency 450 // control on the existing |store| manifest, and can fail with |ErrConcurrentManifestWriteDuringOverwrite| 451 // if the |store|'s view is stale. If contents should be unconditionally replaced without regard for the existing 452 // contents, run this in a loop, rebasing |store| after each failure. 453 // 454 // Regardless of success or failure, |OverwriteStoreManifest| does *not* Rebase the |store|. The persisted 455 // manifest contents will have been updated, but nothing about the in-memory view of the |store| will reflect 456 // those updates. If |store| is Rebase'd, then the new upstream contents will be picked up. 457 // 458 // Extreme care should be taken when updating manifest contents through this interface. Logic typically 459 // assumes that stores grow monotonically unless the |gcGen| of a manifest changes. Since this interface 460 // cannot set |gcGen|, callers must ensure that calls to this function grow the store monotonically. 461 func OverwriteStoreManifest(ctx context.Context, store *NomsBlockStore, root hash.Hash, tableFiles map[hash.Hash]uint32, appendixTableFiles map[hash.Hash]uint32) (err error) { 462 store.mu.Lock() 463 defer store.mu.Unlock() 464 err = store.waitForGC(ctx) 465 if err != nil { 466 return 467 } 468 469 contents := manifestContents{ 470 root: root, 471 nbfVers: store.upstream.nbfVers, 472 } 473 // Appendix table files should come first in specs 474 for h, c := range appendixTableFiles { 475 s := tableSpec{name: h, chunkCount: c} 476 contents.appendix = append(contents.appendix, s) 477 contents.specs = append(contents.specs, s) 478 } 479 for h, c := range tableFiles { 480 s := tableSpec{name: h, chunkCount: c} 481 contents.specs = append(contents.specs, s) 482 } 483 contents.lock = generateLockHash(contents.root, contents.specs, contents.appendix) 484 485 store.mm.LockForUpdate() 486 defer func() { 487 unlockErr := store.mm.UnlockForUpdate() 488 489 if err == nil { 490 err = unlockErr 491 } 492 }() 493 updatedContents, err := store.mm.Update(ctx, store.upstream.lock, contents, store.stats, nil) 494 if err != nil { 495 return err 496 } 497 if updatedContents.lock != contents.lock { 498 return ErrConcurrentManifestWriteDuringOverwrite 499 } 500 // We don't update |nbs.upstream| here since the tables have not been rebased 501 return nil 502 } 503 504 func NewAWSStoreWithMMapIndex(ctx context.Context, nbfVerStr string, table, ns, bucket string, s3 s3iface.S3API, ddb ddbsvc, memTableSize uint64, q MemoryQuotaProvider) (*NomsBlockStore, error) { 505 cacheOnce.Do(makeGlobalCaches) 506 readRateLimiter := make(chan struct{}, 32) 507 p := &awsTablePersister{ 508 s3, 509 bucket, 510 readRateLimiter, 511 awsLimits{defaultS3PartSize, minS3PartSize, maxS3PartSize}, 512 ns, 513 q, 514 } 515 mm := makeManifestManager(newDynamoManifest(table, ns, ddb)) 516 return newNomsBlockStore(ctx, nbfVerStr, mm, p, q, inlineConjoiner{defaultMaxTables}, memTableSize) 517 } 518 519 func NewAWSStore(ctx context.Context, nbfVerStr string, table, ns, bucket string, s3 s3iface.S3API, ddb ddbsvc, memTableSize uint64, q MemoryQuotaProvider) (*NomsBlockStore, error) { 520 cacheOnce.Do(makeGlobalCaches) 521 readRateLimiter := make(chan struct{}, 32) 522 p := &awsTablePersister{ 523 s3, 524 bucket, 525 readRateLimiter, 526 awsLimits{defaultS3PartSize, minS3PartSize, maxS3PartSize}, 527 ns, 528 q, 529 } 530 mm := makeManifestManager(newDynamoManifest(table, ns, ddb)) 531 return newNomsBlockStore(ctx, nbfVerStr, mm, p, q, inlineConjoiner{defaultMaxTables}, memTableSize) 532 } 533 534 // NewGCSStore returns an nbs implementation backed by a GCSBlobstore 535 func NewGCSStore(ctx context.Context, nbfVerStr string, bucketName, path string, gcs *storage.Client, memTableSize uint64, q MemoryQuotaProvider) (*NomsBlockStore, error) { 536 cacheOnce.Do(makeGlobalCaches) 537 538 bs := blobstore.NewGCSBlobstore(gcs, bucketName, path) 539 return NewBSStore(ctx, nbfVerStr, bs, memTableSize, q) 540 } 541 542 // NewGCSStore returns an nbs implementation backed by a GCSBlobstore 543 func NewOCISStore(ctx context.Context, nbfVerStr string, bucketName, path string, provider common.ConfigurationProvider, client objectstorage.ObjectStorageClient, memTableSize uint64, q MemoryQuotaProvider) (*NomsBlockStore, error) { 544 cacheOnce.Do(makeGlobalCaches) 545 546 bs, err := blobstore.NewOCIBlobstore(ctx, provider, client, bucketName, path) 547 if err != nil { 548 return nil, err 549 } 550 551 return NewNoConjoinBSStore(ctx, nbfVerStr, bs, memTableSize, q) 552 } 553 554 // NewBSStore returns an nbs implementation backed by a Blobstore 555 func NewBSStore(ctx context.Context, nbfVerStr string, bs blobstore.Blobstore, memTableSize uint64, q MemoryQuotaProvider) (*NomsBlockStore, error) { 556 cacheOnce.Do(makeGlobalCaches) 557 558 mm := makeManifestManager(blobstoreManifest{bs}) 559 560 p := &blobstorePersister{bs, s3BlockSize, q} 561 return newNomsBlockStore(ctx, nbfVerStr, mm, p, q, inlineConjoiner{defaultMaxTables}, memTableSize) 562 } 563 564 // NewNoConjoinBSStore returns a nbs implementation backed by a Blobstore 565 func NewNoConjoinBSStore(ctx context.Context, nbfVerStr string, bs blobstore.Blobstore, memTableSize uint64, q MemoryQuotaProvider) (*NomsBlockStore, error) { 566 cacheOnce.Do(makeGlobalCaches) 567 568 mm := makeManifestManager(blobstoreManifest{bs}) 569 570 p := &noConjoinBlobstorePersister{bs, s3BlockSize, q} 571 return newNomsBlockStore(ctx, nbfVerStr, mm, p, q, noopConjoiner{}, memTableSize) 572 } 573 574 func NewLocalStore(ctx context.Context, nbfVerStr string, dir string, memTableSize uint64, q MemoryQuotaProvider) (*NomsBlockStore, error) { 575 return newLocalStore(ctx, nbfVerStr, dir, memTableSize, defaultMaxTables, q) 576 } 577 578 func newLocalStore(ctx context.Context, nbfVerStr string, dir string, memTableSize uint64, maxTables int, q MemoryQuotaProvider) (*NomsBlockStore, error) { 579 cacheOnce.Do(makeGlobalCaches) 580 if err := checkDir(dir); err != nil { 581 return nil, err 582 } 583 ok, err := fileExists(filepath.Join(dir, chunkJournalAddr)) 584 if err != nil { 585 return nil, err 586 } else if ok { 587 return nil, fmt.Errorf("cannot create NBS store for directory containing chunk journal: %s", dir) 588 } 589 590 m, err := getFileManifest(ctx, dir, asyncFlush) 591 if err != nil { 592 return nil, err 593 } 594 p := newFSTablePersister(dir, q) 595 c := conjoinStrategy(inlineConjoiner{maxTables}) 596 597 return newNomsBlockStore(ctx, nbfVerStr, makeManifestManager(m), p, q, c, memTableSize) 598 } 599 600 func NewLocalJournalingStore(ctx context.Context, nbfVers, dir string, q MemoryQuotaProvider) (*NomsBlockStore, error) { 601 cacheOnce.Do(makeGlobalCaches) 602 if err := checkDir(dir); err != nil { 603 return nil, err 604 } 605 606 m, err := newJournalManifest(ctx, dir) 607 if err != nil { 608 return nil, err 609 } 610 p := newFSTablePersister(dir, q) 611 612 journal, err := newChunkJournal(ctx, nbfVers, dir, m, p.(*fsTablePersister)) 613 if err != nil { 614 return nil, err 615 } 616 617 mm := makeManifestManager(journal) 618 c := journalConjoiner{child: inlineConjoiner{defaultMaxTables}} 619 620 // |journal| serves as the manifest and tablePersister 621 return newNomsBlockStore(ctx, nbfVers, mm, journal, q, c, defaultMemTableSize) 622 } 623 624 func checkDir(dir string) error { 625 stat, err := os.Stat(dir) 626 if err != nil { 627 return err 628 } 629 if !stat.IsDir() { 630 return fmt.Errorf("path is not a directory: %s", dir) 631 } 632 return nil 633 } 634 635 func newNomsBlockStore(ctx context.Context, nbfVerStr string, mm manifestManager, p tablePersister, q MemoryQuotaProvider, c conjoinStrategy, memTableSize uint64) (*NomsBlockStore, error) { 636 if memTableSize == 0 { 637 memTableSize = defaultMemTableSize 638 } 639 640 hasCache, err := lru.New2Q[hash.Hash, struct{}](hasCacheSize) 641 if err != nil { 642 return nil, err 643 } 644 645 nbs := &NomsBlockStore{ 646 mm: mm, 647 p: p, 648 c: c, 649 tables: newTableSet(p, q), 650 upstream: manifestContents{nbfVers: nbfVerStr}, 651 mtSize: memTableSize, 652 hasCache: hasCache, 653 stats: NewStats(), 654 } 655 nbs.cond = sync.NewCond(&nbs.mu) 656 657 t1 := time.Now() 658 defer nbs.stats.OpenLatency.SampleTimeSince(t1) 659 660 exists, contents, _, err := nbs.mm.Fetch(ctx, nbs.stats) 661 662 if err != nil { 663 return nil, err 664 } 665 666 if exists { 667 newTables, err := nbs.tables.rebase(ctx, contents.specs, nbs.stats) 668 669 if err != nil { 670 return nil, err 671 } 672 673 nbs.upstream = contents 674 oldTables := nbs.tables 675 nbs.tables = newTables 676 err = oldTables.close() 677 if err != nil { 678 return nil, err 679 } 680 } 681 682 return nbs, nil 683 } 684 685 // WithoutConjoiner returns a new *NomsBlockStore instance that will not 686 // conjoin table files during manifest updates. Used in some server-side 687 // contexts when things like table file maintenance is done out-of-process. Not 688 // safe for use outside of NomsBlockStore construction. 689 func (nbs *NomsBlockStore) WithoutConjoiner() *NomsBlockStore { 690 return &NomsBlockStore{ 691 mm: nbs.mm, 692 p: nbs.p, 693 c: noopConjoiner{}, 694 mu: sync.RWMutex{}, 695 mt: nbs.mt, 696 tables: nbs.tables, 697 upstream: nbs.upstream, 698 mtSize: nbs.mtSize, 699 putCount: nbs.putCount, 700 hasCache: nbs.hasCache, 701 stats: nbs.stats, 702 } 703 } 704 705 // Wait for GC to complete to continue with writes 706 func (nbs *NomsBlockStore) waitForGC(ctx context.Context) error { 707 stop := make(chan struct{}) 708 defer close(stop) 709 go func() { 710 select { 711 case <-ctx.Done(): 712 nbs.cond.Broadcast() 713 case <-stop: 714 } 715 }() 716 for nbs.gcInProgress && ctx.Err() == nil { 717 nbs.cond.Wait() 718 } 719 return ctx.Err() 720 } 721 722 func (nbs *NomsBlockStore) Put(ctx context.Context, c chunks.Chunk, getAddrs chunks.GetAddrsCurry) error { 723 return nbs.putChunk(ctx, c, getAddrs, nbs.hasMany) 724 } 725 726 func (nbs *NomsBlockStore) putChunk(ctx context.Context, c chunks.Chunk, getAddrs chunks.GetAddrsCurry, checker refCheck) error { 727 t1 := time.Now() 728 729 success, err := nbs.addChunk(ctx, c, getAddrs, checker) 730 if err != nil { 731 return err 732 } else if !success { 733 return errors.New("failed to add chunk") 734 } 735 atomic.AddUint64(&nbs.putCount, 1) 736 737 nbs.stats.PutLatency.SampleTimeSince(t1) 738 739 return nil 740 } 741 742 // When we have chunks with dangling references in our memtable, we have to 743 // throw away the entire memtable. 744 func (nbs *NomsBlockStore) handlePossibleDanglingRefError(err error) { 745 if errors.Is(err, ErrDanglingRef) { 746 nbs.mt = nil 747 } 748 } 749 750 // Writes to a Dolt database typically involve mutating some tuple maps and 751 // then mutating the top-level address map which points to all the branch heads 752 // and working sets. Each internal node of the address map can have many 753 // references and many of them typically change quite slowly. We keep a cache 754 // of recently written references which we know are in the database so that we 755 // don't have to check the table file indexes for these chunks when we write 756 // references to them again in the near future. 757 // 758 // This cache needs to be treated in a principled manner. The integrity checks 759 // that we run against the a set of chunks we are attempting to write consider 760 // the to-be-written chunks themselves as also being in the database. This is 761 // correct, assuming that all the chunks are written at the same time. However, 762 // we should not add the results of those presence checks to the cache until 763 // those chunks actually land in the database. 764 func (nbs *NomsBlockStore) addPendingRefsToHasCache() { 765 for _, e := range nbs.mt.pendingRefs { 766 if e.has { 767 nbs.hasCache.Add(*e.a, struct{}{}) 768 } 769 } 770 } 771 772 func (nbs *NomsBlockStore) addChunk(ctx context.Context, ch chunks.Chunk, getAddrs chunks.GetAddrsCurry, checker refCheck) (bool, error) { 773 if err := ctx.Err(); err != nil { 774 return false, err 775 } 776 nbs.mu.Lock() 777 defer nbs.mu.Unlock() 778 779 retry := true 780 var addChunkRes addChunkResult 781 for retry { 782 retry = false 783 if nbs.mt == nil { 784 nbs.mt = newMemTable(nbs.mtSize) 785 } 786 787 addChunkRes = nbs.mt.addChunk(ch.Hash(), ch.Data()) 788 if addChunkRes == chunkNotAdded { 789 ts, err := nbs.tables.append(ctx, nbs.mt, checker, nbs.hasCache, nbs.stats) 790 if err != nil { 791 nbs.handlePossibleDanglingRefError(err) 792 return false, err 793 } 794 nbs.addPendingRefsToHasCache() 795 nbs.tables = ts 796 nbs.mt = newMemTable(nbs.mtSize) 797 addChunkRes = nbs.mt.addChunk(ch.Hash(), ch.Data()) 798 } 799 if addChunkRes == chunkAdded || addChunkRes == chunkExists { 800 if nbs.keeperFunc != nil && nbs.keeperFunc(ch.Hash()) { 801 retry = true 802 if err := nbs.waitForGC(ctx); err != nil { 803 return false, err 804 } 805 continue 806 } 807 } 808 if addChunkRes == chunkAdded { 809 nbs.mt.addGetChildRefs(getAddrs(ch)) 810 } 811 } 812 813 return addChunkRes == chunkAdded || addChunkRes == chunkExists, nil 814 } 815 816 // refCheck checks that no dangling references are being committed. 817 type refCheck func(reqs []hasRecord) (hash.HashSet, error) 818 819 func (nbs *NomsBlockStore) errorIfDangling(root hash.Hash, checker refCheck) error { 820 if !root.IsEmpty() { 821 if _, ok := nbs.hasCache.Get(root); !ok { 822 var hr [1]hasRecord 823 hr[0].a = &root 824 hr[0].prefix = root.Prefix() 825 absent, err := checker(hr[:]) 826 if err != nil { 827 return err 828 } else if absent.Size() > 0 { 829 return fmt.Errorf("%w: found dangling references to %s", ErrDanglingRef, absent.String()) 830 } 831 nbs.hasCache.Add(root, struct{}{}) 832 } 833 } 834 return nil 835 } 836 837 func (nbs *NomsBlockStore) Get(ctx context.Context, h hash.Hash) (chunks.Chunk, error) { 838 ctx, span := tracer.Start(ctx, "nbs.Get") 839 defer span.End() 840 841 t1 := time.Now() 842 defer func() { 843 nbs.stats.GetLatency.SampleTimeSince(t1) 844 nbs.stats.ChunksPerGet.Sample(1) 845 }() 846 847 data, tables, err := func() ([]byte, chunkReader, error) { 848 var data []byte 849 nbs.mu.RLock() 850 defer nbs.mu.RUnlock() 851 if nbs.mt != nil { 852 var err error 853 data, err = nbs.mt.get(ctx, h, nbs.stats) 854 855 if err != nil { 856 return nil, nil, err 857 } 858 } 859 return data, nbs.tables, nil 860 }() 861 862 if err != nil { 863 return chunks.EmptyChunk, err 864 } 865 866 if data != nil { 867 return chunks.NewChunkWithHash(h, data), nil 868 } 869 870 data, err = tables.get(ctx, h, nbs.stats) 871 872 if err != nil { 873 return chunks.EmptyChunk, err 874 } 875 876 if data != nil { 877 return chunks.NewChunkWithHash(h, data), nil 878 } 879 880 return chunks.EmptyChunk, nil 881 } 882 883 func (nbs *NomsBlockStore) GetMany(ctx context.Context, hashes hash.HashSet, found func(context.Context, *chunks.Chunk)) error { 884 ctx, span := tracer.Start(ctx, "nbs.GetMany", trace.WithAttributes(attribute.Int("num_hashes", len(hashes)))) 885 span.End() 886 return nbs.getManyWithFunc(ctx, hashes, func(ctx context.Context, cr chunkReader, eg *errgroup.Group, reqs []getRecord, stats *Stats) (bool, error) { 887 return cr.getMany(ctx, eg, reqs, found, nbs.stats) 888 }) 889 } 890 891 func (nbs *NomsBlockStore) GetManyCompressed(ctx context.Context, hashes hash.HashSet, found func(context.Context, CompressedChunk)) error { 892 ctx, span := tracer.Start(ctx, "nbs.GetManyCompressed", trace.WithAttributes(attribute.Int("num_hashes", len(hashes)))) 893 defer span.End() 894 return nbs.getManyWithFunc(ctx, hashes, func(ctx context.Context, cr chunkReader, eg *errgroup.Group, reqs []getRecord, stats *Stats) (bool, error) { 895 return cr.getManyCompressed(ctx, eg, reqs, found, nbs.stats) 896 }) 897 } 898 899 func (nbs *NomsBlockStore) getManyWithFunc( 900 ctx context.Context, 901 hashes hash.HashSet, 902 getManyFunc func(ctx context.Context, cr chunkReader, eg *errgroup.Group, reqs []getRecord, stats *Stats) (bool, error), 903 ) error { 904 t1 := time.Now() 905 reqs := toGetRecords(hashes) 906 907 defer func() { 908 if len(hashes) > 0 { 909 nbs.stats.GetLatency.SampleTimeSince(t1) 910 nbs.stats.ChunksPerGet.Sample(uint64(len(reqs))) 911 } 912 }() 913 914 eg, ctx := errgroup.WithContext(ctx) 915 const ioParallelism = 16 916 eg.SetLimit(ioParallelism) 917 918 tables, remaining, err := func() (tables chunkReader, remaining bool, err error) { 919 nbs.mu.RLock() 920 defer nbs.mu.RUnlock() 921 tables = nbs.tables 922 remaining = true 923 if nbs.mt != nil { 924 remaining, err = getManyFunc(ctx, nbs.mt, eg, reqs, nbs.stats) 925 } 926 return 927 }() 928 if err != nil { 929 return err 930 } 931 932 if remaining { 933 _, err = getManyFunc(ctx, tables, eg, reqs, nbs.stats) 934 } 935 936 if err != nil { 937 eg.Wait() 938 return err 939 } 940 return eg.Wait() 941 } 942 943 func toGetRecords(hashes hash.HashSet) []getRecord { 944 reqs := make([]getRecord, len(hashes)) 945 idx := 0 946 for h := range hashes { 947 h := h 948 reqs[idx] = getRecord{ 949 a: &h, 950 prefix: h.Prefix(), 951 } 952 idx++ 953 } 954 955 sort.Sort(getRecordByPrefix(reqs)) 956 return reqs 957 } 958 959 func (nbs *NomsBlockStore) Count() (uint32, error) { 960 count, tables, err := func() (count uint32, tables chunkReader, err error) { 961 nbs.mu.RLock() 962 defer nbs.mu.RUnlock() 963 if nbs.mt != nil { 964 count, err = nbs.mt.count() 965 } 966 967 if err != nil { 968 return 0, nil, err 969 } 970 971 return count, nbs.tables, nil 972 }() 973 974 if err != nil { 975 return 0, err 976 } 977 978 tablesCount, err := tables.count() 979 980 if err != nil { 981 return 0, err 982 } 983 984 return count + tablesCount, nil 985 } 986 987 func (nbs *NomsBlockStore) Has(ctx context.Context, h hash.Hash) (bool, error) { 988 t1 := time.Now() 989 defer func() { 990 nbs.stats.HasLatency.SampleTimeSince(t1) 991 nbs.stats.AddressesPerHas.Sample(1) 992 }() 993 994 has, tables, err := func() (bool, chunkReader, error) { 995 nbs.mu.RLock() 996 defer nbs.mu.RUnlock() 997 998 if nbs.mt != nil { 999 has, err := nbs.mt.has(h) 1000 1001 if err != nil { 1002 return false, nil, err 1003 } 1004 1005 return has, nbs.tables, nil 1006 } 1007 1008 return false, nbs.tables, nil 1009 }() 1010 1011 if err != nil { 1012 return false, err 1013 } 1014 1015 if !has { 1016 has, err = tables.has(h) 1017 1018 if err != nil { 1019 return false, err 1020 } 1021 } 1022 1023 return has, nil 1024 } 1025 1026 func (nbs *NomsBlockStore) HasMany(ctx context.Context, hashes hash.HashSet) (hash.HashSet, error) { 1027 if hashes.Size() == 0 { 1028 return nil, nil 1029 } 1030 1031 t1 := time.Now() 1032 defer nbs.stats.HasLatency.SampleTimeSince(t1) 1033 nbs.stats.AddressesPerHas.SampleLen(hashes.Size()) 1034 1035 nbs.mu.RLock() 1036 defer nbs.mu.RUnlock() 1037 return nbs.hasMany(toHasRecords(hashes)) 1038 } 1039 1040 func (nbs *NomsBlockStore) hasMany(reqs []hasRecord) (hash.HashSet, error) { 1041 tables, remaining, err := func() (tables chunkReader, remaining bool, err error) { 1042 tables = nbs.tables 1043 1044 remaining = true 1045 if nbs.mt != nil { 1046 remaining, err = nbs.mt.hasMany(reqs) 1047 1048 if err != nil { 1049 return nil, false, err 1050 } 1051 } 1052 1053 return tables, remaining, nil 1054 }() 1055 1056 if err != nil { 1057 return nil, err 1058 } 1059 1060 if remaining { 1061 _, err := tables.hasMany(reqs) 1062 1063 if err != nil { 1064 return nil, err 1065 } 1066 } 1067 1068 absent := hash.HashSet{} 1069 for _, r := range reqs { 1070 if !r.has { 1071 absent.Insert(*r.a) 1072 } 1073 } 1074 return absent, nil 1075 } 1076 1077 func toHasRecords(hashes hash.HashSet) []hasRecord { 1078 reqs := make([]hasRecord, len(hashes)) 1079 idx := 0 1080 for h := range hashes { 1081 h := h 1082 reqs[idx] = hasRecord{ 1083 a: &h, 1084 prefix: h.Prefix(), 1085 order: idx, 1086 } 1087 idx++ 1088 } 1089 1090 sort.Sort(hasRecordByPrefix(reqs)) 1091 return reqs 1092 } 1093 1094 func (nbs *NomsBlockStore) Rebase(ctx context.Context) error { 1095 nbs.mu.Lock() 1096 defer nbs.mu.Unlock() 1097 return nbs.rebase(ctx) 1098 } 1099 1100 func (nbs *NomsBlockStore) rebase(ctx context.Context) error { 1101 exists, contents, _, err := nbs.mm.Fetch(ctx, nbs.stats) 1102 if err != nil { 1103 return err 1104 } 1105 1106 if exists { 1107 if contents.lock == nbs.upstream.lock { 1108 // short-circuit if manifest is unchanged 1109 return nil 1110 } 1111 1112 newTables, err := nbs.tables.rebase(ctx, contents.specs, nbs.stats) 1113 if err != nil { 1114 return err 1115 } 1116 1117 nbs.upstream = contents 1118 oldTables := nbs.tables 1119 nbs.tables = newTables 1120 err = oldTables.close() 1121 if err != nil { 1122 return err 1123 } 1124 } 1125 1126 return nil 1127 } 1128 1129 func (nbs *NomsBlockStore) Root(ctx context.Context) (hash.Hash, error) { 1130 nbs.mu.RLock() 1131 defer nbs.mu.RUnlock() 1132 return nbs.upstream.root, nil 1133 } 1134 1135 func (nbs *NomsBlockStore) Commit(ctx context.Context, current, last hash.Hash) (success bool, err error) { 1136 return nbs.commit(ctx, current, last, nbs.hasMany) 1137 } 1138 1139 func (nbs *NomsBlockStore) commit(ctx context.Context, current, last hash.Hash, checker refCheck) (success bool, err error) { 1140 t1 := time.Now() 1141 defer nbs.stats.CommitLatency.SampleTimeSince(t1) 1142 1143 nbs.mu.Lock() 1144 defer nbs.mu.Unlock() 1145 1146 if nbs.keeperFunc != nil { 1147 if nbs.keeperFunc(current) { 1148 err = nbs.waitForGC(ctx) 1149 if err != nil { 1150 return false, err 1151 } 1152 } 1153 } 1154 1155 anyPossiblyNovelChunks := nbs.mt != nil || len(nbs.tables.novel) > 0 1156 1157 if !anyPossiblyNovelChunks && current == last { 1158 err := nbs.rebase(ctx) 1159 if err != nil { 1160 return false, err 1161 } 1162 return true, nil 1163 } 1164 1165 nbs.mm.LockForUpdate() 1166 defer func() { 1167 unlockErr := nbs.mm.UnlockForUpdate() 1168 1169 if err == nil { 1170 err = unlockErr 1171 } 1172 }() 1173 1174 for { 1175 if err := nbs.updateManifest(ctx, current, last, checker); err == nil { 1176 return true, nil 1177 } else if err == errOptimisticLockFailedRoot || err == errLastRootMismatch { 1178 return false, nil 1179 } else if err != errOptimisticLockFailedTables { 1180 return false, err 1181 } 1182 } 1183 } 1184 1185 var ( 1186 errLastRootMismatch = fmt.Errorf("last does not match nbs.Root()") 1187 errOptimisticLockFailedRoot = fmt.Errorf("root moved") 1188 errOptimisticLockFailedTables = fmt.Errorf("tables changed") 1189 errReadOnlyManifest = fmt.Errorf("cannot update manifest: database is read only") 1190 ) 1191 1192 // callers must acquire lock |nbs.mu| 1193 func (nbs *NomsBlockStore) updateManifest(ctx context.Context, current, last hash.Hash, checker refCheck) error { 1194 if nbs.upstream.root != last { 1195 return errLastRootMismatch 1196 } 1197 1198 handleOptimisticLockFailure := func(upstream manifestContents) error { 1199 newTables, err := nbs.tables.rebase(ctx, upstream.specs, nbs.stats) 1200 if err != nil { 1201 return err 1202 } 1203 1204 nbs.upstream = upstream 1205 oldTables := nbs.tables 1206 nbs.tables = newTables 1207 err = oldTables.close() 1208 1209 if last != upstream.root { 1210 return errOptimisticLockFailedRoot 1211 } 1212 1213 if err != nil { 1214 return err 1215 } 1216 1217 return errOptimisticLockFailedTables 1218 } 1219 1220 if cached, doomed := nbs.mm.updateWillFail(nbs.upstream.lock); doomed { 1221 // Pre-emptive optimistic lock failure. Someone else in-process moved to the root, the set of tables, or both out from under us. 1222 return handleOptimisticLockFailure(cached) 1223 } 1224 1225 if nbs.mt != nil { 1226 cnt, err := nbs.mt.count() 1227 1228 if err != nil { 1229 return err 1230 } 1231 1232 if cnt > 0 { 1233 ts, err := nbs.tables.append(ctx, nbs.mt, checker, nbs.hasCache, nbs.stats) 1234 if err != nil { 1235 nbs.handlePossibleDanglingRefError(err) 1236 return err 1237 } 1238 nbs.addPendingRefsToHasCache() 1239 nbs.tables, nbs.mt = ts, nil 1240 } 1241 } 1242 1243 didConjoin, err := nbs.conjoinIfRequired(ctx) 1244 if err != nil { 1245 return err 1246 } 1247 if didConjoin { 1248 return errOptimisticLockFailedTables 1249 } 1250 1251 // check for dangling reference to the new root 1252 if err = nbs.errorIfDangling(current, checker); err != nil { 1253 nbs.handlePossibleDanglingRefError(err) 1254 return err 1255 } 1256 1257 specs, err := nbs.tables.toSpecs() 1258 if err != nil { 1259 return err 1260 } 1261 1262 // ensure we don't drop appendices on commit 1263 var appendixSpecs []tableSpec 1264 if nbs.upstream.appendix != nil && len(nbs.upstream.appendix) > 0 { 1265 appendixSet := nbs.upstream.getAppendixSet() 1266 1267 filtered := make([]tableSpec, 0, len(specs)) 1268 for _, s := range specs { 1269 if _, present := appendixSet[s.name]; !present { 1270 filtered = append(filtered, s) 1271 } 1272 } 1273 1274 _, appendixSpecs = nbs.upstream.removeAppendixSpecs() 1275 prepended := append([]tableSpec{}, appendixSpecs...) 1276 specs = append(prepended, filtered...) 1277 } 1278 1279 newContents := manifestContents{ 1280 nbfVers: nbs.upstream.nbfVers, 1281 root: current, 1282 lock: generateLockHash(current, specs, appendixSpecs), 1283 gcGen: nbs.upstream.gcGen, 1284 specs: specs, 1285 appendix: appendixSpecs, 1286 } 1287 1288 upstream, err := nbs.mm.Update(ctx, nbs.upstream.lock, newContents, nbs.stats, nil) 1289 if err != nil { 1290 return err 1291 } 1292 1293 if newContents.lock != upstream.lock { 1294 // Optimistic lock failure. Someone else moved to the root, the set of tables, or both out from under us. 1295 return handleOptimisticLockFailure(upstream) 1296 } 1297 1298 newTables, err := nbs.tables.flatten(ctx) 1299 1300 if err != nil { 1301 return err 1302 } 1303 1304 nbs.upstream = newContents 1305 nbs.tables = newTables 1306 1307 return nil 1308 } 1309 1310 func (nbs *NomsBlockStore) Version() string { 1311 nbs.mu.RLock() 1312 defer nbs.mu.RUnlock() 1313 return nbs.upstream.nbfVers 1314 } 1315 1316 func (nbs *NomsBlockStore) AccessMode() chunks.ExclusiveAccessMode { 1317 return nbs.p.AccessMode() 1318 } 1319 1320 func (nbs *NomsBlockStore) Close() (err error) { 1321 if cerr := nbs.p.Close(); cerr != nil { 1322 err = cerr 1323 } 1324 if cerr := nbs.tables.close(); cerr != nil { 1325 err = cerr 1326 } 1327 if cerr := nbs.mm.Close(); cerr != nil { 1328 err = cerr 1329 } 1330 return 1331 } 1332 1333 func (nbs *NomsBlockStore) Stats() interface{} { 1334 return nbs.stats.Clone() 1335 } 1336 1337 func (nbs *NomsBlockStore) StatsSummary() string { 1338 nbs.mu.Lock() 1339 defer nbs.mu.Unlock() 1340 cnt, _ := nbs.tables.count() 1341 physLen, _ := nbs.tables.physicalLen() 1342 return fmt.Sprintf("Root: %s; Chunk Count %d; Physical Bytes %s", nbs.upstream.root, cnt, humanize.Bytes(physLen)) 1343 } 1344 1345 // tableFile is our implementation of TableFile. 1346 type tableFile struct { 1347 info TableSpecInfo 1348 open func(ctx context.Context) (io.ReadCloser, uint64, error) 1349 } 1350 1351 // LocationPrefix 1352 func (tf tableFile) LocationPrefix() string { 1353 return "" 1354 } 1355 1356 // FileID gets the id of the file 1357 func (tf tableFile) FileID() string { 1358 return tf.info.GetName() 1359 } 1360 1361 // NumChunks returns the number of chunks in a table file 1362 func (tf tableFile) NumChunks() int { 1363 return int(tf.info.GetChunkCount()) 1364 } 1365 1366 // Open returns an io.ReadCloser which can be used to read the bytes of a table file and the content length in bytes. 1367 func (tf tableFile) Open(ctx context.Context) (io.ReadCloser, uint64, error) { 1368 return tf.open(ctx) 1369 } 1370 1371 // Sources retrieves the current root hash, a list of all table files (which may include appendix tablefiles), 1372 // and a second list of only the appendix table files 1373 func (nbs *NomsBlockStore) Sources(ctx context.Context) (hash.Hash, []chunks.TableFile, []chunks.TableFile, error) { 1374 nbs.mu.Lock() 1375 defer nbs.mu.Unlock() 1376 1377 exists, contents, err := nbs.mm.m.ParseIfExists(ctx, nbs.stats, nil) 1378 1379 if err != nil { 1380 return hash.Hash{}, nil, nil, err 1381 } 1382 1383 if !exists { 1384 return hash.Hash{}, nil, nil, nil 1385 } 1386 1387 css, err := nbs.chunkSourcesByAddr() 1388 if err != nil { 1389 return hash.Hash{}, nil, nil, err 1390 } 1391 1392 appendixTableFiles, err := getTableFiles(css, contents, contents.NumAppendixSpecs(), func(mc manifestContents, idx int) tableSpec { 1393 return mc.getAppendixSpec(idx) 1394 }) 1395 if err != nil { 1396 return hash.Hash{}, nil, nil, err 1397 } 1398 1399 allTableFiles, err := getTableFiles(css, contents, contents.NumTableSpecs(), func(mc manifestContents, idx int) tableSpec { 1400 return mc.getSpec(idx) 1401 }) 1402 if err != nil { 1403 return hash.Hash{}, nil, nil, err 1404 } 1405 1406 return contents.GetRoot(), allTableFiles, appendixTableFiles, nil 1407 } 1408 1409 func getTableFiles(css map[hash.Hash]chunkSource, contents manifestContents, numSpecs int, specFunc func(mc manifestContents, idx int) tableSpec) ([]chunks.TableFile, error) { 1410 tableFiles := make([]chunks.TableFile, 0) 1411 if numSpecs == 0 { 1412 return tableFiles, nil 1413 } 1414 for i := 0; i < numSpecs; i++ { 1415 info := specFunc(contents, i) 1416 cs, ok := css[info.name] 1417 if !ok { 1418 return nil, ErrSpecWithoutChunkSource 1419 } 1420 tableFiles = append(tableFiles, newTableFile(cs, info)) 1421 } 1422 return tableFiles, nil 1423 } 1424 1425 func newTableFile(cs chunkSource, info tableSpec) tableFile { 1426 return tableFile{ 1427 info: info, 1428 open: func(ctx context.Context) (io.ReadCloser, uint64, error) { 1429 r, s, err := cs.reader(ctx) 1430 if err != nil { 1431 return nil, 0, err 1432 } 1433 return r, s, nil 1434 }, 1435 } 1436 } 1437 1438 func (nbs *NomsBlockStore) Size(ctx context.Context) (uint64, error) { 1439 nbs.mu.Lock() 1440 defer nbs.mu.Unlock() 1441 1442 size := uint64(0) 1443 for _, cs := range nbs.tables.upstream { 1444 size += cs.currentSize() 1445 } 1446 for _, cs := range nbs.tables.novel { 1447 size += cs.currentSize() 1448 } 1449 1450 return size, nil 1451 } 1452 1453 func (nbs *NomsBlockStore) chunkSourcesByAddr() (map[hash.Hash]chunkSource, error) { 1454 css := make(map[hash.Hash]chunkSource, len(nbs.tables.upstream)+len(nbs.tables.novel)) 1455 for _, cs := range nbs.tables.upstream { 1456 css[cs.hash()] = cs 1457 } 1458 for _, cs := range nbs.tables.novel { 1459 css[cs.hash()] = cs 1460 } 1461 return css, nil 1462 1463 } 1464 1465 func (nbs *NomsBlockStore) SupportedOperations() chunks.TableFileStoreOps { 1466 var ok bool 1467 _, ok = nbs.p.(tableFilePersister) 1468 1469 return chunks.TableFileStoreOps{ 1470 CanRead: true, 1471 CanWrite: ok, 1472 CanPrune: ok, 1473 CanGC: ok, 1474 } 1475 } 1476 1477 func (nbs *NomsBlockStore) Path() (string, bool) { 1478 if tfp, ok := nbs.p.(tableFilePersister); ok { 1479 switch p := tfp.(type) { 1480 case *fsTablePersister, *ChunkJournal: 1481 return p.Path(), true 1482 default: 1483 return "", false 1484 } 1485 } 1486 return "", false 1487 } 1488 1489 // WriteTableFile will read a table file from the provided reader and write it to the TableFileStore 1490 func (nbs *NomsBlockStore) WriteTableFile(ctx context.Context, fileId string, numChunks int, contentHash []byte, getRd func() (io.ReadCloser, uint64, error)) error { 1491 tfp, ok := nbs.p.(tableFilePersister) 1492 if !ok { 1493 return errors.New("Not implemented") 1494 } 1495 1496 r, sz, err := getRd() 1497 if err != nil { 1498 return err 1499 } 1500 defer r.Close() 1501 return tfp.CopyTableFile(ctx, r, fileId, sz, uint32(numChunks)) 1502 } 1503 1504 // AddTableFilesToManifest adds table files to the manifest 1505 func (nbs *NomsBlockStore) AddTableFilesToManifest(ctx context.Context, fileIdToNumChunks map[string]int) error { 1506 var totalChunks int 1507 fileIdHashToNumChunks := make(map[hash.Hash]uint32) 1508 for fileId, numChunks := range fileIdToNumChunks { 1509 fileIdHash, ok := hash.MaybeParse(fileId) 1510 1511 if !ok { 1512 return errors.New("invalid base32 encoded hash: " + fileId) 1513 } 1514 1515 fileIdHashToNumChunks[fileIdHash] = uint32(numChunks) 1516 totalChunks += numChunks 1517 } 1518 1519 if totalChunks == 0 { 1520 return nil 1521 } 1522 1523 _, err := nbs.UpdateManifest(ctx, fileIdHashToNumChunks) 1524 return err 1525 } 1526 1527 // PruneTableFiles deletes old table files that are no longer referenced in the manifest. 1528 func (nbs *NomsBlockStore) PruneTableFiles(ctx context.Context) (err error) { 1529 return nbs.pruneTableFiles(ctx, nbs.hasMany) 1530 } 1531 1532 func (nbs *NomsBlockStore) pruneTableFiles(ctx context.Context, checker refCheck) (err error) { 1533 mtime := time.Now() 1534 1535 return nbs.p.PruneTableFiles(ctx, func() []hash.Hash { 1536 nbs.mu.Lock() 1537 defer nbs.mu.Unlock() 1538 keepers := make([]hash.Hash, 0, len(nbs.tables.novel)+len(nbs.tables.upstream)) 1539 for a, _ := range nbs.tables.novel { 1540 keepers = append(keepers, a) 1541 } 1542 for a, _ := range nbs.tables.upstream { 1543 keepers = append(keepers, a) 1544 } 1545 return keepers 1546 }, mtime) 1547 } 1548 1549 func (nbs *NomsBlockStore) BeginGC(keeper func(hash.Hash) bool) error { 1550 nbs.cond.L.Lock() 1551 defer nbs.cond.L.Unlock() 1552 if nbs.gcInProgress { 1553 return errors.New("gc already in progress") 1554 } 1555 nbs.gcInProgress = true 1556 nbs.keeperFunc = keeper 1557 nbs.cond.Broadcast() 1558 return nil 1559 } 1560 1561 func (nbs *NomsBlockStore) EndGC() { 1562 nbs.cond.L.Lock() 1563 defer nbs.cond.L.Unlock() 1564 if !nbs.gcInProgress { 1565 panic("EndGC called when gc was not in progress") 1566 } 1567 nbs.gcInProgress = false 1568 nbs.keeperFunc = nil 1569 nbs.cond.Broadcast() 1570 } 1571 1572 func (nbs *NomsBlockStore) MarkAndSweepChunks(ctx context.Context, hashes <-chan []hash.Hash, dest chunks.ChunkStore) error { 1573 ops := nbs.SupportedOperations() 1574 if !ops.CanGC || !ops.CanPrune { 1575 return chunks.ErrUnsupportedOperation 1576 } 1577 1578 precheck := func() error { 1579 nbs.mu.RLock() 1580 defer nbs.mu.RUnlock() 1581 1582 // check to see if the specs have changed since last gc. If they haven't bail early. 1583 gcGenCheck := generateLockHash(nbs.upstream.root, nbs.upstream.specs, nbs.upstream.appendix) 1584 if nbs.upstream.gcGen == gcGenCheck { 1585 return chunks.ErrNothingToCollect 1586 } 1587 1588 return nil 1589 } 1590 err := precheck() 1591 if err != nil { 1592 return err 1593 } 1594 1595 destNBS := nbs 1596 if dest != nil { 1597 switch typed := dest.(type) { 1598 case *NomsBlockStore: 1599 destNBS = typed 1600 case NBSMetricWrapper: 1601 destNBS = typed.nbs 1602 } 1603 } 1604 1605 specs, err := nbs.copyMarkedChunks(ctx, hashes, destNBS) 1606 if err != nil { 1607 return err 1608 } 1609 if ctx.Err() != nil { 1610 return ctx.Err() 1611 } 1612 1613 if destNBS == nbs { 1614 return nbs.swapTables(ctx, specs) 1615 } else { 1616 fileIdToNumChunks := tableSpecsToMap(specs) 1617 return destNBS.AddTableFilesToManifest(ctx, fileIdToNumChunks) 1618 } 1619 } 1620 1621 func (nbs *NomsBlockStore) copyMarkedChunks(ctx context.Context, keepChunks <-chan []hash.Hash, dest *NomsBlockStore) ([]tableSpec, error) { 1622 tfp, ok := dest.p.(tableFilePersister) 1623 if !ok { 1624 return nil, fmt.Errorf("NBS does not support copying garbage collection") 1625 } 1626 1627 gcc, err := newGarbageCollectionCopier() 1628 if err != nil { 1629 return nil, err 1630 } 1631 1632 // TODO: We should clean up gcc on error. 1633 1634 LOOP: 1635 for { 1636 select { 1637 case hs, ok := <-keepChunks: 1638 if !ok { 1639 break LOOP 1640 } 1641 var addErr error 1642 mu := new(sync.Mutex) 1643 hashset := hash.NewHashSet(hs...) 1644 found := 0 1645 err := nbs.GetManyCompressed(ctx, hashset, func(ctx context.Context, c CompressedChunk) { 1646 mu.Lock() 1647 defer mu.Unlock() 1648 if addErr != nil { 1649 return 1650 } 1651 found += 1 1652 addErr = gcc.addChunk(ctx, c) 1653 }) 1654 if err != nil { 1655 return nil, err 1656 } 1657 if addErr != nil { 1658 return nil, addErr 1659 } 1660 if found != len(hashset) { 1661 return nil, fmt.Errorf("dangling references requested during GC. GC not successful. %v", hashset) 1662 } 1663 case <-ctx.Done(): 1664 return nil, ctx.Err() 1665 } 1666 } 1667 return gcc.copyTablesToDir(ctx, tfp) 1668 } 1669 1670 func (nbs *NomsBlockStore) swapTables(ctx context.Context, specs []tableSpec) (err error) { 1671 nbs.mu.Lock() 1672 defer nbs.mu.Unlock() 1673 1674 nbs.mm.LockForUpdate() 1675 defer func() { 1676 unlockErr := nbs.mm.UnlockForUpdate() 1677 if err == nil { 1678 err = unlockErr 1679 } 1680 }() 1681 1682 newLock := generateLockHash(nbs.upstream.root, specs, []tableSpec{}) 1683 newContents := manifestContents{ 1684 nbfVers: nbs.upstream.nbfVers, 1685 root: nbs.upstream.root, 1686 lock: newLock, 1687 gcGen: newLock, 1688 specs: specs, 1689 } 1690 1691 // Nothing has changed. Bail early. 1692 if newContents.gcGen == nbs.upstream.gcGen { 1693 return nil 1694 } 1695 1696 upstream, uerr := nbs.mm.UpdateGCGen(ctx, nbs.upstream.lock, newContents, nbs.stats, nil) 1697 if uerr != nil { 1698 return uerr 1699 } 1700 1701 if upstream.lock != newContents.lock { 1702 return errors.New("concurrent manifest edit during GC, before swapTables. GC failed.") 1703 } 1704 1705 // We purge the hasCache here, since |swapTables| is the only place where 1706 // chunks can actually be removed from the block store. Other times when 1707 // we update the table set, we are appending new table files to it, or 1708 // replacing table files in it with a file into which they were conjoined. 1709 nbs.hasCache.Purge() 1710 1711 // replace nbs.tables.upstream with gc compacted tables 1712 ts, err := nbs.tables.rebase(ctx, upstream.specs, nbs.stats) 1713 if err != nil { 1714 return err 1715 } 1716 oldTables := nbs.tables 1717 nbs.tables, nbs.upstream = ts, upstream 1718 err = oldTables.close() 1719 if err != nil { 1720 return err 1721 } 1722 1723 // When this is called, we are at a safepoint in the GC process. 1724 // We clear novel and the memtable, which are not coming with us 1725 // into the new store. 1726 oldNovel := nbs.tables.novel 1727 nbs.tables.novel = make(chunkSourceSet) 1728 for _, css := range oldNovel { 1729 err = css.close() 1730 if err != nil { 1731 return err 1732 } 1733 } 1734 if nbs.mt != nil { 1735 var thrown []string 1736 for a := range nbs.mt.chunks { 1737 thrown = append(thrown, a.String()) 1738 } 1739 } 1740 nbs.mt = nil 1741 return nil 1742 } 1743 1744 // SetRootChunk changes the root chunk hash from the previous value to the new root. 1745 func (nbs *NomsBlockStore) SetRootChunk(ctx context.Context, root, previous hash.Hash) error { 1746 return nbs.setRootChunk(ctx, root, previous, nbs.hasMany) 1747 } 1748 1749 func (nbs *NomsBlockStore) setRootChunk(ctx context.Context, root, previous hash.Hash, checker refCheck) error { 1750 nbs.mu.Lock() 1751 defer nbs.mu.Unlock() 1752 err := nbs.waitForGC(ctx) 1753 if err != nil { 1754 return err 1755 } 1756 for { 1757 err := nbs.updateManifest(ctx, root, previous, checker) 1758 1759 if err == nil { 1760 return nil 1761 } else if err == errOptimisticLockFailedTables { 1762 continue 1763 } else { 1764 return err 1765 } 1766 1767 // Same behavior as Commit 1768 // I guess this thing infinitely retries without backoff in the case off errOptimisticLockFailedTables 1769 } 1770 } 1771 1772 // CalcReads computes the number of IO operations necessary to fetch |hashes|. 1773 func CalcReads(nbs *NomsBlockStore, hashes hash.HashSet, blockSize uint64) (reads int, split bool, err error) { 1774 reqs := toGetRecords(hashes) 1775 tables := func() (tables tableSet) { 1776 nbs.mu.RLock() 1777 defer nbs.mu.RUnlock() 1778 tables = nbs.tables 1779 1780 return 1781 }() 1782 1783 reads, split, remaining, err := tableSetCalcReads(tables, reqs, blockSize) 1784 1785 if err != nil { 1786 return 0, false, err 1787 } 1788 1789 if remaining { 1790 return 0, false, errors.New("failed to find all chunks") 1791 } 1792 1793 return 1794 }