From 0702ee25b829fe1bd8304ac319fcc35d0f476cd9 Mon Sep 17 00:00:00 2001 From: Wyatt Alt Date: Wed, 20 Mar 2024 19:54:55 -0700 Subject: [PATCH] Add write-ahead log This replaces the sqlite-based write-ahead log with a more featureful implementation. The new implementation uses append-only log files and a "wal manager" that is in charge of scheduling batched merges from WAL to storage and rotating logs. This is substantially faster to write to than sqlite, and also fixes a bug in the service around handling records that exceed the max blob size in sqlite. In the course of adding support for the new merging flow, I have made changes to the nodestore/treemgr/tree relationships. There is no more concept of "staging". Instead, the tree functions work on "tree writers" and "tree readers", and there is one implementation (memtree) that is used for staging writes to the WAL as well as writes to storage from WAL. Another implementation of the tree reader is backed by the nodestore, allowing data from the WAL to be merged with data from storage in one operation. With this change, the "tree-aware" machinery in the nodestore such as tree merging has been moved to either the tree or treemgr module. The tree module contains methods for traversing generic tree readers/writers while the treemgr module contains methods related to interfacing trees with the WAL. --- Makefile | 5 + README.md | 2 +- client/dp3/cmd/walinspect.go | 61 +++++ nodestore/memwal.go | 65 ----- nodestore/nodeid.go | 17 ++ nodestore/nodeid_test.go | 2 +- nodestore/nodestore.go | 470 ++------------------------------ nodestore/nodestore_test.go | 261 +++--------------- nodestore/sqlwal.go | 132 --------- nodestore/sqlwal_test.go | 46 ---- nodestore/testutils.go | 3 +- nodestore/wal.go | 46 ---- routes/import.go | 10 - service/dp3.go | 32 ++- tree/byo.go | 51 ++++ tree/bytetree.go | 81 ++++++ tree/errors.go | 22 ++ tree/iterator.go | 15 +- tree/iterator_test.go | 116 +++----- tree/memtree.go | 181 +++++++++++++ tree/memtree_test.go | 133 +++++++++ tree/testutils.go | 45 ++++ tree/tree.go | 361 ++++++++++++++++++------- tree/tree_test.go | 391 +++++++++++++-------------- tree/treereader.go | 78 ++++++ tree/treewriter.go | 18 ++ treemgr/options.go | 47 ++++ treemgr/treemgr.go | 474 +++++++++++++++++++------------- treemgr/treemgr_test.go | 119 ++++---- util/encoding.go | 69 +++++ util/encoding_test.go | 64 +++++ wal/errors.go | 53 ++++ wal/options.go | 53 ++++ wal/wal.go | 123 +++++++++ wal/wal_reader.go | 133 +++++++++ wal/wal_reader_test.go | 133 +++++++++ wal/wal_writer.go | 142 ++++++++++ wal/walmgr.go | 510 +++++++++++++++++++++++++++++++++++ wal/walmgr_test.go | 328 ++++++++++++++++++++++ 39 files changed, 3268 insertions(+), 1624 deletions(-) create mode 100644 client/dp3/cmd/walinspect.go delete mode 100644 nodestore/memwal.go delete mode 100644 nodestore/sqlwal.go delete mode 100644 nodestore/sqlwal_test.go delete mode 100644 nodestore/wal.go create mode 100644 tree/byo.go create mode 100644 tree/bytetree.go create mode 100644 tree/memtree.go create mode 100644 tree/memtree_test.go create mode 100644 tree/testutils.go create mode 100644 tree/treereader.go create mode 100644 tree/treewriter.go create mode 100644 treemgr/options.go create mode 100644 util/encoding.go create mode 100644 util/encoding_test.go create mode 100644 wal/errors.go create mode 100644 wal/options.go create mode 100644 wal/wal.go create mode 100644 wal/wal_reader.go create mode 100644 wal/wal_reader_test.go create mode 100644 wal/wal_writer.go create mode 100644 wal/walmgr.go create mode 100644 wal/walmgr_test.go diff --git a/Makefile b/Makefile index fdc82db..734bbe9 100644 --- a/Makefile +++ b/Makefile @@ -6,3 +6,8 @@ lint: build: go build -o dp3 ./client/dp3 + +clean: + rm -rf data/* + rm -rf waldir/* + rm dp3.db* diff --git a/README.md b/README.md index 1712332..1d505f7 100644 --- a/README.md +++ b/README.md @@ -150,7 +150,7 @@ In the service, nodes are cached on read in an LRU cache of configurable byte capacity. In production deployments, this cache will be sized such that most important inner nodes will fit within it at all times. Multigranular summarization requires traversing the tree down to a sufficiently granular -depth, and then scanning the statistical summaries at that depth for the +height, and then scanning the statistical summaries at that height for the requested range of time. If the cache is performing well this operation can be done in RAM. diff --git a/client/dp3/cmd/walinspect.go b/client/dp3/cmd/walinspect.go new file mode 100644 index 0000000..8974e03 --- /dev/null +++ b/client/dp3/cmd/walinspect.go @@ -0,0 +1,61 @@ +package cmd + +import ( + "errors" + "fmt" + "io" + "os" + + "github.com/spf13/cobra" + "github.com/wkalt/dp3/wal" +) + +// walinspectCmd represents the walinspect command +var walinspectCmd = &cobra.Command{ + Use: "walinspect [file] | less", + Run: func(cmd *cobra.Command, args []string) { + if len(args) != 1 { + bailf("Usage: dp3 walinspect [file] | less") + } + f, err := os.Open(args[0]) + if err != nil { + bailf("failed to open WAL file: %v", err) + } + defer f.Close() + reader, err := wal.NewReader(f) + if err != nil { + bailf("failed to create WAL reader: %v", err) + } + for { + offset := reader.Offset() + rectype, record, err := reader.Next() + if err != nil { + if errors.Is(err, io.EOF) { + return + } + bailf("failed to read record: %v", err) + } + switch rectype { + case wal.WALInsert: + req := wal.ParseInsertRecord(record) + fmt.Printf("%d: insert (%s) %d bytes\n", offset, req.Topic, len(record)) + case wal.WALMergeRequest: + req := wal.ParseMergeRequestRecord(record) + addrs := "" + for _, addr := range req.Addrs { + addrs += fmt.Sprintf("%s ", addr) + } + fmt.Printf("%d: merge request %s (%s) addrs: %s\n", offset, req.BatchID, req.Topic, addrs) + case wal.WALMergeComplete: + req := wal.ParseMergeCompleteRecord(record) + fmt.Printf("%d: merge complete %s\n", offset, req.BatchID) + default: + bailf("unknown record type: %v", rectype) + } + } + }, +} + +func init() { + rootCmd.AddCommand(walinspectCmd) +} diff --git a/nodestore/memwal.go b/nodestore/memwal.go deleted file mode 100644 index fd14dc4..0000000 --- a/nodestore/memwal.go +++ /dev/null @@ -1,65 +0,0 @@ -package nodestore - -import ( - "context" - - "golang.org/x/exp/maps" -) - -/* -memwal is an in-memory implementation of the WAL interface. It is only suitable -for usage in testing. -*/ - -//////////////////////////////////////////////////////////////////////////////// - -type memwal struct { - entries []*WALEntry -} - -func NewMemWAL() WAL { - return &memwal{} -} - -func (m *memwal) Put(ctx context.Context, entry WALEntry) error { - m.entries = append(m.entries, &entry) - return nil -} - -func (m *memwal) Get(ctx context.Context, nodeID NodeID) ([]byte, error) { - for _, entry := range m.entries { - if entry.NodeID == nodeID && !entry.Deleted { - return entry.Data, nil - } - } - return nil, NodeNotFoundError{nodeID} -} - -func (m *memwal) Delete(ctx context.Context, nodeID NodeID) error { - for _, entry := range m.entries { - if entry.NodeID == nodeID && !entry.Deleted { - entry.Deleted = true - return nil - } - } - return NodeNotFoundError{nodeID} -} - -func (m *memwal) List(ctx context.Context) ([]WALListing, error) { - streams := make(map[string]WALListing) - for _, entry := range m.entries { - key := entry.ProducerID + entry.Topic - listing, ok := streams[key] - if !ok { - listing = WALListing{ - ProducerID: entry.ProducerID, - Topic: entry.Topic, - Versions: make(map[uint64][]NodeID), - } - streams[key] = listing - } - listing.Versions[entry.Version] = append(listing.Versions[entry.Version], entry.NodeID) - streams[key] = listing - } - return maps.Values(streams), nil -} diff --git a/nodestore/nodeid.go b/nodestore/nodeid.go index 97fc9df..1018a5c 100644 --- a/nodestore/nodeid.go +++ b/nodestore/nodeid.go @@ -37,6 +37,15 @@ func (n NodeID) Length() int { return int(binary.LittleEndian.Uint32(n[12:])) } +// NewNodeID creates a new node ID from an object ID, offset, and length. +func NewNodeID(oid uint64, offset, length int) NodeID { + var id NodeID + binary.LittleEndian.PutUint64(id[:8], oid) + binary.LittleEndian.PutUint32(id[8:12], uint32(offset)) + binary.LittleEndian.PutUint32(id[12:], uint32(length)) + return id +} + // String returns a string representation of the node ID. func (n NodeID) String() string { return fmt.Sprintf("%s:%d:%d", n.OID(), n.Offset(), n.Length()) @@ -78,3 +87,11 @@ func (n *NodeID) Scan(value interface{}) error { func (n NodeID) Value() (driver.Value, error) { return driver.Value(n.String()), nil } + +func generateNodeID(oid objectID, offset int, length int) NodeID { + var id NodeID + binary.LittleEndian.PutUint64(id[:], uint64(oid)) + binary.LittleEndian.PutUint32(id[8:], uint32(offset)) + binary.LittleEndian.PutUint32(id[12:], uint32(length)) + return id +} diff --git a/nodestore/nodeid_test.go b/nodestore/nodeid_test.go index f6feda0..9257983 100644 --- a/nodestore/nodeid_test.go +++ b/nodestore/nodeid_test.go @@ -10,7 +10,7 @@ import ( func TestNodeIDScanner(t *testing.T) { for i := 0; i < 1e3; i++ { - id := genNodeID(t) + id := nodestore.RandomNodeID() value, err := id.Value() require.NoError(t, err) var id2 nodestore.NodeID diff --git a/nodestore/nodestore.go b/nodestore/nodestore.go index c57caab..2ddb154 100644 --- a/nodestore/nodestore.go +++ b/nodestore/nodestore.go @@ -1,19 +1,13 @@ package nodestore import ( - "bytes" "context" - "crypto/rand" - "encoding/binary" "errors" "fmt" "io" - "slices" - "strings" + "strconv" "sync" - fmcap "github.com/foxglove/mcap/go/mcap" - "github.com/wkalt/dp3/mcap" "github.com/wkalt/dp3/storage" "github.com/wkalt/dp3/util" ) @@ -55,38 +49,32 @@ type Nodestore struct { store storage.Provider cache *util.LRU[NodeID, Node] - staging map[NodeID]Node - mtx *sync.RWMutex - - wal WAL + mtx *sync.RWMutex } // NewNodestore creates a new nodestore. func NewNodestore( store storage.Provider, cache *util.LRU[NodeID, Node], - wal WAL, ) *Nodestore { return &Nodestore{ - store: store, - cache: cache, - mtx: &sync.RWMutex{}, - staging: make(map[NodeID]Node), - wal: wal, + store: store, + cache: cache, + mtx: &sync.RWMutex{}, } } -func (n *Nodestore) GetStaged(ctx context.Context, id NodeID) (Node, error) { - node, ok := n.getStagedNode(id) - if !ok { - return nil, NodeNotFoundError{id} +// Put an object to storage. +func (n *Nodestore) Put(ctx context.Context, oid uint64, data []byte) error { + objectname := strconv.FormatUint(oid, 10) + if err := n.store.Put(ctx, objectname, data); err != nil { + return fmt.Errorf("failed to put object %d: %w", oid, err) } - return node, nil + return nil } -// Get retrieves a node from the nodestore. It will check the cache and staging -// areas prior to storage. Staging node IDs and "real" node IDs will never -// conflict. +// Get retrieves a node from the nodestore. It will check the cache prior to +// storage. func (n *Nodestore) Get(ctx context.Context, id NodeID) (Node, error) { if value, ok := n.cache.Get(id); ok { return value, nil @@ -103,7 +91,7 @@ func (n *Nodestore) Get(ctx context.Context, id NodeID) (Node, error) { if err != nil { return nil, fmt.Errorf("failed to read node: %w", err) } - node, err := n.bytesToNode(data) + node, err := n.BytesToNode(data) if err != nil { return nil, fmt.Errorf("failed to parse node: %w", err) } @@ -114,7 +102,7 @@ func (n *Nodestore) Get(ctx context.Context, id NodeID) (Node, error) { // GetLeaf retrieves a leaf node from the nodestore. It returns a ReadSeekCloser // over the leaf data, the closing of which is the caller's responsibility. It // does not cache data. -func (n *Nodestore) GetLeaf(ctx context.Context, id NodeID) (io.ReadSeekCloser, error) { +func (n *Nodestore) GetLeafData(ctx context.Context, id NodeID) (io.ReadSeekCloser, error) { // NB: Add one to the offset to skip the type byte. reader, err := n.store.GetRange(ctx, id.OID(), id.Offset()+1, id.Length()-1) if err != nil { @@ -126,225 +114,11 @@ func (n *Nodestore) GetLeaf(ctx context.Context, id NodeID) (io.ReadSeekCloser, return reader, nil } -// WALDelete deletes a node from the WAL. -func (n *Nodestore) WALDelete(ctx context.Context, id NodeID) error { - if err := n.wal.Delete(ctx, id); err != nil { - return fmt.Errorf("failed to delete node: %w", err) - } - return nil -} - -// ListWAL returns a list of WAL entries. -func (n *Nodestore) ListWAL(ctx context.Context) ([]WALListing, error) { - wal, err := n.wal.List(ctx) - if err != nil { - return nil, fmt.Errorf("failed to list wal: %w", err) - } - return wal, nil -} - -// FlushStagingToWAL flushes a list of nodes from staging to the WAL. -func (n *Nodestore) FlushStagingToWAL( - ctx context.Context, - producerID string, - topic string, - version uint64, - ids []NodeID, -) error { - for _, id := range ids { - node, ok := n.getStagedNode(id) - if !ok { - return fmt.Errorf("node %s not found in staging", id) - } - bytes := node.ToBytes() - entry := WALEntry{ - ProducerID: producerID, - Topic: topic, - NodeID: id, - Version: version, - Data: bytes, - } - if err := n.wal.Put(ctx, entry); err != nil { - return fmt.Errorf("failed to write WAL: %w", err) - } // todo: make transactional - n.deleteStagedNode(id) - } - return nil -} - -// StageWithID stages a node in the staging map with a specific ID. -func (n *Nodestore) StageWithID(id NodeID, node Node) error { - n.mtx.Lock() - defer n.mtx.Unlock() - n.staging[id] = node - return nil -} - -// generateStagingID generates a temporary ID that will not collide with "real" -// node IDs. -func (n *Nodestore) generateStagingID() NodeID { - var id NodeID - _, _ = rand.Read(id[:]) - return id -} - -// Stage a node in the staging map, returning an ID that can be used later. -func (n *Nodestore) Stage(node Node) NodeID { - id := n.generateStagingID() - n.mtx.Lock() - defer n.mtx.Unlock() - n.staging[id] = node - return id -} - -// Flush flushes a list of node IDs to the store in a single object. The IDs are -// assumed to be in root -> leaf order, such that the reversed list will capture -// dependency ordering. All nodes will be removed from staging under any exit -// condition. Existing content in the same logical location is cloned and copied -// into the final output tree. -func (n *Nodestore) Flush(ctx context.Context, version uint64, ids ...NodeID) (newRootID NodeID, err error) { - defer func() { - for _, id := range ids { - n.deleteStagedNode(id) - } - }() - newIDs := make([]NodeID, len(ids)) - buf := &bytes.Buffer{} - offset := 0 - slices.Reverse(ids) - processed := make(map[NodeID]NodeID) - oid := objectID(version) - for i, id := range ids { - node, ok := n.getStagedNode(id) - if !ok { - return newRootID, ErrNodeNotStaged - } - - if inner, ok := node.(*InnerNode); ok { - for _, child := range inner.Children { - if child != nil { - if remapped, ok := processed[child.ID]; ok { - child.ID = remapped - child.Version = version - } - } - } - } - n, err := buf.Write(node.ToBytes()) - if err != nil { - return newRootID, fmt.Errorf("failed to write node to buffer: %w", err) - } - nodeID := generateNodeID(oid, offset, n) - offset += n - processed[id] = nodeID - newIDs[i] = nodeID - } - if err := n.store.Put(ctx, oid.String(), buf.Bytes()); err != nil { - return newRootID, fmt.Errorf("failed to put object: %w", err) - } - return newIDs[len(newIDs)-1], nil -} - -// NewRoot creates a new root node with the given height and range, and persists -// it to storage, returning the ID. -func (n *Nodestore) NewRoot( - ctx context.Context, - start uint64, - end uint64, - leafWidthSecs int, - bfactor int, -) (nodeID NodeID, err error) { - var height int - span := end - start - coverage := leafWidthSecs - for uint64(coverage) < span { - coverage *= bfactor - height++ - } - root := NewInnerNode(uint8(height), start, start+uint64(coverage), bfactor) - tmpid := n.Stage(root) - id, err := n.Flush(ctx, 0, tmpid) - if err != nil { - return nodeID, fmt.Errorf("failed to flush root %s: %w", tmpid, err) - } - return id, nil -} - -// FlushWALPath flushes a list of node IDs from the WAL to permanent storage. -// The path of node IDs is expected to be in root to leaf order. -func (n *Nodestore) FlushWALPath(ctx context.Context, version uint64, path []NodeID) (nodeID NodeID, err error) { - result := []NodeID{} - for _, nodeID := range path { - data, err := n.wal.Get(ctx, nodeID) - if err != nil { - return nodeID, fmt.Errorf("failed to get node %s from wal: %w", nodeID, err) - } - node, err := n.bytesToNode(data) - if err != nil { - return nodeID, fmt.Errorf("failed to parse node: %w", err) - } - if err := n.StageWithID(nodeID, node); err != nil { - return nodeID, fmt.Errorf("failed to stage node: %w", err) - } - result = append(result, nodeID) - } - flushedRoot, err := n.Flush(ctx, version, result...) - if err != nil { - return nodeID, fmt.Errorf("failed to flush root: %w", err) - } - for _, node := range path { // todo transaction - if err := n.wal.Delete(ctx, node); err != nil { - return nodeID, fmt.Errorf("failed to delete node: %w", err) - } - } - return flushedRoot, nil -} - -// MergeWALToStorage merges a list of root node IDs that exist in WAL into a single -// partial tree in the staging area, and then merges that tree into persistent -// storage. -func (n *Nodestore) MergeWALToStorage( - ctx context.Context, - rootID NodeID, - version uint64, - nodeIDs []NodeID, -) (nodeID NodeID, err error) { - ids := make([]NodeID, len(nodeIDs)+1) - ids[0] = rootID - copy(ids[1:], nodeIDs) - mergedPath, err := n.nodeMerge(ctx, version, ids) - if err != nil { - return nodeID, fmt.Errorf("failed to merge nodes: %w", err) - } - flushedRoot, err := n.Flush(ctx, version, mergedPath...) - if err != nil { - return nodeID, fmt.Errorf("failed to flush root: %w", err) - } - return flushedRoot, nil -} - -// Print returns a string representation of the tree rooted at the provided node -// ID. Top-level calls should pass a nil statistics parameter. -func (n *Nodestore) Print(ctx context.Context, nodeID NodeID, version uint64, statistics *Statistics) (string, error) { - node, err := n.Get(ctx, nodeID) - if err != nil { - return "", fmt.Errorf("failed to get node %d: %w", nodeID, err) - } - switch node := node.(type) { - case *InnerNode: - return n.printInnerNode(ctx, node, version, statistics) - case *LeafNode: - return node.String(), nil - default: - return "", fmt.Errorf("unexpected node type: %+v", node) - } -} - func isLeaf(data []byte) bool { return data[0] > 128 } -func (n *Nodestore) bytesToNode(value []byte) (Node, error) { +func (n *Nodestore) BytesToNode(value []byte) (Node, error) { if isLeaf(value) { node := NewLeafNode(nil) if err := node.FromBytes(value); err != nil { @@ -358,215 +132,3 @@ func (n *Nodestore) bytesToNode(value []byte) (Node, error) { } return node, nil } - -func (n *Nodestore) deleteStagedNode(id NodeID) { - n.mtx.Lock() - defer n.mtx.Unlock() - delete(n.staging, id) -} - -func (n *Nodestore) getStagedNode(id NodeID) (Node, bool) { - n.mtx.RLock() - defer n.mtx.RUnlock() - node, ok := n.staging[id] - return node, ok -} - -func generateNodeID(oid objectID, offset int, length int) NodeID { - var id NodeID - binary.LittleEndian.PutUint64(id[:], uint64(oid)) - binary.LittleEndian.PutUint32(id[8:], uint32(offset)) - binary.LittleEndian.PutUint32(id[12:], uint32(length)) - return id -} - -// nodeMerge does an N-way tree merge, returning a "path" from the root to leaf -// of the new tree. All nodes are from the same level of the tree, and can thus -// be assumed to have the same type and same number of children. -func (n *Nodestore) nodeMerge( - ctx context.Context, - version uint64, - nodeIDs []NodeID, -) (path []NodeID, err error) { - if len(nodeIDs) == 0 { - return nil, errors.New("no nodes to merge") - } - nodes := make([]Node, len(nodeIDs)) - for i, id := range nodeIDs { - node, err := n.getWALOrStorage(ctx, id) - if err != nil { - return nil, fmt.Errorf("failed to parse node: %w", err) - } - nodes[i] = node - } - switch node := (nodes[0]).(type) { - case *InnerNode: - inner := make([]*InnerNode, 0, len(nodes)+1) - - for _, node := range nodes { - inner = append(inner, node.(*InnerNode)) - } - - if len(nodes) == 1 { - // if there is just one node, add a fake, empty node to the list to - // represent the logical leaves in the root that don't exist yet. - // Seems to make the downstream code simpler but might be confusing - // in the future. - inner = append(inner, NewInnerNode(node.Height, node.Start, node.End, len(node.Children))) - } - - nodeIDs, err := n.mergeInnerNodes(ctx, version, inner) - if err != nil { - return nil, err - } - return nodeIDs, nil - case *LeafNode: - merged, err := n.mergeLeaves(ctx, nodeIDs) - if err != nil { - return nil, err - } - return []NodeID{merged}, nil - default: - return nil, fmt.Errorf("unrecognized node type: %T", node) - } -} - -func (n *Nodestore) mergeInnerNodes( - ctx context.Context, version uint64, nodes []*InnerNode, -) ([]NodeID, error) { - conflicts := []int{} - node := nodes[0] - for i, child := range node.Children { - var conflicted bool - for _, sibling := range nodes { - cousin := sibling.Children[i] - if child == nil && cousin != nil || - child != nil && cousin == nil || - child != nil && cousin != nil && child.ID != cousin.ID { - conflicted = true - break - } - } - if conflicted { - conflicts = append(conflicts, i) - } - } - newInner := NewInnerNode(node.Height, node.Start, node.End, len(node.Children)) - newID := n.Stage(newInner) - result := []NodeID{newID} - for _, conflict := range conflicts { - children := []NodeID{} // set of not-null children mapping to conflicts - stats := &Statistics{} - for _, node := range nodes { - if inner := node.Children[conflict]; inner != nil && !slices.Contains(children, inner.ID) { - children = append(children, inner.ID) - if err := stats.Add(inner.Statistics); err != nil { - return nil, fmt.Errorf("failed to add statistics: %w", err) - } - } - } - merged, err := n.nodeMerge(ctx, version, children) // merged child for this conflict - if err != nil { - return nil, err - } - newInner.Children[conflict] = &Child{ID: merged[0], Version: version, Statistics: stats} - result = append(result, merged...) - } - for i := range node.Children { - if !slices.Contains(conflicts, i) { - newInner.Children[i] = node.Children[i] - } - } - return result, nil -} - -func (n *Nodestore) getWALOrStorage(ctx context.Context, nodeID NodeID) (Node, error) { - data, err := n.wal.Get(ctx, nodeID) - if err != nil { - if errors.Is(err, NodeNotFoundError{}) { - node, err := n.Get(ctx, nodeID) - if err != nil { - return nil, fmt.Errorf("failed to get node %s: %w", nodeID, err) - } - return node, nil - } - return nil, fmt.Errorf("failed to get node %d from wal: %w", nodeID, err) - } - node, err := n.bytesToNode(data) - if err != nil { - return nil, fmt.Errorf("failed to parse node: %w", err) - } - return node, nil -} - -func (n *Nodestore) mergeLeaves(ctx context.Context, nodeIDs []NodeID) (NodeID, error) { - if len(nodeIDs) == 1 { - id := nodeIDs[0] - var node Node - node, err := n.getWALOrStorage(ctx, id) - if err != nil { - return NodeID{}, fmt.Errorf("failed to get node: %w", err) - } - nodeID := n.Stage(node) - return nodeID, nil - } - iterators := make([]fmcap.MessageIterator, len(nodeIDs)) - for i, id := range nodeIDs { - leaf, err := n.getWALOrStorage(ctx, id) - if err != nil { - return NodeID{}, fmt.Errorf("failed to get leaf %d from wal: %w", id, err) - } - reader, err := fmcap.NewReader(leaf.(*LeafNode).Data()) - if err != nil { - return NodeID{}, fmt.Errorf("failed to create reader: %w", err) - } - defer reader.Close() - iterators[i], err = reader.Messages() - if err != nil { - return NodeID{}, fmt.Errorf("failed to create iterator: %w", err) - } - } - buf := &bytes.Buffer{} - if err := mcap.Nmerge(buf, iterators...); err != nil { - return NodeID{}, fmt.Errorf("failed to merge leaves: %w", err) - } - newLeaf := NewLeafNode(buf.Bytes()) - return n.Stage(newLeaf), nil -} - -func (n *Nodestore) printInnerNode( - ctx context.Context, - node *InnerNode, - version uint64, - statistics *Statistics, -) (string, error) { - sb := &strings.Builder{} - if statistics != nil { - sb.WriteString(fmt.Sprintf("[%d-%d:%d %s", node.Start, node.End, version, statistics)) - } else { - sb.WriteString(fmt.Sprintf("[%d-%d:%d", node.Start, node.End, version)) - } - span := node.End - node.Start - for i, child := range node.Children { - if child == nil { - continue - } - childNode, err := n.Get(ctx, child.ID) - if err != nil { - return "", fmt.Errorf("failed to get node %d: %w", child.ID, err) - } - if cnode, ok := childNode.(*LeafNode); ok { - start := node.Start + uint64(i)*span/uint64(len(node.Children)) - end := node.Start + uint64(i+1)*span/uint64(len(node.Children)) - sb.WriteString(fmt.Sprintf(" [%d-%d:%d %s %s]", start, end, child.Version, child.Statistics, cnode)) - } else { - childStr, err := n.Print(ctx, child.ID, child.Version, child.Statistics) - if err != nil { - return "", err - } - sb.WriteString(" " + childStr) - } - } - sb.WriteString("]") - return sb.String(), nil -} diff --git a/nodestore/nodestore_test.go b/nodestore/nodestore_test.go index f1ec992..da05432 100644 --- a/nodestore/nodestore_test.go +++ b/nodestore/nodestore_test.go @@ -3,252 +3,57 @@ package nodestore_test import ( "bytes" "context" - "database/sql" - "strings" + "io" "testing" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/wkalt/dp3/mcap" "github.com/wkalt/dp3/nodestore" "github.com/wkalt/dp3/storage" - "github.com/wkalt/dp3/tree" "github.com/wkalt/dp3/util" ) -func TestNodestoreErrors(t *testing.T) { - ctx := context.Background() - store := storage.NewMemStore() - cache := util.NewLRU[nodestore.NodeID, nodestore.Node](1e6) - db, err := sql.Open("sqlite3", ":memory:") - require.NoError(t, err) - wal, err := nodestore.NewSQLWAL(ctx, db) - require.NoError(t, err) - ns := nodestore.NewNodestore(store, cache, wal) - t.Run("get nonexistent node", func(t *testing.T) { - id := nodestore.RandomNodeID() - _, err := ns.Get(ctx, id) - assert.ErrorIs(t, err, nodestore.NodeNotFoundError{id}) +func TestNodestore(t *testing.T) { + t.Run("Put", func(t *testing.T) { + ctx := context.Background() + store := storage.NewMemStore() + cache := util.NewLRU[nodestore.NodeID, nodestore.Node](1024) + ns := nodestore.NewNodestore(store, cache) + node := nodestore.NewLeafNode([]byte("test")) + bytes := node.ToBytes() + require.NoError(t, ns.Put(ctx, 1, bytes)) }) - t.Run("flush a nonexistent node", func(t *testing.T) { - _, err := ns.Flush(ctx, 1, nodestore.NodeID{}) - assert.ErrorIs(t, err, nodestore.ErrNodeNotStaged) - }) -} - -func removeSpace(s string) string { - s = strings.ReplaceAll(s, "\n", " ") - s = strings.ReplaceAll(s, " ", "") - s = strings.ReplaceAll(s, "\t", "") - return s -} - -func assertEqualTrees(t *testing.T, a, b string) { - t.Helper() - require.Equal(t, removeSpace(a), removeSpace(b), "%s != %s", a, b) -} - -func makeTestNodestore(t *testing.T) *nodestore.Nodestore { - t.Helper() - store := storage.NewMemStore() - cache := util.NewLRU[nodestore.NodeID, nodestore.Node](1e6) - wal := nodestore.NewMemWAL() - return nodestore.NewNodestore(store, cache, wal) -} - -func TestWALMerge(t *testing.T) { - ctx := context.Background() - cases := []struct { - assertion string - root []string - nodes [][]string - expected string - }{ - { - "single node into empty root", - []string{}, - [][]string{{"1970-01-03"}}, - "[0-707788800:2 [0-11059200:2 (count=1) [172800-345600:2 (count=1) [leaf 1 msg]]]]", - }, - { - "single node into populated, nonoverlapping root", - []string{"1970-01-01"}, - [][]string{{"1970-01-03"}}, - `[0-707788800:3 [0-11059200:3 (count=2) [0-172800:1 (count=1) [leaf 1 msg]] - [172800-345600:3 (count=1) [leaf 1 msg]]]]`, - }, - { - "two nonoverlapping nodes into empty root", - []string{}, - [][]string{{"1970-01-03"}, {"1970-01-05"}}, - `[0-707788800:3 [0-11059200:3 (count=2) [172800-345600:3 (count=1) [leaf 1 msg]] - [345600-518400:3 (count=1) [leaf 1 msg]]]]`, - }, - { - "two nonoverlapping nodes into nonempty empty root", - []string{"1970-01-01"}, - [][]string{{"1970-01-03"}, {"1970-01-05"}}, - `[0-707788800:4 [0-11059200:4 (count=3) [0-172800:1 (count=1) [leaf 1 msg]] - [172800-345600:4 (count=1) [leaf 1 msg]] [345600-518400:4 (count=1) [leaf 1 msg]]]]`, - }, - { - "overlapping nodes into empty root", - []string{}, - [][]string{{"1970-01-01"}, {"1970-01-02"}}, - "[0-707788800:3 [0-11059200:3 (count=2) [0-172800:3 (count=2) [leaf 2 msgs]]]]", - }, - { - "overlapping nodes into nonempty root", - []string{"1970-01-01"}, - [][]string{{"1970-01-01"}, {"1970-01-02"}}, - "[0-707788800:4 [0-11059200:4 (count=3) [0-172800:4 (count=3) [leaf 3 msgs]]]]", - }, - } - for _, c := range cases { - t.Run(c.assertion, func(t *testing.T) { - ns := makeTestNodestore(t) - rootID, err := ns.NewRoot( - ctx, - util.DateSeconds("1970-01-01"), - util.DateSeconds("1975-01-01"), - 60*60*24*2, // five years, two day buckets - 64, - ) - require.NoError(t, err) - version := uint64(1) - if len(c.root) > 0 { - var nodeIDs []nodestore.NodeID - buf := &bytes.Buffer{} - secs := util.DateSeconds(c.root[0]) - nsecs := secs * 1e9 - mcap.WriteFile(t, buf, []uint64{nsecs}) - stats := &nodestore.Statistics{ - MessageCount: uint64(len(c.root)), - } - _, nodeIDs, err = tree.Insert(ctx, ns, rootID, 1, nsecs, buf.Bytes(), stats) - require.NoError(t, err) - rootID, err = ns.Flush(ctx, version, nodeIDs...) - require.NoError(t, err) - version++ - } - - producer := "producer" - topic := "topic" - roots := make([]nodestore.NodeID, len(c.nodes)) - for i, node := range c.nodes { - buf := &bytes.Buffer{} + t.Run("Get", func(t *testing.T) { + ctx := context.Background() + buf := &bytes.Buffer{} + n, err := buf.Write(make([]byte, 1000)) + require.NoError(t, err) - times := []uint64{} - for _, time := range node { - times = append(times, 1e9*util.DateSeconds(time)) - } - mcap.WriteFile(t, buf, times) - stats := &nodestore.Statistics{ - MessageCount: uint64(len(node)), - } - newRootID, nodeIDs, err := tree.Insert( - ctx, ns, rootID, version, times[0], buf.Bytes(), stats) // into staging - require.NoError(t, err) - require.NoError(t, ns.FlushStagingToWAL(ctx, producer, topic, version, nodeIDs)) // staging -> wal - version++ - roots[i] = newRootID - } + store := storage.NewMemStore() + cache := util.NewLRU[nodestore.NodeID, nodestore.Node](1024) + ns := nodestore.NewNodestore(store, cache) - rootID, err = ns.MergeWALToStorage(ctx, rootID, version, roots) - require.NoError(t, err) + data := []byte("hello") + node := nodestore.NewLeafNode(data) + bytes := node.ToBytes() + _, err = buf.Write(bytes) + require.NoError(t, err) + addr := nodestore.NewNodeID(1, n, len(bytes)) - str, err := ns.Print(ctx, rootID, version, nil) - require.NoError(t, err) - assertEqualTrees(t, c.expected, str) - }) - } -} + require.NoError(t, ns.Put(ctx, 1, buf.Bytes())) -func TestNewRoot(t *testing.T) { - ctx := context.Background() - store := storage.NewMemStore() - cache := util.NewLRU[nodestore.NodeID, nodestore.Node](1e6) - wal := nodestore.NewMemWAL() - ns := nodestore.NewNodestore(store, cache, wal) - root, err := ns.NewRoot( - ctx, - util.DateSeconds("1970-01-01"), - util.DateSeconds("2030-01-01"), - 60, - 64, - ) - require.NoError(t, err) - _, err = ns.Get(ctx, root) - require.NoError(t, err) -} - -func TestNodeStore(t *testing.T) { - ctx := context.Background() - store := storage.NewMemStore() - cache := util.NewLRU[nodestore.NodeID, nodestore.Node](1e6) - db, err := sql.Open("sqlite3", ":memory:") - require.NoError(t, err) - wal, err := nodestore.NewSQLWAL(ctx, db) - require.NoError(t, err) - ns := nodestore.NewNodestore(store, cache, wal) - t.Run("store and retrieve an inner node", func(t *testing.T) { - node := nodestore.NewInnerNode(0, 10, 20, 64) - nodeID := ns.Stage(node) - rootID, err := ns.Flush(ctx, 1, nodeID) - require.NoError(t, err) - retrieved, err := ns.Get(ctx, rootID) - require.NoError(t, err) - assert.Equal(t, node, retrieved) - }) - t.Run("store and retrieve a leaf node", func(t *testing.T) { - node := nodestore.NewLeafNode(nil) - nodeID := ns.Stage(node) - rootID, err := ns.Flush(ctx, 1, nodeID) - require.NoError(t, err) - retrieved, err := ns.Get(ctx, rootID) - require.NoError(t, err) - assert.Equal(t, node, retrieved) - }) - t.Run("store and retrieve inner node that has been evicted from cache", func(t *testing.T) { - node := nodestore.NewInnerNode(0, 90, 100, 64) - tmpid := ns.Stage(node) - rootID, err := ns.Flush(ctx, 1, tmpid) + retrieved, err := ns.Get(ctx, addr) require.NoError(t, err) - cache.Reset() - retrieved, err := ns.Get(ctx, rootID) - require.NoError(t, err) - assert.Equal(t, node, retrieved) - }) - t.Run("store and retrieve leaf node that has been evicted from cache", func(t *testing.T) { - node := nodestore.NewLeafNode(nil) - nodeID := ns.Stage(node) - rootID, err := ns.Flush(ctx, 1, nodeID) - require.NoError(t, err) - cache.Reset() - retrieved, err := ns.Get(ctx, rootID) - require.NoError(t, err) - assert.Equal(t, node, retrieved) - }) - t.Run("flush a set of nodes", func(t *testing.T) { - root := nodestore.NewInnerNode(0, 0, 0, 1) - rootID := ns.Stage(root) - inner1 := nodestore.NewInnerNode(0, 1, 1, 1) - inner1ID := ns.Stage(inner1) - inner2 := nodestore.NewInnerNode(0, 2, 2, 1) - inner2ID := ns.Stage(inner2) - leaf := nodestore.NewLeafNode([]byte("leaf1")) - leafID := ns.Stage(leaf) - stats := &nodestore.Statistics{ - MessageCount: 1, - } + leaf, ok := retrieved.(*nodestore.LeafNode) + require.True(t, ok) - root.PlaceChild(0, inner1ID, 1, stats) - inner1.PlaceChild(0, inner2ID, 1, stats) - inner2.PlaceChild(0, leafID, 1, stats) + found, err := io.ReadAll(leaf.Data()) + require.NoError(t, err) + require.Equal(t, data, found) - _, err = ns.Flush(ctx, 1, rootID, inner1ID, inner2ID, leafID) + // hits the cache + _, err = ns.Get(ctx, addr) require.NoError(t, err) }) } diff --git a/nodestore/sqlwal.go b/nodestore/sqlwal.go deleted file mode 100644 index aad85a0..0000000 --- a/nodestore/sqlwal.go +++ /dev/null @@ -1,132 +0,0 @@ -package nodestore - -import ( - "context" - "database/sql" - "errors" - "fmt" - "sync" - - "github.com/wkalt/dp3/util" -) - -/* -SQLWAL is a write-ahead log for the nodestore backed by an SQL database. The -only database that has been used or tested is sqlite. -*/ - -//////////////////////////////////////////////////////////////////////////////// - -type sqlWAL struct { - db *sql.DB - mtx *sync.Mutex -} - -func NewSQLWAL(ctx context.Context, db *sql.DB) (WAL, error) { - wal := &sqlWAL{db: db, mtx: &sync.Mutex{}} - if err := wal.initialize(ctx); err != nil { - return nil, err - } - return wal, nil -} - -func (w *sqlWAL) initialize(ctx context.Context) error { - _, err := w.db.ExecContext(ctx, ` - create table if not exists wal ( - id serial primary key, - producer_id text, - topic text, - node_id text, - version bigint, - deleted text, - data blob - ); - create unique index if not exists producer_id_topic_node_id_uniq_idx on wal(producer_id, topic, node_id); - create index if not exists wal_node_id_idx on wal(node_id); - `) - if err != nil { - return fmt.Errorf("failed to create wal: %w", err) - } - return nil -} - -func (w *sqlWAL) Put(ctx context.Context, entry WALEntry) error { - w.mtx.Lock() - defer w.mtx.Unlock() - stmt := `insert into wal (producer_id, topic, node_id, version, data) - values - ($1, $2, $3, $4, $5)` - params := []interface{}{entry.ProducerID, entry.Topic, entry.NodeID, entry.Version, entry.Data} - _, err := w.db.ExecContext(ctx, stmt, params...) - if err != nil { - return fmt.Errorf("failed to insert wal: %w", err) - } - return nil -} - -func (w *sqlWAL) Get(ctx context.Context, nodeID NodeID) (data []byte, err error) { - w.mtx.Lock() - defer w.mtx.Unlock() - stmt := `select data from wal where node_id = $1 and deleted is null` - err = w.db.QueryRowContext(ctx, stmt, nodeID).Scan(&data) - if err != nil { - if errors.Is(err, sql.ErrNoRows) { - return nil, NodeNotFoundError{nodeID} - } - return nil, fmt.Errorf("failed to get node %s from wal: %w", nodeID, err) - } - return data, nil -} - -func (w *sqlWAL) Delete(ctx context.Context, nodeID NodeID) error { - w.mtx.Lock() - defer w.mtx.Unlock() - stmt := `update wal set deleted = current_timestamp where node_id = $1` - _, err := w.db.ExecContext(ctx, stmt, nodeID) - if err != nil { - if errors.Is(err, sql.ErrNoRows) { - return NodeNotFoundError{nodeID} - } - return fmt.Errorf("failed to delete node %s from wal: %w", nodeID, err) - } - return nil -} - -func (w *sqlWAL) List(ctx context.Context) (paths []WALListing, err error) { - w.mtx.Lock() - defer w.mtx.Unlock() - stmt := `select producer_id, topic, version, node_id from wal where deleted is null order by id` - rows, err := w.db.QueryContext(ctx, stmt) - if err != nil { - return nil, fmt.Errorf("failed to list wal: %w", err) - } - streams := make(map[string]WALListing) - defer rows.Close() - for rows.Next() { - var entry WALEntry - if err := rows.Scan(&entry.ProducerID, &entry.Topic, &entry.Version, &entry.NodeID); err != nil { - return nil, fmt.Errorf("failed to scan row: %w", err) - } - key := entry.ProducerID + entry.Topic - if _, ok := streams[key]; !ok { - streams[key] = WALListing{ - ProducerID: entry.ProducerID, - Topic: entry.Topic, - Versions: make(map[uint64][]NodeID), - } - } - if _, ok := streams[key].Versions[entry.Version]; !ok { - streams[key].Versions[entry.Version] = []NodeID{entry.NodeID} - } - streams[key].Versions[entry.Version] = append( - streams[key].Versions[entry.Version], entry.NodeID) - } - if rows.Err() != nil { - return nil, fmt.Errorf("failed to list wal: %w", rows.Err()) - } - result := make([]WALListing, 0, len(streams)) - for _, k := range util.Okeys(streams) { - result = append(result, streams[k]) - } - return result, nil -} diff --git a/nodestore/sqlwal_test.go b/nodestore/sqlwal_test.go deleted file mode 100644 index da4be92..0000000 --- a/nodestore/sqlwal_test.go +++ /dev/null @@ -1,46 +0,0 @@ -package nodestore_test - -import ( - "context" - "crypto/rand" - "database/sql" - "testing" - - _ "github.com/mattn/go-sqlite3" - "github.com/stretchr/testify/require" - "github.com/wkalt/dp3/nodestore" -) - -func genNodeID(t *testing.T) nodestore.NodeID { - t.Helper() - bytes := make([]byte, 16) - _, err := rand.Read(bytes) - require.NoError(t, err) - return nodestore.NodeID(bytes) -} - -func TestSQLWAL(t *testing.T) { - ctx := context.Background() - db, err := sql.Open("sqlite3", ":memory:") - require.NoError(t, err) - wal, err := nodestore.NewSQLWAL(ctx, db) - require.NoError(t, err) - - producer := "my-device" - topic := "my-topic" - - t.Run("test get", func(t *testing.T) { - nodeID := genNodeID(t) - entry := nodestore.WALEntry{ - ProducerID: producer, - Topic: topic, - NodeID: nodeID, - Version: 10, - Data: []byte("data"), - } - err := wal.Put(ctx, entry) - require.NoError(t, err) - _, err = wal.Get(ctx, nodeID) - require.NoError(t, err) - }) -} diff --git a/nodestore/testutils.go b/nodestore/testutils.go index ee7d3b4..0e62c6a 100644 --- a/nodestore/testutils.go +++ b/nodestore/testutils.go @@ -14,8 +14,7 @@ func MockNodestore(ctx context.Context, t *testing.T) *Nodestore { t.Helper() store := storage.NewMemStore() cache := util.NewLRU[NodeID, Node](1e6) - wal := NewMemWAL() - return NewNodestore(store, cache, wal) + return NewNodestore(store, cache) } func RandomNodeID() NodeID { diff --git a/nodestore/wal.go b/nodestore/wal.go deleted file mode 100644 index ecb86f7..0000000 --- a/nodestore/wal.go +++ /dev/null @@ -1,46 +0,0 @@ -package nodestore - -import "context" - -/* -The WAL is used to aggregate data across multiple writes before flushing to -final storage, while still guaranteeeing those writes are durably persisted. - -We do not currently merge the WAL with resultsets on read. That seems like a -logical thing to do but may complicate the clustering story. - -Staging through the WAL is useful because it decouples the size of our data -writes from the size of our input files. The worst case scenario for us is -repeated writes to the same leaf file, requiring repeated reading, merging, and -rewriting of the same data from object storage. By staging through the WAL we -can deal with these problems when they are cheap to handle - on local disk. -*/ - -//////////////////////////////////////////////////////////////////////////////// - -// WALEntry is a single entry in the WAL. -type WALEntry struct { - RootID NodeID - ProducerID string - Topic string - NodeID NodeID - Version uint64 - Data []byte - Deleted bool -} - -// WALListing is a listing of the WAL. -type WALListing struct { - RootID NodeID - ProducerID string - Topic string - Versions map[uint64][]NodeID -} - -// WAL is the interface for the Write Ahead Log. -type WAL interface { - Put(ctx context.Context, entry WALEntry) error - Get(ctx context.Context, nodeID NodeID) ([]byte, error) - List(ctx context.Context) ([]WALListing, error) - Delete(ctx context.Context, nodeID NodeID) error -} diff --git a/routes/import.go b/routes/import.go index 2cf0f3f..abaca89 100644 --- a/routes/import.go +++ b/routes/import.go @@ -1,7 +1,6 @@ package routes import ( - "context" "encoding/json" "net/http" "os" @@ -38,15 +37,6 @@ func newImportHandler(tmgr *treemgr.TreeManager) http.HandlerFunc { httputil.BadRequest(ctx, w, "error receiving file: %s", err) return } - - // todo: WAL syncing should be done in a background thread to enable - // bigger writes to final storage, as well as to decouple write size - // from input size. For now we just kick it off on request though. - go func() { - if err := tmgr.SyncWAL(context.WithoutCancel(ctx)); err != nil { - log.Errorf(ctx, "Failed to sync WAL: %s", err) - } - }() log.Infow(ctx, "Imported", "location", req.Path, "producer_id", req.ProducerID) } } diff --git a/service/dp3.go b/service/dp3.go index a6569fa..2582224 100644 --- a/service/dp3.go +++ b/service/dp3.go @@ -21,6 +21,7 @@ import ( "github.com/wkalt/dp3/util" "github.com/wkalt/dp3/util/log" "github.com/wkalt/dp3/versionstore" + "github.com/wkalt/dp3/wal" ) /* @@ -51,26 +52,36 @@ func (dp3 *DP3) Start(ctx context.Context, options ...DP3Option) error { //nolin log.Debugf(ctx, "Debug logging enabled") store := opts.StorageProvider cache := util.NewLRU[nodestore.NodeID, nodestore.Node](opts.CacheSizeBytes) - walpath := "wal.db?_journal=WAL&mode=rwc" - db, err := sql.Open("sqlite3", walpath) + dbpath := "dp3.db?_journal=WAL&mode=rwc" + db, err := sql.Open("sqlite3", dbpath) if err != nil { return fmt.Errorf("failed to open database: %w", err) } if err = db.Ping(); err != nil { - return fmt.Errorf("failed to ping database at %s: %w", walpath, err) + return fmt.Errorf("failed to ping database at %s: %w", dbpath, err) } - wal, err := nodestore.NewSQLWAL(ctx, db) - if err != nil { - return fmt.Errorf("failed to open wal at %s: %w", walpath, err) - } - ns := nodestore.NewNodestore(store, cache, wal) + ns := nodestore.NewNodestore(store, cache) rm, err := rootmap.NewSQLRootmap(db) if err != nil { return fmt.Errorf("failed to open rootmap: %w", err) } vs := versionstore.NewSQLVersionstore(db, 1e9) - tmgr := treemgr.NewTreeManager(ns, vs, rm, 2, opts.SyncWorkers) - // go tmgr.StartWALSyncLoop(ctx) + walopts := []wal.Option{ + wal.WithInactiveBatchMergeInterval(2), + } + waldir := "waldir" + tmgr, err := treemgr.NewTreeManager( + ctx, + ns, + vs, + rm, + treemgr.WithSyncWorkers(opts.SyncWorkers), + treemgr.WithWALOpts(walopts...), + treemgr.WithWALDir(waldir), + ) + if err != nil { + return fmt.Errorf("failed to create tree manager: %w", err) + } r := routes.MakeRoutes(tmgr) srv := &http.Server{ Addr: fmt.Sprintf(":%d", opts.Port), @@ -91,6 +102,7 @@ func (dp3 *DP3) Start(ctx context.Context, options ...DP3Option) error { //nolin log.Errorf(ctx, "failed to start pprof server: %s", err) } }() + <-done log.Infof(ctx, "Allowing 10 seconds for existing connections to close") ctx, cancel := context.WithTimeout(ctx, 10*time.Second) diff --git a/tree/byo.go b/tree/byo.go new file mode 100644 index 0000000..12e9bf8 --- /dev/null +++ b/tree/byo.go @@ -0,0 +1,51 @@ +package tree + +import ( + "context" + "io" + + "github.com/wkalt/dp3/nodestore" + "github.com/wkalt/dp3/util" +) + +/* +A BYOTreeReader is a TreeReader that delegates to a user-provided function for +node retrieval. This is used to construct nodestore-backed tree readers. +*/ + +type byoTreeReader struct { + root nodestore.NodeID + get func(context.Context, nodestore.NodeID) (nodestore.Node, error) +} + +// Root returns the root node ID. +func (t *byoTreeReader) Root() nodestore.NodeID { + return t.root +} + +// Get returns the node with the given ID. +func (t *byoTreeReader) Get(ctx context.Context, id nodestore.NodeID) (nodestore.Node, error) { + return t.get(ctx, id) +} + +// GetLeafData returns the data for a leaf node. +func (t *byoTreeReader) GetLeafData(ctx context.Context, id nodestore.NodeID) (io.ReadSeekCloser, error) { + node, err := t.get(ctx, id) + if err != nil { + return nil, err + } + + leaf, ok := node.(*nodestore.LeafNode) + if !ok { + return nil, newUnexpectedNodeError(nodestore.Leaf, node) + } + return util.NewReadSeekNopCloser(leaf.Data()), nil +} + +// NewBYOTreeReader creates a new BYOTreeReader. +func NewBYOTreeReader( + root nodestore.NodeID, + get func(context.Context, nodestore.NodeID) (nodestore.Node, error), +) TreeReader { + return &byoTreeReader{root, get} +} diff --git a/tree/bytetree.go b/tree/bytetree.go new file mode 100644 index 0000000..c11ec1f --- /dev/null +++ b/tree/bytetree.go @@ -0,0 +1,81 @@ +package tree + +import ( + "bytes" + "context" + "errors" + "fmt" + "io" + + "github.com/wkalt/dp3/nodestore" + "github.com/wkalt/dp3/util" +) + +/* +ByteTree is a treereader implementation backed by the byte serialization of a +memtree. The memtree uses it to deserialize its on-disk representation from WAL. +*/ + +//////////////////////////////////////////////////////////////////////////////// + +type byteTree struct{ r *bytes.Reader } + +func isLeaf(data []byte) bool { + return data[0] > 128 +} + +func (b *byteTree) GetReader(ctx context.Context, id nodestore.NodeID) (io.ReadSeekCloser, error) { + offset := id.Offset() + length := id.Length() + _, err := b.r.Seek(int64(offset), io.SeekStart) + if err != nil { + return nil, fmt.Errorf("failed to seek to offset: %w", err) + } + buf := make([]byte, length) + _, err = io.ReadFull(b.r, buf) + if err != nil { + return nil, fmt.Errorf("failed to read node data: %w", err) + } + return util.NewReadSeekNopCloser(bytes.NewReader(buf[1:])), nil +} + +func (b *byteTree) Get(ctx context.Context, id nodestore.NodeID) (nodestore.Node, error) { + offset := id.Offset() + length := id.Length() + _, err := b.r.Seek(int64(offset), io.SeekStart) + if err != nil { + return nil, fmt.Errorf("failed to seek to offset: %w", err) + } + buf := make([]byte, length) + _, err = io.ReadFull(b.r, buf) + if err != nil { + return nil, fmt.Errorf("failed to read node data: %w", err) + } + if isLeaf(buf) { + node := nodestore.NewLeafNode(nil) + if err := node.FromBytes(buf); err != nil { + return nil, fmt.Errorf("failed to deserialize leaf node: %w", err) + } + return node, nil + } + node := nodestore.NewInnerNode(0, 0, 0, 0) + if err := node.FromBytes(buf); err != nil { + return nil, fmt.Errorf("failed to deserialize inner node: %w", err) + } + return node, nil +} + +func (b *byteTree) Put(ctx context.Context, id nodestore.NodeID, node nodestore.Node) error { + return errors.New("not implemented") +} + +func (b *byteTree) Root() nodestore.NodeID { + _, _ = b.r.Seek(-16, io.SeekEnd) + id := make([]byte, 16) + _, _ = io.ReadFull(b.r, id) + return nodestore.NodeID(id) +} + +func (b *byteTree) SetRoot(nodestore.NodeID) { + panic("not implemented") +} diff --git a/tree/errors.go b/tree/errors.go index bc0119a..a453345 100644 --- a/tree/errors.go +++ b/tree/errors.go @@ -6,6 +6,12 @@ import ( "github.com/wkalt/dp3/nodestore" ) +/* +Errors that can be returned by the tree package. +*/ + +//////////////////////////////////////////////////////////////////////////////// + // UnexpectedNodeError is returned when a node of the wrong type is found in the // nodestore. type UnexpectedNodeError struct { @@ -48,3 +54,19 @@ func (e OutOfBoundsError) Is(target error) bool { _, ok := target.(OutOfBoundsError) return ok } + +// MismatchedHeightsError is returned if a merge is attempted on two nodes of +// different height. +type MismatchedHeightsError struct { + height1 uint8 + height2 uint8 +} + +func (e MismatchedHeightsError) Error() string { + return fmt.Sprintf("mismatched depths: %d and %d", e.height1, e.height2) +} + +func (e MismatchedHeightsError) Is(target error) bool { + _, ok := target.(MismatchedHeightsError) + return ok +} diff --git a/tree/iterator.go b/tree/iterator.go index 91a73c0..029a55e 100644 --- a/tree/iterator.go +++ b/tree/iterator.go @@ -27,19 +27,18 @@ type Iterator struct { reader *mcap.Reader rsc io.ReadSeekCloser msgIterator mcap.MessageIterator - ns *nodestore.Nodestore + tr TreeReader } // NewTreeIterator returns a new iterator over the given tree. func NewTreeIterator( ctx context.Context, - ns *nodestore.Nodestore, - rootID nodestore.NodeID, + tr TreeReader, start uint64, end uint64, ) (*Iterator, error) { - it := &Iterator{start: start, end: end, ns: ns} - if err := it.initialize(ctx, rootID); err != nil { + it := &Iterator{start: start, end: end, tr: tr} + if err := it.initialize(ctx, tr.Root()); err != nil { return nil, err } return it, nil @@ -98,7 +97,7 @@ func (ti *Iterator) initialize(ctx context.Context, rootID nodestore.NodeID) err for len(stack) > 0 { nodeID := stack[len(stack)-1] stack = stack[:len(stack)-1] - node, err := ti.ns.Get(ctx, nodeID) + node, err := ti.tr.Get(ctx, nodeID) if err != nil { return fmt.Errorf("failed to get node %s: %w", nodeID, err) } @@ -129,9 +128,9 @@ func (ti *Iterator) initialize(ctx context.Context, rootID nodestore.NodeID) err // openNextLeaf opens the next leaf in the iterator. func (ti *Iterator) openNextLeaf(ctx context.Context) error { leafID := ti.leafIDs[ti.nextLeaf] - rsc, err := ti.ns.GetLeaf(ctx, leafID) + rsc, err := ti.tr.GetLeafData(ctx, leafID) if err != nil { - return fmt.Errorf("failed to get node: %w", err) + return fmt.Errorf("failed to get reader: %w", err) } reader, err := mcap.NewReader(rsc) if err != nil { diff --git a/tree/iterator_test.go b/tree/iterator_test.go index 895746f..092db84 100644 --- a/tree/iterator_test.go +++ b/tree/iterator_test.go @@ -1,96 +1,58 @@ package tree_test import ( - "bytes" "context" "errors" "io" "testing" - fm "github.com/foxglove/mcap/go/mcap" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/wkalt/dp3/mcap" - "github.com/wkalt/dp3/nodestore" "github.com/wkalt/dp3/tree" - "github.com/wkalt/dp3/util" ) func TestTreeIterator(t *testing.T) { ctx := context.Background() - ns := nodestore.MockNodestore(ctx, t) - rootID, err := ns.NewRoot(ctx, 0, util.Pow(uint64(64), 3), 64, 64) - require.NoError(t, err) - // create two mcap files and stick them into the tree - buf1 := &bytes.Buffer{} - buf2 := &bytes.Buffer{} - offset := 0 - for _, buf := range []*bytes.Buffer{buf1, buf2} { - w, err := mcap.NewWriter(buf) - require.NoError(t, err) - require.NoError(t, w.WriteHeader(&fm.Header{})) - require.NoError(t, w.WriteSchema(&fm.Schema{ - ID: 1, - Name: "schema1", - Encoding: "ros1", - Data: []byte{0x01, 0x02, 0x03}, - })) - require.NoError(t, w.WriteChannel(&fm.Channel{ - ID: 0, - SchemaID: 1, - Topic: "/topic", - MessageEncoding: "ros1msg", - })) - for i := 0; i < 10; i++ { - require.NoError(t, w.WriteMessage(&fm.Message{ - LogTime: uint64(i + offset), - Data: []byte("hello"), - })) - } - require.NoError(t, w.Close()) - offset += 64 - } - stats := &nodestore.Statistics{ - MessageCount: 10, - } - version := uint64(1) - rootID = insertLeaf(t, ctx, ns, rootID, version, 0, buf1.Bytes(), stats) - version++ - rootID = insertLeaf(t, ctx, ns, rootID, version, 64*1e9, buf2.Bytes(), stats) + cases := []struct { + assertion string + times [][]uint64 + expectedMessageCount int + }{ + { + "empty tree", + [][]uint64{}, + 0, + }, + { + "one message", + [][]uint64{{100}}, + 1, + }, - it, err := tree.NewTreeIterator(ctx, ns, rootID, 0, 128*1e9) - require.NoError(t, err) - defer require.NoError(t, it.Close()) - count := 0 - for it.More() { - schema, channel, message, err := it.Next(ctx) - if errors.Is(err, io.EOF) { - break - } - require.NoError(t, err) - require.Equal(t, []byte("hello"), message.Data) - require.Equal(t, "schema1", schema.Name) - require.Equal(t, "/topic", channel.Topic) - count++ + { + "three messages", + [][]uint64{{100, 1000, 10000}}, + 3, + }, } - assert.Equal(t, 20, count) -} -func insertLeaf( - t *testing.T, - ctx context.Context, - ns *nodestore.Nodestore, - nodeID nodestore.NodeID, - version uint64, - time uint64, - data []byte, - stats *nodestore.Statistics, -) nodestore.NodeID { - t.Helper() - _, path, err := tree.Insert(ctx, ns, nodeID, version, time, data, stats) - require.NoError(t, err) - rootID, err := ns.Flush(ctx, version, path...) - require.NoError(t, err) - return rootID + for _, c := range cases { + t.Run(c.assertion, func(t *testing.T) { + tr := tree.MergeInserts(ctx, t, 0, 128*1e9, 2, 64, c.times) + + it, err := tree.NewTreeIterator(ctx, tr, 0, 128*1e9) + require.NoError(t, err) + var count int + for { + _, _, _, err := it.Next(ctx) + if errors.Is(err, io.EOF) { + break + } + require.NoError(t, err) + count += 1 + } + require.Equal(t, c.expectedMessageCount, count) + require.NoError(t, it.Close()) + }) + } } diff --git a/tree/memtree.go b/tree/memtree.go new file mode 100644 index 0000000..53b6588 --- /dev/null +++ b/tree/memtree.go @@ -0,0 +1,181 @@ +package tree + +import ( + "bytes" + "context" + "errors" + "fmt" + "io" + "slices" + + "github.com/wkalt/dp3/nodestore" + "github.com/wkalt/dp3/util" +) + +/* +MemTree is an in-memory implementation of a tree, with serialization methods +to/from disk. It is used for building trees in memory, either while receiving +input data and preparing to flush to WAL, or when reading from the WAL and +preparing to serialize to storage. +*/ + +////////////////////////////////////////////////////////////////////////////// + +// MemTree implements both TreeReader and TreeWriter. +type MemTree struct { + root nodestore.NodeID + nodes map[nodestore.NodeID]nodestore.Node +} + +// Get returns the node with the given ID. +func (m *MemTree) Get(ctx context.Context, id nodestore.NodeID) (nodestore.Node, error) { + node, ok := m.nodes[id] + if !ok { + return nil, nodestore.NodeNotFoundError{NodeID: id} + } + return node, nil +} + +// GetLeafData returns the data for a leaf node. +func (m *MemTree) GetLeafData(ctx context.Context, id nodestore.NodeID) (io.ReadSeekCloser, error) { + node, err := m.Get(ctx, id) + if err != nil { + return nil, err + } + leaf, ok := node.(*nodestore.LeafNode) + if !ok { + return nil, errors.New("node is not a leaf") + } + return util.NewReadSeekNopCloser(leaf.Data()), nil +} + +// Put inserts a node into the MemTree. +func (m *MemTree) Put(ctx context.Context, id nodestore.NodeID, node nodestore.Node) error { + if m.nodes == nil { + m.nodes = make(map[nodestore.NodeID]nodestore.Node) + } + m.nodes[id] = node + return nil +} + +// Root returns the root node ID. +func (m *MemTree) Root() nodestore.NodeID { + return m.root +} + +// FromBytes deserializes the byte array into the memtree. Memtrees are +// serialized to disk as bytetrees. +func (m *MemTree) FromBytes(ctx context.Context, data []byte) error { + bt := byteTree{r: bytes.NewReader(data)} + rootID := bt.Root() + m.SetRoot(rootID) + ids := []nodestore.NodeID{rootID} + for len(ids) > 0 { + id := ids[0] + ids = ids[1:] + node, err := bt.Get(ctx, id) + if err != nil { + return fmt.Errorf("failed to look up node %s: %w", id, err) + } + if err := m.Put(ctx, id, node); err != nil { + return fmt.Errorf("failed to put node %s: %w", id, err) + } + if inner, ok := node.(*nodestore.InnerNode); ok { + for _, child := range inner.Children { + if child == nil { + continue + } + ids = append(ids, child.ID) + } + } + } + return nil +} + +// ToBytes serializes the memtree to a byte array suitable for storage in the +// WAL or in an object. The nodes are serialized from leaf to root. The IDs are +// boosted by offset. The last 16 bytes are the root ID. +func (m *MemTree) ToBytes(ctx context.Context, oid uint64) ([]byte, error) { // nolint: funlen + root, err := m.Get(ctx, m.root) + if err != nil { + return nil, fmt.Errorf("failed to read root: %w", err) + } + node, ok := root.(*nodestore.InnerNode) + if !ok { + return nil, newUnexpectedNodeError(nodestore.Inner, root) + } + path := []nodestore.NodeID{m.root} + nodes := []*nodestore.InnerNode{node} + for len(nodes) > 0 { + node, nodes = nodes[len(nodes)-1], nodes[:len(nodes)-1] + for _, child := range node.Children { + if child == nil { + continue + } + path = append(path, child.ID) + childNode, err := m.Get(ctx, child.ID) + if err != nil { + return nil, fmt.Errorf("failed to get child node: %w", err) + } + if inner, ok := childNode.(*nodestore.InnerNode); ok { + nodes = append(nodes, inner) + } + } + } + + // serialize the path in reverse order + buf := &bytes.Buffer{} + offset := 0 + slices.Reverse(path) + processed := make(map[nodestore.NodeID]nodestore.NodeID) + for _, id := range path { + node, err := m.Get(ctx, id) + if err != nil { + return nil, fmt.Errorf("failed to get node: %w", err) + } + if inner, ok := node.(*nodestore.InnerNode); ok { + for _, child := range inner.Children { + if child == nil { + continue + } + if remapped, ok := processed[child.ID]; ok { + child.ID = remapped + } + } + } + n, err := buf.Write(node.ToBytes()) + if err != nil { + return nil, fmt.Errorf("failed to write node: %w", err) + } + nodeID := nodestore.NewNodeID(oid, offset, n) + offset += n + processed[id] = nodeID + } + + // write the root ID onto the end + remappedRootID := processed[m.root] + if _, err = buf.Write(remappedRootID[:]); err != nil { + return nil, fmt.Errorf("failed to write remapped root: %w", err) + } + return buf.Bytes(), nil +} + +// SetRoot sets the root node of the MemTree. +func (m *MemTree) SetRoot(id nodestore.NodeID) { + m.root = id +} + +// NewMemTree creates a new MemTree with the given root node. +func NewMemTree(rootID nodestore.NodeID, root *nodestore.InnerNode) *MemTree { + nodes := make(map[nodestore.NodeID]nodestore.Node) + nodes[rootID] = root + return &MemTree{root: rootID, nodes: nodes} +} + +func (m *MemTree) String(ctx context.Context) string { + s, err := Print(ctx, m) + if err != nil { + return err.Error() + } + return s +} diff --git a/tree/memtree_test.go b/tree/memtree_test.go new file mode 100644 index 0000000..e12a141 --- /dev/null +++ b/tree/memtree_test.go @@ -0,0 +1,133 @@ +package tree_test + +import ( + "bytes" + "context" + "testing" + + "github.com/stretchr/testify/require" + "github.com/wkalt/dp3/mcap" + "github.com/wkalt/dp3/nodestore" + "github.com/wkalt/dp3/tree" + "github.com/wkalt/dp3/util" +) + +func TestMemtreeSerialization(t *testing.T) { + ctx := context.Background() + cases := []struct { + assertion string + height uint8 + timestamps []uint64 + expected string + }{ + { + "two trees", + 1, + []uint64{100, 120}, + "[0-4096 [0-64:1 (count=2) [leaf 2 msgs]]]", + }, + { + "tree trees", + 1, + []uint64{100, 120, 1024 * 1e9}, + "[0-4096 [0-64:1 (count=2) [leaf 2 msgs]] [1024-1088:2 (count=1) [leaf 1 msg]]]", + }, + { + "height 2", + 2, + []uint64{100, 120, 1024 * 1e9}, + "[0-262144 [0-4096:2 (count=3) [0-64:1 (count=2) [leaf 2 msgs]] [1024-1088:2 (count=1) [leaf 1 msg]]]]", + }, + } + + for _, c := range cases { + root := nodestore.NewInnerNode( + c.height, + 0, + util.Pow(uint64(64), int(c.height+1)), + 64, + ) + version := uint64(0) + + trees := make([]tree.TreeReader, len(c.timestamps)) + for i, ts := range c.timestamps { + data := &bytes.Buffer{} + id := nodestore.RandomNodeID() + mt := tree.NewMemTree(id, root) + mcap.WriteFile(t, data, []uint64{ts}) + require.NoError(t, tree.Insert(ctx, mt, version, ts, data.Bytes(), &nodestore.Statistics{ + MessageCount: 1, + })) + + version++ + trees[i] = mt + } + + var merged tree.MemTree + require.NoError(t, tree.Merge(ctx, &merged, trees...)) + + s1, err := tree.Print(ctx, &merged) + require.NoError(t, err) + + bytes, err := merged.ToBytes(ctx, version) + require.NoError(t, err) + + var merged2 tree.MemTree + require.NoError(t, merged2.FromBytes(ctx, bytes)) + + s2, err := tree.Print(ctx, &merged2) + require.NoError(t, err) + + require.Equal(t, c.expected, s1, s2) + } +} + +func TestMemtreeMerge(t *testing.T) { + cases := []struct { + assertion string + height uint8 + inputs [][]uint64 + expected string + }{ + { + "one tree", + 2, + [][]uint64{{100}}, + "[0-262144 [0-4096:1 (count=1) [0-64:1 (count=1) [leaf 1 msg]]]]", + }, + { + "two trees, overlapping", + 2, + [][]uint64{{100e9}, {65e9}}, + "[0-262144 [0-4096:2 (count=2) [64-128:2 (count=2) [leaf 2 msgs]]]]", + }, + { + "two trees, disjoint", + 1, + [][]uint64{{100e9}, {1000e9}}, + "[0-4096 [64-128:1 (count=1) [leaf 1 msg]] [960-1024:2 (count=1) [leaf 1 msg]]]", + }, + } + ctx := context.Background() + for _, c := range cases { + t.Run(c.assertion, func(t *testing.T) { + version := uint64(1) + trees := []tree.TreeReader{} + for _, input := range c.inputs { + node := nodestore.NewInnerNode(c.height, 0, uint64(util.Pow(64, int(c.height)+1)), 64) + pt := tree.NewMemTree(nodestore.RandomNodeID(), node) + data := &bytes.Buffer{} + mcap.WriteFile(t, data, input) + require.NoError(t, tree.Insert(ctx, pt, version, input[0], data.Bytes(), &nodestore.Statistics{MessageCount: 1})) + version++ + trees = append(trees, pt) + } + + outputNode := nodestore.NewInnerNode(2, 0, uint64(util.Pow(64, int(c.height)+1)), 64) + output := tree.NewMemTree(nodestore.RandomNodeID(), outputNode) + require.NoError(t, tree.Merge(ctx, output, trees...)) + + require.Equal(t, c.expected, output.String(ctx)) + }) + } +} diff --git a/tree/testutils.go b/tree/testutils.go new file mode 100644 index 0000000..b77ceb2 --- /dev/null +++ b/tree/testutils.go @@ -0,0 +1,45 @@ +package tree + +import ( + "bytes" + "context" + "testing" + + "github.com/stretchr/testify/require" + "github.com/wkalt/dp3/mcap" + "github.com/wkalt/dp3/nodestore" +) + +// MergeInserts executes a list of inserts and then merges the resulting partial +// trees into a single tree. +func MergeInserts( + ctx context.Context, + t *testing.T, + start uint64, + end uint64, + height uint8, + bfactor int, + times [][]uint64, +) TreeReader { + t.Helper() + version := uint64(1) + trees := make([]TreeReader, len(times)) + for i, batch := range times { + root := nodestore.NewInnerNode(height, start, end, bfactor) + id := nodestore.RandomNodeID() + tmp := NewMemTree(id, root) + buf := &bytes.Buffer{} + mcap.WriteFile(t, buf, batch) + require.NoError(t, Insert( + ctx, tmp, version, batch[0]*1e9, buf.Bytes(), &nodestore.Statistics{ + MessageCount: 1, + }, + )) + version++ + trees[i] = tmp + } + root := nodestore.NewInnerNode(height, start, end, bfactor) + output := NewMemTree(nodestore.RandomNodeID(), root) + require.NoError(t, Merge(ctx, output, trees...)) + return output +} diff --git a/tree/tree.go b/tree/tree.go index c616f90..7e10f71 100644 --- a/tree/tree.go +++ b/tree/tree.go @@ -1,10 +1,13 @@ package tree import ( + "bytes" "context" "errors" "fmt" + fmcap "github.com/foxglove/mcap/go/mcap" + "github.com/wkalt/dp3/mcap" "github.com/wkalt/dp3/nodestore" ) @@ -19,7 +22,80 @@ module. // ////////////////////////////////////////////////////////////////////////////// -// StatRange is a range of multigranular statistics computed on inner nodes. +// Insert writes the provided slice of data into the supplied tree writer, into +// the leaf of the tree that spans the requested timestamp. Assuming no error, +// after insert has returned the tree writer will reflect a partial tree from +// root to leaf. In practice the tree writer passed to insert is always a +// "memtree". The partial trees that result from insert are serialized to the +// WAL, and later merged into the main tree in batches by the tree manager. +// +// Accordingly and confusingly, "insert" is only ever called on empty trees, and +// the test helpers for insertion combine it with a merge operation to produce +// more interesting examples. +func Insert( + ctx context.Context, + tw TreeWriter, + version uint64, + timestamp uint64, + data []byte, + statistics *nodestore.Statistics, +) error { + oldRootID := tw.Root() + oldRootNode, err := tw.Get(ctx, oldRootID) + if err != nil { + return nodestore.NodeNotFoundError{NodeID: oldRootID} + } + oldroot, ok := oldRootNode.(*nodestore.InnerNode) + if !ok || oldroot == nil { + return newUnexpectedNodeError(nodestore.Inner, oldroot) + } + if timestamp < oldroot.Start*1e9 || timestamp >= oldroot.End*1e9 { + return OutOfBoundsError{timestamp, oldroot.Start, oldroot.End} + } + root := nodestore.NewInnerNode( + oldroot.Height, + oldroot.Start, + oldroot.End, + len(oldroot.Children), + ) + ids := make([]nodestore.NodeID, 0, oldroot.Height+1) + for range oldroot.Height + 1 { + id := nodestore.RandomNodeID() + ids = append(ids, id) + } + rootID := ids[0] + if err := tw.Put(ctx, rootID, root); err != nil { + return fmt.Errorf("failed to store new root: %w", err) + } + current := root + for i := 1; current.Height > 1; i++ { + bucket := bucket(timestamp, current) + bwidth := bwidth(current) + node := nodestore.NewInnerNode( + current.Height-1, + current.Start+bucket*bwidth, + current.Start+(bucket+1)*bwidth, + len(current.Children), + ) + if err := tw.Put(ctx, ids[i], node); err != nil { + return fmt.Errorf("failed to store inner node: %w", err) + } + current.PlaceChild(bucket, ids[i], version, statistics) + current = node + } + // now at the parent of the leaf + nodeID := ids[len(ids)-1] + bucket := bucket(timestamp, current) + node := nodestore.NewLeafNode(data) + if err := tw.Put(ctx, nodeID, node); err != nil { + return fmt.Errorf("failed to store leaf node: %w", err) + } + current.PlaceChild(bucket, nodeID, version, statistics) + tw.SetRoot(rootID) + return nil +} + +// StatRange is a range of statistics. type StatRange struct { Start uint64 `json:"start"` End uint64 `json:"end"` @@ -33,18 +109,17 @@ type StatRange struct { // lead to confusing results so clients must be prepared to handle it. func GetStatRange( ctx context.Context, - ns *nodestore.Nodestore, - rootID nodestore.NodeID, + tr TreeReader, start uint64, end uint64, granularity uint64, ) ([]StatRange, error) { ranges := []StatRange{} - stack := []nodestore.NodeID{rootID} + stack := []nodestore.NodeID{tr.Root()} for len(stack) > 0 { nodeID := stack[len(stack)-1] stack = stack[:len(stack)-1] - node, err := ns.Get(ctx, nodeID) + node, err := tr.Get(ctx, nodeID) if err != nil { return nil, fmt.Errorf("failed to get node %s: %w", nodeID, err) } @@ -76,52 +151,197 @@ func GetStatRange( return ranges, nil } -// Insert stages leaf data into the tree rooted at nodeID, in the leaf location -// appropriate for the supplied timestamp. -// -// All new nodes down to the leaf are assigned the version parameter as the -// version, and associated with the provided statistics record. -// new leaf are assigned the version parameter. -// -// If there is no error, a new root node ID is returned, corresponding to the root -// of the new tree. At the time it is returned, the new node exists only in the nodestore's -// staging map. The caller is responsible for flushing the ID to WAL. -// -// On error, the rootID is returned unchanged, and the path is nil. -func Insert( - ctx context.Context, - ns *nodestore.Nodestore, - rootID nodestore.NodeID, - version uint64, - timestamp uint64, - data []byte, - statistics *nodestore.Statistics, -) (newRootID nodestore.NodeID, path []nodestore.NodeID, err error) { - root, err := cloneInnerNode(ctx, ns, rootID) +// Merge a list of TreeReaders into an empty TreeWriter. At each level of +// conflict, merged nodes are versioned with the highest version of the +// conflicted inputs. +func Merge(ctx context.Context, tw TreeWriter, trees ...TreeReader) error { + if len(trees) == 0 { + return nil + } + ids := make([]nodestore.NodeID, 0, len(trees)) + roots := make([]*nodestore.InnerNode, 0, len(trees)) + for _, tree := range trees { + id := tree.Root() + root, err := tree.Get(ctx, id) + if err != nil { + return fmt.Errorf("failed to get root: %w", err) + } + innerNode, ok := root.(*nodestore.InnerNode) + if !ok { + return newUnexpectedNodeError(nodestore.Inner, root) + } + roots = append(roots, innerNode) + ids = append(ids, id) + } + for _, root := range roots[1:] { + if root.Height != roots[0].Height { + return MismatchedHeightsError{root.Height, roots[0].Height} + } + } + mergedRoot, err := mergeLevel(ctx, tw, ids, trees) if err != nil { - return rootID, nil, err + return fmt.Errorf("failed to merge: %w", err) } - if timestamp < root.Start*1e9 || timestamp >= root.End*1e9 { - return rootID, nil, OutOfBoundsError{timestamp, root.Start, root.End} + tw.SetRoot(mergedRoot) + return nil +} + +// mergeInnerNodes merges a list of inner nodes into a single inner node in the +// new tree writer. It does this by identifying "conflicted" child indexes, and +// then calling mergeLevel on each of the conflicted sets of children. The +// mergeLevel function will either dispatch back to mergeInnerNodes, or to +// mergeLeaves depending on the height at which it is called. + +// The passed nodes array must correspond in order and match in length with the +// supplied list of tree readers. +func mergeInnerNodes( + ctx context.Context, + pt TreeWriter, + nodes []*nodestore.InnerNode, + trees []TreeReader, +) (nodestore.Node, error) { + conflicts := []int{} + node := nodes[0] + singleton := len(nodes) == 1 + for i, child := range node.Children { + if child != nil && singleton { + conflicts = append(conflicts, i) + continue + } else { + var conflicted bool + for _, sibling := range nodes[1:] { + cousin := sibling.Children[i] + if child == nil && cousin != nil || + child != nil && cousin == nil || + child != nil && cousin != nil { + conflicted = true + break + } + } + if conflicted { + conflicts = append(conflicts, i) + } + } } - rootID = ns.Stage(root) - nodes := []nodestore.NodeID{rootID} - current := root - for current.Height > 1 { - current, err = descend(ctx, ns, &nodes, current, timestamp, version, statistics) + // Create a new merged child in the location of each conflict + newInner := nodestore.NewInnerNode(node.Height, node.Start, node.End, len(node.Children)) + for _, conflict := range conflicts { + conflictedNodes := make([]nodestore.NodeID, 0, len(trees)) + conflictedTrees := make([]TreeReader, 0, len(trees)) + stats := &nodestore.Statistics{} + maxVersion := uint64(0) + for i, node := range nodes { + child := node.Children[conflict] + if child == nil { + continue + } + if child.Version > maxVersion { + maxVersion = child.Version + } + conflictedNodes = append(conflictedNodes, child.ID) + conflictedTrees = append(conflictedTrees, trees[i]) + if err := stats.Add(child.Statistics); err != nil { + return nil, fmt.Errorf("failed to add statistics: %w", err) + } + } + merged, err := mergeLevel(ctx, pt, conflictedNodes, conflictedTrees) if err != nil { - return rootID, nil, err + return nil, err + } + newInner.Children[conflict] = &nodestore.Child{ + ID: merged, + Version: maxVersion, + Statistics: stats, } } - bucket := bucket(timestamp, current) - node := nodestore.NewLeafNode(data) - stagedID := ns.Stage(node) // todo: should insert flush to WAL? - nodes = append(nodes, stagedID) - current.PlaceChild(bucket, stagedID, version, statistics) - return nodes[0], nodes, nil + return newInner, nil +} + +// mergeLeaves merges a set of leaf nodes into a single leaf node. Data is +// merged in timestamp order. +func mergeLeaves( + leaves []nodestore.Node, +) (nodestore.Node, error) { + if len(leaves) == 0 { + return nil, errors.New("no leaves to merge") + } + if len(leaves) == 1 { + return leaves[0], nil + } + iterators := make([]fmcap.MessageIterator, len(leaves)) + for i, leaf := range leaves { + leaf, ok := leaf.(*nodestore.LeafNode) + if !ok { + return nil, newUnexpectedNodeError(nodestore.Leaf, leaf) + } + reader, err := mcap.NewReader(leaf.Data()) + if err != nil { + return nil, fmt.Errorf("failed to build mcap reader: %w", err) + } + defer reader.Close() + iterators[i], err = reader.Messages() + if err != nil { + return nil, fmt.Errorf("failed to create iterator: %w", err) + } + } + buf := &bytes.Buffer{} + if err := mcap.Nmerge(buf, iterators...); err != nil { + return nil, fmt.Errorf("failed to merge: %w", err) + } + return nodestore.NewLeafNode(buf.Bytes()), nil +} + +// mergeLevel recursively merges a set of nodes at the same level in their +// respective input trees starting at the roots. If all passed nodes are not of +// the same type and height (for inner nodes), an error will result. The final +// merged result is placed into the output tree writer with a random node ID. +func mergeLevel( + ctx context.Context, + pt TreeWriter, + ids []nodestore.NodeID, + trees []TreeReader, +) (nodeID nodestore.NodeID, err error) { + if len(ids) == 0 { + return nodeID, errors.New("no nodes to merge") + } + nodes := make([]nodestore.Node, len(ids)) + for i, id := range ids { + node, err := trees[i].Get(ctx, id) + if err != nil { + return nodeID, fmt.Errorf("failed to get node: %w", err) + } + nodes[i] = node + } + for _, node := range nodes { + if node.Type() != nodes[0].Type() { + return nodeID, errors.New("mismatched node types") + } + } + var node nodestore.Node + switch nodes[0].Type() { + case nodestore.Leaf: + node, err = mergeLeaves(nodes) + if err != nil { + return nodeID, fmt.Errorf("failed to merge leaves: %w", err) + } + case nodestore.Inner: + innerNodes := make([]*nodestore.InnerNode, len(nodes)) + for i, node := range nodes { + innerNodes[i] = node.(*nodestore.InnerNode) + } + node, err = mergeInnerNodes(ctx, pt, innerNodes, trees) + if err != nil { + return nodeID, fmt.Errorf("failed to merge inner nodes: %w", err) + } + } + id := nodestore.RandomNodeID() + if err := pt.Put(ctx, id, node); err != nil { + return nodeID, fmt.Errorf("failed to insert node: %w", err) + } + return id, nil } -// bwidth returns the width of each bucket in seconds. +// // bwidth returns the width of each bucket in seconds. func bwidth(n *nodestore.InnerNode) uint64 { bwidth := (n.End - n.Start) / uint64(len(n.Children)) return bwidth @@ -134,58 +354,3 @@ func bucket(nanos uint64, n *nodestore.InnerNode) uint64 { bucket := (nanos - n.Start*1e9) / (1e9 * bwidth) return bucket } - -// cloneInnerNode returns a new inner node with the same contents as the node -// with the given id, but with the version changed to the one supplied. -func cloneInnerNode(ctx context.Context, ns *nodestore.Nodestore, id nodestore.NodeID) (*nodestore.InnerNode, error) { - node, err := ns.Get(ctx, id) - if err != nil { - return nil, fmt.Errorf("failed to clone inner node %d: %w", id, err) - } - oldNode, ok := node.(*nodestore.InnerNode) - if !ok { - return nil, newUnexpectedNodeError(nodestore.Inner, node) - } - newNode := nodestore.NewInnerNode(oldNode.Height, oldNode.Start, oldNode.End, len(oldNode.Children)) - for i := range oldNode.Children { - if oldNode.Children[i] != nil { - newNode.Children[i] = &nodestore.Child{ - ID: oldNode.Children[i].ID, - Version: oldNode.Children[i].Version, - } - } - } - return newNode, nil -} - -// descend the tree to the node that contains the given timestamp, copying nodes -// at each step and recording the path taken. -func descend( - ctx context.Context, - ns *nodestore.Nodestore, - nodeIDs *[]nodestore.NodeID, - current *nodestore.InnerNode, - timestamp uint64, - version uint64, - stats *nodestore.Statistics, -) (node *nodestore.InnerNode, err error) { - bucket := bucket(timestamp, current) - if existing := current.Children[bucket]; existing != nil { - node, err = cloneInnerNode(ctx, ns, existing.ID) - if err != nil { - return nil, err - } - } else { - bwidth := bwidth(current) - node = nodestore.NewInnerNode( - current.Height-1, - current.Start+bucket*bwidth, - current.Start+(bucket+1)*bwidth, - len(current.Children), - ) - } - nodeID := ns.Stage(node) - current.PlaceChild(bucket, nodeID, version, stats) - *nodeIDs = append(*nodeIDs, nodeID) - return node, nil -} diff --git a/tree/tree_test.go b/tree/tree_test.go index 89f26be..63e1770 100644 --- a/tree/tree_test.go +++ b/tree/tree_test.go @@ -1,14 +1,12 @@ package tree_test import ( - "bytes" "context" "math" "strings" "testing" "github.com/stretchr/testify/require" - "github.com/wkalt/dp3/mcap" "github.com/wkalt/dp3/nodestore" "github.com/wkalt/dp3/tree" "github.com/wkalt/dp3/util" @@ -16,118 +14,131 @@ import ( func TestTreeErrors(t *testing.T) { ctx := context.Background() - ns := nodestore.MockNodestore(ctx, t) t.Run("inserting into a non-existent root", func(t *testing.T) { - id := nodestore.RandomNodeID() - _, _, err := tree.Insert(ctx, ns, id, 0, 0, []byte{}, &nodestore.Statistics{}) - require.ErrorIs(t, err, nodestore.NodeNotFoundError{NodeID: id}) + root := nodestore.NewInnerNode(1, 0, 4096, 64) + tw := tree.NewMemTree(nodestore.RandomNodeID(), root) + tw.SetRoot(nodestore.RandomNodeID()) + err := tree.Insert(ctx, tw, 0, 0, []byte{0x01}, &nodestore.Statistics{}) + require.ErrorIs(t, err, nodestore.NodeNotFoundError{}) }) + + t.Run("root is nil", func(t *testing.T) { + tw := tree.NewMemTree(nodestore.RandomNodeID(), nil) + require.Error(t, tree.Insert(ctx, tw, 0, 0, []byte{0x01}, &nodestore.Statistics{})) + }) + t.Run("out of bounds insert", func(t *testing.T) { - rootID, err := ns.NewRoot(ctx, 100, 1e9, 64, 64) - require.NoError(t, err) - _, _, err = tree.Insert(ctx, ns, rootID, 10, 1e9, []byte{}, &nodestore.Statistics{}) + root := nodestore.NewInnerNode(1, 0, 4096, 64) + tw := tree.NewMemTree(nodestore.RandomNodeID(), root) + err := tree.Insert(ctx, tw, 0, 10000*1e9, []byte{0x01}, &nodestore.Statistics{}) require.ErrorIs(t, err, tree.OutOfBoundsError{}) }) - t.Run("inserting into a node with mislinked children", func(t *testing.T) { - id1 := nodestore.RandomNodeID() - node1 := nodestore.NewInnerNode(5, 0, 1e9, 64) - id2 := nodestore.RandomNodeID() - node1.Children[0] = &nodestore.Child{ - ID: id2, - Version: 2, - } - require.NoError(t, ns.StageWithID(id1, node1)) - rootID, err := ns.Flush(ctx, 2, id1) - require.NoError(t, err) - _, _, err = tree.Insert(ctx, ns, rootID, 10, 3, []byte{}, &nodestore.Statistics{}) - require.ErrorIs(t, err, nodestore.NodeNotFoundError{NodeID: id2}) + + t.Run("root is not an inner node", func(t *testing.T) { + root := nodestore.NewInnerNode(1, 0, 4096, 64) + tw := tree.NewMemTree(nodestore.RandomNodeID(), root) + require.NoError(t, tree.Insert(ctx, tw, 0, 1000*1e9, []byte{0x01}, &nodestore.Statistics{})) + + leaf := nodestore.NewLeafNode([]byte{0x01}) + leafid := nodestore.RandomNodeID() + require.NoError(t, tw.Put(ctx, leafid, leaf)) + tw.SetRoot(leafid) + + err := tree.Insert(ctx, tw, 0, 64*1e9, []byte{0x01}, &nodestore.Statistics{}) + require.ErrorIs(t, err, tree.UnexpectedNodeError{}) }) } -func TestStatRange(t *testing.T) { +func TestMergeErrors(t *testing.T) { ctx := context.Background() - cases := []struct { - assertion string - messageTimes []uint64 - start uint64 - end uint64 - granularity uint64 - expected []tree.StatRange - }{ - { - "empty tree", - []uint64{}, - 0, - 100e9, - 1e9, - []tree.StatRange{}, - }, - { - "single insert", - []uint64{100}, - 0, - 100e9, - 600e9, - []tree.StatRange{ - { - Start: 64e9, - End: 128e9, - Statistics: &nodestore.Statistics{MessageCount: 1}, - }, - }, - }, - { - "inserts spanning buckets", - []uint64{100, 4097}, - 0, - 5000e9, - 600e9, - []tree.StatRange{ - {Start: 4096000000000, End: 4160000000000, Statistics: &nodestore.Statistics{MessageCount: 1}}, - {Start: 64000000000, End: 128000000000, Statistics: &nodestore.Statistics{MessageCount: 1}}, - }, - }, - { - "inserts spanning disjoint buckets", - []uint64{100, 262143}, - 0, - math.MaxUint64, - 600e9, - []tree.StatRange{ - {Start: 262080000000000, End: 262144000000000, Statistics: &nodestore.Statistics{MessageCount: 1}}, - {Start: 64000000000, End: 128000000000, Statistics: &nodestore.Statistics{MessageCount: 1}}, - }, - }, - } - for _, c := range cases { - t.Run(c.assertion, func(t *testing.T) { - ns := nodestore.MockNodestore(ctx, t) - rootID, err := ns.NewRoot(ctx, 0, util.Pow(uint64(64), 3), 64, 64) - require.NoError(t, err) - version := uint64(0) - roots := make([]nodestore.NodeID, len(c.messageTimes)) - for i, time := range c.messageTimes { - buf := &bytes.Buffer{} - mcap.WriteFile(t, buf, []uint64{time}) - version++ - rootID, path, err := tree.Insert( - ctx, ns, rootID, version, time*1e9, buf.Bytes(), - &nodestore.Statistics{ - MessageCount: 1, - }, - ) - require.NoError(t, err) - require.NoError(t, ns.FlushStagingToWAL(ctx, "producer", "topic", version, path)) - roots[i] = rootID - } - rootID, err = ns.MergeWALToStorage(ctx, rootID, version, roots) - require.NoError(t, err) + t.Run("merging trees of different height", func(t *testing.T) { + root := nodestore.NewInnerNode(1, 0, 4096, 64) + tw := tree.NewMemTree(nodestore.RandomNodeID(), root) + require.NoError(t, tree.Insert(ctx, tw, 0, 1000*1e9, []byte{0x01}, &nodestore.Statistics{})) - statrange, err := tree.GetStatRange(ctx, ns, rootID, c.start, c.end, c.granularity) - require.NoError(t, err) - require.Equal(t, c.expected, statrange) - }) - } + root2 := nodestore.NewInnerNode(2, 0, 4096, 64) + tw2 := tree.NewMemTree(nodestore.RandomNodeID(), root2) + require.NoError(t, tree.Insert(ctx, tw2, 0, 64*1e9, []byte{0x01}, &nodestore.Statistics{})) + + root3 := nodestore.NewInnerNode(2, 0, 4096, 64) + tw3 := tree.NewMemTree(nodestore.RandomNodeID(), root3) + + err := tree.Merge(ctx, tw3, tw, tw2) + require.ErrorIs(t, err, tree.MismatchedHeightsError{}) + }) + + t.Run("merging tree with wrong height", func(t *testing.T) { + // build a corrupt tree + root := nodestore.NewInnerNode(2, 0, 64*64*64, 64) + tw := tree.NewMemTree(nodestore.RandomNodeID(), root) + + // child + childnode := nodestore.NewInnerNode(1, 0, 64*64, 64) + childid := nodestore.RandomNodeID() + child := nodestore.Child{ + ID: childid, + Version: 0, + Statistics: &nodestore.Statistics{}, + } + require.NoError(t, tw.Put(ctx, childid, childnode)) + root.Children[0] = &child + + // grandchild should be a leaf, based on tree dimensions + gchildnode := nodestore.NewInnerNode(0, 0, 64*64, 64) + gchildid := nodestore.RandomNodeID() + gchild := nodestore.Child{ + ID: gchildid, + Version: 0, + Statistics: &nodestore.Statistics{}, + } + require.NoError(t, tw.Put(ctx, gchildid, gchildnode)) + childnode.Children[0] = &gchild + + // merge with a correct tree + root2 := nodestore.NewInnerNode(2, 0, 64*64*64, 64) + tw2 := tree.NewMemTree(nodestore.RandomNodeID(), root2) + require.NoError(t, tree.Insert(ctx, tw2, 0, 63*1e9, []byte{0x01}, &nodestore.Statistics{})) + + root3 := nodestore.NewInnerNode(2, 0, 64*64*64, 64) + tw3 := tree.NewMemTree(nodestore.RandomNodeID(), root3) + + err := tree.Merge(ctx, tw3, tw, tw2) + require.Error(t, err) + }) + + t.Run("root is not an inner node", func(t *testing.T) { + root := nodestore.NewInnerNode(1, 0, 4096, 64) + tw := tree.NewMemTree(nodestore.RandomNodeID(), root) + require.NoError(t, tree.Insert(ctx, tw, 0, 1000*1e9, []byte{0x01}, &nodestore.Statistics{})) + + leaf := nodestore.NewLeafNode([]byte{0x01}) + leafid := nodestore.RandomNodeID() + require.NoError(t, tw.Put(ctx, leafid, leaf)) + tw.SetRoot(leafid) + + root2 := nodestore.NewInnerNode(2, 0, 4096, 64) + tw2 := tree.NewMemTree(nodestore.RandomNodeID(), root2) + require.NoError(t, tree.Insert(ctx, tw2, 0, 64*1e9, []byte{0x01}, &nodestore.Statistics{})) + + root3 := nodestore.NewInnerNode(2, 0, 4096, 64) + tw3 := tree.NewMemTree(nodestore.RandomNodeID(), root3) + + err := tree.Merge(ctx, tw3, tw, tw2) + require.ErrorIs(t, err, tree.UnexpectedNodeError{}) + }) + + t.Run("root does not exist", func(t *testing.T) { + root := nodestore.NewInnerNode(1, 0, 4096, 64) + tw := tree.NewMemTree(nodestore.RandomNodeID(), root) + require.NoError(t, tree.Insert(ctx, tw, 0, 1000*1e9, []byte{0x01}, &nodestore.Statistics{})) + tw.SetRoot(nodestore.RandomNodeID()) + + root2 := nodestore.NewInnerNode(2, 0, 4096, 64) + tw2 := tree.NewMemTree(nodestore.RandomNodeID(), root2) + + err := tree.Merge(ctx, tw2, tw) + require.ErrorIs(t, err, nodestore.NodeNotFoundError{}) + }) } func TestTreeInsert(t *testing.T) { @@ -135,69 +146,45 @@ func TestTreeInsert(t *testing.T) { cases := []struct { assertion string height uint8 - times []uint64 + times [][]uint64 repr string }{ { "empty tree", 1, - []uint64{}, - "[0-4096:0]", + [][]uint64{}, + "[0-4096]", }, { "single insert", 1, - []uint64{10}, - "[0-4096:1 [0-64:1 (count=1) [leaf 1 msg]]]", + [][]uint64{{10}}, + "[0-4096 [0-64:1 (count=1) [leaf 1 msg]]]", }, { - "two inserts same bucket get merged", - 1, - []uint64{10, 20}, - "[0-4096:2 [0-64:2 (count=2) [leaf 2 msgs]]]", + "height 2", + 2, + [][]uint64{{10}}, + "[0-262144 [0-4096:1 (count=1) [0-64:1 (count=1) [leaf 1 msg]]]]", }, { - "inserts in different bucket, simulate single inserts", + "two inserts into same bucket are merged", 1, - []uint64{10, 20, 128, 256}, - `[0-4096:4 [0-64:4 (count=2) [leaf 2 msgs]] [128-192:4 (count=1) - [leaf 1 msg]] [256-320:4 (count=1) [leaf 1 msg]]]`, + [][]uint64{{10}, {20}}, + "[0-4096 [0-64:2 (count=2) [leaf 2 msgs]]]", }, { - "height 2", - 2, - []uint64{10, 20, 4097}, - `[0-262144:3 [0-4096:3 (count=2) [0-64:3 (count=2) [leaf 2 msgs]]] - [4096-8192:3 (count=1) [4096-4160:3 (count=1) [leaf 1 msg]]]]`, + "inserts into different buckets", + 1, + [][]uint64{{10}, {100}, {256}, {1000}}, + `[0-4096 [0-64:1 (count=1) [leaf 1 msg]] [64-128:2 (count=1) [leaf 1 msg]] + [256-320:3 (count=1) [leaf 1 msg]] [960-1024:4 (count=1) [leaf 1 msg]]]`, }, } for _, c := range cases { t.Run(c.assertion, func(t *testing.T) { - ns := nodestore.MockNodestore(ctx, t) - rootID, err := ns.NewRoot(ctx, 0, util.Pow(uint64(64), int(c.height)+1), 64, 64) - require.NoError(t, err) - version := uint64(0) - roots := make([]nodestore.NodeID, len(c.times)) - for i, time := range c.times { - buf := &bytes.Buffer{} - mcap.WriteFile(t, buf, []uint64{time}) - version++ - rootID, path, err := tree.Insert( - ctx, ns, rootID, version, time*1e9, buf.Bytes(), - &nodestore.Statistics{ - MessageCount: 1, - }, - ) - require.NoError(t, err) - - require.NoError(t, ns.FlushStagingToWAL(ctx, "producer", "topic", version, path)) - roots[i] = rootID - } - - rootID, err = ns.MergeWALToStorage(ctx, rootID, version, roots) - require.NoError(t, err) - - repr, err := ns.Print(ctx, rootID, version, nil) + output := tree.MergeInserts(ctx, t, 0, util.Pow(uint64(64), int(c.height+1)), c.height, 64, c.times) + repr, err := tree.Print(ctx, output) require.NoError(t, err) assertEqualTrees(t, c.repr, repr) }) @@ -216,86 +203,68 @@ func removeSpace(s string) string { return s } -func TestRootConstruction(t *testing.T) { +func TestStatRange(t *testing.T) { ctx := context.Background() cases := []struct { - assertion string - start string - end string - width int - bfactor int - time string - repr string + assertion string + messages [][]uint64 + start uint64 + end uint64 + granularity uint64 + expected []tree.StatRange }{ { - "epoch to 2050 with 60 second buckets", - "1970-01-01", - "2030-01-01", - 60, - 64, - "1970-01-01", - `[0-64424509440:1 [0-1006632960:1 (count=1) [0-15728640:1 (count=1) - [0-245760:1 (count=1) [0-3840:1 (count=1) [0-60:1 (count=1) [leaf 1 msg]]]]]]]`, + "empty tree", + [][]uint64{}, + 0, + 100e9, + 1e9, + []tree.StatRange{}, }, { - "single year 60 second buckets", - "1970-01-01", - "1971-01-01", - 60, - 64, - "1970-01-02", - `[0-1006632960:1 [0-15728640:1 (count=1) [0-245760:1 (count=1) [84480-88320:1 (count=1) - [86400-86460:1 (count=1) [leaf 1 msg]]]]]]`, + "single insert", + [][]uint64{{100}}, + 0, + 100e9, + 600e9, + []tree.StatRange{ + { + Start: 64e9, + End: 128e9, + Statistics: &nodestore.Statistics{MessageCount: 1}, + }, + }, }, { - "single year 60 second buckets bfactor 20", - "1970-01-01", - "1971-01-01", - 60, - 20, - "1970-01-02", - `[0-192000000:1 [0-9600000:1 (count=1) [0-480000:1 (count=1) [72000-96000:1 (count=1) - [86400-87600:1 (count=1) [86400-86460:1 (count=1) [leaf 1 msg]]]]]]]`, + "inserts spanning buckets", + [][]uint64{{100}, {4097}}, + 0, + 5000e9, + 600e9, + []tree.StatRange{ + {Start: 4096000000000, End: 4160000000000, Statistics: &nodestore.Statistics{MessageCount: 1}}, + {Start: 64000000000, End: 128000000000, Statistics: &nodestore.Statistics{MessageCount: 1}}, + }, }, { - "single year full day buckets bfactor 365", - "1970-01-01", - "1971-01-01", - 60 * 60 * 24, - 365, - "1970-01-02", - `[0-31536000:1 [86400-172800:1 (count=1) [leaf 1 msg]]]`, + "inserts spanning disjoint buckets", + [][]uint64{{100}, {262143}}, + 0, + math.MaxUint64, + 600e9, + []tree.StatRange{ + {Start: 262080000000000, End: 262144000000000, Statistics: &nodestore.Statistics{MessageCount: 1}}, + {Start: 64000000000, End: 128000000000, Statistics: &nodestore.Statistics{MessageCount: 1}}, + }, }, } - for _, c := range cases { t.Run(c.assertion, func(t *testing.T) { - ns := nodestore.MockNodestore(ctx, t) - rootID, err := ns.NewRoot( - ctx, - util.DateSeconds(c.start), - util.DateSeconds(c.end), - c.width, - c.bfactor, - ) - require.NoError(t, err) - - version := uint64(1) - - buf := &bytes.Buffer{} - mcap.WriteFile(t, buf, []uint64{10}) - timestamp := util.DateSeconds(c.time) * 1e9 - stats := &nodestore.Statistics{ - MessageCount: 1, - } - _, path, err := tree.Insert(ctx, ns, rootID, version, timestamp, buf.Bytes(), stats) - require.NoError(t, err) - rootID, err = ns.Flush(ctx, version, path...) - require.NoError(t, err) + tw := tree.MergeInserts(ctx, t, 0, 64*64*64, 2, 64, c.messages) - repr, err := ns.Print(ctx, rootID, version, nil) + statrange, err := tree.GetStatRange(ctx, tw, c.start, c.end, c.granularity) require.NoError(t, err) - assertEqualTrees(t, c.repr, repr) + require.Equal(t, c.expected, statrange) }) } } diff --git a/tree/treereader.go b/tree/treereader.go new file mode 100644 index 0000000..e5ae1bd --- /dev/null +++ b/tree/treereader.go @@ -0,0 +1,78 @@ +package tree + +import ( + "context" + "fmt" + "io" + "strings" + + "github.com/wkalt/dp3/nodestore" +) + +/* +The tree reader interface is used to traverse trees stored in various ways - +either from memory, from WAL, or from the nodestore. +*/ + +//////////////////////////////////////////////////////////////////////////////// + +type TreeReader interface { + Root() nodestore.NodeID + Get(ctx context.Context, id nodestore.NodeID) (nodestore.Node, error) + // todo: remove this in favor of a closure on leaf nodes. + GetLeafData(ctx context.Context, id nodestore.NodeID) (io.ReadSeekCloser, error) +} + +// Print a tree reader in human-readable format. +func Print(ctx context.Context, reader TreeReader) (string, error) { + node, err := reader.Get(ctx, reader.Root()) + if err != nil { + return "", fmt.Errorf("failed to get root node: %w", err) + } + return printInnerNode(ctx, reader, node.(*nodestore.InnerNode), 0, nil) +} + +func printInnerNode( + ctx context.Context, + reader TreeReader, + node *nodestore.InnerNode, + version uint64, + stats *nodestore.Statistics, +) (string, error) { + sb := &strings.Builder{} + sb.WriteString(fmt.Sprintf("[%d-%d", node.Start, node.End)) + if version > 0 { + sb.WriteString(fmt.Sprintf(":%d %s", version, stats)) + } + for i, child := range node.Children { + if child == nil { + continue + } + sb.WriteString(" ") + childNode, err := reader.Get(ctx, child.ID) + if err != nil { + return "", fmt.Errorf("failed to get node %d: %w", child.ID, err) + } + var body string + switch cnode := childNode.(type) { + case *nodestore.InnerNode: + body, err = printInnerNode(ctx, reader, cnode, child.Version, child.Statistics) + if err != nil { + return "", err + } + sb.WriteString(body) + case *nodestore.LeafNode: + width := (node.End - node.Start) / uint64(len(node.Children)) + body = cnode.String() + sb.WriteString(fmt.Sprintf("[%d-%d:%d %s %s]", + node.Start+uint64(i)*width, + node.Start+uint64(i+1)*width, + child.Version, + child.Statistics, + body, + )) + } + } + sb.WriteString("]") + return sb.String(), nil +} diff --git a/tree/treewriter.go b/tree/treewriter.go new file mode 100644 index 0000000..f175d61 --- /dev/null +++ b/tree/treewriter.go @@ -0,0 +1,18 @@ +package tree + +import ( + "context" + + "github.com/wkalt/dp3/nodestore" +) + +/* +The TreeWriter interface is used to write nodes to a tree. It extends the +TreeReader interface with methods to write nodes and set the tree root. +*/ + +type TreeWriter interface { + Put(ctx context.Context, id nodestore.NodeID, node nodestore.Node) error + SetRoot(id nodestore.NodeID) + TreeReader +} diff --git a/treemgr/options.go b/treemgr/options.go new file mode 100644 index 0000000..aeddfb5 --- /dev/null +++ b/treemgr/options.go @@ -0,0 +1,47 @@ +package treemgr + +import ( + "github.com/wkalt/dp3/wal" +) + +type config struct { + waldir string + walopts []wal.Option + walBufferSize int + syncWorkers int +} + +// Option is an option for the tree manager. +type Option func(*config) + +// WithWALOpts passes the supplied set of options to the walmgr constructor. +func WithWALOpts(opts ...wal.Option) Option { + return func(c *config) { + c.walopts = append(c.walopts, opts...) + } +} + +// WithWALDir sets the directory for the WAL. +func WithWALDir(dir string) Option { + return func(c *config) { + c.waldir = dir + } +} + +// WithWALBufferSize sets the buffer size of the notification channel between +// the WAL and the tree manager's batch sync workers. A zero value will result +// in an unbuffered channel. +func WithWALBufferSize(size int) Option { + return func(c *config) { + c.walBufferSize = size + } +} + +// WithSyncWorkers sets the tree manager's sync worker count. Sync workers are +// responsible for merging partial trees from the WAL into storage. A given +// producer/topic will always be routed to a single sync worker. +func WithSyncWorkers(n int) Option { + return func(c *config) { + c.syncWorkers = n + } +} diff --git a/treemgr/treemgr.go b/treemgr/treemgr.go index 61318e6..7957fab 100644 --- a/treemgr/treemgr.go +++ b/treemgr/treemgr.go @@ -5,9 +5,9 @@ import ( "context" "errors" "fmt" + "hash/maphash" "io" - "slices" - "time" + "sort" fmcap "github.com/foxglove/mcap/go/mcap" "github.com/wkalt/dp3/mcap" @@ -17,8 +17,8 @@ import ( "github.com/wkalt/dp3/util" "github.com/wkalt/dp3/util/log" "github.com/wkalt/dp3/versionstore" + "github.com/wkalt/dp3/wal" "golang.org/x/exp/maps" - "golang.org/x/sync/errgroup" ) /* @@ -37,26 +37,80 @@ type TreeManager struct { ns *nodestore.Nodestore vs versionstore.Versionstore rootmap rootmap.Rootmap + merges <-chan *wal.Batch - batchsize int syncWorkers int + + wal *wal.WALManager +} + +// NewRoot creates a new root for the provided producer and topic. +func (tm *TreeManager) NewRoot(ctx context.Context, producer string, topic string) error { + rootID, version, err := tm.newRoot( + ctx, + util.DateSeconds("1970-01-01"), + util.DateSeconds("2038-01-19"), + 60, + 64, + ) + if err != nil { + return fmt.Errorf("failed to create new root: %w", err) + } + if err := tm.rootmap.Put(ctx, producer, topic, version, rootID); err != nil { + return fmt.Errorf("failed to put root: %w", err) + } + return nil } // NewTreeManager returns a new TreeManager. func NewTreeManager( + ctx context.Context, ns *nodestore.Nodestore, vs versionstore.Versionstore, rm rootmap.Rootmap, - batchsize int, - syncWorkers int, -) *TreeManager { - return &TreeManager{ + opts ...Option, +) (*TreeManager, error) { + conf := config{ + walBufferSize: 0, + syncWorkers: 1, + waldir: "", + } + for _, opt := range opts { + opt(&conf) + } + if conf.waldir == "" { + return nil, errors.New("wal directory not specified") + } + merges := make(chan *wal.Batch, conf.walBufferSize) + wmgr, err := wal.NewWALManager(ctx, conf.waldir, merges, conf.walopts...) + if err != nil { + return nil, fmt.Errorf("failed to create WAL manager: %w", err) + } + + tm := &TreeManager{ ns: ns, vs: vs, + wal: wmgr, + merges: merges, rootmap: rm, - batchsize: batchsize, - syncWorkers: syncWorkers, + syncWorkers: conf.syncWorkers, } + + // start listeners on the merges channel that will multiplex merge requests + // over a pool of workers, arranged so that each producer/topic pair is + // routed to a single worker only. + if conf.syncWorkers > 0 { + go tm.spawnWALConsumers(ctx) + } + + // Recover the WAL. This will feed the merges channel with any uncommitted + // work. + if err := wmgr.Recover(ctx); err != nil { + return nil, fmt.Errorf("failed to recover: %w", err) + } + + // ready for requests + return tm, nil } // Receive an MCAP data stream on behalf of a particular producer. The data is @@ -95,7 +149,7 @@ func (tm *TreeManager) Receive(ctx context.Context, producerID string, data io.R if _, _, err := tm.rootmap.GetLatest(ctx, producerID, channel.Topic); err != nil { switch { case errors.Is(err, rootmap.StreamNotFoundError{}): - if err := tm.newRoot(ctx, producerID, channel.Topic); err != nil { + if err := tm.NewRoot(ctx, producerID, channel.Topic); err != nil { return fmt.Errorf("failed to create new root: %w", err) } default: @@ -112,7 +166,11 @@ func (tm *TreeManager) Receive(ctx context.Context, producerID string, data io.R return fmt.Errorf("failed to write message: %w", err) } } - for _, writer := range writers { + + keys := maps.Keys(writers) + sort.Strings(keys) + for _, k := range keys { + writer := writers[k] if err := writer.Close(ctx); err != nil { return fmt.Errorf("failed to close writer: %w", err) } @@ -140,13 +198,222 @@ func (tm *TreeManager) GetStatistics( if err != nil { return nil, fmt.Errorf("failed to get latest root: %w", err) } - ranges, err := tree.GetStatRange(ctx, tm.ns, rootID, start, end, granularity) + tr := tree.NewBYOTreeReader(rootID, tm.ns.Get) + ranges, err := tree.GetStatRange(ctx, tr, start, end, granularity) if err != nil { return nil, fmt.Errorf("failed to get stat range: %w", err) } return ranges, nil } +// GetMessagesLatest returns a stream of messages for the given time range. +func (tm *TreeManager) GetMessagesLatest( + ctx context.Context, + w io.Writer, + start, end uint64, + producerID string, + topics []string, +) error { + roots := make([]nodestore.NodeID, 0, len(topics)) + for _, topic := range topics { + root, _, err := tm.rootmap.GetLatest(ctx, producerID, topic) + if err != nil { + if !errors.Is(err, rootmap.StreamNotFoundError{}) { + return fmt.Errorf("failed to get latest root for %s/%s: %w", producerID, topic, err) + } + log.Debugf(ctx, "no root found for %s/%s - omitting from output", producerID, topic) + continue + } + roots = append(roots, root) + } + if err := tm.getMessages(ctx, w, start, end, roots); err != nil { + return fmt.Errorf("failed to get messages for %s: %w", producerID, err) + } + return nil +} + +// GetStatisticsLatest returns a summary of statistics for the given time range. +func (tm *TreeManager) GetStatisticsLatest( + ctx context.Context, + start, end uint64, + producerID string, + topic string, + granularity uint64, +) ([]tree.StatRange, error) { + rootID, _, err := tm.rootmap.GetLatest(ctx, producerID, topic) + if err != nil { + return nil, fmt.Errorf("failed to get latest root: %w", err) + } + tr := tree.NewBYOTreeReader(rootID, tm.ns.Get) + ranges, err := tree.GetStatRange(ctx, tr, start, end, granularity) + if err != nil { + return nil, fmt.Errorf("failed to get stat range: %w", err) + } + return ranges, nil +} + +// ForceFlush forces a synchronous flush of WAL data to the tree. Used in tests only. +func (tm *TreeManager) ForceFlush(ctx context.Context) error { + c, err := tm.wal.ForceMerge(ctx) + if err != nil { + return fmt.Errorf("failed to force merge: %w", err) + } + err = tm.drainWAL(ctx, c) + if err != nil { + return fmt.Errorf("failed to drain WAL: %w", err) + } + return nil +} + +// PrintStream returns a string representation of the tree for the given stream. +func (tm *TreeManager) PrintStream(ctx context.Context, producerID string, topic string) string { + root, _, err := tm.rootmap.GetLatest(ctx, producerID, topic) + if err != nil { + return fmt.Sprintf("failed to get latest root: %v", err) + } + tr := tree.NewBYOTreeReader(root, tm.ns.Get) + s, err := tree.Print(ctx, tr) + if err != nil { + return fmt.Sprintf("failed to print tree: %v", err) + } + return s +} + +func (tm *TreeManager) newRoot( + ctx context.Context, + start uint64, + end uint64, + leafWidthSecs int, + bfactor int, +) (nodestore.NodeID, uint64, error) { + var height int + span := end - start + coverage := leafWidthSecs + for uint64(coverage) < span { + coverage *= bfactor + height++ + } + root := nodestore.NewInnerNode(uint8(height), start, start+uint64(coverage), bfactor) + data := root.ToBytes() + version, err := tm.vs.Next(ctx) + if err != nil { + return nodestore.NodeID{}, 0, fmt.Errorf("failed to get next version: %w", err) + } + if err := tm.ns.Put(ctx, version, data); err != nil { + return nodestore.NodeID{}, 0, fmt.Errorf("failed to put root: %w", err) + } + nodeID := nodestore.NewNodeID(version, 0, len(data)) + return nodeID, version, nil +} + +func (tm *TreeManager) mergeBatch(ctx context.Context, batch *wal.Batch) error { + trees := make([]tree.TreeReader, 0, len(batch.Addrs)+1) + for _, addr := range batch.Addrs { + page, err := tm.wal.Get(addr) + if err != nil { + return fmt.Errorf("failed to get page %s: %w", addr, err) + } + var mt tree.MemTree + if err := mt.FromBytes(ctx, page); err != nil { + return fmt.Errorf("failed to deserialize tree: %w", err) + } + trees = append(trees, &mt) + } + version, err := tm.vs.Next(ctx) + if err != nil { + return fmt.Errorf("failed to get next version: %w", err) + } + + // If there is an existing root, then we need to add that data to the merge. + existingRootID, _, err := tm.rootmap.GetLatest(ctx, batch.ProducerID, batch.Topic) + if err != nil && !errors.Is(err, rootmap.StreamNotFoundError{}) { + return fmt.Errorf("failed to get root: %w", err) + } + if err == nil { + treedata := tree.NewBYOTreeReader(existingRootID, tm.ns.Get) + trees = append(trees, treedata) + } + + var merged tree.MemTree + if err := tree.Merge(ctx, &merged, trees...); err != nil { + return fmt.Errorf("failed to merge partial trees: %w", err) + } + data, err := merged.ToBytes(ctx, version) + if err != nil { + return fmt.Errorf("failed to serialize partial tree: %w", err) + } + rootID := data[len(data)-16:] + if err := tm.ns.Put(ctx, version, data[:len(data)-16]); err != nil { + return fmt.Errorf("failed to put tree: %w", err) + } + if err := tm.rootmap.Put(ctx, batch.ProducerID, batch.Topic, version, nodestore.NodeID(rootID)); err != nil { + return fmt.Errorf("failed to put root: %w", err) + } + return nil +} + +func (tm *TreeManager) drainWAL(ctx context.Context, n int) error { + c := 0 + for c < n { + select { + case <-ctx.Done(): + return nil + case batch, ok := <-tm.merges: + if !ok { + return nil + } + if err := tm.mergeBatch(ctx, batch); err != nil { + return fmt.Errorf("failed to merge batch %s: %w", batch.ID, err) + } + if err := batch.Finish(); err != nil { + return fmt.Errorf("failed to commit batch success: %w", err) + } + } + c++ + } + return nil +} + +func (tm *TreeManager) spawnWALConsumers(ctx context.Context) { + mergechans := make([]chan *wal.Batch, tm.syncWorkers) + for i := 0; i < tm.syncWorkers; i++ { + mergechans[i] = make(chan *wal.Batch) + } + seed := maphash.MakeSeed() + go func() { + for { + select { + case <-ctx.Done(): + return + case batch := <-tm.merges: + hash := maphash.Hash{} + hash.SetSeed(seed) + _, _ = hash.WriteString(batch.ProducerID + batch.Topic) + bucket := hash.Sum64() % uint64(tm.syncWorkers) + mergechans[bucket] <- batch + } + } + }() + + for i := 0; i < tm.syncWorkers; i++ { + mergechan := mergechans[i] + go func() { + for batch := range mergechan { + if err := tm.mergeBatch(ctx, batch); err != nil { + // todo - retry strategy? + log.Errorf(ctx, "failed to merge batch %s: %v", batch.ID, err) + } else { + if err := batch.Finish(); err != nil { + // todo retry strategy + log.Errorf(ctx, "failed to commit batch success to wal: %s", err) + } + } + } + }() + } + log.Infof(ctx, "spawned %d WAL consumers", tm.syncWorkers) +} + func closeAll(ctx context.Context, closers ...*tree.Iterator) { var errs []error for _, closer := range closers { @@ -169,7 +436,8 @@ func (tm *TreeManager) getMessages( ) error { iterators := make([]*tree.Iterator, 0, len(roots)) for _, root := range roots { - it, err := tree.NewTreeIterator(ctx, tm.ns, root, start, end) + tr := tree.NewBYOTreeReader(root, tm.ns.Get) + it, err := tree.NewTreeIterator(ctx, tr, start, end) if err != nil { return fmt.Errorf("failed to create iterator for %s: %w", root, err) } @@ -177,10 +445,9 @@ func (tm *TreeManager) getMessages( } defer closeAll(ctx, iterators...) + // pop one message from each iterator and push it onto the priority queue pq := util.NewPriorityQueue[record, uint64]() heap.Init(pq) - - // pop one message from each iterator and push it onto the priority queue for i, it := range iterators { if it.More() { schema, channel, message, err := it.Next(ctx) @@ -224,51 +491,6 @@ func (tm *TreeManager) getMessages( return nil } -// GetMessagesLatest returns a stream of messages for the given time range. -func (tm *TreeManager) GetMessagesLatest( - ctx context.Context, - w io.Writer, - start, end uint64, - producerID string, - topics []string, -) error { - roots := make([]nodestore.NodeID, 0, len(topics)) - for _, topic := range topics { - root, _, err := tm.rootmap.GetLatest(ctx, producerID, topic) - if err != nil { - if !errors.Is(err, rootmap.StreamNotFoundError{}) { - return fmt.Errorf("failed to get latest root for %s/%s: %w", producerID, topic, err) - } - log.Debugf(ctx, "no root found for %s/%s - omitting from output", producerID, topic) - continue - } - roots = append(roots, root) - } - if err := tm.getMessages(ctx, w, start, end, roots); err != nil { - return fmt.Errorf("failed to get messages for %s: %w", producerID, err) - } - return nil -} - -// GetStatisticsLatest returns a summary of statistics for the given time range. -func (tm *TreeManager) GetStatisticsLatest( - ctx context.Context, - start, end uint64, - producerID string, - topic string, - granularity uint64, -) ([]tree.StatRange, error) { - rootID, _, err := tm.rootmap.GetLatest(ctx, producerID, topic) - if err != nil { - return nil, fmt.Errorf("failed to get latest root: %w", err) - } - ranges, err := tree.GetStatRange(ctx, tm.ns, rootID, start, end, granularity) - if err != nil { - return nil, fmt.Errorf("failed to get stat range: %w", err) - } - return ranges, nil -} - // insert data into the tree and flush it to the WAL. func (tm *TreeManager) insert( ctx context.Context, @@ -286,105 +508,21 @@ func (tm *TreeManager) insert( if err != nil { return fmt.Errorf("failed to get next version: %w", err) } - _, nodeIDs, err := tree.Insert(ctx, tm.ns, rootID, version, time, data, statistics) + currentRoot, err := tm.ns.Get(ctx, rootID) if err != nil { - return fmt.Errorf("insertion failure: %w", err) - } - if err := tm.ns.FlushStagingToWAL(ctx, producerID, topic, version, nodeIDs); err != nil { - return fmt.Errorf("failed to flush to WAL: %w", err) + return fmt.Errorf("failed to get root: %w", err) } - return nil -} - -// StartWALSyncLoop starts a loop that periodically syncs the WAL to the tree. -func (tm *TreeManager) StartWALSyncLoop(ctx context.Context) { - ticker := time.NewTicker(120 * time.Second) - for { - select { - case <-ticker.C: - log.Infof(ctx, "syncing WAL") - if err := tm.SyncWAL(ctx); err != nil { - log.Errorw(ctx, "failed to sync WAL", "error", err.Error()) - } - case <-ctx.Done(): - return - } - } -} - -// SyncWAL syncs the WAL to persistent storage. -func (tm *TreeManager) SyncWAL(ctx context.Context) error { - listings, err := tm.ns.ListWAL(ctx) - if err != nil { - return fmt.Errorf("failed to list WAL: %w", err) - } - grp := errgroup.Group{} - log.Infof(ctx, "Syncing %d WAL listings", len(listings)) - grp.SetLimit(tm.syncWorkers) - for _, listing := range listings { - grp.Go(func() error { - return tm.syncWALListing(ctx, listing) - }) - } - if err := grp.Wait(); err != nil { - return fmt.Errorf("failed to sync WAL: %w", err) + mt := tree.NewMemTree(rootID, currentRoot.(*nodestore.InnerNode)) + if err := tree.Insert(ctx, mt, version, time, data, statistics); err != nil { + return fmt.Errorf("insertion failure: %w", err) } - log.Infof(ctx, "WAL sync complete") - return nil -} - -func (tm *TreeManager) syncWALListing( - ctx context.Context, - listing nodestore.WALListing, -) error { - rootID, _, err := tm.rootmap.GetLatest(ctx, listing.ProducerID, listing.Topic) + serialized, err := mt.ToBytes(ctx, 0) // zero is a temporary oid if err != nil { - return fmt.Errorf("failed to get latest root: %w", err) + return fmt.Errorf("failed to serialize tree: %w", err) } - roots := []nodestore.NodeID{} - version, err := tm.vs.Next(ctx) + _, err = tm.wal.Insert(producerID, topic, serialized) if err != nil { - return fmt.Errorf("failed to get next version: %w", err) - } - var newRootID nodestore.NodeID - versions := maps.Keys(listing.Versions) - slices.Sort(versions) - if len(versions) == 1 { - log.Debugw(ctx, "flushing WAL entries", - "producerID", listing.ProducerID, - "topic", listing.Topic, - "count", len(listing.Versions), - "version", version, - ) - value := maps.Values(listing.Versions)[0] - newRootID, err = tm.ns.FlushWALPath(ctx, version, value) - if err != nil { - return fmt.Errorf("failed to flush wal path: %w", err) - } - } else { - log.Debugw(ctx, "merging WAL entries", - "producerID", listing.ProducerID, - "topic", listing.Topic, - "count", len(listing.Versions)) - for _, version := range versions { - nodeIDs := listing.Versions[version] - roots = append(roots, nodeIDs[0]) - } - newRootID, err = tm.ns.MergeWALToStorage(ctx, rootID, version, roots) - if err != nil { - return fmt.Errorf("failed to merge WAL into tree: %w", err) - } - } - for _, version := range versions { - nodeIDs := listing.Versions[version] - for _, nodeID := range nodeIDs { - if err = tm.ns.WALDelete(ctx, nodeID); err != nil { // todo transaction - return fmt.Errorf("failed to delete node %s from WAL: %w", nodeID, err) - } - } - } - if err := tm.rootmap.Put(ctx, listing.ProducerID, listing.Topic, version, newRootID); err != nil { - return fmt.Errorf("failed to update rootmap: %w", err) + return fmt.Errorf("failed to insert into WAL: %w", err) } return nil } @@ -434,31 +572,3 @@ func (tm *TreeManager) dimensions( end: inner.End, }, nil } - -func (tm *TreeManager) newRoot(ctx context.Context, producerID string, topic string) error { - rootID, err := tm.ns.NewRoot(ctx, util.DateSeconds("1970-01-01"), util.DateSeconds("2038-01-19"), 60, 64) - if err != nil { - return fmt.Errorf("failed to create new root: %w", err) - } - version, err := tm.vs.Next(ctx) - if err != nil { - return fmt.Errorf("failed to get next version: %w", err) - } - if err := tm.rootmap.Put(ctx, producerID, topic, version, rootID); err != nil { - return fmt.Errorf("failed to update rootmap: %w", err) - } - return nil -} - -// PrintStream returns a string representation of the tree for the given stream. -func (tm *TreeManager) PrintStream(ctx context.Context, producerID string, topic string) string { - root, version, err := tm.rootmap.GetLatest(ctx, producerID, topic) - if err != nil { - return fmt.Sprintf("failed to get latest root: %v", err) - } - s, err := tm.ns.Print(ctx, root, version, nil) - if err != nil { - return fmt.Sprintf("failed to print tree: %v", err) - } - return s -} diff --git a/treemgr/treemgr_test.go b/treemgr/treemgr_test.go index b3a121f..96e6190 100644 --- a/treemgr/treemgr_test.go +++ b/treemgr/treemgr_test.go @@ -3,7 +3,6 @@ package treemgr_test import ( "bytes" "context" - "database/sql" "errors" "fmt" "io" @@ -224,9 +223,10 @@ func TestGetStatisticsLatest(t *testing.T) { t.Run(c.assertion, func(t *testing.T) { buf := &bytes.Buffer{} mcap.WriteFile(t, buf, c.input...) - tmgr := testTreeManager(ctx, t) + tmgr, finish := testTreeManager(ctx, t) + defer finish() require.NoError(t, tmgr.Receive(ctx, "my-device", buf)) - require.NoError(t, tmgr.SyncWAL(ctx)) + require.NoError(t, tmgr.ForceFlush(ctx)) start := c.bounds[0] end := c.bounds[1] @@ -319,9 +319,10 @@ func TestGetMessagesLatest(t *testing.T) { t.Run(c.assertion, func(t *testing.T) { buf := &bytes.Buffer{} mcap.WriteFile(t, buf, c.input...) - tmgr := testTreeManager(ctx, t) + tmgr, finish := testTreeManager(ctx, t) + defer finish() require.NoError(t, tmgr.Receive(ctx, "my-device", buf)) - require.NoError(t, tmgr.SyncWAL(ctx)) + require.NoError(t, tmgr.ForceFlush(ctx)) output := &bytes.Buffer{} start := c.bounds[0] @@ -361,38 +362,40 @@ func assertEqualTrees(t *testing.T, a, b string) { require.Equal(t, removeSpace(a), removeSpace(b), "%s != %s", a, b) } -func BenchmarkReceive(b *testing.B) { +func TestStreamingAcrossMultipleReceives(t *testing.T) { ctx := context.Background() - cases := []struct { - assertion string - inputfile string - }{ - { - "cal loop", - "/home/wyatt/data/bags/cal_loop.mcap", - }, - } + buf := &bytes.Buffer{} + mcap.WriteFile(t, buf, []uint64{10e9}) - for _, c := range cases { - b.Run(c.assertion, func(b *testing.B) { - f, err := os.Open(c.inputfile) - require.NoError(b, err) - data, err := io.ReadAll(f) - require.NoError(b, err) - b.ResetTimer() - for i := 0; i < b.N; i++ { - tmgr := testTreeManager(ctx, b) - require.NoError(b, tmgr.Receive(ctx, "my-device", bytes.NewReader(data))) - require.NoError(b, f.Close()) - } - }) - } -} + tmgr, finish := testTreeManager(ctx, t) + defer finish() + require.NoError(t, tmgr.Receive(ctx, "my-device", buf)) + require.NoError(t, tmgr.ForceFlush(ctx)) + + // overlapping + buf.Reset() + mcap.WriteFile(t, buf, []uint64{10e9}) + require.NoError(t, tmgr.Receive(ctx, "my-device", buf)) + require.NoError(t, tmgr.ForceFlush(ctx)) + + // nonoverlapping + buf.Reset() + mcap.WriteFile(t, buf, []uint64{1000e9}) + require.NoError(t, tmgr.Receive(ctx, "my-device", buf)) + require.NoError(t, tmgr.ForceFlush(ctx)) + output := &bytes.Buffer{} + require.NoError(t, tmgr.GetMessagesLatest(ctx, output, 0, 100000e9, "my-device", []string{"topic-0"})) + + reader, err := mcap.NewReader(bytes.NewReader(output.Bytes())) + require.NoError(t, err) + + info, err := reader.Info() + require.NoError(t, err) + require.Equal(t, 3, int(info.Statistics.MessageCount)) +} func TestReceive(t *testing.T) { ctx := context.Background() - buf := &bytes.Buffer{} - cases := []struct { assertion string input [][]uint64 @@ -400,39 +403,41 @@ func TestReceive(t *testing.T) { }{ { "single-topic file, single message", - [][]uint64{{10}}, + [][]uint64{{10e9}}, []string{ - `[0-64424509440:4 [0-1006632960:4 (count=1) [0-15728640:4 (count=1) - [0-245760:4 (count=1) [0-3840:4 (count=1) [0-60:4 (count=1) [leaf 1 msg]]]]]]]`, + `[0-64424509440 [0-1006632960:3 (count=1) [0-15728640:3 (count=1) + [0-245760:3 (count=1) [0-3840:3 (count=1) [0-60:3 (count=1) [leaf 1 msg]]]]]]]`, }, }, { "two topics, single messages, nonoverlapping", [][]uint64{{10e9}, {100e9}}, []string{ - `[0-64424509440:6 [0-1006632960:6 (count=1) [0-15728640:6 (count=1) [0-245760:6 (count=1) - [0-3840:6 (count=1) [0-60:6 (count=1) [leaf 1 msg]]]]]]]`, - `[0-64424509440:7 [0-1006632960:7 (count=1) [0-15728640:7 (count=1) - [0-245760:7 (count=1) [0-3840:7 (count=1) [60-120:7 (count=1) [leaf 1 msg]]]]]]]`, + `[0-64424509440 [0-1006632960:4 (count=1) [0-15728640:4 (count=1) + [0-245760:4 (count=1) [0-3840:4 (count=1) [0-60:4 (count=1) [leaf 1 msg]]]]]]]`, + `[0-64424509440 [0-1006632960:5 (count=1) [0-15728640:5 (count=1) + [0-245760:5 (count=1) [0-3840:5 (count=1) [60-120:5 (count=1) [leaf 1 msg]]]]]]]`, }, }, { "single-topic file, spanning leaf boundaries", [][]uint64{{10e9, 100e9}}, []string{ - `[0-64424509440:5 [0-1006632960:5 (count=2) [0-15728640:5 (count=2) - [0-245760:5 (count=2) [0-3840:5 (count=2) [0-60:5 (count=1) [leaf 1 msg]] - [60-120:5 (count=1) [leaf 1 msg]]]]]]]`, + `[0-64424509440 [0-1006632960:4 (count=2) [0-15728640:4 (count=2) + [0-245760:4 (count=2) [0-3840:4 (count=2) [0-60:3 (count=1) [leaf 1 msg]] + [60-120:4 (count=1) [leaf 1 msg]]]]]]]`, }, }, } for _, c := range cases { t.Run(c.assertion, func(t *testing.T) { + buf := &bytes.Buffer{} buf.Reset() mcap.WriteFile(t, buf, c.input...) - tmgr := testTreeManager(ctx, t) + tmgr, finish := testTreeManager(ctx, t) + defer finish() require.NoError(t, tmgr.Receive(ctx, "my-device", buf)) - require.NoError(t, tmgr.SyncWAL(ctx)) + require.NoError(t, tmgr.ForceFlush(ctx)) for i := range c.output { topic := fmt.Sprintf("topic-%d", i) @@ -445,18 +450,26 @@ func TestReceive(t *testing.T) { } } -func testTreeManager(ctx context.Context, tb testing.TB) *treemgr.TreeManager { +func testTreeManager(ctx context.Context, tb testing.TB) (*treemgr.TreeManager, func()) { tb.Helper() store := storage.NewMemStore() cache := util.NewLRU[nodestore.NodeID, nodestore.Node](1000) - db, err := sql.Open("sqlite3", ":memory:") - require.NoError(tb, err) - db.SetMaxOpenConns(1) - wal, err := nodestore.NewSQLWAL(ctx, db) - require.NoError(tb, err) - ns := nodestore.NewNodestore(store, cache, wal) + ns := nodestore.NewNodestore(store, cache) vs := versionstore.NewMemVersionStore() rm := rootmap.NewMemRootmap() - tmgr := treemgr.NewTreeManager(ns, vs, rm, 2, 1) - return tmgr + tmpdir, err := os.MkdirTemp("", "dp3-test") + require.NoError(tb, err) + tmgr, err := treemgr.NewTreeManager( + ctx, + ns, + vs, + rm, + treemgr.WithWALBufferSize(100), + treemgr.WithSyncWorkers(0), // control syncing manually + treemgr.WithWALDir(tmpdir), + ) + require.NoError(tb, err) + return tmgr, func() { + os.RemoveAll(tmpdir) + } } diff --git a/util/encoding.go b/util/encoding.go new file mode 100644 index 0000000..80ea8e8 --- /dev/null +++ b/util/encoding.go @@ -0,0 +1,69 @@ +package util + +/* +Encoding utilities. Note that these utilities do not check lengths - it is +necessary to ensure buffers passed (to write functions) are large enough and +that parsed string data is valid (i.e via crc content validation first), or a +panic may result. +*/ + +import "encoding/binary" + +// ReadU8 reads a uint8 from src and stores it in x, returning the written length. +func ReadU8(src []byte, x *uint8) int { + *x = src[0] + return 1 +} + +// ReadU32 reads a uint32 from src and stores it in x, returning the written length. +func ReadU32(src []byte, x *uint32) int { + *x = binary.LittleEndian.Uint32(src) + return 4 +} + +// ReadU64 reads a uint64 from src and stores it in x, returning the written length. +func ReadU64(src []byte, x *uint64) int { + *x = binary.LittleEndian.Uint64(src) + return 8 +} + +// ReadPrefixedString reads a string from data and stores it in s, returning the +// written length. +func ReadPrefixedString(data []byte, s *string) int { + if len(data) < 4 { + panic("short buffer") + } + length := int(binary.LittleEndian.Uint32(data)) + if len(data[4:]) < length { + panic("short buffer") + } + *s = string(data[4 : length+4]) + return 4 + length +} + +// ReadPrefixedString reads a string from src and stores it in x, returning the written length. +func U8(dst []byte, src uint8) int { + dst[0] = src + return 1 +} + +// U32 writes a uint32 to dst and returns the written length. +func U32(dst []byte, src uint32) int { + binary.LittleEndian.PutUint32(dst, src) + return 4 +} + +// U64 writes a uint64 to dst and returns the written length. +func U64(dst []byte, src uint64) int { + binary.LittleEndian.PutUint64(dst, src) + return 8 +} + +// WritePrefixedString writes a string to buf and returns the written length. +func WritePrefixedString(buf []byte, s string) int { + if len(buf) < 4+len(s) { + panic("buffer too small") + } + binary.LittleEndian.PutUint32(buf, uint32(len(s))) + return 4 + copy(buf[4:], s) +} diff --git a/util/encoding_test.go b/util/encoding_test.go new file mode 100644 index 0000000..71cb4d4 --- /dev/null +++ b/util/encoding_test.go @@ -0,0 +1,64 @@ +package util_test + +import ( + "testing" + + "github.com/stretchr/testify/require" + "github.com/wkalt/dp3/util" +) + +func TestReadU8(t *testing.T) { + var x uint8 + n := util.ReadU8([]byte{0x01}, &x) + require.Equal(t, 1, n) + require.Equal(t, uint8(0x01), x) +} + +func TestReadU32(t *testing.T) { + var x uint32 + n := util.ReadU32([]byte{0x01, 0x02, 0x03, 0x04}, &x) + require.Equal(t, 4, n) + require.Equal(t, uint32(0x04030201), x) +} + +func TestReadU64(t *testing.T) { + var x uint64 + n := util.ReadU64([]byte{0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08}, &x) + require.Equal(t, 8, n) + require.Equal(t, uint64(0x0807060504030201), x) +} + +func TestU8(t *testing.T) { + buf := make([]byte, 1) + n := util.U8(buf, 0x01) + require.Equal(t, 1, n) + require.Equal(t, []byte{0x01}, buf) +} + +func TestU32(t *testing.T) { + buf := make([]byte, 4) + n := util.U32(buf, 0x04030201) + require.Equal(t, 4, n) + require.Equal(t, []byte{0x01, 0x02, 0x03, 0x04}, buf) +} + +func TestU64(t *testing.T) { + buf := make([]byte, 8) + n := util.U64(buf, 0x0807060504030201) + require.Equal(t, 8, n) + require.Equal(t, []byte{0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08}, buf) +} + +func TestWritePrefixedString(t *testing.T) { + buf := make([]byte, 4+4) + n := util.WritePrefixedString(buf, "test") + require.Equal(t, 8, n) + require.Equal(t, []byte{0x04, 0x00, 0x00, 0x00, 't', 'e', 's', 't'}, buf) +} + +func TestReadPrefixedString(t *testing.T) { + var s string + n := util.ReadPrefixedString([]byte{0x04, 0x00, 0x00, 0x00, 't', 'e', 's', 't'}, &s) + require.Equal(t, 8, n) + require.Equal(t, "test", s) +} diff --git a/wal/errors.go b/wal/errors.go new file mode 100644 index 0000000..d2c32f2 --- /dev/null +++ b/wal/errors.go @@ -0,0 +1,53 @@ +package wal + +import ( + "errors" + "fmt" + + "github.com/wkalt/dp3/nodestore" +) + +// NodeNotFoundError is returned when a node is not found in the WAL. +type NodeNotFoundError struct { + NodeID nodestore.NodeID +} + +func (e NodeNotFoundError) Error() string { + return fmt.Sprintf("node %s not found", e.NodeID) +} + +func (e NodeNotFoundError) Is(target error) bool { + _, ok := target.(NodeNotFoundError) + return ok +} + +// ErrBadMagic is returned when the WAL magic is not as expected. +var ErrBadMagic = errors.New("bad WAL magic") + +// UnsupportedWALError is returned when the WAL version is not supported by the server. +type UnsupportedWALError struct { + major, minor uint8 +} + +func (e UnsupportedWALError) Error() string { + return fmt.Sprintf("unsupported WAL version: %d.%d (current: %d.%d)", e.major, e.minor, currentMajor, currentMinor) +} + +func (e UnsupportedWALError) Is(target error) bool { + _, ok := target.(UnsupportedWALError) + return ok +} + +// CRCMismatchError is returned when the CRC of a record does not match the computed CRC. +type CRCMismatchError struct { + expected, actual uint32 +} + +func (e CRCMismatchError) Error() string { + return fmt.Sprintf("expected CRC %d, got %d", e.expected, e.actual) +} + +func (e CRCMismatchError) Is(target error) bool { + _, ok := target.(CRCMismatchError) + return ok +} diff --git a/wal/options.go b/wal/options.go new file mode 100644 index 0000000..19b296d --- /dev/null +++ b/wal/options.go @@ -0,0 +1,53 @@ +package wal + +/* +Options for the WAL manager. +*/ + +//////////////////////////////////////////////////////////////////////////////// + +import "time" + +type config struct { + mergeSizeThreshold int + staleBatchThreshold time.Duration + targetFileSize int + + inactiveBatchMergeInterval time.Duration +} + +// Option is a function that modifies the WAL manager configuration. +type Option func(*config) + +// WithMergeSizeThreshold sets the batch data size at which the WAL manager +// merge a batch, regardless of whether data is still coming in. Supplied in +// bytes. +func WithMergeSizeThreshold(size int) Option { + return func(c *config) { + c.mergeSizeThreshold = size + } +} + +// WithStaleBatchThreshold sets the time threshold after which a batch is +// considered to be no longer receiving writes, thus indicating it should be +// merged into storage (regardless of size). +func WithStaleBatchThreshold(d time.Duration) Option { + return func(c *config) { + c.staleBatchThreshold = d + } +} + +// WithTargetFileSize sets the target size for WAL files. When the active file +// exceeds this size, the WAL manager will close it and rotate the log. +func WithTargetFileSize(size int) Option { + return func(c *config) { + c.targetFileSize = size + } +} + +// WithInactiveBatchMergeInterval sets the polling interval to check for inactive batches. +func WithInactiveBatchMergeInterval(secs int) Option { + return func(c *config) { + c.inactiveBatchMergeInterval = time.Duration(secs) * time.Second + } +} diff --git a/wal/wal.go b/wal/wal.go new file mode 100644 index 0000000..f9c241c --- /dev/null +++ b/wal/wal.go @@ -0,0 +1,123 @@ +package wal + +import ( + "encoding/binary" + "fmt" + "strconv" + "time" +) + +/* +Common types used by the WAL package. +*/ + +//////////////////////////////////////////////////////////////////////////////// + +// RecordType is the type of record in the WAL. +type RecordType uint8 + +func (r RecordType) String() string { + switch r { + case WALInsert: + return "insert" + case WALMergeRequest: + return "mergeRequest" + case WALMergeComplete: + return "mergeComplete" + default: + return "invalid" + } +} + +const ( + WALInvalid RecordType = iota + WALInsert + WALMergeRequest + WALMergeComplete +) + +const ( + gigabyte = 1 << 30 +) + +// InsertRecord is a record of a single insert operation. The data is a byte +// representation of a partial tree. +type InsertRecord struct { + Producer string + Topic string + BatchID string + Addr Address + Data []byte +} + +// MergeRequestRecord records a request to merge a batch of inserts into +// storage. +type MergeRequestRecord struct { + Producer string + Topic string + BatchID string + Addrs []Address +} + +// MergeCompleteRecord records the completion of a merge into storage. +type MergeCompleteRecord struct { + BatchID string +} + +// Batch represents a collection of inserts that will be merged into the tree +// together. The WAL manager maintains a batch for each producer/topic combo +// that is receiving writes, and intelligently dispatches them for merging based +// on either size or inactivit, and intelligently dispatches them for merging +// based on either size or inactivity. +type Batch struct { + ID string + ProducerID string + Topic string + Size int + LastUpdate time.Time + Addrs []Address + OnDone func() error +} + +func (b *Batch) Finish() error { + return b.OnDone() +} + +// Addresses are 24-byte values consisting of an object ID, offset, and length. +type Address [24]byte + +// NewAddress creates a new address from the given object, offset, and length. +func NewAddress(object, offset, length uint64) Address { + var a Address + binary.LittleEndian.PutUint64(a[:8], object) + binary.LittleEndian.PutUint64(a[8:16], offset) + binary.LittleEndian.PutUint64(a[16:24], length) + return a +} + +func (a Address) object() string { + object := binary.LittleEndian.Uint64(a[:8]) + return strconv.FormatUint(object, 10) +} + +func (a Address) offset() int64 { + return int64(binary.LittleEndian.Uint64(a[8:16])) +} + +func (a Address) String() string { + return fmt.Sprintf("%d:%d:%d", + binary.LittleEndian.Uint64(a[:8]), + binary.LittleEndian.Uint64(a[8:16]), + binary.LittleEndian.Uint64(a[16:24]), + ) +} + +// TreeID is a unique identifier for a tree in the WAL. +type TreeID struct { + Producer string + Topic string +} + +func (id TreeID) String() string { + return id.Producer + id.Topic +} diff --git a/wal/wal_reader.go b/wal/wal_reader.go new file mode 100644 index 0000000..9cafd0a --- /dev/null +++ b/wal/wal_reader.go @@ -0,0 +1,133 @@ +package wal + +import ( + "bytes" + "encoding/binary" + "errors" + "fmt" + "hash/crc32" + "io" + + "github.com/wkalt/dp3/util" +) + +/* +The WAL reader provides an iterator interface for reading records from a WAL +file. It is used during recovery on startup and not thereafter (not for address +lookups). +*/ + +//////////////////////////////////////////////////////////////////////////////// + +type walReader struct { + r io.ReadSeeker + offset int64 +} + +func (r *walReader) Offset() int64 { + return r.offset +} + +// NewReader creates a new WAL reader. +func NewReader(r io.ReadSeeker) (*walReader, error) { + if err := validateMagic(r); err != nil { + return nil, err + } + offset := int64(len(Magic) + 2) + return &walReader{r: r, offset: offset}, nil +} + +// ParseMergeRequestRecord parses a merge request record. +func ParseMergeRequestRecord(data []byte) *MergeRequestRecord { + offset := 0 + var producer, topic, batchID string + offset += util.ReadPrefixedString(data[offset:], &producer) + offset += util.ReadPrefixedString(data[offset:], &topic) + offset += util.ReadPrefixedString(data[offset:], &batchID) + var addrs []Address + for offset < len(data) { + var a Address + offset += copy(a[:], data[offset:offset+24]) + addrs = append(addrs, a) + } + return &MergeRequestRecord{ + Producer: producer, + Topic: topic, + BatchID: batchID, + Addrs: addrs, + } +} + +// ParseMergeCompleteRecord parses a merge complete record. +func ParseMergeCompleteRecord(data []byte) *MergeCompleteRecord { + var id string + _ = util.ReadPrefixedString(data, &id) + return &MergeCompleteRecord{BatchID: id} +} + +// ParseInsertRecord parses an insert record. +func ParseInsertRecord(data []byte) *InsertRecord { + offset := 0 + var producer, topic, batchID string + offset += util.ReadPrefixedString(data[offset:], &producer) + offset += util.ReadPrefixedString(data[offset:], &topic) + offset += util.ReadPrefixedString(data[offset:], &batchID) + + var addr Address + offset += copy(addr[:], data[offset:offset+24]) + return &InsertRecord{ + Producer: producer, + Topic: topic, + BatchID: batchID, + Addr: addr, + Data: data[offset:], + } +} + +// Next returns the next record from the WAL file, with CRC validation. +func (r *walReader) Next() (RecordType, []byte, error) { + header := make([]byte, 1+8) + _, err := io.ReadFull(r.r, header) + if err != nil { + return 0, nil, fmt.Errorf("failed to read record header: %w", err) + } + var offset int + var rectype uint8 + var length uint64 + offset += util.ReadU8(header[offset:], &rectype) + util.ReadU64(header[offset:], &length) + body := make([]byte, length+4) + _, err = io.ReadFull(r.r, body) + if err != nil { + return 0, nil, fmt.Errorf("failed to read record body: %w", err) + } + dataEnd := len(body) - 4 + computed := crc32.ChecksumIEEE(header) + computed = crc32.Update(computed, crc32.IEEETable, body[:dataEnd]) + crc := binary.LittleEndian.Uint32(body[dataEnd:]) + if crc != computed { + return 0, nil, CRCMismatchError{crc, computed} + } + r.offset += int64(1 + 8 + len(body)) + return RecordType(rectype), body[:dataEnd], nil +} + +func validateMagic(r io.Reader) error { + buf := make([]byte, 8) + _, err := io.ReadFull(r, buf) + if err != nil { + if errors.Is(err, io.ErrUnexpectedEOF) { + return ErrBadMagic + } + return fmt.Errorf("failed to read WAL magic: %w", err) + } + if !bytes.Equal(buf[:6], Magic) { + return ErrBadMagic + } + major := buf[6] + minor := buf[7] + if major > currentMajor || (major == currentMajor && minor > currentMinor) { + return UnsupportedWALError{major, minor} + } + return nil +} diff --git a/wal/wal_reader_test.go b/wal/wal_reader_test.go new file mode 100644 index 0000000..37f7a20 --- /dev/null +++ b/wal/wal_reader_test.go @@ -0,0 +1,133 @@ +package wal_test + +import ( + "bytes" + "testing" + + "github.com/stretchr/testify/require" + "github.com/wkalt/dp3/wal" +) + +func TestBadMagic(t *testing.T) { + t.Run("unsupported major", func(t *testing.T) { + buf := make([]byte, 8) + copy(buf, wal.Magic) + buf[6] = 1 + _, err := wal.NewReader(bytes.NewReader(buf)) + require.ErrorIs(t, err, wal.UnsupportedWALError{}) + }) + + t.Run("unsupported minor", func(t *testing.T) { + buf := make([]byte, 8) + copy(buf, wal.Magic) + buf[7] = 1 + _, err := wal.NewReader(bytes.NewReader(buf)) + require.ErrorIs(t, err, wal.UnsupportedWALError{}) + }) + + t.Run("invalid magic", func(t *testing.T) { + buf := make([]byte, 8) + copy(buf, wal.Magic) + buf[0] = 0 + _, err := wal.NewReader(bytes.NewReader(buf)) + require.ErrorIs(t, err, wal.ErrBadMagic) + }) + + t.Run("short magic", func(t *testing.T) { + buf := make([]byte, 7) + copy(buf, wal.Magic) + _, err := wal.NewReader(bytes.NewReader(buf)) + require.ErrorIs(t, err, wal.ErrBadMagic) + }) +} + +func TestWALCorruption(t *testing.T) { + buf := &bytes.Buffer{} + writer, err := wal.NewWriter(buf, 1, 0) + require.NoError(t, err) + _, _, err = writer.WriteInsert(wal.InsertRecord{ + Producer: "producer", + Topic: "topic", + Data: []byte{0x01, 0x02}, + }) + require.NoError(t, err) + + data := buf.Bytes() + data[len(data)-8] = 0x99 // corrupt it + + reader, err := wal.NewReader(bytes.NewReader(data)) + require.NoError(t, err) + _, _, err = reader.Next() + require.ErrorIs(t, err, wal.CRCMismatchError{}) +} + +func TestWALReader(t *testing.T) { + t.Run("insert", func(t *testing.T) { + record := wal.InsertRecord{ + Producer: "producer", + Topic: "topic", + BatchID: "foobar", + Addr: wal.NewAddress(1, 8, 72), + Data: []byte("data"), + } + buf := &bytes.Buffer{} + w, err := wal.NewWriter(buf, 1, 0) + require.NoError(t, err) + _, _, err = w.WriteInsert(record) + require.NoError(t, err) + reader, err := wal.NewReader(bytes.NewReader(buf.Bytes())) + require.NoError(t, err) + for { + rectype, data, err := reader.Next() + require.NoError(t, err) + if rectype == wal.WALInsert { + result := wal.ParseInsertRecord(data) + require.Equal(t, record.Addr.String(), result.Addr.String()) + break + } + } + }) + t.Run("mergeRequest", func(t *testing.T) { + record := wal.MergeRequestRecord{ + Producer: "producer", + Topic: "topic", + BatchID: "id", + Addrs: []wal.Address{wal.NewAddress(1, 2, 3), wal.NewAddress(4, 5, 6)}, + } + buf := &bytes.Buffer{} + w, err := wal.NewWriter(buf, 1, 0) + require.NoError(t, err) + _, _, err = w.WriteMergeRequest(record) + require.NoError(t, err) + reader, err := wal.NewReader(bytes.NewReader(buf.Bytes())) + require.NoError(t, err) + for { + rectype, data, err := reader.Next() + require.NoError(t, err) + if rectype == wal.WALMergeRequest { + result := wal.ParseMergeRequestRecord(data) + require.Equal(t, *result, record) + break + } + } + }) + t.Run("mergeComplete", func(t *testing.T) { + buf := &bytes.Buffer{} + w, err := wal.NewWriter(buf, 1, 0) + require.NoError(t, err) + _, _, err = w.WriteMergeComplete(wal.MergeCompleteRecord{"id"}) + require.NoError(t, err) + require.NoError(t, err) + reader, err := wal.NewReader(bytes.NewReader(buf.Bytes())) + require.NoError(t, err) + for { + rectype, data, err := reader.Next() + require.NoError(t, err) + if rectype == wal.WALMergeComplete { + result := wal.ParseMergeCompleteRecord(data) + require.Equal(t, &wal.MergeCompleteRecord{BatchID: "id"}, result) + break + } + } + }) +} diff --git a/wal/wal_writer.go b/wal/wal_writer.go new file mode 100644 index 0000000..830f88c --- /dev/null +++ b/wal/wal_writer.go @@ -0,0 +1,142 @@ +package wal + +import ( + "fmt" + "hash/crc32" + "io" + "sync" + + "github.com/wkalt/dp3/util" +) + +/* +The WAL writer is used to write records to the write-ahead log. The format of +the write-ahead log is as follows: + + Magic: 6 bytes (dp3wal) + Version: 2 bytes (major, minor) + [Record]* + +Where each Record is: + Type: 1 byte + Length: 8 bytes + Data: [Length]byte + CRC32: 4 bytes + +The CRC is calculated over all preceding bytes of the record - i.e including the +record type. The type may be "insert", "merge request", or "merge complete". +*/ + +//////////////////////////////////////////////////////////////////////////////// + +// Magic is the magic number for the WAL file. +var Magic = []byte{'d', 'p', '3', 'w', 'a', 'l'} // nolint:gochecknoglobals + +const ( + currentMajor = uint8(0) + currentMinor = uint8(0) +) + +type Writer struct { + id uint64 + writer io.Writer + offset int64 + mtx *sync.Mutex +} + +// NewWriter creates a new WAL writer. +func NewWriter(w io.Writer, id uint64, initialOffset int64) (*Writer, error) { + if initialOffset == 0 { + buf := make([]byte, 8) + offset := copy(buf, Magic) + offset += util.U8(buf[offset:], currentMajor) + util.U8(buf[offset:], currentMinor) + n, err := w.Write(buf) + if err != nil { + return nil, fmt.Errorf("failed to write WAL magic: %w", err) + } + initialOffset = int64(n) + } + return &Writer{ + id: id, + writer: w, + mtx: &sync.Mutex{}, + offset: initialOffset, + }, nil +} + +// WriteMergeComplete writes a merge complete record. +func (w *Writer) WriteMergeComplete(rec MergeCompleteRecord) (Address, int, error) { + w.mtx.Lock() + defer w.mtx.Unlock() + + length := 4 + len(rec.BatchID) + data := make([]byte, length) + _ = util.WritePrefixedString(data, rec.BatchID) + return w.writeRecord(WALMergeComplete, nil, data) +} + +// WriteMergeRequest writes a merge request record. +func (w *Writer) WriteMergeRequest(rec MergeRequestRecord) (Address, int, error) { + w.mtx.Lock() + defer w.mtx.Unlock() + + length := 4 + len(rec.Producer) + 4 + len(rec.Topic) + 4 + len(rec.BatchID) + 24*len(rec.Addrs) + data := make([]byte, length) + offset := util.WritePrefixedString(data, rec.Producer) + offset += util.WritePrefixedString(data[offset:], rec.Topic) + offset += util.WritePrefixedString(data[offset:], rec.BatchID) + for _, addr := range rec.Addrs { + offset += copy(data[offset:], addr[:]) + } + return w.writeRecord(WALMergeRequest, nil, data) +} + +// WriteInsert writes an insert record. +func (w *Writer) WriteInsert(rec InsertRecord) (Address, int, error) { + w.mtx.Lock() + defer w.mtx.Unlock() + + header := make([]byte, 4+len(rec.Producer)+4+len(rec.Topic)+4+len(rec.BatchID)+24) + var offset int + offset += util.WritePrefixedString(header[offset:], rec.Producer) + offset += util.WritePrefixedString(header[offset:], rec.Topic) + offset += util.WritePrefixedString(header[offset:], rec.BatchID) + + addr := NewAddress(w.id, uint64(w.size()), uint64(1+8+len(header)+len(rec.Data)+4)) + copy(header[offset:], addr[:]) + + return w.writeRecord(WALInsert, header, rec.Data) +} + +func (w *Writer) size() int64 { + return w.offset +} + +// writeRecord writes a record to the WAL. +func (w *Writer) writeRecord(rectype RecordType, header []byte, data []byte) (addr Address, n int, err error) { + buf := make([]byte, 1+8+len(header)+len(data)+4) + offset := 0 + offset += util.U8(buf[offset:], uint8(rectype)) + offset += util.U64(buf[offset:], uint64(len(header)+len(data))) + if len(header) > 0 { + offset += copy(buf[offset:], header) + } + offset += copy(buf[offset:], data) + + crc := crc32.ChecksumIEEE(buf[:offset]) + util.U32(buf[offset:], crc) + + n, err = w.writer.Write(buf) + w.offset += int64(n) + if err != nil { + return addr, n, fmt.Errorf("failed to write record header: %w", err) + } + if f, ok := w.writer.(interface{ Flush() error }); ok { + if err := f.Flush(); err != nil { + return addr, n, fmt.Errorf("failed to flush writer: %w", err) + } + } + addr = NewAddress(w.id, uint64(w.offset-int64(n)), uint64(n)) + return addr, n, nil +} diff --git a/wal/walmgr.go b/wal/walmgr.go new file mode 100644 index 0000000..7323899 --- /dev/null +++ b/wal/walmgr.go @@ -0,0 +1,510 @@ +package wal + +import ( + "context" + "encoding/binary" + "errors" + "fmt" + "hash/crc32" + "io" + "os" + "sort" + "strconv" + "sync" + "time" + + "github.com/google/uuid" + "github.com/wkalt/dp3/util" + "github.com/wkalt/dp3/util/log" + "golang.org/x/exp/maps" +) + +/* +The WAL manager is responsible for managing the write-ahead log, including +read/write interfaces, log rotation, recovery, and scheduling of WAL data for +merge into the tree. + +The WAL manager is instantiated and controlled by the tree manager. On startup, +it does a linear scan of all data existing in the WAL, playing records through +the WAL state machine. + +There are two kinds of state the walmgr stores in memory: "pending inserts" and +"pending merges". The pending inserts map is an association between "tree IDs" +(composed of producer + topic) to "batches". A batch is a collection of inserts +to a single tree ID. A single call to "receive" may produce multiple batches, or +a batch may span multiple calls to receive. + +After either the size of a batch exceeds a configured threshold, or a configured +amount of time has passed since the last update to a batch, a "merge request" +covering the addresses of the batch is recorded to the WAL, and the batch is +moved from "pending inserts" to "pending merges" in the walmgr's state. + +The actual merge of a batch into the tree is handled by the treemgr. At the +conclusion of the merge, the treemgr calls a callback on the batch that causes +the walmgr to record a "merge complete" record. + +After scanning the WAL (recovery), the state in the WAL manager is up-to-date, +reflecting any inserts that have been stored but not yet queued for merge, as +well as any batches requested for merge that have not yet completed. + +It notifies the treemgr of pending merges over a shared channel, and is then +ready to take new requests. + +There are three kinds of records stored in the WAL: +* inserts: contain the data to be stored in the tree. The data of an insert + record is a "partial tree" of new nodes, in leaf-to-root order. It is the + byte serialization of a "memtree". +* merge requests: contain the addresses of a batch of inserts that are ready to + be merged, as well as a UUID-valued identifier. +* merge complete: contain the UUID of a batch that has been successfully merged. + +Addresses in the WAL are 24 bytes long, composed of a file ID, an offset, and a +length, each 8 bytes. WAL files are named according to the decimal +representation of the file ID. +*/ + +//////////////////////////////////////////////////////////////////////////////// + +// WALManager is the write-ahead log manager. +type WALManager struct { + f io.WriteCloser + writer *Writer + counter uint64 + pendingInserts map[TreeID]*Batch + pendingMerges map[string]*Batch + mtx *sync.Mutex + config *config + merges chan<- *Batch + waldir string +} + +// NewWALManager constructs a new WAL manager. +func NewWALManager( + ctx context.Context, + waldir string, + merges chan<- *Batch, + opts ...Option, +) (*WALManager, error) { + conf := &config{ + mergeSizeThreshold: 2 * gigabyte, + staleBatchThreshold: 5 * time.Second, + targetFileSize: 2 * gigabyte, + inactiveBatchMergeInterval: 0, + } + for _, opt := range opts { + opt(conf) + } + wmgr := &WALManager{ + merges: merges, + mtx: &sync.Mutex{}, + config: conf, + pendingInserts: map[TreeID]*Batch{}, + pendingMerges: map[string]*Batch{}, + waldir: waldir, + } + if conf.inactiveBatchMergeInterval > 0 { + go func() { + for range time.NewTicker(conf.inactiveBatchMergeInterval).C { + if err := wmgr.mergeInactiveBatches(ctx); err != nil { + log.Errorf(ctx, "failed to merge inactive batches: %s", err) + } + } + }() + } + return wmgr, nil +} + +// Insert data into the WAL for a producer and topic. The data should be the +// serialized representation of a memtree, i.e a partial tree. +func (w *WALManager) Insert(producer string, topic string, data []byte) (Address, error) { + tid := TreeID{producer, topic} + bat, ok := w.pendingInserts[tid] + if !ok { + id := uuid.New().String() + bat = w.newBatch(id, producer, topic) + w.pendingInserts[tid] = bat + } + addr, err := w.insert(tid, bat.ID, data) + if err != nil { + return Address{}, err + } + bat.Size += len(data) + bat.LastUpdate = time.Now() + bat.Addrs = append(bat.Addrs, addr) + + // if the batch is over a certain size now, merge it. + if bat.Size > w.config.mergeSizeThreshold { + if err := w.mergeBatch(bat); err != nil { + return addr, err + } + } + if w.writer.size() > int64(w.config.targetFileSize) { + if err := w.rotate(); err != nil { + return addr, err + } + } + return addr, nil +} + +// Get data from the WAL at a given address. +func (w *WALManager) Get(addr Address) ([]byte, error) { + w.mtx.Lock() + defer w.mtx.Unlock() + f, err := w.openr(addr.object()) + if err != nil { + return nil, fmt.Errorf("failed to open wal file: %w", err) + } + defer f.Close() + _, err = f.Seek(addr.offset(), io.SeekStart) + if err != nil { + return nil, fmt.Errorf("failed to seek to record offset: %w", err) + } + header := make([]byte, 1+8) + _, err = io.ReadFull(f, header) + if err != nil { + return nil, fmt.Errorf("failed to read record header: %w", err) + } + var offset int + var rectype uint8 + var length uint64 + offset += util.ReadU8(header[offset:], &rectype) + if rectype != uint8(WALInsert) { + return nil, fmt.Errorf("expected insert record at %s, got %s", addr, RecordType(rectype)) + } + util.ReadU64(header[offset:], &length) + body := make([]byte, length+4) + _, err = io.ReadFull(f, body) + if err != nil { + return nil, fmt.Errorf("failed to read record body: %w", err) + } + dataEnd := len(body) - 4 + computed := crc32.ChecksumIEEE(header) + computed = crc32.Update(computed, crc32.IEEETable, body[:dataEnd]) + crc := binary.LittleEndian.Uint32(body[dataEnd:]) + if crc != computed { + return nil, CRCMismatchError{crc, computed} + } + insert := ParseInsertRecord(body[:dataEnd]) + return insert.Data, nil +} + +// ForceMerge forces the merge of all pending inserts in the WAL. This is used +// in tests only - in operation merges are async. +func (w *WALManager) ForceMerge(ctx context.Context) (int, error) { + batches := []*Batch{} + w.mtx.Lock() + + keys := maps.Keys(w.pendingInserts) + sort.Slice(keys, func(i, j int) bool { + return w.pendingInserts[keys[i]].Topic > w.pendingInserts[keys[j]].Topic + }) + + for _, k := range keys { + batch := w.pendingInserts[k] + delete(w.pendingInserts, k) + batches = append(batches, batch) + } + w.mtx.Unlock() + c := 0 + for _, batch := range batches { + if err := w.mergeBatch(batch); err != nil { + return c, fmt.Errorf("failed to merge wal batch: %w", err) + } + c++ + } + return c, nil +} + +// WALStats contains statistics about the WAL manager. +type WALStats struct { + PendingInserts map[TreeID][]Address + PendingMerges map[string]*Batch +} + +// Stats returns statistics about the WAL manager. +func (w *WALManager) Stats() WALStats { + w.mtx.Lock() + defer w.mtx.Unlock() + inserts := map[TreeID][]Address{} + for k, v := range w.pendingInserts { + inserts[k] = v.Addrs + } + merges := map[string]*Batch{} + for k, v := range w.pendingMerges { + merges[k] = v + } + + return WALStats{ + PendingInserts: inserts, + PendingMerges: merges, + } +} + +// Recover scans the files in the waldir and replays them through the walmgr's +// internal state. Recover must be called before the WAL manager is ready to +// use. +func (w *WALManager) Recover(ctx context.Context) error { + paths, err := listDir(w.waldir) + if err != nil { + return fmt.Errorf("failed to list wal directory: %w", err) + } + ids := make([]uint64, 0, len(paths)) + log.Infof(ctx, "Replaying %d log files", len(paths)) + for _, path := range paths { + id, err := strconv.ParseUint(path, 10, 64) + if err != nil { + return fmt.Errorf("failed to parse wal path: %w", err) + } + ids = append(ids, id) + } + if len(paths) == 0 { + return w.rotate() + } + sort.Slice(ids, func(i, j int) bool { + return ids[i] < ids[j] + }) + for _, id := range ids { + if err := w.scanfile(ctx, id); err != nil { + return err + } + } + + // todo this can block if there is a lot of pending merges. Ensure the + // channel consumers are started up first. + keys := maps.Keys(w.pendingMerges) + sort.Slice(keys, func(i, j int) bool { + return w.pendingMerges[keys[i]].Topic > w.pendingMerges[keys[j]].Topic + }) + for _, k := range keys { + w.merges <- w.pendingMerges[k] + log.Infof(ctx, "Enqueued pending merge %s", k) + } + err = w.setfile(ids[len(ids)-1]) + if err != nil { + return err + } + log.Infow(ctx, "Recovery complete", + "pending inserts", len(w.pendingInserts), + "pending merges", len(w.pendingMerges), + ) + return nil +} + +func listDir(dir string) ([]string, error) { + files, err := os.ReadDir(dir) + if err != nil { + return nil, fmt.Errorf("failed to list directory: %w", err) + } + result := make([]string, 0, len(files)) + for _, f := range files { + if f.IsDir() { + continue + } + result = append(result, f.Name()) + } + return result, nil +} + +func (w *WALManager) mergeBatch(batch *Batch) error { + if _, _, err := w.writer.WriteMergeRequest(MergeRequestRecord{ + Producer: batch.ProducerID, + Topic: batch.Topic, + BatchID: batch.ID, + Addrs: batch.Addrs, + }); err != nil { + return err + } + w.mtx.Lock() + delete(w.pendingInserts, TreeID{Producer: batch.ProducerID, Topic: batch.Topic}) + w.pendingMerges[batch.ID] = batch + w.mtx.Unlock() + w.merges <- batch + return nil +} + +// merges bounded length queue; bound = 1 in tests? + +func (w *WALManager) mergeInactiveBatches(ctx context.Context) error { + batches := []*Batch{} + w.mtx.Lock() + keys := maps.Keys(w.pendingInserts) + sort.Slice(keys, func(i, j int) bool { + return w.pendingInserts[keys[i]].Topic > w.pendingInserts[keys[j]].Topic + }) + for _, k := range keys { + batch := w.pendingInserts[k] + if time.Since(batch.LastUpdate) > w.config.staleBatchThreshold { + delete(w.pendingInserts, k) + batches = append(batches, batch) + } + } + w.mtx.Unlock() + if len(batches) == 0 { + return nil + } + t0 := time.Now() + log.Infof(ctx, "Merging %d inactive batches", len(batches)) + for _, batch := range batches { + if err := w.mergeBatch(batch); err != nil { + return err + } + } + log.Infof(ctx, "Merged %d inactive batches in %s", len(batches), time.Since(t0)) + return nil +} + +func (w *WALManager) insert(tid TreeID, batchID string, data []byte) (Address, error) { + addr, _, err := w.writer.WriteInsert(InsertRecord{ + Producer: tid.Producer, + Topic: tid.Topic, + BatchID: batchID, + Data: data, + }) + if err != nil { + return Address{}, err + } + return addr, nil +} + +func (w *WALManager) openw(path string) (*os.File, error) { + f, err := os.OpenFile(w.waldir+"/"+path, os.O_RDWR|os.O_CREATE, 0644) + if err != nil { + return nil, fmt.Errorf("failed to open file: %w", err) + } + return f, nil +} + +func (w *WALManager) openr(path string) (*os.File, error) { + f, err := os.OpenFile(w.waldir+"/"+path, os.O_RDWR, 0644) + if err != nil { + return nil, fmt.Errorf("failed to open file: %w", err) + } + return f, nil +} + +func (w *WALManager) completeMerge(id string) error { + if _, _, err := w.writer.WriteMergeComplete(MergeCompleteRecord{ + BatchID: id, + }); err != nil { + return err + } + w.mtx.Lock() + delete(w.pendingMerges, id) + w.mtx.Unlock() + return nil +} + +func (w *WALManager) rotate() error { + if w.f != nil { + if err := w.f.Close(); err != nil { + return fmt.Errorf("failed to close log file: %w", err) + } + } + w.counter++ + f, err := w.openw(strconv.FormatUint(w.counter, 10)) + if err != nil { + return fmt.Errorf("failed to create new log file: %w", err) + } + w.f = f + w.writer, err = NewWriter(f, w.counter, 0) + if err != nil { + return fmt.Errorf("failed to create new log writer: %w", err) + } + + return nil +} + +func (w *WALManager) scanfile(ctx context.Context, id uint64) error { + f, err := w.openr(strconv.FormatUint(id, 10)) + if err != nil { + return fmt.Errorf("failed to open wal file: %w", err) + } + defer f.Close() + r, err := NewReader(f) + if err != nil { + return fmt.Errorf("failed to construct wal reader: %w", err) + } + for { + rectype, data, err := r.Next() + if err != nil { + switch { + case errors.Is(err, io.ErrUnexpectedEOF): + n, err := f.Seek(r.offset, io.SeekStart) + if err != nil { + return fmt.Errorf("failed to seek to last good record end: %w", err) + } + if err := f.Truncate(n); err != nil { + return fmt.Errorf("failed to truncate file: %w", err) + } + log.Infof(ctx, "Truncated log file at %d", n) + return nil + case errors.Is(err, io.EOF): + return nil + } + return fmt.Errorf("failed to read next wal record: %w", err) + } + switch rectype { + case WALInsert: + rec := ParseInsertRecord(data) + bat, ok := w.pendingInserts[TreeID{Producer: rec.Producer, Topic: rec.Topic}] + if !ok { + bat = &Batch{ + ID: rec.BatchID, + ProducerID: rec.Producer, + Topic: rec.Topic, + Addrs: []Address{}, + OnDone: func() error { + return w.completeMerge(rec.BatchID) + }, + } + w.pendingInserts[TreeID{Producer: rec.Producer, Topic: rec.Topic}] = bat + } + bat.Size += len(rec.Data) + bat.Addrs = append(bat.Addrs, rec.Addr) + case WALMergeRequest: + rec := ParseMergeRequestRecord(data) + key := TreeID{Producer: rec.Producer, Topic: rec.Topic} + batch := w.pendingInserts[key] + delete(w.pendingInserts, key) + w.pendingMerges[rec.BatchID] = batch + case WALMergeComplete: + rec := ParseMergeCompleteRecord(data) + delete(w.pendingMerges, rec.BatchID) + } + } +} + +func (w *WALManager) setfile(id uint64) error { + f, err := w.openw(strconv.FormatUint(id, 10)) + if err != nil { + return fmt.Errorf("failed to open wal file: %w", err) + } + n, err := f.Seek(0, io.SeekEnd) + if err != nil { + return fmt.Errorf("failed to seek to file end: %w", err) + } + w.f = f + w.counter = id + w.writer, err = NewWriter(f, id, n) + if err != nil { + return fmt.Errorf("failed to construct wal writer: %w", err) + } + return nil +} + +func (w *WALManager) newBatch( + id string, + producerID string, + topic string, +) *Batch { + return &Batch{ + ID: id, + ProducerID: producerID, + Topic: topic, + Size: 0, + Addrs: []Address{}, + OnDone: func() error { + return w.completeMerge(id) + }, + } +} diff --git a/wal/walmgr_test.go b/wal/walmgr_test.go new file mode 100644 index 0000000..a1b802f --- /dev/null +++ b/wal/walmgr_test.go @@ -0,0 +1,328 @@ +package wal_test + +import ( + "context" + "io" + "os" + "testing" + + "github.com/stretchr/testify/require" + "github.com/wkalt/dp3/wal" +) + +func TestWALRotation(t *testing.T) { + ctx := context.Background() + tmpdir, err := os.MkdirTemp("", "wal_test") + require.NoError(t, err) + defer os.RemoveAll(tmpdir) + merges := make(chan *wal.Batch) + + wm, err := wal.NewWALManager( + ctx, tmpdir, merges, + wal.WithTargetFileSize(1024), + ) + require.NoError(t, err) + require.NoError(t, wm.Recover(ctx)) + for i := 0; i < 100; i++ { + _, err := wm.Insert("producer", "topic", []byte{0x01, 0x02}) + require.NoError(t, err) + } + + paths, err := os.ReadDir(tmpdir) + require.NoError(t, err) + + require.Len(t, paths, 10) +} + +func TestWALLookups(t *testing.T) { + cases := []struct { + assertion string + records []wal.RecordType + }{ + { + "insert", + []wal.RecordType{wal.WALInsert}, + }, + { + "2x insert", + []wal.RecordType{wal.WALInsert, wal.WALInsert}, + }, + { + "2x insert + merge request", + []wal.RecordType{wal.WALInsert, wal.WALInsert, wal.WALMergeRequest}, + }, + { + "2x insert + merge request + merge complete", + []wal.RecordType{wal.WALInsert, wal.WALInsert, wal.WALMergeRequest, wal.WALMergeComplete}, + }, + } + for _, c := range cases { + t.Run(c.assertion, func(t *testing.T) { + ctx := context.Background() + tmpdir, err := os.MkdirTemp("", "wal_test") + require.NoError(t, err) + defer os.RemoveAll(tmpdir) + wm, _, teardown := testWALManager(ctx, t, tmpdir, 10000, func(batch *wal.Batch) { + require.NoError(t, batch.Finish()) + }) + require.NoError(t, wm.Recover(ctx)) + addrs := []wal.Address{} + data := make([]byte, 100) + for _, record := range c.records { + if record == wal.WALInsert { + addr, err := wm.Insert("producer", "topic", data) + require.NoError(t, err) + addrs = append(addrs, addr) + } + // todo don't think this is testing anything + } + for _, addr := range addrs { + value, err := wm.Get(addr) + require.NoError(t, err) + require.Equal(t, data, value) + } + teardown() + }) + } +} + +func TestWALRecovery(t *testing.T) { + ctx := context.Background() + cases := []struct { + assertion string + processBatches bool + mergeSizeThreshold int + inserts []string // producers + expectedInsertsCounts map[string]int + expectedMerges map[string]int + expectedBatchCount int + }{ + { + "empty", + true, + 100, + []string{}, + map[string]int{}, + map[string]int{}, + 0, + }, + { + "one insert unprocessed", + true, + 1000, + []string{ + "producer", + }, + map[string]int{ + "producer": 1, + }, + map[string]int{}, + 0, + }, + { + "two inserts unprocessed", + true, + 1000, + []string{ + "producer", + "producer", + }, + map[string]int{ + "producer": 2, + }, + map[string]int{}, + 0, + }, + { + "two inserts unprocessed, two producers", + true, + 1000, + []string{ + "producer", + "producer2", + }, + map[string]int{ + "producer": 1, + "producer2": 1, + }, + map[string]int{}, + 0, + }, + { + "one insert processed", + true, + 50, + []string{ + "producer", + }, + map[string]int{}, + map[string]int{}, + 1, + }, + { + "two processed one unprocessed", + true, + 150, + []string{"producer", "producer", "producer"}, + map[string]int{"producer": 1}, + map[string]int{}, + 1, + }, + { + "requested for processing but not processed", + false, + 150, + []string{"producer", "producer", "producer"}, + map[string]int{"producer": 1}, + map[string]int{ + "producer": 2, + }, + 1, + }, + } + + for _, c := range cases { + for i := 0; i < 1; i++ { + t.Run(c.assertion, func(t *testing.T) { + tmpdir, err := os.MkdirTemp("", "wal_test") + require.NoError(t, err) + defer os.RemoveAll(tmpdir) + + wm, await, teardown := testWALManager(ctx, t, tmpdir, c.mergeSizeThreshold, func(batch *wal.Batch) { + if c.processBatches { + require.NoError(t, batch.Finish()) + } + }) + + require.NoError(t, wm.Recover(ctx)) + topic := "topic" + for _, producer := range c.inserts { + data := make([]byte, 100) + _, err := wm.Insert(producer, topic, data) + require.NoError(t, err) + } + await(c.expectedBatchCount) // wait for any batches to process prior to shutdown + teardown() + + wm2, _, teardown := testWALManager(ctx, t, tmpdir, c.mergeSizeThreshold, func(batch *wal.Batch) { + if c.processBatches { + require.NoError(t, batch.Finish()) + } + }) + require.NoError(t, wm2.Recover(ctx)) + teardown() + + for treeID, count := range wm2.Stats().PendingInserts { + require.Equal(t, len(count), c.expectedInsertsCounts[treeID.Producer]) + } + + foundMerges := map[string]int{} + for _, batch := range wm2.Stats().PendingMerges { + foundMerges[batch.ProducerID] = len(batch.Addrs) + } + require.Equal(t, c.expectedMerges, foundMerges) + }) + } + } +} + +func TestScenarios(t *testing.T) { + t.Run("truncated final write on shutdown is truncated", func(t *testing.T) { + ctx := context.Background() + tmpdir, err := os.MkdirTemp("", "wal_test") + require.NoError(t, err) + defer os.RemoveAll(tmpdir) + + wm, _, teardown := testWALManager(ctx, t, tmpdir, 1000, func(batch *wal.Batch) { + require.NoError(t, batch.Finish()) + }) + require.NoError(t, wm.Recover(ctx)) + _, err = wm.Insert("producer", "topic", []byte{0x01, 0x02}) + require.NoError(t, err) + teardown() + + // truncate the last record to simulate unclean shutdown + f, err := os.OpenFile(tmpdir+"/1", os.O_RDWR, 0644) + require.NoError(t, err) + n, err := f.Seek(-2, io.SeekEnd) + require.NoError(t, err) + require.NoError(t, f.Truncate(n)) + require.NoError(t, f.Close()) + + // New wmgr should see a clean recovery. The truncated record should be + // gone. Inserts should be received as normal. + wm2, _, teardown := testWALManager(ctx, t, tmpdir, 1000, func(batch *wal.Batch) { + require.NoError(t, batch.Finish()) + }) + require.NoError(t, wm2.Recover(ctx)) + require.Empty(t, wm2.Stats().PendingInserts) + _, err = wm2.Insert("producer", "topic", []byte{0x01, 0x02}) + require.NoError(t, err) + require.Len(t, wm2.Stats().PendingInserts, 1) + teardown() + + // New instance successfully picks up inserts over truncated WAL. + wm3, _, teardown := testWALManager(ctx, t, tmpdir, 1000, func(batch *wal.Batch) { + require.NoError(t, batch.Finish()) + }) + require.NoError(t, wm3.Recover(ctx)) + require.Len(t, wm3.Stats().PendingInserts, 1) + teardown() + }) + + t.Run("flushes split across a single insert", func(t *testing.T) { + + }) + + t.Run("multiple topics interleaved", func(t *testing.T) { + + }) + + t.Run("log pruning does not delete data we need", func(t *testing.T) { + + }) + + t.Run("logs rotate based on size", func(t *testing.T) { + + }) +} + +func testWALManager( + ctx context.Context, + t *testing.T, + tmpdir string, + threshold int, + batchfn func(*wal.Batch), +) (*wal.WALManager, func(n int), func()) { + t.Helper() + merges := make(chan *wal.Batch) + wm, err := wal.NewWALManager( + ctx, + tmpdir, + merges, + wal.WithMergeSizeThreshold(threshold), + ) + require.NoError(t, err) + done := make(chan bool) + ctx, cancel := context.WithCancel(ctx) + go func() { + for { + select { + case <-ctx.Done(): + return + case batch := <-merges: + batchfn(batch) + done <- true + } + } + }() + await := func(n int) { + for i := 0; i < n; i++ { + <-done + } + } + teardown := func() { + cancel() + } + return wm, await, teardown +}