go/store/datas: Get TestCommitParentsClosure passing with DOLT_DEV.

This commit is contained in:
Aaron Son
2022-06-09 14:27:56 -07:00
parent 061ea0bd91
commit f3c19bce4e
3 changed files with 181 additions and 22 deletions

View File

@@ -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)

View File

@@ -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)

View File

@@ -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)))