mirror of
https://github.com/dolthub/dolt.git
synced 2026-01-15 03:10:54 -06:00
go/store/datas: Get TestCommitParentsClosure passing with DOLT_DEV.
This commit is contained in:
@@ -27,7 +27,6 @@ import (
|
||||
"errors"
|
||||
"io"
|
||||
"fmt"
|
||||
"encoding/binary"
|
||||
|
||||
flatbuffers "github.com/google/flatbuffers/go"
|
||||
|
||||
@@ -214,11 +213,12 @@ func writeCommitParentClosure(ctx context.Context, cs chunks.ChunkStore, vrw typ
|
||||
}
|
||||
}
|
||||
}
|
||||
// Add all the missing entries from [1, ...) maps to the 0th map.
|
||||
editor := maps[0].Mutate()
|
||||
for i := 1; i < len(maps); i++ {
|
||||
err = prolly.DiffMaps(ctx, maps[0], maps[i], func(ctx context.Context, diff tree.Diff) error {
|
||||
if diff.Type == tree.AddedDiff {
|
||||
return editor.Put(ctx, val.Tuple(diff.Key), val.Tuple{})
|
||||
return editor.Put(ctx, val.Tuple(diff.Key), val.EmptyTuple)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
@@ -226,14 +226,18 @@ func writeCommitParentClosure(ctx context.Context, cs chunks.ChunkStore, vrw typ
|
||||
return hash.Hash{}, fmt.Errorf("writeCommitParentClosure: DiffMaps: %w", err)
|
||||
}
|
||||
}
|
||||
var height [8]byte
|
||||
// Add the parents themselves to the new map.
|
||||
tb := val.NewTupleBuilder(keyDesc)
|
||||
for i := 0; i < len(parents); i++ {
|
||||
binary.LittleEndian.PutUint64(height[:], parents[i].Height())
|
||||
err = editor.Put(ctx, val.NewTuple(ns.Pool(), height[:], parentAddrs[i][:]), val.Tuple{})
|
||||
tb.PutUint64(0, parents[i].Height())
|
||||
tb.PutByteString(1, parentAddrs[i][:])
|
||||
err = editor.Put(ctx, tb.Build(ns.Pool()), val.EmptyTuple)
|
||||
if err != nil {
|
||||
return hash.Hash{}, fmt.Errorf("writeCommitParentClosure: MutableMap.Put: %w", err)
|
||||
}
|
||||
tb.Recycle()
|
||||
}
|
||||
// This puts the map in the NodeStore as well.
|
||||
res, err := editor.Map(ctx)
|
||||
if err != nil {
|
||||
return hash.Hash{}, fmt.Errorf("writeCommitParentClosure: MutableMap.Map: %w", err)
|
||||
@@ -713,6 +717,13 @@ func getRefElementType(t *types.Type) *types.Type {
|
||||
return t.Desc.(types.CompoundDesc).ElemTypes[0]
|
||||
}
|
||||
|
||||
type parentsClosureIter interface {
|
||||
Err() error
|
||||
Hash() hash.Hash
|
||||
Less(*types.NomsBinFormat, parentsClosureIter) bool
|
||||
Next(context.Context) bool
|
||||
}
|
||||
|
||||
type parentsClosureIterator struct {
|
||||
mi types.MapIterator
|
||||
err error
|
||||
@@ -742,7 +753,8 @@ func (i *parentsClosureIterator) Hash() hash.Hash {
|
||||
return h
|
||||
}
|
||||
|
||||
func (i *parentsClosureIterator) Less(f *types.NomsBinFormat, other *parentsClosureIterator) bool {
|
||||
func (i *parentsClosureIterator) Less(f *types.NomsBinFormat, otherI parentsClosureIter) bool {
|
||||
other := otherI.(*parentsClosureIterator)
|
||||
if i.err != nil || other.err != nil {
|
||||
return false
|
||||
}
|
||||
@@ -776,6 +788,41 @@ func (i *parentsClosureIterator) Next(ctx context.Context) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
type fbParentsClosureIterator struct {
|
||||
tuples []val.Tuple
|
||||
i int
|
||||
}
|
||||
|
||||
func (i *fbParentsClosureIterator) Err() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *fbParentsClosureIterator) Hash() hash.Hash {
|
||||
keyDesc := val.NewTupleDescriptor(
|
||||
val.Type{Enc: val.Uint64Enc, Nullable: false},
|
||||
val.Type{Enc: val.ByteStringEnc, Nullable: false},
|
||||
)
|
||||
bs, _ := keyDesc.GetBytes(1, i.tuples[i.i])
|
||||
return hash.New(bs)
|
||||
}
|
||||
|
||||
func (i *fbParentsClosureIterator) Next(ctx context.Context) bool {
|
||||
if i.i == 0 {
|
||||
return false
|
||||
}
|
||||
i.i = i.i-1
|
||||
return true
|
||||
}
|
||||
|
||||
func (i *fbParentsClosureIterator) Less(f *types.NomsBinFormat, otherI parentsClosureIter) bool {
|
||||
other := otherI.(*fbParentsClosureIterator)
|
||||
keyDesc := val.NewTupleDescriptor(
|
||||
val.Type{Enc: val.Uint64Enc, Nullable: false},
|
||||
val.Type{Enc: val.ByteStringEnc, Nullable: false},
|
||||
)
|
||||
return keyDesc.Comparator().Compare(i.tuples[i.i], other.tuples[other.i], keyDesc) == -1
|
||||
}
|
||||
|
||||
func commitToMapKeyTuple(f *types.NomsBinFormat, c *Commit) (types.Tuple, error) {
|
||||
h := c.Addr()
|
||||
ib := make([]byte, len(hash.Hash{}))
|
||||
@@ -790,12 +837,60 @@ func firstError(l, r error) error {
|
||||
return r
|
||||
}
|
||||
|
||||
func newParentsClosureIterator(ctx context.Context, c *Commit, vr types.ValueReader) (*parentsClosureIterator, error) {
|
||||
func hackVRToCS(vr types.ValueReader) chunks.ChunkStore {
|
||||
switch v := vr.(type) {
|
||||
case Database:
|
||||
return ChunkStoreFromDatabase(v)
|
||||
case *types.ValueStore:
|
||||
return v.ChunkStore()
|
||||
}
|
||||
panic("unknown ValueReader implementation...")
|
||||
}
|
||||
|
||||
func newParentsClosureIterator(ctx context.Context, c *Commit, vr types.ValueReader) (parentsClosureIter, error) {
|
||||
sv := c.NomsValue()
|
||||
|
||||
if _, ok := sv.(types.SerialMessage); ok {
|
||||
// TODO: __DOLT_DEV__ should support parent closures.
|
||||
return nil, nil
|
||||
msg := serial.GetRootAsCommit(sv.(types.SerialMessage), 0)
|
||||
addr := hash.New(msg.ParentClosureBytes())
|
||||
if addr.IsEmpty() {
|
||||
return nil, nil
|
||||
}
|
||||
v, err := vr.ReadValue(ctx, addr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if types.IsNull(v) {
|
||||
return nil, fmt.Errorf("internal error or data loss: dangling commit parent closure for addr %s or commit %s", addr.String(), c.Addr().String())
|
||||
}
|
||||
node := tree.NodeFromBytes(v.(types.TupleRowStorage))
|
||||
keyDesc := val.NewTupleDescriptor(
|
||||
val.Type{Enc: val.Uint64Enc, Nullable: false},
|
||||
val.Type{Enc: val.ByteStringEnc, Nullable: false},
|
||||
)
|
||||
valDesc := val.NewTupleDescriptor()
|
||||
ns := tree.NewNodeStore(hackVRToCS(vr))
|
||||
m := prolly.NewMap(node, ns, keyDesc, valDesc)
|
||||
tuples := make([]val.Tuple, m.Count())
|
||||
mi, err := m.IterAll(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
i := 0
|
||||
for {
|
||||
k, _, err := mi.Next(ctx)
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
tuples[i] = k
|
||||
i += 1
|
||||
}
|
||||
return &fbParentsClosureIterator{
|
||||
tuples, len(tuples)-1,
|
||||
}, nil
|
||||
}
|
||||
|
||||
s, ok := sv.(types.Struct)
|
||||
|
||||
@@ -25,15 +25,21 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
"io"
|
||||
"errors"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/dolthub/dolt/go/gen/fb/serial"
|
||||
"github.com/dolthub/dolt/go/store/chunks"
|
||||
"github.com/dolthub/dolt/go/store/d"
|
||||
"github.com/dolthub/dolt/go/store/hash"
|
||||
"github.com/dolthub/dolt/go/store/nomdl"
|
||||
"github.com/dolthub/dolt/go/store/types"
|
||||
"github.com/dolthub/dolt/go/store/prolly/tree"
|
||||
"github.com/dolthub/dolt/go/store/prolly"
|
||||
"github.com/dolthub/dolt/go/store/val"
|
||||
)
|
||||
|
||||
func mustHead(ds Dataset) types.Value {
|
||||
@@ -412,6 +418,7 @@ func TestCommitParentsClosure(t *testing.T) {
|
||||
|
||||
storage := &chunks.TestStorage{}
|
||||
db := NewDatabase(storage.NewViewWithDefaultFormat()).(*database)
|
||||
ctx := context.Background()
|
||||
|
||||
type expected struct {
|
||||
height int
|
||||
@@ -419,8 +426,58 @@ func TestCommitParentsClosure(t *testing.T) {
|
||||
}
|
||||
|
||||
assertCommitParentsClosure := func(v types.Value, es []expected) {
|
||||
if _, ok := v.(types.SerialMessage); ok {
|
||||
t.Skip("__DOLT_DEV__ does not implement ParentsClosure yet.")
|
||||
if v, ok := v.(types.SerialMessage); ok {
|
||||
msg := serial.GetRootAsCommit(v, 0)
|
||||
addr := hash.New(msg.ParentClosureBytes())
|
||||
if len(es) == 0 {
|
||||
assert.True(addr.IsEmpty(), "no parent closure when the closure is empty")
|
||||
return
|
||||
}
|
||||
v, err := db.ReadValue(ctx, addr)
|
||||
if !assert.NoError(err, "no error reading map for parent closure") {
|
||||
return
|
||||
}
|
||||
if !assert.False(types.IsNull(v), "reading parent closure map returns non-null") {
|
||||
return
|
||||
}
|
||||
node := tree.NodeFromBytes(v.(types.TupleRowStorage))
|
||||
keyDesc := val.NewTupleDescriptor(
|
||||
val.Type{Enc: val.Uint64Enc, Nullable: false},
|
||||
val.Type{Enc: val.ByteStringEnc, Nullable: false},
|
||||
)
|
||||
valDesc := val.NewTupleDescriptor()
|
||||
m := prolly.NewMap(node, tree.NewNodeStore(db.chunkStore()), keyDesc, valDesc)
|
||||
if !assert.Equal(len(es), m.Count(), "expected length of commit closure matches") {
|
||||
return
|
||||
}
|
||||
mi, err := m.IterAll(ctx)
|
||||
if !assert.NoError(err, "no error getting map iterator") {
|
||||
return
|
||||
}
|
||||
i := 0
|
||||
for {
|
||||
k, _, err := mi.Next(ctx)
|
||||
if errors.Is(err, io.EOF) {
|
||||
break
|
||||
}
|
||||
if !assert.NoError(err, "no error on MapIterator.Next") {
|
||||
return
|
||||
}
|
||||
height, ok := keyDesc.GetUint64(0, k)
|
||||
if !assert.True(ok, "able to get height from first field of closure key") {
|
||||
return
|
||||
}
|
||||
addrbs, ok := keyDesc.GetBytes(1, k)
|
||||
if !assert.True(ok, "able to get address bytes from second field of closure key") {
|
||||
return
|
||||
}
|
||||
addr := hash.New(addrbs)
|
||||
assert.Equal(es[i].height, int(height), "heights of expected entries match")
|
||||
assert.Equal(es[i].hash, addr, "addresses of expected entries match")
|
||||
i += 1
|
||||
}
|
||||
assert.Equal(len(es), i)
|
||||
return
|
||||
}
|
||||
s, ok := v.(types.Struct)
|
||||
if !assert.True(ok) {
|
||||
@@ -441,7 +498,7 @@ func TestCommitParentsClosure(t *testing.T) {
|
||||
if !assert.True(ok, "parents_closure field must contain a ref value.") {
|
||||
return
|
||||
}
|
||||
tv, err := r.TargetValue(context.Background(), db)
|
||||
tv, err := r.TargetValue(ctx, db)
|
||||
if !assert.NoError(err, "getting target value of parents_closure field must not error") {
|
||||
return
|
||||
}
|
||||
@@ -453,7 +510,7 @@ func TestCommitParentsClosure(t *testing.T) {
|
||||
return
|
||||
}
|
||||
i := 0
|
||||
err = m.IterAll(context.Background(), func(k, v types.Value) error {
|
||||
err = m.IterAll(ctx, func(k, v types.Value) error {
|
||||
j := i
|
||||
i++
|
||||
kt, ok := k.(types.Tuple)
|
||||
@@ -493,14 +550,19 @@ func TestCommitParentsClosure(t *testing.T) {
|
||||
assert.NoError(err)
|
||||
}
|
||||
|
||||
// TODO: These tests rely on the hash values of the commits
|
||||
// to assert the order of commits that are at the same height in the
|
||||
// parent closure map. The values have been tweaked to currently pass
|
||||
// with LD_1 and DOLT_DEV.
|
||||
|
||||
a, b, c, d := "ds-a", "ds-b", "ds-c", "ds-d"
|
||||
a1, a1a := addCommit(t, db, a, "a1")
|
||||
a2, a2a := addCommit(t, db, a, "a2", a1)
|
||||
a3, a3a := addCommit(t, db, a, "a3", a2)
|
||||
a3, a3a := addCommit(t, db, a, "a3 ", a2)
|
||||
|
||||
b1, b1a := addCommit(t, db, b, "b1", a1)
|
||||
b2, b2a := addCommit(t, db, b, "b2", b1)
|
||||
b3, b3a := addCommit(t, db, b, "b3", b2)
|
||||
b2, b2a := addCommit(t, db, b, "b2 ", b1)
|
||||
b3, b3a := addCommit(t, db, b, "b3 ", b2)
|
||||
|
||||
c1, c1a := addCommit(t, db, c, "c1", a3, b3)
|
||||
|
||||
|
||||
@@ -342,12 +342,14 @@ func (sm SerialMessage) walkRefs(nbf *NomsBinFormat, cb RefCallback) error {
|
||||
}
|
||||
|
||||
addr = hash.New(msg.ParentClosureBytes())
|
||||
r, err = constructRef(nbf, addr, PrimitiveTypeMap[ValueKind], SerialMessageRefHeight)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err = cb(r); err != nil {
|
||||
return err
|
||||
if !addr.IsEmpty() {
|
||||
r, err = constructRef(nbf, addr, PrimitiveTypeMap[ValueKind], SerialMessageRefHeight)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err = cb(r); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
default:
|
||||
return fmt.Errorf("unsupported SerialMessage message with FileID: %s", serial.GetFileID([]byte(sm)))
|
||||
|
||||
Reference in New Issue
Block a user