mirror of
https://github.com/dolthub/dolt.git
synced 2026-01-25 18:49:36 -06:00
Test changes to account for dropping nbs.addr
This commit is contained in:
@@ -33,6 +33,7 @@ import (
|
||||
"github.com/aws/aws-sdk-go/aws/request"
|
||||
"github.com/aws/aws-sdk-go/service/s3"
|
||||
"github.com/aws/aws-sdk-go/service/s3/s3iface"
|
||||
"github.com/dolthub/dolt/go/store/hash"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
@@ -160,22 +161,22 @@ func TestAWSTablePersisterPersist(t *testing.T) {
|
||||
}
|
||||
|
||||
type waitOnStoreTableCache struct {
|
||||
readers map[addr]io.ReaderAt
|
||||
readers map[hash.Hash]io.ReaderAt
|
||||
mu sync.RWMutex
|
||||
storeWG sync.WaitGroup
|
||||
}
|
||||
|
||||
func (mtc *waitOnStoreTableCache) checkout(h addr) (io.ReaderAt, error) {
|
||||
func (mtc *waitOnStoreTableCache) checkout(h hash.Hash) (io.ReaderAt, error) {
|
||||
mtc.mu.RLock()
|
||||
defer mtc.mu.RUnlock()
|
||||
return mtc.readers[h], nil
|
||||
}
|
||||
|
||||
func (mtc *waitOnStoreTableCache) checkin(h addr) error {
|
||||
func (mtc *waitOnStoreTableCache) checkin(h hash.Hash) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (mtc *waitOnStoreTableCache) store(h addr, data io.Reader, size uint64) error {
|
||||
func (mtc *waitOnStoreTableCache) store(h hash.Hash, data io.Reader, size uint64) error {
|
||||
defer mtc.storeWG.Done()
|
||||
mtc.mu.Lock()
|
||||
defer mtc.mu.Unlock()
|
||||
|
||||
@@ -111,7 +111,7 @@ func TestConjoin(t *testing.T) {
|
||||
|
||||
func testConjoin(t *testing.T, factory func(t *testing.T) tablePersister) {
|
||||
stats := &Stats{}
|
||||
setup := func(lock addr, root hash.Hash, sizes []uint32) (fm *fakeManifest, p tablePersister, upstream manifestContents) {
|
||||
setup := func(lock hash.Hash, root hash.Hash, sizes []uint32) (fm *fakeManifest, p tablePersister, upstream manifestContents) {
|
||||
p = factory(t)
|
||||
fm = &fakeManifest{}
|
||||
fm.set(constants.FormatLD1String, lock, root, makeTestTableSpecs(t, sizes, p), nil)
|
||||
@@ -257,7 +257,7 @@ func testConjoin(t *testing.T, factory func(t *testing.T) tablePersister) {
|
||||
}
|
||||
})
|
||||
|
||||
setupAppendix := func(lock addr, root hash.Hash, specSizes, appendixSizes []uint32) (fm *fakeManifest, p tablePersister, upstream manifestContents) {
|
||||
setupAppendix := func(lock hash.Hash, root hash.Hash, specSizes, appendixSizes []uint32) (fm *fakeManifest, p tablePersister, upstream manifestContents) {
|
||||
p = newFakeTablePersister(&UnlimitedQuotaProvider{})
|
||||
fm = &fakeManifest{}
|
||||
fm.set(constants.FormatLD1String, lock, root, makeTestTableSpecs(t, specSizes, p), makeTestTableSpecs(t, appendixSizes, p))
|
||||
@@ -403,7 +403,7 @@ type updatePreemptManifest struct {
|
||||
preUpdate func()
|
||||
}
|
||||
|
||||
func (u updatePreemptManifest) Update(ctx context.Context, lastLock addr, newContents manifestContents, stats *Stats, writeHook func() error) (manifestContents, error) {
|
||||
func (u updatePreemptManifest) Update(ctx context.Context, lastLock hash.Hash, newContents manifestContents, stats *Stats, writeHook func() error) (manifestContents, error) {
|
||||
if u.preUpdate != nil {
|
||||
u.preUpdate()
|
||||
}
|
||||
|
||||
@@ -110,7 +110,7 @@ func TestDynamoManifestUpdate(t *testing.T) {
|
||||
|
||||
// First, test winning the race against another process.
|
||||
contents := makeContents("locker", "nuroot", []tableSpec{{computeAddr([]byte("a")), 3}}, nil)
|
||||
upstream, err := mm.Update(context.Background(), addr{}, contents, stats, func() error {
|
||||
upstream, err := mm.Update(context.Background(), hash.Hash{}, contents, stats, func() error {
|
||||
// This should fail to get the lock, and therefore _not_ clobber the manifest. So the Update should succeed.
|
||||
lock := computeAddr([]byte("nolock"))
|
||||
newRoot2 := hash.Of([]byte("noroot"))
|
||||
@@ -124,7 +124,7 @@ func TestDynamoManifestUpdate(t *testing.T) {
|
||||
|
||||
// Now, test the case where the optimistic lock fails, and someone else updated the root since last we checked.
|
||||
rejected := makeContents("locker 2", "new root 2", nil, nil)
|
||||
upstream, err = mm.Update(context.Background(), addr{}, rejected, stats, nil)
|
||||
upstream, err = mm.Update(context.Background(), hash.Hash{}, rejected, stats, nil)
|
||||
require.NoError(t, err)
|
||||
assert.Equal(contents.lock, upstream.lock)
|
||||
assert.Equal(contents.root, upstream.root)
|
||||
@@ -162,7 +162,7 @@ func TestDynamoManifestUpdateAppendix(t *testing.T) {
|
||||
app := []tableSpec{{computeAddr([]byte("app-a")), 3}}
|
||||
contents := makeContents("locker", "nuroot", specs, app)
|
||||
|
||||
upstream, err := mm.Update(context.Background(), addr{}, contents, stats, func() error {
|
||||
upstream, err := mm.Update(context.Background(), hash.Hash{}, contents, stats, func() error {
|
||||
// This should fail to get the lock, and therefore _not_ clobber the manifest. So the Update should succeed.
|
||||
lock := computeAddr([]byte("nolock"))
|
||||
newRoot2 := hash.Of([]byte("noroot"))
|
||||
@@ -177,7 +177,7 @@ func TestDynamoManifestUpdateAppendix(t *testing.T) {
|
||||
|
||||
// Now, test the case where the optimistic lock fails, and someone else updated the root since last we checked.
|
||||
rejected := makeContents("locker 2", "new root 2", nil, nil)
|
||||
upstream, err = mm.Update(context.Background(), addr{}, rejected, stats, nil)
|
||||
upstream, err = mm.Update(context.Background(), hash.Hash{}, rejected, stats, nil)
|
||||
require.NoError(t, err)
|
||||
assert.Equal(contents.lock, upstream.lock)
|
||||
assert.Equal(contents.root, upstream.root)
|
||||
@@ -232,7 +232,7 @@ func TestDynamoManifestCaching(t *testing.T) {
|
||||
// When failing the optimistic lock, we should hit persistent storage.
|
||||
reads = ddb.NumGets()
|
||||
contents := makeContents("lock2", "nuroot", []tableSpec{{computeAddr([]byte("a")), 3}}, nil)
|
||||
upstream, err := mm.Update(context.Background(), addr{}, contents, stats, nil)
|
||||
upstream, err := mm.Update(context.Background(), hash.Hash{}, contents, stats, nil)
|
||||
require.NoError(t, err)
|
||||
assert.NotEqual(contents.lock, upstream.lock)
|
||||
assert.Equal(reads+1, ddb.NumGets())
|
||||
@@ -251,7 +251,7 @@ func TestDynamoManifestUpdateEmpty(t *testing.T) {
|
||||
stats := &Stats{}
|
||||
|
||||
contents := manifestContents{nbfVers: constants.FormatLD1String, lock: computeAddr([]byte{0x01})}
|
||||
upstream, err := mm.Update(context.Background(), addr{}, contents, stats, nil)
|
||||
upstream, err := mm.Update(context.Background(), hash.Hash{}, contents, stats, nil)
|
||||
require.NoError(t, err)
|
||||
assert.Equal(contents.lock, upstream.lock)
|
||||
assert.True(upstream.root.IsEmpty())
|
||||
|
||||
@@ -60,7 +60,7 @@ func TestFileManifestLoadIfExists(t *testing.T) {
|
||||
jerk := computeAddr([]byte("jerk"))
|
||||
newRoot := hash.Of([]byte("new root"))
|
||||
tableName := hash.Of([]byte("table1"))
|
||||
gcGen := addr{}
|
||||
gcGen := hash.Hash{}
|
||||
m := strings.Join([]string{StorageVersion, "0", jerk.String(), newRoot.String(), gcGen.String(), tableName.String(), "0"}, ":")
|
||||
err = clobberManifest(fm.dir, m)
|
||||
require.NoError(t, err)
|
||||
@@ -85,11 +85,11 @@ func TestFileManifestUpdateWontClobberOldVersion(t *testing.T) {
|
||||
stats := &Stats{}
|
||||
|
||||
// Simulate another process having already put old Noms data in dir/.
|
||||
m := strings.Join([]string{StorageVersion, "0", addr{}.String(), hash.Hash{}.String(), addr{}.String()}, ":")
|
||||
m := strings.Join([]string{StorageVersion, "0", hash.Hash{}.String(), hash.Hash{}.String(), hash.Hash{}.String()}, ":")
|
||||
err := clobberManifest(fm.dir, m)
|
||||
require.NoError(t, err)
|
||||
|
||||
_, err = fm.Update(context.Background(), addr{}, manifestContents{}, stats, nil)
|
||||
_, err = fm.Update(context.Background(), hash.Hash{}, manifestContents{}, stats, nil)
|
||||
assert.Error(err)
|
||||
}
|
||||
|
||||
@@ -100,7 +100,7 @@ func TestFileManifestUpdateEmpty(t *testing.T) {
|
||||
stats := &Stats{}
|
||||
|
||||
l := computeAddr([]byte{0x01})
|
||||
upstream, err := fm.Update(context.Background(), addr{}, manifestContents{nbfVers: constants.FormatLD1String, lock: l}, stats, nil)
|
||||
upstream, err := fm.Update(context.Background(), hash.Hash{}, manifestContents{nbfVers: constants.FormatLD1String, lock: l}, stats, nil)
|
||||
require.NoError(t, err)
|
||||
assert.Equal(l, upstream.lock)
|
||||
assert.True(upstream.root.IsEmpty())
|
||||
@@ -136,11 +136,11 @@ func TestFileManifestUpdate(t *testing.T) {
|
||||
root: hash.Of([]byte("new root")),
|
||||
specs: []tableSpec{{computeAddr([]byte("a")), 3}},
|
||||
}
|
||||
upstream, err := fm.Update(context.Background(), addr{}, contents, stats, func() error {
|
||||
upstream, err := fm.Update(context.Background(), hash.Hash{}, contents, stats, func() error {
|
||||
// This should fail to get the lock, and therefore _not_ clobber the manifest. So the Update should succeed.
|
||||
lock := computeAddr([]byte("nolock"))
|
||||
newRoot2 := hash.Of([]byte("noroot"))
|
||||
gcGen := addr{}
|
||||
gcGen := hash.Hash{}
|
||||
m := strings.Join([]string{StorageVersion, constants.FormatLD1String, lock.String(), newRoot2.String(), gcGen.String()}, ":")
|
||||
b, err := tryClobberManifest(fm.dir, m)
|
||||
require.NoError(t, err, string(b))
|
||||
@@ -153,7 +153,7 @@ func TestFileManifestUpdate(t *testing.T) {
|
||||
|
||||
// Now, test the case where the optimistic lock fails, and someone else updated the root since last we checked.
|
||||
contents2 := manifestContents{lock: computeAddr([]byte("locker 2")), root: hash.Of([]byte("new root 2")), nbfVers: constants.FormatLD1String}
|
||||
upstream, err = fm.Update(context.Background(), addr{}, contents2, stats, nil)
|
||||
upstream, err = fm.Update(context.Background(), hash.Hash{}, contents2, stats, nil)
|
||||
require.NoError(t, err)
|
||||
assert.Equal(contents.lock, upstream.lock)
|
||||
assert.Equal(contents.root, upstream.root)
|
||||
@@ -167,7 +167,7 @@ func TestFileManifestUpdate(t *testing.T) {
|
||||
// Now, test the case where the optimistic lock fails because someone else updated only the tables since last we checked
|
||||
jerkLock := computeAddr([]byte("jerk"))
|
||||
tableName := computeAddr([]byte("table1"))
|
||||
gcGen := addr{}
|
||||
gcGen := hash.Hash{}
|
||||
m := strings.Join([]string{StorageVersion, constants.FormatLD1String, jerkLock.String(), contents2.root.String(), gcGen.String(), tableName.String(), "1"}, ":")
|
||||
err = clobberManifest(fm.dir, m)
|
||||
require.NoError(t, err)
|
||||
|
||||
@@ -30,6 +30,7 @@ import (
|
||||
"testing"
|
||||
|
||||
"github.com/dolthub/dolt/go/libraries/utils/file"
|
||||
"github.com/dolthub/dolt/go/store/hash"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
@@ -41,23 +42,23 @@ func makeTempDir(t *testing.T) string {
|
||||
return dir
|
||||
}
|
||||
|
||||
func writeTableData(dir string, chunx ...[]byte) (addr, error) {
|
||||
func writeTableData(dir string, chunx ...[]byte) (hash.Hash, error) {
|
||||
tableData, name, err := buildTable(chunx)
|
||||
|
||||
if err != nil {
|
||||
return addr{}, err
|
||||
return hash.Hash{}, err
|
||||
}
|
||||
|
||||
err = os.WriteFile(filepath.Join(dir, name.String()), tableData, 0666)
|
||||
|
||||
if err != nil {
|
||||
return addr{}, err
|
||||
return hash.Hash{}, err
|
||||
}
|
||||
|
||||
return name, nil
|
||||
}
|
||||
|
||||
func removeTables(dir string, names ...addr) error {
|
||||
func removeTables(dir string, names ...hash.Hash) error {
|
||||
for _, name := range names {
|
||||
if err := file.Remove(filepath.Join(dir, name.String())); err != nil {
|
||||
return err
|
||||
|
||||
@@ -22,6 +22,7 @@ import (
|
||||
"math/rand"
|
||||
"testing"
|
||||
|
||||
"github.com/dolthub/dolt/go/store/hash"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/dolthub/dolt/go/libraries/utils/test"
|
||||
@@ -166,7 +167,7 @@ func TestIndexTransformer(t *testing.T) {
|
||||
offsetBytes := get64Bytes(offsets)
|
||||
|
||||
tupleBytes := test.RandomData(chunkCount * prefixTupleSize)
|
||||
suffixBytes := test.RandomData(chunkCount * addrSuffixSize)
|
||||
suffixBytes := test.RandomData(chunkCount * hash.SuffixLen)
|
||||
|
||||
var inBytes []byte
|
||||
inBytes = append(inBytes, tupleBytes...)
|
||||
|
||||
@@ -190,12 +190,12 @@ func mustPayload(rec indexRec) []byte {
|
||||
|
||||
func makeLookups(cnt int) (lookups []lookup) {
|
||||
lookups = make([]lookup, cnt)
|
||||
buf := make([]byte, cnt*addrSize)
|
||||
buf := make([]byte, cnt*hash.ByteLen)
|
||||
rand.Read(buf)
|
||||
var off uint64
|
||||
for i := range lookups {
|
||||
copy(lookups[i].a[:], buf)
|
||||
buf = buf[addrSize:]
|
||||
buf = buf[hash.ByteLen:]
|
||||
lookups[i].r.Offset = off
|
||||
l := rand.Uint32() % 1024
|
||||
lookups[i].r.Length = l
|
||||
|
||||
@@ -127,11 +127,11 @@ func TestProcessJournalRecords(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
func randomMemTable(cnt int) (*memTable, map[addr]chunks.Chunk) {
|
||||
chnx := make(map[addr]chunks.Chunk, cnt)
|
||||
func randomMemTable(cnt int) (*memTable, map[hash.Hash]chunks.Chunk) {
|
||||
chnx := make(map[hash.Hash]chunks.Chunk, cnt)
|
||||
for i := 0; i < cnt; i++ {
|
||||
ch := chunks.NewChunk(randBuf(100))
|
||||
chnx[addr(ch.Hash())] = ch
|
||||
chnx[ch.Hash()] = ch
|
||||
}
|
||||
mt := newMemTable(uint64(cnt) * 256)
|
||||
for a, ch := range chnx {
|
||||
@@ -173,7 +173,7 @@ func makeChunkRecord() (journalRec, []byte) {
|
||||
r := journalRec{
|
||||
length: uint32(len(buf)),
|
||||
kind: chunkJournalRecKind,
|
||||
address: addr(cc.H),
|
||||
address: cc.H,
|
||||
payload: payload,
|
||||
checksum: c,
|
||||
}
|
||||
@@ -181,7 +181,7 @@ func makeChunkRecord() (journalRec, []byte) {
|
||||
}
|
||||
|
||||
func makeRootHashRecord() (journalRec, []byte) {
|
||||
a := addr(hash.Of(randBuf(8)))
|
||||
a := hash.Of(randBuf(8))
|
||||
var n int
|
||||
buf := make([]byte, rootHashRecordSize())
|
||||
// length
|
||||
|
||||
@@ -112,17 +112,17 @@ func TestReadRecordRanges(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
|
||||
for h, rng := range ranges {
|
||||
b, err := jcs.get(ctx, addr(h), &Stats{})
|
||||
b, err := jcs.get(ctx, h, &Stats{})
|
||||
assert.NoError(t, err)
|
||||
ch1 := chunks.NewChunkWithHash(h, b)
|
||||
assert.Equal(t, data[addr(h)], ch1)
|
||||
assert.Equal(t, data[h], ch1)
|
||||
|
||||
start, stop := rng.Offset, uint32(rng.Offset)+rng.Length
|
||||
cc2, err := NewCompressedChunk(h, buf[start:stop])
|
||||
assert.NoError(t, err)
|
||||
ch2, err := cc2.ToChunk()
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, data[addr(h)], ch2)
|
||||
assert.Equal(t, data[h], ch2)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -234,7 +234,7 @@ func TestJournalWriterBootstrap(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func validateAllLookups(t *testing.T, j *journalWriter, data map[addr]CompressedChunk) {
|
||||
func validateAllLookups(t *testing.T, j *journalWriter, data map[hash.Hash]CompressedChunk) {
|
||||
// move |data| to addr16-keyed map
|
||||
prefixMap := make(map[addr16]CompressedChunk, len(data))
|
||||
var prefix addr16
|
||||
@@ -249,7 +249,7 @@ func validateAllLookups(t *testing.T, j *journalWriter, data map[addr]Compressed
|
||||
}
|
||||
|
||||
func iterRangeIndex(idx rangeIndex, cb func(addr16, Range) (stop bool)) {
|
||||
idx.novel.Iter(func(a addr, r Range) (stop bool) {
|
||||
idx.novel.Iter(func(a hash.Hash, r Range) (stop bool) {
|
||||
return cb(toAddr16(a), r)
|
||||
})
|
||||
idx.cached.Iter(cb)
|
||||
@@ -285,7 +285,7 @@ func newTestFilePath(t *testing.T) string {
|
||||
func TestJournalIndexBootstrap(t *testing.T) {
|
||||
// potentially indexed region of a journal
|
||||
type epoch struct {
|
||||
records map[addr]CompressedChunk
|
||||
records map[hash.Hash]CompressedChunk
|
||||
last hash.Hash
|
||||
}
|
||||
|
||||
@@ -396,8 +396,8 @@ func TestJournalIndexBootstrap(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func randomCompressedChunks(cnt int) (compressed map[addr]CompressedChunk) {
|
||||
compressed = make(map[addr]CompressedChunk)
|
||||
func randomCompressedChunks(cnt int) (compressed map[hash.Hash]CompressedChunk) {
|
||||
compressed = make(map[hash.Hash]CompressedChunk)
|
||||
var buf []byte
|
||||
for i := 0; i < cnt; i++ {
|
||||
k := rand.Intn(51) + 50
|
||||
@@ -407,7 +407,7 @@ func randomCompressedChunks(cnt int) (compressed map[addr]CompressedChunk) {
|
||||
}
|
||||
c := chunks.NewChunk(buf[:k])
|
||||
buf = buf[k:]
|
||||
compressed[addr(c.Hash())] = ChunkToCompressedChunk(c)
|
||||
compressed[c.Hash()] = ChunkToCompressedChunk(c)
|
||||
}
|
||||
return
|
||||
}
|
||||
@@ -427,10 +427,10 @@ func TestRangeIndex(t *testing.T) {
|
||||
data := randomCompressedChunks(1024)
|
||||
idx := newRangeIndex()
|
||||
for _, c := range data {
|
||||
idx.put(addr(c.Hash()), Range{})
|
||||
idx.put(c.Hash(), Range{})
|
||||
}
|
||||
for _, c := range data {
|
||||
_, ok := idx.get(addr(c.Hash()))
|
||||
_, ok := idx.get(c.Hash())
|
||||
assert.True(t, ok)
|
||||
}
|
||||
assert.Equal(t, len(data), idx.novelCount())
|
||||
|
||||
@@ -28,6 +28,7 @@ import (
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/dolthub/dolt/go/store/hash"
|
||||
"github.com/golang/snappy"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
@@ -243,7 +244,7 @@ func (o *outOfLineSnappy) Encode(dst, src []byte) []byte {
|
||||
|
||||
type chunkReaderGroup []chunkReader
|
||||
|
||||
func (crg chunkReaderGroup) has(h addr) (bool, error) {
|
||||
func (crg chunkReaderGroup) has(h hash.Hash) (bool, error) {
|
||||
for _, haver := range crg {
|
||||
ok, err := haver.has(h)
|
||||
|
||||
@@ -258,7 +259,7 @@ func (crg chunkReaderGroup) has(h addr) (bool, error) {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
func (crg chunkReaderGroup) get(ctx context.Context, h addr, stats *Stats) ([]byte, error) {
|
||||
func (crg chunkReaderGroup) get(ctx context.Context, h hash.Hash, stats *Stats) ([]byte, error) {
|
||||
for _, haver := range crg {
|
||||
if data, err := haver.get(ctx, h, stats); err != nil {
|
||||
return nil, err
|
||||
|
||||
@@ -14,7 +14,10 @@
|
||||
|
||||
package nbs
|
||||
|
||||
import "github.com/dolthub/dolt/go/store/d"
|
||||
import (
|
||||
"github.com/dolthub/dolt/go/store/d"
|
||||
"github.com/dolthub/dolt/go/store/hash"
|
||||
)
|
||||
|
||||
func mustUint32(val uint32, err error) uint32 {
|
||||
d.PanicIfError(err)
|
||||
@@ -26,7 +29,7 @@ func mustUint64(val uint64, err error) uint64 {
|
||||
return val
|
||||
}
|
||||
|
||||
func mustAddr(h addr, err error) addr {
|
||||
func mustAddr(h hash.Hash, err error) hash.Hash {
|
||||
d.PanicIfError(err)
|
||||
return h
|
||||
}
|
||||
|
||||
@@ -446,7 +446,7 @@ func (fm *fakeManifest) Name() string { return fm.name }
|
||||
func (fm *fakeManifest) ParseIfExists(ctx context.Context, stats *Stats, readHook func() error) (bool, manifestContents, error) {
|
||||
fm.mu.RLock()
|
||||
defer fm.mu.RUnlock()
|
||||
if fm.contents.lock != (addr{}) {
|
||||
if fm.contents.lock != (hash.Hash{}) {
|
||||
return true, fm.contents, nil
|
||||
}
|
||||
|
||||
@@ -458,7 +458,7 @@ func (fm *fakeManifest) ParseIfExists(ctx context.Context, stats *Stats, readHoo
|
||||
// to |newLock|, |fm.root| is set to |newRoot|, and the contents of |specs|
|
||||
// replace |fm.tableSpecs|. If |lastLock| != |fm.lock|, then the update
|
||||
// fails. Regardless of success or failure, the current state is returned.
|
||||
func (fm *fakeManifest) Update(ctx context.Context, lastLock addr, newContents manifestContents, stats *Stats, writeHook func() error) (manifestContents, error) {
|
||||
func (fm *fakeManifest) Update(ctx context.Context, lastLock hash.Hash, newContents manifestContents, stats *Stats, writeHook func() error) (manifestContents, error) {
|
||||
fm.mu.Lock()
|
||||
defer fm.mu.Unlock()
|
||||
if fm.contents.lock == lastLock {
|
||||
@@ -467,7 +467,7 @@ func (fm *fakeManifest) Update(ctx context.Context, lastLock addr, newContents m
|
||||
nbfVers: newContents.nbfVers,
|
||||
lock: newContents.lock,
|
||||
root: newContents.root,
|
||||
gcGen: addr(hash.Hash{}),
|
||||
gcGen: hash.Hash{},
|
||||
}
|
||||
fm.contents.specs = make([]tableSpec, len(newContents.specs))
|
||||
copy(fm.contents.specs, newContents.specs)
|
||||
@@ -479,13 +479,13 @@ func (fm *fakeManifest) Update(ctx context.Context, lastLock addr, newContents m
|
||||
return fm.contents, nil
|
||||
}
|
||||
|
||||
func (fm *fakeManifest) set(version string, lock addr, root hash.Hash, specs, appendix []tableSpec) {
|
||||
func (fm *fakeManifest) set(version string, lock hash.Hash, root hash.Hash, specs, appendix []tableSpec) {
|
||||
fm.contents = manifestContents{
|
||||
manifestVers: StorageVersion,
|
||||
nbfVers: version,
|
||||
lock: lock,
|
||||
root: root,
|
||||
gcGen: addr(hash.Hash{}),
|
||||
gcGen: hash.Hash{},
|
||||
specs: specs,
|
||||
appendix: appendix,
|
||||
}
|
||||
@@ -496,14 +496,14 @@ func newFakeTableSet(q MemoryQuotaProvider) tableSet {
|
||||
}
|
||||
|
||||
func newFakeTablePersister(q MemoryQuotaProvider) fakeTablePersister {
|
||||
return fakeTablePersister{q, map[addr][]byte{}, map[addr]bool{}, map[addr]bool{}, &sync.RWMutex{}}
|
||||
return fakeTablePersister{q, map[hash.Hash][]byte{}, map[hash.Hash]bool{}, map[hash.Hash]bool{}, &sync.RWMutex{}}
|
||||
}
|
||||
|
||||
type fakeTablePersister struct {
|
||||
q MemoryQuotaProvider
|
||||
sources map[addr][]byte
|
||||
sourcesToFail map[addr]bool
|
||||
opened map[addr]bool
|
||||
sources map[hash.Hash][]byte
|
||||
sourcesToFail map[hash.Hash]bool
|
||||
opened map[hash.Hash]bool
|
||||
mu *sync.RWMutex
|
||||
}
|
||||
|
||||
@@ -561,7 +561,7 @@ func (ftp fakeTablePersister) ConjoinAll(ctx context.Context, sources chunkSourc
|
||||
return chunkSourceAdapter{cs, name}, func() {}, nil
|
||||
}
|
||||
|
||||
func compactSourcesToBuffer(sources chunkSources) (name addr, data []byte, chunkCount uint32, err error) {
|
||||
func compactSourcesToBuffer(sources chunkSources) (name hash.Hash, data []byte, chunkCount uint32, err error) {
|
||||
totalData := uint64(0)
|
||||
for _, src := range sources {
|
||||
chunkCount += mustUint32(src.count())
|
||||
@@ -599,19 +599,19 @@ func compactSourcesToBuffer(sources chunkSources) (name addr, data []byte, chunk
|
||||
}
|
||||
|
||||
if errString != "" {
|
||||
return addr{}, nil, 0, fmt.Errorf(errString)
|
||||
return hash.Hash{}, nil, 0, fmt.Errorf(errString)
|
||||
}
|
||||
|
||||
tableSize, name, err := tw.finish()
|
||||
|
||||
if err != nil {
|
||||
return addr{}, nil, 0, err
|
||||
return hash.Hash{}, nil, 0, err
|
||||
}
|
||||
|
||||
return name, buff[:tableSize], chunkCount, nil
|
||||
}
|
||||
|
||||
func (ftp fakeTablePersister) Open(ctx context.Context, name addr, chunkCount uint32, stats *Stats) (chunkSource, error) {
|
||||
func (ftp fakeTablePersister) Open(ctx context.Context, name hash.Hash, chunkCount uint32, stats *Stats) (chunkSource, error) {
|
||||
ftp.mu.Lock()
|
||||
defer ftp.mu.Unlock()
|
||||
|
||||
@@ -633,14 +633,14 @@ func (ftp fakeTablePersister) Open(ctx context.Context, name addr, chunkCount ui
|
||||
return chunkSourceAdapter{cs, name}, nil
|
||||
}
|
||||
|
||||
func (ftp fakeTablePersister) Exists(ctx context.Context, name addr, chunkCount uint32, stats *Stats) (bool, error) {
|
||||
func (ftp fakeTablePersister) Exists(ctx context.Context, name hash.Hash, chunkCount uint32, stats *Stats) (bool, error) {
|
||||
if _, ok := ftp.sourcesToFail[name]; ok {
|
||||
return false, errors.New("intentional failure")
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
|
||||
func (ftp fakeTablePersister) PruneTableFiles(_ context.Context, _ func() []addr, _ time.Time) error {
|
||||
func (ftp fakeTablePersister) PruneTableFiles(_ context.Context, _ func() []hash.Hash, _ time.Time) error {
|
||||
return chunks.ErrUnsupportedOperation
|
||||
}
|
||||
|
||||
@@ -658,18 +658,18 @@ func extractAllChunks(ctx context.Context, src chunkSource, cb func(rec extractR
|
||||
return err
|
||||
}
|
||||
|
||||
var a addr
|
||||
for i := uint32(0); i < index.chunkCount(); i++ {
|
||||
_, err = index.indexEntry(i, &a)
|
||||
var h hash.Hash
|
||||
_, err = index.indexEntry(i, &h)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
data, err := src.get(ctx, a, nil)
|
||||
data, err := src.get(ctx, h, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
cb(extractRecord{a: a, data: data})
|
||||
cb(extractRecord{a: h, data: data})
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
@@ -77,7 +77,7 @@ type fakeS3Multipart struct {
|
||||
etags []string
|
||||
}
|
||||
|
||||
func (m *fakeS3) readerForTable(ctx context.Context, name addr) (chunkReader, error) {
|
||||
func (m *fakeS3) readerForTable(ctx context.Context, name hash.Hash) (chunkReader, error) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
if buff, present := m.data[name.String()]; present {
|
||||
@@ -95,7 +95,7 @@ func (m *fakeS3) readerForTable(ctx context.Context, name addr) (chunkReader, er
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (m *fakeS3) readerForTableWithNamespace(ctx context.Context, ns string, name addr) (chunkReader, error) {
|
||||
func (m *fakeS3) readerForTableWithNamespace(ctx context.Context, ns string, name hash.Hash) (chunkReader, error) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
key := name.String()
|
||||
|
||||
@@ -49,7 +49,7 @@ func makeTestLocalStore(t *testing.T, maxTableFiles int) (st *NomsBlockStore, no
|
||||
// create a v5 manifest
|
||||
fm, err := getFileManifest(ctx, nomsDir, asyncFlush)
|
||||
require.NoError(t, err)
|
||||
_, err = fm.Update(ctx, addr{}, manifestContents{}, &Stats{}, nil)
|
||||
_, err = fm.Update(ctx, hash.Hash{}, manifestContents{}, &Stats{}, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
q = NewUnlimitedMemQuotaProvider()
|
||||
|
||||
@@ -21,6 +21,7 @@ import (
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/dolthub/dolt/go/store/hash"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
@@ -36,9 +37,9 @@ func TestParseTableIndex(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
defer idx.Close()
|
||||
assert.Equal(t, uint32(596), idx.chunkCount())
|
||||
seen := make(map[addr]bool)
|
||||
seen := make(map[hash.Hash]bool)
|
||||
for i := uint32(0); i < idx.chunkCount(); i++ {
|
||||
var onheapaddr addr
|
||||
var onheapaddr hash.Hash
|
||||
e, err := idx.indexEntry(i, &onheapaddr)
|
||||
require.NoError(t, err)
|
||||
if _, ok := seen[onheapaddr]; !ok {
|
||||
@@ -229,25 +230,24 @@ func TestAmbiguousShortHash(t *testing.T) {
|
||||
|
||||
// fakeChunk is chunk with a faked address
|
||||
type fakeChunk struct {
|
||||
address addr
|
||||
address hash.Hash
|
||||
data []byte
|
||||
}
|
||||
|
||||
var fakeData = []byte("supercalifragilisticexpialidocious")
|
||||
|
||||
func addrFromPrefix(prefix string) (a addr) {
|
||||
func addrFromPrefix(prefix string) hash.Hash {
|
||||
// create a full length addr from a prefix
|
||||
for i := 0; i < addrSize; i++ {
|
||||
for i := 0; i < hash.ByteLen; i++ {
|
||||
prefix += "0"
|
||||
}
|
||||
|
||||
// base32 decode string
|
||||
h, _ := encoding.DecodeString(prefix)
|
||||
copy(a[:], h)
|
||||
return
|
||||
return hash.New(h[:hash.ByteLen])
|
||||
}
|
||||
|
||||
func buildFakeChunkTable(chunks []fakeChunk) ([]byte, addr, error) {
|
||||
func buildFakeChunkTable(chunks []fakeChunk) ([]byte, hash.Hash, error) {
|
||||
totalData := uint64(0)
|
||||
for _, chunk := range chunks {
|
||||
totalData += uint64(len(chunk.data))
|
||||
@@ -265,7 +265,7 @@ func buildFakeChunkTable(chunks []fakeChunk) ([]byte, addr, error) {
|
||||
length, blockHash, err := tw.finish()
|
||||
|
||||
if err != nil {
|
||||
return nil, addr{}, err
|
||||
return nil, hash.Hash{}, err
|
||||
}
|
||||
|
||||
return buff[:length], blockHash, nil
|
||||
|
||||
@@ -39,7 +39,7 @@ var hasManyHasAll = func([]hasRecord) (hash.HashSet, error) {
|
||||
}
|
||||
|
||||
func TestTableSetPrependEmpty(t *testing.T) {
|
||||
hasCache, err := lru.New2Q[addr, struct{}](1024)
|
||||
hasCache, err := lru.New2Q[hash.Hash, struct{}](1024)
|
||||
require.NoError(t, err)
|
||||
ts, err := newFakeTableSet(&UnlimitedQuotaProvider{}).append(context.Background(), newMemTable(testMemTableSize), hasManyHasAll, hasCache, &Stats{})
|
||||
require.NoError(t, err)
|
||||
@@ -59,7 +59,7 @@ func TestTableSetPrepend(t *testing.T) {
|
||||
assert.Empty(specs)
|
||||
mt := newMemTable(testMemTableSize)
|
||||
mt.addChunk(computeAddr(testChunks[0]), testChunks[0])
|
||||
hasCache, err := lru.New2Q[addr, struct{}](1024)
|
||||
hasCache, err := lru.New2Q[hash.Hash, struct{}](1024)
|
||||
require.NoError(t, err)
|
||||
ts, err = ts.append(context.Background(), mt, hasManyHasAll, hasCache, &Stats{})
|
||||
require.NoError(t, err)
|
||||
@@ -91,7 +91,7 @@ func TestTableSetToSpecsExcludesEmptyTable(t *testing.T) {
|
||||
assert.Empty(specs)
|
||||
mt := newMemTable(testMemTableSize)
|
||||
mt.addChunk(computeAddr(testChunks[0]), testChunks[0])
|
||||
hasCache, err := lru.New2Q[addr, struct{}](1024)
|
||||
hasCache, err := lru.New2Q[hash.Hash, struct{}](1024)
|
||||
require.NoError(t, err)
|
||||
ts, err = ts.append(context.Background(), mt, hasManyHasAll, hasCache, &Stats{})
|
||||
require.NoError(t, err)
|
||||
@@ -122,7 +122,7 @@ func TestTableSetFlattenExcludesEmptyTable(t *testing.T) {
|
||||
assert.Empty(specs)
|
||||
mt := newMemTable(testMemTableSize)
|
||||
mt.addChunk(computeAddr(testChunks[0]), testChunks[0])
|
||||
hasCache, err := lru.New2Q[addr, struct{}](1024)
|
||||
hasCache, err := lru.New2Q[hash.Hash, struct{}](1024)
|
||||
require.NoError(t, err)
|
||||
ts, err = ts.append(context.Background(), mt, hasManyHasAll, hasCache, &Stats{})
|
||||
require.NoError(t, err)
|
||||
@@ -156,7 +156,7 @@ func TestTableSetRebase(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
q := NewUnlimitedMemQuotaProvider()
|
||||
persister := newFakeTablePersister(q)
|
||||
hasCache, err := lru.New2Q[addr, struct{}](1024)
|
||||
hasCache, err := lru.New2Q[hash.Hash, struct{}](1024)
|
||||
require.NoError(t, err)
|
||||
|
||||
insert := func(ts tableSet, chunks ...[]byte) tableSet {
|
||||
@@ -211,7 +211,7 @@ func TestTableSetPhysicalLen(t *testing.T) {
|
||||
assert.Empty(specs)
|
||||
mt := newMemTable(testMemTableSize)
|
||||
mt.addChunk(computeAddr(testChunks[0]), testChunks[0])
|
||||
hasCache, err := lru.New2Q[addr, struct{}](1024)
|
||||
hasCache, err := lru.New2Q[hash.Hash, struct{}](1024)
|
||||
require.NoError(t, err)
|
||||
ts, err = ts.append(context.Background(), mt, hasManyHasAll, hasCache, &Stats{})
|
||||
require.NoError(t, err)
|
||||
|
||||
@@ -37,7 +37,7 @@ import (
|
||||
"github.com/dolthub/dolt/go/store/hash"
|
||||
)
|
||||
|
||||
func buildTable(chunks [][]byte) ([]byte, addr, error) {
|
||||
func buildTable(chunks [][]byte) ([]byte, hash.Hash, error) {
|
||||
totalData := uint64(0)
|
||||
for _, chunk := range chunks {
|
||||
totalData += uint64(len(chunk))
|
||||
@@ -55,7 +55,7 @@ func buildTable(chunks [][]byte) ([]byte, addr, error) {
|
||||
length, blockHash, err := tw.finish()
|
||||
|
||||
if err != nil {
|
||||
return nil, addr{}, err
|
||||
return nil, hash.Hash{}, err
|
||||
}
|
||||
|
||||
return buff[:length], blockHash, nil
|
||||
@@ -136,9 +136,9 @@ func TestHasMany(t *testing.T) {
|
||||
|
||||
addrs := addrSlice{computeAddr(chunks[0]), computeAddr(chunks[1]), computeAddr(chunks[2])}
|
||||
hasAddrs := []hasRecord{
|
||||
{&addrs[0], binary.BigEndian.Uint64(addrs[0][:addrPrefixSize]), 0, false},
|
||||
{&addrs[1], binary.BigEndian.Uint64(addrs[1][:addrPrefixSize]), 1, false},
|
||||
{&addrs[2], binary.BigEndian.Uint64(addrs[2][:addrPrefixSize]), 2, false},
|
||||
{&addrs[0], binary.BigEndian.Uint64(addrs[0][:hash.PrefixLen]), 0, false},
|
||||
{&addrs[1], binary.BigEndian.Uint64(addrs[1][:hash.PrefixLen]), 1, false},
|
||||
{&addrs[2], binary.BigEndian.Uint64(addrs[2][:hash.PrefixLen]), 2, false},
|
||||
}
|
||||
sort.Sort(hasRecordByPrefix(hasAddrs))
|
||||
|
||||
@@ -161,9 +161,9 @@ func TestHasManySequentialPrefix(t *testing.T) {
|
||||
"0rfgadopg6h3fk7d253ivbjsij4qo9nv",
|
||||
}
|
||||
|
||||
addrs := make([]addr, len(addrStrings))
|
||||
addrs := make([]hash.Hash, len(addrStrings))
|
||||
for i, s := range addrStrings {
|
||||
addrs[i] = addr(hash.Parse(s))
|
||||
addrs[i] = hash.Parse(s)
|
||||
}
|
||||
|
||||
bogusData := []byte("bogus") // doesn't matter what this is. hasMany() won't check chunkRecords
|
||||
@@ -218,7 +218,7 @@ func BenchmarkHasMany(b *testing.B) {
|
||||
for i := range hrecs {
|
||||
hrecs[i] = hasRecord{
|
||||
a: &addrs[i],
|
||||
prefix: prefixOf(addrs[i]),
|
||||
prefix: addrs[i].Prefix(),
|
||||
order: i,
|
||||
}
|
||||
}
|
||||
@@ -226,7 +226,7 @@ func BenchmarkHasMany(b *testing.B) {
|
||||
j := i * 64
|
||||
hrecs[i] = hasRecord{
|
||||
a: &addrs[j],
|
||||
prefix: prefixOf(addrs[j]),
|
||||
prefix: addrs[j].Prefix(),
|
||||
order: j,
|
||||
}
|
||||
}
|
||||
@@ -261,10 +261,6 @@ func BenchmarkHasMany(b *testing.B) {
|
||||
})
|
||||
}
|
||||
|
||||
func prefixOf(a addr) uint64 {
|
||||
return binary.BigEndian.Uint64(a[:addrPrefixSize])
|
||||
}
|
||||
|
||||
func TestGetMany(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
assert := assert.New(t)
|
||||
@@ -285,9 +281,9 @@ func TestGetMany(t *testing.T) {
|
||||
|
||||
addrs := addrSlice{computeAddr(data[0]), computeAddr(data[1]), computeAddr(data[2])}
|
||||
getBatch := []getRecord{
|
||||
{&addrs[0], binary.BigEndian.Uint64(addrs[0][:addrPrefixSize]), false},
|
||||
{&addrs[1], binary.BigEndian.Uint64(addrs[1][:addrPrefixSize]), false},
|
||||
{&addrs[2], binary.BigEndian.Uint64(addrs[2][:addrPrefixSize]), false},
|
||||
{&addrs[0], binary.BigEndian.Uint64(addrs[0][:hash.PrefixLen]), false},
|
||||
{&addrs[1], binary.BigEndian.Uint64(addrs[1][:hash.PrefixLen]), false},
|
||||
{&addrs[2], binary.BigEndian.Uint64(addrs[2][:hash.PrefixLen]), false},
|
||||
}
|
||||
sort.Sort(getRecordByPrefix(getBatch))
|
||||
|
||||
@@ -320,9 +316,9 @@ func TestCalcReads(t *testing.T) {
|
||||
defer tr.close()
|
||||
addrs := addrSlice{computeAddr(chunks[0]), computeAddr(chunks[1]), computeAddr(chunks[2])}
|
||||
getBatch := []getRecord{
|
||||
{&addrs[0], binary.BigEndian.Uint64(addrs[0][:addrPrefixSize]), false},
|
||||
{&addrs[1], binary.BigEndian.Uint64(addrs[1][:addrPrefixSize]), false},
|
||||
{&addrs[2], binary.BigEndian.Uint64(addrs[2][:addrPrefixSize]), false},
|
||||
{&addrs[0], binary.BigEndian.Uint64(addrs[0][:hash.PrefixLen]), false},
|
||||
{&addrs[1], binary.BigEndian.Uint64(addrs[1][:hash.PrefixLen]), false},
|
||||
{&addrs[2], binary.BigEndian.Uint64(addrs[2][:hash.PrefixLen]), false},
|
||||
}
|
||||
|
||||
gb2 := []getRecord{getBatch[0], getBatch[2]}
|
||||
@@ -420,7 +416,7 @@ func Test65k(t *testing.T) {
|
||||
|
||||
// Ensure all addresses share the first 7 bytes. Useful for easily generating tests which have
|
||||
// "prefix" collisions.
|
||||
func computeAddrCommonPrefix(data []byte) addr {
|
||||
func computeAddrCommonPrefix(data []byte) hash.Hash {
|
||||
a := computeAddrDefault(data)
|
||||
a[0] = 0x01
|
||||
a[1] = 0x23
|
||||
|
||||
Reference in New Issue
Block a user