diff --git a/accum/block.go b/accum/block.go
new file mode 100644
index 00000000..89a0c989
--- /dev/null
+++ b/accum/block.go
@@ -0,0 +1,179 @@
+package accum
+
+import (
+ "context"
+ "errors"
+ "io"
+ "sync"
+
+ "github.com/ipfs/go-cid"
+ "github.com/rpcpool/yellowstone-faithful/carreader"
+ "github.com/rpcpool/yellowstone-faithful/iplddecoders"
+)
+
+type ObjectAccumulator struct {
+ flushOnKind iplddecoders.Kind
+ reader *carreader.CarReader
+ ignoreKinds iplddecoders.KindSlice
+ callback func(*ObjectWithMetadata, []ObjectWithMetadata) error
+ flushWg sync.WaitGroup
+ flushQueue chan *flushBuffer
+}
+
+var ErrStop = errors.New("stop")
+
+func isStop(err error) bool {
+ return errors.Is(err, ErrStop)
+}
+
+func NewObjectAccumulator(
+ reader *carreader.CarReader,
+ flushOnKind iplddecoders.Kind,
+ callback func(*ObjectWithMetadata, []ObjectWithMetadata) error,
+ ignoreKinds ...iplddecoders.Kind,
+) *ObjectAccumulator {
+ return &ObjectAccumulator{
+ reader: reader,
+ ignoreKinds: ignoreKinds,
+ flushOnKind: flushOnKind,
+ callback: callback,
+ flushQueue: make(chan *flushBuffer, 1000),
+ }
+}
+
+var flushBufferPool = sync.Pool{
+ New: func() interface{} {
+ return &flushBuffer{}
+ },
+}
+
+func getFlushBuffer() *flushBuffer {
+ return flushBufferPool.Get().(*flushBuffer)
+}
+
+func putFlushBuffer(fb *flushBuffer) {
+ fb.Reset()
+ flushBufferPool.Put(fb)
+}
+
+type flushBuffer struct {
+ head *ObjectWithMetadata
+ other []ObjectWithMetadata
+}
+
+// Reset resets the flushBuffer.
+func (fb *flushBuffer) Reset() {
+ fb.head = nil
+ clear(fb.other)
+}
+
+type ObjectWithMetadata struct {
+ Cid cid.Cid
+ Offset uint64
+ SectionLength uint64
+ ObjectData []byte
+}
+
+func (oa *ObjectAccumulator) startFlusher(ctx context.Context) {
+ for {
+ select {
+ case <-ctx.Done():
+ return
+ case fb := <-oa.flushQueue:
+ if fb == nil {
+ return
+ }
+ if err := oa.flush(fb.head, fb.other); err != nil {
+ if isStop(err) {
+ return
+ }
+ panic(err)
+ }
+ oa.flushWg.Done()
+ putFlushBuffer(fb)
+ }
+ }
+}
+
+func (oa *ObjectAccumulator) sendToFlusher(head *ObjectWithMetadata, other []ObjectWithMetadata) {
+ oa.flushWg.Add(1)
+ fb := getFlushBuffer()
+ fb.head = head
+ fb.other = clone(other)
+ oa.flushQueue <- fb
+}
+
+func (oa *ObjectAccumulator) Run(ctx context.Context) error {
+ go oa.startFlusher(ctx)
+ defer func() {
+ oa.flushWg.Wait()
+ close(oa.flushQueue)
+ }()
+ totalOffset := uint64(0)
+ {
+ if size, err := oa.reader.HeaderSize(); err != nil {
+ return err
+ } else {
+ totalOffset += size
+ }
+ }
+ objectCap := 5000
+buffersLoop:
+ for {
+ objects := make([]ObjectWithMetadata, 0, objectCap)
+ currentBufferLoop:
+ for {
+ if ctx.Err() != nil {
+ return ctx.Err()
+ }
+ c, sectionLength, data, err := oa.reader.NextNodeBytes()
+ if err != nil {
+ if errors.Is(err, io.EOF) {
+ oa.sendToFlusher(nil, objects)
+ break buffersLoop
+ }
+ return err
+ }
+ currentOffset := totalOffset
+ totalOffset += sectionLength
+
+ if data == nil {
+ oa.sendToFlusher(nil, objects)
+ break buffersLoop
+ }
+
+ objm := ObjectWithMetadata{
+ Cid: c,
+ Offset: currentOffset,
+ SectionLength: sectionLength,
+ ObjectData: data,
+ }
+
+ kind := iplddecoders.Kind(data[1])
+ if kind == oa.flushOnKind {
+ oa.sendToFlusher(&objm, (objects))
+ break currentBufferLoop
+ } else {
+ if len(oa.ignoreKinds) > 0 && oa.ignoreKinds.Has(kind) {
+ continue
+ }
+ objects = append(objects, objm)
+ }
+ }
+ }
+
+ return nil
+}
+
+func (oa *ObjectAccumulator) flush(head *ObjectWithMetadata, other []ObjectWithMetadata) error {
+ if head == nil && len(other) == 0 {
+ return nil
+ }
+ return oa.callback(head, other)
+}
+
+func clone[T any](s []T) []T {
+ v := make([]T, len(s))
+ copy(v, s)
+ return v
+}
diff --git a/adapters.go b/adapters.go
index e248a5d4..20e3669c 100644
--- a/adapters.go
+++ b/adapters.go
@@ -234,17 +234,18 @@ func adaptTransactionMetaToExpectedOutput(m map[string]any) map[string]any {
} else {
instruction["accounts"] = []any{}
}
- if data, ok := instruction["data"]; ok {
- // as string
- dataStr, ok := data.(string)
- if ok {
- decoded, err := base64.StdEncoding.DecodeString(dataStr)
- if err == nil {
- // TODO: the data in the `innerInstructions` is always base58 encoded (even if the transaction is base64 encoded)
- instruction["data"] = base58.Encode(decoded)
- }
- }
- }
+ // if data, ok := instruction["data"]; ok {
+ // // as string
+ // dataStr, ok := data.(string)
+ // if ok {
+ // decoded, err := base64.StdEncoding.DecodeString(dataStr)
+ // if err == nil {
+ // // TODO: the data in the `innerInstructions` is always base58 encoded (even if the transaction is base64 encoded)
+ // // instruction["data"] = base58.Encode(decoded)
+ // _ = decoded
+ // }
+ // }
+ // }
}
}
meta["innerInstructions"].([]any)[i] = innerInstruction
diff --git a/carreader/reader.go b/carreader/reader.go
new file mode 100644
index 00000000..1336ae80
--- /dev/null
+++ b/carreader/reader.go
@@ -0,0 +1,188 @@
+package carreader
+
+import (
+ "bufio"
+ "bytes"
+ "encoding/binary"
+ "errors"
+ "fmt"
+ "io"
+ "os"
+
+ "github.com/ipfs/go-cid"
+ cbor "github.com/ipfs/go-ipld-cbor"
+ "github.com/ipfs/go-libipfs/blocks"
+ carv1 "github.com/ipld/go-car"
+ "github.com/ipld/go-car/util"
+ "github.com/rpcpool/yellowstone-faithful/readahead"
+)
+
+type CarReader struct {
+ headerSize *uint64
+ br *bufio.Reader
+ Header *carv1.CarHeader
+}
+
+func alignValueToPageSize(value int) int {
+ pageSize := os.Getpagesize()
+ return (value + pageSize - 1) &^ (pageSize - 1)
+}
+
+func New(r io.ReadCloser) (*CarReader, error) {
+ br := bufio.NewReaderSize(r, alignValueToPageSize(readahead.DefaultChunkSize))
+ ch, err := ReadHeader(br)
+ if err != nil {
+ return nil, err
+ }
+
+ if ch.Version != 1 {
+ return nil, fmt.Errorf("invalid car version: %d", ch.Version)
+ }
+
+ if len(ch.Roots) == 0 {
+ return nil, fmt.Errorf("empty car, no roots")
+ }
+
+ return &CarReader{
+ br: br,
+ Header: ch,
+ }, nil
+}
+
+func ReadHeader(br io.Reader) (*carv1.CarHeader, error) {
+ hb, err := util.LdRead(bufio.NewReader(br))
+ if err != nil {
+ return nil, err
+ }
+
+ var ch carv1.CarHeader
+ if err := cbor.DecodeInto(hb, &ch); err != nil {
+ return nil, fmt.Errorf("invalid header: %v", err)
+ }
+
+ return &ch, nil
+}
+
+func (cr *CarReader) NextInfo() (cid.Cid, uint64, error) {
+ c, sectionLen, err := ReadNodeInfoWithoutData(cr.br)
+ if err != nil {
+ return c, 0, err
+ }
+ return c, sectionLen, nil
+}
+
+func (cr *CarReader) NextNode() (cid.Cid, uint64, *blocks.BasicBlock, error) {
+ c, sectionLen, data, err := ReadNodeInfoWithData(cr.br)
+ if err != nil {
+ return c, 0, nil, fmt.Errorf("failed to read node info: %w", err)
+ }
+ bl, err := blocks.NewBlockWithCid(data, c)
+ if err != nil {
+ return c, 0, nil, fmt.Errorf("failed to create block: %w", err)
+ }
+ return c, sectionLen, bl, nil
+}
+
+func (cr *CarReader) NextNodeBytes() (cid.Cid, uint64, []byte, error) {
+ c, sectionLen, data, err := ReadNodeInfoWithData(cr.br)
+ if err != nil {
+ return c, 0, nil, fmt.Errorf("failed to read node info: %w", err)
+ }
+ return c, sectionLen, data, nil
+}
+
+func (cr *CarReader) HeaderSize() (uint64, error) {
+ if cr.headerSize == nil {
+ var buf bytes.Buffer
+ if err := carv1.WriteHeader(cr.Header, &buf); err != nil {
+ return 0, err
+ }
+ size := uint64(buf.Len())
+ cr.headerSize = &size
+ }
+ return *cr.headerSize, nil
+}
+
+func ReadNodeInfoWithoutData(br *bufio.Reader) (cid.Cid, uint64, error) {
+ sectionLen, ll, err := ReadSectionLength(br)
+ if err != nil {
+ return cid.Cid{}, 0, err
+ }
+
+ cidLen, c, err := cid.CidFromReader(br)
+ if err != nil {
+ return cid.Cid{}, 0, err
+ }
+
+ // Seek to the next section by skipping the block.
+ // The section length includes the CID, so subtract it.
+ remainingSectionLen := int64(sectionLen) - int64(cidLen)
+
+ _, err = io.CopyN(io.Discard, br, remainingSectionLen)
+ if err != nil {
+ return cid.Cid{}, 0, err
+ }
+
+ return c, sectionLen + ll, nil
+}
+
+func ReadNodeInfoWithData(br *bufio.Reader) (cid.Cid, uint64, []byte, error) {
+ sectionLen, ll, err := ReadSectionLength(br)
+ if err != nil {
+ return cid.Cid{}, 0, nil, fmt.Errorf("failed to read section length: %w", err)
+ }
+
+ cidLen, c, err := cid.CidFromReader(br)
+ if err != nil {
+ return cid.Cid{}, 0, nil, fmt.Errorf("failed to read cid: %w", err)
+ }
+
+ // Seek to the next section by skipping the block.
+ // The section length includes the CID, so subtract it.
+ remainingSectionLen := int64(sectionLen) - int64(cidLen)
+
+ buf := make([]byte, remainingSectionLen)
+ _, err = io.ReadFull(br, buf)
+ if err != nil {
+ return cid.Cid{}, 0, nil, fmt.Errorf("failed to read block: %w", err)
+ }
+
+ return c, sectionLen + ll, buf, nil
+}
+
+func ReadSectionLength(r *bufio.Reader) (uint64, uint64, error) {
+ if _, err := r.Peek(1); err != nil { // no more blocks, likely clean io.EOF
+ if errors.Is(err, io.ErrNoProgress) {
+ return 0, 0, io.EOF
+ }
+ return 0, 0, fmt.Errorf("failed to peek: %w", err)
+ }
+
+ br := byteReaderWithCounter{r, 0}
+ l, err := binary.ReadUvarint(&br)
+ if err != nil {
+ if errors.Is(err, io.EOF) {
+ return 0, 0, io.ErrUnexpectedEOF // don't silently pretend this is a clean EOF
+ }
+ return 0, 0, err
+ }
+
+ if l > uint64(util.MaxAllowedSectionSize) { // Don't OOM
+ return 0, 0, errors.New("malformed car; header is bigger than util.MaxAllowedSectionSize")
+ }
+
+ return l, br.Offset, nil
+}
+
+type byteReaderWithCounter struct {
+ io.ByteReader
+ Offset uint64
+}
+
+func (b *byteReaderWithCounter) ReadByte() (byte, error) {
+ c, err := b.ByteReader.ReadByte()
+ if err == nil {
+ b.Offset++
+ }
+ return c, err
+}
diff --git a/cmd-rpc.go b/cmd-rpc.go
index 80e40c1f..5f8d4685 100644
--- a/cmd-rpc.go
+++ b/cmd-rpc.go
@@ -9,6 +9,7 @@ import (
"runtime"
"sort"
"sync"
+ "sync/atomic"
"time"
"github.com/allegro/bigcache/v3"
@@ -170,6 +171,9 @@ func newCmd_rpc() *cli.Command {
return *configs[i].Epoch < *configs[j].Epoch
})
+ numFailed := new(atomic.Int32)
+ numSucceeded := new(atomic.Int32)
+
wg := new(errgroup.Group)
wg.SetLimit(epochLoadConcurrency)
for confIndex := range configs {
@@ -194,10 +198,12 @@ func newCmd_rpc() *cli.Command {
if err != nil {
metrics_epochsAvailable.WithLabelValues(fmt.Sprintf("%d", epochNum)).Set(0)
klog.Error(err)
+ numFailed.Add(1)
// NOTE: DO NOT return the error here, as we want to continue loading other epochs
return nil
}
metrics_epochsAvailable.WithLabelValues(fmt.Sprintf("%d", epochNum)).Set(1)
+ numSucceeded.Add(1)
return nil
})
}
@@ -205,7 +211,7 @@ func newCmd_rpc() *cli.Command {
klog.Errorf("fatal error initializing epochs: %s", err.Error())
}
tookInitializingEpochs := time.Since(startedInitiatingEpochsAt)
- klog.Infof("Initialized %d epochs in %s", len(configs), tookInitializingEpochs)
+ klog.Infof("Initialized %d/%d epochs in %s", numSucceeded.Load(), len(configs), tookInitializingEpochs)
}()
if watch {
diff --git a/cmd-x-index-all.go b/cmd-x-index-all.go
index 2a2e3513..b4b15eac 100644
--- a/cmd-x-index-all.go
+++ b/cmd-x-index-all.go
@@ -13,8 +13,8 @@ import (
"github.com/dustin/go-humanize"
"github.com/ipfs/go-cid"
- carv1 "github.com/ipld/go-car"
"github.com/rpcpool/yellowstone-faithful/bucketteer"
+ "github.com/rpcpool/yellowstone-faithful/carreader"
"github.com/rpcpool/yellowstone-faithful/indexes"
"github.com/rpcpool/yellowstone-faithful/indexmeta"
"github.com/rpcpool/yellowstone-faithful/iplddecoders"
@@ -46,7 +46,7 @@ func newCmd_Index_all() *cli.Command {
&cli.StringFlag{
Name: "tmp-dir",
Usage: "temporary directory to use for storing intermediate files",
- Value: "",
+ Value: os.TempDir(),
},
&cli.StringFlag{
Name: "network",
@@ -135,19 +135,19 @@ func createAllIndexes(
}
defer carFile.Close()
- rd, err := newCarReader(carFile)
+ rd, err := carreader.New(carFile)
if err != nil {
return nil, 0, fmt.Errorf("failed to create car reader: %w", err)
}
// check it has 1 root
- if len(rd.header.Roots) != 1 {
- return nil, 0, fmt.Errorf("car file must have exactly 1 root, but has %d", len(rd.header.Roots))
+ if len(rd.Header.Roots) != 1 {
+ return nil, 0, fmt.Errorf("car file must have exactly 1 root, but has %d", len(rd.Header.Roots))
}
// print roots:
- for _, root := range rd.header.Roots {
+ for _, root := range rd.Header.Roots {
klog.Infof("- Root: %s", root)
}
- rootCID := rd.header.Roots[0]
+ rootCID := rd.Header.Roots[0]
klog.Infof("Getting car file size")
@@ -222,11 +222,11 @@ func createAllIndexes(
totalOffset := uint64(0)
{
- var buf bytes.Buffer
- if err = carv1.WriteHeader(rd.header, &buf); err != nil {
+ if size, err := rd.HeaderSize(); err != nil {
return nil, 0, err
+ } else {
+ totalOffset += size
}
- totalOffset = uint64(buf.Len())
}
numIndexedOffsets := uint64(0)
@@ -529,13 +529,13 @@ func verifyAllIndexes(
}
defer carFile.Close()
- rd, err := newCarReader(carFile)
+ rd, err := carreader.New(carFile)
if err != nil {
return fmt.Errorf("failed to create car reader: %w", err)
}
// check it has 1 root
- if len(rd.header.Roots) != 1 {
- return fmt.Errorf("car file must have exactly 1 root, but has %d", len(rd.header.Roots))
+ if len(rd.Header.Roots) != 1 {
+ return fmt.Errorf("car file must have exactly 1 root, but has %d", len(rd.Header.Roots))
}
cid_to_offset_and_size, err := OpenIndex_CidToOffset(
@@ -575,11 +575,11 @@ func verifyAllIndexes(
totalOffset := uint64(0)
{
- var buf bytes.Buffer
- if err = carv1.WriteHeader(rd.header, &buf); err != nil {
+ if size, err := rd.HeaderSize(); err != nil {
return err
+ } else {
+ totalOffset += size
}
- totalOffset = uint64(buf.Len())
}
numIndexedOffsets := uint64(0)
diff --git a/cmd-x-index-cid2offset.go b/cmd-x-index-cid2offset.go
index 1af14a87..77653c23 100644
--- a/cmd-x-index-cid2offset.go
+++ b/cmd-x-index-cid2offset.go
@@ -3,6 +3,7 @@ package main
import (
"context"
"fmt"
+ "os"
"time"
"github.com/rpcpool/yellowstone-faithful/indexes"
@@ -33,7 +34,7 @@ func newCmd_Index_cid2offset() *cli.Command {
&cli.StringFlag{
Name: "tmp-dir",
Usage: "temporary directory to use for storing intermediate files",
- Value: "",
+ Value: os.TempDir(),
},
&cli.Uint64Flag{
Name: "epoch",
diff --git a/cmd-x-index-gsfa.go b/cmd-x-index-gsfa.go
index 86df0208..eb1ef7b3 100644
--- a/cmd-x-index-gsfa.go
+++ b/cmd-x-index-gsfa.go
@@ -2,30 +2,25 @@ package main
import (
"context"
- "errors"
"fmt"
- "io"
"io/fs"
"os"
"path/filepath"
"runtime"
- "sync"
"sync/atomic"
"time"
+ "github.com/davecgh/go-spew/spew"
"github.com/dustin/go-humanize"
- bin "github.com/gagliardetto/binary"
"github.com/gagliardetto/solana-go"
"github.com/ipfs/go-cid"
- "github.com/ipfs/go-libipfs/blocks"
- "github.com/ipld/go-car"
+ "github.com/rpcpool/yellowstone-faithful/accum"
+ "github.com/rpcpool/yellowstone-faithful/carreader"
"github.com/rpcpool/yellowstone-faithful/gsfa"
"github.com/rpcpool/yellowstone-faithful/indexes"
"github.com/rpcpool/yellowstone-faithful/indexmeta"
"github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode"
"github.com/rpcpool/yellowstone-faithful/iplddecoders"
- "github.com/rpcpool/yellowstone-faithful/readahead"
- concurrently "github.com/tejzpr/ordered-concurrently/v3"
"github.com/urfave/cli/v2"
"k8s.io/klog/v2"
)
@@ -44,11 +39,6 @@ func newCmd_Index_gsfa() *cli.Command {
return nil
},
Flags: []cli.Flag{
- &cli.Uint64Flag{
- Name: "flush-every",
- Usage: "flush every N transactions",
- Value: 1_000_000,
- },
// verify hash of transactions:
&cli.BoolFlag{
Name: "verify-hash",
@@ -78,6 +68,11 @@ func newCmd_Index_gsfa() *cli.Command {
return nil
},
},
+ &cli.StringFlag{
+ Name: "tmp-dir",
+ Usage: "temporary directory to use for storing intermediate files",
+ Value: os.TempDir(),
+ },
},
Action: func(c *cli.Context) error {
carPath := c.Args().First()
@@ -93,11 +88,7 @@ func newCmd_Index_gsfa() *cli.Command {
defer file.Close()
}
- cachingReader, err := readahead.NewCachingReaderFromReader(file, readahead.DefaultChunkSize)
- if err != nil {
- klog.Exitf("Failed to create caching reader: %s", err)
- }
- rd, err := car.NewCarReader(cachingReader)
+ rd, err := carreader.New(file)
if err != nil {
klog.Exitf("Failed to open CAR: %s", err)
}
@@ -135,12 +126,6 @@ func newCmd_Index_gsfa() *cli.Command {
return fmt.Errorf("failed to create index dir: %w", err)
}
- flushEvery := c.Uint64("flush-every")
- if flushEvery == 0 {
- return fmt.Errorf("flush-every must be > 0")
- }
- klog.Infof("Will flush to index every %s transactions", humanize.Comma(int64(flushEvery)))
-
meta := indexmeta.Meta{}
if err := meta.AddUint64(indexmeta.MetadataKey_Epoch, epoch); err != nil {
return fmt.Errorf("failed to add epoch to sig_exists index metadata: %w", err)
@@ -151,119 +136,165 @@ func newCmd_Index_gsfa() *cli.Command {
if err := meta.AddString(indexmeta.MetadataKey_Network, string(network)); err != nil {
return fmt.Errorf("failed to add network to sig_exists index metadata: %w", err)
}
- accu, err := gsfa.NewGsfaWriter(
+ tmpDir := c.String("tmp-dir")
+ indexW, err := gsfa.NewGsfaWriter(
gsfaIndexDir,
- flushEvery,
meta,
+ epoch,
+ rootCID,
+ network,
+ tmpDir,
)
if err != nil {
return fmt.Errorf("error while opening gsfa index writer: %w", err)
}
+ numProcessedTransactions := new(atomic.Int64)
+ startedAt := time.Now()
defer func() {
- if err := accu.Flush(); err != nil {
- klog.Errorf("Error while flushing: %s", err)
- }
- if err := accu.Close(); err != nil {
+ klog.Infof("Indexed %s transactions", humanize.Comma(int64(numProcessedTransactions.Load())))
+ klog.Info("Finalizing index -- this may take a while, DO NOT EXIT")
+ klog.Info("Closing index")
+ if err := indexW.Close(); err != nil {
klog.Errorf("Error while closing: %s", err)
}
- }()
-
- startedAt := time.Now()
- numTransactionsSeen := 0
- defer func() {
+ klog.Infof("Success: gSFA index created at %s with %d transactions", gsfaIndexDir, numProcessedTransactions.Load())
klog.Infof("Finished in %s", time.Since(startedAt))
- klog.Infof("Indexed %s transactions", humanize.Comma(int64(numTransactionsSeen)))
}()
- dotEvery := 100_000
- klog.Infof("A dot is printed every %s transactions", humanize.Comma(int64(dotEvery)))
- verifyHash = c.Bool("verify-hash")
- numWorkers := c.Uint("w")
+ verifyHash := c.Bool("verify-hash")
+ ipldbindcode.DisableHashVerification = !verifyHash
- if numWorkers == 0 {
- numWorkers = uint(runtime.NumCPU())
- }
- workerInputChan := make(chan concurrently.WorkFunction, numWorkers)
- waitExecuted := new(sync.WaitGroup)
- waitResultsReceived := new(sync.WaitGroup)
- numReceivedAtomic := new(atomic.Int64)
+ epochStart, epochEnd := CalcEpochLimits(epoch)
- outputChan := concurrently.Process(
- context.Background(),
- workerInputChan,
- &concurrently.Options{PoolSize: int(numWorkers), OutChannelBuffer: int(numWorkers)},
- )
- go func() {
- // process the results from the workers
- for result := range outputChan {
- switch resValue := result.Value.(type) {
- case error:
- panic(resValue)
- case TransactionWithSlot:
- tx := resValue.Transaction
- slot := resValue.Slot
- sig := tx.Signatures[0]
- err = accu.Push(slot, sig, tx.Message.AccountKeys)
+ numSlots := uint64(0)
+ numMaxObjects := uint64(0)
+
+ lastPrintedAt := time.Now()
+ lastTimeDid1kSlots := time.Now()
+ var eta time.Duration
+ etaSampleSlots := uint64(2_000)
+ var tookToDo1kSlots time.Duration
+ accum := accum.NewObjectAccumulator(
+ rd,
+ iplddecoders.KindBlock,
+ func(owm1 *accum.ObjectWithMetadata, owm2 []accum.ObjectWithMetadata) error {
+ numSlots++
+ numObjects := len(owm2) + 1
+ if numObjects > int(numMaxObjects) {
+ numMaxObjects = uint64(numObjects)
+ }
+
+ if owm1 == nil {
+ transactions, err := objectsToTransactions(&ipldbindcode.Block{
+ Meta: ipldbindcode.SlotMeta{
+ Blocktime: 0,
+ },
+ }, owm2)
if err != nil {
- klog.Exitf("Error while pushing to gsfa index: %s", err)
+ return fmt.Errorf("error while converting objects to transactions: %w", err)
}
- waitResultsReceived.Done()
- numReceivedAtomic.Add(-1)
- default:
- panic(fmt.Errorf("unexpected result type: %T", result.Value))
+ if len(transactions) == 0 {
+ return nil
+ }
+ spew.Dump(owm1, transactions, len(owm2))
}
- }
- }()
- for {
- block, err := rd.Next()
- if err != nil {
- if errors.Is(err, io.EOF) {
- fmt.Println("EOF")
- break
+ // decode the block:
+ block, err := iplddecoders.DecodeBlock(owm1.ObjectData)
+ if err != nil {
+ return fmt.Errorf("error while decoding block: %w", err)
}
- return err
- }
- kind := iplddecoders.Kind(block.RawData()[1])
-
- switch kind {
- case iplddecoders.KindTransaction:
- numTransactionsSeen++
- if numTransactionsSeen%dotEvery == 0 {
- fmt.Print(".")
+ if numSlots%etaSampleSlots == 0 {
+ tookToDo1kSlots = time.Since(lastTimeDid1kSlots)
+ lastTimeDid1kSlots = time.Now()
}
- {
- waitExecuted.Add(1)
- waitResultsReceived.Add(1)
- numReceivedAtomic.Add(1)
- workerInputChan <- newTxParserWorker(
- block,
- func() {
- waitExecuted.Done()
- },
- )
+ if tookToDo1kSlots > 0 {
+ eta = time.Duration(float64(tookToDo1kSlots) / float64(etaSampleSlots) * float64(epochEnd-epochStart-numSlots))
}
- default:
- continue
- }
- }
+ transactions, err := objectsToTransactions(block, owm2)
+ if err != nil {
+ return fmt.Errorf("error while converting objects to transactions: %w", err)
+ }
+ for ii := range transactions {
+ txWithInfo := transactions[ii]
+ numProcessedTransactions.Add(1)
+ err = indexW.Push(
+ txWithInfo.Offset,
+ txWithInfo.Length,
+ txWithInfo.Slot,
+ txWithInfo.Blocktime,
+ txWithInfo.Transaction.Message.AccountKeys,
+ )
+ if err != nil {
+ klog.Exitf("Error while pushing to gsfa index: %s", err)
+ }
- {
- klog.Infof("Waiting for all transactions to be parsed...")
- waitExecuted.Wait()
- klog.Infof("All transactions parsed.")
+ if time.Since(lastPrintedAt) > time.Millisecond*500 {
+ percentDone := float64(txWithInfo.Slot-epochStart) / float64(epochEnd-epochStart) * 100
+ // clear line, then print progress
+ msg := fmt.Sprintf(
+ "\rCreating gSFA index for epoch %d - %s | %s | %.2f%% | slot %s | tx %s",
+ epoch,
+ time.Now().Format("2006-01-02 15:04:05"),
+ time.Since(startedAt).Truncate(time.Second),
+ percentDone,
+ humanize.Comma(int64(txWithInfo.Slot)),
+ humanize.Comma(int64(numProcessedTransactions.Load())),
+ )
+ if eta > 0 {
+ msg += fmt.Sprintf(" | ETA %s", eta.Truncate(time.Second))
+ }
+ fmt.Print(msg)
+ lastPrintedAt = time.Now()
+ }
+ }
+ return nil
+ },
+ // Ignore these kinds in the accumulator (only need transactions):
+ iplddecoders.KindEntry,
+ iplddecoders.KindRewards,
+ iplddecoders.KindDataFrame,
+ )
- klog.Infof("Waiting to receive all results...")
- close(workerInputChan)
- waitResultsReceived.Wait()
- klog.Infof("All results received")
+ if err := accum.Run(context.Background()); err != nil {
+ return fmt.Errorf("error while accumulating objects: %w", err)
}
- klog.Infof("Success: GSFA index created at %s", gsfaIndexDir)
+
return nil
},
}
}
+func objectsToTransactions(
+ block *ipldbindcode.Block,
+ objects []accum.ObjectWithMetadata,
+) ([]*TransactionWithSlot, error) {
+ transactions := make([]*TransactionWithSlot, 0, len(objects))
+ for _, object := range objects {
+ // check if the object is a transaction:
+ kind := iplddecoders.Kind(object.ObjectData[1])
+ if kind != iplddecoders.KindTransaction {
+ continue
+ }
+ decoded, err := iplddecoders.DecodeTransaction(object.ObjectData)
+ if err != nil {
+ return nil, fmt.Errorf("error while decoding transaction from nodex %s: %w", object.Cid, err)
+ }
+ tx, err := decoded.GetSolanaTransaction()
+ if err != nil {
+ return nil, fmt.Errorf("error while getting solana transaction from object %s: %w", object.Cid, err)
+ }
+ transactions = append(transactions, &TransactionWithSlot{
+ Offset: object.Offset,
+ Length: object.SectionLength,
+ Slot: uint64(decoded.Slot),
+ Blocktime: uint64(block.Meta.Blocktime),
+ Transaction: *tx,
+ })
+ }
+ return transactions, nil
+}
+
func formatIndexDirname_gsfa(epoch uint64, rootCid cid.Cid, network indexes.Network) string {
return fmt.Sprintf(
"epoch-%d-%s-%s-%s",
@@ -275,63 +306,9 @@ func formatIndexDirname_gsfa(epoch uint64, rootCid cid.Cid, network indexes.Netw
}
type TransactionWithSlot struct {
+ Offset uint64
+ Length uint64
Slot uint64
+ Blocktime uint64
Transaction solana.Transaction
}
-
-type txParserWorker struct {
- blk blocks.Block
- done func()
-}
-
-func newTxParserWorker(
- blk blocks.Block,
- done func(),
-) *txParserWorker {
- return &txParserWorker{
- blk: blk,
- done: done,
- }
-}
-
-var verifyHash bool
-
-func (w txParserWorker) Run(ctx context.Context) interface{} {
- defer func() {
- w.done()
- }()
-
- block := w.blk
-
- decoded, err := iplddecoders.DecodeTransaction(block.RawData())
- if err != nil {
- return fmt.Errorf("error while decoding transaction from nodex %s: %w", block.Cid(), err)
- }
- {
- if total, ok := decoded.Data.GetTotal(); !ok || total == 1 {
- completeData := decoded.Data.Bytes()
- if verifyHash {
- // verify hash (if present)
- if ha, ok := decoded.Data.GetHash(); ok {
- err := ipldbindcode.VerifyHash(completeData, ha)
- if err != nil {
- klog.Exitf("Error while verifying hash for %s: %s", block.Cid(), err)
- }
- }
- }
- var tx solana.Transaction
- if err := bin.UnmarshalBin(&tx, completeData); err != nil {
- klog.Exitf("Error while unmarshaling transaction from nodex %s: %s", block.Cid(), err)
- } else if len(tx.Signatures) == 0 {
- klog.Exitf("Error while unmarshaling transaction from nodex %s: no signatures", block.Cid())
- }
- return TransactionWithSlot{
- Slot: uint64(decoded.Slot),
- Transaction: tx,
- }
- } else {
- klog.Warningf("Transaction data is split into multiple objects for %s; skipping", block.Cid())
- }
- }
- return nil
-}
diff --git a/cmd-x-index-sig-exists.go b/cmd-x-index-sig-exists.go
index 37517139..1b2a953f 100644
--- a/cmd-x-index-sig-exists.go
+++ b/cmd-x-index-sig-exists.go
@@ -145,7 +145,6 @@ func newCmd_Index_sigExists() *cli.Command {
dotEvery := 100_000
klog.Infof("A dot is printed every %s transactions", humanize.Comma(int64(dotEvery)))
- verifyHash = c.Bool("verify-hash")
numWorkers := c.Uint("w")
if numWorkers == 0 {
diff --git a/cmd-x-index-sig2cid.go b/cmd-x-index-sig2cid.go
index 1755c633..11efad2d 100644
--- a/cmd-x-index-sig2cid.go
+++ b/cmd-x-index-sig2cid.go
@@ -3,6 +3,7 @@ package main
import (
"context"
"fmt"
+ "os"
"time"
"github.com/rpcpool/yellowstone-faithful/indexes"
@@ -33,7 +34,7 @@ func newCmd_Index_sig2cid() *cli.Command {
&cli.StringFlag{
Name: "tmp-dir",
Usage: "temporary directory to use for storing intermediate files",
- Value: "",
+ Value: os.TempDir(),
},
&cli.Uint64Flag{
Name: "epoch",
diff --git a/cmd-x-index-slot2cid.go b/cmd-x-index-slot2cid.go
index 0b09b941..5b876bb5 100644
--- a/cmd-x-index-slot2cid.go
+++ b/cmd-x-index-slot2cid.go
@@ -3,6 +3,7 @@ package main
import (
"context"
"fmt"
+ "os"
"time"
"github.com/rpcpool/yellowstone-faithful/indexes"
@@ -33,7 +34,7 @@ func newCmd_Index_slot2cid() *cli.Command {
&cli.StringFlag{
Name: "tmp-dir",
Usage: "temporary directory to use for storing intermediate files",
- Value: "",
+ Value: os.TempDir(),
},
&cli.Uint64Flag{
Name: "epoch",
diff --git a/compactindexsized/build.go b/compactindexsized/build.go
index 4dfda8f4..77c0d6e4 100644
--- a/compactindexsized/build.go
+++ b/compactindexsized/build.go
@@ -40,7 +40,7 @@ type Builder struct {
func NewBuilderSized(
tmpDir string,
numItems uint,
- valueSize uint,
+ valueSizeBytes uint,
) (*Builder, error) {
if tmpDir == "" {
var err error
@@ -49,11 +49,11 @@ func NewBuilderSized(
return nil, fmt.Errorf("failed to create temp dir: %w", err)
}
}
- if valueSize == 0 {
- return nil, fmt.Errorf("valueSize must be > 0")
+ if valueSizeBytes == 0 {
+ return nil, fmt.Errorf("valueSizeBytes must be > 0")
}
- if valueSize > 256 {
- return nil, fmt.Errorf("valueSize must be <= 256")
+ if valueSizeBytes > 255 {
+ return nil, fmt.Errorf("valueSizeBytes must be <= 255")
}
if numItems == 0 {
return nil, fmt.Errorf("numItems must be > 0")
@@ -74,12 +74,12 @@ func NewBuilderSized(
closers = append(closers, f)
buckets[i].file = f
buckets[i].writer = bufio.NewWriter(f)
- buckets[i].valueSize = uint(valueSize)
+ buckets[i].valueSize = uint(valueSizeBytes)
}
return &Builder{
Header: Header{
- ValueSize: uint64(valueSize),
+ ValueSize: uint64(valueSizeBytes),
NumBuckets: uint32(numBuckets),
Metadata: &indexmeta.Meta{},
},
diff --git a/compactindexsized/query.go b/compactindexsized/query.go
index 4130f9fb..d49a6576 100644
--- a/compactindexsized/query.go
+++ b/compactindexsized/query.go
@@ -237,6 +237,10 @@ func (b *Bucket) loadEntry(i int) (Entry, error) {
// ErrNotFound marks a missing entry.
var ErrNotFound = errors.New("not found")
+func IsNotFound(err error) bool {
+ return errors.Is(err, ErrNotFound)
+}
+
func searchEytzinger(min int, max int, x uint64, getter func(int) (Entry, error)) ([]byte, error) {
var index int
for index < max {
@@ -251,6 +255,9 @@ func searchEytzinger(min int, max int, x uint64, getter func(int) (Entry, error)
if k.Hash < x {
index++
}
+ if index < min {
+ return nil, ErrNotFound
+ }
}
return nil, ErrNotFound
}
diff --git a/epoch.go b/epoch.go
index 3e85ecdd..b2414063 100644
--- a/epoch.go
+++ b/epoch.go
@@ -23,6 +23,7 @@ import (
cidlink "github.com/ipld/go-ipld-prime/linking/cid"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/rpcpool/yellowstone-faithful/bucketteer"
+ "github.com/rpcpool/yellowstone-faithful/carreader"
deprecatedbucketter "github.com/rpcpool/yellowstone-faithful/deprecated/bucketteer"
"github.com/rpcpool/yellowstone-faithful/gsfa"
hugecache "github.com/rpcpool/yellowstone-faithful/huge-cache"
@@ -420,7 +421,7 @@ func NewEpochFromConfig(
if err != nil {
return nil, fmt.Errorf("failed to get local CAR data reader: %w", err)
}
- header, err := readHeader(dr)
+ header, err := carreader.ReadHeader(dr)
if err != nil {
return nil, fmt.Errorf("failed to read local CAR header: %w", err)
}
@@ -622,7 +623,7 @@ func (s *Epoch) GetNodeByCid(ctx context.Context, wantedCid cid.Cid) ([]byte, er
// not found or error
return nil, fmt.Errorf("failed to find offset for CID %s: %w", wantedCid, err)
}
- return s.GetNodeByOffsetAndSize(ctx, wantedCid, oas)
+ return s.GetNodeByOffsetAndSize(ctx, &wantedCid, oas)
}
func (s *Epoch) ReadAtFromCar(ctx context.Context, offset uint64, length uint64) ([]byte, error) {
@@ -647,7 +648,7 @@ func (s *Epoch) ReadAtFromCar(ctx context.Context, offset uint64, length uint64)
return data, nil
}
-func (s *Epoch) GetNodeByOffsetAndSize(ctx context.Context, wantedCid cid.Cid, offsetAndSize *indexes.OffsetAndSize) ([]byte, error) {
+func (s *Epoch) GetNodeByOffsetAndSize(ctx context.Context, wantedCid *cid.Cid, offsetAndSize *indexes.OffsetAndSize) ([]byte, error) {
if offsetAndSize == nil {
return nil, fmt.Errorf("offsetAndSize must not be nil")
}
@@ -706,7 +707,7 @@ func readNodeSizeFromReaderAtWithOffset(reader io.ReaderAt, offset uint64) (uint
return dataLen, nil
}
-func readNodeWithKnownSize(br *bufio.Reader, wantedCid cid.Cid, length uint64) ([]byte, error) {
+func readNodeWithKnownSize(br *bufio.Reader, wantedCid *cid.Cid, length uint64) ([]byte, error) {
section := make([]byte, length)
_, err := io.ReadFull(br, section)
if err != nil {
@@ -715,7 +716,7 @@ func readNodeWithKnownSize(br *bufio.Reader, wantedCid cid.Cid, length uint64) (
return parseNodeFromSection(section, wantedCid)
}
-func parseNodeFromSection(section []byte, wantedCid cid.Cid) ([]byte, error) {
+func parseNodeFromSection(section []byte, wantedCid *cid.Cid) ([]byte, error) {
// read an uvarint from the buffer
gotLen, usize := binary.Uvarint(section)
if usize <= 0 {
@@ -730,7 +731,7 @@ func parseNodeFromSection(section []byte, wantedCid cid.Cid) ([]byte, error) {
return nil, fmt.Errorf("failed to read cid: %w", err)
}
// verify that the CID we read matches the one we expected.
- if !gotCid.Equals(wantedCid) {
+ if wantedCid != nil && !gotCid.Equals(*wantedCid) {
return nil, fmt.Errorf("CID mismatch: expected %s, got %s", wantedCid, gotCid)
}
return data[cidLen:], nil
diff --git a/err.go b/err.go
new file mode 100644
index 00000000..1ccdf958
--- /dev/null
+++ b/err.go
@@ -0,0 +1,326 @@
+package main
+
+import (
+ metalatest "github.com/rpcpool/yellowstone-faithful/parse_legacy_transaction_status_meta/v-latest"
+ metaoldest "github.com/rpcpool/yellowstone-faithful/parse_legacy_transaction_status_meta/v-oldest"
+ "github.com/rpcpool/yellowstone-faithful/third_party/solana_proto/confirmed_block"
+)
+
+func getErr(meta any) any {
+ switch metaValue := meta.(type) {
+ case *confirmed_block.TransactionStatusMeta:
+ out, _ := parseTransactionError(metaValue.Err)
+ return out
+ case *metalatest.TransactionStatusMeta:
+ switch status := metaValue.Status.(type) {
+ case *metalatest.Result__Ok:
+ return nil // no error
+ case *metalatest.Result__Err:
+ switch err_ := status.Value.(type) {
+ case *metalatest.TransactionError__AccountInUse:
+ return map[string]any{
+ "AccountInUse": []any{0},
+ }
+ case *metalatest.TransactionError__AccountLoadedTwice:
+ return map[string]any{
+ "AccountLoadedTwice": []any{1},
+ }
+ case *metalatest.TransactionError__AccountNotFound:
+ return map[string]any{
+ "AccountNotFound": []any{2},
+ }
+ case *metalatest.TransactionError__ProgramAccountNotFound:
+ return map[string]any{
+ "ProgramAccountNotFound": []any{3},
+ }
+ case *metalatest.TransactionError__InsufficientFundsForFee:
+ return map[string]any{
+ "InsufficientFundsForFee": []any{4},
+ }
+ case *metalatest.TransactionError__InvalidAccountForFee:
+ return map[string]any{
+ "InvalidAccountForFee": []any{5},
+ }
+ case *metalatest.TransactionError__DuplicateSignature:
+ return map[string]any{
+ "DuplicateSignature": []any{6},
+ }
+ case *metalatest.TransactionError__BlockhashNotFound:
+ return map[string]any{
+ "BlockhashNotFound": []any{7},
+ }
+ case *metalatest.TransactionError__CallChainTooDeep:
+ return map[string]any{
+ "CallChainTooDeep": []any{8},
+ }
+ case *metalatest.TransactionError__MissingSignatureForFee:
+ return map[string]any{
+ "MissingSignatureForFee": []any{9},
+ }
+ case *metalatest.TransactionError__InvalidAccountIndex:
+ return map[string]any{
+ "InvalidAccountIndex": []any{10},
+ }
+ case *metalatest.TransactionError__SignatureFailure:
+ return map[string]any{
+ "SignatureFailure": []any{11},
+ }
+ case *metalatest.TransactionError__InvalidProgramForExecution:
+ return map[string]any{
+ "InvalidProgramForExecution": []any{12},
+ }
+ case *metalatest.TransactionError__SanitizeFailure:
+ return map[string]any{
+ "SanitizeFailure": []any{13},
+ }
+ case *metalatest.TransactionError__ClusterMaintenance:
+ return map[string]any{
+ "ClusterMaintenance": []any{14},
+ }
+
+ case *metalatest.TransactionError__InstructionError:
+ transactionErrorType := err_.Field0
+ instructionErrorType := err_.Field1
+
+ gotInstructionErrorName := func() any {
+ switch realInstructionError := instructionErrorType.(type) {
+ case *metalatest.InstructionError__GenericError:
+ return "GenericError"
+ case *metalatest.InstructionError__InvalidArgument:
+ return "InvalidArgument"
+ case *metalatest.InstructionError__InvalidInstructionData:
+ return "InvalidInstructionData"
+ case *metalatest.InstructionError__InvalidAccountData:
+ return "InvalidAccountData"
+ case *metalatest.InstructionError__AccountDataTooSmall:
+ return "AccountDataTooSmall"
+ case *metalatest.InstructionError__InsufficientFunds:
+ return "InsufficientFunds"
+ case *metalatest.InstructionError__IncorrectProgramId:
+ return "IncorrectProgramId"
+ case *metalatest.InstructionError__MissingRequiredSignature:
+ return "MissingRequiredSignature"
+ case *metalatest.InstructionError__AccountAlreadyInitialized:
+ return "AccountAlreadyInitialized"
+ case *metalatest.InstructionError__UninitializedAccount:
+ return "UninitializedAccount"
+ case *metalatest.InstructionError__UnbalancedInstruction:
+ return "UnbalancedInstruction"
+ case *metalatest.InstructionError__ModifiedProgramId:
+ return "ModifiedProgramId"
+ case *metalatest.InstructionError__ExternalAccountLamportSpend:
+ return "ExternalAccountLamportSpend"
+ case *metalatest.InstructionError__ExternalAccountDataModified:
+ return "ExternalAccountDataModified"
+ case *metalatest.InstructionError__ReadonlyLamportChange:
+ return "ReadonlyLamportChange"
+ case *metalatest.InstructionError__ReadonlyDataModified:
+ return "ReadonlyDataModified"
+ case *metalatest.InstructionError__DuplicateAccountIndex:
+ return "DuplicateAccountIndex"
+ case *metalatest.InstructionError__ExecutableModified:
+ return "ExecutableModified"
+ case *metalatest.InstructionError__RentEpochModified:
+ return "RentEpochModified"
+ case *metalatest.InstructionError__NotEnoughAccountKeys:
+ return "NotEnoughAccountKeys"
+ case *metalatest.InstructionError__AccountDataSizeChanged:
+ return "AccountDataSizeChanged"
+ case *metalatest.InstructionError__AccountNotExecutable:
+ return "AccountNotExecutable"
+ case *metalatest.InstructionError__AccountBorrowFailed:
+ return "AccountBorrowFailed"
+ case *metalatest.InstructionError__AccountBorrowOutstanding:
+ return "AccountBorrowOutstanding"
+ case *metalatest.InstructionError__DuplicateAccountOutOfSync:
+ return "DuplicateAccountOutOfSync"
+ case *metalatest.InstructionError__Custom:
+ return map[string]any{
+ "Custom": realInstructionError,
+ }
+ case *metalatest.InstructionError__InvalidError:
+ return "InvalidError"
+ case *metalatest.InstructionError__ExecutableDataModified:
+ return "ExecutableDataModified"
+ case *metalatest.InstructionError__ExecutableLamportChange:
+ return "ExecutableLamportChange"
+ case *metalatest.InstructionError__ExecutableAccountNotRentExempt:
+ return "ExecutableAccountNotRentExempt"
+ case *metalatest.InstructionError__UnsupportedProgramId:
+ return "UnsupportedProgramId"
+ case *metalatest.InstructionError__CallDepth:
+ return "CallDepth"
+ case *metalatest.InstructionError__MissingAccount:
+ return "MissingAccount"
+ case *metalatest.InstructionError__ReentrancyNotAllowed:
+ return "ReentrancyNotAllowed"
+ case *metalatest.InstructionError__MaxSeedLengthExceeded:
+ return "MaxSeedLengthExceeded"
+ case *metalatest.InstructionError__InvalidSeeds:
+ return "InvalidSeeds"
+ case *metalatest.InstructionError__InvalidRealloc:
+ return "InvalidRealloc"
+ case *metalatest.InstructionError__ComputationalBudgetExceeded:
+ return "ComputationalBudgetExceeded"
+ default:
+ return map[string]any{
+ "unknown": []any{}, // unknown; could not parse
+ }
+ }
+ }()
+ return map[string]any{
+ "InstructionError": []any{
+ transactionErrorType,
+ gotInstructionErrorName,
+ },
+ }
+ }
+ }
+
+ case *metaoldest.TransactionStatusMeta:
+ switch status := metaValue.Status.(type) {
+ case *metaoldest.Result__Ok:
+ return nil // no error
+ case *metaoldest.Result__Err:
+ switch err_ := status.Value.(type) {
+ case *metaoldest.TransactionError__AccountInUse:
+ return map[string]any{
+ "AccountInUse": []any{0},
+ }
+ case *metaoldest.TransactionError__AccountLoadedTwice:
+ return map[string]any{
+ "AccountLoadedTwice": []any{1},
+ }
+ case *metaoldest.TransactionError__AccountNotFound:
+ return map[string]any{
+ "AccountNotFound": []any{2},
+ }
+ case *metaoldest.TransactionError__ProgramAccountNotFound:
+ return map[string]any{
+ "ProgramAccountNotFound": []any{3},
+ }
+ case *metaoldest.TransactionError__InsufficientFundsForFee:
+ return map[string]any{
+ "InsufficientFundsForFee": []any{4},
+ }
+ case *metaoldest.TransactionError__InvalidAccountForFee:
+ return map[string]any{
+ "InvalidAccountForFee": []any{5},
+ }
+ case *metaoldest.TransactionError__DuplicateSignature:
+ return map[string]any{
+ "DuplicateSignature": []any{6},
+ }
+ case *metaoldest.TransactionError__BlockhashNotFound:
+ return map[string]any{
+ "BlockhashNotFound": []any{7},
+ }
+ case *metaoldest.TransactionError__InstructionError:
+ transactionErrorType := err_.Field0
+ instructionErrorType := err_.Field1
+
+ gotInstructionErrorName := func() any {
+ switch realInstructionError := instructionErrorType.(type) {
+ case *metaoldest.InstructionError__GenericError:
+ return "GenericError"
+ case *metaoldest.InstructionError__InvalidArgument:
+ return "InvalidArgument"
+ case *metaoldest.InstructionError__InvalidInstructionData:
+ return "InvalidInstructionData"
+ case *metaoldest.InstructionError__InvalidAccountData:
+ return "InvalidAccountData"
+ case *metaoldest.InstructionError__AccountDataTooSmall:
+ return "AccountDataTooSmall"
+ case *metaoldest.InstructionError__InsufficientFunds:
+ return "InsufficientFunds"
+ case *metaoldest.InstructionError__IncorrectProgramId:
+ return "IncorrectProgramId"
+ case *metaoldest.InstructionError__MissingRequiredSignature:
+ return "MissingRequiredSignature"
+ case *metaoldest.InstructionError__AccountAlreadyInitialized:
+ return "AccountAlreadyInitialized"
+ case *metaoldest.InstructionError__UninitializedAccount:
+ return "UninitializedAccount"
+ case *metaoldest.InstructionError__UnbalancedInstruction:
+ return "UnbalancedInstruction"
+ case *metaoldest.InstructionError__ModifiedProgramId:
+ return "ModifiedProgramId"
+ case *metaoldest.InstructionError__ExternalAccountLamportSpend:
+ return "ExternalAccountLamportSpend"
+ case *metaoldest.InstructionError__ExternalAccountDataModified:
+ return "ExternalAccountDataModified"
+ case *metaoldest.InstructionError__ReadonlyLamportChange:
+ return "ReadonlyLamportChange"
+ case *metaoldest.InstructionError__ReadonlyDataModified:
+ return "ReadonlyDataModified"
+ case *metaoldest.InstructionError__DuplicateAccountIndex:
+ return "DuplicateAccountIndex"
+ case *metaoldest.InstructionError__ExecutableModified:
+ return "ExecutableModified"
+ case *metaoldest.InstructionError__RentEpochModified:
+ return "RentEpochModified"
+ case *metaoldest.InstructionError__NotEnoughAccountKeys:
+ return "NotEnoughAccountKeys"
+ case *metaoldest.InstructionError__AccountDataSizeChanged:
+ return "AccountDataSizeChanged"
+ case *metaoldest.InstructionError__AccountNotExecutable:
+ return "AccountNotExecutable"
+ case *metaoldest.InstructionError__AccountBorrowFailed:
+ return "AccountBorrowFailed"
+ case *metaoldest.InstructionError__AccountBorrowOutstanding:
+ return "AccountBorrowOutstanding"
+ case *metaoldest.InstructionError__DuplicateAccountOutOfSync:
+ return "DuplicateAccountOutOfSync"
+ case *metaoldest.InstructionError__CustomError:
+ return map[string]any{
+ "Custom": realInstructionError,
+ }
+ case *metaoldest.InstructionError__InvalidError:
+ return "InvalidError"
+ default:
+ return map[string]any{
+ "unknown": []any{}, // unknown; could not parse
+ }
+ }
+ }()
+ return map[string]any{
+ "InstructionError": []any{
+ transactionErrorType,
+ gotInstructionErrorName,
+ },
+ }
+ case *metaoldest.TransactionError__CallChainTooDeep:
+ return map[string]any{
+ "CallChainTooDeep": []any{8},
+ }
+ case *metaoldest.TransactionError__MissingSignatureForFee:
+ return map[string]any{
+ "MissingSignatureForFee": []any{9},
+ }
+ case *metaoldest.TransactionError__InvalidAccountIndex:
+ return map[string]any{
+ "InvalidAccountIndex": []any{10},
+ }
+ case *metaoldest.TransactionError__SignatureFailure:
+ return map[string]any{
+ "SignatureFailure": []any{11},
+ }
+ case *metaoldest.TransactionError__InvalidProgramForExecution:
+ return map[string]any{
+ "InvalidProgramForExecution": []any{12},
+ }
+ default:
+ return map[string]any{
+ "unknown": []any{}, // unknown; could not parse
+ }
+ }
+ }
+ default:
+ return map[string]any{
+ "unknown": []any{}, // unknown; could not parse
+ }
+ }
+ return map[string]any{
+ "unknown": []any{}, // unknown; could not parse
+ }
+}
diff --git a/go.mod b/go.mod
index aa684c21..071a1811 100644
--- a/go.mod
+++ b/go.mod
@@ -13,8 +13,8 @@ require (
github.com/dustin/go-humanize v1.0.1
github.com/filecoin-project/go-data-transfer/v2 v2.0.0-rc7 // indirect
github.com/filecoin-project/go-state-types v0.10.0 // indirect
- github.com/gagliardetto/binary v0.7.8
- github.com/gagliardetto/solana-go v1.8.4
+ github.com/gagliardetto/binary v0.8.0
+ github.com/gagliardetto/solana-go v1.10.0
github.com/google/uuid v1.6.0
github.com/hannahhoward/go-pubsub v1.0.0 // indirect
github.com/ipfs/go-blockservice v0.5.0 // indirect
@@ -72,9 +72,9 @@ require (
github.com/mostynb/zstdpool-freelist v0.0.0-20201229113212-927304c0c3b1
github.com/mr-tron/base58 v1.2.0
github.com/prometheus/client_golang v1.18.0
- github.com/ronanh/intcomp v1.1.0
github.com/ryanuber/go-glob v1.0.0
github.com/tejzpr/ordered-concurrently/v3 v3.0.1
+ github.com/tidwall/hashmap v1.8.1
github.com/valyala/fasthttp v1.47.0
github.com/ybbus/jsonrpc/v3 v3.1.5
golang.org/x/exp v0.0.0-20231006140011-7918f672742d
@@ -85,7 +85,6 @@ require (
)
require (
- contrib.go.opencensus.io/exporter/stackdriver v0.13.14 // indirect
filippo.io/edwards25519 v1.0.0 // indirect
github.com/Jorropo/jsync v1.0.1 // indirect
github.com/PuerkitoBio/purell v1.1.1 // indirect
@@ -104,7 +103,6 @@ require (
github.com/cskr/pubsub v1.0.2 // indirect
github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c // indirect
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect
- github.com/dfuse-io/logging v0.0.0-20210109005628-b97a57253f70 // indirect
github.com/docker/go-units v0.5.0 // indirect
github.com/elastic/gosigar v0.14.2 // indirect
github.com/fatih/color v1.14.1 // indirect
@@ -125,7 +123,6 @@ require (
github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 // indirect
github.com/godbus/dbus/v5 v5.1.0 // indirect
github.com/gogo/protobuf v1.3.2 // indirect
- github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect
github.com/google/gopacket v1.1.19 // indirect
github.com/google/pprof v0.0.0-20231023181126-ff6d637d2a7b // indirect
github.com/gorilla/websocket v1.5.0 // indirect
@@ -203,14 +200,13 @@ require (
github.com/rivo/uniseg v0.4.4 // indirect
github.com/russross/blackfriday/v2 v2.1.0 // indirect
github.com/spaolacci/murmur3 v1.1.0 // indirect
- github.com/streamingfast/logging v0.0.0-20221209193439-bff11742bf4c // indirect
- github.com/teris-io/shortid v0.0.0-20220617161101-71ec9f2aa569 // indirect
+ github.com/streamingfast/logging v0.0.0-20230608130331-f22c91403091 // indirect
github.com/valyala/bytebufferpool v1.0.0 // indirect
github.com/whyrusleeping/cbor v0.0.0-20171005072247-63513f603b11 // indirect
github.com/whyrusleeping/cbor-gen v0.0.0-20230818171029-f91ae536ca25 // indirect
github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 // indirect
+ github.com/zeebo/xxh3 v1.0.2 // indirect
go.mongodb.org/mongo-driver v1.11.2 // indirect
- go.opencensus.io v0.24.0 // indirect
go.opentelemetry.io/otel/metric v1.16.0 // indirect
go.uber.org/atomic v1.11.0 // indirect
go.uber.org/dig v1.17.1 // indirect
diff --git a/go.sum b/go.sum
index 8d6ad29e..9a3da629 100644
--- a/go.sum
+++ b/go.sum
@@ -3,37 +3,24 @@ cloud.google.com/go v0.31.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT
cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
cloud.google.com/go v0.37.0/go.mod h1:TS1dMSSfndXH133OKGwekG838Om/cQT0BUHV3HcBgoo=
cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU=
-cloud.google.com/go v0.43.0/go.mod h1:BOSR3VbTLkk6FDC/TcffxP4NF/FFBGA5ku+jvKOP7pg=
cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU=
cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY=
cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc=
cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0=
cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To=
-cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4=
cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M=
-cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk=
cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o=
cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE=
-cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc=
cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE=
-cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk=
-cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk=
cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I=
cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw=
-cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA=
cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw=
cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos=
-cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk=
-contrib.go.opencensus.io/exporter/stackdriver v0.12.6/go.mod h1:8x999/OcIPy5ivx/wDiV7Gx4D+VUPODf0mWRGRc5kSk=
-contrib.go.opencensus.io/exporter/stackdriver v0.13.4/go.mod h1:aXENhDJ1Y4lIg4EUaVTwzvYETVNZk10Pu26tevFKLUc=
-contrib.go.opencensus.io/exporter/stackdriver v0.13.14 h1:zBakwHardp9Jcb8sQHcHpXy/0+JIb1M8KjigCJzx7+4=
-contrib.go.opencensus.io/exporter/stackdriver v0.13.14/go.mod h1:5pSSGY0Bhuk7waTHuDf4aQ8D2DrhgETRo9fy6k3Xlzc=
dmitri.shuralyov.com/app/changes v0.0.0-20180602232624-0a106ad413e3/go.mod h1:Yl+fi1br7+Rr3LqpNJf1/uxUdtRUV+Tnj0o93V2B9MU=
dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
dmitri.shuralyov.com/html/belt v0.0.0-20180602232347-f7d459c86be0/go.mod h1:JLBrvjyP0v+ecvNYvCpyZgu5/xkfAUhi6wJj28eUfSU=
dmitri.shuralyov.com/service/change v0.0.0-20181023043359-a85b471d5412/go.mod h1:a1inKt/atXimZ4Mv927x+r7UpyzRUf4emIoiiSC2TN4=
dmitri.shuralyov.com/state v0.0.0-20180228185332-28bcc343414c/go.mod h1:0PRwlb0D6DFvNNtx+9ybjezNCa8XF0xaYcETyp6rHWU=
-filippo.io/edwards25519 v1.0.0-rc.1/go.mod h1:N1IkdkCkiLB6tki+MYJoSx2JTY9NUlxZE7eHn5EwJns=
filippo.io/edwards25519 v1.0.0 h1:0wAIcmJUqRdI8IJ/3eGi5/HwXZWPujYXXlkrQogz0Ek=
filippo.io/edwards25519 v1.0.0/go.mod h1:N1IkdkCkiLB6tki+MYJoSx2JTY9NUlxZE7eHn5EwJns=
git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg=
@@ -41,8 +28,6 @@ github.com/AlekSi/pointer v1.1.0 h1:SSDMPcXD9jSl8FPy9cRzoRaMJtm9g9ggGTxecRUbQoI=
github.com/AlekSi/pointer v1.1.0/go.mod h1:y7BvfRI3wXPWKXEBhU71nbnIEEZX0QTSB2Bj48UJIZE=
github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo=
-github.com/GeertJohan/go.incremental v1.0.0/go.mod h1:6fAjUhbVuX1KcMD3c8TEgVUqmo4seqhv0i0kdATSkM0=
-github.com/GeertJohan/go.rice v1.0.0/go.mod h1:eH6gbSOAUv07dQuZVnBmoDP8mgsM1rtixis4Tib9if0=
github.com/Jorropo/jsync v1.0.1 h1:6HgRolFZnsdfzRUj+ImB9og1JYOxQoReSywkHOGSaUU=
github.com/Jorropo/jsync v1.0.1/go.mod h1:jCOZj3vrBCri3bSU3ErUYvevKlnbssrXeCivybS5ABQ=
github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE=
@@ -55,9 +40,6 @@ github.com/VividCortex/ewma v1.2.0 h1:f58SaIzcDXrSy3kWaHNvuJgJ3Nmz59Zji6XoJR/q1o
github.com/VividCortex/ewma v1.2.0/go.mod h1:nz4BbCtbLyFDeC9SUHbtcT5644juEuWfUAUnGx7j5l4=
github.com/acarl005/stripansi v0.0.0-20180116102854-5a71ef0e047d h1:licZJFw2RwpHMqeKTCYkitsPqHNxTmd4SNR5r94FGM8=
github.com/acarl005/stripansi v0.0.0-20180116102854-5a71ef0e047d/go.mod h1:asat636LX7Bqt5lYEZ27JNDcqxfjdBQuJ/MM4CN/Lzo=
-github.com/akavel/rsrc v0.8.0/go.mod h1:uLoCtb9J+EyAqh+26kdrTgmzRBFPGOolLWKpdxkKq+c=
-github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
-github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137 h1:s6gZFSlWYmbqAuRjVTiNNhvNRfY2Wxp9nhfyel4rklc=
github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE=
github.com/allegro/bigcache/v3 v3.1.0 h1:H2Vp8VOvxcrB91o86fUSVJFqeuz8kpyyB02eH3bSzwk=
@@ -67,28 +49,19 @@ github.com/andres-erbsen/clock v0.0.0-20160526145045-9e14626cd129/go.mod h1:rFgp
github.com/andybalholm/brotli v1.0.5 h1:8uQZIdzKmjc/iuPu7O2ioW48L81FgatrcpfFmiq/cCs=
github.com/andybalholm/brotli v1.0.5/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig=
github.com/anmitsu/go-shlex v0.0.0-20161002113705-648efa622239/go.mod h1:2FmKhYUyUczH0OGQWaF5ceTx0UBShxjsH6f8oGKYe2c=
-github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o=
-github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY=
-github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8=
-github.com/aws/aws-sdk-go v1.22.1/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
-github.com/aws/aws-sdk-go v1.23.20/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA=
github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA=
github.com/benbjohnson/clock v1.3.5 h1:VvXlSJBzZpA/zum6Sj74hxwYI2DIxRWuNIoXAzHZz5o=
github.com/benbjohnson/clock v1.3.5/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA=
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
-github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
github.com/bep/debounce v1.2.1 h1:v67fRdBA9UQu2NhLFXrSg0Brw7CexQekrBwDMM8bzeY=
github.com/bep/debounce v1.2.1/go.mod h1:H8yggRPQKLUhUoqrJC1bO2xNya7vanpDl7xR3ISbCJ0=
-github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
-github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84=
github.com/blendle/zapdriver v1.3.1 h1:C3dydBOWYRiOk+B8X9IVZ5IOe+7cl+tGOexN4QqHfpE=
github.com/blendle/zapdriver v1.3.1/go.mod h1:mdXfREi6u5MArG4j9fewC+FGnXaBR+T4Ox4J2u4eHCc=
github.com/bradfitz/go-smtpd v0.0.0-20170404230938-deb6d6237625/go.mod h1:HYsPBTaaSFSlLx/70C2HPIMNZpVV8+vt/A+FMnYP11g=
github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s=
-github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0=
github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko=
github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc=
@@ -99,19 +72,13 @@ github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5P
github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU=
github.com/cilium/ebpf v0.2.0/go.mod h1:To2CFviqOWL/M0gIMsvSMlqe7em/l1ALkX1PyjrX2Qs=
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
-github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc=
github.com/containerd/cgroups v0.0.0-20201119153540-4cbc285b3327/go.mod h1:ZJeTFisyysqgcCdecO57Dj79RfL0LNeGiFUqLYQRYLE=
github.com/containerd/cgroups v1.1.0 h1:v8rEWFl6EoqHB+swVNjVoCJE8o3jX7e8nqBGPLaDFBM=
github.com/containerd/cgroups v1.1.0/go.mod h1:6ppBcbh/NOOUU+dMKrykgaBnK9lCIBxHqJDGwsa1mIw=
-github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk=
-github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE=
-github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk=
github.com/coreos/go-systemd v0.0.0-20181012123002-c6f51f82210d/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
-github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
github.com/coreos/go-systemd/v22 v22.1.0/go.mod h1:xO0FLkIi5MaZafQlIrOotqXZ90ih+1atmu1JpKERPPk=
github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs=
github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc=
-github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU=
github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU=
github.com/cpuguy83/go-md2man/v2 v2.0.2 h1:p1EgwI/C7NhT0JmVkwCD2ZBK8j4aeHQX2pMHHBfMQ6w=
@@ -120,7 +87,6 @@ github.com/crackcomm/go-gitignore v0.0.0-20170627025303-887ab5e44cc3 h1:HVTnpeuv
github.com/crackcomm/go-gitignore v0.0.0-20170627025303-887ab5e44cc3/go.mod h1:p1d6YEZWvFzEh4KLyvBcVSnrfNDDvK2zfK/4x2v/4pE=
github.com/cskr/pubsub v1.0.2 h1:vlOzMhl6PFn60gRlTQQsIfVwaPB/B/8MziK8FhEPt/0=
github.com/cskr/pubsub v1.0.2/go.mod h1:/8MzYXk/NJAz782G8RPkFzXTZVu63VotefPnR9TIRis=
-github.com/daaku/go.zipexe v1.0.0/go.mod h1:z8IiR6TsVLEYKwXAoE/I+8ys/sDkgTzSL0CLnGVd57E=
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
@@ -130,11 +96,6 @@ github.com/decred/dcrd/crypto/blake256 v1.0.1 h1:7PltbUIQB7u/FfZ39+DGa/ShuMyJ5il
github.com/decred/dcrd/crypto/blake256 v1.0.1/go.mod h1:2OfgNZ5wDpcsFmHmCK5gZTPcCXqlm2ArzUIkw9czNJo=
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 h1:8UrgZ3GkP4i/CLijOJx79Yu+etlyjdBU4sfcs2WYQMs=
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0/go.mod h1:v57UDF4pDQJcEfFUCRop3lJL149eHGSe9Jvczhzjo/0=
-github.com/dfuse-io/logging v0.0.0-20201110202154-26697de88c79/go.mod h1:V+ED4kT/t/lKtH99JQmKIb0v9WL3VaYkJ36CfHlVECI=
-github.com/dfuse-io/logging v0.0.0-20210109005628-b97a57253f70 h1:CuJS05R9jmNlUK8GOxrEELPbfXm0EuGh/30LjkjN5vo=
-github.com/dfuse-io/logging v0.0.0-20210109005628-b97a57253f70/go.mod h1:EoK/8RFbMEteaCaz89uessDTnCWjbbcr+DXcBh4el5o=
-github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
-github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no=
github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk=
github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4=
github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk=
@@ -144,12 +105,8 @@ github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+m
github.com/elastic/gosigar v0.12.0/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs=
github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/4=
github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs=
-github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
-github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98=
github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
-github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4=
-github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU=
github.com/fatih/color v1.14.1 h1:qfhVLaG5s+nCROl1zJsZRxFeYrHLqWroPOQ8BWiNb4w=
github.com/fatih/color v1.14.1/go.mod h1:2oHN61fhTpgcxD3TSWCgKDiH1+x4OiDVVGH8WlgGZGg=
github.com/filecoin-project/filecoin-ffi v0.30.4-0.20200910194244-f640612a1a1f/go.mod h1:+If3s2VxyjZn+KGGZIoRXBDSFQ9xL404JBJGf4WhEj0=
@@ -211,12 +168,10 @@ github.com/frankban/quicktest v1.14.6/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7z
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
github.com/fsnotify/fsnotify v1.5.4 h1:jRbGcIw6P2Meqdwuo0H1p6JVLbL5DHKAKlYndzMwVZI=
github.com/fsnotify/fsnotify v1.5.4/go.mod h1:OVB6XrOHzAwXMpEM7uPOzcehqUV2UqJxmVXmkdnm1bU=
-github.com/gagliardetto/binary v0.7.7/go.mod h1:mUuay5LL8wFVnIlecHakSZMvcdqfs+CsotR5n77kyjM=
-github.com/gagliardetto/binary v0.7.8 h1:hbIUIP8BWhPm/BIdODxY2Lnv4NlJwNdbtsi1xkhNOec=
-github.com/gagliardetto/binary v0.7.8/go.mod h1:Cn70Gnvyk1OWkNJXwVh3oYqSYhKLHJN+C/Wguw3fc3U=
-github.com/gagliardetto/gofuzz v1.2.2/go.mod h1:bkH/3hYLZrMLbfYWA0pWzXmi5TTRZnu4pMGZBkqMKvY=
-github.com/gagliardetto/solana-go v1.8.4 h1:vmD/JmTlonyXGy39bAo0inMhmbdAwV7rXZtLDMZeodE=
-github.com/gagliardetto/solana-go v1.8.4/go.mod h1:i+7aAyNDTHG0jK8GZIBSI4OVvDqkt2Qx+LklYclRNG8=
+github.com/gagliardetto/binary v0.8.0 h1:U9ahc45v9HW0d15LoN++vIXSJyqR/pWw8DDlhd7zvxg=
+github.com/gagliardetto/binary v0.8.0/go.mod h1:2tfj51g5o9dnvsc+fL3Jxr22MuWzYXwx9wEoN0XQ7/c=
+github.com/gagliardetto/solana-go v1.10.0 h1:lDuHGC+XLxw9j8fCHBZM9tv4trI0PVhev1m9NAMaIdM=
+github.com/gagliardetto/solana-go v1.10.0/go.mod h1:afBEcIRrDLJst3lvAahTr63m6W2Ns6dajZxe2irF7Jg=
github.com/gagliardetto/treeout v0.1.4 h1:ozeYerrLCmCubo1TcIjFiOWTTGteOOHND1twdFpgwaw=
github.com/gagliardetto/treeout v0.1.4/go.mod h1:loUefvXTrlRG5rYmJmExNryyBRh8f89VZhmMOyCyqok=
github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
@@ -225,10 +180,6 @@ github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclK
github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q=
github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
-github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
-github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
-github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
-github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas=
github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
@@ -236,7 +187,6 @@ github.com/go-logr/logr v1.2.4 h1:g01GSCwiDw2xSZfjJ2/T9M+S6pFdcNtFYsp+Y43HYDQ=
github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag=
github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE=
-github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 h1:tfuBGBXKqDEevZMzYi5KSi8KkcZtzBcTgAUUtapy0OI=
github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572/go.mod h1:9Pwr4B2jHnOSGXyyzV8ROjYa2ojvAY6HCGYYfMoC3Ls=
github.com/go-yaml/yaml v2.1.0+incompatible/go.mod h1:w2MrLa16VYP0jy6N7M5kHaCkaLENm+P+Tv+MfurjSw0=
@@ -250,33 +200,21 @@ github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXP
github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q=
github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q=
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
-github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
-github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE=
-github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E=
github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y=
github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw=
-github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw=
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw=
-github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk=
-github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
-github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
-github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
-github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
-github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
-github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8=
-github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
-github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
-github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek=
-github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps=
+github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
+github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg=
+github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
@@ -284,7 +222,6 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a
github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
-github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI=
@@ -297,14 +234,11 @@ github.com/google/gopacket v1.1.19/go.mod h1:iJ8V8n6KS+z2U1A8pUwu8bW5SyEMkXJB8Yo
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
-github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM=
github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM=
-github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM=
github.com/google/pprof v0.0.0-20231023181126-ff6d637d2a7b h1:RMpPgZTSApbPf7xaVel+QkoGPRLFLrwFO89uDUHEGf0=
github.com/google/pprof v0.0.0-20231023181126-ff6d637d2a7b/go.mod h1:czg5+yv1E0ZGTi6S6vVK1mke0fV+FaUhNGcd6VRS9Ik=
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0=
github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY=
@@ -314,58 +248,33 @@ github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5m
github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
github.com/gopherjs/gopherjs v0.0.0-20190812055157-5d271430af9f h1:KMlcu9X58lhTA/KrfX8Bi1LQSO4pzoVjTiL3h4Jk+Zk=
github.com/gopherjs/gopherjs v0.0.0-20190812055157-5d271430af9f/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
-github.com/gorilla/rpc v1.2.0/go.mod h1:V4h9r+4sF5HnzqbwIez0fKSpANP0zlYd3qR7p36jkTQ=
github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
-github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
github.com/gorilla/websocket v1.5.0 h1:PPwGk2jz7EePpoHN/+ClbZu8SPxiqlu12wZP/3sWmnc=
github.com/gorilla/websocket v1.5.0/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
github.com/goware/urlx v0.3.2 h1:gdoo4kBHlkqZNaf6XlQ12LGtQOmpKJrR04Rc3RnpJEo=
github.com/goware/urlx v0.3.2/go.mod h1:h8uwbJy68o+tQXCGZNa9D73WN8n0r9OBae5bUnLcgjw=
github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA=
-github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs=
-github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk=
github.com/grpc-ecosystem/grpc-gateway v1.5.0/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw=
-github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
github.com/gxed/hashland/keccakpg v0.0.1/go.mod h1:kRzw3HkwxFU1mpmPP8v1WyQzwdGfmKFJ6tItnhQ67kU=
github.com/gxed/hashland/murmur3 v0.0.1/go.mod h1:KjXop02n4/ckmZSnY2+HKcLud/tcmvhST0bie/0lS48=
github.com/hannahhoward/cbor-gen-for v0.0.0-20230214144701-5d17c9d5243c h1:iiD+p+U0M6n/FsO6XIZuOgobnNa48FxtyYFfWwLttUQ=
github.com/hannahhoward/cbor-gen-for v0.0.0-20230214144701-5d17c9d5243c/go.mod h1:jvfsLIxk0fY/2BKSQ1xf2406AKA5dwMmKKv0ADcOfN8=
github.com/hannahhoward/go-pubsub v1.0.0 h1:yONMbY9blu+FFlamGzRZVocoY6WHPJa08h3yX7nOGuA=
github.com/hannahhoward/go-pubsub v1.0.0/go.mod h1:3lHsAt5uM7YFHauT5whoifwfgIgVwEX2fMDxPDrkpU4=
-github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q=
-github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8=
github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I=
github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
-github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80=
-github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
-github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM=
-github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk=
github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo=
github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM=
-github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU=
-github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU=
-github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4=
-github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
-github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
-github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90=
github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
-github.com/hashicorp/golang-lru v0.5.3/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
github.com/hashicorp/golang-lru v0.5.4 h1:YDjusn29QI/Das2iO9M0BHnIbxPeyuCHsjMW+lJfyTc=
github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
github.com/hashicorp/golang-lru/v2 v2.0.5 h1:wW7h1TG88eUIJ2i69gaE3uNVtEPIagzhGvHgwfx2Vm4=
github.com/hashicorp/golang-lru/v2 v2.0.5/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM=
-github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ=
-github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64=
-github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ=
-github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I=
-github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc=
-github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
github.com/huin/goupnp v1.3.0 h1:UvLUlWDNpoUdYzb2TCn+MuTWtcjXKSza2n6CBdQ0xXc=
github.com/huin/goupnp v1.3.0/go.mod h1:gnGPsThkYa7bFi/KWmEysQRf48l2dvR5bxr2OFckNX8=
github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc=
-github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
github.com/ipfs/bbloom v0.0.4 h1:Gi+8EGJ2y5qiD5FbsbpX/TMNcJw8gSqr7eyjHa4Fhvs=
github.com/ipfs/bbloom v0.0.4/go.mod h1:cS9YprKXpoZ9lT0n/Mw/a6/aFV6DTjTLYHeA+gyqMG0=
github.com/ipfs/boxo v0.11.1-0.20230817065640-7ec68c5e5adf h1:toUvJ0yELWjrVmFX8AdriAfzl/EtqvYrpkfEniAJiFo=
@@ -498,9 +407,6 @@ github.com/jbenet/goprocess v0.1.4/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZl
github.com/jellevandenhooff/dkim v0.0.0-20150330215556-f50fe3d243e1/go.mod h1:E0B/fFc00Y+Rasa88328GlI/XbtyysCtTHZS8h7IrBU=
github.com/jellydator/ttlcache/v3 v3.1.0 h1:0gPFG0IHHP6xyUyXq+JaD8fwkDCqgqwohXNJBcYE71g=
github.com/jellydator/ttlcache/v3 v3.1.0/go.mod h1:hi7MGFdMAwZna5n2tuvh63DvFLzVKySzCVW6+0gA2n4=
-github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI=
-github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k=
-github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA=
github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4=
github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
@@ -511,7 +417,6 @@ github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/X
github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU=
github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo=
github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU=
-github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w=
github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q=
github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00=
github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
@@ -523,10 +428,8 @@ github.com/klauspost/compress v1.17.2/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQs
github.com/klauspost/cpuid/v2 v2.0.4/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg=
github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg=
github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws=
-github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
github.com/koron/go-ssdp v0.0.4 h1:1IDwrghSKYM7yLf7XCzbByg2sJ/JcNOZRXS2jczTwz0=
github.com/koron/go-ssdp v0.0.4/go.mod h1:oDXq+E5IL5q0U8uSBcoAXzTzInwy5lEgC91HoKtbmZk=
-github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
@@ -566,17 +469,11 @@ github.com/libp2p/go-yamux/v4 v4.0.1/go.mod h1:NWjl8ZTLOGlozrXSOZ/HlfG++39iKNnM5
github.com/logrusorgru/aurora v2.0.3+incompatible h1:tOpm7WcpBTn4fjmVfgpQq0EfczGlG91VSDkswnjF5A8=
github.com/logrusorgru/aurora v2.0.3+incompatible/go.mod h1:7rIyQOR62GCctdiQpZ/zOJlFyk6y+94wXzv6RNZgaR4=
github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI=
-github.com/magiconair/properties v1.8.1/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ=
github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd h1:br0buuQ854V8u83wA0rVZ8ttrq5CpaPZdvrK0LP2lOk=
github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd/go.mod h1:QuCEs1Nt24+FYQEqAAncTDPJIuGs+LxK1MCiFL25pMU=
-github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU=
-github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE=
github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA=
github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg=
-github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4=
-github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
-github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE=
github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94=
github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM=
github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY=
@@ -587,7 +484,6 @@ github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5
github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg=
github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k=
github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4=
-github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI=
github.com/miekg/dns v1.1.56 h1:5imZaSeoRNvpM9SzWNhEcP9QliKiz20/dA2QabIGVnE=
github.com/miekg/dns v1.1.56/go.mod h1:cRm6Oo2C8TY9ZS/TqsSrseAcncm74lfK5G+ikN2SWWY=
@@ -605,16 +501,8 @@ github.com/minio/sha256-simd v0.1.1/go.mod h1:B5e1o+1/KgNmWrSQK08Y6Z1Vb5pwIktudl
github.com/minio/sha256-simd v1.0.0/go.mod h1:OuYzVNI5vcoYIAmbIvHPl3N3jUzVedXbKy5RFepssQM=
github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM=
github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8=
-github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc=
-github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
-github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
-github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI=
github.com/mitchellh/go-testing-interface v1.14.1 h1:jrgshOhYAUVNMAJiKbEu7EqAwgJJ2JqpQmpLJOu07cU=
github.com/mitchellh/go-testing-interface v1.14.1/go.mod h1:gfgS7OtZj6MA4U1UrDRp04twqAjfvlZyCfX3sDjEym8=
-github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg=
-github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY=
-github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
-github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg=
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
@@ -665,20 +553,12 @@ github.com/multiformats/go-varint v0.0.5/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXS
github.com/multiformats/go-varint v0.0.6/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE=
github.com/multiformats/go-varint v0.0.7 h1:sWSGR+f/eu5ABZA2ZpYKBILXTTs9JWpdEM/nEGOHFS8=
github.com/multiformats/go-varint v0.0.7/go.mod h1:r8PUYw/fD/SjBCiKOoDlGF6QawOELpZAu9eioSos/OU=
-github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
github.com/neelance/astrewrite v0.0.0-20160511093645-99348263ae86/go.mod h1:kHJEU3ofeGjhHklVoIGuVj85JJwZ6kWPaJwCIxgnFmo=
github.com/neelance/sourcemap v0.0.0-20151028013722-8c68805598ab/go.mod h1:Qr6/a/Q4r9LP1IltGz7tA7iOK1WonHEYhu1HRBA7ZiM=
-github.com/nkovacs/streamquote v0.0.0-20170412213628-49af9bddb229/go.mod h1:0aYXnNPJ8l7uZxf45rWW1a/uME32OF0rhiYGNQ2oF2E=
github.com/novifinancial/serde-reflection/serde-generate/runtime/golang v0.0.0-20220519162058-e5cd3c3b3f3a h1:oMG8C4E7DFkat7WQicw4JNa/dYUaqO7RvLPbkFdADIA=
github.com/novifinancial/serde-reflection/serde-generate/runtime/golang v0.0.0-20220519162058-e5cd3c3b3f3a/go.mod h1:NrRYJCFtaewjIRr4B9V2AyWsAEMW0Zqdjs8Bm+bACbM=
-github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A=
-github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U=
-github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
-github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk=
github.com/onsi/ginkgo/v2 v2.13.0 h1:0jY9lJquiL8fcf3M4LAXN5aMlS/b2BV86HFFPCPMgE4=
github.com/onsi/ginkgo/v2 v2.13.0/go.mod h1:TE309ZR8s5FsKKpuB1YAQYBzCaAfUgatB/xlT/ETL/o=
-github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY=
-github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo=
github.com/onsi/gomega v1.27.10 h1:naR28SdDFlqrG6kScpT8VWpu1xWY5nJRCF3XaYyBjhI=
github.com/onsi/gomega v1.27.10/go.mod h1:RsS8tutOdbdgzbPtzzATp12yT7kM5I5aElG3evPbQ0M=
github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0=
@@ -688,13 +568,10 @@ github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFSt
github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs=
github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc=
github.com/openzipkin/zipkin-go v0.1.1/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTmOf0Erfk+hxe8=
-github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 h1:onHthvaw9LFnH4t2DcNVpwGmV9E1BkGknEliJkfwQj0=
github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58/go.mod h1:DXv8WO4yhMYhSNPKjeNKa5WY9YCIEBRbNzFFPJbWO6Y=
-github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic=
github.com/petar/GoLLRB v0.0.0-20210522233825-ae3b015fd3e9 h1:1/WtZae0yGtPq+TI6+Tv1WTxkukpXeMlviSxvL7SRgk=
github.com/petar/GoLLRB v0.0.0-20210522233825-ae3b015fd3e9/go.mod h1:x3N5drFsm2uilKKuuYo6LdyD8vZAW55sH/9w+pbo1sw=
-github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
@@ -705,28 +582,19 @@ github.com/polydawn/refmt v0.0.0-20190807091052-3d65705ee9f1/go.mod h1:uIp+gprXx
github.com/polydawn/refmt v0.0.0-20190809202753-05966cbd336a/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o=
github.com/polydawn/refmt v0.89.0 h1:ADJTApkvkeBZsN0tBTx8QjpD9JkmxbKp0cxfr9qszm4=
github.com/polydawn/refmt v0.89.0/go.mod h1:/zvteZs/GwLtCgZ4BL6CBsk9IKIlexP43ObX9AxTqTw=
-github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI=
github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
-github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
-github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso=
github.com/prometheus/client_golang v1.18.0 h1:HzFfmkOzH5Q8L8G+kSJKUx5dtG87sewO+FoDDqP5Tbk=
github.com/prometheus/client_golang v1.18.0/go.mod h1:T+GXkCk5wSJyOqMIzVgvvjFDlkOQntgjkJWKrN5txjA=
github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
-github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw=
github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI=
github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro=
-github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro=
-github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM=
github.com/prometheus/common v0.45.0/go.mod h1:YJmSTw9BoKxJplESWWxlbyttQR4uaEcGyv9MZjVOJsY=
github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
-github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
-github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo=
github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo=
-github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU=
github.com/quic-go/qpack v0.4.0 h1:Cr9BXA1sQS2SmDUWjSofMPNKmvF6IiIfDRmgU0w1ZCo=
github.com/quic-go/qpack v0.4.0/go.mod h1:UZVnYIfi5GRk+zI9UMaCPsmZ2xKJP7XBUvVyT1Knj9A=
github.com/quic-go/qtls-go1-20 v0.4.1 h1:D33340mCNDAIKBqXuAvexTNMUByrYmFYVfKfDN5nfFs=
@@ -741,12 +609,9 @@ github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qq
github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc=
github.com/rivo/uniseg v0.4.4 h1:8TfxU8dW6PdqD27gjM8MVNuicgxIjxpm4K7x4jp8sis=
github.com/rivo/uniseg v0.4.4/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88=
-github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ=
github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog=
-github.com/ronanh/intcomp v1.1.0 h1:i54kxmpmSoOZFcWPMWryuakN0vLxLswASsGa07zkvLU=
-github.com/ronanh/intcomp v1.1.0/go.mod h1:7FOLy3P3Zj3er/kVrU/pl+Ql7JFZj7bwliMGketo0IU=
github.com/rpcpool/carlet v0.0.4 h1:ZrDMvrS1Jewy4rQkj/ODy0SG8jXG0mljeNcY76kEnYg=
github.com/rpcpool/carlet v0.0.4/go.mod h1:sTpcN668YLQ1cvCT2RcIxifc/soFheI9wbNeDGCJX74=
github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g=
@@ -754,10 +619,8 @@ github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD
github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk=
github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
github.com/rwcarlsen/goexif v0.0.0-20190401172101-9e8deecbddbd/go.mod h1:hPqNNc0+uJM6H+SuU8sEs5K5IQeKccPqeSjfgcKGgPk=
-github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
github.com/ryanuber/go-glob v1.0.0 h1:iQh3xXAumdQ+4Ufa5b25cRpC5TYKlno6hsv6Cb3pkBk=
github.com/ryanuber/go-glob v1.0.0/go.mod h1:807d1WSdnB0XRJzKNil9Om6lcp/3a0v4qIHxIXzX/Yc=
-github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc=
github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo=
github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8=
github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o=
@@ -784,7 +647,6 @@ github.com/shurcooL/sanitized_anchor_name v0.0.0-20170918181015-86672fcb3f95/go.
github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc=
github.com/shurcooL/users v0.0.0-20180125191416-49c67e49c537/go.mod h1:QJTqeLYEDaXHZDBsXlPCDqdhQuJkuw4NOtaxYe3xii4=
github.com/shurcooL/webdavfs v0.0.0-20170829043945-18c3829fa133/go.mod h1:hKmq5kWdCj2z2KEozexVbfEZIWiTjhE0+UjmZgPqehw=
-github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0=
github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc=
github.com/smartystreets/assertions v1.0.1/go.mod h1:kHHU4qYBaI3q23Pp3VPrmWhuIUrLW/7eUrw0BU5VaoM=
@@ -793,10 +655,8 @@ github.com/smartystreets/assertions v1.13.0 h1:Dx1kYM01xsSqKPno3aqLnrwac2LetPvN2
github.com/smartystreets/assertions v1.13.0/go.mod h1:wDmR7qL282YbGsPy6H/yAsesrxfxaaSlJazyFLYVFx8=
github.com/smartystreets/goconvey v0.0.0-20190222223459-a17d461953aa/go.mod h1:2RVY1rIf+2J2o/IM9+vPq9RzmHDSseB7FoXiSNIUsoU=
github.com/smartystreets/goconvey v0.0.0-20190731233626-505e41936337/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA=
-github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA=
github.com/smartystreets/goconvey v1.7.2 h1:9RBaZCeXEQ3UselpuwUQHltGVXvdwm6cv1hgR6gDIPg=
github.com/smartystreets/goconvey v1.7.2/go.mod h1:Vw0tHAZW6lzCRk3xgdin6fKYcG+G3Pg9vgXWeJpQFMM=
-github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
github.com/sourcegraph/annotate v0.0.0-20160123013949-f4cad6c6324d/go.mod h1:UdhH50NIW0fCiwBSr0co2m7BnFLdv4fQTgdqdJTHFeE=
github.com/sourcegraph/jsonrpc2 v0.2.0 h1:KjN/dC4fP6aN9030MZCJs9WQbTOjWHhrtKVpzzSrr/U=
github.com/sourcegraph/jsonrpc2 v0.2.0/go.mod h1:ZafdZgk/axhT1cvZAPOhw+95nz2I/Ra5qMlU4gTRwIo=
@@ -804,48 +664,27 @@ github.com/sourcegraph/syntaxhighlight v0.0.0-20170531221838-bd320f5d308e/go.mod
github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI=
github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
-github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ=
-github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE=
-github.com/spf13/cobra v1.1.1/go.mod h1:WnodtKOvamDL/PwE2M4iKs8aMDBZ5Q5klgD3qfVJQMI=
-github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo=
-github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
-github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg=
-github.com/spf13/viper v1.7.0/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg=
-github.com/spf13/viper v1.7.1/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg=
-github.com/streamingfast/logging v0.0.0-20220405224725-2755dab2ce75/go.mod h1:VlduQ80JcGJSargkRU4Sg9Xo63wZD/l8A5NC/Uo1/uU=
-github.com/streamingfast/logging v0.0.0-20221209193439-bff11742bf4c h1:dV1ye/S2PiW9uIWvLtMrxWoTLcZS+yhjZDSKEV102Ho=
-github.com/streamingfast/logging v0.0.0-20221209193439-bff11742bf4c/go.mod h1:VlduQ80JcGJSargkRU4Sg9Xo63wZD/l8A5NC/Uo1/uU=
+github.com/streamingfast/logging v0.0.0-20230608130331-f22c91403091 h1:RN5mrigyirb8anBEtdjtHFIufXdacyTi6i4KBfeNXeo=
+github.com/streamingfast/logging v0.0.0-20230608130331-f22c91403091/go.mod h1:VlduQ80JcGJSargkRU4Sg9Xo63wZD/l8A5NC/Uo1/uU=
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
-github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
-github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
-github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=
github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
-github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
-github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
-github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4=
github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk=
github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo=
-github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw=
github.com/tarm/serial v0.0.0-20180830185346-98f6abe2eb07/go.mod h1:kDXzergiv9cbyO7IOYJZWg1U88JhDg3PB6klq9Hg2pA=
github.com/tejzpr/ordered-concurrently/v3 v3.0.1 h1:TLHtzlQEDshbmGveS8S+hxLw4s5u67aoJw5LLf+X2xY=
github.com/tejzpr/ordered-concurrently/v3 v3.0.1/go.mod h1:mu/neZ6AGXm5jdPc7PEgViYK3rkYNPvVCEm15Cx/iRI=
-github.com/teris-io/shortid v0.0.0-20171029131806-771a37caa5cf/go.mod h1:M8agBzgqHIhgj7wEn9/0hJUZcrvt9VY+Ln+S1I5Mha0=
-github.com/teris-io/shortid v0.0.0-20201117134242-e59966efd125/go.mod h1:M8agBzgqHIhgj7wEn9/0hJUZcrvt9VY+Ln+S1I5Mha0=
-github.com/teris-io/shortid v0.0.0-20220617161101-71ec9f2aa569 h1:xzABM9let0HLLqFypcxvLmlvEciCHL7+Lv+4vwZqecI=
-github.com/teris-io/shortid v0.0.0-20220617161101-71ec9f2aa569/go.mod h1:2Ly+NIftZN4de9zRmENdYbvPQeaVIYKWpLFStLFEBgI=
github.com/test-go/testify v1.1.4 h1:Tf9lntrKUMHiXQ07qBScBTSA0dhYQlu83hswqelv1iE=
github.com/test-go/testify v1.1.4/go.mod h1:rH7cfJo/47vWGdi4GPj16x3/t1xGOj2YxzmNQzk2ghU=
-github.com/tidwall/gjson v1.9.3/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk=
-github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM=
+github.com/tidwall/hashmap v1.8.1 h1:hXNzBfSJ2Jwvt0lbkWD59O/r3OfatSIcbuWT0VKEVns=
+github.com/tidwall/hashmap v1.8.1/go.mod h1:v+0qJrJn7l+l2dB8+fAFpC62p2G0SMP2Teu8ejkebg8=
github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk=
github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs=
github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU=
github.com/tj/go-spin v1.1.0/go.mod h1:Mg1mzmePZm4dva8Qz60H2lHwmJ2loum4VIrLgVnKwh4=
-github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
github.com/urfave/cli v1.22.10/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
github.com/urfave/cli/v2 v2.0.0/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ=
@@ -855,7 +694,6 @@ github.com/valyala/bytebufferpool v1.0.0 h1:GqA5TC/0021Y/b9FG4Oi9Mr3q7XYx6Kllzaw
github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc=
github.com/valyala/fasthttp v1.47.0 h1:y7moDoxYzMooFpT5aHgNgVOQDrS3qlkfiP9mDtGGK9c=
github.com/valyala/fasthttp v1.47.0/go.mod h1:k2zXd82h/7UZc3VOdJ2WaUqt1uZ/XpXAfE9i+HBC3lA=
-github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8=
github.com/vbauerster/mpb/v8 v8.2.1 h1:7V3DLM8rkK4BpgDUqu8l/ExBDfAfMbWOECW5phzVHx0=
github.com/vbauerster/mpb/v8 v8.2.1/go.mod h1:DqGePwrIYW6Bs5pXaGAuGgP0PYgu5VZKIjfLZkOsdZw=
github.com/viant/assertly v0.4.8/go.mod h1:aGifi++jvCrUaklKEKT0BU95igDNaqkvz+49uaYMPRU=
@@ -888,7 +726,6 @@ github.com/whyrusleeping/chunker v0.0.0-20181014151217-fe64bd25879f/go.mod h1:p9
github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI=
github.com/xdg-go/scram v1.1.1/go.mod h1:RaEWvsqvNKKvBPvcKeFjrG2cJqOkHTiyTpzz23ni57g=
github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgklLGvcBnW8=
-github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU=
github.com/xlab/c-for-go v0.0.0-20200718154222-87b0065af829/go.mod h1:h/1PEBwj7Ym/8kOuMWvO2ujZ6Lt+TMbySEXNhjjR87I=
github.com/xlab/pkgconfig v0.0.0-20170226114623-cea12a0fd245/go.mod h1:C+diUUz7pxhNY6KAoLgrTYARGWnt82zWTylZlxT92vk=
github.com/xorcare/golden v0.6.0/go.mod h1:7T39/ZMvaSEZlBPoYfVFmsBLmUl3uz9IuzWj/U6FtvQ=
@@ -897,25 +734,21 @@ github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673/go.mod h1:N3UwUGtsr
github.com/ybbus/jsonrpc/v3 v3.1.5 h1:0cC/QzS8OCuXYqqDbYnKKhsEe+IZLrNlDx8KPCieeW0=
github.com/ybbus/jsonrpc/v3 v3.1.5/go.mod h1:U1QbyNfL5Pvi2roT0OpRbJeyvGxfWYSgKJHjxWdAEeE=
github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d/go.mod h1:rHwXgn7JulP+udvsHwJoVG1YGAP6VLg4y9I5dyZdqmA=
-github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k=
-go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
-go.mongodb.org/mongo-driver v1.11.0/go.mod h1:s7p5vEtfbeR1gYi6pnj3c3/urpbLv2T5Sfd6Rp2HBB8=
+github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ=
+github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0=
+github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0=
+github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA=
go.mongodb.org/mongo-driver v1.11.2 h1:+1v2rDQUWNcGW7/7E0Jvdz51V38XXxJfhzbV17aNHCw=
go.mongodb.org/mongo-driver v1.11.2/go.mod h1:s7p5vEtfbeR1gYi6pnj3c3/urpbLv2T5Sfd6Rp2HBB8=
go.opencensus.io v0.18.0/go.mod h1:vKdFvxhtzZ9onBp9VKHK8z/sRpBMnKAsufL7wlDrCOA=
go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU=
go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8=
-go.opencensus.io v0.22.1/go.mod h1:Ap50jQcDJrx6rB6VgeeFPtuPIf3wMRvRfrfYDO6+BmA=
go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
-go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
-go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk=
-go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0=
-go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo=
go.opentelemetry.io/otel v1.16.0 h1:Z7GVAX/UkAXPKsy94IU+i6thsQS4nb7LviLpnaNeW8s=
go.opentelemetry.io/otel v1.16.0/go.mod h1:vl0h9NUa1D5s1nv3A5vZOYWn8av4K8Ml6JDeHrT/bx4=
go.opentelemetry.io/otel/metric v1.16.0 h1:RbrpwVG1Hfv85LgnZ7+txXioPDoh6EdbZHo26Q3hqOo=
@@ -941,7 +774,6 @@ go.uber.org/goleak v1.2.1/go.mod h1:qlT2yGI9QafXHhZZLxlSuNsMw3FFLxBr+tBRlmO1xH4=
go.uber.org/mock v0.3.0 h1:3mUxI1No2/60yUYax92Pt8eNOEecx2D3lcXZh2NEZJo=
go.uber.org/mock v0.3.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc=
go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
-go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU=
go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU=
@@ -951,7 +783,6 @@ go.uber.org/ratelimit v0.2.0 h1:UQE2Bgi7p2B85uP5dC2bbRtig0C+OeNRnNEafLjsLPA=
go.uber.org/ratelimit v0.2.0/go.mod h1:YYBV4e4naJvhpitQrWJu1vCpgB7CboMe0qhltKt6mUg=
go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA=
go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
-go.uber.org/zap v1.14.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM=
go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc=
go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ=
go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI=
@@ -962,8 +793,6 @@ go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1
go4.org v0.0.0-20200411211856-f5505b9728dd h1:BNJlw5kRTzdmyfh5U8F93HA2OwkP7ZGwA51eJ/0wKOU=
go4.org v0.0.0-20200411211856-f5505b9728dd/go.mod h1:CIiUVy99QCPfoE13bO4EZaz5GZMZXMSBGhxRdsvzbkg=
golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d/go.mod h1:OWs+y06UdEOHN4y+MfF/py+xQ/tYqIWW03b70/CG9Rw=
-golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
-golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20181030102418-4d3f4d9ffa16/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190211182817-74369b46fc67/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
@@ -988,9 +817,7 @@ golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm0
golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY=
golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4=
golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4=
-golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4=
golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM=
-golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU=
golang.org/x/exp v0.0.0-20231006140011-7918f672742d h1:jtJma62tbqLibJ5sFQz8bKtEM8rJBtfilJ2qTU199MI=
golang.org/x/exp v0.0.0-20231006140011-7918f672742d/go.mod h1:ldy0pHrwJyGW56pPQzzkH36rKxoZW1tw7ZJpeKx+hdo=
golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js=
@@ -1011,7 +838,6 @@ golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCc
golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc=
golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY=
golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg=
-golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg=
golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
@@ -1020,12 +846,8 @@ golang.org/x/mod v0.13.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c=
golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181029044818-c44066c5c816/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181106065722-10aee1819953/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
@@ -1036,17 +858,12 @@ golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn
golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks=
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20190923162816-aa69164e4478/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
-golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
-golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM=
@@ -1067,7 +884,6 @@ golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJ
golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
@@ -1075,17 +891,11 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ
golang.org/x/sync v0.6.0 h1:5BMeUDZ7vkXGfEr1x9B4bRcTH4lpkTkpdh0T/J+qjbQ=
golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk=
golang.org/x/sys v0.0.0-20180810173357-98c5dad5d1a0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20181029174526-d69651ed3497/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190219092855-153ac476189d/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190316082340-a2f829d7f35f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@@ -1095,22 +905,14 @@ golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190804053845-51ab0e2deafa/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200124204421-9fbb57f87de9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200602225109-6fdc65e7d980/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@@ -1165,27 +967,18 @@ golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgw
golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
-golang.org/x/tools v0.0.0-20191010075000-0337d82405ff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
-golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
-golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
-golang.org/x/tools v0.0.0-20191216052735-49a3e744a425/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
-golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
-golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
-golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
-golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
-golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8=
golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
golang.org/x/tools v0.0.0-20200711155855-7342f9734a7d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA=
golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
@@ -1205,21 +998,16 @@ google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEt
google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M=
google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
-google.golang.org/api v0.10.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI=
google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI=
google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI=
google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE=
-google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE=
-google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE=
-google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM=
google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
google.golang.org/appengine v1.3.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0=
-google.golang.org/appengine v1.6.2/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0=
google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc=
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
@@ -1230,7 +1018,6 @@ google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRn
google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
-google.golang.org/genproto v0.0.0-20190716160619-c506a9f90610/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8=
@@ -1238,9 +1025,6 @@ google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvx
google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc=
google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc=
google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc=
-google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc=
-google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc=
-google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA=
google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
@@ -1253,10 +1037,7 @@ google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3
google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38=
google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM=
-google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY=
google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
@@ -1282,12 +1063,7 @@ gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI=
-gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw=
-gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
-gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo=
-gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
-gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74=
gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
@@ -1308,7 +1084,6 @@ honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWh
honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg=
-honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k=
k8s.io/klog v1.0.0 h1:Pt+yjF5aB1xDSVbau4VsWe+dQNzA0qv1LlXdC2dF6Q8=
k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I=
k8s.io/klog/v2 v2.90.1 h1:m4bYOKall2MmOiRaR1J+We67Do7vm9KiQVlT96lnHUw=
diff --git a/gsfa/gsfa-read-multiepoch.go b/gsfa/gsfa-read-multiepoch.go
index 488c1536..6141bdd3 100644
--- a/gsfa/gsfa-read-multiepoch.go
+++ b/gsfa/gsfa-read-multiepoch.go
@@ -4,9 +4,13 @@ import (
"context"
"errors"
"fmt"
+ "time"
"github.com/gagliardetto/solana-go"
- "github.com/rpcpool/yellowstone-faithful/gsfa/offsetstore"
+ "github.com/rpcpool/yellowstone-faithful/compactindexsized"
+ "github.com/rpcpool/yellowstone-faithful/gsfa/linkedlog"
+ "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode"
+ "k8s.io/klog/v2"
)
type GsfaReaderMultiepoch struct {
@@ -46,11 +50,12 @@ func (gsfa *GsfaReaderMultiepoch) Get(
ctx context.Context,
pk solana.PublicKey,
limit int,
-) (EpochToSignatures, error) {
+ fetcher func(uint64, linkedlog.OffsetAndSizeAndBlocktime) (*ipldbindcode.Transaction, error),
+) (EpochToTransactionObjects, error) {
if limit <= 0 {
return nil, nil
}
- sigs := make(EpochToSignatures)
+ sigs := make(EpochToTransactionObjects)
currentLimit := limit
epochLoop:
for _, epoch := range gsfa.epochs {
@@ -62,8 +67,12 @@ epochLoop:
if !ok {
return nil, fmt.Errorf("epoch is not set for the provided gsfa reader")
}
- for _, sig := range epochSigs {
- sigs[epochNum] = append(sigs[epochNum], sig)
+ for _, txLoc := range epochSigs {
+ tx, err := fetcher(epochNum, txLoc)
+ if err != nil {
+ return nil, fmt.Errorf("error while fetching signature: %w", err)
+ }
+ sigs[epochNum] = append(sigs[epochNum], tx)
currentLimit--
if currentLimit <= 0 {
break epochLoop
@@ -76,10 +85,10 @@ epochLoop:
return sigs, nil
}
-type EpochToSignatures map[uint64][]solana.Signature
+type EpochToTransactionObjects map[uint64][]*ipldbindcode.Transaction
// Count returns the number of signatures in the EpochToSignatures.
-func (e EpochToSignatures) Count() int {
+func (e EpochToTransactionObjects) Count() int {
var count int
for _, sigs := range e {
count += len(sigs)
@@ -93,11 +102,12 @@ func (multi *GsfaReaderMultiepoch) GetBeforeUntil(
limit int,
before *solana.Signature, // Before this signature, exclusive (i.e. get signatures older than this signature, excluding it).
until *solana.Signature, // Until this signature, inclusive (i.e. stop at this signature, including it).
-) (EpochToSignatures, error) {
+ fetcher func(uint64, linkedlog.OffsetAndSizeAndBlocktime) (*ipldbindcode.Transaction, error),
+) (EpochToTransactionObjects, error) {
if limit <= 0 {
- return make(EpochToSignatures), nil
+ return make(EpochToTransactionObjects), nil
}
- return multi.iterBeforeUntil(ctx, pk, limit, before, until)
+ return multi.iterBeforeUntil(ctx, pk, limit, before, until, fetcher)
}
// GetBeforeUntil gets the signatures for the given public key,
@@ -108,12 +118,13 @@ func (multi *GsfaReaderMultiepoch) iterBeforeUntil(
limit int,
before *solana.Signature, // Before this signature, exclusive (i.e. get signatures older than this signature, excluding it).
until *solana.Signature, // Until this signature, inclusive (i.e. stop at this signature, including it).
-) (EpochToSignatures, error) {
+ fetcher func(uint64, linkedlog.OffsetAndSizeAndBlocktime) (*ipldbindcode.Transaction, error),
+) (EpochToTransactionObjects, error) {
if limit <= 0 {
- return make(EpochToSignatures), nil
+ return make(EpochToTransactionObjects), nil
}
- sigs := make(EpochToSignatures)
+ transactions := make(EpochToTransactionObjects)
reachedBefore := false
if before == nil {
reachedBefore = true
@@ -121,40 +132,55 @@ func (multi *GsfaReaderMultiepoch) iterBeforeUntil(
epochLoop:
for readerIndex, index := range multi.epochs {
+ if ctx.Err() != nil {
+ return nil, ctx.Err()
+ }
epochNum, ok := index.GetEpoch()
if !ok {
return nil, fmt.Errorf("epoch is not set for the #%d provided gsfa reader", readerIndex)
}
- locs, err := index.offsets.Get(context.Background(), pk)
+ locsStartedAt := time.Now()
+ locs, err := index.offsets.Get(pk)
if err != nil {
- if offsetstore.IsNotFound(err) {
+ if compactindexsized.IsNotFound(err) {
continue epochLoop
}
return nil, fmt.Errorf("error while getting initial offset: %w", err)
}
+ klog.V(5).Infof("locs.OffsetToFirst took %s", time.Since(locsStartedAt))
debugln("locs.OffsetToFirst:", locs)
- next := locs.OffsetToLatest // Start from the latest, and go back in time.
+ next := locs // Start from the latest, and go back in time.
for {
- if next == 0 {
+ if next == nil || next.IsZero() { // no previous.
continue epochLoop
}
- if limit > 0 && sigs.Count() >= limit {
+ if limit > 0 && transactions.Count() >= limit {
break epochLoop
}
- sigIndexes, newNext, err := index.ll.Read(next)
+ startedReadAt := time.Now()
+ locations, newNext, err := index.ll.ReadWithSize(next.Offset, next.Size)
if err != nil {
- return nil, fmt.Errorf("error while reading linked log with next=%d: %w", next, err)
+ return nil, fmt.Errorf("error while reading linked log with next=%v: %w", next, err)
+ }
+ klog.V(5).Infof("ReadWithSize took %s to get %d locs", time.Since(startedReadAt), len(locations))
+ if len(locations) == 0 {
+ continue epochLoop
}
- debugln("sigIndexes:", sigIndexes, "newNext:", newNext)
- next = newNext
- for _, sigIndex := range sigIndexes {
- sig, err := index.sff.Get(sigIndex)
+ debugln("sigIndexes:", locations, "newNext:", newNext)
+ next = &newNext
+ for locIndex, txLoc := range locations {
+ tx, err := fetcher(epochNum, txLoc)
+ if err != nil {
+ return nil, fmt.Errorf("error while getting signature at index=%v: %w", txLoc, err)
+ }
+ sig, err := tx.Signature()
if err != nil {
- return nil, fmt.Errorf("error while getting signature at index=%d: %w", sigIndex, err)
+ return nil, fmt.Errorf("error while getting signature: %w", err)
}
+ klog.V(5).Infoln(locIndex, "sig:", sig, "epoch:", epochNum)
if !reachedBefore && sig == *before {
reachedBefore = true
continue
@@ -162,15 +188,15 @@ epochLoop:
if !reachedBefore {
continue
}
- if limit > 0 && sigs.Count() >= limit {
+ if limit > 0 && transactions.Count() >= limit {
break epochLoop
}
- sigs[epochNum] = append(sigs[epochNum], sig)
+ transactions[epochNum] = append(transactions[epochNum], tx)
if until != nil && sig == *until {
break epochLoop
}
}
}
}
- return sigs, nil
+ return transactions, nil
}
diff --git a/gsfa/gsfa-read.go b/gsfa/gsfa-read.go
index 40c1ced8..eaa01426 100644
--- a/gsfa/gsfa-read.go
+++ b/gsfa/gsfa-read.go
@@ -8,18 +8,17 @@ import (
"path/filepath"
"github.com/gagliardetto/solana-go"
+ "github.com/rpcpool/yellowstone-faithful/compactindexsized"
"github.com/rpcpool/yellowstone-faithful/gsfa/linkedlog"
"github.com/rpcpool/yellowstone-faithful/gsfa/manifest"
- "github.com/rpcpool/yellowstone-faithful/gsfa/offsetstore"
- "github.com/rpcpool/yellowstone-faithful/gsfa/sff"
+ "github.com/rpcpool/yellowstone-faithful/indexes"
"github.com/rpcpool/yellowstone-faithful/indexmeta"
)
type GsfaReader struct {
epoch *uint64
- offsets *offsetstore.OffsetStore
+ offsets *indexes.PubkeyToOffsetAndSize_Reader
ll *linkedlog.LinkedLog
- sff *sff.SignaturesFlatFile
man *manifest.Manifest
}
@@ -40,18 +39,12 @@ func NewGsfaReader(indexRootDir string) (*GsfaReader, error) {
}
index := &GsfaReader{}
{
- offsetsIndexDir := filepath.Join(indexRootDir, "offsets-index")
- offsets, err := offsetstore.Open(
- context.Background(),
- filepath.Join(offsetsIndexDir, "index"),
- filepath.Join(offsetsIndexDir, "data"),
- offsetstoreOptions...,
- )
+ offsetsIndex := filepath.Join(indexRootDir, string(indexes.Kind_PubkeyToOffsetAndSize)+".index")
+ offsets, err := indexes.Open_PubkeyToOffsetAndSize(offsetsIndex)
if err != nil {
- return nil, fmt.Errorf("error while opening index: %w", err)
+ return nil, fmt.Errorf("error while opening offsets index: %w", err)
}
index.offsets = offsets
- index.offsets.Start()
}
{
ll, err := linkedlog.NewLinkedLog(filepath.Join(indexRootDir, "linked-log"))
@@ -60,13 +53,6 @@ func NewGsfaReader(indexRootDir string) (*GsfaReader, error) {
}
index.ll = ll
}
- {
- sff, err := sff.NewSignaturesFlatFile(filepath.Join(indexRootDir, "signatures-flatfile"))
- if err != nil {
- return nil, err
- }
- index.sff = sff
- }
{
man, err := manifest.NewManifest(filepath.Join(indexRootDir, "manifest"), indexmeta.Meta{})
if err != nil {
@@ -92,7 +78,6 @@ func (index *GsfaReader) Close() error {
return errors.Join(
index.offsets.Close(),
index.ll.Close(),
- index.sff.Close(),
)
}
@@ -108,47 +93,43 @@ func (index *GsfaReader) Get(
ctx context.Context,
pk solana.PublicKey,
limit int,
-) ([]solana.Signature, error) {
+) ([]linkedlog.OffsetAndSizeAndBlocktime, error) {
if limit <= 0 {
- return []solana.Signature{}, nil
+ return []linkedlog.OffsetAndSizeAndBlocktime{}, nil
}
- locs, err := index.offsets.Get(context.Background(), pk)
+ lastOffset, err := index.offsets.Get(pk)
if err != nil {
- if offsetstore.IsNotFound(err) {
- return nil, offsetstore.ErrNotFound{PubKey: pk}
+ if compactindexsized.IsNotFound(err) {
+ return nil, fmt.Errorf("pubkey %s not found: %w", pk, err)
}
return nil, fmt.Errorf("error while getting initial offset: %w", err)
}
- debugln("locs.OffsetToFirst:", locs)
+ debugln("locs.OffsetToFirst:", lastOffset)
- var sigs []solana.Signature
- next := locs.OffsetToLatest // Start from the latest, and go back in time.
+ var allTransactionLocations []linkedlog.OffsetAndSizeAndBlocktime
+ next := lastOffset // Start from the latest, and go back in time.
for {
- if next == 0 {
+ if next == nil || next.IsZero() { // no previous.
break
}
- if limit > 0 && len(sigs) >= limit {
+ if limit > 0 && len(allTransactionLocations) >= limit {
break
}
- sigIndexes, newNext, err := index.ll.Read(next)
+ locations, newNext, err := index.ll.ReadWithSize(next.Offset, next.Size)
if err != nil {
return nil, fmt.Errorf("error while reading linked log with next=%d: %w", next, err)
}
- debugln("sigIndexes:", sigIndexes, "newNext:", newNext)
- next = newNext
- for _, sigIndex := range sigIndexes {
- sig, err := index.sff.Get(sigIndex)
- if err != nil {
- return nil, fmt.Errorf("error while getting signature at index=%d: %w", sigIndex, err)
- }
- if limit > 0 && len(sigs) >= limit {
+ debugln("sigIndexes:", locations, "newNext:", newNext)
+ next = &newNext
+ for _, sigIndex := range locations {
+ if limit > 0 && len(allTransactionLocations) >= limit {
break
}
- sigs = append(sigs, sig)
+ allTransactionLocations = append(allTransactionLocations, sigIndex)
}
}
- return sigs, nil
+ return allTransactionLocations, nil
}
func (index *GsfaReader) GetBeforeUntil(
@@ -157,21 +138,22 @@ func (index *GsfaReader) GetBeforeUntil(
limit int,
before *solana.Signature, // Before this signature, exclusive (i.e. get signatures older than this signature, excluding it).
until *solana.Signature, // Until this signature, inclusive (i.e. stop at this signature, including it).
-) ([]solana.Signature, error) {
+ fetcher func(sigIndex linkedlog.OffsetAndSizeAndBlocktime) (solana.Signature, error),
+) ([]linkedlog.OffsetAndSizeAndBlocktime, error) {
if limit <= 0 {
- return []solana.Signature{}, nil
+ return []linkedlog.OffsetAndSizeAndBlocktime{}, nil
}
- locs, err := index.offsets.Get(context.Background(), pk)
+ locs, err := index.offsets.Get(pk)
if err != nil {
- if offsetstore.IsNotFound(err) {
- return nil, offsetstore.ErrNotFound{PubKey: pk}
+ if compactindexsized.IsNotFound(err) {
+ return nil, fmt.Errorf("pubkey %s not found: %w", pk, err)
}
return nil, fmt.Errorf("error while getting initial offset: %w", err)
}
debugln("locs.OffsetToFirst:", locs)
- var sigs []solana.Signature
- next := locs.OffsetToLatest // Start from the latest, and go back in time.
+ var allTransactionLocations []linkedlog.OffsetAndSizeAndBlocktime
+ next := locs // Start from the latest, and go back in time.
reachedBefore := false
if before == nil {
@@ -180,22 +162,22 @@ func (index *GsfaReader) GetBeforeUntil(
bigLoop:
for {
- if next == 0 {
+ if next == nil || next.IsZero() { // no previous.
break
}
- if limit > 0 && len(sigs) >= limit {
+ if limit > 0 && len(allTransactionLocations) >= limit {
break
}
- sigIndexes, newNext, err := index.ll.Read(next)
+ locations, newNext, err := index.ll.ReadWithSize(next.Offset, next.Size)
if err != nil {
- return nil, fmt.Errorf("error while reading linked log with next=%d: %w", next, err)
+ return nil, fmt.Errorf("error while reading linked log with next=%v: %w", next, err)
}
- debugln("sigIndexes:", sigIndexes, "newNext:", newNext)
- next = newNext
- for _, sigIndex := range sigIndexes {
- sig, err := index.sff.Get(sigIndex)
+ debugln("sigIndexes:", locations, "newNext:", newNext)
+ next = &newNext
+ for _, txLoc := range locations {
+ sig, err := fetcher(txLoc)
if err != nil {
- return nil, fmt.Errorf("error while getting signature at index=%d: %w", sigIndex, err)
+ return nil, fmt.Errorf("error while getting signature at index=%v: %w", txLoc, err)
}
if !reachedBefore && sig == *before {
reachedBefore = true
@@ -204,14 +186,14 @@ bigLoop:
if !reachedBefore {
continue
}
- if limit > 0 && len(sigs) >= limit {
+ if limit > 0 && len(allTransactionLocations) >= limit {
break
}
- sigs = append(sigs, sig)
+ allTransactionLocations = append(allTransactionLocations, txLoc)
if until != nil && sig == *until {
break bigLoop
}
}
}
- return sigs, nil
+ return allTransactionLocations, nil
}
diff --git a/gsfa/gsfa-write.go b/gsfa/gsfa-write.go
index e0d477fe..640fa64f 100644
--- a/gsfa/gsfa-write.go
+++ b/gsfa/gsfa-write.go
@@ -7,41 +7,42 @@ import (
"os"
"path/filepath"
"sync"
+ "sync/atomic"
"time"
"github.com/gagliardetto/solana-go"
+ "github.com/ipfs/go-cid"
"github.com/rpcpool/yellowstone-faithful/gsfa/linkedlog"
"github.com/rpcpool/yellowstone-faithful/gsfa/manifest"
- "github.com/rpcpool/yellowstone-faithful/gsfa/offsetstore"
- "github.com/rpcpool/yellowstone-faithful/gsfa/sff"
+ "github.com/rpcpool/yellowstone-faithful/indexes"
"github.com/rpcpool/yellowstone-faithful/indexmeta"
- "github.com/rpcpool/yellowstone-faithful/store"
- "k8s.io/klog"
+ "github.com/tidwall/hashmap"
+ "k8s.io/klog/v2"
)
type GsfaWriter struct {
- sff *sff.SignaturesFlatFile
- batch map[solana.PublicKey][]uint64
- numCurrentBatchSignatures uint64
- optAutoflushAtNumSigs uint64
- mu sync.Mutex
- offsets *offsetstore.OffsetStore
- ll *linkedlog.LinkedLog
- man *manifest.Manifest
- lastSlot uint64
- firstSlotOfCurrentBatch uint64
-}
-
-var offsetstoreOptions = []store.Option{
- store.IndexBitSize(22),
- store.GCInterval(time.Hour),
+ mu sync.Mutex
+ indexRootDir string
+ offsets *hashmap.Map[solana.PublicKey, [2]uint64]
+ ll *linkedlog.LinkedLog
+ man *manifest.Manifest
+ fullBufferWriterChan chan linkedlog.KeyToOffsetAndSizeAndBlocktime
+ accum *hashmap.Map[solana.PublicKey, []*linkedlog.OffsetAndSizeAndBlocktime]
+ offsetsWriter *indexes.PubkeyToOffsetAndSize_Writer
+ ctx context.Context
+ cancel context.CancelFunc
+ exiting *atomic.Bool
+ fullBufferWriterDone chan struct{}
}
// NewGsfaWriter creates or opens an existing index in WRITE mode.
func NewGsfaWriter(
indexRootDir string,
- flushEveryXSigs uint64,
meta indexmeta.Meta,
+ epoch uint64,
+ rootCid cid.Cid,
+ network indexes.Network,
+ tmpDir string,
) (*GsfaWriter, error) {
// if exists and is dir, open.
// if exists and is not dir, error.
@@ -57,178 +58,253 @@ func NewGsfaWriter(
} else if !ok {
return nil, fmt.Errorf("provided path is not a directory: %s", indexRootDir)
}
- if flushEveryXSigs == 0 {
- return nil, fmt.Errorf("flushAt must be greater than 0")
- }
+ ctx, cancel := context.WithCancel(context.Background())
index := &GsfaWriter{
- batch: make(map[solana.PublicKey][]uint64),
- optAutoflushAtNumSigs: flushEveryXSigs,
- }
- {
- offsetsIndexDir := filepath.Join(indexRootDir, "offsets-index")
- if err := os.MkdirAll(offsetsIndexDir, 0o755); err != nil {
- return nil, err
- }
- offsets, err := offsetstore.Open(
- context.Background(),
- filepath.Join(offsetsIndexDir, "index"),
- filepath.Join(offsetsIndexDir, "data"),
- offsetstoreOptions...,
- )
- if err != nil {
- return nil, fmt.Errorf("error while opening offset index: %w", err)
- }
- index.offsets = offsets
- index.offsets.Start()
+ fullBufferWriterChan: make(chan linkedlog.KeyToOffsetAndSizeAndBlocktime, 50), // TODO: make this configurable
+ offsets: hashmap.New[solana.PublicKey, [2]uint64](int(1_000_000)),
+ accum: hashmap.New[solana.PublicKey, []*linkedlog.OffsetAndSizeAndBlocktime](int(1_000_000)),
+ ctx: ctx,
+ cancel: cancel,
+ fullBufferWriterDone: make(chan struct{}),
+ indexRootDir: indexRootDir,
+ exiting: new(atomic.Bool),
}
{
ll, err := linkedlog.NewLinkedLog(filepath.Join(indexRootDir, "linked-log"))
if err != nil {
- return nil, err
+ return nil, fmt.Errorf("error while opening linked log: %w", err)
}
index.ll = ll
}
{
- sff, err := sff.NewSignaturesFlatFile(filepath.Join(indexRootDir, "signatures-flatfile"))
+ man, err := manifest.NewManifest(filepath.Join(indexRootDir, "manifest"), meta)
if err != nil {
- return nil, err
+ return nil, fmt.Errorf("error while opening manifest: %w", err)
}
- index.sff = sff
+ index.man = man
}
{
- man, err := manifest.NewManifest(filepath.Join(indexRootDir, "manifest"), meta)
+ offsetsWriter, err := indexes.NewWriter_PubkeyToOffsetAndSize(
+ epoch,
+ rootCid,
+ network,
+ tmpDir,
+ )
if err != nil {
- return nil, err
+ return nil, fmt.Errorf("error while opening pubkey-to-offset-and-size writer: %w", err)
}
- index.man = man
+ index.offsetsWriter = offsetsWriter
}
+ go index.fullBufferWriter()
return index, nil
}
-func (a *GsfaWriter) Push(slot uint64, signature solana.Signature, publicKeys []solana.PublicKey) error {
+func (a *GsfaWriter) fullBufferWriter() {
+ numReadFromChan := uint64(0)
+ howManyBuffersToFlushConcurrently := 256
+ tmpBuf := make(linkedlog.KeyToOffsetAndSizeAndBlocktimeSlice, howManyBuffersToFlushConcurrently)
+
+ for {
+ // fmt.Println("numReadFromChan", numReadFromChan, "len(a.fullBufferWriterChan)", len(a.fullBufferWriterChan), "a.exiting.Load()", a.exiting.Load())
+ if a.exiting.Load() {
+ klog.Infof("remaining %d buffers to flush", len(a.fullBufferWriterChan))
+ }
+ if a.exiting.Load() && len(a.fullBufferWriterChan) == 0 {
+ a.fullBufferWriterDone <- struct{}{}
+ return // exit
+ }
+ select {
+ case buffer := <-a.fullBufferWriterChan:
+ {
+ numReadFromChan++
+ has := tmpBuf.Has(buffer.Key)
+ if len(tmpBuf) == howManyBuffersToFlushConcurrently || has {
+ for _, buf := range tmpBuf {
+ // Write the buffer to the linked log.
+ klog.V(5).Infof("Flushing %d transactions for key %s", len(buf.Values), buf.Key)
+ if err := a.flushKVs(buf); err != nil {
+ klog.Errorf("Error while flushing transactions for key %s: %v", buf.Key, err)
+ }
+ }
+ tmpBuf = make(linkedlog.KeyToOffsetAndSizeAndBlocktimeSlice, howManyBuffersToFlushConcurrently)
+ }
+ tmpBuf = append(tmpBuf, buffer)
+ }
+ case <-time.After(1 * time.Second):
+ klog.Infof("Read %d buffers from channel", numReadFromChan)
+ }
+ }
+}
+
+func (a *GsfaWriter) Push(
+ offset uint64,
+ length uint64,
+ slot uint64,
+ blocktime uint64,
+ publicKeys solana.PublicKeySlice,
+) error {
a.mu.Lock()
defer a.mu.Unlock()
- if a.numCurrentBatchSignatures >= a.optAutoflushAtNumSigs && slot != a.lastSlot {
- // Flush the current batch. Only flush if the slot is different from the last one.
- // This is to avoid flushing mid-slot.
- if err := a.flush(); err != nil {
- return fmt.Errorf("error while flushing current batch: %w", err)
+
+ oas := &linkedlog.OffsetAndSizeAndBlocktime{
+ Offset: offset,
+ Size: length,
+ Blocktime: blocktime,
+ }
+ publicKeys = publicKeys.Dedupe()
+ publicKeys.Sort()
+ if slot%1000 == 0 {
+ if a.accum.Len() > 130_000 {
+ // flush all
+ klog.Infof("Flushing all %d keys", a.accum.Len())
+
+ var keys solana.PublicKeySlice = a.accum.Keys()
+ keys.Sort()
+
+ for iii := range keys {
+ key := keys[iii]
+ values, _ := a.accum.Get(key)
+
+ if len(values) < 100 && len(values) > 0 {
+ if err := a.flushKVs(linkedlog.KeyToOffsetAndSizeAndBlocktime{
+ Key: key,
+ Values: values,
+ }); err != nil {
+ return err
+ }
+ a.accum.Delete(key)
+ }
+ }
}
- a.firstSlotOfCurrentBatch = slot
}
- index, err := a.sff.Put(signature)
for _, publicKey := range publicKeys {
- a.batch[publicKey] = append(a.batch[publicKey], index)
- }
- a.numCurrentBatchSignatures++
- a.lastSlot = slot
- if a.firstSlotOfCurrentBatch == 0 {
- a.firstSlotOfCurrentBatch = slot
+ current, ok := a.accum.Get(publicKey)
+ if !ok {
+ current = make([]*linkedlog.OffsetAndSizeAndBlocktime, 0)
+ current = append(current, oas)
+ a.accum.Set(publicKey, current)
+ } else {
+ current = append(current, oas)
+ if len(current) >= itemsPerBatch {
+ a.fullBufferWriterChan <- linkedlog.KeyToOffsetAndSizeAndBlocktime{
+ Key: publicKey,
+ Values: clone(current),
+ }
+ clear(current)
+ a.accum.Delete(publicKey)
+ } else {
+ a.accum.Set(publicKey, current)
+ }
+ }
}
- return err
+ return nil
}
-// Flush forces a flush of the current batch to disk.
-func (a *GsfaWriter) Flush() error {
- a.mu.Lock()
- defer a.mu.Unlock()
- return a.flush()
+func clone[T any](slice []T) []T {
+ s := make([]T, len(slice))
+ copy(s, slice)
+ return s
}
+const itemsPerBatch = 1000
+
// Close closes the accumulator.
func (a *GsfaWriter) Close() error {
a.mu.Lock()
defer a.mu.Unlock()
- if err := a.flush(); err != nil {
+ if err := a.flushAccum(a.accum); err != nil {
return err
}
+ a.exiting.Store(true)
+ klog.Info("Closing linked log...")
+ <-a.fullBufferWriterDone
+ klog.Info("Closing full buffer writer...")
+ a.cancel()
+ {
+ {
+ keys := solana.PublicKeySlice(a.offsets.Keys())
+ keys.Sort()
+ klog.Infof("Writing %d starting offsets for as many pubkeys ...", len(keys))
+ for _, key := range keys {
+ offSize, _ := a.offsets.Get(key)
+ err := a.offsetsWriter.Put(key, offSize[0], offSize[1])
+ if err != nil {
+ return fmt.Errorf("error while writing pubkey-to-offset-and-size: %w", err)
+ }
+ }
+ }
+ offsetsIndex := filepath.Join(a.indexRootDir, string(indexes.Kind_PubkeyToOffsetAndSize)+".index")
+ klog.Info("Sealing pubkey-to-offset-and-size writer...")
+ err := a.offsetsWriter.SealWithFilename(context.Background(), offsetsIndex)
+ if err != nil {
+ return fmt.Errorf("error while sealing pubkey-to-offset-and-size writer: %w", err)
+ }
+ }
+
return errors.Join(
- a.sff.Close(),
- a.offsets.Close(),
+ a.offsetsWriter.Close(),
a.ll.Close(),
a.man.Close(),
)
}
-func (a *GsfaWriter) flush() error {
- if err := a.sff.Flush(); err != nil {
- return err
+func (a *GsfaWriter) flushAccum(m *hashmap.Map[solana.PublicKey, []*linkedlog.OffsetAndSizeAndBlocktime]) error {
+ keys := solana.PublicKeySlice(m.Keys())
+ keys.Sort()
+ for ii := range keys {
+ key := keys[ii]
+ vals, _ := m.Get(key)
+ if err := a.flushKVs(linkedlog.KeyToOffsetAndSizeAndBlocktime{
+ Key: key,
+ Values: vals,
+ }); err != nil {
+ return err
+ }
+ m.Delete(key)
}
- if len(a.batch) == 0 {
+ return nil
+}
+
+func (a *GsfaWriter) flushKVs(kvs ...linkedlog.KeyToOffsetAndSizeAndBlocktime) error {
+ if len(kvs) == 0 {
return nil
}
- klog.Infof("Flushing %d key-to-sigs...", len(a.batch))
startedAt := time.Now()
defer func() {
- klog.Infof(" Flushed key-to-sigs in %s.", time.Since(startedAt))
+ klog.V(5).Infof(" Flushed %d key-to-sigs in %s.", len(kvs), time.Since(startedAt))
}()
- // Flush the offsets store.
- err := a.offsets.Flush()
+ // Flush the linked log cache.
+ // err := a.ll.Flush()
+ // if err != nil {
+ // return fmt.Errorf("error while flushing linked log cache: %w", err)
+ // }
+ _, err := a.ll.Put(
+ func(pk solana.PublicKey) (indexes.OffsetAndSize, error) {
+ got, ok := a.offsets.Get(pk)
+ if !ok {
+ // This is the first time we see this account.
+ // And there is no offset for the previous list.
+ return indexes.OffsetAndSize{}, nil
+ }
+ return indexes.OffsetAndSize{Offset: got[0], Size: got[1]}, nil
+ },
+ func(pk solana.PublicKey, offset uint64, ln uint32) error {
+ a.offsets.Set(pk, [2]uint64{offset, uint64(ln)})
+ return nil
+ },
+ kvs...,
+ )
if err != nil {
- return fmt.Errorf("error while flushing account store: %w", err)
+ return fmt.Errorf("error while writing account lists batch to linked log: %w", err)
}
- {
- // Flush the linked log cache.
- err = a.ll.Flush()
- if err != nil {
- return fmt.Errorf("error while flushing linked log cache: %w", err)
- }
- debugf("Writing %d account batches to linked log...", len(a.batch))
- startOffset, err := a.ll.Put(
- a.batch,
- func(pk solana.PublicKey) (uint64, error) {
- got, err := a.offsets.Get(context.Background(), pk)
- if err != nil {
- if offsetstore.IsNotFound(err) {
- // This is the first time we see this account.
- // And there is no offset for the previous list.
- return 0, nil
- } else {
- return 0, fmt.Errorf("error while getting account: %w", err)
- }
- }
- return got.OffsetToLatest, nil
- },
- func(pk solana.PublicKey, offset uint64, ln uint32) error {
- return a.offsets.Put(
- context.Background(),
- pk,
- offsetstore.Locs{
- OffsetToLatest: offset, // in case this is the first time we see this account.
- })
- },
- )
- if err != nil {
- return fmt.Errorf("error while writing account lists batch to linked log: %w", err)
- }
- // Maps first slot of the batch to the offset of the batch in the linked log.
- err = a.man.Put(a.firstSlotOfCurrentBatch, startOffset)
- if err != nil {
- return fmt.Errorf("error while writing entry to manifest: %w", err)
- }
- }
- a.batch = make(map[solana.PublicKey][]uint64)
- a.numCurrentBatchSignatures = 0
return nil
}
var enableDebug = false
-func debugf(format string, args ...interface{}) {
- if enableDebug {
- klog.Infof(format, args...)
- }
-}
-
func debugln(args ...interface{}) {
if enableDebug {
klog.Infoln(args...)
}
}
-
-func debugln_(c func() []any) {
- if enableDebug {
- klog.Infoln(c()...)
- }
-}
diff --git a/gsfa/linkedlog/compress.go b/gsfa/linkedlog/compress.go
new file mode 100644
index 00000000..b89ceb46
--- /dev/null
+++ b/gsfa/linkedlog/compress.go
@@ -0,0 +1,38 @@
+package linkedlog
+
+import (
+ "fmt"
+
+ "github.com/klauspost/compress/zstd"
+ "github.com/mostynb/zstdpool-freelist"
+)
+
+var zstdDecoderPool = zstdpool.NewDecoderPool()
+
+func decompressZSTD(data []byte) ([]byte, error) {
+ dec, err := zstdDecoderPool.Get(nil)
+ if err != nil {
+ return nil, fmt.Errorf("failed to get zstd decoder from pool: %w", err)
+ }
+ defer zstdDecoderPool.Put(dec)
+
+ content, err := dec.DecodeAll(data, nil)
+ if err != nil {
+ return nil, fmt.Errorf("failed to decompress zstd data: %w", err)
+ }
+ return content, nil
+}
+
+var zstdEncoderPool = zstdpool.NewEncoderPool(
+ zstd.WithEncoderLevel(zstd.SpeedBetterCompression),
+ // zstd.WithEncoderLevel(zstd.SpeedFastest),
+)
+
+func compressZSTD(data []byte) ([]byte, error) {
+ enc, err := zstdEncoderPool.Get(nil)
+ if err != nil {
+ return nil, err
+ }
+ defer zstdEncoderPool.Put(enc)
+ return enc.EncodeAll(data, nil), nil
+}
diff --git a/gsfa/linkedlog/linked-log.go b/gsfa/linkedlog/linked-log.go
index b95ed167..4a1cbfbd 100644
--- a/gsfa/linkedlog/linked-log.go
+++ b/gsfa/linkedlog/linked-log.go
@@ -2,21 +2,23 @@ package linkedlog
import (
"bufio"
+ "bytes"
"encoding/binary"
"errors"
"fmt"
"io"
"os"
+ "slices"
+ "sort"
"sync"
"github.com/gagliardetto/solana-go"
- "github.com/ronanh/intcomp"
- "golang.org/x/sync/errgroup"
+ "github.com/rpcpool/yellowstone-faithful/indexes"
)
type LinkedLog struct {
file *os.File
- cache *bufio.Writer
+ buffer *bufio.Writer
mu sync.Mutex
offset uint64
writeMu sync.Mutex
@@ -38,10 +40,10 @@ func NewLinkedLog(filename string) (*LinkedLog, error) {
if err != nil {
return nil, err
}
- cache := bufio.NewWriterSize(file, MiB*256)
+ buf := bufio.NewWriterSize(file, MiB*12)
ll := &LinkedLog{
- file: file,
- cache: cache,
+ file: file,
+ buffer: buf,
}
currentOffset, err := ll.getCurrentOffset()
if err != nil {
@@ -58,7 +60,7 @@ func (s *LinkedLog) Close() error {
func (c *LinkedLog) close() (err error) {
c.mu.Lock()
defer c.mu.Unlock()
- if err = c.cache.Flush(); err != nil {
+ if err = c.buffer.Flush(); err != nil {
return err
}
err = c.file.Close()
@@ -68,29 +70,27 @@ func (c *LinkedLog) close() (err error) {
return
}
-// Flush flushes the cache to disk
+// Flush flushes the buffer to disk
func (s *LinkedLog) Flush() error {
s.mu.Lock()
defer s.mu.Unlock()
- return s.cache.Flush()
+ return s.buffer.Flush()
}
// getCurrentOffset returns the number of bytes in the file
func (s *LinkedLog) getCurrentOffset() (uint64, error) {
- size, err := s.getSize()
+ stat, err := s.file.Stat()
if err != nil {
return 0, err
}
- return uint64(size), nil
+ return uint64(stat.Size()), nil
}
-// getSize returns the size of the file in bytes
+// getSize returns the size of the file in bytes considering the buffer
func (s *LinkedLog) getSize() (int64, error) {
- fi, err := s.file.Stat()
- if err != nil {
- return 0, err
- }
- return fi.Size(), nil
+ s.writeMu.Lock()
+ defer s.writeMu.Unlock()
+ return int64(s.offset), nil
}
// write writes the given bytes to the file and returns the offset at which
@@ -98,7 +98,7 @@ func (s *LinkedLog) getSize() (int64, error) {
func (s *LinkedLog) write(b []byte) (uint64, uint32, error) {
s.writeMu.Lock()
defer s.writeMu.Unlock()
- numWritten, err := s.cache.Write(b)
+ numWritten, err := s.buffer.Write(b)
if err != nil {
return 0, 0, err
}
@@ -110,123 +110,147 @@ func (s *LinkedLog) write(b []byte) (uint64, uint32, error) {
const mib = 1024 * 1024
// Read reads the block stored at the given offset.
-func (s *LinkedLog) Read(offset uint64) ([]uint64, uint64, error) {
+func (s *LinkedLog) Read(offset uint64) ([]OffsetAndSizeAndBlocktime, indexes.OffsetAndSize, error) {
lenBuf := make([]byte, binary.MaxVarintLen64)
_, err := s.file.ReadAt(lenBuf, int64(offset))
if err != nil {
- return nil, 0, err
+ return nil, indexes.OffsetAndSize{}, err
}
// debugln_(func() []any { return []any{"lenBuf:", bin.FormatByteSlice(lenBuf)} })
// Read the length of the compressed indexes
compactedIndexesLen, n := binary.Uvarint(lenBuf)
if n <= 0 {
- return nil, 0, errors.New("invalid compacted indexes length")
+ return nil, indexes.OffsetAndSize{}, errors.New("invalid compacted indexes length")
}
- if compactedIndexesLen > 256*mib {
- return nil, 0, fmt.Errorf("compacted indexes length too large: %d", compactedIndexesLen)
+ return s.ReadWithSize(offset, compactedIndexesLen)
+}
+
+func sizeOfUvarint(n uint64) int {
+ return binary.PutUvarint(make([]byte, binary.MaxVarintLen64), n)
+}
+
+func (s *LinkedLog) ReadWithSize(offset uint64, size uint64) ([]OffsetAndSizeAndBlocktime, indexes.OffsetAndSize, error) {
+ if size > 256*mib {
+ return nil, indexes.OffsetAndSize{}, fmt.Errorf("compacted indexes length too large: %d", size)
}
// debugln("compactedIndexesLen:", compactedIndexesLen)
// Read the compressed indexes
- data := make([]byte, compactedIndexesLen)
- _, err = s.file.ReadAt(data, int64(offset)+int64(n))
+ data := make([]byte, size-uint64(sizeOfUvarint(size))) // The size bytes have already been read.
+ _, err := s.file.ReadAt(data, int64(offset)+int64(sizeOfUvarint(size)))
if err != nil {
- return nil, 0, err
+ return nil, indexes.OffsetAndSize{}, err
}
// debugln_(func() []any { return []any{"data:", bin.FormatByteSlice(data)} })
- // the indexes are up until the last 8 bytes, which are the `next` offset.
- indexes := data[:len(data)-8]
- nextOffset := binary.LittleEndian.Uint64(data[len(data)-8:])
+ // the indexesBytes are up until the last 8 bytes, which are the `next` offset.
+ indexesBytes := data[:len(data)-9]
+ var nextOffset indexes.OffsetAndSize
+ err = nextOffset.FromBytes(data[len(data)-9:])
+ if err != nil {
+ return nil, indexes.OffsetAndSize{}, fmt.Errorf("error while reading next offset: %w", err)
+ }
+ // fmt.Println("nextOffset:", nextOffset, offset, size, bin.FormatByteSlice(data)) // DEBUG
// Decompress the indexes
- sigIndexes := intcomp.UncompressUint64(uint64SliceFromBytes(indexes), make([]uint64, 0))
+ sigIndexes, err := decompressIndexes(indexesBytes)
+ if err != nil {
+ return nil, indexes.OffsetAndSize{}, fmt.Errorf("error while decompressing indexes: %w", err)
+ }
return sigIndexes, nextOffset, nil
}
-// Put map[PublicKey][]uint64 to file
+func decompressIndexes(data []byte) ([]OffsetAndSizeAndBlocktime, error) {
+ decompressed, err := decompressZSTD(data)
+ if err != nil {
+ return nil, fmt.Errorf("error while decompressing data: %w", err)
+ }
+ return OffsetAndSizeAndBlocktimeSliceFromBytes(decompressed)
+}
+
+type KeyToOffsetAndSizeAndBlocktimeSlice []KeyToOffsetAndSizeAndBlocktime
+
+// Has returns true if the given public key is in the slice.
+func (s KeyToOffsetAndSizeAndBlocktimeSlice) Has(key solana.PublicKey) bool {
+ for _, k := range s {
+ if k.Key == key {
+ return true
+ }
+ }
+ return false
+}
+
+type KeyToOffsetAndSizeAndBlocktime struct {
+ Key solana.PublicKey
+ Values []*OffsetAndSizeAndBlocktime
+}
+
func (s *LinkedLog) Put(
- dataMap map[solana.PublicKey][]uint64,
- callbackBefore func(pk solana.PublicKey) (uint64, error),
+ callbackBefore func(pk solana.PublicKey) (indexes.OffsetAndSize, error),
callbackAfter func(pk solana.PublicKey, offset uint64, ln uint32) error,
+ values ...KeyToOffsetAndSizeAndBlocktime,
) (uint64, error) {
s.mu.Lock()
defer s.mu.Unlock()
- pubkeys := make(solana.PublicKeySlice, 0, len(dataMap))
- for k := range dataMap {
- pubkeys = append(pubkeys, k)
- }
- // Sort pubkeys
- pubkeys.Sort()
+ // sort by public key:
+ sort.Slice(values, func(i, j int) bool {
+ return bytes.Compare(values[i].Key[:], values[j].Key[:]) < 0
+ })
previousSize, err := s.getSize()
if err != nil {
return 0, err
}
- wg := new(errgroup.Group)
- wg.SetLimit(256)
- for pkIndex := range pubkeys {
- pk := pubkeys[pkIndex]
- sigIndexes := dataMap[pk]
- reverseUint64Slice(sigIndexes) // reverse the slice so that the most recent indexes are first
- wg.Go(func() error {
- compactedIndexes := intcomp.CompressUint64(sigIndexes, make([]uint64, 0))
-
- encodedIndexes := uint64SliceToBytes(compactedIndexes)
- finalPayload := make([]byte, 0)
+ for pkIndex := range values {
+ val := values[pkIndex]
+ if len(val.Values) == 0 {
+ continue
+ }
+ slices.Reverse[[]*OffsetAndSizeAndBlocktime](val.Values) // reverse the slice so that the most recent indexes are first
+ err := func() error {
+ encodedIndexes, err := createIndexesPayload(val.Values)
+ if err != nil {
+ return fmt.Errorf("error while creating payload: %w", err)
+ }
+ payloadLen := uint64(len(encodedIndexes)) + indexes.IndexValueSize_CidToOffsetAndSize
+ payloadLenAsBytes := encodeUvarint(payloadLen)
- // Write the size of the compressed indexes
- uvLen := encodeUvarint(uint64(len(encodedIndexes)) + 8)
- finalPayload = append(finalPayload, uvLen...)
- // Write the compressed indexes
+ // The payload:
+ finalPayload := make([]byte, 0, len(payloadLenAsBytes)+len(encodedIndexes)+indexes.IndexValueSize_CidToOffsetAndSize)
+ // 1/3 - the size of the compressed indexes
+ finalPayload = append(finalPayload, payloadLenAsBytes...)
+ // 2/3 - the compressed indexes
finalPayload = append(finalPayload, encodedIndexes...)
{
- previousListOffset, err := callbackBefore(pk)
+ previousListOffset, err := callbackBefore(val.Key)
if err != nil {
return err
}
- // Write the offset of the previous list for this pubkey:
- finalPayload = append(finalPayload, uint64ToBytes(previousListOffset)...)
+ // 3/3 - the offset and size of the previous list for this pubkey:
+ finalPayload = append(finalPayload, previousListOffset.Bytes()...)
}
offset, numWrittenBytes, err := s.write(finalPayload)
if err != nil {
return err
}
- return callbackAfter(pk, offset, numWrittenBytes)
- })
- }
- return uint64(previousSize), wg.Wait()
-}
-
-func reverseUint64Slice(s []uint64) {
- for i, j := 0, len(s)-1; i < j; i, j = i+1, j-1 {
- s[i], s[j] = s[j], s[i]
- }
-}
-
-func uint64ToBytes(i uint64) []byte {
- b := make([]byte, 8)
- binary.LittleEndian.PutUint64(b, i)
- return b
-}
-
-func uint64SliceFromBytes(buf []byte) []uint64 {
- if len(buf)%8 != 0 {
- panic(fmt.Sprintf("buf length must be a multiple of 8, got %d", len(buf)))
- }
- slice := make([]uint64, len(buf)/8)
- for i := 0; i < len(slice); i++ {
- slice[i] = binary.LittleEndian.Uint64(buf[i*8:])
+ // fmt.Printf("offset=%d, numWrittenBytes=%d ll=%d\n", offset, numWrittenBytes, ll) // DEBUG
+ // fmt.Println("finalPayload:", bin.FormatByteSlice(finalPayload)) // DEBUG
+ return callbackAfter(val.Key, offset, numWrittenBytes)
+ }()
+ if err != nil {
+ return 0, err
+ }
}
- return slice
+ return uint64(previousSize), nil
}
-func uint64SliceToBytes(slice []uint64) []byte {
- buf := make([]byte, len(slice)*8)
- for i, num := range slice {
- binary.LittleEndian.PutUint64(buf[i*8:], num)
+func createIndexesPayload(indexes []*OffsetAndSizeAndBlocktime) ([]byte, error) {
+ buf := make([]byte, 0, 9*len(indexes))
+ for _, index := range indexes {
+ buf = append(buf, index.Bytes()...)
}
- return buf
+ buf = slices.Clip(buf)
+ return (compressZSTD(buf))
}
func encodeUvarint(n uint64) []byte {
diff --git a/gsfa/linkedlog/offset-size-blocktime.go b/gsfa/linkedlog/offset-size-blocktime.go
new file mode 100644
index 00000000..5e5977cb
--- /dev/null
+++ b/gsfa/linkedlog/offset-size-blocktime.go
@@ -0,0 +1,110 @@
+package linkedlog
+
+import (
+ "encoding/binary"
+ "errors"
+ "fmt"
+ "io"
+ "slices"
+)
+
+func NewOffsetAndSizeAndBlocktime(offset uint64, size uint64, blocktime uint64) *OffsetAndSizeAndBlocktime {
+ return &OffsetAndSizeAndBlocktime{
+ Offset: offset,
+ Size: size,
+ Blocktime: blocktime,
+ }
+}
+
+type OffsetAndSizeAndBlocktime struct {
+ Offset uint64 // uint48, 6 bytes, max 281.5 TB (terabytes)
+ Size uint64 // uint24, 3 bytes, max 16.7 MB (megabytes)
+ Blocktime uint64 // uint40, 5 bytes, max 1099511627775 (seconds since epoch)
+}
+
+// Bytes returns the offset and size as a byte slice.
+func (oas OffsetAndSizeAndBlocktime) Bytes() []byte {
+ buf := make([]byte, 0, binary.MaxVarintLen64*3)
+ buf = binary.AppendUvarint(buf, oas.Offset)
+ buf = binary.AppendUvarint(buf, oas.Size)
+ buf = binary.AppendUvarint(buf, oas.Blocktime)
+ buf = slices.Clip(buf)
+ return buf
+}
+
+// FromBytes parses the offset and size from a byte slice.
+func (oas *OffsetAndSizeAndBlocktime) FromBytes(buf []byte) error {
+ if len(buf) > binary.MaxVarintLen64*3 {
+ return errors.New("invalid byte slice length")
+ }
+ var n int
+ oas.Offset, n = binary.Uvarint(buf)
+ if n <= 0 {
+ return errors.New("failed to parse offset")
+ }
+ buf = buf[n:]
+ oas.Size, n = binary.Uvarint(buf)
+ if n <= 0 {
+ return errors.New("failed to parse size")
+ }
+ buf = buf[n:]
+ oas.Blocktime, n = binary.Uvarint(buf)
+ if n <= 0 {
+ return errors.New("failed to parse blocktime")
+ }
+ return nil
+}
+
+func (oas *OffsetAndSizeAndBlocktime) FromReader(r UvarintReader) error {
+ var err error
+ oas.Offset, err = r.ReadUvarint()
+ if err != nil {
+ return fmt.Errorf("failed to read offset: %w", err)
+ }
+ oas.Size, err = r.ReadUvarint()
+ if err != nil {
+ return fmt.Errorf("failed to read size: %w", err)
+ }
+ oas.Blocktime, err = r.ReadUvarint()
+ if err != nil {
+ return fmt.Errorf("failed to read blocktime: %w", err)
+ }
+ return nil
+}
+
+type UvarintReader interface {
+ ReadUvarint() (uint64, error)
+}
+type uvarintReader struct {
+ pos int
+ buf []byte
+}
+
+func (r *uvarintReader) ReadUvarint() (uint64, error) {
+ if r.pos >= len(r.buf) {
+ return 0, io.EOF
+ }
+ v, n := binary.Uvarint(r.buf[r.pos:])
+ if n <= 0 {
+ return 0, errors.New("failed to parse uvarint")
+ }
+ r.pos += n
+ return v, nil
+}
+
+func OffsetAndSizeAndBlocktimeSliceFromBytes(buf []byte) ([]OffsetAndSizeAndBlocktime, error) {
+ r := &uvarintReader{buf: buf}
+ oass := make([]OffsetAndSizeAndBlocktime, 0)
+ for {
+ oas := OffsetAndSizeAndBlocktime{}
+ err := oas.FromReader(r)
+ if err != nil {
+ if errors.Is(err, io.EOF) {
+ break
+ }
+ return nil, fmt.Errorf("failed to parse offset and size: %w", err)
+ }
+ oass = append(oass, oas)
+ }
+ return oass, nil
+}
diff --git a/gsfa/linkedlog/offset-size-blocktime_test.go b/gsfa/linkedlog/offset-size-blocktime_test.go
new file mode 100644
index 00000000..99d1ac6a
--- /dev/null
+++ b/gsfa/linkedlog/offset-size-blocktime_test.go
@@ -0,0 +1,87 @@
+package linkedlog
+
+import (
+ "encoding/binary"
+ "fmt"
+ "testing"
+)
+
+func TestOffsetAndSizeAndBlocktime(t *testing.T) {
+ {
+ ca := OffsetAndSizeAndBlocktime{
+ Offset: 1,
+ Size: 2,
+ Blocktime: 3,
+ }
+ buf := ca.Bytes()
+
+ {
+ ca2 := OffsetAndSizeAndBlocktime{}
+ err := ca2.FromBytes(buf)
+ if err != nil {
+ panic(err)
+ }
+ if ca != ca2 {
+ panic(fmt.Sprintf("expected %v, got %v", ca, ca2))
+ }
+ }
+ }
+ {
+ // now with very high values
+ ca := OffsetAndSizeAndBlocktime{
+ Offset: 281474976710655,
+ Size: 16777215,
+ Blocktime: 1099511627775,
+ }
+ buf := ca.Bytes()
+
+ {
+ ca2 := OffsetAndSizeAndBlocktime{}
+ err := ca2.FromBytes(buf)
+ if err != nil {
+ panic(err)
+ }
+ if ca != ca2 {
+ panic(fmt.Sprintf("expected %v, got %v", ca, ca2))
+ }
+ }
+ }
+ {
+ many := []OffsetAndSizeAndBlocktime{
+ {
+ Offset: 1,
+ Size: 2,
+ Blocktime: 3,
+ },
+ {
+ Offset: 4,
+ Size: 5,
+ Blocktime: 6,
+ },
+ {
+ Offset: 281474976710655,
+ Size: 16777215,
+ Blocktime: 1099511627775,
+ },
+ }
+ buf := make([]byte, 0, binary.MaxVarintLen64*3*len(many))
+ for _, ca := range many {
+ buf = append(buf, ca.Bytes()...)
+ }
+
+ {
+ many2, err := OffsetAndSizeAndBlocktimeSliceFromBytes(buf)
+ if err != nil {
+ panic(err)
+ }
+ if len(many) != len(many2) {
+ panic(fmt.Sprintf("expected %v, got %v", many, many2))
+ }
+ for i := range many {
+ if many[i] != many2[i] {
+ panic(fmt.Sprintf("expected %v, got %v", many, many2))
+ }
+ }
+ }
+ }
+}
diff --git a/gsfa/manifest/manifest.go b/gsfa/manifest/manifest.go
index bfc6f2a4..96bf4a03 100644
--- a/gsfa/manifest/manifest.go
+++ b/gsfa/manifest/manifest.go
@@ -20,7 +20,7 @@ type Manifest struct {
var (
_MAGIC = [...]byte{'g', 's', 'f', 'a', 'm', 'n', 'f', 's'}
- _Version = uint64(2)
+ _Version = uint64(3)
)
var headerLenWithoutMeta = len(_MAGIC) + 8 // 8 bytes for the version
@@ -124,7 +124,7 @@ func NewManifest(filename string, meta indexmeta.Meta) (*Manifest, error) {
if err != nil {
return nil, err
}
- if header.Version() != _Version && header.Version() != 1 {
+ if header.Version() != _Version {
return nil, fmt.Errorf("unsupported manifest version: %d", header.Version())
}
man.header = header
diff --git a/gsfa/offsetstore/offsetstore.go b/gsfa/offsetstore/offsetstore.go
deleted file mode 100644
index 3b47d647..00000000
--- a/gsfa/offsetstore/offsetstore.go
+++ /dev/null
@@ -1,246 +0,0 @@
-package offsetstore
-
-import (
- "context"
- "encoding/binary"
- "errors"
-
- "github.com/gagliardetto/solana-go"
- store "github.com/rpcpool/yellowstone-faithful/store"
- storetypes "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-type errorType string
-
-func (e errorType) Error() string {
- return string(e)
-}
-
-const (
- ErrNotSupported = errorType("Operation not supported")
- ErrWrongHash = errorType("Wrong hash")
-)
-
-type OffsetStore struct {
- store *store.Store
-}
-
-type Locs struct {
- OffsetToLatest uint64
-}
-
-// Open opens a HashedBlockstore with the default index size
-func Open(ctx context.Context, indexPath string, dataPath string, options ...store.Option) (*OffsetStore, error) {
- store, err := store.OpenStore(
- ctx,
- store.GsfaPrimary,
- dataPath,
- indexPath,
- options...,
- )
- if err != nil {
- return nil, err
- }
- return &OffsetStore{store}, nil
-}
-
-func (as *OffsetStore) Start() {
- as.store.Start()
-}
-
-func (as *OffsetStore) Close() error {
- return as.store.Close()
-}
-
-func (as *OffsetStore) Delete(ctx context.Context, pk solana.PublicKey) error {
- if ctx.Err() != nil {
- return ctx.Err()
- }
- _, err := as.store.Remove(pk[:])
- return err
-}
-
-// Has indicates if an account exists in the store.
-func (as *OffsetStore) Has(ctx context.Context, pk solana.PublicKey) (bool, error) {
- if ctx.Err() != nil {
- return false, ctx.Err()
- }
- return as.store.Has(pk[:])
-}
-
-// Get returns an account from the store.
-func (as *OffsetStore) Get(ctx context.Context, pk solana.PublicKey) (Locs, error) {
- if ctx.Err() != nil {
- return Locs{}, ctx.Err()
- }
- value, found, err := as.store.Get(pk[:])
- if err != nil {
- return Locs{}, err
- }
- if !found {
- return Locs{}, ErrNotFound{PubKey: pk}
- }
- parsed, err := parseLocs(value)
- if err != nil {
- return Locs{}, err
- }
- return parsed, nil
-}
-
-func parseLocs(value []byte) (Locs, error) {
- if len(value) != 8 {
- return Locs{}, errors.New("invalid Loc size")
- }
- return Locs{
- readUint64(value[0:8]),
- }, nil
-}
-
-// Encode returns the encoded bytes of an account.
-func (a Locs) Bytes() []byte {
- buf := make([]byte, 8)
- binary.LittleEndian.PutUint64(buf[0:8], a.OffsetToLatest)
- return buf
-}
-
-func readUint64(data []byte) uint64 {
- return binary.LittleEndian.Uint64(data)
-}
-
-// GetSize returns the size of an account in the store.
-func (as *OffsetStore) GetSize(ctx context.Context, pk solana.PublicKey) (int, error) {
- if ctx.Err() != nil {
- return 0, ctx.Err()
- }
- // unoptimized implementation for now
- size, found, err := as.store.GetSize(pk[:])
- if err != nil {
- return 0, err
- }
- if !found {
- return 0, ErrNotFound{PubKey: pk}
- }
- return int(size), nil
-}
-
-// Put puts a given account in the underlying store.
-func (as *OffsetStore) Put(ctx context.Context, pk solana.PublicKey, loc Locs) error {
- if ctx.Err() != nil {
- return ctx.Err()
- }
- err := as.store.Put(pk[:], loc.Bytes())
- // suppress key exist error because this is not expected behavior for a blockstore
- if err == storetypes.ErrKeyExists {
- // TODO: can we make the store mutable?
- return nil
- }
- return err
-}
-
-func (as *OffsetStore) Flush() error {
- if err := as.store.Flush(); err != nil {
- return err
- }
- if _, err := as.store.Primary().Flush(); err != nil {
- return err
- }
- if err := as.store.Primary().Sync(); err != nil {
- return err
- }
- return nil
-}
-
-// AllKeysChan returns a channel from which
-// the pubkeys in the AccountStore can be read. It should respect
-// the given context, closing the channel if it becomes Done.
-func (as *OffsetStore) AllKeysChan(ctx context.Context) (<-chan solana.PublicKey, error) {
- if ctx.Err() != nil {
- return nil, ctx.Err()
- }
-
- iter := as.store.NewIterator()
-
- ch := make(chan solana.PublicKey)
- go func() {
- defer close(ch)
- for keyHash, val, err := iter.Next(); err == nil; keyHash, _, err = iter.Next() {
- _ = keyHash
- // parse val[:32] as a pubkey
- pubkey := solana.PublicKeyFromBytes(val[:32])
- select {
- case ch <- pubkey:
- case <-ctx.Done():
- return
- }
- }
- }()
- return ch, nil
-}
-
-func (as *OffsetStore) AllValuesChan(ctx context.Context) (<-chan Locs, error) {
- if ctx.Err() != nil {
- return nil, ctx.Err()
- }
-
- iter := as.store.NewIterator()
-
- ch := make(chan Locs)
- go func() {
- defer close(ch)
- for keyHash, value, err := iter.Next(); err == nil; keyHash, _, err = iter.Next() {
- _ = keyHash
- parsed, err := parseLocs(value)
- if err != nil {
- // TODO: log error
- continue
- }
- select {
- case ch <- parsed:
- case <-ctx.Done():
- return
- }
- }
- }()
- return ch, nil
-}
-
-// ErrNotFound is used to signal when a Node could not be found. The specific
-// meaning will depend on the DAGService implementation, which may be trying
-// to read nodes locally but also, trying to find them remotely.
-//
-// The Cid field can be filled in to provide additional context.
-type ErrNotFound struct {
- PubKey solana.PublicKey
-}
-
-// Error implements the error interface and returns a human-readable
-// message for this error.
-func (e ErrNotFound) Error() string {
- if e.PubKey.IsZero() {
- return "not found"
- }
-
- return "could not find entries for " + e.PubKey.String()
-}
-
-// Is allows to check whether any error is of this ErrNotFound type.
-// Do not use this directly, but rather errors.Is(yourError, ErrNotFound).
-func (e ErrNotFound) Is(err error) bool {
- switch err.(type) {
- case ErrNotFound:
- return true
- default:
- return false
- }
-}
-
-// NotFound returns true.
-func (e ErrNotFound) NotFound() bool {
- return true
-}
-
-// IsNotFound returns if the given error is or wraps an ErrNotFound
-// (equivalent to errors.Is(err, ErrNotFound{}))
-func IsNotFound(err error) bool {
- return errors.Is(err, ErrNotFound{})
-}
diff --git a/gsfa/sff/signatures-flatfile.go b/gsfa/sff/signatures-flatfile.go
deleted file mode 100644
index a4001833..00000000
--- a/gsfa/sff/signatures-flatfile.go
+++ /dev/null
@@ -1,137 +0,0 @@
-package sff
-
-import (
- "bufio"
- "fmt"
- "io"
- "os"
- "sync"
-
- "github.com/gagliardetto/solana-go"
-)
-
-const (
- SignatureSize = 64
-)
-
-const (
- writeBufSize = SignatureSize * 1024
-)
-
-type SignaturesFlatFile struct {
- file *os.File
- cache *bufio.Writer
- mu sync.Mutex
- count uint64
-}
-
-func NewSignaturesFlatFile(filename string) (*SignaturesFlatFile, error) {
- file, err := os.OpenFile(filename, os.O_CREATE|os.O_RDWR|os.O_APPEND, 0o644)
- if err != nil {
- return nil, err
- }
- cache := bufio.NewWriterSize(file, writeBufSize)
- sfl := &SignaturesFlatFile{
- file: file,
- cache: cache,
- }
- currentCount, err := sfl.getCurrentCount()
- if err != nil {
- return nil, err
- }
- sfl.count = currentCount
- return sfl, nil
-}
-
-// getSize returns the size of the file in bytes
-func (s *SignaturesFlatFile) getSize() (int64, error) {
- fi, err := s.file.Stat()
- if err != nil {
- return 0, err
- }
- return fi.Size(), nil
-}
-
-// getCurrentCount returns the number of signatures in the file
-func (s *SignaturesFlatFile) getCurrentCount() (uint64, error) {
- size, err := s.getSize()
- if err != nil {
- return 0, err
- }
- // if it's not a multiple of the signature size, it's corrupt
- if size != 0 && size%SignatureSize != 0 {
- return 0, fmt.Errorf("file size is not a multiple of signature size: %d", size)
- }
- return uint64(size / SignatureSize), nil
-}
-
-// NumSignatures returns the number of signatures in the file
-func (s *SignaturesFlatFile) NumSignatures() uint64 {
- s.mu.Lock()
- defer s.mu.Unlock()
- return s.count
-}
-
-func (s *SignaturesFlatFile) Close() error {
- return s.close()
-}
-
-func (c *SignaturesFlatFile) close() (err error) {
- c.mu.Lock()
- defer c.mu.Unlock()
- if err = c.cache.Flush(); err != nil {
- return err
- }
- err = c.file.Close()
- if err != nil {
- return err
- }
- return
-}
-
-// Flush flushes the cache to disk
-func (s *SignaturesFlatFile) Flush() error {
- s.mu.Lock()
- defer s.mu.Unlock()
- return s.cache.Flush()
-}
-
-func (s *SignaturesFlatFile) Put(sig solana.Signature) (uint64, error) {
- if sig.IsZero() {
- return 0, os.ErrInvalid
- }
- s.mu.Lock()
- defer s.mu.Unlock()
- numWritten, err := s.cache.Write(sig[:])
- if err != nil {
- return 0, err
- }
- if numWritten != SignatureSize {
- return 0, os.ErrInvalid
- }
- s.count++
- return s.count - 1, nil
-}
-
-// IsEmpty returns true if the signature is empty
-func IsEmpty(sig solana.Signature) bool {
- return sig.IsZero()
-}
-
-// Get returns the signature at the given index.
-// If the index is out of bounds, os.ErrNotExist is returned.
-// NOTE: Just-written signatures may not be available until the cache is flushed.
-func (s *SignaturesFlatFile) Get(index uint64) (solana.Signature, error) {
- s.mu.Lock()
- defer s.mu.Unlock()
- if index >= s.count {
- return solana.Signature{}, os.ErrNotExist
- }
- sectionReader := io.NewSectionReader(s.file, int64(index*SignatureSize), SignatureSize)
- var sig solana.Signature
- _, err := io.ReadFull(sectionReader, sig[:])
- if err != nil {
- return solana.Signature{}, err
- }
- return sig, nil
-}
diff --git a/gsfa/sff/signatures-flatfile_test.go b/gsfa/sff/signatures-flatfile_test.go
deleted file mode 100644
index ffbfa9de..00000000
--- a/gsfa/sff/signatures-flatfile_test.go
+++ /dev/null
@@ -1,53 +0,0 @@
-package sff
-
-import (
- "math/rand"
- "testing"
-
- "github.com/gagliardetto/solana-go"
- "github.com/stretchr/testify/require"
-)
-
-func TestSignaturesFlatFile(t *testing.T) {
- tmpFilePath := t.TempDir() + "/signatures-flatfile_test"
-
- sfl, err := NewSignaturesFlatFile(tmpFilePath)
- require.NoError(t, err)
- require.NotNil(t, sfl)
- require.Equal(t, uint64(0), sfl.NumSignatures())
-
- // Add a signature
- {
- sig := newRandomSignature()
- index, err := sfl.Put(sig)
- require.NoError(t, err)
- require.Equal(t, uint64(0), index)
- require.Equal(t, uint64(1), sfl.NumSignatures())
- require.NoError(t, sfl.Flush())
- got, err := sfl.Get(index)
- require.NoError(t, err)
- require.Equal(t, sig, got)
- }
- {
- sig := newRandomSignature()
- index, err := sfl.Put(sig)
- require.NoError(t, err)
- require.Equal(t, uint64(1), index)
- require.Equal(t, uint64(2), sfl.NumSignatures())
- require.NoError(t, sfl.Flush())
- got, err := sfl.Get(index)
- require.NoError(t, err)
- require.Equal(t, sig, got)
- }
- {
- // non-existent index
- _, err := sfl.Get(2)
- require.Error(t, err)
- }
-}
-
-func newRandomSignature() solana.Signature {
- var sig solana.Signature
- rand.Read(sig[:])
- return sig
-}
diff --git a/gsfa/worker.go b/gsfa/worker.go
deleted file mode 100644
index d72c4746..00000000
--- a/gsfa/worker.go
+++ /dev/null
@@ -1,208 +0,0 @@
-package gsfa
-
-import (
- "context"
- "fmt"
- "os"
- "path/filepath"
- "time"
-
- "github.com/davecgh/go-spew/spew"
- "github.com/dustin/go-humanize"
- "github.com/gagliardetto/solana-go"
- "github.com/rpcpool/yellowstone-faithful/indexmeta"
-)
-
-func workerRead(indexRoot string, pubkey string, limit int) error {
- gsfa, err := NewGsfaReader(indexRoot)
- if err != nil {
- return err
- }
- defer gsfa.Close()
- gsfa.Get(context.Background(), solana.NewWallet().PublicKey(), limit) // warmup
-
- pk, err := solana.PublicKeyFromBase58(pubkey)
- if err != nil {
- return err
- }
-
- startedAt := time.Now()
- sigs, err := gsfa.Get(context.Background(), pk, limit)
- if err != nil {
- return err
- }
- fmt.Printf("Got %d signatures in %s\n", len(sigs), time.Since(startedAt))
- spew.Dump(sigs)
- {
- took("Get", func() {
- gsfa.Get(context.Background(), pk, limit)
- })
- pk2 := solana.MPK("CiDwVBFgWV9VwPVuU3sPsph2xgPzRYhuhVVA7EcZoKd")
- took("Get", func() {
- gsfa.Get(context.Background(), pk2, limit)
- })
- took("Get", func() {
- gsfa.Get(context.Background(), pk, limit)
- })
- took("Get", func() {
- gsfa.Get(context.Background(), pk2, limit)
- })
- took("Get", func() {
- gsfa.Get(context.Background(), solana.MPK("c8fpTXm3XTSVpAViQ9cBdU56t3ByGe7j6UTUKhqhPxs"), limit)
- })
- }
- return nil
-}
-
-func took(name string, cb func()) {
- startedAt := time.Now()
- cb()
- fmt.Printf("%s took %s\n", name, time.Since(startedAt))
-}
-
-func workerDemoLoad(root string, numGlobalAccounts uint64, numSigs int) error {
- defer func() {
- {
- // print the size of the index.
- size, err := getDirSize(root)
- if err != nil {
- panic(err)
- }
- fmt.Printf("Index folder size: %s\n", humanize.Bytes(size))
- }
- }()
- ///----------------------------------------
-
- accu, err := NewGsfaWriter(
- root,
- 500_000,
- indexmeta.Meta{},
- )
- if err != nil {
- return fmt.Errorf("error while opening accumulator: %w", err)
- }
- defer func() {
- accu.Close()
- }()
-
- for i := 0; i < numSigs; i++ {
- sig := generateRandomSignature()
- howManyKeys := genRandomNumberBetween(1, 10)
- keys := genRandomKeys(numGlobalAccounts, howManyKeys)
- err = accu.Push(111, sig, keys)
- if err != nil {
- panic(err)
- }
- if i%(numSigs/10) == 0 {
- fmt.Println(keys[0], i/(numSigs/10))
- }
- }
- // force flush:
- if err := accu.Flush(); err != nil {
- panic(err)
- }
- fmt.Println("Flushed.")
- return nil
-}
-
-func worker(root string) error {
- indexFolder := filepath.Join(root, "offsets-index")
- os.MkdirAll(indexFolder, os.ModePerm)
-
- defer func() {
- {
- // print the size of the index.
- size, err := getDirSize(root)
- if err != nil {
- panic(err)
- }
- fmt.Printf("Index folder size: %s\n", humanize.Bytes(size))
- }
- }()
- ///----------------------------------------
-
- accu, err := NewGsfaWriter(
- root,
- 1000000,
- indexmeta.Meta{},
- )
- if err != nil {
- return fmt.Errorf("error while opening accumulator: %w", err)
- }
- defer func() {
- accu.Close()
- }()
- keyStrings := []string{
- "CeTwZkgj9bLSkBJ8WPJZFe9zeQ2Z9HJwK8CMfUTEgWYQ",
- "BMRAPhUR3NirnaSAUCWRUwC62jyqgEUAyGbzR8wz3c64",
- "DUN6D2M598AHrhHggySMDWzvzoZwgJuPXAUKVeRQZ6DZ",
- "38aNxeW6VhSu9sAwvfqSEoqEeXZ5B6KrHX4QwPG9NdTZ",
- "EhgYh1aR5jXKzpVJnZgQqYYyhfXCbgA39W1td1EGHbe9",
- "HN7uRdKmGJKaXjK9HayxaXG5TSDCbZTDR5tR4zT9NeUy",
- "5T8kksicHbDXMETjRHFN5LGWXeTjn3n9dNkqfc6CceFA",
- "BfjScBGJ2KUXKRNPdpi2XCBEhYGyzz8BLY7CLw1SJany",
- "AAzR77VN31T68J3LbLu64XBkrtjr1T9aNycvsR1nitzJ",
- "71rkqrWSqqEccxFhSwHZFmaEwZBvWFxLX66C4VJBBioV",
- }
- keys := make([]solana.PublicKey, len(keyStrings))
- for i, keyString := range keyStrings {
- keys[i] = solana.MustPublicKeyFromBase58(keyString)
- }
-
- slot := uint64(123456789)
-
- {
- sig := solana.MustSignatureFromBase58("5YXMTLhABRRs5NtE66kjS6re17pN7xoC8UYvChYMuHySBviyTjxKcsw7riibTtmbRBGxqXX7C3FHgbbsfNk6z2Ga")
- err = accu.Push(slot, sig, keys)
- if err != nil {
- panic(err)
- }
- }
- {
- sig := solana.MustSignatureFromBase58("5kyKrGTGHJhPgWohMW7kn3YS7rq3rdKAHf7J7SnZNZjwPta7jT5tKV9oNejvKQX6k2DcCbk358zKSdQFawMnr8ci")
- err = accu.Push(slot, sig, keys[0:1])
- if err != nil {
- panic(err)
- }
-
- }
- {
- sig := solana.MustSignatureFromBase58("5tPrsxsjifTuXJKvw4KvpGAG73s3JWwhgGbQckYYQC4gexHqiV9tBRnSan5YjMv7vwvJjZfC6rvC8AX8HaRpjA13")
- err = accu.Push(slot, sig, keys[0:3])
- if err != nil {
- panic(err)
- }
- }
- // force flush:
- if err := accu.Flush(); err != nil {
- panic(err)
- }
- fmt.Println("Flushed.")
- {
- sig := solana.MustSignatureFromBase58("3ighZ7KKy1SQsayDX4sfsoC5HAKqZG8XT8NfQh9GmSjq3DCHBnNY9Vc6JNinAxnUdCHxTKoNVALeUA24yd98ZEWt")
- err = accu.Push(slot, sig, keys)
- if err != nil {
- panic(err)
- }
- }
- {
- sig := solana.MustSignatureFromBase58("4LoJumTFxec2viccvKZZL2gieYDbUu7EsuDzNkr73aKxrF4Kb5FNqgQvUpthYoGbjU46iDVsfEYTpMtZEFZy5RCG")
- err = accu.Push(slot, sig, keys[0:1])
- if err != nil {
- panic(err)
- }
- }
- {
- sig := solana.MustSignatureFromBase58("3MGAb27HPFka3JhoLYwoR268EVHe7NMa8mLuTV7Z9sPXayDhFEmNLGvjDR1aBoPzoVKD4i6ws38vRZ7X45NkneeS")
- err = accu.Push(slot, sig, keys[0:3])
- if err != nil {
- panic(err)
- }
- }
- // force flush:
- if err := accu.Flush(); err != nil {
- panic(err)
- }
- fmt.Println("Flushed.")
- return nil
-}
diff --git a/index-cid-to-offset.go b/index-cid-to-offset.go
index 6ce43cb8..05825034 100644
--- a/index-cid-to-offset.go
+++ b/index-cid-to-offset.go
@@ -2,7 +2,6 @@ package main
import (
"bufio"
- "bytes"
"context"
"errors"
"fmt"
@@ -13,9 +12,9 @@ import (
"github.com/davecgh/go-spew/spew"
"github.com/dustin/go-humanize"
- carv1 "github.com/ipld/go-car"
"github.com/ipld/go-car/util"
carv2 "github.com/ipld/go-car/v2"
+ "github.com/rpcpool/yellowstone-faithful/carreader"
"github.com/rpcpool/yellowstone-faithful/indexes"
"github.com/rpcpool/yellowstone-faithful/iplddecoders"
"k8s.io/klog/v2"
@@ -45,13 +44,13 @@ func CreateIndex_cid2offset(
}
defer carFile.Close()
- rd, err := newCarReader(carFile)
+ rd, err := carreader.New(carFile)
if err != nil {
return "", fmt.Errorf("failed to create car reader: %w", err)
}
// check it has 1 root
- if len(rd.header.Roots) != 1 {
- return "", fmt.Errorf("car file must have exactly 1 root, but has %d", len(rd.header.Roots))
+ if len(rd.Header.Roots) != 1 {
+ return "", fmt.Errorf("car file must have exactly 1 root, but has %d", len(rd.Header.Roots))
}
klog.Infof("Getting car file size")
@@ -72,7 +71,7 @@ func CreateIndex_cid2offset(
return "", fmt.Errorf("failed to create tmp dir: %w", err)
}
- rootCid := rd.header.Roots[0]
+ rootCid := rd.Header.Roots[0]
klog.Infof("Creating builder with %d items and target file size %d", numItems, targetFileSize)
c2o, err := indexes.NewWriter_CidToOffsetAndSize(
@@ -88,11 +87,11 @@ func CreateIndex_cid2offset(
defer c2o.Close()
totalOffset := uint64(0)
{
- var buf bytes.Buffer
- if err = carv1.WriteHeader(rd.header, &buf); err != nil {
+ if size, err := rd.HeaderSize(); err != nil {
return "", err
+ } else {
+ totalOffset += size
}
- totalOffset = uint64(buf.Len())
}
numItemsIndexed := uint64(0)
klog.Infof("Indexing...")
@@ -157,13 +156,13 @@ func VerifyIndex_cid2offset(ctx context.Context, carPath string, indexFilePath s
}
defer carFile.Close()
- rd, err := newCarReader(carFile)
+ rd, err := carreader.New(carFile)
if err != nil {
return fmt.Errorf("failed to create car reader: %w", err)
}
// check it has 1 root
- if len(rd.header.Roots) != 1 {
- return fmt.Errorf("car file must have exactly 1 root, but has %d", len(rd.header.Roots))
+ if len(rd.Header.Roots) != 1 {
+ return fmt.Errorf("car file must have exactly 1 root, but has %d", len(rd.Header.Roots))
}
c2o, err := indexes.Open_CidToOffsetAndSize(indexFilePath)
@@ -172,7 +171,7 @@ func VerifyIndex_cid2offset(ctx context.Context, carPath string, indexFilePath s
}
{
// find root cid
- rootCID := rd.header.Roots[0]
+ rootCID := rd.Header.Roots[0]
offset, err := c2o.Get(rootCID)
if err != nil {
return fmt.Errorf("failed to get offset from index: %w", err)
@@ -215,11 +214,11 @@ func VerifyIndex_cid2offset(ctx context.Context, carPath string, indexFilePath s
totalOffset := uint64(0)
{
- var buf bytes.Buffer
- if err = carv1.WriteHeader(rd.header, &buf); err != nil {
+ if size, err := rd.HeaderSize(); err != nil {
return err
+ } else {
+ totalOffset += size
}
- totalOffset = uint64(buf.Len())
}
for {
c, sectionLen, err := rd.NextInfo()
diff --git a/indexes/index-cid-to-offset-and-size.go b/indexes/index-cid-to-offset-and-size.go
index 1c18cb58..513560f1 100644
--- a/indexes/index-cid-to-offset-and-size.go
+++ b/indexes/index-cid-to-offset-and-size.go
@@ -2,7 +2,6 @@ package indexes
import (
"context"
- "errors"
"fmt"
"io"
"os"
@@ -79,14 +78,14 @@ func (w *CidToOffsetAndSize_Writer) Put(cid_ cid.Cid, offset uint64, size uint64
if cid_ == cid.Undef {
return fmt.Errorf("cid is undefined")
}
- if offset > maxUint48 {
- return fmt.Errorf("offset is too large; max is %d, but got %d", maxUint48, offset)
+ if offset > MaxUint48 {
+ return fmt.Errorf("offset is too large; max is %d, but got %d", MaxUint48, offset)
}
- if size > maxUint24 {
- return fmt.Errorf("size is too large; max is %d, but got %d", maxUint24, size)
+ if size > MaxUint24 {
+ return fmt.Errorf("size is too large; max is %d, but got %d", MaxUint24, size)
}
key := cid_.Bytes()
- value := append(uint48tob(offset), uint24tob(uint32(size))...)
+ value := append(Uint48tob(offset), Uint24tob(uint32(size))...)
return w.index.Insert(key, value)
}
@@ -124,39 +123,6 @@ func (w *CidToOffsetAndSize_Writer) GetFilepath() string {
return w.finalPath
}
-func NewOffsetAndSize(offset uint64, size uint64) *OffsetAndSize {
- return &OffsetAndSize{
- Offset: offset,
- Size: size,
- }
-}
-
-// IsValid returns true if the offset and size are valid.
-func (oas *OffsetAndSize) IsValid() bool {
- return oas.Offset <= maxUint48 && oas.Size <= maxUint24
-}
-
-type OffsetAndSize struct {
- Offset uint64 // uint48, 6 bytes, max 281.5 TB (terabytes)
- Size uint64 // uint24, 3 bytes, max 16.7 MB (megabytes)
-}
-
-// Bytes returns the offset and size as a byte slice.
-func (oas *OffsetAndSize) Bytes() []byte {
- return append(uint48tob(oas.Offset), uint24tob(uint32(oas.Size))...)
-}
-
-// FromBytes parses the offset and size from a byte slice.
-func (oas *OffsetAndSize) FromBytes(buf []byte) error {
- if len(buf) != IndexValueSize_CidToOffsetAndSize {
- return errors.New("invalid byte slice length")
- }
- _ = buf[IndexValueSize_CidToOffsetAndSize-1] // bounds check hint to compiler
- oas.Offset = btoUint48(buf[:6])
- oas.Size = uint64(btoUint24(buf[6:]))
- return nil
-}
-
type CidToOffsetAndSize_Reader struct {
file io.Closer
meta *Metadata
diff --git a/indexes/index-pubkey-to-offset-and-size.go b/indexes/index-pubkey-to-offset-and-size.go
new file mode 100644
index 00000000..2bd12ebe
--- /dev/null
+++ b/indexes/index-pubkey-to-offset-and-size.go
@@ -0,0 +1,195 @@
+package indexes
+
+import (
+ "context"
+ "fmt"
+ "io"
+ "os"
+ "path/filepath"
+
+ "github.com/gagliardetto/solana-go"
+ "github.com/ipfs/go-cid"
+ "github.com/rpcpool/yellowstone-faithful/compactindexsized"
+)
+
+type PubkeyToOffsetAndSize_Writer struct {
+ sealed bool
+ tmpDir string
+ finalPath string
+ meta *Metadata
+ index *compactindexsized.Builder
+}
+
+const (
+ // 6 bytes for offset (uint48, max 281.5 TB (terabytes)),
+ // 3 bytes for size (uint24, max 16.7 MB (megabytes), which is plenty considering the max object size is ~1 MB)
+ IndexValueSize_PubkeyToOffsetAndSize = 6 + 3
+)
+
+func FormatFilename_PubkeyToOffsetAndSize(epoch uint64, rootCid cid.Cid, network Network) string {
+ return fmt.Sprintf(
+ "epoch-%d-%s-%s-%s",
+ epoch,
+ rootCid.String(),
+ network,
+ "pubkey-to-offset-and-size.index",
+ )
+}
+
+var Kind_PubkeyToOffsetAndSize = []byte("pubkey-to-offset-and-size")
+
+func NewWriter_PubkeyToOffsetAndSize(
+ epoch uint64,
+ rootCid cid.Cid,
+ network Network,
+ tmpDir string, // Where to put the temporary index files; WILL BE DELETED.
+) (*PubkeyToOffsetAndSize_Writer, error) {
+ if !IsValidNetwork(network) {
+ return nil, ErrInvalidNetwork
+ }
+ if rootCid == cid.Undef {
+ return nil, ErrInvalidRootCid
+ }
+ index, err := compactindexsized.NewBuilderSized(
+ tmpDir,
+ uint(1000000), // TODO: can this be not precise?
+ IndexValueSize_PubkeyToOffsetAndSize,
+ )
+ if err != nil {
+ return nil, err
+ }
+ meta := &Metadata{
+ Epoch: epoch,
+ RootCid: rootCid,
+ Network: network,
+ IndexKind: Kind_PubkeyToOffsetAndSize,
+ }
+ if err := setDefaultMetadata(index, meta); err != nil {
+ return nil, err
+ }
+ return &PubkeyToOffsetAndSize_Writer{
+ tmpDir: tmpDir,
+ meta: meta,
+ index: index,
+ }, nil
+}
+
+func (w *PubkeyToOffsetAndSize_Writer) Put(pk solana.PublicKey, offset uint64, size uint64) error {
+ if offset > MaxUint48 {
+ return fmt.Errorf("offset is too large; max is %d, but got %d", MaxUint48, offset)
+ }
+ if size > MaxUint24 {
+ return fmt.Errorf("size is too large; max is %d, but got %d", MaxUint24, size)
+ }
+ key := pk.Bytes()
+ value := append(Uint48tob(offset), Uint24tob(uint32(size))...)
+ return w.index.Insert(key, value)
+}
+
+func (w *PubkeyToOffsetAndSize_Writer) Seal(ctx context.Context, dstDir string) error {
+ if w.sealed {
+ return fmt.Errorf("already sealed")
+ }
+
+ filepath := filepath.Join(dstDir, FormatFilename_PubkeyToOffsetAndSize(w.meta.Epoch, w.meta.RootCid, w.meta.Network))
+ return w.SealWithFilename(ctx, filepath)
+}
+
+func (w *PubkeyToOffsetAndSize_Writer) SealWithFilename(ctx context.Context, dstFilepath string) error {
+ if w.sealed {
+ return fmt.Errorf("already sealed")
+ }
+
+ filepath := dstFilepath
+ w.finalPath = filepath
+
+ file, err := os.Create(filepath)
+ if err != nil {
+ return fmt.Errorf("failed to create file: %w", err)
+ }
+ defer file.Close()
+
+ if err := w.index.Seal(ctx, file); err != nil {
+ return fmt.Errorf("failed to seal index: %w", err)
+ }
+ w.sealed = true
+
+ return nil
+}
+
+func (w *PubkeyToOffsetAndSize_Writer) Close() error {
+ if !w.sealed {
+ return fmt.Errorf("attempted to close a pubkey-to-offset-and-size index that was not sealed")
+ }
+ return w.index.Close()
+}
+
+// GetFilepath returns the path to the sealed index file.
+func (w *PubkeyToOffsetAndSize_Writer) GetFilepath() string {
+ return w.finalPath
+}
+
+type PubkeyToOffsetAndSize_Reader struct {
+ file io.Closer
+ meta *Metadata
+ index *compactindexsized.DB
+}
+
+func Open_PubkeyToOffsetAndSize(file string) (*PubkeyToOffsetAndSize_Reader, error) {
+ reader, err := os.Open(file)
+ if err != nil {
+ return nil, fmt.Errorf("failed to open index file: %w", err)
+ }
+ return OpenWithReader_PubkeyToOffsetAndSize(reader)
+}
+
+func OpenWithReader_PubkeyToOffsetAndSize(reader ReaderAtCloser) (*PubkeyToOffsetAndSize_Reader, error) {
+ index, err := compactindexsized.Open(reader)
+ if err != nil {
+ return nil, fmt.Errorf("failed to open index: %w", err)
+ }
+ meta, err := getDefaultMetadata(index)
+ if err != nil {
+ return nil, err
+ }
+ if !IsValidNetwork(meta.Network) {
+ return nil, fmt.Errorf("invalid network")
+ }
+ if meta.RootCid == cid.Undef {
+ return nil, fmt.Errorf("root cid is undefined")
+ }
+ if err := meta.AssertIndexKind(Kind_PubkeyToOffsetAndSize); err != nil {
+ return nil, err
+ }
+ return &PubkeyToOffsetAndSize_Reader{
+ file: reader,
+ meta: meta,
+ index: index,
+ }, nil
+}
+
+func (r *PubkeyToOffsetAndSize_Reader) Get(pk solana.PublicKey) (*OffsetAndSize, error) {
+ key := pk.Bytes()
+ value, err := r.index.Lookup(key)
+ if err != nil {
+ return nil, err
+ }
+ oas := &OffsetAndSize{}
+ if err := oas.FromBytes(value); err != nil {
+ return nil, err
+ }
+ return oas, nil
+}
+
+func (r *PubkeyToOffsetAndSize_Reader) Close() error {
+ return r.file.Close()
+}
+
+// Meta returns the metadata for the index.
+func (r *PubkeyToOffsetAndSize_Reader) Meta() *Metadata {
+ return r.meta
+}
+
+func (r *PubkeyToOffsetAndSize_Reader) Prefetch(b bool) {
+ r.index.Prefetch(b)
+}
diff --git a/indexes/index-slot-to-cid.go b/indexes/index-slot-to-cid.go
index 59f9cc44..35ae518c 100644
--- a/indexes/index-slot-to-cid.go
+++ b/indexes/index-slot-to-cid.go
@@ -81,7 +81,7 @@ func (w *SlotToCid_Writer) Put(slot uint64, cid_ cid.Cid) error {
if cid_ == cid.Undef {
return fmt.Errorf("cid is undefined")
}
- key := uint64tob(slot)
+ key := Uint64tob(slot)
value := cid_.Bytes()
return w.index.Insert(key, value)
}
@@ -184,7 +184,7 @@ func (r *SlotToCid_Reader) IsDeprecatedOldVersion() bool {
func (r *SlotToCid_Reader) Get(slot uint64) (cid.Cid, error) {
if r.IsDeprecatedOldVersion() {
- key := uint64tob(slot)
+ key := Uint64tob(slot)
value, err := r.deprecatedIndex.Lookup(key)
if err != nil {
return cid.Undef, err
@@ -195,7 +195,7 @@ func (r *SlotToCid_Reader) Get(slot uint64) (cid.Cid, error) {
}
return c, nil
}
- key := uint64tob(slot)
+ key := Uint64tob(slot)
value, err := r.index.Lookup(key)
if err != nil {
return cid.Undef, err
diff --git a/indexes/metadata.go b/indexes/metadata.go
index 16fd4459..1fb88370 100644
--- a/indexes/metadata.go
+++ b/indexes/metadata.go
@@ -57,7 +57,7 @@ func setDefaultMetadata(index *compactindexsized.Builder, metadata *Metadata) er
}
setter := index.Metadata()
- if err := setter.Add(indexmeta.MetadataKey_Epoch, uint64tob(metadata.Epoch)); err != nil {
+ if err := setter.Add(indexmeta.MetadataKey_Epoch, Uint64tob(metadata.Epoch)); err != nil {
return err
}
@@ -96,7 +96,7 @@ func getDefaultMetadata(index *compactindexsized.DB) (*Metadata, error) {
epochBytes, ok := meta.Get(indexmeta.MetadataKey_Epoch)
if ok {
- out.Epoch = btoUint64(epochBytes)
+ out.Epoch = BtoUint64(epochBytes)
} else {
return nil, fmt.Errorf("metadata.epoch is empty")
}
diff --git a/indexes/offset-and-size.go b/indexes/offset-and-size.go
new file mode 100644
index 00000000..d3ccbc18
--- /dev/null
+++ b/indexes/offset-and-size.go
@@ -0,0 +1,57 @@
+package indexes
+
+import (
+ "errors"
+ "fmt"
+)
+
+func NewOffsetAndSize(offset uint64, size uint64) *OffsetAndSize {
+ return &OffsetAndSize{
+ Offset: offset,
+ Size: size,
+ }
+}
+
+// IsValid returns true if the offset and size are valid.
+func (oas *OffsetAndSize) IsValid() bool {
+ return oas.Offset <= MaxUint48 && oas.Size <= MaxUint24
+}
+
+type OffsetAndSize struct {
+ Offset uint64 // uint48, 6 bytes, max 281.5 TB (terabytes)
+ Size uint64 // uint24, 3 bytes, max 16.7 MB (megabytes)
+}
+
+// IsZero
+func (oas OffsetAndSize) IsZero() bool {
+ return oas.Offset == 0 && oas.Size == 0
+}
+
+// Bytes returns the offset and size as a byte slice.
+func (oas OffsetAndSize) Bytes() []byte {
+ return append(Uint48tob(oas.Offset), Uint24tob(uint32(oas.Size))...)
+}
+
+// FromBytes parses the offset and size from a byte slice.
+func (oas *OffsetAndSize) FromBytes(buf []byte) error {
+ if len(buf) != IndexValueSize_CidToOffsetAndSize {
+ return errors.New("invalid byte slice length")
+ }
+ _ = buf[IndexValueSize_CidToOffsetAndSize-1] // bounds check hint to compiler
+ oas.Offset = BtoUint48(buf[:6])
+ oas.Size = uint64(BtoUint24(buf[6:]))
+ return nil
+}
+
+func OffsetAndSizeSliceFromBytes(buf []byte) ([]OffsetAndSize, error) {
+ if len(buf)%IndexValueSize_CidToOffsetAndSize != 0 {
+ return nil, errors.New("invalid byte slice length")
+ }
+ oass := make([]OffsetAndSize, len(buf)/IndexValueSize_CidToOffsetAndSize)
+ for i := 0; i < len(oass); i++ {
+ if err := oass[i].FromBytes(buf[i*IndexValueSize_CidToOffsetAndSize : (i+1)*IndexValueSize_CidToOffsetAndSize]); err != nil {
+ return nil, fmt.Errorf("failed to parse offset and size at index %d: %w", i, err)
+ }
+ }
+ return oass, nil
+}
diff --git a/indexes/uints.go b/indexes/uints.go
index b9844b34..c42e7604 100644
--- a/indexes/uints.go
+++ b/indexes/uints.go
@@ -3,15 +3,15 @@ package indexes
import "encoding/binary"
const (
- maxUint24 = 1<<24 - 1
- maxUint40 = 1<<40 - 1
- maxUint48 = 1<<48 - 1
- maxUint64 = 1<<64 - 1
+ MaxUint24 = 1<<24 - 1
+ MaxUint40 = 1<<40 - 1
+ MaxUint48 = 1<<48 - 1
+ MaxUint64 = 1<<64 - 1
)
-// uint24tob converts a uint32 to a 3-byte slice; panics if v > maxUint24.
-func uint24tob(v uint32) []byte {
- if v > maxUint24 {
+// Uint24tob converts a uint32 to a 3-byte slice; panics if v > maxUint24.
+func Uint24tob(v uint32) []byte {
+ if v > MaxUint24 {
panic("uint24tob: value out of range")
}
buf := make([]byte, 4)
@@ -19,15 +19,15 @@ func uint24tob(v uint32) []byte {
return buf[:3]
}
-// btoUint24 converts a 3-byte slice to a uint32.
-func btoUint24(buf []byte) uint32 {
+// BtoUint24 converts a 3-byte slice to a uint32.
+func BtoUint24(buf []byte) uint32 {
_ = buf[2] // bounds check hint to compiler
return binary.LittleEndian.Uint32(cloneAndPad(buf, 1))
}
-// uint40tob converts a uint64 to a 5-byte slice; panics if v > maxUint40.
-func uint40tob(v uint64) []byte {
- if v > maxUint40 {
+// Uint40tob converts a uint64 to a 5-byte slice; panics if v > maxUint40.
+func Uint40tob(v uint64) []byte {
+ if v > MaxUint40 {
panic("uint40tob: value out of range")
}
buf := make([]byte, 8)
@@ -35,15 +35,15 @@ func uint40tob(v uint64) []byte {
return buf[:5]
}
-// btoUint40 converts a 5-byte slice to a uint64.
-func btoUint40(buf []byte) uint64 {
+// BtoUint40 converts a 5-byte slice to a uint64.
+func BtoUint40(buf []byte) uint64 {
_ = buf[4] // bounds check hint to compiler
return binary.LittleEndian.Uint64(cloneAndPad(buf, 3))
}
-// uint48tob converts a uint64 to a 6-byte slice; panics if v > maxUint48.
-func uint48tob(v uint64) []byte {
- if v > maxUint48 {
+// Uint48tob converts a uint64 to a 6-byte slice; panics if v > maxUint48.
+func Uint48tob(v uint64) []byte {
+ if v > MaxUint48 {
panic("uint48tob: value out of range")
}
buf := make([]byte, 8)
@@ -51,21 +51,21 @@ func uint48tob(v uint64) []byte {
return buf[:6]
}
-// btoUint48 converts a 6-byte slice to a uint64.
-func btoUint48(buf []byte) uint64 {
+// BtoUint48 converts a 6-byte slice to a uint64.
+func BtoUint48(buf []byte) uint64 {
_ = buf[5] // bounds check hint to compiler
return binary.LittleEndian.Uint64(cloneAndPad(buf, 2))
}
-// uint64tob converts a uint64 to an 8-byte little-endian slice.
-func uint64tob(v uint64) []byte {
+// Uint64tob converts a uint64 to an 8-byte little-endian slice.
+func Uint64tob(v uint64) []byte {
buf := make([]byte, 8)
binary.LittleEndian.PutUint64(buf, v)
return buf
}
-// btoUint64 converts an 8-byte little-endian slice to a uint64.
-func btoUint64(buf []byte) uint64 {
+// BtoUint64 converts an 8-byte little-endian slice to a uint64.
+func BtoUint64(buf []byte) uint64 {
_ = buf[7] // bounds check hint to compiler
return binary.LittleEndian.Uint64(buf)
}
diff --git a/indexes/uints_test.go b/indexes/uints_test.go
index e6dfb5d2..f0ed2cd6 100644
--- a/indexes/uints_test.go
+++ b/indexes/uints_test.go
@@ -9,116 +9,116 @@ import (
func TestUints(t *testing.T) {
{
- require.Equal(t, int(16_777_215), maxUint24)
- require.Equal(t, int(1_099_511_627_775), maxUint40)
- require.Equal(t, int(281_474_976_710_655), maxUint48)
- require.Equal(t, uint(math.MaxUint64), uint(maxUint64))
+ require.Equal(t, int(16_777_215), MaxUint24)
+ require.Equal(t, int(1_099_511_627_775), MaxUint40)
+ require.Equal(t, int(281_474_976_710_655), MaxUint48)
+ require.Equal(t, uint(math.MaxUint64), uint(MaxUint64))
}
{
- v := uint24tob(0)
+ v := Uint24tob(0)
require.Equal(t, []byte{0, 0, 0}, v)
- require.Equal(t, uint32(0), btoUint24(v))
+ require.Equal(t, uint32(0), BtoUint24(v))
- v = uint24tob(1)
+ v = Uint24tob(1)
require.Equal(t, []byte{1, 0, 0}, v)
- require.Equal(t, uint32(1), btoUint24(v))
+ require.Equal(t, uint32(1), BtoUint24(v))
- v = uint24tob(maxUint24)
+ v = Uint24tob(MaxUint24)
require.Equal(t, []byte{255, 255, 255}, v)
- require.Equal(t, uint32(maxUint24), btoUint24(v))
+ require.Equal(t, uint32(MaxUint24), BtoUint24(v))
- v = uint24tob(123)
+ v = Uint24tob(123)
require.Equal(t, []byte{0x7b, 0x0, 0x0}, v)
- require.Equal(t, uint32(123), btoUint24(v))
+ require.Equal(t, uint32(123), BtoUint24(v))
require.Panics(t, func() {
- v = uint24tob(maxUint24 + 1)
+ v = Uint24tob(MaxUint24 + 1)
require.Equal(t, []byte{0, 0, 0}, v)
- require.Equal(t, uint32(0), btoUint24(v))
+ require.Equal(t, uint32(0), BtoUint24(v))
})
}
{
- v := uint40tob(0)
+ v := Uint40tob(0)
require.Equal(t, []byte{0, 0, 0, 0, 0}, v)
- require.Equal(t, uint64(0), btoUint40(v))
+ require.Equal(t, uint64(0), BtoUint40(v))
- v = uint40tob(1)
+ v = Uint40tob(1)
require.Equal(t, []byte{1, 0, 0, 0, 0}, v)
- require.Equal(t, uint64(1), btoUint40(v))
+ require.Equal(t, uint64(1), BtoUint40(v))
- v = uint40tob(123)
+ v = Uint40tob(123)
require.Equal(t, []byte{0x7b, 0x0, 0x0, 0x0, 0x0}, v)
- require.Equal(t, uint64(123), btoUint40(v))
+ require.Equal(t, uint64(123), BtoUint40(v))
- v = uint40tob(maxUint40)
+ v = Uint40tob(MaxUint40)
require.Equal(t, []byte{255, 255, 255, 255, 255}, v)
- require.Equal(t, uint64(maxUint40), btoUint40(v))
+ require.Equal(t, uint64(MaxUint40), BtoUint40(v))
require.Panics(t, func() {
- v = uint40tob(maxUint40 + 1)
+ v = Uint40tob(MaxUint40 + 1)
require.Equal(t, []byte{0, 0, 0, 0, 0}, v)
- require.Equal(t, uint64(0), btoUint40(v))
+ require.Equal(t, uint64(0), BtoUint40(v))
})
}
{
- v := uint48tob(0)
+ v := Uint48tob(0)
require.Equal(t, []byte{0, 0, 0, 0, 0, 0}, v)
- require.Equal(t, uint64(0), btoUint48(v))
+ require.Equal(t, uint64(0), BtoUint48(v))
- v = uint48tob(1)
+ v = Uint48tob(1)
require.Equal(t, []byte{1, 0, 0, 0, 0, 0}, v)
- require.Equal(t, uint64(1), btoUint48(v))
+ require.Equal(t, uint64(1), BtoUint48(v))
- v = uint48tob(123)
+ v = Uint48tob(123)
require.Equal(t, []byte{0x7b, 0x0, 0x0, 0x0, 0x0, 0x0}, v)
- require.Equal(t, uint64(123), btoUint48(v))
+ require.Equal(t, uint64(123), BtoUint48(v))
- v = uint48tob(maxUint48)
+ v = Uint48tob(MaxUint48)
require.Equal(t, []byte{255, 255, 255, 255, 255, 255}, v)
- require.Equal(t, uint64(maxUint48), btoUint48(v))
+ require.Equal(t, uint64(MaxUint48), BtoUint48(v))
require.Panics(t, func() {
- v = uint48tob(maxUint48 + 1)
+ v = Uint48tob(MaxUint48 + 1)
require.Equal(t, []byte{0, 0, 0, 0, 0, 0}, v)
- require.Equal(t, uint64(0), btoUint48(v))
+ require.Equal(t, uint64(0), BtoUint48(v))
})
}
{
- v := uint64tob(0)
+ v := Uint64tob(0)
require.Equal(t, []byte{0, 0, 0, 0, 0, 0, 0, 0}, v)
- require.Equal(t, uint64(0), btoUint64(v))
+ require.Equal(t, uint64(0), BtoUint64(v))
- v = uint64tob(1)
+ v = Uint64tob(1)
require.Equal(t, []byte{1, 0, 0, 0, 0, 0, 0, 0}, v)
- require.Equal(t, uint64(1), btoUint64(v))
+ require.Equal(t, uint64(1), BtoUint64(v))
- v = uint64tob(123)
+ v = Uint64tob(123)
require.Equal(t, []byte{0x7b, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}, v)
- require.Equal(t, uint64(123), btoUint64(v))
+ require.Equal(t, uint64(123), BtoUint64(v))
- v = uint64tob(math.MaxUint64)
+ v = Uint64tob(math.MaxUint64)
require.Equal(t, []byte{255, 255, 255, 255, 255, 255, 255, 255}, v)
- require.Equal(t, uint64(math.MaxUint64), btoUint64(v))
+ require.Equal(t, uint64(math.MaxUint64), BtoUint64(v))
- v = uint64tob(math.MaxUint64 - 1)
+ v = Uint64tob(math.MaxUint64 - 1)
require.Equal(t, []byte{254, 255, 255, 255, 255, 255, 255, 255}, v)
- require.Equal(t, uint64(math.MaxUint64-1), btoUint64(v))
+ require.Equal(t, uint64(math.MaxUint64-1), BtoUint64(v))
}
{
buf := make([]byte, 9)
- copy(buf[:6], uint48tob(123))
- copy(buf[6:], uint24tob(uint32(456)))
+ copy(buf[:6], Uint48tob(123))
+ copy(buf[6:], Uint24tob(uint32(456)))
{
- require.Equal(t, buf[:6], uint48tob(123))
- require.Equal(t, buf[6:], uint24tob(uint32(456)))
+ require.Equal(t, buf[:6], Uint48tob(123))
+ require.Equal(t, buf[6:], Uint24tob(uint32(456)))
}
{
- v := btoUint48(buf[:6])
+ v := BtoUint48(buf[:6])
require.Equal(t, uint64(123), v)
require.Equal(t, uint32(123), uint32(v))
}
{
- v := btoUint24(buf[6:])
+ v := BtoUint24(buf[6:])
require.Equal(t, uint32(456), v)
require.Equal(t, uint64(uint32(456)), uint64(v))
}
@@ -132,11 +132,11 @@ func TestUints(t *testing.T) {
require.Equal(t, []byte{0x7b, 0x00, 0x00, 0x00, 0x00, 0x00, 0xc8, 0x01, 0x00}, encoded)
require.Equal(t, buf, encoded)
}
- require.Equal(t, uint48tob(123), buf[:6])
- require.Equal(t, uint24tob(uint32(456)), buf[6:])
- require.Equal(t, uint64(123), btoUint48(buf[:6]))
- require.Equal(t, uint32(456), btoUint24(uint24tob(uint32(456))))
- require.Equal(t, uint32(456), btoUint24(buf[6:]))
- require.Equal(t, uint64(uint32(456)), uint64(btoUint24(buf[6:])))
+ require.Equal(t, Uint48tob(123), buf[:6])
+ require.Equal(t, Uint24tob(uint32(456)), buf[6:])
+ require.Equal(t, uint64(123), BtoUint48(buf[:6]))
+ require.Equal(t, uint32(456), BtoUint24(Uint24tob(uint32(456))))
+ require.Equal(t, uint32(456), BtoUint24(buf[6:]))
+ require.Equal(t, uint64(uint32(456)), uint64(BtoUint24(buf[6:])))
}
}
diff --git a/ipld/ipldbindcode/methods.go b/ipld/ipldbindcode/methods.go
index 463fec10..11c9205b 100644
--- a/ipld/ipldbindcode/methods.go
+++ b/ipld/ipldbindcode/methods.go
@@ -2,12 +2,15 @@ package ipldbindcode
import (
"encoding/json"
+ "errors"
"fmt"
"hash/crc64"
"hash/fnv"
"strconv"
"strings"
+ bin "github.com/gagliardetto/binary"
+ "github.com/gagliardetto/solana-go"
"github.com/ipfs/go-cid"
cidlink "github.com/ipld/go-ipld-prime/linking/cid"
)
@@ -116,6 +119,92 @@ func (n Transaction) GetPositionIndex() (int, bool) {
return **n.Index, true
}
+var DisableHashVerification bool
+
+func (decoded Transaction) GetSolanaTransaction() (*solana.Transaction, error) {
+ if total, ok := decoded.Data.GetTotal(); !ok || total == 1 {
+ completeData := decoded.Data.Bytes()
+ if !DisableHashVerification {
+ // verify hash (if present)
+ if ha, ok := decoded.Data.GetHash(); ok {
+ err := VerifyHash(completeData, ha)
+ if err != nil {
+ return nil, fmt.Errorf("error while verifying hash: %w", err)
+ }
+ }
+ }
+ var tx solana.Transaction
+ if err := bin.UnmarshalBin(&tx, completeData); err != nil {
+ return nil, fmt.Errorf("error while unmarshaling transaction: %w", err)
+ } else if len(tx.Signatures) == 0 {
+ return nil, fmt.Errorf("transaction has no signatures")
+ }
+ return &tx, nil
+ } else {
+ return nil, errors.New("transaction data is split into multiple objects")
+ }
+}
+
+func (decoded Transaction) Signatures() ([]solana.Signature, error) {
+ return readAllSignatures(decoded.Data.Bytes())
+}
+
+func (decoded Transaction) Signature() (solana.Signature, error) {
+ return readFirstSignature(decoded.Data.Bytes())
+}
+
+func readAllSignatures(buf []byte) ([]solana.Signature, error) {
+ decoder := bin.NewCompactU16Decoder(buf)
+ numSigs, err := decoder.ReadCompactU16()
+ if err != nil {
+ return nil, err
+ }
+ if numSigs == 0 {
+ return nil, fmt.Errorf("no signatures")
+ }
+ // check that there is at least 64 bytes * numSigs left:
+ if decoder.Remaining() < (64 * numSigs) {
+ return nil, fmt.Errorf("not enough bytes left to read %d signatures", numSigs)
+ }
+
+ sigs := make([]solana.Signature, numSigs)
+ for i := 0; i < numSigs; i++ {
+ numRead, err := decoder.Read(sigs[i][:])
+ if err != nil {
+ return nil, err
+ }
+ if numRead != 64 {
+ return nil, fmt.Errorf("unexpected signature length %d", numRead)
+ }
+ }
+ return sigs, nil
+}
+
+func readFirstSignature(buf []byte) (solana.Signature, error) {
+ decoder := bin.NewCompactU16Decoder(buf)
+ numSigs, err := decoder.ReadCompactU16()
+ if err != nil {
+ return solana.Signature{}, err
+ }
+ if numSigs == 0 {
+ return solana.Signature{}, fmt.Errorf("no signatures")
+ }
+ // check that there is at least 64 bytes left:
+ if decoder.Remaining() < 64 {
+ return solana.Signature{}, fmt.Errorf("not enough bytes left to read a signature")
+ }
+
+ var sig solana.Signature
+ numRead, err := decoder.Read(sig[:])
+ if err != nil {
+ return sig, err
+ }
+ if numRead != 64 {
+ return sig, fmt.Errorf("unexpected signature length %d", numRead)
+ }
+ return sig, nil
+}
+
// GetBlockHeight returns the 'block_height' field, which indicates
// the height of the block, and
// a flag indicating whether the field has a value.
diff --git a/multiepoch-getBlock.go b/multiepoch-getBlock.go
index ac7bc3b7..6f1e5dbd 100644
--- a/multiepoch-getBlock.go
+++ b/multiepoch-getBlock.go
@@ -397,9 +397,8 @@ func (multi *MultiEpoch) handleGetBlock(ctx context.Context, conn *requestContex
} else {
txResp.Version = "legacy"
}
- txResp.Meta = meta
- encodedTx, err := encodeTransactionResponseBasedOnWantedEncoding(*params.Options.Encoding, tx, meta)
+ encodedTx, encodedMeta, err := encodeTransactionResponseBasedOnWantedEncoding(*params.Options.Encoding, tx, meta)
if err != nil {
return &jsonrpc2.Error{
Code: jsonrpc2.CodeInternalError,
@@ -407,6 +406,7 @@ func (multi *MultiEpoch) handleGetBlock(ctx context.Context, conn *requestContex
}, fmt.Errorf("failed to encode transaction: %v", err)
}
txResp.Transaction = encodedTx
+ txResp.Meta = encodedMeta
}
allTransactions = append(allTransactions, txResp)
diff --git a/multiepoch-getSignaturesForAddress.go b/multiepoch-getSignaturesForAddress.go
index 6aaf85ff..d51dcdb3 100644
--- a/multiepoch-getSignaturesForAddress.go
+++ b/multiepoch-getSignaturesForAddress.go
@@ -3,17 +3,15 @@ package main
import (
"context"
"fmt"
- "runtime"
"sort"
"sync"
- "github.com/gagliardetto/solana-go"
"github.com/rpcpool/yellowstone-faithful/gsfa"
- metalatest "github.com/rpcpool/yellowstone-faithful/parse_legacy_transaction_status_meta/v-latest"
- metaoldest "github.com/rpcpool/yellowstone-faithful/parse_legacy_transaction_status_meta/v-oldest"
- "github.com/rpcpool/yellowstone-faithful/third_party/solana_proto/confirmed_block"
+ "github.com/rpcpool/yellowstone-faithful/gsfa/linkedlog"
+ "github.com/rpcpool/yellowstone-faithful/indexes"
+ "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode"
+ "github.com/rpcpool/yellowstone-faithful/iplddecoders"
"github.com/sourcegraph/jsonrpc2"
- "golang.org/x/sync/errgroup"
"k8s.io/klog/v2"
)
@@ -44,10 +42,10 @@ func (ser *MultiEpoch) getGsfaReadersInEpochDescendingOrder() ([]*gsfa.GsfaReade
return gsfaReaders, epochNums
}
-func countSignatures(v map[uint64][]solana.Signature) int {
+func countTransactions(v gsfa.EpochToTransactionObjects) int {
var count int
- for _, sigs := range v {
- count += len(sigs)
+ for _, txs := range v {
+ count += len(txs)
}
return count
}
@@ -85,13 +83,55 @@ func (multi *MultiEpoch) handleGetSignaturesForAddress(ctx context.Context, conn
}, fmt.Errorf("failed to create gsfa multiepoch reader: %w", err)
}
- // Get the signatures:
- foundSignatures, err := gsfaMulti.GetBeforeUntil(
+ var blockTimeCache struct {
+ m map[uint64]uint64
+ mu sync.Mutex
+ }
+ blockTimeCache.m = make(map[uint64]uint64)
+ getBlockTime := func(slot uint64, ser *Epoch) uint64 {
+ // NOTE: this means that you have to potentially fetch 1k blocks to get the blocktime for each transaction.
+ // TODO: include blocktime into the transaction data, or in the gsfaindex.
+ // return 0
+ blockTimeCache.mu.Lock()
+ defer blockTimeCache.mu.Unlock()
+ if blockTime, ok := blockTimeCache.m[slot]; ok {
+ return blockTime
+ }
+ block, _, err := ser.GetBlock(ctx, slot)
+ if err != nil {
+ klog.Errorf("failed to get block time for slot %d: %v", slot, err)
+ return 0
+ }
+ blockTimeCache.m[slot] = uint64(block.Meta.Blocktime)
+ return uint64(block.Meta.Blocktime)
+ }
+
+ // Get the transactions:
+ foundTransactions, err := gsfaMulti.GetBeforeUntil(
ctx,
pk,
limit,
params.Before,
params.Until,
+ func(epochNum uint64, oas linkedlog.OffsetAndSizeAndBlocktime) (*ipldbindcode.Transaction, error) {
+ epoch, err := multi.GetEpoch(epochNum)
+ if err != nil {
+ return nil, fmt.Errorf("failed to get epoch %d: %w", epochNum, err)
+ }
+ raw, err := epoch.GetNodeByOffsetAndSize(ctx, nil, &indexes.OffsetAndSize{
+ Offset: oas.Offset,
+ Size: oas.Size,
+ })
+ if err != nil {
+ return nil, fmt.Errorf("failed to get signature: %w", err)
+ }
+ decoded, err := iplddecoders.DecodeTransaction(raw)
+ if err != nil {
+ return nil, fmt.Errorf("error while decoding transaction from nodex at offset %d: %w", oas.Offset, err)
+ }
+ blockTimeCache.m[uint64(decoded.Slot)] = uint64(oas.Blocktime)
+ return decoded, nil
+ },
)
if err != nil {
return &jsonrpc2.Error{
@@ -100,7 +140,7 @@ func (multi *MultiEpoch) handleGetSignaturesForAddress(ctx context.Context, conn
}, fmt.Errorf("failed to get signatures: %w", err)
}
- if len(foundSignatures) == 0 {
+ if len(foundTransactions) == 0 {
err = conn.ReplyRaw(
ctx,
req.ID,
@@ -112,32 +152,10 @@ func (multi *MultiEpoch) handleGetSignaturesForAddress(ctx context.Context, conn
return nil, nil
}
- var blockTimeCache struct {
- m map[uint64]uint64
- mu sync.Mutex
- }
- blockTimeCache.m = make(map[uint64]uint64)
- getBlockTime := func(slot uint64, ser *Epoch) uint64 {
- blockTimeCache.mu.Lock()
- defer blockTimeCache.mu.Unlock()
- if blockTime, ok := blockTimeCache.m[slot]; ok {
- return blockTime
- }
- block, _, err := ser.GetBlock(ctx, slot)
- if err != nil {
- klog.Errorf("failed to get block time for slot %d: %v", slot, err)
- return 0
- }
- blockTimeCache.m[slot] = uint64(block.Meta.Blocktime)
- return uint64(block.Meta.Blocktime)
- }
-
- wg := new(errgroup.Group)
- wg.SetLimit(runtime.NumCPU() * 2)
// The response is an array of objects: [{signature: string}]
- response := make([]map[string]any, countSignatures(foundSignatures))
+ response := make([]map[string]any, countTransactions(foundTransactions))
numBefore := 0
- for ei := range foundSignatures {
+ for ei := range foundTransactions {
epoch := ei
ser, err := multi.GetEpoch(epoch)
if err != nil {
@@ -147,43 +165,35 @@ func (multi *MultiEpoch) handleGetSignaturesForAddress(ctx context.Context, conn
}, fmt.Errorf("failed to get epoch %d: %w", epoch, err)
}
- sigs := foundSignatures[ei]
+ sigs := foundTransactions[ei]
for i := range sigs {
ii := numBefore + i
- sig := sigs[i]
- wg.Go(func() error {
+ transactionNode := sigs[i]
+ err := func() error {
+ sig, err := transactionNode.Signature()
+ if err != nil {
+ klog.Errorf("failed to get signature: %v", err)
+ return nil
+ }
response[ii] = map[string]any{
"signature": sig.String(),
}
if signaturesOnly {
return nil
}
- transactionNode, _, err := ser.GetTransaction(ctx, sig)
- if err != nil {
- klog.Errorf("failed to get tx %s: %v", sig, err)
- return nil
- }
- if transactionNode != nil {
+
+ {
{
tx, meta, err := parseTransactionAndMetaFromNode(transactionNode, ser.GetDataFrameByCid)
if err == nil {
- switch metaValue := meta.(type) {
- case *confirmed_block.TransactionStatusMeta:
- response[ii]["err"] = metaValue.Err
- case *metalatest.TransactionStatusMeta:
- response[ii]["err"] = metaValue.Status
- case *metaoldest.TransactionStatusMeta:
- response[ii]["err"] = metaValue.Status
- }
-
- if _, ok := response[ii]["err"]; ok {
- response[ii]["err"], _ = parseTransactionError(response[ii]["err"])
- }
+ response[ii]["err"] = getErr(meta)
memoData := getMemoInstructionDataFromTransaction(&tx)
if memoData != nil {
response[ii]["memo"] = string(memoData)
}
+ } else {
+ klog.Errorf("failed to parse transaction and meta for signature %s: %v", sig, err)
}
if _, ok := response[ii]["memo"]; !ok {
@@ -203,17 +213,16 @@ func (multi *MultiEpoch) handleGetSignaturesForAddress(ctx context.Context, conn
response[ii]["confirmationStatus"] = "finalized"
}
return nil
- })
+ }()
+ if err != nil {
+ return &jsonrpc2.Error{
+ Code: jsonrpc2.CodeInternalError,
+ Message: "Internal error",
+ }, fmt.Errorf("failed to get tx data: %w", err)
+ }
}
numBefore += len(sigs)
}
- if err := wg.Wait(); err != nil {
- return &jsonrpc2.Error{
- Code: jsonrpc2.CodeInternalError,
- Message: "Internal error",
- }, fmt.Errorf("failed to get tx data: %w", err)
- }
-
// reply with the data
err = conn.ReplyRaw(
ctx,
diff --git a/multiepoch-getTransaction.go b/multiepoch-getTransaction.go
index 1dae2b71..86689ebf 100644
--- a/multiepoch-getTransaction.go
+++ b/multiepoch-getTransaction.go
@@ -191,9 +191,8 @@ func (multi *MultiEpoch) handleGetTransaction(ctx context.Context, conn *request
} else {
response.Version = "legacy"
}
- response.Meta = meta
- encodedTx, err := encodeTransactionResponseBasedOnWantedEncoding(*params.Options.Encoding, tx, meta)
+ encodedTx, encodedMeta, err := encodeTransactionResponseBasedOnWantedEncoding(*params.Options.Encoding, tx, meta)
if err != nil {
return &jsonrpc2.Error{
Code: jsonrpc2.CodeInternalError,
@@ -201,6 +200,7 @@ func (multi *MultiEpoch) handleGetTransaction(ctx context.Context, conn *request
}, fmt.Errorf("failed to encode transaction: %w", err)
}
response.Transaction = encodedTx
+ response.Meta = encodedMeta
}
// reply with the data
diff --git a/parse_legacy_transaction_status_meta/v-latest/parse_legacy_transaction_status_meta_ce598c5c98e7384c104fe7f5121e32c2c5a2d2eb.go b/parse_legacy_transaction_status_meta/v-latest/parse_legacy_transaction_status_meta_ce598c5c98e7384c104fe7f5121e32c2c5a2d2eb.go
index 6be9ad93..c28fd935 100644
--- a/parse_legacy_transaction_status_meta/v-latest/parse_legacy_transaction_status_meta_ce598c5c98e7384c104fe7f5121e32c2c5a2d2eb.go
+++ b/parse_legacy_transaction_status_meta/v-latest/parse_legacy_transaction_status_meta_ce598c5c98e7384c104fe7f5121e32c2c5a2d2eb.go
@@ -5,7 +5,7 @@ import (
"github.com/novifinancial/serde-reflection/serde-generate/runtime/golang/bincode"
"github.com/novifinancial/serde-reflection/serde-generate/runtime/golang/serde"
- "k8s.io/klog"
+ "k8s.io/klog/v2"
)
type CompiledInstruction struct {
diff --git a/readers.go b/readers.go
index cd0430b7..2b796a72 100644
--- a/readers.go
+++ b/readers.go
@@ -1,8 +1,6 @@
package main
import (
- "bufio"
- "encoding/binary"
"errors"
"fmt"
"io"
@@ -10,160 +8,11 @@ import (
"time"
"github.com/dustin/go-humanize"
- "github.com/ipfs/go-cid"
- cbor "github.com/ipfs/go-ipld-cbor"
- "github.com/ipfs/go-libipfs/blocks"
- carv1 "github.com/ipld/go-car"
- "github.com/ipld/go-car/util"
+ "github.com/rpcpool/yellowstone-faithful/carreader"
"github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode"
"github.com/rpcpool/yellowstone-faithful/iplddecoders"
- "github.com/rpcpool/yellowstone-faithful/readahead"
)
-func readHeader(br io.Reader) (*carv1.CarHeader, error) {
- hb, err := util.LdRead(bufio.NewReader(br))
- if err != nil {
- return nil, err
- }
-
- var ch carv1.CarHeader
- if err := cbor.DecodeInto(hb, &ch); err != nil {
- return nil, fmt.Errorf("invalid header: %v", err)
- }
-
- return &ch, nil
-}
-
-type carReader struct {
- br *bufio.Reader
- header *carv1.CarHeader
-}
-
-func newCarReader(r io.ReadCloser) (*carReader, error) {
- br := bufio.NewReaderSize(r, readahead.DefaultChunkSize)
- ch, err := readHeader(br)
- if err != nil {
- return nil, err
- }
-
- if ch.Version != 1 {
- return nil, fmt.Errorf("invalid car version: %d", ch.Version)
- }
-
- if len(ch.Roots) == 0 {
- return nil, fmt.Errorf("empty car, no roots")
- }
-
- return &carReader{
- br: br,
- header: ch,
- }, nil
-}
-
-func readNodeInfoWithoutData(br *bufio.Reader) (cid.Cid, uint64, error) {
- sectionLen, ll, err := readSectionLength(br)
- if err != nil {
- return cid.Cid{}, 0, err
- }
-
- cidLen, c, err := cid.CidFromReader(br)
- if err != nil {
- return cid.Cid{}, 0, err
- }
-
- // Seek to the next section by skipping the block.
- // The section length includes the CID, so subtract it.
- remainingSectionLen := int64(sectionLen) - int64(cidLen)
-
- _, err = io.CopyN(io.Discard, br, remainingSectionLen)
- if err != nil {
- return cid.Cid{}, 0, err
- }
-
- return c, sectionLen + ll, nil
-}
-
-func readNodeInfoWithData(br *bufio.Reader) (cid.Cid, uint64, []byte, error) {
- sectionLen, ll, err := readSectionLength(br)
- if err != nil {
- return cid.Cid{}, 0, nil, fmt.Errorf("failed to read section length: %w", err)
- }
-
- cidLen, c, err := cid.CidFromReader(br)
- if err != nil {
- return cid.Cid{}, 0, nil, fmt.Errorf("failed to read cid: %w", err)
- }
-
- // Seek to the next section by skipping the block.
- // The section length includes the CID, so subtract it.
- remainingSectionLen := int64(sectionLen) - int64(cidLen)
-
- buf := make([]byte, remainingSectionLen)
- _, err = io.ReadFull(br, buf)
- if err != nil {
- return cid.Cid{}, 0, nil, fmt.Errorf("failed to read block: %w", err)
- }
-
- return c, sectionLen + ll, buf, nil
-}
-
-type byteReaderWithCounter struct {
- io.ByteReader
- Offset uint64
-}
-
-func (b *byteReaderWithCounter) ReadByte() (byte, error) {
- c, err := b.ByteReader.ReadByte()
- if err == nil {
- b.Offset++
- }
- return c, err
-}
-
-func readSectionLength(r *bufio.Reader) (uint64, uint64, error) {
- if _, err := r.Peek(1); err != nil { // no more blocks, likely clean io.EOF
- if errors.Is(err, io.ErrNoProgress) {
- return 0, 0, io.EOF
- }
- return 0, 0, fmt.Errorf("failed to peek: %w", err)
- }
-
- br := byteReaderWithCounter{r, 0}
- l, err := binary.ReadUvarint(&br)
- if err != nil {
- if errors.Is(err, io.EOF) {
- return 0, 0, io.ErrUnexpectedEOF // don't silently pretend this is a clean EOF
- }
- return 0, 0, err
- }
-
- if l > uint64(util.MaxAllowedSectionSize) { // Don't OOM
- return 0, 0, errors.New("malformed car; header is bigger than util.MaxAllowedSectionSize")
- }
-
- return l, br.Offset, nil
-}
-
-func (cr *carReader) NextInfo() (cid.Cid, uint64, error) {
- c, sectionLen, err := readNodeInfoWithoutData(cr.br)
- if err != nil {
- return c, 0, err
- }
- return c, sectionLen, nil
-}
-
-func (cr *carReader) NextNode() (cid.Cid, uint64, *blocks.BasicBlock, error) {
- c, sectionLen, data, err := readNodeInfoWithData(cr.br)
- if err != nil {
- return c, 0, nil, fmt.Errorf("failed to read node info: %w", err)
- }
- bl, err := blocks.NewBlockWithCid(data, c)
- if err != nil {
- return c, 0, nil, fmt.Errorf("failed to create block: %w", err)
- }
- return c, sectionLen, bl, nil
-}
-
func isDirEmpty(dir string) (bool, error) {
file, err := os.Open(dir)
if err != nil {
@@ -193,7 +42,7 @@ func carCountItems(carPath string) (uint64, error) {
}
defer file.Close()
- rd, err := newCarReader(file)
+ rd, err := carreader.New(file)
if err != nil {
return 0, fmt.Errorf("failed to open car file: %w", err)
}
@@ -220,7 +69,7 @@ func carCountItemsByFirstByte(carPath string) (map[byte]uint64, *ipldbindcode.Ep
}
defer file.Close()
- rd, err := newCarReader(file)
+ rd, err := carreader.New(file)
if err != nil {
return nil, nil, fmt.Errorf("failed to open car file: %w", err)
}
@@ -230,7 +79,7 @@ func carCountItemsByFirstByte(carPath string) (map[byte]uint64, *ipldbindcode.Ep
startedCountAt := time.Now()
var epochObject *ipldbindcode.Epoch
for {
- _, _, block, err := rd.NextNode()
+ _, _, block, err := rd.NextNodeBytes()
if err != nil {
if errors.Is(err, io.EOF) {
break
@@ -238,7 +87,7 @@ func carCountItemsByFirstByte(carPath string) (map[byte]uint64, *ipldbindcode.Ep
return nil, nil, err
}
// the first data byte is the block type (after the CBOR tag)
- firstDataByte := block.RawData()[1]
+ firstDataByte := block[1]
counts[firstDataByte]++
numTotalItems++
@@ -249,7 +98,7 @@ func carCountItemsByFirstByte(carPath string) (map[byte]uint64, *ipldbindcode.Ep
}
if iplddecoders.Kind(firstDataByte) == iplddecoders.KindEpoch {
- epochObject, err = iplddecoders.DecodeEpoch(block.RawData())
+ epochObject, err = iplddecoders.DecodeEpoch(block)
if err != nil {
return nil, nil, fmt.Errorf("failed to decode Epoch node: %w", err)
}
diff --git a/request-response.go b/request-response.go
index 92e8873b..97c5e823 100644
--- a/request-response.go
+++ b/request-response.go
@@ -6,6 +6,7 @@ import (
"encoding/json"
"fmt"
"net/http"
+ "sort"
"strings"
bin "github.com/gagliardetto/binary"
@@ -376,116 +377,262 @@ func parseGetTransactionRequest(raw *json.RawMessage) (*GetTransactionRequest, e
var zstdEncoderPool = zstdpool.NewEncoderPool()
+func compiledInstructionsToJsonParsed(
+ tx solana.Transaction,
+ inst solana.CompiledInstruction,
+ meta any,
+) (json.RawMessage, error) {
+ programId, err := tx.ResolveProgramIDIndex(inst.ProgramIDIndex)
+ if err != nil {
+ return nil, fmt.Errorf("failed to resolve program ID index: %w", err)
+ }
+ keys := tx.Message.AccountKeys
+ instrParams := txstatus.Parameters{
+ ProgramID: programId,
+ Instruction: txstatus.CompiledInstruction{
+ ProgramIDIndex: uint8(inst.ProgramIDIndex),
+ Accounts: func() []uint8 {
+ out := make([]uint8, len(inst.Accounts))
+ for i, v := range inst.Accounts {
+ out[i] = uint8(v)
+ }
+ return out
+ }(),
+ Data: inst.Data,
+ },
+ AccountKeys: txstatus.AccountKeys{
+ StaticKeys: func() []solana.PublicKey {
+ return clone(keys)
+ }(),
+ // TODO: test this:
+ DynamicKeys: func() *txstatus.LoadedAddresses {
+ switch vv := meta.(type) {
+ case *confirmed_block.TransactionStatusMeta:
+ return &txstatus.LoadedAddresses{
+ Writable: func() []solana.PublicKey {
+ return byteSlicesToKeySlices(vv.LoadedWritableAddresses)
+ }(),
+ Readonly: func() []solana.PublicKey {
+ return byteSlicesToKeySlices(vv.LoadedReadonlyAddresses)
+ }(),
+ }
+ default:
+ return nil
+ }
+ }(),
+ },
+ StackHeight: func() *uint32 {
+ // TODO: get the stack height from somewhere
+ return nil
+ }(),
+ }
+
+ parsedInstructionJSON, err := instrParams.ParseInstruction()
+ if err != nil || parsedInstructionJSON == nil || !strings.HasPrefix(strings.TrimSpace(string(parsedInstructionJSON)), "{") {
+ nonParseadInstructionJSON := map[string]any{
+ "accounts": func() []string {
+ out := make([]string, len(inst.Accounts))
+ for i, v := range inst.Accounts {
+ out[i] = tx.Message.AccountKeys[v].String()
+ }
+ return out
+ }(),
+ "data": base58.Encode(inst.Data),
+ "programId": programId.String(),
+ "stackHeight": nil,
+ }
+ asRaw, _ := jsoniter.ConfigCompatibleWithStandardLibrary.Marshal(nonParseadInstructionJSON)
+ return asRaw, nil
+ } else {
+ return parsedInstructionJSON, nil
+ }
+}
+
func encodeTransactionResponseBasedOnWantedEncoding(
encoding solana.EncodingType,
tx solana.Transaction,
meta any,
-) (any, error) {
+) (any, any, error) {
switch encoding {
case solana.EncodingBase58, solana.EncodingBase64, solana.EncodingBase64Zstd:
txBuf, err := tx.MarshalBinary()
if err != nil {
- return nil, fmt.Errorf("failed to marshal transaction: %w", err)
+ return nil, nil, fmt.Errorf("failed to marshal transaction: %w", err)
}
- return encodeBytesResponseBasedOnWantedEncoding(encoding, txBuf)
+ tOut, err := encodeBytesResponseBasedOnWantedEncoding(encoding, txBuf)
+ return tOut, meta, err
case solana.EncodingJSONParsed:
if !txstatus.IsEnabled() {
- return nil, fmt.Errorf("unsupported encoding")
+ return nil, nil, fmt.Errorf("unsupported encoding")
}
- parsedInstructions := make([]json.RawMessage, 0)
-
- for _, inst := range tx.Message.Instructions {
- programId, _ := tx.ResolveProgramIDIndex(inst.ProgramIDIndex)
- instrParams := txstatus.Parameters{
- ProgramID: programId,
- Instruction: txstatus.CompiledInstruction{
- ProgramIDIndex: uint8(inst.ProgramIDIndex),
- Accounts: func() []uint8 {
- out := make([]uint8, len(inst.Accounts))
- for i, v := range inst.Accounts {
- out[i] = uint8(v)
- }
- return out
- }(),
- Data: inst.Data,
- },
- AccountKeys: txstatus.AccountKeys{
- StaticKeys: tx.Message.AccountKeys,
- // TODO: test this:
- DynamicKeys: func() *txstatus.LoadedAddresses {
- switch v := meta.(type) {
- case *confirmed_block.TransactionStatusMeta:
- return &txstatus.LoadedAddresses{
- Writable: func() []solana.PublicKey {
- out := make([]solana.PublicKey, len(v.LoadedWritableAddresses))
- for i, v := range v.LoadedWritableAddresses {
- out[i] = solana.PublicKeyFromBytes(v)
- }
- return out
- }(),
- Readonly: func() []solana.PublicKey {
- out := make([]solana.PublicKey, len(v.LoadedReadonlyAddresses))
- for i, v := range v.LoadedReadonlyAddresses {
- out[i] = solana.PublicKeyFromBytes(v)
- }
- return out
- }(),
+ {
+ unwrappedMeta, ok := meta.(*confirmed_block.TransactionStatusMeta)
+ if ok {
+ {
+ tables := map[solana.PublicKey]solana.PublicKeySlice{}
+ writable := byteSlicesToKeySlices(unwrappedMeta.LoadedWritableAddresses)
+ readonly := byteSlicesToKeySlices(unwrappedMeta.LoadedReadonlyAddresses)
+ for _, addr := range tx.Message.AddressTableLookups {
+ numTakeWritable := len(addr.WritableIndexes)
+ numTakeReadonly := len(addr.ReadonlyIndexes)
+ tableKey := addr.AccountKey
+ {
+ // now need to rebuild the address table taking into account the indexes, and put the keys into the tables
+ maxIndex := 0
+ for _, indexB := range addr.WritableIndexes {
+ index := int(indexB)
+ if index > maxIndex {
+ maxIndex = index
+ }
}
- default:
- return nil
- }
- }(),
- },
- StackHeight: nil,
- }
-
- parsedInstructionJSON, err := instrParams.ParseInstruction()
- if err != nil || parsedInstructionJSON == nil || !strings.HasPrefix(strings.TrimSpace(string(parsedInstructionJSON)), "{") {
- nonParseadInstructionJSON := map[string]any{
- "accounts": func() []string {
- out := make([]string, len(inst.Accounts))
- for i, v := range inst.Accounts {
- if v >= uint16(len(tx.Message.AccountKeys)) {
- continue
+ for _, indexB := range addr.ReadonlyIndexes {
+ index := int(indexB)
+ if index > maxIndex {
+ maxIndex = index
+ }
}
- out[i] = tx.Message.AccountKeys[v].String()
+ tables[tableKey] = make([]solana.PublicKey, maxIndex+1)
}
- // TODO: validate that the order is correct
- switch v := meta.(type) {
- case *confirmed_block.TransactionStatusMeta:
- for _, wr := range v.LoadedWritableAddresses {
- out = append(out, solana.PublicKeyFromBytes(wr).String())
+ if numTakeWritable > 0 {
+ writableForTable := writable[:numTakeWritable]
+ for i, indexB := range addr.WritableIndexes {
+ index := int(indexB)
+ tables[tableKey][index] = writableForTable[i]
}
- for _, ro := range v.LoadedReadonlyAddresses {
- out = append(out, solana.PublicKeyFromBytes(ro).String())
+ writable = writable[numTakeWritable:]
+ }
+ if numTakeReadonly > 0 {
+ readableForTable := readonly[:numTakeReadonly]
+ for i, indexB := range addr.ReadonlyIndexes {
+ index := int(indexB)
+ tables[tableKey][index] = readableForTable[i]
}
+ readonly = readonly[numTakeReadonly:]
}
- return out
- }(),
- "data": base58.Encode(inst.Data),
- "programId": programId.String(),
- "stackHeight": nil,
+ }
+ err := tx.Message.SetAddressTables(tables)
+ if err != nil {
+ return nil, nil, fmt.Errorf("failed to set address tables: %w", err)
+ }
+ }
+ if tx.Message.IsVersioned() {
+ err := tx.Message.ResolveLookups()
+ if err != nil {
+ panic(err)
+ }
}
- asRaw, _ := jsoniter.ConfigCompatibleWithStandardLibrary.Marshal(nonParseadInstructionJSON)
- parsedInstructions = append(parsedInstructions, asRaw)
- } else {
- parsedInstructions = append(parsedInstructions, parsedInstructionJSON)
}
}
+ parsedInstructions := make([]json.RawMessage, 0)
+
+ for _, inst := range tx.Message.Instructions {
+ parsedInstructionJSON, err := compiledInstructionsToJsonParsed(tx, inst, meta)
+ if err != nil {
+ return nil, nil, fmt.Errorf("failed to compile instruction: %w", err)
+ }
+ parsedInstructions = append(parsedInstructions, parsedInstructionJSON)
+ }
+
resp, err := txstatus.FromTransaction(tx)
if err != nil {
- return nil, fmt.Errorf("failed to convert transaction to txstatus.Transaction: %w", err)
+ return nil, nil, fmt.Errorf("failed to convert transaction to txstatus.Transaction: %w", err)
}
resp.Message.Instructions = parsedInstructions
- return resp, nil
+ {
+ // now try to encode unwrappedMeta:
+ unwrappedMeta, ok := meta.(*confirmed_block.TransactionStatusMeta)
+ if ok {
+ // convert meta to json:
+ metaJSON, err := toMapAny(unwrappedMeta)
+ if err != nil {
+ return nil, nil, fmt.Errorf("failed to marshal meta: %w", err)
+ }
+ for innerIndex, insts := range unwrappedMeta.InnerInstructions {
+ inner := make([]solana.CompiledInstruction, len(insts.Instructions))
+ for j, inst := range insts.Instructions {
+ inner[j] = solana.CompiledInstruction{
+ ProgramIDIndex: uint16(inst.ProgramIdIndex),
+ Accounts: byeSliceToUint16Slice(inst.Accounts),
+ Data: clone(inst.Data),
+ }
+ }
+ for instIndex, inst := range inner {
+ parsedInstructionJSON, err := compiledInstructionsToJsonParsed(tx, inst, unwrappedMeta)
+ if err != nil {
+ return nil, nil, fmt.Errorf("failed to compile instruction: %w", err)
+ }
+ // now replace the inner instruction with the parsed instruction:
+ {
+ if _, ok := metaJSON["inner_instructions"]; !ok {
+ metaJSON["inner_instructions"] = []any{}
+ } else {
+ innerInstructions, ok := metaJSON["inner_instructions"].([]any)
+ if ok && len(innerInstructions) > innerIndex {
+ relevantInner := innerInstructions[innerIndex].(map[string]any)
+ {
+ _, ok := relevantInner["instructions"].([]any)
+ if ok {
+ metaJSON["inner_instructions"].([]any)[innerIndex].(map[string]any)["instructions"].([]any)[instIndex] = parsedInstructionJSON
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ return resp, metaJSON, nil
+ }
+ }
+
+ return resp, meta, nil
case solana.EncodingJSON:
- return tx, nil
+ return tx, meta, nil
default:
- return nil, fmt.Errorf("unsupported encoding")
+ return nil, nil, fmt.Errorf("unsupported encoding")
+ }
+}
+
+func clone[T any](in []T) []T {
+ out := make([]T, len(in))
+ copy(out, in)
+ return out
+}
+
+func byeSliceToUint16Slice(in []byte) []uint16 {
+ out := make([]uint16, len(in))
+ for i, v := range in {
+ out[i] = uint16(v)
+ }
+ return out
+}
+
+func byteSlicesToKeySlices(keys [][]byte) []solana.PublicKey {
+ var out []solana.PublicKey
+ for _, key := range keys {
+ var k solana.PublicKey
+ copy(k[:], key)
+ out = append(out, k)
+ }
+ return out
+}
+
+func toUniqueSorted(accountIndexes []uint16) []uint16 {
+ seen := make(map[uint16]struct{})
+ var out []uint16
+ for _, v := range accountIndexes {
+ if _, ok := seen[v]; ok {
+ continue
+ }
+ seen[v] = struct{}{}
+ out = append(out, v)
}
+ sort.Slice(out, func(i, j int) bool {
+ return out[i] < out[j]
+ })
+ return out
}
func encodeBytesResponseBasedOnWantedEncoding(
diff --git a/storage.go b/storage.go
index 05cf8565..1587580b 100644
--- a/storage.go
+++ b/storage.go
@@ -92,7 +92,7 @@ func readSectionFromReaderAt(reader ReaderAtCloser, offset uint64, length uint64
return data, nil
}
-func readNodeFromReaderAtWithOffsetAndSize(reader ReaderAtCloser, wantedCid cid.Cid, offset uint64, length uint64) ([]byte, error) {
+func readNodeFromReaderAtWithOffsetAndSize(reader ReaderAtCloser, wantedCid *cid.Cid, offset uint64, length uint64) ([]byte, error) {
// read MaxVarintLen64 bytes
section := make([]byte, length)
_, err := reader.ReadAt(section, int64(offset))
diff --git a/store/LICENSE.md b/store/LICENSE.md
deleted file mode 100644
index 15601cba..00000000
--- a/store/LICENSE.md
+++ /dev/null
@@ -1,229 +0,0 @@
-The contents of this repository are Copyright (c) corresponding authors and
-contributors, licensed under the `Permissive License Stack` meaning either of:
-
-- Apache-2.0 Software License: https://www.apache.org/licenses/LICENSE-2.0
- ([...4tr2kfsq](https://gateway.ipfs.io/ipfs/bafkreiankqxazcae4onkp436wag2lj3ccso4nawxqkkfckd6cg4tr2kfsq))
-
-- MIT Software License: https://opensource.org/licenses/MIT
- ([...vljevcba](https://gateway.ipfs.io/ipfs/bafkreiepofszg4gfe2gzuhojmksgemsub2h4uy2gewdnr35kswvljevcba))
-
-You may not use the contents of this repository except in compliance
-with one of the listed Licenses. For an extended clarification of the
-intent behind the choice of Licensing please refer to
-https://protocol.ai/blog/announcing-the-permissive-license-stack/
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the terms listed in this notice is distributed on
-an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
-either express or implied. See each License for the specific language
-governing permissions and limitations under that License.
-
-
-`SPDX-License-Identifier: Apache-2.0 OR MIT`
-
-Verbatim copies of both licenses are included below:
-
-Apache-2.0 Software License
-
-```
- Apache License
- Version 2.0, January 2004
- http://www.apache.org/licenses/
-
- TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
- 1. Definitions.
-
- "License" shall mean the terms and conditions for use, reproduction,
- and distribution as defined by Sections 1 through 9 of this document.
-
- "Licensor" shall mean the copyright owner or entity authorized by
- the copyright owner that is granting the License.
-
- "Legal Entity" shall mean the union of the acting entity and all
- other entities that control, are controlled by, or are under common
- control with that entity. For the purposes of this definition,
- "control" means (i) the power, direct or indirect, to cause the
- direction or management of such entity, whether by contract or
- otherwise, or (ii) ownership of fifty percent (50%) or more of the
- outstanding shares, or (iii) beneficial ownership of such entity.
-
- "You" (or "Your") shall mean an individual or Legal Entity
- exercising permissions granted by this License.
-
- "Source" form shall mean the preferred form for making modifications,
- including but not limited to software source code, documentation
- source, and configuration files.
-
- "Object" form shall mean any form resulting from mechanical
- transformation or translation of a Source form, including but
- not limited to compiled object code, generated documentation,
- and conversions to other media types.
-
- "Work" shall mean the work of authorship, whether in Source or
- Object form, made available under the License, as indicated by a
- copyright notice that is included in or attached to the work
- (an example is provided in the Appendix below).
-
- "Derivative Works" shall mean any work, whether in Source or Object
- form, that is based on (or derived from) the Work and for which the
- editorial revisions, annotations, elaborations, or other modifications
- represent, as a whole, an original work of authorship. For the purposes
- of this License, Derivative Works shall not include works that remain
- separable from, or merely link (or bind by name) to the interfaces of,
- the Work and Derivative Works thereof.
-
- "Contribution" shall mean any work of authorship, including
- the original version of the Work and any modifications or additions
- to that Work or Derivative Works thereof, that is intentionally
- submitted to Licensor for inclusion in the Work by the copyright owner
- or by an individual or Legal Entity authorized to submit on behalf of
- the copyright owner. For the purposes of this definition, "submitted"
- means any form of electronic, verbal, or written communication sent
- to the Licensor or its representatives, including but not limited to
- communication on electronic mailing lists, source code control systems,
- and issue tracking systems that are managed by, or on behalf of, the
- Licensor for the purpose of discussing and improving the Work, but
- excluding communication that is conspicuously marked or otherwise
- designated in writing by the copyright owner as "Not a Contribution."
-
- "Contributor" shall mean Licensor and any individual or Legal Entity
- on behalf of whom a Contribution has been received by Licensor and
- subsequently incorporated within the Work.
-
- 2. Grant of Copyright License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- copyright license to reproduce, prepare Derivative Works of,
- publicly display, publicly perform, sublicense, and distribute the
- Work and such Derivative Works in Source or Object form.
-
- 3. Grant of Patent License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- (except as stated in this section) patent license to make, have made,
- use, offer to sell, sell, import, and otherwise transfer the Work,
- where such license applies only to those patent claims licensable
- by such Contributor that are necessarily infringed by their
- Contribution(s) alone or by combination of their Contribution(s)
- with the Work to which such Contribution(s) was submitted. If You
- institute patent litigation against any entity (including a
- cross-claim or counterclaim in a lawsuit) alleging that the Work
- or a Contribution incorporated within the Work constitutes direct
- or contributory patent infringement, then any patent licenses
- granted to You under this License for that Work shall terminate
- as of the date such litigation is filed.
-
- 4. Redistribution. You may reproduce and distribute copies of the
- Work or Derivative Works thereof in any medium, with or without
- modifications, and in Source or Object form, provided that You
- meet the following conditions:
-
- (a) You must give any other recipients of the Work or
- Derivative Works a copy of this License; and
-
- (b) You must cause any modified files to carry prominent notices
- stating that You changed the files; and
-
- (c) You must retain, in the Source form of any Derivative Works
- that You distribute, all copyright, patent, trademark, and
- attribution notices from the Source form of the Work,
- excluding those notices that do not pertain to any part of
- the Derivative Works; and
-
- (d) If the Work includes a "NOTICE" text file as part of its
- distribution, then any Derivative Works that You distribute must
- include a readable copy of the attribution notices contained
- within such NOTICE file, excluding those notices that do not
- pertain to any part of the Derivative Works, in at least one
- of the following places: within a NOTICE text file distributed
- as part of the Derivative Works; within the Source form or
- documentation, if provided along with the Derivative Works; or,
- within a display generated by the Derivative Works, if and
- wherever such third-party notices normally appear. The contents
- of the NOTICE file are for informational purposes only and
- do not modify the License. You may add Your own attribution
- notices within Derivative Works that You distribute, alongside
- or as an addendum to the NOTICE text from the Work, provided
- that such additional attribution notices cannot be construed
- as modifying the License.
-
- You may add Your own copyright statement to Your modifications and
- may provide additional or different license terms and conditions
- for use, reproduction, or distribution of Your modifications, or
- for any such Derivative Works as a whole, provided Your use,
- reproduction, and distribution of the Work otherwise complies with
- the conditions stated in this License.
-
- 5. Submission of Contributions. Unless You explicitly state otherwise,
- any Contribution intentionally submitted for inclusion in the Work
- by You to the Licensor shall be under the terms and conditions of
- this License, without any additional terms or conditions.
- Notwithstanding the above, nothing herein shall supersede or modify
- the terms of any separate license agreement you may have executed
- with Licensor regarding such Contributions.
-
- 6. Trademarks. This License does not grant permission to use the trade
- names, trademarks, service marks, or product names of the Licensor,
- except as required for reasonable and customary use in describing the
- origin of the Work and reproducing the content of the NOTICE file.
-
- 7. Disclaimer of Warranty. Unless required by applicable law or
- agreed to in writing, Licensor provides the Work (and each
- Contributor provides its Contributions) on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
- implied, including, without limitation, any warranties or conditions
- of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
- PARTICULAR PURPOSE. You are solely responsible for determining the
- appropriateness of using or redistributing the Work and assume any
- risks associated with Your exercise of permissions under this License.
-
- 8. Limitation of Liability. In no event and under no legal theory,
- whether in tort (including negligence), contract, or otherwise,
- unless required by applicable law (such as deliberate and grossly
- negligent acts) or agreed to in writing, shall any Contributor be
- liable to You for damages, including any direct, indirect, special,
- incidental, or consequential damages of any character arising as a
- result of this License or out of the use or inability to use the
- Work (including but not limited to damages for loss of goodwill,
- work stoppage, computer failure or malfunction, or any and all
- other commercial damages or losses), even if such Contributor
- has been advised of the possibility of such damages.
-
- 9. Accepting Warranty or Additional Liability. While redistributing
- the Work or Derivative Works thereof, You may choose to offer,
- and charge a fee for, acceptance of support, warranty, indemnity,
- or other liability obligations and/or rights consistent with this
- License. However, in accepting such obligations, You may act only
- on Your own behalf and on Your sole responsibility, not on behalf
- of any other Contributor, and only if You agree to indemnify,
- defend, and hold each Contributor harmless for any liability
- incurred by, or claims asserted against, such Contributor by reason
- of your accepting any such warranty or additional liability.
-
- END OF TERMS AND CONDITIONS
-```
-
-
-MIT Software License
-
-```
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
-```
-
diff --git a/store/filecache/filecache.go b/store/filecache/filecache.go
deleted file mode 100644
index 45938427..00000000
--- a/store/filecache/filecache.go
+++ /dev/null
@@ -1,264 +0,0 @@
-// Package filecache provides an LRU cache of opened files. If the same files
-// are frequently opened and closed this is useful for reducing the number of
-// syscalls for opening and closing the files.
-package filecache
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "container/list"
- "os"
- "sync"
-)
-
-// FileCache maintains a LRU cache of opened files. Its methods are safe to
-// call concurrently.
-type FileCache struct {
- cache map[string]*list.Element
- capacity int
- ll *list.List
- lock sync.Mutex
- onEvicted func(*os.File, int)
- openFlag int
- openPerm os.FileMode
- removed map[*os.File]int
-
- // Stats
- hit int
- miss int
-}
-
-type entry struct {
- file *os.File
- refs int
-}
-
-// New creates a new FileCache that can hold up to specified capacity of open
-// files. If capacity is 0, then there is no limit. Files are opened read-only.
-// If other open flags and permissions are needed, use NewOpenFile.
-func New(capacity int) *FileCache {
- return NewOpenFile(capacity, os.O_RDONLY, 0)
-}
-
-// NewOpenFile created a new FileCache that opens files using the specified
-// arguments to os.OpenFile.
-func NewOpenFile(capacity int, openFlag int, openPerm os.FileMode) *FileCache {
- if capacity < 0 {
- capacity = 0
- }
- return &FileCache{
- capacity: capacity,
- openFlag: openFlag,
- openPerm: openPerm,
- }
-}
-
-// Open returns the already opened file, or opens the named file and returns
-// that. The file is subsequently retrievable without opening it again, unless
-// it has been removed from the FileCache.
-//
-// All returned os.File instances are shared, so opeartions on these files must
-// use methods that do not depend on the current file position.
-//
-// Every call to Open must be accompanied by a call to Close. Otherwise,
-// reference counts will not be adjusted correctly and file handles will leak.
-func (c *FileCache) Open(name string) (*os.File, error) {
- return c.OpenWithFlag(name, c.openFlag)
-}
-
-func (c *FileCache) OpenWithFlag(name string, flag int) (*os.File, error) {
- c.lock.Lock()
- defer c.lock.Unlock()
-
- if c.capacity == 0 {
- return os.OpenFile(name, flag, c.openPerm)
- }
-
- if c.cache == nil {
- c.cache = make(map[string]*list.Element)
- c.ll = list.New()
- }
-
- if elem, ok := c.cache[name]; ok {
- c.ll.MoveToFront(elem)
- ent := elem.Value.(*entry)
- ent.refs++
- c.hit++
- return ent.file, nil
- }
- c.miss++
-
- file, err := os.OpenFile(name, flag, c.openPerm)
- if err != nil {
- return nil, err
- }
-
- c.cache[name] = c.ll.PushFront(&entry{file, 1})
- if c.capacity != 0 && c.ll.Len() > c.capacity {
- c.removeOldest()
- }
-
- return file, nil
-}
-
-// Close decrements the reference count on the file. If the file has been
-// removed from the cache and the reference count is zero, then the file is
-// closed.
-func (c *FileCache) Close(file *os.File) error {
- name := file.Name()
-
- c.lock.Lock()
- defer c.lock.Unlock()
-
- // First check if the file is in removed. A different File with the same
- // name may be in the cache, and would be the wrong one to close.
- refs, ok := c.removed[file]
- if ok {
- if refs == 1 {
- delete(c.removed, file)
- if len(c.removed) == 0 {
- c.removed = nil
- }
- return file.Close()
- }
- // Removed from cache, but still in use.
- c.removed[file] = refs - 1
- return nil
- }
-
- if elem, ok := c.cache[name]; ok {
- ent := elem.Value.(*entry)
- if ent.refs == 0 {
- return &os.PathError{Op: "close", Path: name, Err: os.ErrClosed}
- }
- ent.refs--
- return nil
- }
-
- // File is not in removed or in cache, so just close it
- return file.Close()
-}
-
-// Len return the number of open files in the FileCache.
-func (c *FileCache) Len() int {
- c.lock.Lock()
- defer c.lock.Unlock()
-
- if c.cache == nil {
- return 0
- }
- return c.ll.Len()
-}
-
-// Capacity return the capacity of the FileCache.
-func (c *FileCache) Cap() int {
- c.lock.Lock()
- defer c.lock.Unlock()
-
- return c.capacity
-}
-
-// Clear removes all files in the FileCache and closes those that have a zero
-// reference count.
-func (c *FileCache) Clear() {
- c.lock.Lock()
- defer c.lock.Unlock()
-
- for _, elem := range c.cache {
- c.removeElement(elem)
- }
- c.ll = nil
- c.cache = nil
-}
-
-// Remove removes the named file from the cache and closes it if it has a zero
-// reference count.
-func (c *FileCache) Remove(name string) {
- c.lock.Lock()
- defer c.lock.Unlock()
-
- if elem, ok := c.cache[name]; ok {
- c.removeElement(elem)
- }
-}
-
-// SetCacheSize sets the capacity of the FileCache. If the change reduces the
-// capacity to fewer items than are currently in the cache, then the oldest
-// items are removed until the number of items in the cache is equal to the new
-// capacity.
-func (c *FileCache) SetCacheSize(capacity int) {
- if capacity < 0 {
- capacity = 0
- }
-
- c.lock.Lock()
- defer c.lock.Unlock()
-
- if capacity < c.capacity {
- if capacity == 0 {
- for _, elem := range c.cache {
- c.removeElement(elem)
- }
- c.ll = nil
- c.cache = nil
- } else {
- for i := capacity; i < c.capacity; i++ {
- c.removeOldest()
- }
- }
- }
- c.capacity = capacity
-}
-
-// SetOnEvicted specifies a function to call when a file is removed from cache.
-func (c *FileCache) SetOnEvicted(f func(*os.File, int)) {
- c.lock.Lock()
- defer c.lock.Unlock()
-
- c.onEvicted = f
-}
-
-// Stats returns hit count, miss count, items in cache, and cache capacity.
-func (c *FileCache) Stats() (int, int, int, int) {
- c.lock.Lock()
- defer c.lock.Unlock()
-
- var items int
- if c.cache != nil {
- items = c.ll.Len()
- }
- // If exceeded max, reset.
- if c.hit < 0 || c.miss < 0 {
- c.hit = 0
- c.miss = 0
- }
- return c.hit, c.miss, items, c.capacity
-}
-
-func (c *FileCache) removeOldest() {
- elem := c.ll.Back()
- if elem != nil {
- c.removeElement(elem)
- }
-}
-
-func (c *FileCache) removeElement(elem *list.Element) {
- c.ll.Remove(elem)
- ent := elem.Value.(*entry)
- delete(c.cache, ent.file.Name())
- if c.onEvicted != nil {
- c.onEvicted(ent.file, ent.refs)
- }
- if ent.refs == 0 {
- ent.file.Close()
- return
- }
- // Removed from cache, but still in use.
- if c.removed == nil {
- c.removed = make(map[*os.File]int)
- }
- c.removed[ent.file] = ent.refs
-}
diff --git a/store/filecache/filecache_test.go b/store/filecache/filecache_test.go
deleted file mode 100644
index 8d050ef8..00000000
--- a/store/filecache/filecache_test.go
+++ /dev/null
@@ -1,356 +0,0 @@
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-package filecache
-
-import (
- "os"
- "path/filepath"
- "sync"
- "testing"
- "time"
-
- "github.com/stretchr/testify/require"
-)
-
-func TestOpen(t *testing.T) {
- var (
- evictedCount int
- evictedName string
- evictedRefs int
- )
- onEvicted := func(file *os.File, refs int) {
- t.Logf("Removed %q from cache", filepath.Base(file.Name()))
- evictedCount++
- evictedName = file.Name()
- evictedRefs = refs
- }
-
- fc := NewOpenFile(2, os.O_CREATE|os.O_RDWR, 0o644)
- fc.SetOnEvicted(onEvicted)
-
- tmp := t.TempDir()
- fooName := filepath.Join(tmp, "foo")
- barName := filepath.Join(tmp, "bar")
- bazName := filepath.Join(tmp, "baz")
-
- _, err := fc.Open(fooName)
- require.NoError(t, err)
-
- barFile, err := fc.Open(barName)
- require.NoError(t, err)
-
- fooFile, err := fc.Open(fooName)
- require.NoError(t, err)
-
- require.Zero(t, evictedCount)
-
- bazFile, err := fc.Open(bazName)
- require.NoError(t, err)
-
- require.Equal(t, 1, evictedCount)
- require.Equal(t, barName, evictedName)
- require.Equal(t, 1, evictedRefs)
-
- require.NoError(t, fc.Close(barFile))
-
- barFile, err = fc.Open(barName)
- require.NoError(t, err)
- require.NoError(t, fc.Close(barFile))
-
- require.Equal(t, 2, evictedCount)
- require.Equal(t, fooName, evictedName)
- require.Equal(t, 2, evictedRefs)
-
- require.NoError(t, fc.Close(fooFile))
- require.NoError(t, fc.Close(fooFile))
- err = fc.Close(fooFile)
- require.ErrorContains(t, err, os.ErrClosed.Error())
-
- fc.Remove(bazName)
-
- require.Equal(t, 3, evictedCount)
- require.Equal(t, bazName, evictedName)
- require.Equal(t, 1, evictedRefs)
-
- require.NoError(t, fc.Close(bazFile))
-
- err = fc.Close(bazFile)
- require.ErrorContains(t, err, os.ErrClosed.Error())
-
- // barFile closed, but still in cache, with zero references.
- require.Equal(t, 1, fc.Len())
- fc.Remove(barName)
-
- require.Zero(t, fc.Len())
- require.Zero(t, len(fc.removed))
-
- // Check that double close returns error
- fooFile, err = fc.Open(fooName)
- require.NoError(t, err)
- require.NoError(t, fc.Close(fooFile))
- err = fc.Close(fooFile)
- require.ErrorContains(t, err, os.ErrClosed.Error())
-
- fc.Clear()
-
- require.Zero(t, fc.Len())
- require.Zero(t, len(fc.removed))
-
- hits, misses, ln, cp := fc.Stats()
- t.Logf("Cache stats: hits=%d misses=%d len=%d cap=%d", hits, misses, ln, cp)
-}
-
-func TestMultiFileInstances(t *testing.T) {
- var (
- evictedCount int
- evictedName string
- evictedRefs int
- )
- onEvicted := func(file *os.File, refs int) {
- t.Logf("Removed %q from cache", filepath.Base(file.Name()))
- evictedCount++
- evictedName = file.Name()
- evictedRefs = refs
- }
- fc := NewOpenFile(2, os.O_CREATE|os.O_RDWR, 0o644)
- fc.SetOnEvicted(onEvicted)
-
- tmp := t.TempDir()
- fooName := filepath.Join(tmp, "foo")
- barName := filepath.Join(tmp, "bar")
- bazName := filepath.Join(tmp, "baz")
-
- // Incr reference count to 3.
- fooFile, err := fc.Open(fooName)
- require.NoError(t, err)
- _, err = fc.Open(fooName)
- require.NoError(t, err)
- _, err = fc.Open(fooName)
- require.NoError(t, err)
-
- barFile, err := fc.Open(barName)
- require.NoError(t, err)
- require.NoError(t, fc.Close(barFile))
-
- require.Equal(t, 0, len(fc.removed))
- require.Equal(t, 2, fc.Len())
-
- // Cause foo to be evicted.
- bazFile, err := fc.Open(bazName)
- require.NoError(t, err)
- require.NoError(t, fc.Close(bazFile))
- require.Equal(t, 2, fc.Len())
-
- // Since foo is still referenced, so should be put into removed.
- require.Equal(t, 1, len(fc.removed))
-
- require.Equal(t, 1, evictedCount)
- require.Equal(t, fooName, evictedName)
- require.Equal(t, 3, evictedRefs)
-
- // Open foo again, should be one file with reference of 1.
- fooFileX, err := fc.Open(fooName)
- require.NoError(t, err)
- require.NotEqual(t, fooFile, fooFileX)
-
- // Check that bar was evicted.
- require.Equal(t, 2, evictedCount)
- require.Equal(t, barName, evictedName)
- require.Equal(t, 0, evictedRefs)
-
- // Since bar was evicted with no references, it should not be put into
- // removed. Make sure that only fooFile is in removed.
- require.Equal(t, 1, len(fc.removed))
- refs, ok := fc.removed[fooFile]
- require.True(t, ok)
- require.Equal(t, 3, refs)
-
- // Remove the fooFileX from cache, without closing the file first. Since it
- // still has a non-zero reference count, it is put into removed, along with
- // the other instance of fooFile.
- fc.Remove(fooName)
-
- // Check that there are two distinct files in removed, with different
- // reference counts.
- require.Equal(t, 2, len(fc.removed))
- refs, ok = fc.removed[fooFile]
- require.True(t, ok)
- require.Equal(t, 3, refs)
- refs, ok = fc.removed[fooFileX]
- require.True(t, ok)
- require.Equal(t, 1, refs)
-
- // Close fooFileX and check that is no longer in removed.
- require.NoError(t, fc.Close(fooFileX))
- require.Equal(t, 1, len(fc.removed))
- refs, ok = fc.removed[fooFile]
- require.True(t, ok)
- require.Equal(t, 3, refs)
-
- // Closing fooFileX again should result in error.
- err = fc.Close(fooFileX)
- require.ErrorContains(t, err, os.ErrClosed.Error())
-
- // Make sure 3 closes are required to remove fooFile.
- require.NoError(t, fc.Close(fooFile))
- require.Equal(t, 1, len(fc.removed))
- require.NoError(t, fc.Close(fooFile))
- require.Equal(t, 1, len(fc.removed))
- require.NoError(t, fc.Close(fooFile))
- require.Equal(t, 0, len(fc.removed))
- err = fc.Close(fooFile)
- require.ErrorContains(t, err, os.ErrClosed.Error())
-
- // baz should still be in cache.
- require.Equal(t, 1, fc.Len())
-
- fc.Clear()
- require.Zero(t, fc.Len())
- require.Zero(t, len(fc.removed))
-
- hits, misses, ln, cp := fc.Stats()
- require.Equal(t, 2, hits)
- require.Equal(t, 4, misses)
- t.Logf("Cache stats: hits=%d misses=%d len=%d cap=%d", hits, misses, ln, cp)
-}
-
-func TestZeroSize(t *testing.T) {
- fc := NewOpenFile(0, os.O_CREATE|os.O_RDWR, 0o644)
-
- var evicted bool
- fc.SetOnEvicted(func(file *os.File, refs int) {
- evicted = true
- })
- require.Zero(t, fc.Len())
- require.Zero(t, fc.Cap())
-
- tmp := t.TempDir()
- fooName := filepath.Join(tmp, "foo")
- barName := filepath.Join(tmp, "bar")
-
- file1, err := fc.Open(fooName)
- require.NoError(t, err)
- require.Zero(t, fc.Len())
- require.False(t, evicted)
-
- file2, err := fc.Open(barName)
- require.NoError(t, err)
- require.False(t, evicted)
-
- require.Zero(t, fc.Len())
- require.Zero(t, len(fc.removed))
-
- require.NoError(t, fc.Close(file1))
- require.Zero(t, len(fc.removed))
-
- require.NoError(t, fc.Close(file2))
- require.Zero(t, len(fc.removed))
-
- require.Zero(t, fc.Len())
-}
-
-func TestFuzz(t *testing.T) {
- const (
- capacity = 3
- concurrency = 1000
- reps = 100
- delay = 500 * time.Microsecond
- )
-
- fc := NewOpenFile(capacity, os.O_CREATE|os.O_RDWR, 0o644)
-
- tmp := t.TempDir()
- fooName := filepath.Join(tmp, "foo")
- barName := filepath.Join(tmp, "bar")
- bazName := filepath.Join(tmp, "baz")
- bifName := filepath.Join(tmp, "bif")
- names := []string{fooName, barName, bazName, bifName}
-
- var wg sync.WaitGroup
- wg.Add(concurrency)
- for i := 0; i < concurrency; i++ {
- go func(name string) {
- for x := 0; x < reps; x++ {
- f, err := fc.Open(name)
- require.NoError(t, err, "opening file", name)
- time.Sleep(delay)
- err = fc.Close(f)
- require.NoError(t, err, "closing file", name)
- }
- defer wg.Done()
- }(names[i%len(names)])
- }
-
- wg.Wait()
-
- for name, elem := range fc.cache {
- ent := elem.Value.(*entry)
- require.Zero(t, ent.refs, "expected zero ref count for cached file", name)
- }
-
- require.Zero(t, len(fc.removed))
- fc.Clear()
- require.Zero(t, fc.Len())
- require.Zero(t, len(fc.cache))
- require.Zero(t, len(fc.removed))
-}
-
-func TestEvict(t *testing.T) {
- // Open a file to get 1st *File, let it be evicted, open that file again to
- // get 2nd *File, then close both 1st and 2nd *File. This tests that the
- // first Close closes the evicted *File, not the one in cache.
- const (
- capacity = 2
- )
-
- var evictions int
- onEvicted := func(file *os.File, refs int) {
- t.Logf("Removed %q from cache, refs: %d", filepath.Base(file.Name()), refs)
- evictions++
- }
- fc := NewOpenFile(capacity, os.O_CREATE|os.O_RDWR, 0o644)
- fc.SetOnEvicted(onEvicted)
-
- tmp := t.TempDir()
- fooName := filepath.Join(tmp, "foo")
- barName := filepath.Join(tmp, "bar")
- bazName := filepath.Join(tmp, "baz")
- names := []string{fooName, barName, bazName}
-
- var err error
- files := make([]*os.File, len(names))
- for i, name := range names {
- t.Log("Opening", name)
- files[i], err = fc.Open(name)
- require.NoError(t, err, "opening file", name)
- }
-
- t.Log("Opening", fooName, "again")
- f2, err := fc.Open(fooName)
- require.NoError(t, err, "opening file", fooName)
-
- for _, file := range files {
- t.Log("Closing", file.Name())
- err = fc.Close(file)
- require.NoError(t, err, "closing file", file.Name())
- }
-
- t.Log("Closing other handle for", f2.Name())
- err = fc.Close(f2)
- require.NoError(t, err, "closing file", f2.Name())
-
- require.Equal(t, len(names)+1-capacity, evictions)
- fc.Clear()
- require.Zero(t, fc.Len())
- require.Zero(t, len(fc.cache))
- require.Zero(t, len(fc.removed))
-
- // Check that all files are really closed.
- for _, file := range files {
- require.Error(t, file.Close())
- }
- require.Error(t, f2.Close())
-}
diff --git a/store/freelist/freelist.go b/store/freelist/freelist.go
deleted file mode 100644
index 78661311..00000000
--- a/store/freelist/freelist.go
+++ /dev/null
@@ -1,220 +0,0 @@
-package freelist
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bufio"
- "encoding/binary"
- "fmt"
- "io"
- "os"
- "sync"
-
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-const CIDSizePrefix = 4
-
-// A primary storage that is CID aware.
-type FreeList struct {
- file *os.File
- writer *bufio.Writer
- outstandingWork types.Work
- blockPool []types.Block
- poolLk sync.RWMutex
- flushLock sync.Mutex
-}
-
-const (
- // blockBufferSize is the size of I/O buffers. If has the same size as the
- // linux pipe size.
- blockBufferSize = 16 * 4096
- // blockPoolSize is the size of the freelist cache.
- blockPoolSize = 1024
-)
-
-func Open(path string) (*FreeList, error) {
- file, err := os.OpenFile(path, os.O_RDWR|os.O_APPEND|os.O_CREATE, 0o644)
- if err != nil {
- return nil, err
- }
- return &FreeList{
- file: file,
- writer: bufio.NewWriterSize(file, blockBufferSize),
- blockPool: make([]types.Block, 0, blockPoolSize),
- }, nil
-}
-
-func (cp *FreeList) Put(blk types.Block) error {
- cp.poolLk.Lock()
- defer cp.poolLk.Unlock()
- cp.blockPool = append(cp.blockPool, blk)
- // Offset = 8bytes + Size = 4bytes = 12 Bytes
- cp.outstandingWork += types.Work(types.SizeBytesLen + types.OffBytesLen)
- return nil
-}
-
-func (cp *FreeList) flushBlock(blk types.Block) (types.Work, error) {
- sizeBuf := make([]byte, types.SizeBytesLen)
- offBuf := make([]byte, types.OffBytesLen)
- // NOTE: If Position or Size types change, this needs to change.
- binary.LittleEndian.PutUint64(offBuf, uint64(blk.Offset))
- binary.LittleEndian.PutUint32(sizeBuf, uint32(blk.Size))
- // We append offset to size in free list
- if _, err := cp.writer.Write(offBuf); err != nil {
- return 0, err
- }
- if _, err := cp.writer.Write(sizeBuf); err != nil {
- return 0, err
- }
- return types.Work(types.SizeBytesLen + types.OffBytesLen), nil
-}
-
-// Flush writes outstanding work and buffered data to the freelist file.
-func (cp *FreeList) Flush() (types.Work, error) {
- cp.flushLock.Lock()
- defer cp.flushLock.Unlock()
-
- cp.poolLk.Lock()
- if len(cp.blockPool) == 0 {
- cp.poolLk.Unlock()
- return 0, nil
- }
- blocks := cp.blockPool
- cp.blockPool = make([]types.Block, 0, blockPoolSize)
- cp.outstandingWork = 0
- cp.poolLk.Unlock()
-
- // The pool lock is released allowing Put to write to nextPool. The
- // flushLock is still held, preventing concurrent flushes from changing the
- // pool or accessing writer.
-
- if len(blocks) == 0 {
- return 0, nil
- }
-
- var work types.Work
- for _, record := range blocks {
- blockWork, err := cp.flushBlock(record)
- if err != nil {
- return 0, err
- }
- work += blockWork
- }
- err := cp.writer.Flush()
- if err != nil {
- return 0, fmt.Errorf("cannot flush data to freelist file %s: %w", cp.file.Name(), err)
- }
-
- return work, nil
-}
-
-// Sync commits the contents of the freelist file to disk. Flush should be
-// called before calling Sync.
-func (cp *FreeList) Sync() error {
- cp.flushLock.Lock()
- defer cp.flushLock.Unlock()
- return cp.file.Sync()
-}
-
-// Close calls Flush to write work and data to the freelist file, and then
-// closes the file.
-func (cp *FreeList) Close() error {
- _, err := cp.Flush()
- if err != nil {
- cp.file.Close()
- return err
- }
- return cp.file.Close()
-}
-
-func (cp *FreeList) OutstandingWork() types.Work {
- cp.poolLk.RLock()
- defer cp.poolLk.RUnlock()
- return cp.outstandingWork
-}
-
-func (cp *FreeList) Iter() (*Iterator, error) {
- return NewIterator(cp.file), nil
-}
-
-func NewIterator(reader io.Reader) *Iterator {
- return &Iterator{
- reader: reader,
- }
-}
-
-type Iterator struct {
- reader io.Reader
-}
-
-func (cpi *Iterator) Next() (*types.Block, error) {
- data := make([]byte, types.OffBytesLen+types.SizeBytesLen)
- _, err := io.ReadFull(cpi.reader, data)
- if err != nil {
- return nil, err
- }
- offset := binary.LittleEndian.Uint64(data)
- size := binary.LittleEndian.Uint32(data[types.OffBytesLen:])
- return &types.Block{Size: types.Size(size), Offset: types.Position(offset)}, nil
-}
-
-// StorageSize returns bytes of storage used by the freelist.
-func (fl *FreeList) StorageSize() (int64, error) {
- fi, err := fl.file.Stat()
- if err != nil {
- if os.IsNotExist(err) {
- return 0, nil
- }
- return 0, err
- }
-
- return fi.Size(), nil
-}
-
-// ToGC moves the current freelist file into a ".gc" file and creates a new
-// freelist file. This allows the garbage collector to then process the .gc
-// freelist file while allowing the freelist to continue to operate on a new
-// file.
-func (cp *FreeList) ToGC() (string, error) {
- fileName := cp.file.Name()
- workFilePath := fileName + ".gc"
-
- // If a .gc file already exists, return the existing one becuase it means
- // that GC did not finish processing it.
- _, err := os.Stat(workFilePath)
- if !os.IsNotExist(err) {
- if err != nil {
- return "", err
- }
- return workFilePath, nil
- }
-
- _, err = cp.Flush()
- if err != nil {
- return "", err
- }
-
- cp.flushLock.Lock()
- defer cp.flushLock.Unlock()
-
- // Flush any buffered data and close the file. Safe to do with flushLock
- // acquired.
- cp.writer.Flush()
- cp.file.Close()
- err = os.Rename(fileName, workFilePath)
- if err != nil {
- return "", err
- }
-
- cp.file, err = os.OpenFile(fileName, os.O_RDWR|os.O_APPEND|os.O_CREATE, 0o644)
- if err != nil {
- return "", err
- }
- cp.writer.Reset(cp.file)
-
- return workFilePath, nil
-}
diff --git a/store/freelist/freelist_test.go b/store/freelist/freelist_test.go
deleted file mode 100644
index 50410f84..00000000
--- a/store/freelist/freelist_test.go
+++ /dev/null
@@ -1,99 +0,0 @@
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-package freelist_test
-
-import (
- "io"
- "math/rand"
- "os"
- "path/filepath"
- "testing"
-
- "github.com/rpcpool/yellowstone-faithful/store/freelist"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-func TestFLPut(t *testing.T) {
- tempDir := t.TempDir()
- flPath := filepath.Join(tempDir, "storethehash.free")
- fl, err := freelist.Open(flPath)
- require.NoError(t, err)
-
- blks := generateFreeListEntries(100)
- for _, blk := range blks {
- err := fl.Put(blk)
- require.NoError(t, err)
- }
-
- outstandingWork := fl.OutstandingWork()
- expectedStorage := 100 * (types.SizeBytesLen + types.OffBytesLen)
- require.Equal(t, types.Work(expectedStorage), outstandingWork)
- work, err := fl.Flush()
- require.NoError(t, err)
- require.Equal(t, types.Work(expectedStorage), work)
- err = fl.Sync()
- require.NoError(t, err)
- err = fl.Close()
- require.NoError(t, err)
-
- file, err := os.Open(flPath)
- t.Cleanup(func() { file.Close() })
- require.NoError(t, err)
- iter := freelist.NewIterator(file)
- for _, expectedBlk := range blks {
- blk, err := iter.Next()
- require.NoError(t, err)
- require.Equal(t, expectedBlk.Size, blk.Size)
- require.Equal(t, expectedBlk.Offset, blk.Offset)
- }
- _, err = iter.Next()
- require.EqualError(t, err, io.EOF.Error())
-
- err = file.Close()
- require.NoError(t, err)
-}
-
-func TestToGC(t *testing.T) {
- tempDir := t.TempDir()
- flPath := filepath.Join(tempDir, "storethehash.free")
- fl, err := freelist.Open(flPath)
- require.NoError(t, err)
- t.Cleanup(func() { fl.Close() })
-
- blks := generateFreeListEntries(100)
- for _, blk := range blks {
- err := fl.Put(blk)
- require.NoError(t, err)
- }
- _, err = fl.Flush()
- require.NoError(t, err)
-
- flsize, err := fl.StorageSize()
- require.NoError(t, err)
-
- gcfile, err := fl.ToGC()
- require.NoError(t, err)
-
- fi, err := os.Stat(gcfile)
- require.NoError(t, err)
- require.Equal(t, flsize, fi.Size())
-
- flsize, err = fl.StorageSize()
- require.NoError(t, err)
- require.Zero(t, flsize)
-}
-
-func generateFreeListEntries(n int) []types.Block {
- blks := make([]types.Block, 0)
- for i := 0; i < n; i++ {
- blks = append(blks, types.Block{
- Size: types.Size(rand.Int31()),
- Offset: types.Position(rand.Int63()),
- })
- }
- return blks
-}
diff --git a/store/index/buckets.go b/store/index/buckets.go
deleted file mode 100644
index c3a22d90..00000000
--- a/store/index/buckets.go
+++ /dev/null
@@ -1,37 +0,0 @@
-package index
-
-import "github.com/rpcpool/yellowstone-faithful/store/types"
-
-// BucketIndex is an index to a bucket
-type BucketIndex uint32
-
-// Buckets contains pointers to file offsets
-//
-// The generic specifies how many bits are used to create the buckets. The number of buckets is
-// 2 ^ bits.
-type Buckets []types.Position
-
-// NewBuckets returns a list of buckets for the given index size in bits
-func NewBuckets(indexSizeBits uint8) (Buckets, error) {
- if indexSizeBits > 32 {
- return nil, types.ErrIndexTooLarge
- }
- return make(Buckets, 1< len(b)-1 {
- return types.ErrOutOfBounds
- }
- b[int(index)] = offset
- return nil
-}
-
-// Get updates returns the value at the given index
-func (b Buckets) Get(index BucketIndex) (types.Position, error) {
- if int(index) > len(b)-1 {
- return 0, types.ErrOutOfBounds
- }
- return b[int(index)], nil
-}
diff --git a/store/index/buckets_test.go b/store/index/buckets_test.go
deleted file mode 100644
index 9c1ce162..00000000
--- a/store/index/buckets_test.go
+++ /dev/null
@@ -1,63 +0,0 @@
-package index_test
-
-import (
- "testing"
-
- "github.com/rpcpool/yellowstone-faithful/store/index"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-func TestNewBuckets(t *testing.T) {
- var bucketBits uint8 = 24
- buckets, err := index.NewBuckets(bucketBits)
- require.NoError(t, err)
- require.Equal(t, len(buckets), 1< 0 {
- // Files were truncated, skip scan for fewer gc cycles.
- freeSkipIncr--
- }
- freeSkip = freeSkipIncr
- } else {
- // One less cycle until truncateFreeFiles tried again.
- freeSkip--
- }
- }(ctx)
- case <-gcDone:
- gcDone = nil
- t.Reset(interval)
- }
- }
-}
-
-// gc searches for and removes stale index files. Returns the number of unused
-// index files that were removed and the number of freeFiles that were found.
-func (index *Index) gc(ctx context.Context, scanFree bool) (int64, int, error) {
- var emptied int
- var reclaimed int64
- var err error
-
- if scanFree {
- reclaimed, emptied, err = index.truncateFreeFiles(ctx)
- if err != nil {
- if err == context.DeadlineExceeded {
- return reclaimed, emptied, err
- }
- return 0, 0, err
- }
- log.Debugf("Emptied %d unused index files", emptied)
- }
-
- header, err := readHeader(index.headerPath)
- if err != nil {
- return 0, 0, err
- }
-
- index.flushLock.Lock()
- lastFileNum := index.fileNum
- index.flushLock.Unlock()
-
- if header.FirstFile == lastFileNum {
- return reclaimed, emptied, nil
- }
-
- var firstFileNum uint32
- if index.gcResume {
- firstFileNum = index.gcResumeAt
- index.gcResume = false
- log.Debugw("Resuming GC", "file", filepath.Base(indexFileName(index.basePath, firstFileNum)))
- } else {
- firstFileNum = header.FirstFile
- }
-
- var seenFirst bool
- for fileNum := firstFileNum; fileNum != lastFileNum; {
- indexPath := indexFileName(index.basePath, fileNum)
-
- stale, err := index.reapIndexRecords(ctx, fileNum, indexPath)
- if err != nil {
- if err == context.DeadlineExceeded {
- index.gcResumeAt = fileNum
- index.gcResume = true
- return reclaimed, emptied, err
- }
- return 0, 0, err
- }
- if stale {
- index.fileCache.Remove(indexPath)
-
- // If this is first index file, then update header and remove file.
- if header.FirstFile == fileNum {
- header.FirstFile++
- err = writeHeader(index.headerPath, header)
- if err != nil {
- return 0, 0, err
- }
- err = os.Remove(indexPath)
- if err != nil {
- return 0, 0, err
- }
- seenFirst = true
- }
- }
-
- fileNum++
- if fileNum == lastFileNum {
- if seenFirst {
- break
- }
- fileNum = header.FirstFile
- }
- if fileNum == firstFileNum {
- // Back to where gc started, all done.
- break
- }
- }
- return reclaimed, emptied, nil
-}
-
-func (index *Index) truncateFreeFiles(ctx context.Context) (int64, int, error) {
- header, err := readHeader(index.headerPath)
- if err != nil {
- return 0, 0, fmt.Errorf("cannot read index header: %w", err)
- }
- index.flushLock.Lock()
- lastFileNum := index.fileNum
- index.flushLock.Unlock()
-
- fileCount := lastFileNum - header.FirstFile
- if fileCount == 0 {
- return 0, 0, nil
- }
-
- busySet := make(map[uint32]struct{}, fileCount)
- maxFileSize := index.maxFileSize
- end := 1 << index.sizeBits
- tmpBuckets := make([]types.Position, 4096)
- for i := 0; i < end; {
- index.bucketLk.RLock()
- i += copy(tmpBuckets, index.buckets[i:])
- index.bucketLk.RUnlock()
- for _, offset := range tmpBuckets {
- ok, fileNum := bucketPosToFileNum(offset, maxFileSize)
- if ok {
- busySet[fileNum] = struct{}{}
- }
- }
- }
-
- var emptied int
- var reclaimed int64
- basePath := index.basePath
-
- for fileNum := header.FirstFile; fileNum != lastFileNum; fileNum++ {
- if _, busy := busySet[fileNum]; busy {
- continue
- }
-
- if ctx.Err() != nil {
- return reclaimed, emptied, ctx.Err()
- }
-
- indexPath := indexFileName(basePath, fileNum)
-
- index.fileCache.Remove(indexPath)
-
- fi, err := os.Stat(indexPath)
- if err != nil {
- log.Errorw("Cannot stat index file", "err", err, "file", indexPath)
- continue
- }
- reclaimed += fi.Size()
-
- // If this is first index file, then update header and remove file.
- if header.FirstFile == fileNum {
- header.FirstFile++
- if err = writeHeader(index.headerPath, header); err != nil {
- return 0, 0, err
- }
- if err = os.Remove(indexPath); err != nil {
- return 0, 0, err
- }
- emptied++
- log.Debugw("Removed unused index file", "file", indexPath)
- continue
- }
-
- if fi.Size() == 0 {
- continue
- }
-
- err = os.Truncate(indexPath, 0)
- if err != nil {
- log.Errorw("Error truncating index file", "err", err, "file", indexPath)
- continue
- }
- emptied++
- log.Debugw("Emptied unused index file", "file", indexPath)
- }
-
- return reclaimed, emptied, nil
-}
-
-// reapIndexRecords scans a single index file, logically deleting records that
-// are not referenced by a bucket, merging spans of deleted records, and
-// truncating deleted records from the end of the file.
-func (index *Index) reapIndexRecords(ctx context.Context, fileNum uint32, indexPath string) (bool, error) {
- fi, err := os.Stat(indexPath)
- if err != nil {
- return false, fmt.Errorf("cannot stat index file: %w", err)
- }
- if fi.Size() == 0 {
- // File is empty, so OK to delete if it is first file.
- return true, nil
- }
-
- file, err := os.OpenFile(indexPath, os.O_RDWR, 0o644)
- if err != nil {
- return false, err
- }
- defer file.Close()
-
- var freedCount, mergedCount int
- var freeAtSize uint32
- var busyAt, freeAt int64
- freeAt = -1
- busyAt = -1
-
- sizeBuf := make([]byte, sizePrefixSize)
- scratch := make([]byte, 256)
- var pos int64
- for {
- if ctx.Err() != nil {
- return false, ctx.Err()
- }
- if _, err = file.ReadAt(sizeBuf, pos); err != nil {
- if errors.Is(err, io.EOF) {
- // Finished reading entire index.
- break
- }
- return false, err
- }
-
- size := binary.LittleEndian.Uint32(sizeBuf)
- if size&deletedBit != 0 {
- // Record is already deleted.
- size ^= deletedBit
- if freeAt > busyAt {
- // Previous record free, so merge this record into the last.
- freeAtSize += sizePrefixSize + size
- if freeAtSize >= deletedBit {
- log.Warnf("Records are too large to merge %d >= %d", freeAtSize, deletedBit)
- freeAt = pos
- freeAtSize = size
- } else {
- binary.LittleEndian.PutUint32(sizeBuf, freeAtSize|deletedBit)
- _, err = file.WriteAt(sizeBuf, freeAt)
- if err != nil {
- return false, fmt.Errorf("cannot write to index file %s: %w", file.Name(), err)
- }
- mergedCount++
- }
- } else {
- // Previous record was not free, so mark new free position.
- freeAt = pos
- freeAtSize = size
- }
- pos += sizePrefixSize + int64(size)
- continue
- }
-
- if int(size) > len(scratch) {
- scratch = make([]byte, size)
- }
- data := scratch[:size]
- if _, err = file.ReadAt(data, pos+sizePrefixSize); err != nil {
- if errors.Is(err, io.EOF) {
- // The data has not been written yet, or the file is corrupt.
- // Take the data we are able to use and move on.
- break
- }
- return false, fmt.Errorf("error reading data from index: %w", err)
- }
-
- bucketPrefix := BucketIndex(binary.LittleEndian.Uint32(data))
- inUse, err := index.busy(bucketPrefix, pos+sizePrefixSize, fileNum)
- if err != nil {
- return false, err
- }
- if inUse {
- // Record is in use.
- busyAt = pos
- } else {
- // Record is free.
- if freeAt > busyAt {
- // Merge this free record into the last
- freeAtSize += sizePrefixSize + size
- if freeAtSize >= deletedBit {
- log.Warn("Records are too large to merge")
- freeAt = pos
- freeAtSize = size
- } else {
- mergedCount++
- }
- } else {
- freeAt = pos
- freeAtSize = size
- }
-
- // Mark the record as deleted by setting the highest bit in the
- // size. This assumes that the size of an individual index record
- // will always be less than 2^30.
- binary.LittleEndian.PutUint32(sizeBuf, freeAtSize|deletedBit)
- if _, err = file.WriteAt(sizeBuf, freeAt); err != nil {
- return false, fmt.Errorf("cannot write to index file %s: %w", file.Name(), err)
- }
- freedCount++
- }
- pos += sizePrefixSize + int64(size)
- }
-
- fileName := filepath.Base(file.Name())
- log.Debugw("Marked index records as free", "freed", freedCount, "merged", mergedCount, "file", fileName)
-
- // If there is a span of free records at end of file, truncate file.
- if freeAt > busyAt {
- // End of primary is free.
- if err = file.Truncate(freeAt); err != nil {
- return false, fmt.Errorf("failed to truncate index file: %w", err)
- }
- log.Debugw("Removed free records from end of index file", "file", fileName, "at", freeAt, "bytes", freeAtSize)
- if freeAt == 0 {
- return true, nil
- }
- }
-
- return false, nil
-}
-
-func (index *Index) busy(bucketPrefix BucketIndex, localPos int64, fileNum uint32) (bool, error) {
- index.bucketLk.RLock()
- bucketPos, err := index.buckets.Get(bucketPrefix)
- index.bucketLk.RUnlock()
- if err != nil {
- return false, err
- }
- localPosInBucket, fileNumInBucket := localizeBucketPos(bucketPos, index.maxFileSize)
- if fileNum == fileNumInBucket && localPos == int64(localPosInBucket) {
- return true, nil
- }
- return false, nil
-}
diff --git a/store/index/gc_test.go b/store/index/gc_test.go
deleted file mode 100644
index 12e56858..00000000
--- a/store/index/gc_test.go
+++ /dev/null
@@ -1,152 +0,0 @@
-package index
-
-import (
- "context"
- "encoding/binary"
- "fmt"
- "os"
- "path/filepath"
- "testing"
-
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/stretchr/testify/require"
-)
-
-func TestGC(t *testing.T) {
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, filepath.Base(testIndexPath))
-
- // Copy test file.
- err := copyFile(testIndexPath, indexPath)
- require.NoError(t, err)
-
- fc := filecache.New(1)
-
- // Open index and with nil primary to avoid attempting to remap.
- idx, err := Open(context.Background(), indexPath, nil, 24, 1024, 0, 0, fc)
- require.NoError(t, err)
- defer idx.Close()
-
- // All index files in use, so gc should not remove any files.
- reclaimed, emptied, err := idx.gc(context.Background(), true)
- require.NoError(t, err)
- require.Zero(t, reclaimed)
- require.Zero(t, emptied)
-
- require.NoError(t, idx.Close())
-
- // Copy the first two files as the last two files so that the indexes in
- // them are associated with the last files.
- err = copyFile(indexPath+".0", fmt.Sprintf("%s.%d", indexPath, idx.fileNum+1))
- require.NoError(t, err)
- err = copyFile(indexPath+".1", fmt.Sprintf("%s.%d", indexPath, idx.fileNum+2))
- require.NoError(t, err)
-
- // Adding index files invalidates the saved bucket state.
- require.NoError(t, RemoveSavedBuckets(indexPath))
-
- // Open the index with the duplicated files.
- idx, err = Open(context.Background(), indexPath, nil, 24, 1024, 0, 0, fc)
- require.NoError(t, err)
- defer idx.Close()
-
- // GC should now remove the first 2 files only.
- reclaimed, emptied, err = idx.gc(context.Background(), true)
- require.NoError(t, err)
- require.Equal(t, int64(2068), reclaimed)
- require.Equal(t, 2, emptied)
-
- // Another GC should not remove files.
- reclaimed, emptied, err = idx.gc(context.Background(), true)
- require.NoError(t, err)
- require.Zero(t, reclaimed)
- require.Zero(t, emptied)
-
- // Check that first file is .2 and last file is .24
- header, err := readHeader(idx.headerPath)
- require.NoError(t, err)
- require.Equal(t, header.FirstFile, uint32(2))
- require.Equal(t, idx.fileNum, uint32(24))
-
- // --- Test truncation ---
-
- // Remove buckets for last two records in 2nd to last index file.
- bucketY := 7143210
- bucketZ := 12228148
- idx.buckets[bucketY] = 0
- idx.buckets[bucketZ] = 0
-
- recordSize := int64(18 + sizePrefixSize)
-
- fileName := indexFileName(idx.basePath, 23)
- fi, err := os.Stat(fileName)
- require.NoError(t, err)
- sizeBefore := fi.Size()
- t.Log("File size before truncation:", sizeBefore)
-
- // Run GC and check that second to last file was truncated by two records.
- reclaimed, _, err = idx.gc(context.Background(), false)
- require.NoError(t, err)
- require.Zero(t, reclaimed)
-
- fi, err = os.Stat(fileName)
- require.NoError(t, err)
- sizeAfter := fi.Size()
- t.Log("File size after trucation:", sizeAfter)
- require.Equal(t, sizeAfter, sizeBefore-(2*recordSize))
-
- // --- Test dead record merge ---
-
- // Remove buckets for first two records in 2nd to last index file.
- bucketY = 719032
- bucketZ = 5851659
- idx.buckets[bucketY] = 0
- idx.buckets[bucketZ] = 0
-
- sizeBefore = fi.Size()
-
- var deleted bool
- sizeBuffer := make([]byte, sizePrefixSize)
-
- // Read first record size and deleted bit before GC.
- file, err := openFileForScan(fileName)
- require.NoError(t, err)
- _, err = file.ReadAt(sizeBuffer, 0)
- require.NoError(t, err)
- size := binary.LittleEndian.Uint32(sizeBuffer)
- if size&deletedBit != 0 {
- deleted = true
- size ^= deletedBit
- }
- size1Before := size
- require.False(t, deleted)
- file.Close()
- t.Log("Record size before:", size1Before)
-
- // Run GC and check that first and second records were merged into one free record.
- reclaimed, _, err = idx.gc(context.Background(), false)
- require.NoError(t, err)
- require.Zero(t, reclaimed)
-
- fi, err = os.Stat(fileName)
- require.NoError(t, err)
- sizeAfter = fi.Size()
-
- // File should not have changed size.
- require.Equal(t, sizeAfter, sizeBefore)
-
- // Read first record size and deleted bit before GC.
- file, err = openFileForScan(fileName)
- require.NoError(t, err)
- _, err = file.ReadAt(sizeBuffer, 0)
- require.NoError(t, err)
- size = binary.LittleEndian.Uint32(sizeBuffer)
- if size&deletedBit != 0 {
- deleted = true
- size ^= deletedBit
- }
- t.Log("Record size after:", size)
- require.True(t, deleted)
- require.Equal(t, size1Before+sizePrefixSize+size1Before, size)
- file.Close()
-}
diff --git a/store/index/header.go b/store/index/header.go
deleted file mode 100644
index 6b53d2e6..00000000
--- a/store/index/header.go
+++ /dev/null
@@ -1,59 +0,0 @@
-package index
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "encoding/json"
- "os"
-)
-
-// Header contains information about the index. This is actually stored in a
-// separate ".info" file, but is the first file read when the index is opened.
-type Header struct {
- // A version number in case we change the header
- Version int
- // The number of bits used to determine the in-memory buckets
- BucketsBits byte
- // MaxFileSize is the size limit of each index file. This cannot be greater
- // than 4GiB.
- MaxFileSize uint32
- // First index file number
- FirstFile uint32
- // PrimaryFileSize is the primary's maximum size, if applicable.
- PrimaryFileSize uint32
-}
-
-func newHeader(bucketsBits byte, maxFileSize uint32) Header {
- return Header{
- Version: IndexVersion,
- BucketsBits: bucketsBits,
- MaxFileSize: maxFileSize,
- }
-}
-
-func readHeader(filePath string) (Header, error) {
- data, err := os.ReadFile(filePath)
- if err != nil {
- return Header{}, err
- }
-
- var header Header
- err = json.Unmarshal(data, &header)
- if err != nil {
- return Header{}, err
- }
-
- return header, nil
-}
-
-func writeHeader(headerPath string, header Header) error {
- data, err := json.Marshal(&header)
- if err != nil {
- return err
- }
-
- return os.WriteFile(headerPath, data, 0o666)
-}
diff --git a/store/index/index.go b/store/index/index.go
deleted file mode 100644
index 66015daa..00000000
--- a/store/index/index.go
+++ /dev/null
@@ -1,1519 +0,0 @@
-package index
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bufio"
- "bytes"
- "context"
- "encoding/binary"
- "errors"
- "fmt"
- "io"
- "os"
- "path/filepath"
- "sync"
- "time"
-
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/rpcpool/yellowstone-faithful/store/primary"
- "github.com/rpcpool/yellowstone-faithful/store/primary/gsfaprimary"
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-/* An append-only log [`recordlist`]s.
-
-The format of that append only log is:
-
-```text
- | Repeated |
- | |
- | 4 bytes | Variable size | … |
- | Size of the Recordlist | Recordlist | … |
-```
-*/
-
-// In-memory buckets are used to track the location of records within the index
-// files. The buckets map a bit-prefix to a bucketPos value. The bucketPos
-// encodes both the index file number and the record offset within that file.
-// If 1GiB is the maximum size for a file, then the local data offset is kept
-// in the first GiB worth of bits (30) of the bucketPos. The file number is
-// kept in the bits above that. It is necessary for the file number to wrap
-// before it reaches a value greater than the number of bits available to
-// record it in the buckerPos. This results in a trade-off between allowing
-// larger files or allowing more files, but with the same overall maximum
-// storage.
-//
-// With a 1GiB local offset taking the first 30 bits of a 64 bit number, that
-// leaves 34 bits left to encode the file number. Instead of having logic to
-// wrap the file number at the largest value allowed by the available bits, the
-// file number is represented as a 32-bit value that always wraps at 2^32.
-//
-// Since the file number wraps 2^32 this means there can never be more than
-// 2^32 active index files. This also means that maxFileSize should never be
-// greater than 2^32. Using a maxFileSize of 2^30, the default, and a 32-bit
-// file number, results in 2 bits unused in the bucketPos address space. With a
-// smaller maxFileSize more bits would be unused.
-//
-// Smaller values for maxFileSize result in more files needed to hold the
-// index, but also more granular GC. A value too small risks running out of
-// inodes on the file system, and a value too large means that there is more
-// stale data that GC cannot remove. Using a 1GiB index file size limit offers
-// a good balance, and this value should not be changed (other than for
-// testing) by more than a factor of 4.
-
-const (
- // IndexVersion is stored in the header data to indicate how to interpret
- // index data.
- IndexVersion = 3
-
- // defaultIndexSizeBits is the default number of bits in an index prefix.
- defaultIndexSizeBits = uint8(24)
-
- // defaultMaxFileSize is the default size at which to start a new file.
- defaultMaxFileSize = 1024 * 1024 * 1024
-
- // sizePrefixSize is the number of bytes used for the size prefix of a
- // record list.
- sizePrefixSize = 4
-
- // indexBufferSize is the size of I/O buffers. If has the same size as the
- // linux pipe size.
- indexBufferSize = 16 * 4096
-
- // bucketPoolSize is the bucket cache size.
- bucketPoolSize = 1024
-
- // deletedBit is the highest order bit in the uint32 size part of a file
- // record, and when set, indicates that the record is deleted. This means
- // that record sizes must be less than 2^31.
- deletedBit = uint32(1 << 31)
-)
-
-// stripBucketPrefix removes the prefix that is used for the bucket.
-//
-// The first bits of a key are used to determine the bucket to put the key
-// into. This function removes those bytes. Only bytes that are fully covered
-// by the bits are removed. E.g. a bit value of 19 will remove only 2 bytes,
-// whereas 24 bits removes 3 bytes.
-func stripBucketPrefix(key []byte, bits byte) []byte {
- prefixLen := int(bits / 8)
- if len(key) < prefixLen {
- return nil
- }
- return key[prefixLen:]
-}
-
-type Index struct {
- sizeBits uint8
- maxFileSize uint32
- buckets Buckets
- file *os.File
- fileNum uint32
- headerPath string
- writer *bufio.Writer
- Primary primary.PrimaryStorage
- bucketLk sync.RWMutex
- flushLock sync.Mutex
- outstandingWork types.Work
- curPool, nextPool bucketPool
- length types.Position
- basePath string
- fileCache *filecache.FileCache
- closeOnce sync.Once
-
- gcDone chan struct{}
- gcResumeAt uint32
- gcResume bool
- gcStop chan struct{}
-}
-
-type bucketPool map[BucketIndex][]byte
-
-// Open opens the index for the given primary. The index is created if there is
-// no existing index at the specified path. If there is an older version index,
-// then it is automatically upgraded.
-//
-// Specifying 0 for indexSizeBits and maxFileSize results in using the values
-// of the existing index, or default values if no index exists. A gcInterval of
-// 0 disables garbage collection.
-func Open(ctx context.Context, path string, primary primary.PrimaryStorage, indexSizeBits uint8, maxFileSize uint32, gcInterval, gcTimeLimit time.Duration, fileCache *filecache.FileCache) (*Index, error) {
- var file *os.File
- headerPath := headerName(path)
-
- if indexSizeBits != 0 && (indexSizeBits > 31 || indexSizeBits < 8) {
- return nil, fmt.Errorf("indexSizeBits must be between 8 and 31, default is %d", defaultIndexSizeBits)
- }
- if maxFileSize > defaultMaxFileSize {
- return nil, fmt.Errorf("maximum file size cannot exceed %d", defaultMaxFileSize)
- }
-
- upgradeFileSize := maxFileSize
- if upgradeFileSize == 0 {
- upgradeFileSize = defaultMaxFileSize
- }
- err := upgradeIndex(ctx, path, headerPath, upgradeFileSize)
- if err != nil {
- return nil, fmt.Errorf("could not upgrade index: %w", err)
- }
-
- var existingHeader bool
- header, err := readHeader(headerPath)
- if err != nil {
- if !os.IsNotExist(err) {
- return nil, err
- }
- if indexSizeBits == 0 {
- indexSizeBits = defaultIndexSizeBits
- }
- if maxFileSize == 0 {
- maxFileSize = defaultMaxFileSize
- }
- header = newHeader(indexSizeBits, maxFileSize)
- mp, ok := primary.(*gsfaprimary.GsfaPrimary)
- if ok {
- header.PrimaryFileSize = mp.FileSize()
- }
- if err = writeHeader(headerPath, header); err != nil {
- return nil, err
- }
- } else {
- existingHeader = true
- if indexSizeBits == 0 {
- indexSizeBits = header.BucketsBits
- }
- if maxFileSize == 0 {
- maxFileSize = header.MaxFileSize
- }
- }
-
- buckets, err := NewBuckets(indexSizeBits)
- if err != nil {
- return nil, err
- }
-
- var rmPool bucketPool
- var lastIndexNum uint32
- if existingHeader {
- if header.BucketsBits != indexSizeBits {
- return nil, types.ErrIndexWrongBitSize{header.BucketsBits, indexSizeBits}
- }
-
- if header.MaxFileSize != maxFileSize {
- return nil, types.ErrIndexWrongFileSize{header.MaxFileSize, maxFileSize}
- }
-
- err = loadBucketState(ctx, path, buckets, maxFileSize)
- if err != nil {
- log.Warnw("Could not load bucket state, scanning index file", "err", err)
- lastIndexNum, err = scanIndex(ctx, path, header.FirstFile, buckets, maxFileSize)
- if err != nil {
- return nil, err
- }
- } else {
- lastIndexNum, err = findLastIndex(path, header.FirstFile)
- if err != nil {
- return nil, fmt.Errorf("could not find most recent index file: %w", err)
- }
- }
-
- mp, ok := primary.(*gsfaprimary.GsfaPrimary)
- if ok {
- switch header.PrimaryFileSize {
- case 0:
- // Primary file size is not yet known, so may need to remap index.
- rmPool, err = remapIndex(ctx, mp, buckets, path, headerPath, header)
- if err != nil {
- return nil, err
- }
- case mp.FileSize():
- default:
- return nil, types.ErrPrimaryWrongFileSize{mp.FileSize(), header.PrimaryFileSize}
- }
- }
- }
-
- if ctx.Err() != nil {
- return nil, ctx.Err()
- }
-
- file, err = openFileAppend(indexFileName(path, lastIndexNum))
- if err != nil {
- return nil, err
- }
-
- fi, err := file.Stat()
- if err != nil {
- return nil, err
- }
-
- idx := &Index{
- sizeBits: indexSizeBits,
- maxFileSize: maxFileSize,
- buckets: buckets,
- file: file,
- fileNum: lastIndexNum,
- headerPath: headerPath,
- writer: bufio.NewWriterSize(file, indexBufferSize),
- Primary: primary,
- nextPool: make(bucketPool, bucketPoolSize),
- length: types.Position(fi.Size()),
- basePath: path,
- fileCache: fileCache,
- }
-
- if len(rmPool) != 0 {
- idx.nextPool = rmPool
- idx.Flush()
- idx.curPool = nil
- }
-
- if gcInterval == 0 {
- log.Warn("Index garbage collection disabled")
- } else {
- idx.gcDone = make(chan struct{})
- idx.gcStop = make(chan struct{})
- go idx.garbageCollector(gcInterval, gcTimeLimit)
- }
-
- return idx, nil
-}
-
-func indexFileName(basePath string, fileNum uint32) string {
- return fmt.Sprintf("%s.%d", basePath, fileNum)
-}
-
-func savedBucketsName(basePath string) string {
- return basePath + ".buckets"
-}
-
-func scanIndex(ctx context.Context, basePath string, fileNum uint32, buckets Buckets, maxFileSize uint32) (uint32, error) {
- var lastFileNum uint32
- for {
- if ctx.Err() != nil {
- return 0, ctx.Err()
- }
- err := scanIndexFile(ctx, basePath, fileNum, buckets, maxFileSize)
- if err != nil {
- if os.IsNotExist(err) {
- break
- }
- return 0, fmt.Errorf("error scanning index file %s: %w", indexFileName(basePath, fileNum), err)
- }
- lastFileNum = fileNum
- fileNum++
- }
- return lastFileNum, nil
-}
-
-// StorageSize returns bytes of storage used by the index files.
-func (idx *Index) StorageSize() (int64, error) {
- header, err := readHeader(idx.headerPath)
- if err != nil {
- if os.IsNotExist(err) {
- return 0, nil
- }
- return 0, err
- }
- fi, err := os.Stat(idx.headerPath)
- if err != nil {
- return 0, err
- }
- size := fi.Size()
-
- fileNum := header.FirstFile
- for {
- fi, err = os.Stat(indexFileName(idx.basePath, fileNum))
- if err != nil {
- if os.IsNotExist(err) {
- break
- }
- return 0, err
- }
- size += fi.Size()
- fileNum++
- }
- return size, nil
-}
-
-func scanIndexFile(ctx context.Context, basePath string, fileNum uint32, buckets Buckets, maxFileSize uint32) error {
- indexPath := indexFileName(basePath, fileNum)
-
- // This is a single sequential read across the index file.
- file, err := openFileForScan(indexPath)
- if err != nil {
- return err
- }
- defer file.Close()
-
- fi, err := file.Stat()
- if err != nil {
- return fmt.Errorf("cannot stat index file: %w", err)
- }
- if fi.Size() == 0 {
- return nil
- }
-
- sizeBuffer := make([]byte, sizePrefixSize)
- scratch := make([]byte, 256)
- var pos int64
- var i int
- for {
- if _, err = file.ReadAt(sizeBuffer, pos); err != nil {
- if errors.Is(err, io.EOF) {
- // Finished reading entire index.
- break
- }
- if err == io.ErrUnexpectedEOF {
- log.Errorw("Unexpected EOF scanning index", "file", indexPath)
- file.Close()
- // Cut off incomplete data
- e := os.Truncate(indexPath, pos)
- if e != nil {
- log.Errorw("Error truncating file", "err", e, "file", indexPath)
- }
- break
- }
- return err
- }
- pos += sizePrefixSize
-
- size := binary.LittleEndian.Uint32(sizeBuffer)
- if size&deletedBit != 0 {
- // Record is deleted, so skip.
- pos += int64(size ^ deletedBit)
- continue
- }
-
- if int(size) > len(scratch) {
- scratch = make([]byte, size)
- }
- data := scratch[:size]
- if _, err = file.ReadAt(data, pos); err != nil {
- if err == io.ErrUnexpectedEOF || errors.Is(err, io.EOF) {
- // The file is corrupt since the expected data could not be
- // read. Take the usable data and move on.
- log.Errorw("Unexpected EOF scanning index record", "file", indexPath)
- file.Close()
- // Cut off incomplete data
- e := os.Truncate(indexPath, pos-sizePrefixSize)
- if e != nil {
- log.Errorw("Error truncating file", "err", e, "file", indexPath)
- }
- break
- }
- return err
- }
-
- i++
- if i&1023 == 0 && ctx.Err() != nil {
- return ctx.Err()
- }
-
- bucketPrefix := BucketIndex(binary.LittleEndian.Uint32(data))
- err = buckets.Put(bucketPrefix, localPosToBucketPos(pos, fileNum, maxFileSize))
- if err != nil {
- return err
- }
- pos += int64(size)
- }
- log.Infof("Scanned %s", indexPath)
- return nil
-}
-
-// Put puts a key together with a file offset into the index.
-//
-// The key needs to be a cryptographically secure hash that is at least 4 bytes
-// long.
-func (idx *Index) Put(key []byte, location types.Block) error {
- // Get record list and bucket index
- bucket, err := idx.getBucketIndex(key)
- if err != nil {
- return err
- }
-
- // The key does not need the prefix that was used to find the right
- // bucket. For simplicity only full bytes are trimmed off.
- indexKey := stripBucketPrefix(key, idx.sizeBits)
-
- idx.bucketLk.Lock()
- defer idx.bucketLk.Unlock()
-
- records, err := idx.getRecordsFromBucket(bucket)
- if err != nil {
- return err
- }
-
- // No records stored in that bucket yet
- var newData []byte
- if records == nil {
- // As it's the first key a single byte is enough as it does not need to
- // be distinguished from other keys.
- trimmedIndexKey := indexKey[:1]
- newData = EncodeKeyPosition(KeyPositionPair{trimmedIndexKey, location})
- } else {
- // Read the record list from disk and insert the new key
- pos, prevRecord, has := records.FindKeyPosition(indexKey)
-
- if has && bytes.HasPrefix(indexKey, prevRecord.Key) {
- // The previous key is fully contained in the current key. We need to read the full
- // key from the main data file in order to retrieve a key that is distinguishable
- // from the one that should get inserted.
- fullPrevKey, err := idx.Primary.GetIndexKey(prevRecord.Block)
- if err != nil {
- return fmt.Errorf("error reading previous key from primary: %w", err)
- }
- // The index key has already removed the prefix that is used to determine the
- // bucket. Do the same for the full previous key.
- prevKey := stripBucketPrefix(fullPrevKey, idx.sizeBits)
- if prevKey == nil {
- // The previous key, read from the primary, was bad. This means
- // that the data in the primary at prevRecord.Bucket is not
- // good, or that data in the index is bad and prevRecord.Bucket
- // has a wrong location in the primary. Log the error with
- // diagnostic information.
- cached, indexOffset, fileNum, err := idx.readBucketInfo(bucket)
- if err != nil {
- log.Errorw("Cannot read bucket", "err", err)
- } else {
- msg := "Read bad pevious key data, too short"
- if cached == nil {
- log.Errorw(msg, "offset", indexOffset, "size", indexFileName(idx.basePath, fileNum))
- } else {
- log.Error(msg)
- }
- }
- // Either way, the previous key record is not usable, so
- // overwrite it with a record for the new key. Use the same
- // key in the index record as the previous record, since the
- // previous key is being replaced so there is no need to
- // differentiate old from new.
- //
- // This results in the data for the previous keys being lost,
- // but it may not have been present in the first place, in which
- // case that was the cause of this problem.
- newData = records.PutKeys([]KeyPositionPair{{prevRecord.Key, location}}, prevRecord.Pos, pos)
- idx.outstandingWork += types.Work(len(newData) + BucketPrefixSize + sizePrefixSize)
- idx.nextPool[bucket] = newData
- return nil
- }
-
- keyTrimPos := firstNonCommonByte(indexKey, prevKey)
- // Only store the new key if it doesn't exist yet.
- if keyTrimPos >= len(indexKey) {
- return nil
- }
-
- trimmedPrevKey := prevKey
- if keyTrimPos < len(prevKey) {
- trimmedPrevKey = prevKey[:keyTrimPos+1]
- } else {
- // trimmedPrevKey should always be a prefix. since it is not
- // here, collect some diagnostic logs.
- cached, indexOffset, fileNum, err := idx.readBucketInfo(bucket)
- if err != nil {
- log.Errorw("Cannot read bucket", "err", err)
- } else {
- msg := "Read bad pevious key data"
- if cached == nil {
- log.Errorw(msg, "offset", indexOffset, "size", indexFileName(idx.basePath, fileNum))
- } else {
- log.Error(msg)
- }
- }
- }
- trimmedIndexKey := indexKey[:keyTrimPos+1]
- var keys []KeyPositionPair
-
- // Replace the existing previous key (which is too short) with a
- // new one and also insert the new key.
- if bytes.Compare(trimmedPrevKey, trimmedIndexKey) == -1 {
- keys = []KeyPositionPair{
- {trimmedPrevKey, prevRecord.Block},
- {trimmedIndexKey, location},
- }
- } else {
- keys = []KeyPositionPair{
- {trimmedIndexKey, location},
- {trimmedPrevKey, prevRecord.Block},
- }
- }
- newData = records.PutKeys(keys, prevRecord.Pos, pos)
- // There is no need to do anything with the next key as the next key is
- // already guaranteed to be distinguishable from the new key as it was already
- // distinguishable from the previous key.
- } else {
- // The previous key is not fully contained in the key that should get inserted.
- // Hence we only need to trim the new key to the smallest one possible that is
- // still distinguishable from the previous (in case there is one) and next key
- // (in case there is one).
- prevRecordNonCommonBytePos := 0
- if has {
- prevRecordNonCommonBytePos = firstNonCommonByte(indexKey, prevRecord.Key)
- }
- // The new record will not be the last record.
- nextRecordNonCommonBytePos := 0
- if pos < records.Len() {
- // In order to determine the minimal key size, we need to get
- // the next key as well.
- nextRecord := records.ReadRecord(pos)
- nextRecordNonCommonBytePos = firstNonCommonByte(indexKey, nextRecord.Key)
- }
-
- // Minimum prefix of the key that is different in at least one byte
- // from the previous as well as the next key.
- minPrefix := max(
- prevRecordNonCommonBytePos,
- nextRecordNonCommonBytePos,
- )
-
- // We cannot trim beyond the key length.
- keyTrimPos := min(minPrefix, len(indexKey)-1)
-
- trimmedIndexKey := indexKey[:keyTrimPos+1]
- newData = records.PutKeys([]KeyPositionPair{{trimmedIndexKey, location}}, pos, pos)
- }
- }
- idx.outstandingWork += types.Work(len(newData) + BucketPrefixSize + sizePrefixSize)
- idx.nextPool[bucket] = newData
- return nil
-}
-
-// Update updates a key together with a file offset into the index.
-func (idx *Index) Update(key []byte, location types.Block) error {
- // Get record list and bucket index
- bucket, err := idx.getBucketIndex(key)
- if err != nil {
- return err
- }
-
- // The key does not need the prefix that was used to find its bucket. For
- // simplicity only full bytes are trimmed off.
- indexKey := stripBucketPrefix(key, idx.sizeBits)
-
- idx.bucketLk.Lock()
- defer idx.bucketLk.Unlock()
- records, err := idx.getRecordsFromBucket(bucket)
- if err != nil {
- return err
- }
-
- var newData []byte
- // If no records are stored in that bucket yet, it means there is no key to
- // be updated.
- if records == nil {
- return fmt.Errorf("no records found in index, unable to update key")
- }
-
- // Read the record list to find the key and position.
- r := records.GetRecord(indexKey)
- if r == nil {
- return fmt.Errorf("key to update not found in index")
- }
- // Update key in position.
- newData = records.PutKeys([]KeyPositionPair{{r.Key, location}}, r.Pos, r.NextPos())
-
- idx.outstandingWork += types.Work(len(newData) + BucketPrefixSize + sizePrefixSize)
- idx.nextPool[bucket] = newData
- return nil
-}
-
-// Remove removes a key from the index.
-func (idx *Index) Remove(key []byte) (bool, error) {
- // Get record list and bucket index
- bucket, err := idx.getBucketIndex(key)
- if err != nil {
- return false, err
- }
-
- // The key does not need the prefix that was used to find its bucket. For
- // simplicity only full bytes are trimmed off.
- indexKey := stripBucketPrefix(key, idx.sizeBits)
-
- idx.bucketLk.Lock()
- defer idx.bucketLk.Unlock()
-
- records, err := idx.getRecordsFromBucket(bucket)
- if err != nil {
- return false, err
- }
-
- // If no records are stored in that bucket yet, it means there is no key to
- // be removed.
- if records == nil {
- // No records in index. Nothing to remove.
- return false, nil
- }
-
- // Read the record list to find the key and its position.
- r := records.GetRecord(indexKey)
- if r == nil {
- // The record does not exist. Nothing to remove.
- return false, nil
- }
-
- // Remove key from record.
- newData := records.PutKeys([]KeyPositionPair{}, r.Pos, r.NextPos())
- // NOTE: We are removing the key without changing any keys. If we want
- // to optimize for storage we need to check the keys with the same prefix
- // and see if any of them can be shortened. This process will be similar
- // to finding where to put a new key.
-
- idx.outstandingWork += types.Work(len(newData) + BucketPrefixSize + sizePrefixSize)
- idx.nextPool[bucket] = newData
- return true, nil
-}
-
-func (idx *Index) getBucketIndex(key []byte) (BucketIndex, error) {
- if len(key) < 4 {
- return 0, types.ErrKeyTooShort
- }
-
- // Determine which bucket a key falls into. Use the first few bytes of they
- // key for it and interpret them as a little-endian integer.
- prefix := BucketIndex(binary.LittleEndian.Uint32(key))
- var leadingBits BucketIndex = (1 << idx.sizeBits) - 1
- return prefix & leadingBits, nil
-}
-
-// getRecordsFromBucket returns the recordList and bucket the key belongs to.
-func (idx *Index) getRecordsFromBucket(bucket BucketIndex) (RecordList, error) {
- // Get the index file offset of the record list the key is in.
- cached, indexOffset, fileNum, err := idx.readBucketInfo(bucket)
- if err != nil {
- return nil, fmt.Errorf("error reading bucket info: %w", err)
- }
- var records RecordList
- if cached != nil {
- records = NewRecordListRaw(cached)
- } else {
- records, err = idx.readDiskBucket(indexOffset, fileNum)
- if err != nil {
- return nil, fmt.Errorf("error reading index records from disk: %w", err)
- }
- }
- return records, nil
-}
-
-func (idx *Index) flushBucket(bucket BucketIndex, newData []byte) (types.Block, types.Work, error) {
- if idx.length >= types.Position(idx.maxFileSize) {
- fileNum := idx.fileNum + 1
- indexPath := indexFileName(idx.basePath, fileNum)
- // If the index file being opened already exists then fileNum has
- // wrapped and there are max uint32 of index files. This means that
- // maxFileSize is set far too small or GC is disabled.
- if _, err := os.Stat(indexPath); !os.IsNotExist(err) {
- log.Warnw("Creating index file overwrites existing. Check that file size limit is not too small resulting in too many files.",
- "maxFileSize", idx.maxFileSize, "indexPath", indexPath)
- }
- file, err := openFileAppend(indexPath)
- if err != nil {
- return types.Block{}, 0, fmt.Errorf("cannot open new index file %s: %w", indexPath, err)
- }
- if err = idx.writer.Flush(); err != nil {
- return types.Block{}, 0, fmt.Errorf("cannot write to index file %s: %w", idx.file.Name(), err)
- }
- idx.file.Close()
- idx.writer.Reset(file)
- idx.file = file
- idx.fileNum = fileNum
- idx.length = 0
- }
-
- // Write new data to disk. The record list is prefixed with the bucket they
- // are in. This is needed in order to reconstruct the in-memory buckets
- // from the index itself.
- //
- // If the size of new data is too large to fit in 30 bits, then a bigger
- // bit prefix, or s 2nd level index is needed.
- //
- // TODO: If size >= 1<<30 then the size takes up the following 62 bits.
- newDataSize := make([]byte, sizePrefixSize)
- binary.LittleEndian.PutUint32(newDataSize, uint32(len(newData))+uint32(BucketPrefixSize))
- _, err := idx.writer.Write(newDataSize)
- if err != nil {
- return types.Block{}, 0, err
- }
-
- bucketPrefixBuffer := make([]byte, BucketPrefixSize)
- binary.LittleEndian.PutUint32(bucketPrefixBuffer, uint32(bucket))
- if _, err = idx.writer.Write(bucketPrefixBuffer); err != nil {
- return types.Block{}, 0, err
- }
-
- if _, err = idx.writer.Write(newData); err != nil {
- return types.Block{}, 0, err
- }
- length := idx.length
- toWrite := types.Position(len(newData) + BucketPrefixSize + sizePrefixSize)
- idx.length += toWrite
- // Fsyncs are expensive, so do not do them here; do in explicit Sync().
-
- // Keep the reference to the stored data in the bucket.
- return types.Block{
- Offset: localPosToBucketPos(int64(length+sizePrefixSize), idx.fileNum, idx.maxFileSize),
- Size: types.Size(len(newData) + BucketPrefixSize),
- }, types.Work(toWrite), nil
-}
-
-type bucketBlock struct {
- bucket BucketIndex
- blk types.Block
-}
-
-func (idx *Index) readCached(bucket BucketIndex) ([]byte, bool) {
- data, ok := idx.nextPool[bucket]
- if ok {
- return data, true
- }
- data, ok = idx.curPool[bucket]
- if ok {
- return data, true
- }
- return nil, false
-}
-
-func (idx *Index) readBucketInfo(bucket BucketIndex) ([]byte, types.Position, uint32, error) {
- data, ok := idx.readCached(bucket)
- if ok {
- return data, 0, 0, nil
- }
- bucketPos, err := idx.buckets.Get(bucket)
- if err != nil {
- return nil, 0, 0, fmt.Errorf("error reading bucket: %w", err)
- }
- localPos, fileNum := localizeBucketPos(bucketPos, idx.maxFileSize)
- return nil, localPos, fileNum, nil
-}
-
-func (idx *Index) readDiskBucket(indexOffset types.Position, fileNum uint32) (RecordList, error) {
- // indexOffset should never be 0 if there is a bucket, because it is always
- // at lease sizePrefixSize into the stored data.
- if indexOffset == 0 {
- return nil, nil
- }
-
- file, err := idx.fileCache.Open(indexFileName(idx.basePath, fileNum))
- if err != nil {
- return nil, err
- }
- defer idx.fileCache.Close(file)
-
- // Read the record list from disk and get the file offset of that key in
- // the primary storage.
- sizeBuf := make([]byte, sizePrefixSize)
- if _, err = file.ReadAt(sizeBuf, int64(indexOffset-4)); err != nil {
- return nil, err
- }
- data := make([]byte, binary.LittleEndian.Uint32(sizeBuf))
- if _, err = file.ReadAt(data, int64(indexOffset)); err != nil {
- return nil, err
- }
- return NewRecordList(data), nil
-}
-
-// Get the file offset in the primary storage of a key.
-func (idx *Index) Get(key []byte) (types.Block, bool, error) {
- // Get record list and bucket index.
- bucket, err := idx.getBucketIndex(key)
- if err != nil {
- return types.Block{}, false, err
- }
-
- // Here we just need an RLock since there will not be changes over buckets.
- // So, do not use getRecordsFromBucket and instead only wrap this line of
- // code in the RLock.
- idx.bucketLk.RLock()
- cached, indexOffset, fileNum, err := idx.readBucketInfo(bucket)
- idx.bucketLk.RUnlock()
- if err != nil {
- return types.Block{}, false, fmt.Errorf("error reading bucket: %w", err)
- }
- var records RecordList
- if cached != nil {
- records = NewRecordListRaw(cached)
- } else {
- records, err = idx.readDiskBucket(indexOffset, fileNum)
- if err != nil {
- return types.Block{}, false, fmt.Errorf("error reading index records from disk: %w", err)
- }
- }
- if records == nil {
- return types.Block{}, false, nil
- }
-
- // The key does not need the prefix that was used to find its bucket. For
- // simplicity only full bytes are trimmed off.
- indexKey := stripBucketPrefix(key, idx.sizeBits)
-
- fileOffset, found := records.Get(indexKey)
- return fileOffset, found, nil
-}
-
-// Flush writes outstanding work and buffered data to the current index file
-// and updates buckets.
-func (idx *Index) Flush() (types.Work, error) {
- // Only one Flush at a time, otherwise the 2nd Flush can swap the pools
- // while the 1st Flush is still reading the pool being flushed. That could
- // cause the pool being read by the 1st Flush to be written to
- // concurrently.
- idx.flushLock.Lock()
- defer idx.flushLock.Unlock()
-
- idx.bucketLk.Lock()
- // If no new data, then nothing to do.
- if len(idx.nextPool) == 0 {
- idx.bucketLk.Unlock()
- return 0, nil
- }
- idx.curPool = idx.nextPool
- idx.nextPool = make(bucketPool, bucketPoolSize)
- idx.outstandingWork = 0
- idx.bucketLk.Unlock()
-
- blks := make([]bucketBlock, 0, len(idx.curPool))
- var work types.Work
- for bucket, data := range idx.curPool {
- blk, newWork, err := idx.flushBucket(bucket, data)
- if err != nil {
- return 0, err
- }
- blks = append(blks, bucketBlock{bucket, blk})
- work += newWork
- }
- err := idx.writer.Flush()
- if err != nil {
- return 0, fmt.Errorf("cannot flush data to index file %s: %w", idx.file.Name(), err)
- }
- idx.bucketLk.Lock()
- defer idx.bucketLk.Unlock()
- for _, blk := range blks {
- if err = idx.buckets.Put(blk.bucket, blk.blk.Offset); err != nil {
- return 0, fmt.Errorf("error commiting bucket: %w", err)
- }
- }
-
- return work, nil
-}
-
-// Sync commits the contents of the current index file to disk. Flush should be
-// called before calling Sync.
-func (idx *Index) Sync() error {
- idx.flushLock.Lock()
- defer idx.flushLock.Unlock()
- return idx.file.Sync()
-}
-
-// Close calls Flush to write work and data to the current index file, and then
-// closes the file.
-func (idx *Index) Close() error {
- var err error
- idx.closeOnce.Do(func() {
- idx.fileCache.Clear()
- if idx.gcStop != nil {
- close(idx.gcStop)
- <-idx.gcDone
- idx.gcStop = nil
- }
- _, err = idx.Flush()
- if err != nil {
- idx.file.Close()
- return
- }
- if err = idx.file.Close(); err != nil {
- return
- }
- err = idx.saveBucketState()
- })
- return err
-}
-
-func (idx *Index) saveBucketState() error {
- bucketsFileName := savedBucketsName(idx.basePath)
- bucketsFileNameTemp := bucketsFileName + ".tmp"
-
- file, err := os.Create(bucketsFileNameTemp)
- if err != nil {
- return err
- }
- writer := bufio.NewWriterSize(file, indexBufferSize)
- buf := make([]byte, types.OffBytesLen)
-
- for _, offset := range idx.buckets {
- binary.LittleEndian.PutUint64(buf, uint64(offset))
-
- _, err = writer.Write(buf)
- if err != nil {
- return err
- }
- }
- if err = writer.Flush(); err != nil {
- return err
- }
- if err = file.Close(); err != nil {
- return err
- }
-
- // Only create the file after saving all buckets.
- return os.Rename(bucketsFileNameTemp, bucketsFileName)
-}
-
-func loadBucketState(ctx context.Context, basePath string, buckets Buckets, maxFileSize uint32) error {
- if ctx.Err() != nil {
- return ctx.Err()
- }
- bucketsFileName := savedBucketsName(basePath)
- file, err := os.Open(bucketsFileName)
- if err != nil {
- return err
- }
- defer func() {
- e := file.Close()
- if e != nil {
- log.Error("Error closing saved buckets file", "err", err)
- }
- if e = os.Remove(bucketsFileName); e != nil {
- log.Error("Error removing saved buckets file", "err", err)
- }
- }()
-
- fi, err := file.Stat()
- if err != nil {
- return err
- }
-
- // If the file is not the expected size then do not use it.
- if fi.Size() != int64(types.OffBytesLen*len(buckets)) {
- return fmt.Errorf("bucket state file is wrong size, expected %d actual %d", types.OffBytesLen*len(buckets), fi.Size())
- }
-
- reader := bufio.NewReaderSize(file, indexBufferSize)
- buf := make([]byte, types.OffBytesLen)
-
- for i := 0; i < len(buckets); i++ {
- // Read offset from bucket state.
- _, err = io.ReadFull(reader, buf)
- if err != nil {
- return err
- }
- buckets[i] = types.Position(binary.LittleEndian.Uint64(buf))
- }
-
- return nil
-}
-
-func RemoveSavedBuckets(basePath string) error {
- err := os.Remove(savedBucketsName(basePath))
- if err != nil && !os.IsNotExist(err) {
- return err
- }
- return nil
-}
-
-func (i *Index) OutstandingWork() types.Work {
- i.bucketLk.RLock()
- defer i.bucketLk.RUnlock()
- return i.outstandingWork
-}
-
-// RawIterator iterates raw index file entries, whether or not they are still
-// valid. Deleted entries are skipped.
-//
-// This is primarily to inspect index files for testing.
-type RawIterator struct {
- // The index data we are iterating over
- file *os.File
- // The current position within the index
- pos int64
- // The base index file path
- base string
- // The current index file number
- fileNum uint32
-}
-
-func NewRawIterator(basePath string, fileNum uint32) *RawIterator {
- return &RawIterator{
- base: basePath,
- fileNum: fileNum,
- }
-}
-
-func (iter *RawIterator) Next() ([]byte, types.Position, bool, error) {
- if iter.file == nil {
- file, err := openFileForScan(indexFileName(iter.base, iter.fileNum))
- if err != nil {
- if os.IsNotExist(err) {
- return nil, 0, true, nil
- }
- return nil, 0, false, err
- }
- iter.file = file
- iter.pos = 0
- }
-
- var size uint32
- sizeBuf := make([]byte, sizePrefixSize)
- for {
- _, err := iter.file.ReadAt(sizeBuf, iter.pos)
- if err != nil {
- iter.file.Close()
- if errors.Is(err, io.EOF) {
- iter.file = nil
- iter.fileNum++
- return iter.Next()
- }
- return nil, 0, false, err
- }
- size = binary.LittleEndian.Uint32(sizeBuf)
- if size&deletedBit != 0 {
- size ^= deletedBit
- iter.pos += int64(sizePrefixSize + size)
- } else {
- break
- }
- }
- pos := iter.pos + int64(sizePrefixSize)
- data := make([]byte, size)
- _, err := iter.file.ReadAt(data, pos)
- if err != nil {
- iter.file.Close()
- return nil, 0, false, err
- }
-
- iter.pos += int64(sizePrefixSize + size)
- return data, types.Position(pos), false, nil
-}
-
-func (iter *RawIterator) Close() error {
- if iter.file == nil {
- return nil
- }
- return iter.file.Close()
-}
-
-// Iterator is an iterator over only valid index entries.
-//
-// On each iteration it returns the position of the record within the index
-// together with the raw record list data.
-type Iterator struct {
- // bucketIndex is the next bucket to iterate.
- bucketIndex BucketIndex
- // index is the Index being iterated.
- index *Index
- rlIter *RecordListIter
-}
-
-func (idx *Index) NewIterator() *Iterator {
- return &Iterator{
- index: idx,
- }
-}
-
-// Progress returns the percentage of buckets iterated.
-func (iter *Iterator) Progress() float64 {
- return 100.0 * float64(iter.bucketIndex) / float64(len(iter.index.buckets))
-}
-
-func (iter *Iterator) Next() (Record, bool, error) {
- if iter.rlIter != nil {
- if !iter.rlIter.Done() {
- return iter.rlIter.Next(), false, nil
- }
- iter.rlIter = nil
- }
-
- iter.index.flushLock.Lock()
- defer iter.index.flushLock.Unlock()
-
-next:
- iter.index.bucketLk.RLock()
- var bucketPos types.Position
- for {
- if int(iter.bucketIndex) >= len(iter.index.buckets) {
- iter.index.bucketLk.RUnlock()
- return Record{}, true, nil
- }
- bucketPos = iter.index.buckets[iter.bucketIndex]
- if bucketPos != 0 {
- // Got non-empty bucket.
- break
- }
- iter.bucketIndex++
- }
- iter.index.bucketLk.RUnlock()
-
- data, cached := iter.index.readCached(iter.bucketIndex)
- if cached {
- // Add the size prefix to the record data.
- newData := make([]byte, len(data)+sizePrefixSize)
- binary.LittleEndian.PutUint32(newData, uint32(len(data)))
- copy(newData[sizePrefixSize:], data)
- data = newData
- } else {
- localPos, fileNum := localizeBucketPos(bucketPos, iter.index.maxFileSize)
- file, err := iter.index.fileCache.Open(indexFileName(iter.index.basePath, fileNum))
- if err != nil {
- return Record{}, false, err
- }
- defer iter.index.fileCache.Close(file)
-
- sizeBuf := make([]byte, sizePrefixSize)
- if _, err = file.ReadAt(sizeBuf, int64(localPos-sizePrefixSize)); err != nil {
- return Record{}, false, err
- }
- data = make([]byte, binary.LittleEndian.Uint32(sizeBuf))
- if _, err = file.ReadAt(data, int64(localPos)); err != nil {
- return Record{}, false, err
- }
- }
- iter.bucketIndex++
-
- rl := NewRecordList(data)
- iter.rlIter = rl.Iter()
- if iter.rlIter.Done() {
- iter.rlIter = nil
- goto next
- }
-
- return iter.rlIter.Next(), false, nil
-}
-
-func max(a, b int) int {
- if a > b {
- return a
- }
- return b
-}
-
-func min(a, b int) int {
- if a < b {
- return a
- }
- return b
-}
-
-// firstNonCommonByte returns the position of the first character that both
-// given slices have not in common.
-//
-// It might return an index that is bigger than the input strings. If one is
-// full prefix of the other, the index will be `shorterSlice.len() + 1`, if
-// both slices are equal it will be `slice.len() + 1`
-func firstNonCommonByte(aa []byte, bb []byte) int {
- smallerLength := min(len(aa), len(bb))
- index := 0
- for ; index < smallerLength; index++ {
- if aa[index] != bb[index] {
- break
- }
- }
- return index
-}
-
-func openFileAppend(name string) (*os.File, error) {
- return os.OpenFile(name, os.O_WRONLY|os.O_APPEND|os.O_CREATE, 0o644)
-}
-
-func openFileForScan(name string) (*os.File, error) {
- return os.OpenFile(name, os.O_RDONLY, 0o644)
-}
-
-func bucketPosToFileNum(pos types.Position, maxFileSize uint32) (bool, uint32) {
- // Bucket pos 0 means there is no data in the bucket, so indicate empty bucket.
- if pos == 0 {
- return false, 0
- }
- // The start of the entry, not the position of the record, determines which
- // is file is used. The record begins sizePrefixSize before pos. This
- // matters only if pos is slightly after a maxFileSize boundry, but
- // the adjusted position is not.
- return true, uint32((pos - sizePrefixSize) / types.Position(maxFileSize))
-}
-
-func localPosToBucketPos(pos int64, fileNum, maxFileSize uint32) types.Position {
- // Valid position must be non-zero, at least sizePrefixSize.
- if pos == 0 {
- panic("invalid local offset")
- }
- // fileNum is a 32bit value and will wrap at 4GiB, So 4294967296 is the
- // maximum number of index files possible.
- return types.Position(fileNum)*types.Position(maxFileSize) + types.Position(pos)
-}
-
-// localizeBucketPos decodes a bucketPos into a local pos and file number.
-func localizeBucketPos(pos types.Position, maxFileSize uint32) (types.Position, uint32) {
- ok, fileNum := bucketPosToFileNum(pos, maxFileSize)
- if !ok {
- // Return 0 local pos to indicate empty bucket.
- return 0, 0
- }
- // Subtract file offset to get pos within its local file.
- localPos := pos - (types.Position(fileNum) * types.Position(maxFileSize))
- return localPos, fileNum
-}
-
-func findLastIndex(basePath string, fileNum uint32) (uint32, error) {
- var lastFound uint32
- for {
- _, err := os.Stat(indexFileName(basePath, fileNum))
- if err != nil {
- if os.IsNotExist(err) {
- break
- }
- return 0, err
- }
- lastFound = fileNum
- fileNum++
- }
- return lastFound, nil
-}
-
-func copyFile(src, dst string) error {
- fin, err := os.Open(src)
- if err != nil {
- return err
- }
- defer fin.Close()
-
- fout, err := os.Create(dst)
- if err != nil {
- return err
- }
- defer fout.Close()
-
- _, err = io.Copy(fout, fin)
- return err
-}
-
-// remapIndex updates all the primary offset in each record from the location
-// in the single primary file to the correct location in the separate primary
-// files. Remapping is done on a copy of each original index file so that if
-// remapping the index files is not completed, there are no files in a
-// partially remapped state. This allows remapping to resume from where it left
-// off, without corrupting any files that were already remapped.
-func remapIndex(ctx context.Context, mp *gsfaprimary.GsfaPrimary, buckets Buckets, basePath, headerPath string, header Header) (bucketPool, error) {
- remapper, err := mp.NewIndexRemapper()
- if err != nil {
- return nil, err
- }
- if remapper == nil {
- // Update the header to indicate remapping is completed.
- header.PrimaryFileSize = mp.FileSize()
- return nil, writeHeader(headerPath, header)
- }
-
- log.Infow("Remapping primary offsets in index")
-
- maxFileSize := header.MaxFileSize
- fileBuckets := make(map[uint32][]int)
-
- var indexTotal int
- for i, offset := range buckets {
- ok, fileNum := bucketPosToFileNum(offset, maxFileSize)
- if ok {
- fileBuckets[fileNum] = append(fileBuckets[fileNum], i)
- indexTotal++
- }
- }
-
- var rmPool bucketPool
- var fileCount, indexCount, recordCount int
- var scratch []byte
- sizeBuf := make([]byte, sizePrefixSize)
-
- for fileNum, bucketPrefixes := range fileBuckets {
- if ctx.Err() != nil {
- return nil, ctx.Err()
- }
- fileName := indexFileName(basePath, fileNum)
- tmpName := fileName + ".tmp"
- doneName := fileName + ".remapped"
-
- // If this file was already remapped, skip it.
- _, err = os.Stat(doneName)
- if !os.IsNotExist(err) {
- log.Infow("index file already remapped", "file", fileName)
- indexCount += len(bucketPrefixes)
- continue
- }
-
- err = copyFile(fileName, tmpName)
- if err != nil {
- return nil, err
- }
-
- file, err := os.OpenFile(tmpName, os.O_RDWR, 0o644)
- if err != nil {
- return nil, fmt.Errorf("remap cannot open index file %s: %w", fileName, err)
- }
-
- for _, pfx := range bucketPrefixes {
- // Read the record list from disk and remap the primary file offset
- // in each record in the record list.
- localPos := buckets[pfx] - (types.Position(fileNum) * types.Position(maxFileSize))
- if _, err = file.ReadAt(sizeBuf, int64(localPos-sizePrefixSize)); err != nil {
- return nil, fmt.Errorf("cannot read record list size from index file %s: %w", file.Name(), err)
- }
- size := binary.LittleEndian.Uint32(sizeBuf)
- if len(scratch) < int(size) {
- scratch = make([]byte, size)
- }
- data := scratch[:size]
- if _, err = file.ReadAt(data, int64(localPos)); err != nil {
- return nil, fmt.Errorf("cannot read record list from index file %s: %w", file.Name(), err)
- }
- records := NewRecordList(data)
- recIter := records.Iter()
- var delPosList []int
- for !recIter.Done() {
- record := recIter.Next()
- offset, err := remapper.RemapOffset(record.Block.Offset)
- if err != nil {
- // This offset does not exist in the primary. The primary
- // was corrupted and this offset is not present in the new
- // primary. Create new record list data, with the bad
- // record deleted, and add it to a work pool for later
- // deletion from the index.
- delPosList = append(delPosList, record.Pos, record.NextPos())
- log.Errorw("Index has unusable primary offset", "err", err)
- }
- binary.LittleEndian.PutUint64(records[record.Pos:], uint64(offset))
- recordCount++
- }
- if _, err = file.WriteAt(data, int64(localPos)); err != nil {
- return nil, fmt.Errorf("failed to remap primary offset in index file %s: %w", fileName, err)
- }
- if len(delPosList) != 0 {
- for i := len(delPosList) - 1; i >= 0; i -= 2 {
- delNext := delPosList[i]
- delPos := delPosList[i-1]
- data = records.PutKeys([]KeyPositionPair{}, delPos, delNext)
- records = NewRecordListRaw(data)
- }
- if rmPool == nil {
- rmPool = make(bucketPool)
- }
- rmPool[BucketIndex(pfx)] = data
- }
- indexCount++
- }
-
- if err = file.Close(); err != nil {
- log.Errorw("Error closing remapped index file", "err", err, "path", fileName)
- }
-
- // Create a ".remapped" file to indicate this file was remapped, and
- // rename the temp file to the original index file name.
- doneFile, err := os.Create(doneName)
- if err != nil {
- log.Errorw("Error creating remapped file", "err", err, "file", doneName)
- }
- if err = doneFile.Close(); err != nil {
- log.Errorw("Error closeing remapped file", "err", err, "file", doneName)
- }
-
- if err = os.Rename(tmpName, fileName); err != nil {
- return nil, fmt.Errorf("error renaming remapped file %s to %s: %w", tmpName, fileName, err)
- }
-
- fileCount++
- log.Infof("Remapped index file %s: %.1f%% done", filepath.Base(fileName), float64(1000*indexCount/indexTotal)/10)
- }
-
- // Update the header to indicate remapping is completed.
- header.PrimaryFileSize = mp.FileSize()
- if err = writeHeader(headerPath, header); err != nil {
- return nil, err
- }
-
- // Remove the completion marker files.
- for fileNum := range fileBuckets {
- doneName := indexFileName(basePath, fileNum) + ".remapped"
- if err = os.Remove(doneName); err != nil {
- log.Errorw("Error removing remapped marker", "file", doneName, "err", err)
- }
- }
-
- log.Infow("Remapped primary offsets", "fileCount", fileCount, "recordCount", recordCount)
- return rmPool, nil
-}
-
-func headerName(basePath string) string {
- return filepath.Clean(basePath) + ".info"
-}
-
-type fileIter struct {
- basePath string
- fileNum uint32
-}
-
-func newFileIter(basePath string) (*fileIter, error) {
- header, err := readHeader(headerName(basePath))
- if err != nil {
- return nil, err
- }
- return &fileIter{
- basePath: basePath,
- fileNum: header.FirstFile,
- }, nil
-}
-
-// next returns the name of the next index file. Returns io.EOF if there are no
-// more index files.
-func (fi *fileIter) next() (string, error) {
- _, err := os.Stat(indexFileName(fi.basePath, fi.fileNum))
- if err != nil {
- if os.IsNotExist(err) {
- err = io.EOF
- }
- return "", err
- }
- fileName := indexFileName(fi.basePath, fi.fileNum)
- fi.fileNum++
-
- return fileName, nil
-}
-
-func MoveFiles(indexPath, newDir string) error {
- err := os.MkdirAll(newDir, 0o755)
- if err != nil {
- return err
- }
-
- fileIter, err := newFileIter(indexPath)
- if err != nil {
- return err
- }
- for {
- fileName, err := fileIter.next()
- if err != nil {
- if errors.Is(err, io.EOF) {
- break
- }
- return err
- }
- newPath := filepath.Join(newDir, filepath.Base(fileName))
- if err = os.Rename(fileName, newPath); err != nil {
- return err
- }
- }
-
- headerPath := headerName(indexPath)
- newPath := filepath.Join(newDir, filepath.Base(headerPath))
- if err = os.Rename(headerPath, newPath); err != nil {
- return err
- }
-
- bucketsPath := savedBucketsName(indexPath)
- _, err = os.Stat(bucketsPath)
- if !os.IsNotExist(err) {
- newPath = filepath.Join(newDir, filepath.Base(bucketsPath))
- if err = os.Rename(bucketsPath, newPath); err != nil {
- return err
- }
- }
-
- return nil
-}
diff --git a/store/index/index_test.go b/store/index/index_test.go
deleted file mode 100644
index 341932d1..00000000
--- a/store/index/index_test.go
+++ /dev/null
@@ -1,637 +0,0 @@
-package index
-
-import (
- "context"
- "path/filepath"
- "testing"
-
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/rpcpool/yellowstone-faithful/store/primary/inmemory"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-const (
- bucketBits uint8 = 24
- fileSize uint32 = 1024 * 1024 * 1024
-
- // File cache size for testing.
- testFCSize = 64
-)
-
-func TestFirstNonCommonByte(t *testing.T) {
- require.Equal(t, firstNonCommonByte([]byte{0}, []byte{1}), 0)
- require.Equal(t, firstNonCommonByte([]byte{0}, []byte{0}), 1)
- require.Equal(t, firstNonCommonByte([]byte{0, 1, 2, 3}, []byte{0}), 1)
- require.Equal(t, firstNonCommonByte([]byte{0}, []byte{0, 1, 2, 3}), 1)
- require.Equal(t, firstNonCommonByte([]byte{0, 1, 2}, []byte{0, 1, 2, 3}), 3)
- require.Equal(t, firstNonCommonByte([]byte{0, 1, 2, 3}, []byte{0, 1, 2}), 3)
- require.Equal(t, firstNonCommonByte([]byte{3, 2, 1, 0}, []byte{0, 1, 2}), 0)
- require.Equal(t, firstNonCommonByte([]byte{0, 1, 1, 0}, []byte{0, 1, 2}), 2)
- require.Equal(t, firstNonCommonByte([]byte{180, 9, 113, 0}, []byte{180, 0, 113, 0}), 1)
-}
-
-func assertHeader(t *testing.T, headerPath string, bucketsBits uint8) {
- header, err := readHeader(headerPath)
- require.NoError(t, err)
- require.Equal(t, header.Version, IndexVersion)
- require.Equal(t, header.BucketsBits, bucketsBits)
-}
-
-// Asserts that given two keys that on the first insert the key is trimmed to a single byte and on
-// the second insert they are trimmed to the minimal distinguishable prefix
-func assertCommonPrefixTrimmed(t *testing.T, key1 []byte, key2 []byte, expectedKeyLength int) {
- primaryStorage := inmemory.New([][2][]byte{{key1, {0x20}}, {key2, {0x30}}})
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- err = i.Put(key1, types.Block{Offset: 0, Size: 1})
- require.NoError(t, err)
- _, err = i.Flush()
- require.NoError(t, err)
- err = i.Sync()
- require.NoError(t, err)
- err = i.Put(key2, types.Block{Offset: 1, Size: 1})
- require.NoError(t, err)
- _, err = i.Flush()
- require.NoError(t, err)
- err = i.Sync()
- require.NoError(t, err)
- err = i.Close()
- require.NoError(t, err)
-
- iter := NewRawIterator(i.basePath, i.fileNum)
- defer iter.Close()
-
- // The record list is append only, hence the first record list only contains the first insert
- data, _, done, err := iter.Next()
- require.NoError(t, err)
- require.False(t, done)
- recordlist := NewRecordList(data)
- recordIter := recordlist.Iter()
- var keyLengths []int
- for !recordIter.Done() {
- record := recordIter.Next()
- keyLengths = append(keyLengths, len(record.Key))
- }
- require.Equal(t, keyLengths, []int{1}, "Single key has the expected length of 1")
-
- // The second block contains both keys
- data, _, done, err = iter.Next()
- require.NoError(t, err)
- require.False(t, done)
- recordlist = NewRecordList(data)
- recordIter = recordlist.Iter()
- keyLengths = []int{}
- for !recordIter.Done() {
- record := recordIter.Next()
- keyLengths = append(keyLengths, len(record.Key))
- }
- require.Equal(t,
- keyLengths,
- []int{expectedKeyLength, expectedKeyLength},
- "All keys are trimmed to their minimal distringuishable prefix",
- )
-}
-
-// This test is about making sure that inserts into an empty bucket result in a key that is trimmed
-// to a single byte.
-
-func TestIndexPutSingleKey(t *testing.T) {
- primaryStorage := inmemory.New([][2][]byte{})
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- err = i.Put([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, types.Block{Offset: 222, Size: 10})
- require.NoError(t, err)
- _, err = i.Flush()
- require.NoError(t, err)
- err = i.Sync()
- require.NoError(t, err)
- err = i.Close()
- require.NoError(t, err)
-
- // Test double close.
- err = i.Close()
- require.NoError(t, err)
-
- // Skip header
- iter := NewRawIterator(i.basePath, i.fileNum)
- defer iter.Close()
- data, _, done, err := iter.Next()
- require.NoError(t, err)
- require.False(t, done)
- recordlist := NewRecordList(data)
- recordIter := recordlist.Iter()
- require.False(t, recordIter.Done())
- record := recordIter.Next()
- require.Equal(t,
- len(record.Key),
- 1,
- "Key is trimmed to one bytes it's the only key in the record list",
- )
-}
-
-// This test is about making sure that we remove the record for a key successfully
-
-func TestIndexRemoveKey(t *testing.T) {
- k1 := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}
- k2 := []byte{1, 2, 3, 55, 5, 6, 7, 8, 9, 10}
- b1 := types.Block{Offset: 0, Size: 1}
- b2 := types.Block{Offset: 1, Size: 2}
-
- primaryStorage := inmemory.New([][2][]byte{})
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- // Put key 1
- err = i.Put(k1, b1)
- require.NoError(t, err)
- // Put key 2
- err = i.Put(k2, b2)
- require.NoError(t, err)
-
- // Remove key
- removed, err := i.Remove(k1)
- require.NoError(t, err)
- require.True(t, removed)
-
- _, found, err := i.Get(k1)
- require.NoError(t, err)
- require.False(t, found)
-
- secondKeyBlock, found, err := i.Get(k2)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, secondKeyBlock, b2)
-
- // Removing the same key again
- removed, err = i.Remove(k1)
- require.NoError(t, err)
- require.False(t, removed)
-
- // Trying to remove a non-existing key
- removed, err = i.Remove([]byte{1, 2, 3, 78, 5, 6, 7, 8, 9, 10})
- require.NoError(t, err)
- require.False(t, removed)
-
- // Flush and check if it holds
- _, err = i.Flush()
- require.NoError(t, err)
- err = i.Sync()
- require.NoError(t, err)
-
- _, found, err = i.Get(k1)
- require.NoError(t, err)
- require.False(t, found)
-
- secondKeyBlock, found, err = i.Get(k2)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, secondKeyBlock, b2)
-
- // Removing all keys from storage
- removed, err = i.Remove(k2)
- require.NoError(t, err)
- require.True(t, removed)
-
- // Removing over empty record
- removed, err = i.Remove(k2)
- require.NoError(t, err)
- require.False(t, removed)
-
- err = i.Close()
- require.NoError(t, err)
-}
-
-// This test is about making sure that a new key that doesn't share any prefix with other keys
-// within the same bucket is trimmed to a single byte.
-func TestIndexPutDistinctKey(t *testing.T) {
- primaryStorage := inmemory.New([][2][]byte{})
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- err = i.Put([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, types.Block{Offset: 222, Size: 10})
- require.NoError(t, err)
- err = i.Put([]byte{1, 2, 3, 55, 5, 6, 7, 8, 9, 10}, types.Block{Offset: 333, Size: 10})
- require.NoError(t, err)
- _, err = i.Flush()
- require.NoError(t, err)
- err = i.Sync()
- require.NoError(t, err)
- err = i.Close()
- require.NoError(t, err)
-
- iter := NewRawIterator(i.basePath, i.fileNum)
- defer iter.Close()
-
- // The record list is append only, hence the first record list only contains the first insert
- var data []byte
- var hasData bool
- for {
- next, _, done, err := iter.Next()
- require.NoError(t, err)
- if done {
- break
- }
- data = next
- hasData = true
- }
- require.True(t, hasData)
- recordlist := NewRecordList(data)
- recordIter := recordlist.Iter()
- var keys [][]byte
- for !recordIter.Done() {
- record := recordIter.Next()
- keys = append(keys, record.Key)
- }
- require.Equal(t, keys, [][]byte{{4}, {55}}, "All keys are trimmed to a single byte")
-}
-
-func TestCorrectCacheReading(t *testing.T) {
- primaryStorage := inmemory.New([][2][]byte{})
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- // put key in, then flush the cache
- err = i.Put([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, types.Block{Offset: 222, Size: 10})
- require.NoError(t, err)
- _, err = i.Flush()
- require.NoError(t, err)
- // now put two keys in the same bucket
- err = i.Put([]byte{1, 2, 3, 55, 5, 6, 7, 8, 9, 10}, types.Block{Offset: 333, Size: 10})
- require.NoError(t, err)
- err = i.Put([]byte{1, 2, 3, 88, 5, 6, 7, 8, 9, 10}, types.Block{Offset: 500, Size: 10})
- require.NoError(t, err)
-
- block, found, err := i.Get([]byte{1, 2, 3, 55, 5, 6, 7, 8, 9, 10})
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, types.Block{Offset: 333, Size: 10}, block)
-
- err = i.Close()
- require.NoError(t, err)
-}
-
-// This test is about making sure that a key is trimmed correctly if it shares a prefix with the
-// previous key
-
-func TestIndexPutPrevKeyCommonPrefix(t *testing.T) {
- key1 := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}
- key2 := []byte{1, 2, 3, 4, 5, 6, 9, 9, 9, 9}
- assertCommonPrefixTrimmed(t, key1, key2, 4)
-}
-
-// This test is about making sure that a key is trimmed correctly if it shares a prefix with the
-// next key
-func TestIndexPutNextKeyCommonPrefix(t *testing.T) {
- key1 := []byte{1, 2, 3, 4, 5, 6, 9, 9, 9, 9}
- key2 := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}
- assertCommonPrefixTrimmed(t, key1, key2, 4)
-}
-
-// This test is about making sure that a key is trimmed correctly if it shares a prefix with the
-// previous and the next key, where the common prefix with the next key is longer.
-func TestIndexPutPrevAndNextKeyCommonPrefix(t *testing.T) {
- key1 := []byte{1, 2, 3, 4, 5, 6, 9, 9, 9, 9}
- key2 := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}
- key3 := []byte{1, 2, 3, 4, 5, 6, 9, 8, 8, 8}
-
- primaryStorage := inmemory.New([][2][]byte{
- {key1, {0x10}},
- {key2, {0x20}},
- {key3, {0x30}},
- })
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- err = i.Put(key1, types.Block{Offset: 0, Size: 1})
- require.NoError(t, err)
- err = i.Put(key2, types.Block{Offset: 1, Size: 1})
- require.NoError(t, err)
- err = i.Put(key3, types.Block{Offset: 1, Size: 1})
- require.NoError(t, err)
- _, err = i.Flush()
- require.NoError(t, err)
- err = i.Sync()
- require.NoError(t, err)
- err = i.Close()
- require.NoError(t, err)
-
- iter := NewRawIterator(i.basePath, i.fileNum)
- defer iter.Close()
-
- var data []byte
- for {
- next, _, done, err := iter.Next()
- require.NoError(t, err)
- if done {
- break
- }
- data = next
- }
- recordlist := NewRecordList(data)
- recordIter := recordlist.Iter()
- var keys [][]byte
- for !recordIter.Done() {
- record := recordIter.Next()
- keys = append(keys, record.Key)
- }
- require.Equal(t,
- keys,
- [][]byte{{4, 5, 6, 7}, {4, 5, 6, 9, 8}, {4, 5, 6, 9, 9}},
- "Keys are correctly sorted and trimmed",
- )
-}
-
-func TestIndexGetEmptyIndex(t *testing.T) {
- key := []byte{1, 2, 3, 4, 5, 6, 9, 9, 9, 9}
- primaryStorage := inmemory.New([][2][]byte{})
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- index, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- _, found, err := index.Get(key)
- require.NoError(t, err)
- require.False(t, found, "Key was not found")
- err = index.Close()
- require.NoError(t, err)
-}
-
-func TestIndexGet(t *testing.T) {
- key1 := []byte{1, 2, 3, 4, 5, 6, 9, 9, 9, 9}
- key2 := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}
- key3 := []byte{1, 2, 3, 4, 5, 6, 9, 8, 8, 8}
-
- primaryStorage := inmemory.New([][2][]byte{
- {key1, {0x10}},
- {key2, {0x20}},
- {key3, {0x30}},
- })
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- err = i.Put(key1, types.Block{Offset: 0, Size: 1})
- require.NoError(t, err)
- err = i.Put(key2, types.Block{Offset: 1, Size: 1})
- require.NoError(t, err)
- err = i.Put(key3, types.Block{Offset: 2, Size: 1})
- require.NoError(t, err)
-
- firstKeyBlock, found, err := i.Get(key1)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, firstKeyBlock, types.Block{Offset: 0, Size: 1})
-
- secondKeyBlock, found, err := i.Get(key2)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, secondKeyBlock, types.Block{Offset: 1, Size: 1})
-
- thirdKeyBlock, found, err := i.Get(key3)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, thirdKeyBlock, types.Block{Offset: 2, Size: 1})
-
- // It still hits a bucket where there are keys, but that key doesn't exist.
- _, found, err = i.Get([]byte{1, 2, 3, 4, 5, 9})
- require.False(t, found)
- require.NoError(t, err)
-
- // A key that matches some prefixes but it shorter than the prefixes.
- _, found, err = i.Get([]byte{1, 2, 3, 4, 5})
- require.False(t, found)
- require.NoError(t, err)
-
- // same should hold true after flush
- _, err = i.Flush()
- require.NoError(t, err)
- err = i.Sync()
- require.NoError(t, err)
-
- firstKeyBlock, found, err = i.Get(key1)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, firstKeyBlock, types.Block{Offset: 0, Size: 1})
-
- secondKeyBlock, found, err = i.Get(key2)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, secondKeyBlock, types.Block{Offset: 1, Size: 1})
-
- thirdKeyBlock, found, err = i.Get(key3)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, thirdKeyBlock, types.Block{Offset: 2, Size: 1})
-
- // It still hits a bucket where there are keys, but that key doesn't exist.
- _, found, err = i.Get([]byte{1, 2, 3, 4, 5, 9})
- require.False(t, found)
- require.NoError(t, err)
-
- // A key that matches some prefixes but it shorter than the prefixes.
- _, found, err = i.Get([]byte{1, 2, 3, 4, 5})
- require.False(t, found)
- require.NoError(t, err)
-
- err = i.Close()
- require.NoError(t, err)
- i, err = Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
-
- // same should hold true when index is closed and reopened
-
- firstKeyBlock, found, err = i.Get(key1)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, firstKeyBlock, types.Block{Offset: 0, Size: 1})
-
- secondKeyBlock, found, err = i.Get(key2)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, secondKeyBlock, types.Block{Offset: 1, Size: 1})
-
- thirdKeyBlock, found, err = i.Get(key3)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, thirdKeyBlock, types.Block{Offset: 2, Size: 1})
-
- err = i.Close()
- require.NoError(t, err)
- bucketsFileName := indexPath + ".buckets"
- require.FileExists(t, bucketsFileName)
-
- // Open index reading bucket state.
- i, err = Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- t.Cleanup(func() { i.Close() })
- require.NoFileExists(t, bucketsFileName)
-
- // Open index scanning index files.
- i2, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- t.Cleanup(func() { i2.Close() })
-
- // Check that both indexes have same buckets.
- require.Equal(t, i.buckets, i2.buckets)
-}
-
-func TestIndexHeader(t *testing.T) {
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
-
- primaryStorage := inmemory.New([][2][]byte{})
- i1, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- t.Cleanup(func() { i1.Close() })
- assertHeader(t, i1.headerPath, bucketBits)
-
- // Check that the header doesn't change if the index is opened again.
- i2, err := Open(context.Background(), indexPath, inmemory.New([][2][]byte{}), bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- t.Cleanup(func() { i2.Close() })
- assertHeader(t, i2.headerPath, bucketBits)
-}
-
-func TestIndexGetBad(t *testing.T) {
- key1 := []byte{1, 2, 3, 4, 5, 6, 9, 9, 9, 9}
- key2 := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}
- key3 := []byte{1, 2, 3, 4, 5, 6, 9, 8, 8, 8}
- key4 := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11}
-
- primaryStorage := inmemory.New([][2][]byte{
- {key1, {0x10}},
- {[]byte("X"), {0x20}},
- {key3, {0x30}},
- })
-
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, 0, 0, 0, filecache.New(testFCSize))
-
- require.NoError(t, err)
- err = i.Put(key1, types.Block{Offset: 0, Size: 1})
- require.NoError(t, err)
- err = i.Put(key2, types.Block{Offset: 1, Size: 1})
- require.NoError(t, err)
- err = i.Put(key3, types.Block{Offset: 2, Size: 1})
- require.NoError(t, err)
-
- firstKeyBlock, found, err := i.Get(key1)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, firstKeyBlock, types.Block{Offset: 0, Size: 1})
-
- secondKeyBlock, found, err := i.Get(key2)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, secondKeyBlock, types.Block{Offset: 1, Size: 1})
-
- thirdKeyBlock, found, err := i.Get(key3)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, thirdKeyBlock, types.Block{Offset: 2, Size: 1})
-
- // This should result in the record for key2 being replaced.
- err = i.Put(key4, types.Block{Offset: 1, Size: 1})
- require.NoError(t, err)
-
- fourthKeyBlock, found, err := i.Get(key4)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, fourthKeyBlock, secondKeyBlock)
-
- // Index for key2 should be same as index for key4
- secondKeyBlock, found, err = i.Get(key2)
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, secondKeyBlock, fourthKeyBlock)
-
- err = i.Close()
- require.NoError(t, err)
-}
-
-func TestFlushRace(t *testing.T) {
- const goroutines = 64
- key1 := []byte{1, 2, 3, 4, 5, 6, 9, 9, 9, 9}
- key2 := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}
- key3 := []byte{1, 2, 3, 4, 5, 6, 9, 8, 8, 8}
-
- primaryStorage := inmemory.New([][2][]byte{
- {key1, {0x10}},
- {key2, {0x20}},
- {key3, {0x30}},
- })
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- err = i.Put(key1, types.Block{Offset: 0, Size: 1})
- require.NoError(t, err)
- err = i.Put(key2, types.Block{Offset: 1, Size: 1})
- require.NoError(t, err)
- err = i.Put(key3, types.Block{Offset: 2, Size: 1})
- require.NoError(t, err)
-
- start := make(chan struct{})
- errs := make(chan error)
- for n := 0; n < goroutines; n++ {
- go func() {
- <-start
- _, err := i.Flush()
- errs <- err
- }()
- }
- close(start)
- for n := 0; n < goroutines; n++ {
- err := <-errs
- require.NoError(t, err)
- }
-
- require.NoError(t, i.Close())
-}
-
-func TestFlushExcess(t *testing.T) {
- key1 := []byte{1, 2, 3, 4, 5, 6, 9, 9, 9, 9}
- key2 := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}
- key3 := []byte{1, 2, 3, 4, 5, 6, 9, 8, 8, 8}
-
- primaryStorage := inmemory.New([][2][]byte{
- {key1, {0x10}},
- {key2, {0x20}},
- {key3, {0x30}},
- })
- tempDir := t.TempDir()
- indexPath := filepath.Join(tempDir, "storethehash.index")
- i, err := Open(context.Background(), indexPath, primaryStorage, bucketBits, fileSize, 0, 0, filecache.New(testFCSize))
- require.NoError(t, err)
- err = i.Put(key1, types.Block{Offset: 0, Size: 1})
- require.NoError(t, err)
- err = i.Put(key2, types.Block{Offset: 1, Size: 1})
- require.NoError(t, err)
-
- work, err := i.Flush()
- require.NoError(t, err)
- require.NotZero(t, work)
-
- err = i.Put(key3, types.Block{Offset: 2, Size: 1})
- require.NoError(t, err)
-
- work, err = i.Flush()
- require.NoError(t, err)
- require.NotZero(t, work)
-
- // Another flush with no new data should not do work.
- work, err = i.Flush()
- require.NoError(t, err)
- require.Zero(t, work)
-
- require.NoError(t, i.Close())
-}
diff --git a/store/index/recordlist.go b/store/index/recordlist.go
deleted file mode 100644
index 66ecf9b9..00000000
--- a/store/index/recordlist.go
+++ /dev/null
@@ -1,248 +0,0 @@
-package index
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bytes"
- "encoding/binary"
- "io"
-
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-// BucketPrefixSize is how many bytes of bucket prefixes are stored.
-const BucketPrefixSize int = 4
-
-// FileOffsetBytes is the byte size of the file offset
-const FileOffsetBytes int = 8
-
-// FileSizeBytes is the byte size of the file size
-const FileSizeBytes int = 4
-
-// KeySizeBytes is key length slot, a one byte prefix
-const KeySizeBytes int = 1
-
-// KeyPositionPair contains a key, which is the unique prefix of the actual key, and the value
-// which is a file offset.
-type KeyPositionPair struct {
- Key []byte
- // The file offset, into the primary file, where the full key and its value
- // is actually stored.
- Block types.Block
-}
-
-// Record is a KeyPositionPair plus the actual position of the record in the record list
-type Record struct {
- // The current position (in bytes) of the record within the [`RecordList`]
- Pos int
- KeyPositionPair
-}
-
-// RecordList is the main object that contains several [`Record`]s. Records can be stored and retrieved.
-//
-// The underlying data is a continuous range of bytes. The format is:
-//
-// ```text
-//
-// | Once | Repeated |
-// | | |
-// | 4 bytes | Variable size | … |
-// | Bit value used to determine the bucket | Record | … |
-//
-// ```
-type RecordList []byte
-
-// NewRecordList returns an iterable RecordList from the given byte array
-func NewRecordList(data []byte) RecordList {
- return RecordList(data[BucketPrefixSize:])
-}
-
-// NewRecordList returns an iterable RecordList from the given byte array
-func NewRecordListRaw(data []byte) RecordList {
- return RecordList(data)
-}
-
-// FindKeyPosition return the position where a key would be added.
-//
-// Returns the position together with the previous record.
-func (rl RecordList) FindKeyPosition(key []byte) (pos int, prev Record, hasPrev bool) {
- rli := &RecordListIter{rl, 0}
- for !rli.Done() {
- record := rli.Next()
- // Location where the key gets inserted is found
- if bytes.Compare(record.Key, key) == 1 {
- pos = record.Pos
- return
- }
- hasPrev = true
- prev = record
- }
- pos = len(rl)
- return
-}
-
-// PutKeys puts keys at a certain position and returns the new data
-//
-// This method puts a continuous range of keys inside the data structure. The given range
-// is where it is put. *This means that you can also overwrite existing keys.*
-//
-// This is needed if you insert a new key that fully contains an existing key. The existing
-// key needs to replaced by one with a larger prefix, so that it is distinguishable from the
-// new key.
-func (rl RecordList) PutKeys(keys []KeyPositionPair, start int, end int) []byte {
- newKeys := make([]byte, 0,
- len(rl)-(end-start)+
- // Each key might have a different size, so just allocate an arbitrary size to
- // prevent more allocations. I picked 32 bytes as I don't expect hashes (hence
- // keys) to be bigger that that
- (len(keys))*(KeySizeBytes+FileOffsetBytes+FileSizeBytes+32))
- newKeys = append(newKeys, rl[:start]...)
- // Adding new keys to the beginning of the list.
- for i := range keys {
- newKeys = AddKeyPosition(newKeys, keys[i])
- }
- return append(newKeys, rl[end:]...)
-}
-
-// Get the primary storage file offset for that key.
-//
-// As the index is only storing prefixes and not the actual keys, the returned offset might
-// match, it's not guaranteed. Once the key is retieved from the primary storage it needs to
-// be checked if it actually matches.
-func (rl RecordList) Get(key []byte) (types.Block, bool) {
- // Several prefixes can match a `key`, we are only interested in the last one that
- // matches, hence keep a match around until we can be sure it's the last one.
- rli := &RecordListIter{rl, 0}
- var blk types.Block
- var matched bool
- for !rli.Done() {
- record := rli.Next()
- // The stored prefix of the key needs to match the requested key.
- if bytes.HasPrefix(key, record.Key) {
- matched = true
- blk = record.Block
- } else if bytes.Compare(record.Key, key) == 1 {
- // No keys from here on can possibly match, hence stop iterating. If we had a prefix
- // match, return that, else return none
- break
- }
- }
-
- return blk, matched
-}
-
-// GetRecord returns the full record for a key in the recordList
-func (rl RecordList) GetRecord(key []byte) *Record {
- // Several prefixes can match a `key`, we are only interested in the last one that
- // matches
- var r *Record
- rli := &RecordListIter{rl, 0}
- for !rli.Done() {
- record := rli.Next()
- // The stored prefix of the key needs to match the requested key.
- if bytes.HasPrefix(key, record.Key) {
- r = &record
- } else if bytes.Compare(record.Key, key) == 1 {
- // No keys from here on can possibly match, hence stop iterating. If we had a prefix
- // match, return that, else return nil
- break
- }
- }
-
- // Return the record with larger match with prefix.
- return r
-}
-
-// ReadRecord reads a record from a slice at the given position.
-//
-// The given position must point to the first byte where the record starts.
-func (rl RecordList) ReadRecord(pos int) Record {
- sizeOffset := pos + FileOffsetBytes + FileSizeBytes
- size := rl[int(sizeOffset)]
- return Record{
- pos,
- KeyPositionPair{rl[sizeOffset+KeySizeBytes : sizeOffset+KeySizeBytes+int(size)], types.Block{
- Offset: types.Position(binary.LittleEndian.Uint64(rl[pos:])),
- Size: types.Size(binary.LittleEndian.Uint32(rl[pos+FileOffsetBytes:])),
- }},
- }
-}
-
-// Len returns the byte length of the record list.
-func (rl RecordList) Len() int {
- return len(rl)
-}
-
-// Empty eturns true if the record list is empty.
-func (rl RecordList) Empty() bool {
- return len(rl) == 0
-}
-
-// Iter returns an iterator for a record list
-func (rl RecordList) Iter() *RecordListIter {
- return &RecordListIter{rl, 0}
-}
-
-// RecordListIter provides an easy mechanism to iterate a record list
-type RecordListIter struct {
- records RecordList
- // The data we are iterating over
- // The current position within the data
- pos int
-}
-
-// Done indicates whether there are more records to read
-func (rli *RecordListIter) Done() bool {
- return rli.pos >= len(rli.records)
-}
-
-// Next returns the next record in the list
-func (rli *RecordListIter) Next() Record {
- record := rli.records.ReadRecord(rli.pos)
- // Prepare the internal state for the next call
- rli.pos += FileOffsetBytes + FileSizeBytes + KeySizeBytes + len(record.Key)
- return record
-}
-
-// NextPos returns the position of the next record.
-func (r *Record) NextPos() int {
- return r.Pos + FileOffsetBytes + FileSizeBytes + KeySizeBytes + len(r.Key)
-}
-
-// AddKeyPosition extends record data with an encoded key and a file offset.
-//
-// The format is:
-//
-// ```text
-//
-// | 8 bytes | 1 byte | Variable size < 256 bytes |
-// | Pointer to actual data | Size of the key | Key |
-//
-// ```
-func AddKeyPosition(data []byte, keyPos KeyPositionPair) []byte {
- size := byte(len(keyPos.Key))
- offsetBytes := make([]byte, 8)
- binary.LittleEndian.PutUint64(offsetBytes, uint64(keyPos.Block.Offset))
- sizeBytes := make([]byte, 4)
- binary.LittleEndian.PutUint32(sizeBytes, uint32(keyPos.Block.Size))
- return append(append(append(append(data, offsetBytes...), sizeBytes...), size), keyPos.Key...)
-}
-
-// EncodeKeyPosition a key and and offset into a single record
-func EncodeKeyPosition(keyPos KeyPositionPair) []byte {
- encoded := make([]byte, 0, FileOffsetBytes+FileSizeBytes+KeySizeBytes+len(keyPos.Key))
- return AddKeyPosition(encoded, keyPos)
-}
-
-// ReadBucketPrefix reads the bucket prefix and returns it.
-func ReadBucketPrefix(reader io.Reader) (BucketIndex, error) {
- bucketPrefixBuffer := make([]byte, BucketPrefixSize)
- _, err := io.ReadFull(reader, bucketPrefixBuffer)
- if err != nil {
- return 0, err
- }
- return BucketIndex(binary.LittleEndian.Uint32(bucketPrefixBuffer)), nil
-}
diff --git a/store/index/recordlist_test.go b/store/index/recordlist_test.go
deleted file mode 100644
index 35f4f576..00000000
--- a/store/index/recordlist_test.go
+++ /dev/null
@@ -1,305 +0,0 @@
-package index_test
-
-import (
- "fmt"
- "testing"
-
- "github.com/rpcpool/yellowstone-faithful/store/index"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-func TestEncodeKeyPosition(t *testing.T) {
- key := []byte("abcdefg")
- offset := 4326
- size := 64
- encoded := index.EncodeKeyPosition(index.KeyPositionPair{key, types.Block{Offset: types.Position(offset), Size: types.Size(size)}})
- require.Equal(t,
- encoded,
- []byte{
- 0xe6, 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x40, 0x00, 0x00, 0x00, 0x07, 0x61, 0x62, 0x63, 0x64, 0x65,
- 0x66, 0x67,
- },
- )
-}
-
-func TestRecordListIterator(t *testing.T) {
- // Create records
- var keys []string
- for i := 0; i < 20; i++ {
- keys = append(keys, fmt.Sprintf("key-%02d", i))
- }
-
- var expected []index.Record
- for i, key := range keys {
- expected = append(expected, index.Record{
- KeyPositionPair: index.KeyPositionPair{
- Key: []byte(key),
- Block: types.Block{Offset: types.Position(i), Size: types.Size(i)},
- },
- Pos: i * 19,
- })
- }
-
- // Encode them into records list
- var data []byte
- for _, record := range expected {
- encoded := index.EncodeKeyPosition(record.KeyPositionPair)
- data = append(data, encoded...)
- }
-
- // The record list have the bits that were used to determine the bucket as prefix
- prefixedData := append([]byte{0, 0, 0, 0}, data...)
- // Verify that it can be correctly iterated over those encoded records
- records := index.NewRecordList(prefixedData)
- recordsIter := records.Iter()
- for _, record := range expected {
- require.False(t, recordsIter.Done())
- require.Equal(t, record, recordsIter.Next())
- }
-
- // Verify that we can compute next position successfully.
- // First key
- r := records.GetRecord([]byte(keys[1]))
- npos := r.NextPos()
- nr := records.GetRecord([]byte(keys[2]))
- require.Equal(t, npos, nr.Pos)
-}
-
-func TestRecordListFindKeyPosition(t *testing.T) {
- // Create data
- keys := []string{"a", "ac", "b", "d", "de", "dn", "nky", "xrlfg"}
- // Encode them into records list
- var data []byte
- for i, key := range keys {
- encoded := index.EncodeKeyPosition(index.KeyPositionPair{[]byte(key), types.Block{Offset: types.Position(i), Size: types.Size(i)}})
- data = append(data, encoded...)
- }
- // The record list have the bits that were used to determine the bucket as prefix
- prefixedData := append([]byte{0, 0, 0, 0}, data...)
- records := index.NewRecordList(prefixedData)
-
- // First key
- pos, prevRecord, hasPrev := records.FindKeyPosition([]byte("ABCD"))
- require.Equal(t, pos, 0)
- require.False(t, hasPrev)
-
- // Between two keys with same prefix, but first one being shorter
- pos, prevRecord, _ = records.FindKeyPosition([]byte("ab"))
- require.Equal(t, pos, 14)
- require.Equal(t, prevRecord.Key, []byte("a"))
-
- // Between to keys with both having a different prefix
- pos, prevRecord, hasPrev = records.FindKeyPosition([]byte("c"))
- require.True(t, hasPrev)
- require.Equal(t, pos, 43)
- require.Equal(t, prevRecord.Key, []byte("b"))
-
- // Between two keys with both having a different prefix and the input key having a
- // different length
- pos, prevRecord, _ = records.FindKeyPosition([]byte("cabefg"))
-
- require.Equal(t, pos, 43)
- require.Equal(t, prevRecord.Key, []byte("b"))
-
- // Between two keys with both having a different prefix (with one character in common),
- // all keys having the same length
- pos, prevRecord, _ = records.FindKeyPosition([]byte("dg"))
- require.Equal(t, pos, 72)
- require.Equal(t, prevRecord.Key, []byte("de"))
-
- // Between two keys with both having a different prefix, no charachter in in common and
- // different length (shorter than the input key)
- pos, prevRecord, _ = records.FindKeyPosition([]byte("hello"))
- require.Equal(t, pos, 87)
- require.Equal(t, prevRecord.Key, []byte("dn"))
-
- // Between two keys with both having a different prefix, no charachter in in common and
- // different length (longer than the input key)
- pos, prevRecord, _ = records.FindKeyPosition([]byte("pz"))
- require.Equal(t, pos, 103)
- require.Equal(t, prevRecord.Key, []byte("nky"))
-
- // Last key
- pos, prevRecord, _ = records.FindKeyPosition([]byte("z"))
- require.Equal(t, pos, 121)
- require.Equal(t, prevRecord.Key, []byte("xrlfg"))
-}
-
-// Validate that the new key was properly added
-func assertAddKey(t *testing.T, records index.RecordList, key []byte) {
- pos, _, _ := records.FindKeyPosition(key)
- newData := records.PutKeys([]index.KeyPositionPair{{key, types.Block{Offset: types.Position(773), Size: types.Size(48)}}}, pos, pos)
- // The record list have the bits that were used to determine the bucket as prefix
- prefixedNewData := append([]byte{0, 0, 0, 0}, newData...)
- newRecords := index.NewRecordList(prefixedNewData)
- insertedPos, insertedRecord, _ := newRecords.FindKeyPosition(key)
- require.Equal(t,
- insertedPos,
- pos+index.FileOffsetBytes+index.FileSizeBytes+index.KeySizeBytes+len(key),
- )
- require.Equal(t, insertedRecord.Key, key)
-}
-
-func TestRecordListAddKeyWithoutReplacing(t *testing.T) {
- // Create Data
- keys := []string{"a", "ac", "b", "d", "de", "dn", "nky", "xrlfg"}
- // Encode them into records list
- var data []byte
- for i, key := range keys {
- encoded := index.EncodeKeyPosition(index.KeyPositionPair{[]byte(key), types.Block{Offset: types.Position(i), Size: types.Size(i)}})
- data = append(data, encoded...)
- }
- // The record list have the bits that were used to determine the bucket as prefix
- prefixedData := append([]byte{0, 0, 0, 0}, data...)
- records := index.NewRecordList(prefixedData)
-
- // First key
- assertAddKey(t, records, []byte("ABCD"))
-
- // Between two keys with same prefix, but first one being shorter
- assertAddKey(t, records, []byte("ab"))
-
- // Between to keys with both having a different prefix
- assertAddKey(t, records, []byte("c"))
-
- // Between two keys with both having a different prefix and the input key having a
- // different length
- assertAddKey(t, records, []byte("cabefg"))
-
- // Between two keys with both having a different prefix (with one character in common),
- // all keys having the same length
- assertAddKey(t, records, []byte("dg"))
-
- // Between two keys with both having a different prefix, no charachter in in common and
- // different length (shorter than the input key)
- assertAddKey(t, records, []byte("hello"))
-
- // Between two keys with both having a different prefix, no charachter in in common and
- // different length (longer than the input key)
- assertAddKey(t, records, []byte("pz"))
-
- // Last key
- assertAddKey(t, records, []byte("z"))
-}
-
-// Validate that the previous key was properly replaced and the new key was added.
-func assertAddKeyAndReplacePrev(t *testing.T, records index.RecordList, key []byte, newPrevKey []byte) {
- pos, prevRecord, hasPrev := records.FindKeyPosition(key)
- require.True(t, hasPrev)
-
- keys := []index.KeyPositionPair{{newPrevKey, prevRecord.Block}, {key, types.Block{Offset: types.Position(773), Size: types.Size(48)}}}
- newData := records.PutKeys(keys, prevRecord.Pos, pos)
- // The record list have the bits that were used to determine the bucket as prefix
- prefixedNewData := append([]byte{0, 0, 0, 0}, newData...)
- newRecords := index.NewRecordList(prefixedNewData)
-
- // Find the newly added prevKey
- insertedPrevKeyPos, insertedPrevRecord, hasPrev := newRecords.FindKeyPosition(newPrevKey)
- require.True(t, hasPrev)
- require.Equal(t, insertedPrevRecord.Pos, prevRecord.Pos)
- require.Equal(t, insertedPrevRecord.Key, newPrevKey)
-
- // Find the newly added key
- insertedPos, insertedRecord, hasPrev := newRecords.FindKeyPosition(key)
- require.True(t, hasPrev)
- require.Equal(t,
- insertedPos,
- // The prev key is longer, hence use its position instead of the original one
- insertedPrevKeyPos+index.FileOffsetBytes+index.FileSizeBytes+index.KeySizeBytes+len(key),
- )
- require.Equal(t, insertedRecord.Key, key)
-}
-
-// If a new key is added and it fully contains the previous key, them the previous key needs
-// to be updated as well. This is what these tests are about.
-func TestRecordListAddKeyAndReplacePrev(t *testing.T) {
- // Create Data
- keys := []string{"a", "ac", "b", "d", "de", "dn", "nky", "xrlfg"}
- // Encode them into records list
- var data []byte
- for i, key := range keys {
- encoded := index.EncodeKeyPosition(index.KeyPositionPair{[]byte(key), types.Block{Offset: types.Position(i), Size: types.Size(i)}})
- data = append(data, encoded...)
- }
- // The record list have the bits that were used to determine the bucket as prefix
- prefixedData := append([]byte{0, 0, 0, 0}, data...)
- records := index.NewRecordList(prefixedData)
-
- // Between two keys with same prefix, but first one being shorter
- assertAddKeyAndReplacePrev(t, records, []byte("ab"), []byte("aa"))
-
- // Between two keys with same prefix, but first one being shorter. Replacing the previous
- // key which is more than one character longer than the existong one.
- assertAddKeyAndReplacePrev(t, records, []byte("ab"), []byte("aaaa"))
-
- // Between to keys with both having a different prefix
- assertAddKeyAndReplacePrev(t, records, []byte("c"), []byte("bx"))
-
- // Between two keys with both having a different prefix and the input key having a
- // different length
- assertAddKeyAndReplacePrev(t, records, []byte("cabefg"), []byte("bbccdd"))
-
- // Between two keys with both having a different prefix (with one character in common),
- // extending the prev key with an additional character to be distinguishable from the new
- // key
- assertAddKeyAndReplacePrev(t, records, []byte("deq"), []byte("dej"))
-
- // Last key
- assertAddKeyAndReplacePrev(t, records, []byte("xrlfgu"), []byte("xrlfgs"))
-}
-
-func TestRecordListGetKey(t *testing.T) {
- // Create Data
- keys := []string{"a", "ac", "b", "de", "dn", "nky", "xrlfg"}
- // Encode them into records list
- var data []byte
- for i, key := range keys {
- encoded := index.EncodeKeyPosition(index.KeyPositionPair{[]byte(key), types.Block{Offset: types.Position(i), Size: types.Size(i)}})
- data = append(data, encoded...)
- }
- // The record list have the bits that were used to determine the bucket as prefix
- prefixedData := append([]byte{0, 0, 0, 0}, data...)
- records := index.NewRecordList(prefixedData)
-
- // First key
- blk, has := records.Get([]byte("a"))
- require.True(t, has)
- require.Equal(t, blk, types.Block{Offset: types.Position(0), Size: types.Size(0)})
-
- // Key with same prefix, but it's the second one
- blk, has = records.Get([]byte("ac"))
- require.True(t, has)
- require.Equal(t, blk, types.Block{Offset: types.Position(1), Size: types.Size(1)})
-
- // Key with same length as two other keys, sharing a prefix
- blk, has = records.Get([]byte("de"))
- require.True(t, has)
- require.Equal(t, blk, types.Block{Offset: types.Position(3), Size: types.Size(3)})
-
- // Key that is sharing a prefix, but is longer
- blk, has = records.Get([]byte("dngho"))
- require.True(t, has)
- require.Equal(t, blk, types.Block{Offset: types.Position(4), Size: types.Size(4)})
-
- // Key that is the last one
- blk, has = records.Get([]byte("xrlfg"))
- require.True(t, has)
- require.Equal(t, blk, types.Block{Offset: types.Position(6), Size: types.Size(6)})
-
- // Key that is shorter than the inserted ones cannot match
- _, has = records.Get([]byte("d"))
- require.False(t, has)
-
- // Key that is before all keys
- _, has = records.Get([]byte("ABCD"))
- require.False(t, has)
-
- // Key that is after all keys
- _, has = records.Get([]byte("zzzzz"))
- require.False(t, has)
-
- // Key that matches a prefix of some keys, but doesn't match fully
- _, has = records.Get([]byte("dg"))
- require.False(t, has)
-}
diff --git a/store/index/upgrade.go b/store/index/upgrade.go
deleted file mode 100644
index 09a3b2ab..00000000
--- a/store/index/upgrade.go
+++ /dev/null
@@ -1,146 +0,0 @@
-package index
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bufio"
- "context"
- "encoding/binary"
- "errors"
- "fmt"
- "io"
- "os"
-
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-func upgradeIndex(ctx context.Context, name, headerPath string, maxFileSize uint32) error {
- if ctx.Err() != nil {
- return ctx.Err()
- }
-
- inFile, err := os.Open(name)
- if err != nil {
- if os.IsNotExist(err) {
- return nil
- }
- return err
- }
- defer inFile.Close()
-
- version, bucketBits, _, err := readOldHeader(inFile)
- if err != nil {
- return fmt.Errorf("cannot read old index header from %s: %w", name, err)
- }
- if version != 2 {
- return fmt.Errorf("cannot convert unknown header version: %d", version)
- }
-
- fileNum, err := chunkOldIndex(ctx, inFile, name, int64(maxFileSize))
- if err != nil {
- return err
- }
- inFile.Close()
-
- if err = writeHeader(headerPath, newHeader(bucketBits, maxFileSize)); err != nil {
- return err
- }
-
- if err = os.Remove(name); err != nil {
- return err
- }
-
- log.Infow("Replaced old index with multiple files", "replaced", name, "files", fileNum+1)
- log.Infof("Upgraded index from version 2 to %d", IndexVersion)
- return nil
-}
-
-func readOldHeader(file *os.File) (byte, byte, types.Position, error) {
- headerSizeBuffer := make([]byte, sizePrefixSize)
- _, err := io.ReadFull(file, headerSizeBuffer)
- if err != nil {
- return 0, 0, 0, err
- }
- headerSize := binary.LittleEndian.Uint32(headerSizeBuffer)
- headerBytes := make([]byte, headerSize)
- _, err = io.ReadFull(file, headerBytes)
- if err != nil {
- return 0, 0, 0, err
- }
- version := headerBytes[0]
- bucketBits := headerBytes[1]
-
- return version, bucketBits, types.Position(sizePrefixSize + headerSize), nil
-}
-
-func chunkOldIndex(ctx context.Context, file *os.File, name string, fileSizeLimit int64) (uint32, error) {
- var fileNum uint32
- outName := indexFileName(name, fileNum)
- outFile, err := createFileAppend(outName)
- if err != nil {
- return 0, err
- }
- log.Infof("Upgrade created index file %s", outName)
- writer := bufio.NewWriterSize(outFile, indexBufferSize)
- reader := bufio.NewReaderSize(file, indexBufferSize)
-
- sizeBuffer := make([]byte, sizePrefixSize)
- var written int64
- for {
- _, err = io.ReadFull(reader, sizeBuffer)
- if err != nil {
- if errors.Is(err, io.EOF) {
- break
- }
- return 0, err
- }
- size := binary.LittleEndian.Uint32(sizeBuffer)
- if _, err = writer.Write(sizeBuffer); err != nil {
- outFile.Close()
- return 0, err
- }
- n, err := io.CopyN(writer, reader, int64(size))
- if err != nil {
- outFile.Close()
- return 0, err
- }
- if n != int64(size) {
- writer.Flush()
- outFile.Close()
- return 0, fmt.Errorf("count not read complete entry from index")
- }
- written += sizePrefixSize + int64(size)
- if written >= fileSizeLimit {
- if err = writer.Flush(); err != nil {
- return 0, err
- }
- outFile.Close()
- if ctx.Err() != nil {
- return 0, ctx.Err()
- }
- fileNum++
- outName = indexFileName(name, fileNum)
- outFile, err = createFileAppend(outName)
- if err != nil {
- return 0, err
- }
- log.Infof("Upgrade created index file %s", outName)
- writer.Reset(outFile)
- written = 0
- }
- }
- if written != 0 {
- if err = writer.Flush(); err != nil {
- return 0, err
- }
- }
- outFile.Close()
- return fileNum, nil
-}
-
-func createFileAppend(name string) (*os.File, error) {
- return os.OpenFile(name, os.O_WRONLY|os.O_APPEND|os.O_CREATE|os.O_TRUNC, 0o644)
-}
diff --git a/store/index/upgrade_test.go b/store/index/upgrade_test.go
deleted file mode 100644
index 30846c9d..00000000
--- a/store/index/upgrade_test.go
+++ /dev/null
@@ -1,135 +0,0 @@
-package index
-
-import (
- "bufio"
- "context"
- "encoding/binary"
- "errors"
- "io"
- "os"
- "path/filepath"
- "testing"
-
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-const testIndexPath = "valuestore_test/storethehash.index"
-
-// testFileSizeLimt is the maximum size for new index files. Using a small
-// file size for testing so that the test index gets split into multiple files.
-const testFileSizeLimit = 1024
-
-func TestReadOldHeader(t *testing.T) {
- inFile, err := os.Open(testIndexPath)
- require.NoError(t, err)
- defer inFile.Close()
-
- version, bucketBits, _, err := readOldHeader(inFile)
- require.NoError(t, err)
- require.Equal(t, version, byte(2))
- require.Equal(t, bucketBits, byte(24))
-}
-
-func TestChunkOldIndex(t *testing.T) {
- oldFile, err := openFileForScan(testIndexPath)
- require.NoError(t, err)
- defer oldFile.Close()
-
- // Skip header in old file.
- _, bucketBits, headerSize, err := readOldHeader(oldFile)
- require.NoError(t, err)
-
- // Allocate old buckets.
- oldBuckets, err := NewBuckets(bucketBits)
- require.NoError(t, err)
-
- // Scan the old file into the buckets.
- t.Log("Scanning old index")
- err = testScanIndexFile(oldFile, 0, oldBuckets, 0)
- require.NoError(t, err)
-
- // Return to beginning of old file.
- _, err = oldFile.Seek(int64(headerSize), 0)
- require.NoError(t, err)
-
- newIndexPath := filepath.Join(t.TempDir(), "storethehash.index")
-
- // Do the upgrade to split the index into multiple files.
- t.Log("Chunking old index into new index files")
- lastChunkNum, err := chunkOldIndex(context.Background(), oldFile, newIndexPath, testFileSizeLimit)
- require.NoError(t, err)
- t.Logf("Split old index into %d files", lastChunkNum)
-
- // Allocate new buckets.
- newBuckets, err := NewBuckets(bucketBits)
- require.NoError(t, err)
-
- var fileNum, lastFileNum uint32
- var prevSize int64
- for {
- fileName := indexFileName(newIndexPath, fileNum)
- t.Logf("Scanning new index file %s", fileName)
- newFile, err := openFileForScan(fileName)
- if os.IsNotExist(err) {
- break
- }
- require.NoError(t, err)
-
- fi, err := newFile.Stat()
- require.NoError(t, err)
-
- err = testScanIndexFile(newFile, fileNum, newBuckets, prevSize)
- newFile.Close()
- require.NoError(t, err)
-
- prevSize += fi.Size()
-
- lastFileNum = fileNum
- fileNum++
- }
- require.Equal(t, lastFileNum, lastChunkNum)
-
- t.Log("Compare old to new buckets")
- for i := 0; i < len(oldBuckets); i++ {
- require.Equal(t, oldBuckets[i], newBuckets[i])
- }
-}
-
-func testScanIndexFile(file *os.File, fileNum uint32, buckets Buckets, prevSize int64) error {
- buffered := bufio.NewReader(file)
- sizeBuffer := make([]byte, sizePrefixSize)
- scratch := make([]byte, 256)
- var iterPos int64
- for {
- _, err := io.ReadFull(buffered, sizeBuffer)
- if err != nil {
- if errors.Is(err, io.EOF) {
- break
- }
- return err
- }
- size := binary.LittleEndian.Uint32(sizeBuffer)
-
- pos := iterPos + sizePrefixSize
- iterPos = pos + int64(size)
- if int(size) > len(scratch) {
- scratch = make([]byte, size)
- }
- data := scratch[:size]
- _, err = io.ReadFull(buffered, data)
- if err != nil {
- if errors.Is(err, io.EOF) {
- return errors.New("unexpected EOF")
- }
- return err
- }
-
- bucketPrefix := BucketIndex(binary.LittleEndian.Uint32(data))
- err = buckets.Put(bucketPrefix, types.Position(pos+prevSize))
- if err != nil {
- return err
- }
- }
- return nil
-}
diff --git a/store/index/valuestore_test/storethehash.index b/store/index/valuestore_test/storethehash.index
deleted file mode 100644
index 143010a4..00000000
Binary files a/store/index/valuestore_test/storethehash.index and /dev/null differ
diff --git a/store/index/valuestore_test/storethehash.index.free b/store/index/valuestore_test/storethehash.index.free
deleted file mode 100644
index e69de29b..00000000
diff --git a/store/iterator.go b/store/iterator.go
deleted file mode 100644
index 10d458cb..00000000
--- a/store/iterator.go
+++ /dev/null
@@ -1,55 +0,0 @@
-package store
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "io"
-
- "github.com/rpcpool/yellowstone-faithful/store/index"
-)
-
-// Iterator iterates keys and values. Any write to the store potentially
-// invalidates the iterator and may cause values to be missed or seen again.
-type Iterator struct {
- index *index.Index
- indexIter *index.Iterator
-}
-
-// NewIterator creates a new store iterator.
-func (s *Store) NewIterator() *Iterator {
- _ = s.Flush()
- return &Iterator{
- index: s.index,
- indexIter: s.index.NewIterator(),
- }
-}
-
-// Next returns the next key and value. Returns io.EOF error when done.
-func (it *Iterator) Next() ([]byte, []byte, error) {
- for {
- rec, done, err := it.indexIter.Next()
- if err != nil {
- return nil, nil, err
- }
- if done {
- return nil, nil, io.EOF
- }
-
- // Get the key and value stored in primary to see if it is the same
- // (index only stores prefixes).
- key, value, err := it.index.Primary.Get(rec.Block)
- if err != nil || key == nil {
- // Record no longer there, skip.
- continue
- }
-
- return key, value, nil
- }
-}
-
-func (it *Iterator) Progress() float64 {
- return it.indexIter.Progress()
-}
diff --git a/store/option.go b/store/option.go
deleted file mode 100644
index 99a3c5af..00000000
--- a/store/option.go
+++ /dev/null
@@ -1,111 +0,0 @@
-package store
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "time"
-
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-const (
- defaultFileCacheSize = 512
- defaultIndexSizeBits = uint8(24)
- defaultIndexFileSize = uint32(1024 * 1024 * 1024)
- defaultPrimaryFileSize = uint32(1024 * 1024 * 1024)
- defaultBurstRate = 4 * 1024 * 1024
- defaultSyncInterval = time.Second
- defaultGCInterval = 30 * time.Minute
- defaultGCTimeLimit = 5 * time.Minute
-)
-
-type config struct {
- fileCacheSize int
- indexSizeBits uint8
- indexFileSize uint32
- primaryFileSize uint32
- syncInterval time.Duration
- burstRate types.Work
- gcInterval time.Duration
- gcTimeLimit time.Duration
- syncOnFlush bool
-}
-
-type Option func(*config)
-
-// apply applies the given options to this config.
-func (c *config) apply(opts []Option) {
- for _, opt := range opts {
- opt(c)
- }
-}
-
-// FileCacheSize is the number of open files the index file cache may keep.
-func FileCacheSize(size int) Option {
- return func(c *config) {
- c.fileCacheSize = size
- }
-}
-
-// IndexBitSize is the number of bits in an index prefix.
-func IndexBitSize(indexBitSize uint8) Option {
- return func(c *config) {
- c.indexSizeBits = indexBitSize
- }
-}
-
-// IndexFileSize is the maximum offset an index record can have within an
-// individual index file, before the record must be stored in another file.
-func IndexFileSize(indexFileSize uint32) Option {
- return func(c *config) {
- c.indexFileSize = indexFileSize
- }
-}
-
-// PrimaryFileSize is the maximum offset a primary record can have within an
-// individual primary file, before the record must be stored in another file.
-func PrimaryFileSize(fileSize uint32) Option {
- return func(c *config) {
- c.primaryFileSize = fileSize
- }
-}
-
-// SyncInterval determines how frequently changes are flushed to disk.
-func SyncInterval(syncInterval time.Duration) Option {
- return func(c *config) {
- c.syncInterval = syncInterval
- }
-}
-
-// BurstRate specifies how much data can accumulate in memory, at a rate faster
-// than can be flushed, before causing a synchronous flush.
-func BurstRate(burstRate uint64) Option {
- return func(c *config) {
- c.burstRate = types.Work(burstRate)
- }
-}
-
-// GCInterval is the amount of time to wait between GC cycles. A value of 0
-// disables garbage collection.
-func GCInterval(gcInterval time.Duration) Option {
- return func(c *config) {
- c.gcInterval = gcInterval
- }
-}
-
-// GCTimeLimit is the maximum amount of time that a GC cycle may run.
-func GCTimeLimit(gcTimeLimit time.Duration) Option {
- return func(c *config) {
- c.gcTimeLimit = gcTimeLimit
- }
-}
-
-// SyncOnFlush, when set to true, causes fsync to be called as part of Flush.
-func SyncOnFlush(syncOnFlush bool) Option {
- return func(c *config) {
- c.syncOnFlush = syncOnFlush
- }
-}
diff --git a/store/primary/gsfaprimary/gsfaprimary.go b/store/primary/gsfaprimary/gsfaprimary.go
deleted file mode 100644
index d0463422..00000000
--- a/store/primary/gsfaprimary/gsfaprimary.go
+++ /dev/null
@@ -1,595 +0,0 @@
-package gsfaprimary
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bufio"
- "bytes"
- "context"
- "fmt"
- "io"
- "os"
- "path/filepath"
- "sync"
-
- "github.com/gagliardetto/solana-go"
- logging "github.com/ipfs/go-log/v2"
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/rpcpool/yellowstone-faithful/store/freelist"
- "github.com/rpcpool/yellowstone-faithful/store/primary"
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-var log = logging.Logger("storethehash/gsfaprimary")
-
-const (
- // PrimaryVersion is stored in the header data to indicate how to interpret
- // primary data.
- PrimaryVersion = 1
-
- // defaultMaxFileSize is largest the max file size is allowed to be.
- defaultMaxFileSize = uint32(1024 * 1024 * 1024)
-
- // blockBufferSize is the size of primary I/O buffers. If has the same size
- // as the linux pipe size.
- blockBufferSize = 16 * 4096
- // blockPoolSize is the size of the primary cache.
- blockPoolSize = 1024
-
- // TODO: remove sizePrefixSize anywhere it is used.
- sizePrefixSize = 1
-
- // TODO: remove deletedBit anywhere it is used.
- // TODO: replace deletedBit with a byte? or do the same thing
- deletedBit = uint32(1 << 31)
-)
-const primaryRecordSize = 32 + 8
-
-// A primary storage that is multihash aware.
-type GsfaPrimary struct {
- basePath string
- file *os.File
- headerPath string
- maxFileSize uint32
- writer *bufio.Writer
- outstandingWork types.Work
- curPool, nextPool blockPool
- poolLk sync.RWMutex
- flushLock sync.Mutex
- fileCache *filecache.FileCache
-
- // fileNum and length track flushed data.
- fileNum uint32
- length types.Position
-
- // recFileNum and recPos track where each record will be written when they
- // are flushed to disk.
- recFileNum uint32
- recPos types.Position
-
- closed bool
-}
-
-type blockRecord struct {
- key []byte
- value []byte
-}
-type blockPool struct {
- refs map[types.Block]int
- blocks []blockRecord
-}
-
-func newBlockPool() blockPool {
- return blockPool{
- refs: make(map[types.Block]int, blockPoolSize),
- blocks: make([]blockRecord, 0, blockPoolSize),
- }
-}
-
-func _clone(b []byte) []byte {
- if b == nil {
- return nil
- }
- return append(b[:0:0], b...)
-}
-
-// Open opens the gsfa primary storage file. The primary is created if
-// there is no existing primary at the specified path. If there is an older
-// version primary, then it is automatically upgraded.
-func Open(path string, freeList *freelist.FreeList, fileCache *filecache.FileCache, maxFileSize uint32) (*GsfaPrimary, error) {
- headerPath := filepath.Clean(path) + ".info"
-
- if maxFileSize == 0 {
- maxFileSize = defaultMaxFileSize
- } else if maxFileSize > defaultMaxFileSize {
- return nil, fmt.Errorf("maximum primary file size cannot exceed %d", defaultMaxFileSize)
- }
-
- var lastPrimaryNum uint32
- header, err := readHeader(headerPath)
- if os.IsNotExist(err) {
- // If header does not exist, then upgrade primary.
- lastPrimaryNum, err = upgradePrimary(context.Background(), path, headerPath, maxFileSize, freeList)
- if err != nil {
- return nil, fmt.Errorf("error upgrading primary: %w", err)
- }
-
- // Header does not exist, so create new one.
- header = newHeader(maxFileSize)
- if err = writeHeader(headerPath, header); err != nil {
- return nil, err
- }
- } else {
- if err != nil {
- return nil, err
- }
-
- if header.MaxFileSize != maxFileSize {
- return nil, types.ErrPrimaryWrongFileSize{header.MaxFileSize, maxFileSize}
- }
-
- // Find last primary file.
- lastPrimaryNum, err = findLastPrimary(path, header.FirstFile)
- if err != nil {
- return nil, err
- }
- }
-
- file, err := os.OpenFile(primaryFileName(path, lastPrimaryNum), os.O_RDWR|os.O_APPEND|os.O_CREATE, 0o644)
- if err != nil {
- return nil, err
- }
- length, err := file.Seek(0, io.SeekEnd)
- if err != nil {
- return nil, err
- }
-
- mp := &GsfaPrimary{
- basePath: path,
- file: file,
- fileCache: fileCache,
- headerPath: headerPath,
- maxFileSize: maxFileSize,
- writer: bufio.NewWriterSize(file, blockBufferSize),
- curPool: newBlockPool(),
- nextPool: newBlockPool(),
-
- fileNum: lastPrimaryNum,
- length: types.Position(length),
-
- recFileNum: lastPrimaryNum,
- recPos: types.Position(length),
- }
-
- return mp, nil
-}
-
-func (cp *GsfaPrimary) FileSize() uint32 {
- return cp.maxFileSize
-}
-
-// upgradeCachedValue updates the cached value for the given key if it exists.
-// This is used to make sure that the cached value is updated when a new value
-// is written to the primary (overwriting the old value), otherwise the cached
-// value will be stale.
-func (cp *GsfaPrimary) upgradeCachedValue(blk types.Block, key []byte, value []byte) {
- idx, ok := cp.nextPool.refs[blk]
- if ok {
- if !bytes.Equal(cp.nextPool.blocks[idx].key, key) {
- return
- }
- cp.nextPool.blocks[idx].value = value
- }
- idx, ok = cp.curPool.refs[blk]
- if ok {
- if !bytes.Equal(cp.curPool.blocks[idx].key, key) {
- return
- }
- cp.curPool.blocks[idx].value = value
- }
-}
-
-func (cp *GsfaPrimary) getCached(blk types.Block) ([]byte, []byte, error) {
- cp.poolLk.RLock()
- defer cp.poolLk.RUnlock()
- idx, ok := cp.nextPool.refs[blk]
- if ok {
- br := cp.nextPool.blocks[idx]
- return br.key, br.value, nil
- }
- idx, ok = cp.curPool.refs[blk]
- if ok {
- br := cp.curPool.blocks[idx]
- return br.key, br.value, nil
- }
- if blk.Offset >= absolutePrimaryPos(cp.recPos, cp.recFileNum, cp.maxFileSize) {
- return nil, nil, fmt.Errorf("error getting cached multihashed primary: %w", types.ErrOutOfBounds)
- }
- return nil, nil, nil
-}
-
-func (cp *GsfaPrimary) Get(blk types.Block) ([]byte, []byte, error) {
- key, value, err := cp.getCached(blk)
- if err != nil {
- return nil, nil, err
- }
- if key != nil && value != nil {
- return key, value, nil
- }
-
- localPos, fileNum := localizePrimaryPos(blk.Offset, cp.maxFileSize)
-
- file, err := cp.fileCache.Open(primaryFileName(cp.basePath, fileNum))
- if err != nil {
- return nil, nil, err
- }
- defer cp.fileCache.Close(file)
-
- read := make([]byte, int(blk.Size))
- if _, err = file.ReadAt(read, int64(localPos)); err != nil {
- return nil, nil, fmt.Errorf("error reading data from gsfa primary: %w", err)
- }
-
- return readNode(read)
-}
-
-type Pubkey []byte
-
-// readNode extracts the pubkey from the data read and splits key and value.
-func readNode(data []byte) (Pubkey, []byte, error) {
- c, n, err := readPubkey(data)
- if err != nil {
- return Pubkey{}, nil, err
- }
-
- return c, data[n:], nil
-}
-
-func readPubkey(buf []byte) (Pubkey, int, error) {
- // the pubkey is 32 bytes
- if len(buf) < 32 {
- return Pubkey{}, 0, fmt.Errorf("error reading pubkey from primary: expected at least 32 bytes, got %d", len(buf))
- }
- pk := buf[:32]
- return pk, 32, nil
-}
-
-// Put adds a new pending blockRecord to the pool and returns a Block that
-// contains the location that the block will occupy in the primary. The
-// returned primary location must be an absolute position across all primary
-// files.
-func (cp *GsfaPrimary) Put(key []byte, value []byte) (types.Block, error) {
- recSize := int64(len(key) + len(value))
- dataSize := primaryRecordSize
- if recSize != int64(dataSize) {
- return types.Block{}, fmt.Errorf("expected record size %d, got %d", dataSize, recSize)
- }
-
- cp.poolLk.Lock()
- defer cp.poolLk.Unlock()
-
- if cp.recPos >= types.Position(cp.maxFileSize) {
- cp.recFileNum++
- cp.recPos = 0
- }
-
- // Tell index the location that this record will be writtten.
- absRecPos := absolutePrimaryPos(cp.recPos, cp.recFileNum, cp.maxFileSize)
- blk := types.Block{Offset: absRecPos, Size: types.Size(recSize)}
-
- cp.recPos += types.Position(dataSize)
-
- cp.nextPool.refs[blk] = len(cp.nextPool.blocks)
- cp.nextPool.blocks = append(cp.nextPool.blocks, blockRecord{_clone(key), _clone(value)})
- cp.outstandingWork += types.Work(dataSize)
- return blk, nil
-}
-
-func (cp *GsfaPrimary) Overwrite(blk types.Block, key []byte, value []byte) error {
- recSize := int64(len(key) + len(value))
-
- if recSize != int64(blk.Size) {
- return fmt.Errorf("expected record size %d, got %d", blk.Size, recSize)
- }
- cp.poolLk.Lock()
- defer cp.poolLk.Unlock()
-
- localPos, fileNum := localizePrimaryPos(blk.Offset, cp.maxFileSize)
-
- fi, err := os.OpenFile(primaryFileName(cp.basePath, fileNum), os.O_WRONLY, 0o666)
- if err != nil {
- return err
- }
- defer fi.Close()
- payload := append(key, value...)
-
- // overwrite the record
- if _, err = fi.WriteAt(payload, int64(localPos)); err != nil {
- return fmt.Errorf("error writing data to gsfa primary: %w", err)
- }
- cp.upgradeCachedValue(blk, _clone(key), _clone(value))
- return nil
-}
-
-func (cp *GsfaPrimary) flushBlock(key []byte, value []byte) (types.Work, error) {
- if cp.length >= types.Position(cp.maxFileSize) {
- fileNum := cp.fileNum + 1
- primaryPath := primaryFileName(cp.basePath, fileNum)
- // If the primary file being opened already exists then fileNum has
- // wrapped and there are max uint32 of index files. This means that
- // maxFileSize is set far too small or GC is disabled.
- if _, err := os.Stat(primaryPath); !os.IsNotExist(err) {
- return 0, fmt.Errorf("creating primary file overwrites existing, check file size, gc and path (maxFileSize=%d) (path=%s)", cp.maxFileSize, primaryPath)
- }
-
- file, err := os.OpenFile(primaryPath, os.O_RDWR|os.O_APPEND|os.O_CREATE, 0o644)
- if err != nil {
- return 0, fmt.Errorf("cannot open new primary file %s: %w", primaryPath, err)
- }
- if err = cp.writer.Flush(); err != nil {
- return 0, fmt.Errorf("cannot write to primary file %s: %w", cp.file.Name(), err)
- }
-
- cp.file.Close()
- cp.writer.Reset(file)
- cp.file = file
- cp.fileNum = fileNum
- cp.length = 0
- }
-
- size := len(key) + len(value)
- if _, err := cp.writer.Write(append(key, value...)); err != nil {
- return 0, err
- }
-
- writeSize := size
- cp.length += types.Position(writeSize)
-
- return types.Work(writeSize), nil
-}
-
-func (cp *GsfaPrimary) IndexKey(key []byte) ([]byte, error) {
- if len(key) != 32 {
- return nil, fmt.Errorf("invalid key length: %d", len(key))
- }
- // This is a sanity-check to see if it really is a solana pubkey
- decoded := solana.PublicKeyFromBytes(key)
- return decoded[:], nil
-}
-
-func (cp *GsfaPrimary) GetIndexKey(blk types.Block) ([]byte, error) {
- key, _, err := cp.Get(blk)
- if err != nil {
- return nil, err
- }
- if key == nil {
- return nil, nil
- }
- return cp.IndexKey(key)
-}
-
-// Flush writes outstanding work and buffered data to the primary file.
-func (cp *GsfaPrimary) Flush() (types.Work, error) {
- // Only one Flush at a time, otherwise the 2nd Flush can swap the pools
- // while the 1st Flush is still reading the pool being flushed. That could
- // cause the pool being read by the 1st Flush to be written to
- // concurrently.
- cp.flushLock.Lock()
- defer cp.flushLock.Unlock()
-
- cp.poolLk.Lock()
- // If no new data, then nothing to do.
- if len(cp.nextPool.blocks) == 0 {
- cp.poolLk.Unlock()
- return 0, nil
- }
- cp.curPool = cp.nextPool
- cp.nextPool = newBlockPool()
- cp.outstandingWork = 0
- cp.poolLk.Unlock()
-
- // The pool lock is released allowing Put to write to nextPool. The
- // flushLock is still held, preventing concurrent flushes from changing the
- // pools or accessing writer.
-
- var work types.Work
- for _, record := range cp.curPool.blocks {
- blockWork, err := cp.flushBlock(record.key, record.value)
- if err != nil {
- return 0, err
- }
- work += blockWork
- }
- err := cp.writer.Flush()
- if err != nil {
- return 0, fmt.Errorf("cannot flush data to primary file %s: %w", cp.file.Name(), err)
- }
-
- return work, nil
-}
-
-// Sync commits the contents of the primary file to disk. Flush should be
-// called before calling Sync.
-func (mp *GsfaPrimary) Sync() error {
- mp.flushLock.Lock()
- defer mp.flushLock.Unlock()
- return mp.file.Sync()
-}
-
-// Close calls Flush to write work and data to the primary file, and then
-// closes the file.
-func (mp *GsfaPrimary) Close() error {
- if mp.closed {
- return nil
- }
-
- mp.fileCache.Clear()
-
- _, err := mp.Flush()
- if err != nil {
- mp.file.Close()
- return err
- }
-
- return mp.file.Close()
-}
-
-func (cp *GsfaPrimary) OutstandingWork() types.Work {
- cp.poolLk.RLock()
- defer cp.poolLk.RUnlock()
- return cp.outstandingWork
-}
-
-type Iterator struct {
- // The index data we are iterating over
- file *os.File
- // The current position within the index
- pos int64
- // The base index file path
- base string
- // The current index file number
- fileNum uint32
-}
-
-func (cp *GsfaPrimary) Iter() (primary.PrimaryStorageIter, error) {
- header, err := readHeader(cp.headerPath)
- if err != nil {
- if os.IsNotExist(err) {
- return nil, nil
- }
- return nil, err
- }
-
- return NewIterator(cp.basePath, header.FirstFile), nil
-}
-
-func NewIterator(basePath string, fileNum uint32) *Iterator {
- return &Iterator{
- base: basePath,
- fileNum: fileNum,
- }
-}
-
-func (iter *Iterator) Next() ([]byte, []byte, error) {
- if iter == nil {
- return nil, nil, nil
- }
-
- if iter.file == nil {
- file, err := os.OpenFile(primaryFileName(iter.base, iter.fileNum), os.O_RDONLY, 0o644)
- if err != nil {
- if os.IsNotExist(err) {
- return nil, nil, io.EOF
- }
- return nil, nil, err
- }
- iter.file = file
- iter.pos = 0
- }
-
- size := primaryRecordSize
- pos := iter.pos
- data := make([]byte, size)
- _, err := iter.file.ReadAt(data, pos)
- if err != nil {
- iter.file.Close()
- // if errors.Is(err, io.EOF) {
- // err = io.ErrUnexpectedEOF
- // }
- return nil, nil, err
- }
-
- iter.pos += int64(size)
- return readNode(data)
-}
-
-func (iter *Iterator) Close() error {
- if iter.file == nil {
- return nil
- }
- return iter.file.Close()
-}
-
-// StorageSize returns bytes of storage used by the primary files.
-func (cp *GsfaPrimary) StorageSize() (int64, error) {
- header, err := readHeader(cp.headerPath)
- if err != nil {
- if os.IsNotExist(err) {
- return 0, nil
- }
- return 0, err
- }
- fi, err := os.Stat(cp.headerPath)
- if err != nil {
- return 0, err
- }
- size := fi.Size()
-
- fileNum := header.FirstFile
- for {
- primaryName := primaryFileName(cp.basePath, fileNum)
-
- // Get size of primary file.
- fi, err = os.Stat(primaryName)
- if err != nil {
- if os.IsNotExist(err) {
- break
- }
- return 0, err
- }
- size += fi.Size()
-
- fileNum++
- }
- return size, nil
-}
-
-func primaryFileName(basePath string, fileNum uint32) string {
- return fmt.Sprintf("%s.%d", basePath, fileNum)
-}
-
-func primaryPosToFileNum(pos types.Position, maxFileSize uint32) (bool, uint32) {
- // Primary pos 0 means there is no data in the primary, so indicate empty.
- if pos == 0 {
- return false, 0
- }
- // The start of the entry determines which is file is used.
- return true, uint32(pos / types.Position(maxFileSize))
-}
-
-// localizePrimaryPos decodes a position into a local primary offset and file number.
-func localizePrimaryPos(pos types.Position, maxFileSize uint32) (types.Position, uint32) {
- ok, fileNum := primaryPosToFileNum(pos, maxFileSize)
- if !ok {
- // Return 0 local pos to indicate empty bucket.
- return 0, 0
- }
- // Subtract file offset to get pos within its local file.
- localPos := pos - (types.Position(fileNum) * types.Position(maxFileSize))
- return localPos, fileNum
-}
-
-func absolutePrimaryPos(localPos types.Position, fileNum, maxFileSize uint32) types.Position {
- return types.Position(maxFileSize)*types.Position(fileNum) + localPos
-}
-
-func findLastPrimary(basePath string, fileNum uint32) (uint32, error) {
- var lastFound uint32
- for {
- _, err := os.Stat(primaryFileName(basePath, fileNum))
- if err != nil {
- if os.IsNotExist(err) {
- break
- }
- return 0, err
- }
- lastFound = fileNum
- fileNum++
- }
- return lastFound, nil
-}
-
-var _ primary.PrimaryStorage = &GsfaPrimary{}
diff --git a/store/primary/gsfaprimary/gsfaprimary_test.go b/store/primary/gsfaprimary/gsfaprimary_test.go
deleted file mode 100644
index b4df5c42..00000000
--- a/store/primary/gsfaprimary/gsfaprimary_test.go
+++ /dev/null
@@ -1,194 +0,0 @@
-package gsfaprimary_test
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "io"
- "path/filepath"
- "testing"
-
- "github.com/gagliardetto/solana-go"
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/rpcpool/yellowstone-faithful/store/primary/gsfaprimary"
- "github.com/rpcpool/yellowstone-faithful/store/testutil"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-// This test is about making sure that inserts into an empty bucket result in a key that is trimmed
-// to a single byte.
-
-func TestIndexPut(t *testing.T) {
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := gsfaprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
-
- blks := testutil.GenerateEntries(5)
- expectedOffset := types.Position(0)
- for _, blk := range blks {
- expectedSize := len(blk.Key[:]) + len(blk.Value)
- loc, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- require.Equal(t, expectedOffset, loc.Offset)
- require.Equal(t, types.Size(expectedSize), loc.Size)
- expectedOffset += types.Position(expectedSize)
- }
-
- outstandingWork := primaryStorage.OutstandingWork()
- require.Equal(t, types.Work(expectedOffset), outstandingWork)
- work, err := primaryStorage.Flush()
- require.NoError(t, err)
- require.Equal(t, types.Work(expectedOffset), work)
- err = primaryStorage.Sync()
- require.NoError(t, err)
-
- iter := gsfaprimary.NewIterator(primaryPath, 0)
- t.Cleanup(func() { iter.Close() })
-
- for _, expectedBlk := range blks {
- key, value, err := iter.Next()
- require.NoError(t, err)
- blk := testutil.Entry{Key: solana.PublicKeyFromBytes(key), Value: value}
- require.NoError(t, err)
- require.True(t, expectedBlk.Key.Equals(blk.Key))
- require.Equal(t, expectedBlk.Value, blk.Value)
- }
- _, _, err = iter.Next()
- require.EqualError(t, err, io.EOF.Error())
-
- err = primaryStorage.Close()
- require.NoError(t, err)
-}
-
-func TestIndexGetEmptyIndex(t *testing.T) {
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := gsfaprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
- defer primaryStorage.Close()
-
- key, value, err := primaryStorage.Get(types.Block{
- Offset: 0,
- Size: 50,
- })
- require.Nil(t, key)
- require.Nil(t, value)
- require.Error(t, err)
-}
-
-func TestIndexGet(t *testing.T) {
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := gsfaprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
-
- // load blocks
- blks := testutil.GenerateEntries(5)
- var locs []types.Block
- for _, blk := range blks {
- loc, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- locs = append(locs, loc)
- }
-
- // should fetch from memory before flush
- for i, loc := range locs {
- expectedBlk := blks[i]
- key, value, err := primaryStorage.Get(loc)
- require.NoError(t, err)
- blk := testutil.Entry{Key: solana.PublicKeyFromBytes(key), Value: value}
- require.NoError(t, err)
- require.True(t, expectedBlk.Key.Equals(blk.Key))
- require.Equal(t, expectedBlk.Value, blk.Value)
- }
-
- // should fetch from disk after flush
- _, err = primaryStorage.Flush()
- require.NoError(t, err)
- err = primaryStorage.Sync()
- require.NoError(t, err)
-
- for i, loc := range locs {
- expectedBlk := blks[i]
- key, value, err := primaryStorage.Get(loc)
- require.NoError(t, err)
- blk := testutil.Entry{Key: solana.PublicKeyFromBytes(key), Value: value}
- require.NoError(t, err)
- require.True(t, expectedBlk.Key.Equals(blk.Key))
- require.Equal(t, expectedBlk.Value, blk.Value)
- }
-
- err = primaryStorage.Close()
- require.NoError(t, err)
-}
-
-func TestFlushRace(t *testing.T) {
- const goroutines = 64
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := gsfaprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
-
- // load blocks
- blks := testutil.GenerateEntries(5)
- for _, blk := range blks {
- _, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- }
-
- start := make(chan struct{})
- errs := make(chan error)
- for n := 0; n < goroutines; n++ {
- go func() {
- <-start
- _, err := primaryStorage.Flush()
- errs <- err
- }()
- }
- close(start)
- for n := 0; n < goroutines; n++ {
- err := <-errs
- require.NoError(t, err)
- }
-
- require.NoError(t, primaryStorage.Close())
-}
-
-func TestFlushExcess(t *testing.T) {
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := gsfaprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
-
- // load blocks
- blks := testutil.GenerateEntries(5)
- for _, blk := range blks {
- _, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- }
-
- work, err := primaryStorage.Flush()
- require.NoError(t, err)
- require.NotZero(t, work)
-
- blks = testutil.GenerateEntries(5)
- for _, blk := range blks {
- _, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- }
-
- work, err = primaryStorage.Flush()
- require.NoError(t, err)
- require.NotZero(t, work)
-
- // Another flush with no new data should not do work.
- work, err = primaryStorage.Flush()
- require.NoError(t, err)
- require.Zero(t, work)
-
- require.NoError(t, primaryStorage.Close())
-}
diff --git a/store/primary/gsfaprimary/header.go b/store/primary/gsfaprimary/header.go
deleted file mode 100644
index 71aaa106..00000000
--- a/store/primary/gsfaprimary/header.go
+++ /dev/null
@@ -1,53 +0,0 @@
-package gsfaprimary
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "encoding/json"
- "os"
-)
-
-// Header contains information about the primary. This is actually stored in a
-// separate ".info" file, but is the first file read when the index is opened.
-type Header struct {
- // A version number in case we change the header
- Version int
- // MaxFileSize is the size limit of each index file. This cannot be greater
- // than 4GiB.
- MaxFileSize uint32
- // First index file number
- FirstFile uint32
-}
-
-func newHeader(maxFileSize uint32) Header {
- return Header{
- Version: PrimaryVersion,
- MaxFileSize: maxFileSize,
- }
-}
-
-func readHeader(filePath string) (Header, error) {
- data, err := os.ReadFile(filePath)
- if err != nil {
- return Header{}, err
- }
-
- var header Header
- err = json.Unmarshal(data, &header)
- if err != nil {
- return Header{}, err
- }
-
- return header, nil
-}
-
-func writeHeader(headerPath string, header Header) error {
- data, err := json.Marshal(&header)
- if err != nil {
- return err
- }
- return os.WriteFile(headerPath, data, 0o666)
-}
diff --git a/store/primary/gsfaprimary/upgrade.go b/store/primary/gsfaprimary/upgrade.go
deleted file mode 100644
index 9fea0086..00000000
--- a/store/primary/gsfaprimary/upgrade.go
+++ /dev/null
@@ -1,326 +0,0 @@
-package gsfaprimary
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bufio"
- "context"
- "encoding/binary"
- "fmt"
- "io"
- "os"
- "path/filepath"
-
- "github.com/rpcpool/yellowstone-faithful/store/freelist"
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-type IndexRemapper struct {
- firstFile uint32
- maxFileSize uint32
- sizes []int64
-}
-
-func (mp *GsfaPrimary) NewIndexRemapper() (*IndexRemapper, error) {
- header, err := readHeader(mp.headerPath)
- if err != nil {
- return nil, err
- }
-
- var sizes []int64
- for fileNum := header.FirstFile; fileNum <= mp.fileNum; fileNum++ {
- fi, err := os.Stat(primaryFileName(mp.basePath, fileNum))
- if err != nil {
- if os.IsNotExist(err) {
- break
- }
- return nil, err
- }
- sizes = append(sizes, fi.Size())
- }
-
- // If there are no primary files, or the only primary file is small enough
- // that no remapping is needed, return a nil remapper.
- if len(sizes) == 0 || (len(sizes) == 1 && sizes[0] < int64(mp.maxFileSize)) {
- return nil, nil
- }
-
- return &IndexRemapper{
- firstFile: header.FirstFile,
- maxFileSize: mp.maxFileSize,
- sizes: sizes,
- }, nil
-}
-
-func (ir *IndexRemapper) RemapOffset(pos types.Position) (types.Position, error) {
- fileNum := ir.firstFile
- newPos := int64(pos)
- for _, size := range ir.sizes {
- if newPos < size {
- return absolutePrimaryPos(types.Position(newPos), fileNum, ir.maxFileSize), nil
- }
- newPos -= size
- fileNum++
- }
- return 0, fmt.Errorf("cannot convert out-of-range primary position: %d", pos)
-}
-
-func (ir *IndexRemapper) FileSize() uint32 {
- return ir.maxFileSize
-}
-
-func upgradePrimary(ctx context.Context, filePath, headerPath string, maxFileSize uint32, freeList *freelist.FreeList) (uint32, error) {
- // If header already exists, or old primary does not exist, then no upgrade.
- _, err := os.Stat(headerPath)
- if !os.IsNotExist(err) {
- // Header already exists, do nothing.
- return 0, nil
- }
- if _, err = os.Stat(filePath); err != nil {
- if os.IsNotExist(err) {
- // No primary to upgrade.
- return 0, nil
- }
- return 0, err
- }
-
- if ctx.Err() != nil {
- return 0, ctx.Err()
- }
-
- log.Infow("Upgrading primary storage and splitting into separate files", "newVersion", PrimaryVersion, "fileSize", maxFileSize)
- if freeList != nil {
- // Instead of remapping all the primary offsets in the freelist, call
- // the garbage collector function to process the freelist and make the
- // primary records deleted. This is safer because it can be re-applied
- // if there is a failure during this phase.
- err := applyFreeList(ctx, freeList, filePath)
- if err != nil {
- return 0, fmt.Errorf("could not apply freelist to primary: %w", err)
- }
- }
-
- fileNum, err := chunkOldPrimary(ctx, filePath, int64(maxFileSize))
- if err != nil {
- return 0, fmt.Errorf("error chunking primary: %w", err)
- }
-
- if err = writeHeader(headerPath, newHeader(maxFileSize)); err != nil {
- return 0, fmt.Errorf("error writing primary info file: %w", err)
- }
-
- if err = os.Remove(filePath); err != nil {
- return 0, fmt.Errorf("cannot remove old primary: %w", err)
- }
-
- log.Infow("Replaced old primary with multiple files", "replaced", filePath, "files", fileNum+1)
- log.Infof("Upgraded primary from version 0 to %d", PrimaryVersion)
- return fileNum, nil
-}
-
-func chunkOldPrimary(ctx context.Context, name string, fileSizeLimit int64) (uint32, error) {
- file, err := os.Open(name)
- if err != nil {
- return 0, err
- }
- defer file.Close()
-
- fi, err := file.Stat()
- if err != nil {
- return 0, err
- }
- if fi.Size() == 0 {
- return 0, nil
- }
-
- total := fi.Size()
- var fileNum uint32
- outName := primaryFileName(name, fileNum)
- outFile, err := createFileAppend(outName)
- if err != nil {
- return 0, err
- }
- log.Infow("Upgrade created primary file", "file", filepath.Base(outName))
- writer := bufio.NewWriterSize(outFile, blockBufferSize)
-
- sizeBuf := make([]byte, sizePrefixSize)
- var written int64
- var count int
- var pos int64
- scratch := make([]byte, 1024)
-
- for {
- _, err = file.ReadAt(sizeBuf, pos)
- if err != nil {
- if err != io.EOF {
- log.Errorw("Error reading primary", "err", err)
- }
- break
- }
- size := binary.LittleEndian.Uint32(sizeBuf)
- if _, err = writer.Write(sizeBuf); err != nil {
- outFile.Close()
- return 0, err
- }
- pos += sizePrefixSize
-
- del := false
- if size&deletedBit != 0 {
- size ^= deletedBit
- del = true
- }
-
- if int(size) > len(scratch) {
- scratch = make([]byte, size)
- }
- data := scratch[:size]
-
- if !del {
- if _, err = file.ReadAt(data, pos); err != nil {
- log.Errorw("Error reading primary", "err", err)
- break
- }
- }
- _, err := writer.Write(data)
- if err != nil {
- outFile.Close()
- return 0, err
- }
- pos += int64(size)
-
- written += sizePrefixSize + int64(size)
- if written >= fileSizeLimit {
- if err = writer.Flush(); err != nil {
- return 0, err
- }
- outFile.Close()
- if ctx.Err() != nil {
- return 0, ctx.Err()
- }
- fileNum++
- outName = primaryFileName(name, fileNum)
- outFile, err = createFileAppend(outName)
- if err != nil {
- return 0, err
- }
- log.Infof("Upgrade created primary file %s: %.1f%% done", filepath.Base(outName), float64(1000*pos/total)/10)
- writer.Reset(outFile)
- written = 0
- }
- count++
- }
- if written != 0 {
- if err = writer.Flush(); err != nil {
- return 0, err
- }
- }
- outFile.Close()
- return fileNum, nil
-}
-
-func createFileAppend(name string) (*os.File, error) {
- return os.OpenFile(name, os.O_WRONLY|os.O_APPEND|os.O_CREATE|os.O_TRUNC, 0o644)
-}
-
-// applyFreeList reads the freelist and marks the locations in the old primary file
-// as dead by setting the deleted bit in the record size field.
-func applyFreeList(ctx context.Context, freeList *freelist.FreeList, filePath string) error {
- flPath, err := freeList.ToGC()
- if err != nil {
- return fmt.Errorf("cannot get freelist gc file: %w", err)
- }
-
- fi, err := os.Stat(flPath)
- if err != nil {
- return fmt.Errorf("cannot stat freelist gc file: %w", err)
- }
- flSize := fi.Size()
-
- // If the freelist size is non-zero, then process its records.
- var count int
- if flSize != 0 {
- log.Infof("Applying freelist to primary storage")
-
- flFile, err := os.OpenFile(flPath, os.O_RDONLY, 0o644)
- if err != nil {
- return fmt.Errorf("error opening freelist gc file: %w", err)
- }
- defer flFile.Close()
-
- primaryFile, err := os.OpenFile(filePath, os.O_RDWR, 0o644)
- if err != nil {
- return fmt.Errorf("cannot open primary file %s: %w", filePath, err)
- }
- defer primaryFile.Close()
-
- fi, err = primaryFile.Stat()
- if err != nil {
- return fmt.Errorf("cannot stat primary file %s: %w", primaryFile.Name(), err)
- }
- primarySize := fi.Size()
-
- total := int(flSize / (types.OffBytesLen + types.SizeBytesLen))
- flIter := freelist.NewIterator(bufio.NewReader(flFile))
- sizeBuf := make([]byte, sizePrefixSize)
- percentIncr := 1
- nextPercent := percentIncr
-
- for {
- free, err := flIter.Next()
- if err != nil {
- // Done reading freelist; log if error.
- if err != io.EOF {
- log.Errorw("Error reading freelist", "err", err)
- }
- break
- }
-
- offset := int64(free.Offset)
-
- if offset > primarySize {
- log.Errorw("freelist record has out-of-range primary offset", "offset", offset, "fileSize", primarySize)
- continue // skip bad freelist entry
- }
-
- if _, err = primaryFile.ReadAt(sizeBuf, offset); err != nil {
- return err
- }
- recSize := binary.LittleEndian.Uint32(sizeBuf)
- if recSize&deletedBit != 0 {
- // Already deleted.
- continue
- }
- if recSize != uint32(free.Size) {
- log.Errorw("Record size in primary does not match size in freelist", "primaryRecordSize", recSize, "freelistRecordSize", free.Size, "file", flFile.Name(), "offset", offset)
- }
-
- // Mark the record as deleted by setting the highest bit in the
- // size. This assumes that the record size is < 2^31.
- binary.LittleEndian.PutUint32(sizeBuf, recSize|deletedBit)
- _, err = primaryFile.WriteAt(sizeBuf, int64(offset))
- if err != nil {
- return fmt.Errorf("cannot write to primary file %s: %w", flFile.Name(), err)
- }
-
- count++
-
- // Log at every percent increment.
- percent := 100 * count / total
- if percent >= nextPercent {
- log.Infof("Processed %d of %d freelist records: %d%% done", count, total, percent)
- nextPercent += percentIncr
- }
- }
- log.Infow("Marked primary records from freelist as deleted", "count", count)
- flFile.Close()
- }
-
- if err = os.Remove(flPath); err != nil {
- return fmt.Errorf("error removing freelist: %w", err)
- }
-
- return nil
-}
diff --git a/store/primary/gsfaprimary/upgrade_test.go b/store/primary/gsfaprimary/upgrade_test.go
deleted file mode 100644
index f8626cbc..00000000
--- a/store/primary/gsfaprimary/upgrade_test.go
+++ /dev/null
@@ -1,175 +0,0 @@
-package gsfaprimary
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bufio"
- "context"
- "encoding/binary"
- "errors"
- "io"
- "os"
- "path/filepath"
- "testing"
-
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/rpcpool/yellowstone-faithful/store/freelist"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-const testPrimaryPath = "valuestore_test/storethehash.data"
-
-// testFileSizeLimt is the maximum size for new primary files. Using a small
-// file size for testing so that the test primary gets split into multiple files.
-const testFileSizeLimit = 1024
-
-func TestUpgradePrimary(t *testing.T) {
- t.Skip("Skipping upgrade test because there upgrade is not supported yet.")
- oldFile, err := os.OpenFile(testPrimaryPath, os.O_RDONLY, 0o644)
- require.NoError(t, err)
- defer oldFile.Close()
-
- // Scan the old file.
- t.Log("Scanning old primary")
- oldRecs, err := testScanPrimaryFile(oldFile)
- require.NoError(t, err)
-
- // Return to beginning of old file.
- _, err = oldFile.Seek(0, io.SeekStart)
- require.NoError(t, err)
-
- newPrimaryPath := filepath.Join(t.TempDir(), "storethehash.data")
-
- // Copy test file to new location.
- err = copyFile(testPrimaryPath, newPrimaryPath)
- require.NoError(t, err)
-
- newFreeListPath := filepath.Join(t.TempDir(), "storethehash.index.free")
- freeList, err := freelist.Open(newFreeListPath)
- require.NoError(t, err)
- defer freeList.Close()
-
- // Do the upgrade to split the primary into multiple files.
- headerPath := newPrimaryPath + ".info"
- updated, err := upgradePrimary(context.Background(), newPrimaryPath, headerPath, testFileSizeLimit, freeList)
- require.NoError(t, err)
- require.NotZero(t, updated)
-
- lastChunkNum, err := findLastPrimary(newPrimaryPath, 0)
- require.NoError(t, err)
-
- t.Logf("Split old primary into %d files", lastChunkNum)
- require.Equal(t, int(lastChunkNum), 198)
-
- // Make sure original file was removed.
- _, err = os.Stat(newPrimaryPath)
- require.True(t, os.IsNotExist(err))
-
- var newRecs [][]byte
- var fileNum, lastFileNum uint32
- for {
- fileName := primaryFileName(newPrimaryPath, fileNum)
- newFile, err := os.OpenFile(fileName, os.O_RDONLY, 0o644)
- if os.IsNotExist(err) {
- break
- }
- require.NoError(t, err)
-
- _, err = newFile.Stat()
- require.NoError(t, err)
-
- recs, err := testScanPrimaryFile(newFile)
- newFile.Close()
- require.NoError(t, err)
-
- newRecs = append(newRecs, recs...)
-
- lastFileNum = fileNum
- fileNum++
- }
- require.Equal(t, lastFileNum, lastChunkNum)
-
- t.Log("Compare old to new records")
- require.Equal(t, len(oldRecs), len(newRecs))
- for i := 0; i < len(oldRecs); i++ {
- require.Equal(t, len(oldRecs[i]), len(newRecs[i]))
- require.Equal(t, oldRecs[i], newRecs[i])
- }
-
- // Check that header was created
- header, err := readHeader(headerPath)
- require.NoError(t, err)
- require.Equal(t, header.Version, 1)
- require.Equal(t, header.MaxFileSize, uint32(testFileSizeLimit))
- require.Equal(t, header.FirstFile, uint32(0))
-
- fc := filecache.New(16)
- _, err = Open(newPrimaryPath, nil, fc, 0)
- require.Equal(t, err, types.ErrPrimaryWrongFileSize{testFileSizeLimit, defaultMaxFileSize})
-
- mp, err := Open(newPrimaryPath, nil, fc, testFileSizeLimit)
- require.NoError(t, err)
- require.NoError(t, mp.Close())
-
- // Run upgrade again to make sure it does nothing.
- updated, err = upgradePrimary(context.Background(), newPrimaryPath, headerPath, testFileSizeLimit, freeList)
- require.NoError(t, err)
- require.Zero(t, updated)
-}
-
-func testScanPrimaryFile(file *os.File) ([][]byte, error) {
- var recs [][]byte
-
- buffered := bufio.NewReader(file)
- sizeBuffer := make([]byte, sizePrefixSize)
- scratch := make([]byte, 256)
- for {
- _, err := io.ReadFull(buffered, sizeBuffer)
- if err != nil {
- if errors.Is(err, io.EOF) {
- break
- }
- return nil, err
- }
- size := binary.LittleEndian.Uint32(sizeBuffer)
-
- if int(size) > len(scratch) {
- scratch = make([]byte, size)
- }
- data := scratch[:size]
- _, err = io.ReadFull(buffered, data)
- if err != nil {
- if errors.Is(err, io.EOF) {
- return nil, errors.New("unexpected EOF")
- }
- return nil, err
- }
-
- rec := make([]byte, len(sizeBuffer)+len(data))
- copy(rec, sizeBuffer)
- copy(rec[len(sizeBuffer):], data)
- recs = append(recs, rec)
- }
- return recs, nil
-}
-
-func copyFile(src, dst string) error {
- fin, err := os.Open(src)
- if err != nil {
- return err
- }
- defer fin.Close()
-
- fout, err := os.Create(dst)
- if err != nil {
- return err
- }
- defer fout.Close()
-
- _, err = io.Copy(fout, fin)
- return err
-}
diff --git a/store/primary/gsfaprimary/valuestore_test/storethehash.data b/store/primary/gsfaprimary/valuestore_test/storethehash.data
deleted file mode 100644
index 4c2d77c6..00000000
Binary files a/store/primary/gsfaprimary/valuestore_test/storethehash.data and /dev/null differ
diff --git a/store/primary/inmemory/inmemory.go b/store/primary/inmemory/inmemory.go
deleted file mode 100644
index b61fda8c..00000000
--- a/store/primary/inmemory/inmemory.go
+++ /dev/null
@@ -1,95 +0,0 @@
-package inmemory
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "io"
-
- "github.com/rpcpool/yellowstone-faithful/store/primary"
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-//! In-memory primary storage implementation.
-//!
-//! It's using a vector of tuples containing the key-value pairs.
-
-type InMemory [][2][]byte
-
-func New(data [][2][]byte) *InMemory {
- value := InMemory(data)
- return &value
-}
-
-func (im *InMemory) Get(blk types.Block) (key []byte, value []byte, err error) {
- max := len(*im)
- if blk.Offset >= types.Position(max) {
- return nil, nil, types.ErrOutOfBounds
- }
- val := (*im)[blk.Offset]
- return val[0], val[1], nil
-}
-
-func (im *InMemory) Put(key []byte, value []byte) (blk types.Block, err error) {
- pos := len(*im)
- *im = append(*im, [2][]byte{key, value})
- return types.Block{Offset: types.Position(pos), Size: 1}, nil
-}
-
-func (im *InMemory) Flush() (types.Work, error) {
- return 0, nil
-}
-
-func (im *InMemory) Sync() error {
- return nil
-}
-
-func (im *InMemory) Close() error {
- return nil
-}
-
-func (im *InMemory) OutstandingWork() types.Work {
- return 0
-}
-
-func (im *InMemory) IndexKey(key []byte) ([]byte, error) {
- return key, nil
-}
-
-func (im *InMemory) GetIndexKey(blk types.Block) ([]byte, error) {
- key, _, err := im.Get(blk)
- if err != nil {
- return nil, err
- }
- return im.IndexKey(key)
-}
-
-func (im *InMemory) Iter() (primary.PrimaryStorageIter, error) {
- return &inMemoryIter{im, 0}, nil
-}
-
-type inMemoryIter struct {
- im *InMemory
- idx int
-}
-
-func (imi *inMemoryIter) Next() ([]byte, []byte, error) {
- key, value, err := imi.im.Get(types.Block{Offset: types.Position(imi.idx)})
- if err == types.ErrOutOfBounds {
- return nil, nil, io.EOF
- }
- imi.idx++
- return key, value, nil
-}
-
-func (im *InMemory) StorageSize() (int64, error) {
- return 0, nil
-}
-
-func (im *InMemory) Overwrite(blk types.Block, key []byte, value []byte) error {
- return nil
-}
-
-var _ primary.PrimaryStorage = &InMemory{}
diff --git a/store/primary/inmemory/inmemory_test.go b/store/primary/inmemory/inmemory_test.go
deleted file mode 100644
index 1e656072..00000000
--- a/store/primary/inmemory/inmemory_test.go
+++ /dev/null
@@ -1,69 +0,0 @@
-package inmemory_test
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "testing"
-
- "github.com/rpcpool/yellowstone-faithful/store/primary/inmemory"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-func TestGet(t *testing.T) {
- aa := [2][]byte{[]byte("aa"), {0x10}}
- yy := [2][]byte{[]byte("yy"), {0x11}}
- efg := [2][]byte{[]byte("efg"), {0x12}}
- storage := inmemory.New([][2][]byte{aa, yy, efg})
-
- key, value, err := storage.Get(types.Block{Offset: 0})
- require.NoError(t, err)
- result_aa := [2][]byte{key, value}
- require.Equal(t, result_aa, aa)
- key, value, err = storage.Get(types.Block{Offset: 2})
- require.NoError(t, err)
- result_efg := [2][]byte{key, value}
-
- require.Equal(t, result_efg, efg)
- key, value, err = storage.Get(types.Block{Offset: 1})
- require.NoError(t, err)
- result_yy := [2][]byte{key, value}
-
- require.Equal(t, result_yy, yy)
-}
-
-func TestPut(t *testing.T) {
- aa := [2][]byte{[]byte("aa"), {0x10}}
- yy := [2][]byte{[]byte("yy"), {0x11}}
- efg := [2][]byte{[]byte("efg"), {0x12}}
- storage := inmemory.New([][2][]byte{})
-
- put_aa, err := storage.Put(aa[0], aa[1])
- require.NoError(t, err)
- require.Equal(t, put_aa, types.Block{Offset: 0, Size: 1})
- put_yy, err := storage.Put(yy[0], yy[1])
- require.NoError(t, err)
- require.Equal(t, put_yy, types.Block{Offset: 1, Size: 1})
- put_efg, err := storage.Put(efg[0], efg[1])
- require.NoError(t, err)
- require.Equal(t, put_efg, types.Block{Offset: 2, Size: 1})
-
- key, value, err := storage.Get(types.Block{Offset: 0})
- require.NoError(t, err)
- result_aa := [2][]byte{key, value}
-
- require.Equal(t, result_aa, aa)
- key, value, err = storage.Get(types.Block{Offset: 2})
- require.NoError(t, err)
- result_efg := [2][]byte{key, value}
-
- require.Equal(t, result_efg, efg)
- key, value, err = storage.Get(types.Block{Offset: 1})
- require.NoError(t, err)
- result_yy := [2][]byte{key, value}
-
- require.Equal(t, result_yy, yy)
-}
diff --git a/store/primary/primary.go b/store/primary/primary.go
deleted file mode 100644
index c3fb7d0d..00000000
--- a/store/primary/primary.go
+++ /dev/null
@@ -1,48 +0,0 @@
-package primary
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import "github.com/rpcpool/yellowstone-faithful/store/types"
-
-// PrimaryStorage is an interface for storing and retrieving key value pairs on disk
-type PrimaryStorage interface {
- // Returns the key-value pair from the given position.
- Get(blk types.Block) (key []byte, value []byte, err error)
-
- // Saves a key-value pair and returns the position it was stored at.
- Put(key []byte, value []byte) (blk types.Block, err error)
-
- // Overwrites the key-value pair at the given position.
- Overwrite(blk types.Block, key []byte, value []byte) error
-
- // Creates a key that can be used for the index.
- //
- // The index needs a key which is at least 4 bytes long and contains random bytes (the more
- // random the better). In case the keys you are storing don't have this property, you can
- // transform them with this function.
- //
- // By default it just returns the original key with any changes.
- IndexKey(key []byte) ([]byte, error)
-
- // Returns the key that is used for the index which is stored at the given position.
- //
- // Note that this key might differ from the key that is actually stored.
- GetIndexKey(blk types.Block) ([]byte, error)
-
- Flush() (types.Work, error)
- Sync() error
-
- Close() error
- OutstandingWork() types.Work
- Iter() (PrimaryStorageIter, error)
-
- StorageSize() (int64, error)
-}
-
-type PrimaryStorageIter interface {
- // Next should return io.EOF when done
- Next() (key []byte, value []byte, err error)
-}
diff --git a/store/primary/sig2epochprimary/header.go b/store/primary/sig2epochprimary/header.go
deleted file mode 100644
index 54ad7287..00000000
--- a/store/primary/sig2epochprimary/header.go
+++ /dev/null
@@ -1,53 +0,0 @@
-package sig2epochprimary
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "encoding/json"
- "os"
-)
-
-// Header contains information about the primary. This is actually stored in a
-// separate ".info" file, but is the first file read when the index is opened.
-type Header struct {
- // A version number in case we change the header
- Version int
- // MaxFileSize is the size limit of each index file. This cannot be greater
- // than 4GiB.
- MaxFileSize uint32
- // First index file number
- FirstFile uint32
-}
-
-func newHeader(maxFileSize uint32) Header {
- return Header{
- Version: PrimaryVersion,
- MaxFileSize: maxFileSize,
- }
-}
-
-func readHeader(filePath string) (Header, error) {
- data, err := os.ReadFile(filePath)
- if err != nil {
- return Header{}, err
- }
-
- var header Header
- err = json.Unmarshal(data, &header)
- if err != nil {
- return Header{}, err
- }
-
- return header, nil
-}
-
-func writeHeader(headerPath string, header Header) error {
- data, err := json.Marshal(&header)
- if err != nil {
- return err
- }
- return os.WriteFile(headerPath, data, 0o666)
-}
diff --git a/store/primary/sig2epochprimary/sig2epochprimary.go b/store/primary/sig2epochprimary/sig2epochprimary.go
deleted file mode 100644
index e2c20c1b..00000000
--- a/store/primary/sig2epochprimary/sig2epochprimary.go
+++ /dev/null
@@ -1,594 +0,0 @@
-package sig2epochprimary
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bufio"
- "bytes"
- "context"
- "fmt"
- "io"
- "os"
- "path/filepath"
- "sync"
-
- "github.com/gagliardetto/solana-go"
- logging "github.com/ipfs/go-log/v2"
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/rpcpool/yellowstone-faithful/store/freelist"
- "github.com/rpcpool/yellowstone-faithful/store/primary"
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-var log = logging.Logger("storethehash/gsfaprimary")
-
-const (
- // PrimaryVersion is stored in the header data to indicate how to interpret
- // primary data.
- PrimaryVersion = 1
-
- // defaultMaxFileSize is largest the max file size is allowed to be.
- defaultMaxFileSize = uint32(1024 * 1024 * 1024)
-
- // blockBufferSize is the size of primary I/O buffers. If has the same size
- // as the linux pipe size.
- blockBufferSize = 16 * 4096
- // blockPoolSize is the size of the primary cache.
- blockPoolSize = 1024
-
- // TODO: remove sizePrefixSize anywhere it is used.
- sizePrefixSize = 1
-
- // TODO: remove deletedBit anywhere it is used.
- // TODO: replace deletedBit with a byte? or do the same thing
- deletedBit = uint32(1 << 31)
-)
-const primaryRecordSize = 64 + 2
-
-type SigToEpochPrimary struct {
- basePath string
- file *os.File
- headerPath string
- maxFileSize uint32
- writer *bufio.Writer
- outstandingWork types.Work
- curPool, nextPool blockPool
- poolLk sync.RWMutex
- flushLock sync.Mutex
- fileCache *filecache.FileCache
-
- // fileNum and length track flushed data.
- fileNum uint32
- length types.Position
-
- // recFileNum and recPos track where each record will be written when they
- // are flushed to disk.
- recFileNum uint32
- recPos types.Position
-
- closed bool
-}
-
-type blockRecord struct {
- key []byte
- value []byte
-}
-type blockPool struct {
- refs map[types.Block]int
- blocks []blockRecord
-}
-
-func newBlockPool() blockPool {
- return blockPool{
- refs: make(map[types.Block]int, blockPoolSize),
- blocks: make([]blockRecord, 0, blockPoolSize),
- }
-}
-
-// Open opens the gsfa primary storage file. The primary is created if
-// there is no existing primary at the specified path. If there is an older
-// version primary, then it is automatically upgraded.
-func Open(path string, freeList *freelist.FreeList, fileCache *filecache.FileCache, maxFileSize uint32) (*SigToEpochPrimary, error) {
- headerPath := filepath.Clean(path) + ".info"
-
- if maxFileSize == 0 {
- maxFileSize = defaultMaxFileSize
- } else if maxFileSize > defaultMaxFileSize {
- return nil, fmt.Errorf("maximum primary file size cannot exceed %d", defaultMaxFileSize)
- }
-
- var lastPrimaryNum uint32
- header, err := readHeader(headerPath)
- if os.IsNotExist(err) {
- // If header does not exist, then upgrade primary.
- lastPrimaryNum, err = upgradePrimary(context.Background(), path, headerPath, maxFileSize, freeList)
- if err != nil {
- return nil, fmt.Errorf("error upgrading primary: %w", err)
- }
-
- // Header does not exist, so create new one.
- header = newHeader(maxFileSize)
- if err = writeHeader(headerPath, header); err != nil {
- return nil, err
- }
- } else {
- if err != nil {
- return nil, err
- }
-
- if header.MaxFileSize != maxFileSize {
- return nil, types.ErrPrimaryWrongFileSize{header.MaxFileSize, maxFileSize}
- }
-
- // Find last primary file.
- lastPrimaryNum, err = findLastPrimary(path, header.FirstFile)
- if err != nil {
- return nil, err
- }
- }
-
- file, err := os.OpenFile(primaryFileName(path, lastPrimaryNum), os.O_RDWR|os.O_APPEND|os.O_CREATE, 0o644)
- if err != nil {
- return nil, err
- }
- length, err := file.Seek(0, io.SeekEnd)
- if err != nil {
- return nil, err
- }
-
- mp := &SigToEpochPrimary{
- basePath: path,
- file: file,
- fileCache: fileCache,
- headerPath: headerPath,
- maxFileSize: maxFileSize,
- writer: bufio.NewWriterSize(file, blockBufferSize),
- curPool: newBlockPool(),
- nextPool: newBlockPool(),
-
- fileNum: lastPrimaryNum,
- length: types.Position(length),
-
- recFileNum: lastPrimaryNum,
- recPos: types.Position(length),
- }
-
- return mp, nil
-}
-
-func (cp *SigToEpochPrimary) FileSize() uint32 {
- return cp.maxFileSize
-}
-
-// upgradeCachedValue updates the cached value for the given key if it exists.
-// This is used to make sure that the cached value is updated when a new value
-// is written to the primary (overwriting the old value), otherwise the cached
-// value will be stale.
-func (cp *SigToEpochPrimary) upgradeCachedValue(blk types.Block, key []byte, value []byte) {
- idx, ok := cp.nextPool.refs[blk]
- if ok {
- if !bytes.Equal(cp.nextPool.blocks[idx].key, key) {
- return
- }
- cp.nextPool.blocks[idx].value = value
- }
- idx, ok = cp.curPool.refs[blk]
- if ok {
- if !bytes.Equal(cp.curPool.blocks[idx].key, key) {
- return
- }
- cp.curPool.blocks[idx].value = value
- }
-}
-
-func (cp *SigToEpochPrimary) getCached(blk types.Block) ([]byte, []byte, error) {
- cp.poolLk.RLock()
- defer cp.poolLk.RUnlock()
- idx, ok := cp.nextPool.refs[blk]
- if ok {
- br := cp.nextPool.blocks[idx]
- return br.key, br.value, nil
- }
- idx, ok = cp.curPool.refs[blk]
- if ok {
- br := cp.curPool.blocks[idx]
- return br.key, br.value, nil
- }
- if blk.Offset >= absolutePrimaryPos(cp.recPos, cp.recFileNum, cp.maxFileSize) {
- return nil, nil, fmt.Errorf("error getting cached multihashed primary: %w", types.ErrOutOfBounds)
- }
- return nil, nil, nil
-}
-
-func (cp *SigToEpochPrimary) Get(blk types.Block) ([]byte, []byte, error) {
- key, value, err := cp.getCached(blk)
- if err != nil {
- return nil, nil, err
- }
- if key != nil && value != nil {
- return key, value, nil
- }
-
- localPos, fileNum := localizePrimaryPos(blk.Offset, cp.maxFileSize)
-
- file, err := cp.fileCache.Open(primaryFileName(cp.basePath, fileNum))
- if err != nil {
- return nil, nil, err
- }
- defer cp.fileCache.Close(file)
-
- read := make([]byte, int(blk.Size))
- if _, err = file.ReadAt(read, int64(localPos)); err != nil {
- return nil, nil, fmt.Errorf("error reading data from gsfa primary: %w", err)
- }
-
- return readNode(read)
-}
-
-type Signature []byte
-
-// readNode extracts the signature from the data read and splits key and value.
-func readNode(data []byte) (Signature, []byte, error) {
- c, n, err := readSignature(data)
- if err != nil {
- return Signature{}, nil, err
- }
-
- return c, data[n:], nil
-}
-
-func readSignature(buf []byte) (Signature, int, error) {
- // the signature is the first 64 bytes
- if len(buf) < 64 {
- return Signature{}, 0, fmt.Errorf("error reading signature from primary: expected at least 64 bytes, got %d", len(buf))
- }
- sig := buf[:64]
- return sig, 64, nil
-}
-
-func _clone(b []byte) []byte {
- if b == nil {
- return nil
- }
- return append(b[:0:0], b...)
-}
-
-// Put adds a new pending blockRecord to the pool and returns a Block that
-// contains the location that the block will occupy in the primary. The
-// returned primary location must be an absolute position across all primary
-// files.
-func (cp *SigToEpochPrimary) Put(key []byte, value []byte) (types.Block, error) {
- recSize := int64(len(key) + len(value))
- dataSize := primaryRecordSize
- if recSize != int64(dataSize) {
- return types.Block{}, fmt.Errorf("expected record size %d, got %d", dataSize, recSize)
- }
-
- cp.poolLk.Lock()
- defer cp.poolLk.Unlock()
-
- if cp.recPos >= types.Position(cp.maxFileSize) {
- cp.recFileNum++
- cp.recPos = 0
- }
-
- // Tell index the location that this record will be writtten.
- absRecPos := absolutePrimaryPos(cp.recPos, cp.recFileNum, cp.maxFileSize)
- blk := types.Block{Offset: absRecPos, Size: types.Size(recSize)}
-
- cp.recPos += types.Position(dataSize)
-
- cp.nextPool.refs[blk] = len(cp.nextPool.blocks)
- cp.nextPool.blocks = append(cp.nextPool.blocks, blockRecord{_clone(key), _clone(value)})
- cp.outstandingWork += types.Work(dataSize)
- return blk, nil
-}
-
-func (cp *SigToEpochPrimary) Overwrite(blk types.Block, key []byte, value []byte) error {
- recSize := int64(len(key) + len(value))
-
- if recSize != int64(blk.Size) {
- return fmt.Errorf("expected record size %d, got %d", blk.Size, recSize)
- }
- cp.poolLk.Lock()
- defer cp.poolLk.Unlock()
-
- localPos, fileNum := localizePrimaryPos(blk.Offset, cp.maxFileSize)
-
- fi, err := os.OpenFile(primaryFileName(cp.basePath, fileNum), os.O_WRONLY, 0o666)
- if err != nil {
- return err
- }
- defer fi.Close()
- payload := append(key, value...)
-
- // overwrite the record
- if _, err = fi.WriteAt(payload, int64(localPos)); err != nil {
- return fmt.Errorf("error writing data to gsfa primary: %w", err)
- }
- cp.upgradeCachedValue(blk, _clone(key), _clone(value))
- return nil
-}
-
-func (cp *SigToEpochPrimary) flushBlock(key []byte, value []byte) (types.Work, error) {
- if cp.length >= types.Position(cp.maxFileSize) {
- fileNum := cp.fileNum + 1
- primaryPath := primaryFileName(cp.basePath, fileNum)
- // If the primary file being opened already exists then fileNum has
- // wrapped and there are max uint32 of index files. This means that
- // maxFileSize is set far too small or GC is disabled.
- if _, err := os.Stat(primaryPath); !os.IsNotExist(err) {
- return 0, fmt.Errorf("creating primary file overwrites existing, check file size, gc and path (maxFileSize=%d) (path=%s)", cp.maxFileSize, primaryPath)
- }
-
- file, err := os.OpenFile(primaryPath, os.O_RDWR|os.O_APPEND|os.O_CREATE, 0o644)
- if err != nil {
- return 0, fmt.Errorf("cannot open new primary file %s: %w", primaryPath, err)
- }
- if err = cp.writer.Flush(); err != nil {
- return 0, fmt.Errorf("cannot write to primary file %s: %w", cp.file.Name(), err)
- }
-
- cp.file.Close()
- cp.writer.Reset(file)
- cp.file = file
- cp.fileNum = fileNum
- cp.length = 0
- }
-
- size := len(key) + len(value)
- if _, err := cp.writer.Write(append(key, value...)); err != nil {
- return 0, err
- }
-
- writeSize := size
- cp.length += types.Position(writeSize)
-
- return types.Work(writeSize), nil
-}
-
-func (cp *SigToEpochPrimary) IndexKey(key []byte) ([]byte, error) {
- if len(key) != 64 {
- return nil, fmt.Errorf("invalid key length: %d", len(key))
- }
- // This is a sanity-check to see if it really is a solana signature.
- decoded := solana.SignatureFromBytes(key)
- return decoded[:], nil
-}
-
-func (cp *SigToEpochPrimary) GetIndexKey(blk types.Block) ([]byte, error) {
- key, _, err := cp.Get(blk)
- if err != nil {
- return nil, err
- }
- if key == nil {
- return nil, nil
- }
- return cp.IndexKey(key)
-}
-
-// Flush writes outstanding work and buffered data to the primary file.
-func (cp *SigToEpochPrimary) Flush() (types.Work, error) {
- // Only one Flush at a time, otherwise the 2nd Flush can swap the pools
- // while the 1st Flush is still reading the pool being flushed. That could
- // cause the pool being read by the 1st Flush to be written to
- // concurrently.
- cp.flushLock.Lock()
- defer cp.flushLock.Unlock()
-
- cp.poolLk.Lock()
- // If no new data, then nothing to do.
- if len(cp.nextPool.blocks) == 0 {
- cp.poolLk.Unlock()
- return 0, nil
- }
- cp.curPool = cp.nextPool
- cp.nextPool = newBlockPool()
- cp.outstandingWork = 0
- cp.poolLk.Unlock()
-
- // The pool lock is released allowing Put to write to nextPool. The
- // flushLock is still held, preventing concurrent flushes from changing the
- // pools or accessing writer.
-
- var work types.Work
- for _, record := range cp.curPool.blocks {
- blockWork, err := cp.flushBlock(record.key, record.value)
- if err != nil {
- return 0, err
- }
- work += blockWork
- }
- err := cp.writer.Flush()
- if err != nil {
- return 0, fmt.Errorf("cannot flush data to primary file %s: %w", cp.file.Name(), err)
- }
-
- return work, nil
-}
-
-// Sync commits the contents of the primary file to disk. Flush should be
-// called before calling Sync.
-func (mp *SigToEpochPrimary) Sync() error {
- mp.flushLock.Lock()
- defer mp.flushLock.Unlock()
- return mp.file.Sync()
-}
-
-// Close calls Flush to write work and data to the primary file, and then
-// closes the file.
-func (mp *SigToEpochPrimary) Close() error {
- if mp.closed {
- return nil
- }
-
- mp.fileCache.Clear()
-
- _, err := mp.Flush()
- if err != nil {
- mp.file.Close()
- return err
- }
-
- return mp.file.Close()
-}
-
-func (cp *SigToEpochPrimary) OutstandingWork() types.Work {
- cp.poolLk.RLock()
- defer cp.poolLk.RUnlock()
- return cp.outstandingWork
-}
-
-type Iterator struct {
- // The index data we are iterating over
- file *os.File
- // The current position within the index
- pos int64
- // The base index file path
- base string
- // The current index file number
- fileNum uint32
-}
-
-func (cp *SigToEpochPrimary) Iter() (primary.PrimaryStorageIter, error) {
- header, err := readHeader(cp.headerPath)
- if err != nil {
- if os.IsNotExist(err) {
- return nil, nil
- }
- return nil, err
- }
-
- return NewIterator(cp.basePath, header.FirstFile), nil
-}
-
-func NewIterator(basePath string, fileNum uint32) *Iterator {
- return &Iterator{
- base: basePath,
- fileNum: fileNum,
- }
-}
-
-func (iter *Iterator) Next() ([]byte, []byte, error) {
- if iter == nil {
- return nil, nil, nil
- }
-
- if iter.file == nil {
- file, err := os.OpenFile(primaryFileName(iter.base, iter.fileNum), os.O_RDONLY, 0o644)
- if err != nil {
- if os.IsNotExist(err) {
- return nil, nil, io.EOF
- }
- return nil, nil, err
- }
- iter.file = file
- iter.pos = 0
- }
-
- size := primaryRecordSize
- pos := iter.pos
- data := make([]byte, size)
- _, err := iter.file.ReadAt(data, pos)
- if err != nil {
- iter.file.Close()
- // if err == io.EOF {
- // err = io.ErrUnexpectedEOF
- // }
- return nil, nil, err
- }
-
- iter.pos += int64(size)
- return readNode(data)
-}
-
-func (iter *Iterator) Close() error {
- if iter.file == nil {
- return nil
- }
- return iter.file.Close()
-}
-
-// StorageSize returns bytes of storage used by the primary files.
-func (cp *SigToEpochPrimary) StorageSize() (int64, error) {
- header, err := readHeader(cp.headerPath)
- if err != nil {
- if os.IsNotExist(err) {
- return 0, nil
- }
- return 0, err
- }
- fi, err := os.Stat(cp.headerPath)
- if err != nil {
- return 0, err
- }
- size := fi.Size()
-
- fileNum := header.FirstFile
- for {
- primaryName := primaryFileName(cp.basePath, fileNum)
-
- // Get size of primary file.
- fi, err = os.Stat(primaryName)
- if err != nil {
- if os.IsNotExist(err) {
- break
- }
- return 0, err
- }
- size += fi.Size()
-
- fileNum++
- }
- return size, nil
-}
-
-func primaryFileName(basePath string, fileNum uint32) string {
- return fmt.Sprintf("%s.%d", basePath, fileNum)
-}
-
-func primaryPosToFileNum(pos types.Position, maxFileSize uint32) (bool, uint32) {
- // Primary pos 0 means there is no data in the primary, so indicate empty.
- if pos == 0 {
- return false, 0
- }
- // The start of the entry determines which is file is used.
- return true, uint32(pos / types.Position(maxFileSize))
-}
-
-// localizePrimaryPos decodes a position into a local primary offset and file number.
-func localizePrimaryPos(pos types.Position, maxFileSize uint32) (types.Position, uint32) {
- ok, fileNum := primaryPosToFileNum(pos, maxFileSize)
- if !ok {
- // Return 0 local pos to indicate empty bucket.
- return 0, 0
- }
- // Subtract file offset to get pos within its local file.
- localPos := pos - (types.Position(fileNum) * types.Position(maxFileSize))
- return localPos, fileNum
-}
-
-func absolutePrimaryPos(localPos types.Position, fileNum, maxFileSize uint32) types.Position {
- return types.Position(maxFileSize)*types.Position(fileNum) + localPos
-}
-
-func findLastPrimary(basePath string, fileNum uint32) (uint32, error) {
- var lastFound uint32
- for {
- _, err := os.Stat(primaryFileName(basePath, fileNum))
- if err != nil {
- if os.IsNotExist(err) {
- break
- }
- return 0, err
- }
- lastFound = fileNum
- fileNum++
- }
- return lastFound, nil
-}
-
-var _ primary.PrimaryStorage = &SigToEpochPrimary{}
diff --git a/store/primary/sig2epochprimary/sig2epochprimary_test.go b/store/primary/sig2epochprimary/sig2epochprimary_test.go
deleted file mode 100644
index 5aed8fa8..00000000
--- a/store/primary/sig2epochprimary/sig2epochprimary_test.go
+++ /dev/null
@@ -1,198 +0,0 @@
-package sig2epochprimary_test
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "io"
- "path/filepath"
- "testing"
-
- "github.com/davecgh/go-spew/spew"
- "github.com/gagliardetto/solana-go"
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/rpcpool/yellowstone-faithful/store/primary/sig2epochprimary"
- "github.com/rpcpool/yellowstone-faithful/store/testutil"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-// This test is about making sure that inserts into an empty bucket result in a key that is trimmed
-// to a single byte.
-
-func TestIndexPut(t *testing.T) {
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := sig2epochprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
-
- blks := testutil.GenerateEpochs(5)
- expectedOffset := types.Position(0)
- for _, blk := range blks {
- expectedSize := len(blk.Key[:]) + len(blk.Value)
- loc, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- require.Equal(t, expectedOffset, loc.Offset)
- require.Equal(t, types.Size(expectedSize), loc.Size)
- expectedOffset += types.Position(expectedSize)
- }
-
- outstandingWork := primaryStorage.OutstandingWork()
- require.Equal(t, types.Work(expectedOffset), outstandingWork)
- work, err := primaryStorage.Flush()
- require.NoError(t, err)
- require.Equal(t, types.Work(expectedOffset), work)
- err = primaryStorage.Sync()
- require.NoError(t, err)
-
- iter := sig2epochprimary.NewIterator(primaryPath, 0)
- t.Cleanup(func() { iter.Close() })
-
- gotBlocks := make([]testutil.Epoch, 0, len(blks))
- for range blks {
- key, value, err := iter.Next()
- require.NoError(t, err)
- blk := testutil.Epoch{Key: solana.SignatureFromBytes(key), Value: value}
- require.NoError(t, err)
- gotBlocks = append(gotBlocks, blk)
- }
- require.Equal(t, blks, gotBlocks)
- _, _, err = iter.Next()
- require.EqualError(t, err, io.EOF.Error())
-
- err = primaryStorage.Close()
- require.NoError(t, err)
-}
-
-func TestIndexGetEmptyIndex(t *testing.T) {
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := sig2epochprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
- defer primaryStorage.Close()
-
- key, value, err := primaryStorage.Get(types.Block{
- Offset: 0,
- Size: 50,
- })
- require.Nil(t, key)
- require.Nil(t, value)
- require.Error(t, err)
-}
-
-func TestIndexGet(t *testing.T) {
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := sig2epochprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
-
- // load blocks
- blks := testutil.GenerateEpochs(5)
- var locs []types.Block
- for _, blk := range blks {
- loc, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- locs = append(locs, loc)
- }
-
- // should fetch from memory before flush
- spew.Dump(blks)
- for i, loc := range locs {
- expectedBlk := blks[i]
- key, value, err := primaryStorage.Get(loc)
- require.NoError(t, err)
- blk := testutil.Epoch{Key: solana.SignatureFromBytes(key), Value: value}
- require.NoError(t, err)
- spew.Dump(i, expectedBlk, blk)
- require.True(t, expectedBlk.Key.Equals(blk.Key))
- require.Equal(t, expectedBlk.Value, blk.Value)
- }
-
- // should fetch from disk after flush
- _, err = primaryStorage.Flush()
- require.NoError(t, err)
- err = primaryStorage.Sync()
- require.NoError(t, err)
-
- for i, loc := range locs {
- expectedBlk := blks[i]
- key, value, err := primaryStorage.Get(loc)
- require.NoError(t, err)
- blk := testutil.Epoch{Key: solana.SignatureFromBytes(key), Value: value}
- require.NoError(t, err)
- require.True(t, expectedBlk.Key.Equals(blk.Key))
- require.Equal(t, expectedBlk.Value, blk.Value)
- }
-
- err = primaryStorage.Close()
- require.NoError(t, err)
-}
-
-func TestFlushRace(t *testing.T) {
- const goroutines = 64
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := sig2epochprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
-
- // load blocks
- blks := testutil.GenerateEpochs(5)
- for _, blk := range blks {
- _, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- }
-
- start := make(chan struct{})
- errs := make(chan error)
- for n := 0; n < goroutines; n++ {
- go func() {
- <-start
- _, err := primaryStorage.Flush()
- errs <- err
- }()
- }
- close(start)
- for n := 0; n < goroutines; n++ {
- err := <-errs
- require.NoError(t, err)
- }
-
- require.NoError(t, primaryStorage.Close())
-}
-
-func TestFlushExcess(t *testing.T) {
- tempDir := t.TempDir()
- primaryPath := filepath.Join(tempDir, "storethehash.primary")
- primaryStorage, err := sig2epochprimary.Open(primaryPath, nil, filecache.New(1), 0)
- require.NoError(t, err)
-
- // load blocks
- blks := testutil.GenerateEpochs(5)
- for _, blk := range blks {
- _, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- }
-
- work, err := primaryStorage.Flush()
- require.NoError(t, err)
- require.NotZero(t, work)
-
- blks = testutil.GenerateEpochs(5)
- for _, blk := range blks {
- _, err := primaryStorage.Put(blk.Key[:], blk.Value)
- require.NoError(t, err)
- }
-
- work, err = primaryStorage.Flush()
- require.NoError(t, err)
- require.NotZero(t, work)
-
- // Another flush with no new data should not do work.
- work, err = primaryStorage.Flush()
- require.NoError(t, err)
- require.Zero(t, work)
-
- require.NoError(t, primaryStorage.Close())
-}
diff --git a/store/primary/sig2epochprimary/upgrade.go b/store/primary/sig2epochprimary/upgrade.go
deleted file mode 100644
index a0f7750f..00000000
--- a/store/primary/sig2epochprimary/upgrade.go
+++ /dev/null
@@ -1,326 +0,0 @@
-package sig2epochprimary
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bufio"
- "context"
- "encoding/binary"
- "fmt"
- "io"
- "os"
- "path/filepath"
-
- "github.com/rpcpool/yellowstone-faithful/store/freelist"
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-type IndexRemapper struct {
- firstFile uint32
- maxFileSize uint32
- sizes []int64
-}
-
-func (mp *SigToEpochPrimary) NewIndexRemapper() (*IndexRemapper, error) {
- header, err := readHeader(mp.headerPath)
- if err != nil {
- return nil, err
- }
-
- var sizes []int64
- for fileNum := header.FirstFile; fileNum <= mp.fileNum; fileNum++ {
- fi, err := os.Stat(primaryFileName(mp.basePath, fileNum))
- if err != nil {
- if os.IsNotExist(err) {
- break
- }
- return nil, err
- }
- sizes = append(sizes, fi.Size())
- }
-
- // If there are no primary files, or the only primary file is small enough
- // that no remapping is needed, return a nil remapper.
- if len(sizes) == 0 || (len(sizes) == 1 && sizes[0] < int64(mp.maxFileSize)) {
- return nil, nil
- }
-
- return &IndexRemapper{
- firstFile: header.FirstFile,
- maxFileSize: mp.maxFileSize,
- sizes: sizes,
- }, nil
-}
-
-func (ir *IndexRemapper) RemapOffset(pos types.Position) (types.Position, error) {
- fileNum := ir.firstFile
- newPos := int64(pos)
- for _, size := range ir.sizes {
- if newPos < size {
- return absolutePrimaryPos(types.Position(newPos), fileNum, ir.maxFileSize), nil
- }
- newPos -= size
- fileNum++
- }
- return 0, fmt.Errorf("cannot convert out-of-range primary position: %d", pos)
-}
-
-func (ir *IndexRemapper) FileSize() uint32 {
- return ir.maxFileSize
-}
-
-func upgradePrimary(ctx context.Context, filePath, headerPath string, maxFileSize uint32, freeList *freelist.FreeList) (uint32, error) {
- // If header already exists, or old primary does not exist, then no upgrade.
- _, err := os.Stat(headerPath)
- if !os.IsNotExist(err) {
- // Header already exists, do nothing.
- return 0, nil
- }
- if _, err = os.Stat(filePath); err != nil {
- if os.IsNotExist(err) {
- // No primary to upgrade.
- return 0, nil
- }
- return 0, err
- }
-
- if ctx.Err() != nil {
- return 0, ctx.Err()
- }
-
- log.Infow("Upgrading primary storage and splitting into separate files", "newVersion", PrimaryVersion, "fileSize", maxFileSize)
- if freeList != nil {
- // Instead of remapping all the primary offsets in the freelist, call
- // the garbage collector function to process the freelist and make the
- // primary records deleted. This is safer because it can be re-applied
- // if there is a failure during this phase.
- err := applyFreeList(ctx, freeList, filePath)
- if err != nil {
- return 0, fmt.Errorf("could not apply freelist to primary: %w", err)
- }
- }
-
- fileNum, err := chunkOldPrimary(ctx, filePath, int64(maxFileSize))
- if err != nil {
- return 0, fmt.Errorf("error chunking primary: %w", err)
- }
-
- if err = writeHeader(headerPath, newHeader(maxFileSize)); err != nil {
- return 0, fmt.Errorf("error writing primary info file: %w", err)
- }
-
- if err = os.Remove(filePath); err != nil {
- return 0, fmt.Errorf("cannot remove old primary: %w", err)
- }
-
- log.Infow("Replaced old primary with multiple files", "replaced", filePath, "files", fileNum+1)
- log.Infof("Upgraded primary from version 0 to %d", PrimaryVersion)
- return fileNum, nil
-}
-
-func chunkOldPrimary(ctx context.Context, name string, fileSizeLimit int64) (uint32, error) {
- file, err := os.Open(name)
- if err != nil {
- return 0, err
- }
- defer file.Close()
-
- fi, err := file.Stat()
- if err != nil {
- return 0, err
- }
- if fi.Size() == 0 {
- return 0, nil
- }
-
- total := fi.Size()
- var fileNum uint32
- outName := primaryFileName(name, fileNum)
- outFile, err := createFileAppend(outName)
- if err != nil {
- return 0, err
- }
- log.Infow("Upgrade created primary file", "file", filepath.Base(outName))
- writer := bufio.NewWriterSize(outFile, blockBufferSize)
-
- sizeBuf := make([]byte, sizePrefixSize)
- var written int64
- var count int
- var pos int64
- scratch := make([]byte, 1024)
-
- for {
- _, err = file.ReadAt(sizeBuf, pos)
- if err != nil {
- if err != io.EOF {
- log.Errorw("Error reading primary", "err", err)
- }
- break
- }
- size := binary.LittleEndian.Uint32(sizeBuf)
- if _, err = writer.Write(sizeBuf); err != nil {
- outFile.Close()
- return 0, err
- }
- pos += sizePrefixSize
-
- del := false
- if size&deletedBit != 0 {
- size ^= deletedBit
- del = true
- }
-
- if int(size) > len(scratch) {
- scratch = make([]byte, size)
- }
- data := scratch[:size]
-
- if !del {
- if _, err = file.ReadAt(data, pos); err != nil {
- log.Errorw("Error reading primary", "err", err)
- break
- }
- }
- _, err := writer.Write(data)
- if err != nil {
- outFile.Close()
- return 0, err
- }
- pos += int64(size)
-
- written += sizePrefixSize + int64(size)
- if written >= fileSizeLimit {
- if err = writer.Flush(); err != nil {
- return 0, err
- }
- outFile.Close()
- if ctx.Err() != nil {
- return 0, ctx.Err()
- }
- fileNum++
- outName = primaryFileName(name, fileNum)
- outFile, err = createFileAppend(outName)
- if err != nil {
- return 0, err
- }
- log.Infof("Upgrade created primary file %s: %.1f%% done", filepath.Base(outName), float64(1000*pos/total)/10)
- writer.Reset(outFile)
- written = 0
- }
- count++
- }
- if written != 0 {
- if err = writer.Flush(); err != nil {
- return 0, err
- }
- }
- outFile.Close()
- return fileNum, nil
-}
-
-func createFileAppend(name string) (*os.File, error) {
- return os.OpenFile(name, os.O_WRONLY|os.O_APPEND|os.O_CREATE|os.O_TRUNC, 0o644)
-}
-
-// applyFreeList reads the freelist and marks the locations in the old primary file
-// as dead by setting the deleted bit in the record size field.
-func applyFreeList(ctx context.Context, freeList *freelist.FreeList, filePath string) error {
- flPath, err := freeList.ToGC()
- if err != nil {
- return fmt.Errorf("cannot get freelist gc file: %w", err)
- }
-
- fi, err := os.Stat(flPath)
- if err != nil {
- return fmt.Errorf("cannot stat freelist gc file: %w", err)
- }
- flSize := fi.Size()
-
- // If the freelist size is non-zero, then process its records.
- var count int
- if flSize != 0 {
- log.Infof("Applying freelist to primary storage")
-
- flFile, err := os.OpenFile(flPath, os.O_RDONLY, 0o644)
- if err != nil {
- return fmt.Errorf("error opening freelist gc file: %w", err)
- }
- defer flFile.Close()
-
- primaryFile, err := os.OpenFile(filePath, os.O_RDWR, 0o644)
- if err != nil {
- return fmt.Errorf("cannot open primary file %s: %w", filePath, err)
- }
- defer primaryFile.Close()
-
- fi, err = primaryFile.Stat()
- if err != nil {
- return fmt.Errorf("cannot stat primary file %s: %w", primaryFile.Name(), err)
- }
- primarySize := fi.Size()
-
- total := int(flSize / (types.OffBytesLen + types.SizeBytesLen))
- flIter := freelist.NewIterator(bufio.NewReader(flFile))
- sizeBuf := make([]byte, sizePrefixSize)
- percentIncr := 1
- nextPercent := percentIncr
-
- for {
- free, err := flIter.Next()
- if err != nil {
- // Done reading freelist; log if error.
- if err != io.EOF {
- log.Errorw("Error reading freelist", "err", err)
- }
- break
- }
-
- offset := int64(free.Offset)
-
- if offset > primarySize {
- log.Errorw("freelist record has out-of-range primary offset", "offset", offset, "fileSize", primarySize)
- continue // skip bad freelist entry
- }
-
- if _, err = primaryFile.ReadAt(sizeBuf, offset); err != nil {
- return err
- }
- recSize := binary.LittleEndian.Uint32(sizeBuf)
- if recSize&deletedBit != 0 {
- // Already deleted.
- continue
- }
- if recSize != uint32(free.Size) {
- log.Errorw("Record size in primary does not match size in freelist", "primaryRecordSize", recSize, "freelistRecordSize", free.Size, "file", flFile.Name(), "offset", offset)
- }
-
- // Mark the record as deleted by setting the highest bit in the
- // size. This assumes that the record size is < 2^31.
- binary.LittleEndian.PutUint32(sizeBuf, recSize|deletedBit)
- _, err = primaryFile.WriteAt(sizeBuf, int64(offset))
- if err != nil {
- return fmt.Errorf("cannot write to primary file %s: %w", flFile.Name(), err)
- }
-
- count++
-
- // Log at every percent increment.
- percent := 100 * count / total
- if percent >= nextPercent {
- log.Infof("Processed %d of %d freelist records: %d%% done", count, total, percent)
- nextPercent += percentIncr
- }
- }
- log.Infow("Marked primary records from freelist as deleted", "count", count)
- flFile.Close()
- }
-
- if err = os.Remove(flPath); err != nil {
- return fmt.Errorf("error removing freelist: %w", err)
- }
-
- return nil
-}
diff --git a/store/primary/sig2epochprimary/upgrade_test.go b/store/primary/sig2epochprimary/upgrade_test.go
deleted file mode 100644
index ad33b55c..00000000
--- a/store/primary/sig2epochprimary/upgrade_test.go
+++ /dev/null
@@ -1,175 +0,0 @@
-package sig2epochprimary
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bufio"
- "context"
- "encoding/binary"
- "errors"
- "io"
- "os"
- "path/filepath"
- "testing"
-
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/rpcpool/yellowstone-faithful/store/freelist"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-const testPrimaryPath = "valuestore_test/storethehash.data"
-
-// testFileSizeLimt is the maximum size for new primary files. Using a small
-// file size for testing so that the test primary gets split into multiple files.
-const testFileSizeLimit = 1024
-
-func TestUpgradePrimary(t *testing.T) {
- t.Skip("Skipping upgrade test because there upgrade is not supported yet.")
- oldFile, err := os.OpenFile(testPrimaryPath, os.O_RDONLY, 0o644)
- require.NoError(t, err)
- defer oldFile.Close()
-
- // Scan the old file.
- t.Log("Scanning old primary")
- oldRecs, err := testScanPrimaryFile(oldFile)
- require.NoError(t, err)
-
- // Return to beginning of old file.
- _, err = oldFile.Seek(0, io.SeekStart)
- require.NoError(t, err)
-
- newPrimaryPath := filepath.Join(t.TempDir(), "storethehash.data")
-
- // Copy test file to new location.
- err = copyFile(testPrimaryPath, newPrimaryPath)
- require.NoError(t, err)
-
- newFreeListPath := filepath.Join(t.TempDir(), "storethehash.index.free")
- freeList, err := freelist.Open(newFreeListPath)
- require.NoError(t, err)
- defer freeList.Close()
-
- // Do the upgrade to split the primary into multiple files.
- headerPath := newPrimaryPath + ".info"
- updated, err := upgradePrimary(context.Background(), newPrimaryPath, headerPath, testFileSizeLimit, freeList)
- require.NoError(t, err)
- require.NotZero(t, updated)
-
- lastChunkNum, err := findLastPrimary(newPrimaryPath, 0)
- require.NoError(t, err)
-
- t.Logf("Split old primary into %d files", lastChunkNum)
- require.Equal(t, int(lastChunkNum), 198)
-
- // Make sure original file was removed.
- _, err = os.Stat(newPrimaryPath)
- require.True(t, os.IsNotExist(err))
-
- var newRecs [][]byte
- var fileNum, lastFileNum uint32
- for {
- fileName := primaryFileName(newPrimaryPath, fileNum)
- newFile, err := os.OpenFile(fileName, os.O_RDONLY, 0o644)
- if os.IsNotExist(err) {
- break
- }
- require.NoError(t, err)
-
- _, err = newFile.Stat()
- require.NoError(t, err)
-
- recs, err := testScanPrimaryFile(newFile)
- newFile.Close()
- require.NoError(t, err)
-
- newRecs = append(newRecs, recs...)
-
- lastFileNum = fileNum
- fileNum++
- }
- require.Equal(t, lastFileNum, lastChunkNum)
-
- t.Log("Compare old to new records")
- require.Equal(t, len(oldRecs), len(newRecs))
- for i := 0; i < len(oldRecs); i++ {
- require.Equal(t, len(oldRecs[i]), len(newRecs[i]))
- require.Equal(t, oldRecs[i], newRecs[i])
- }
-
- // Check that header was created
- header, err := readHeader(headerPath)
- require.NoError(t, err)
- require.Equal(t, header.Version, 1)
- require.Equal(t, header.MaxFileSize, uint32(testFileSizeLimit))
- require.Equal(t, header.FirstFile, uint32(0))
-
- fc := filecache.New(16)
- _, err = Open(newPrimaryPath, nil, fc, 0)
- require.Equal(t, err, types.ErrPrimaryWrongFileSize{testFileSizeLimit, defaultMaxFileSize})
-
- mp, err := Open(newPrimaryPath, nil, fc, testFileSizeLimit)
- require.NoError(t, err)
- require.NoError(t, mp.Close())
-
- // Run upgrade again to make sure it does nothing.
- updated, err = upgradePrimary(context.Background(), newPrimaryPath, headerPath, testFileSizeLimit, freeList)
- require.NoError(t, err)
- require.Zero(t, updated)
-}
-
-func testScanPrimaryFile(file *os.File) ([][]byte, error) {
- var recs [][]byte
-
- buffered := bufio.NewReader(file)
- sizeBuffer := make([]byte, sizePrefixSize)
- scratch := make([]byte, 256)
- for {
- _, err := io.ReadFull(buffered, sizeBuffer)
- if err != nil {
- if errors.Is(err, io.EOF) {
- break
- }
- return nil, err
- }
- size := binary.LittleEndian.Uint32(sizeBuffer)
-
- if int(size) > len(scratch) {
- scratch = make([]byte, size)
- }
- data := scratch[:size]
- _, err = io.ReadFull(buffered, data)
- if err != nil {
- if errors.Is(err, io.EOF) {
- return nil, errors.New("unexpected EOF")
- }
- return nil, err
- }
-
- rec := make([]byte, len(sizeBuffer)+len(data))
- copy(rec, sizeBuffer)
- copy(rec[len(sizeBuffer):], data)
- recs = append(recs, rec)
- }
- return recs, nil
-}
-
-func copyFile(src, dst string) error {
- fin, err := os.Open(src)
- if err != nil {
- return err
- }
- defer fin.Close()
-
- fout, err := os.Create(dst)
- if err != nil {
- return err
- }
- defer fout.Close()
-
- _, err = io.Copy(fout, fin)
- return err
-}
diff --git a/store/primary/sig2epochprimary/valuestore_test/storethehash.data b/store/primary/sig2epochprimary/valuestore_test/storethehash.data
deleted file mode 100644
index 4c2d77c6..00000000
Binary files a/store/primary/sig2epochprimary/valuestore_test/storethehash.data and /dev/null differ
diff --git a/store/store.go b/store/store.go
deleted file mode 100644
index 15afe49c..00000000
--- a/store/store.go
+++ /dev/null
@@ -1,767 +0,0 @@
-package store
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "bytes"
- "context"
- "errors"
- "fmt"
- "math"
- "os"
- "path/filepath"
- "sync"
- "time"
-
- logging "github.com/ipfs/go-log/v2"
- "github.com/rpcpool/yellowstone-faithful/store/filecache"
- "github.com/rpcpool/yellowstone-faithful/store/freelist"
- "github.com/rpcpool/yellowstone-faithful/store/index"
- "github.com/rpcpool/yellowstone-faithful/store/primary"
- "github.com/rpcpool/yellowstone-faithful/store/primary/gsfaprimary"
- "github.com/rpcpool/yellowstone-faithful/store/primary/sig2epochprimary"
- "github.com/rpcpool/yellowstone-faithful/store/types"
-)
-
-var log = logging.Logger("storethehash")
-
-const (
- // Primary types
- GsfaPrimary = "gsfaprimary"
- SigToEpochPrimary = "sig2epochprimary"
-)
-
-type Store struct {
- index *index.Index
- fileCache *filecache.FileCache
- freelist *freelist.FreeList
-
- stateLk sync.RWMutex
- open bool
- running bool
- err error
-
- rateLk sync.RWMutex
- flushRate float64 // rate at which data can be flushed
- burstRate types.Work
- lastFlush time.Time
- flushNotice chan struct{}
-
- closed chan struct{}
- closing chan struct{}
- flushNow chan struct{}
- syncInterval time.Duration
- syncOnFlush bool
- immutable bool
-}
-
-// SetReturnErrorOnDuplicatePut sets whether to return an error when a duplicate key is
-// inserted.
-func (s *Store) SetReturnErrorOnDuplicatePut(yes bool) {
- s.immutable = yes
-}
-
-// OpenStore opens the index and returns a Store with the specified primary type.
-//
-// Calling Store.Close closes the primary and freelist.
-func OpenStore(ctx context.Context, primaryType string, dataPath, indexPath string, options ...Option) (*Store, error) {
- c := config{
- fileCacheSize: defaultFileCacheSize,
- indexSizeBits: defaultIndexSizeBits,
- indexFileSize: defaultIndexFileSize,
- primaryFileSize: defaultPrimaryFileSize,
- syncInterval: defaultSyncInterval,
- burstRate: defaultBurstRate,
- gcInterval: defaultGCInterval,
- gcTimeLimit: defaultGCTimeLimit,
- }
- c.apply(options)
-
- freeList, err := freelist.Open(indexPath + ".free")
- if err != nil {
- return nil, err
- }
-
- fileCache := filecache.New(c.fileCacheSize)
-
- var primary primary.PrimaryStorage
- switch primaryType {
- case GsfaPrimary:
- primary, err = gsfaprimary.Open(dataPath, freeList, fileCache, c.primaryFileSize)
- case SigToEpochPrimary:
- primary, err = sig2epochprimary.Open(dataPath, freeList, fileCache, c.primaryFileSize)
- default:
- err = fmt.Errorf("unsupported primary type: %s", primaryType)
- }
- if err != nil {
- freeList.Close()
- return nil, err
- }
-
- idx, err := index.Open(ctx, indexPath, primary, c.indexSizeBits, c.indexFileSize, c.gcInterval, c.gcTimeLimit, fileCache)
- var bitSizeError types.ErrIndexWrongBitSize
- if errors.As(err, &bitSizeError) {
- err = translateIndex(ctx, indexPath, primary, c.indexSizeBits, c.indexFileSize)
- if err != nil {
- err = fmt.Errorf("error translating index to %d bit prefix size: %w", c.indexSizeBits, err)
- } else {
- idx, err = index.Open(ctx, indexPath, primary, c.indexSizeBits, c.indexFileSize, c.gcInterval, c.gcTimeLimit, fileCache)
- }
- }
- if err != nil {
- primary.Close()
- freeList.Close()
- return nil, err
- }
-
- store := &Store{
- lastFlush: time.Now(),
- index: idx,
- fileCache: fileCache,
- freelist: freeList,
- open: true,
- running: false,
- syncInterval: c.syncInterval,
- burstRate: c.burstRate,
- closed: make(chan struct{}),
- closing: make(chan struct{}),
- flushNow: make(chan struct{}, 1),
- syncOnFlush: c.syncOnFlush,
- }
- return store, nil
-}
-
-func translateIndex(ctx context.Context, indexPath string, primary primary.PrimaryStorage, indexSizeBits uint8, indexFileSize uint32) error {
- const progressLogInterval = 5 * time.Second
-
- log.Infof("Translating index to %d bit prefix", indexSizeBits)
-
- oldFileCache := filecache.New(64)
- log.Info("Reading old index")
- oldIndex, err := index.Open(ctx, indexPath, primary, 0, indexFileSize, 0, 0, oldFileCache)
- if err != nil {
- return fmt.Errorf("cannot open old index: %w", err)
- }
- defer oldIndex.Close()
-
- indexDir := filepath.Dir(indexPath)
- indexTmp, err := os.MkdirTemp(indexDir, "new_index")
- if err != nil {
- return err
- }
- defer os.RemoveAll(indexTmp)
-
- newIndexPath := filepath.Join(indexTmp, filepath.Base(indexPath))
- newFileCache := filecache.New(64)
- newIndex, err := index.Open(ctx, newIndexPath, primary, indexSizeBits, indexFileSize, 0, 0, newFileCache)
- if err != nil {
- return fmt.Errorf("cannot open new index: %w", err)
- }
- defer newIndex.Close()
-
- iter := oldIndex.NewIterator()
- ticker := time.NewTicker(progressLogInterval)
- defer ticker.Stop()
- var count int
- for {
- select {
- case <-ticker.C:
- log.Infof("Translating index records %.1f%% done", iter.Progress())
- default:
- }
-
- rec, done, err := iter.Next()
- if err != nil {
- return fmt.Errorf("cannot get next index record: %w", err)
- }
- if done {
- break
- }
-
- indexKey, err := primary.GetIndexKey(rec.Block)
- if err != nil {
- return fmt.Errorf("cannot get old index key: %w", err)
- }
-
- if err = newIndex.Put(indexKey, rec.Block); err != nil {
- return fmt.Errorf("cannot put new index record: %w", err)
- }
-
- count++
- }
- ticker.Stop()
-
- log.Infof("Translated %d index records", count)
-
- log.Info("Replacing old index files with new")
- if err = newIndex.Close(); err != nil {
- return fmt.Errorf("error closing new index: %w", err)
- }
- if err = oldIndex.Close(); err != nil {
- return fmt.Errorf("error closing old index: %w", err)
- }
-
- // Create a temp directory for the old index files and move them there.
- oldTmp, err := os.MkdirTemp(indexDir, "old_index")
- if err != nil {
- return err
- }
- if err = index.MoveFiles(indexPath, oldTmp); err != nil {
- return fmt.Errorf("cannot move old index files: %w", err)
- }
-
- // Move the new index file from the temp directory to the index directory.
- if err = index.MoveFiles(newIndexPath, indexDir); err != nil {
- return fmt.Errorf("cannot move new index files: %w", err)
- }
-
- // Remove the old index files.
- if err = os.RemoveAll(oldTmp); err != nil {
- return fmt.Errorf("cannot remove old index files: %w", err)
- }
-
- log.Infof("Finished translating index to %d bit prefix", indexSizeBits)
- return nil
-}
-
-func (s *Store) Start() {
- s.stateLk.Lock()
- running := s.running
- s.running = true
- s.stateLk.Unlock()
- if !running {
- go s.run()
- }
-}
-
-func (s *Store) Index() *index.Index {
- return s.index
-}
-
-func (s *Store) Primary() primary.PrimaryStorage {
- return s.index.Primary
-}
-
-func (s *Store) run() {
- defer close(s.closed)
- d := time.NewTicker(s.syncInterval)
-
- for {
- select {
- case <-s.flushNow:
- if err := s.Flush(); err != nil {
- s.setErr(err)
- }
- case <-s.closing:
- d.Stop()
- select {
- case <-d.C:
- default:
- }
- return
- case <-d.C:
- select {
- case s.flushNow <- struct{}{}:
- default:
- // Already signaled by write, do not need another flush.
- }
- }
- }
-}
-
-// Close stops store goroutines and calls Close on the index, primary, and
-// freelist. This flushes any outstanding work and buffered data to their
-// files.
-func (s *Store) Close() error {
- s.stateLk.Lock()
- if !s.open {
- s.stateLk.Unlock()
- return nil
- }
- s.open = false
-
- running := s.running
- s.running = false
- s.stateLk.Unlock()
-
- if running {
- close(s.closing)
- <-s.closed
- }
-
- cerr := s.Err()
-
- err := s.index.Close()
- if err != nil {
- cerr = err
- }
- if err = s.index.Primary.Close(); err != nil {
- cerr = err
- }
- s.fileCache.Clear()
- if err = s.freelist.Close(); err != nil {
- cerr = err
- }
-
- return cerr
-}
-
-func (s *Store) Get(key []byte) ([]byte, bool, error) {
- err := s.Err()
- if err != nil {
- return nil, false, err
- }
-
- indexKey, err := s.index.Primary.IndexKey(key)
- if err != nil {
- return nil, false, err
- }
- fileOffset, found, err := s.index.Get(indexKey)
- if err != nil {
- return nil, false, err
- }
- if !found {
- return nil, false, nil
- }
-
- primaryKey, value, err := s.getPrimaryKeyData(fileOffset, indexKey)
- if err != nil {
- return nil, false, err
- }
- if primaryKey == nil {
- return nil, false, nil
- }
-
- return value, true, nil
-}
-
-func (s *Store) Err() error {
- s.stateLk.RLock()
- defer s.stateLk.RUnlock()
- return s.err
-}
-
-func (s *Store) setErr(err error) {
- s.stateLk.Lock()
- s.err = err
- s.stateLk.Unlock()
-}
-
-type ErrDuplicate struct {
- Key []byte
- StoredKey []byte
- Value []byte
- StoredValue []byte
-}
-
-func (e *ErrDuplicate) Error() string {
- return fmt.Sprintf("duplicate key: %x", e.Key)
-}
-
-// Is returns true if the error is an ErrDuplicate.
-func (e *ErrDuplicate) Is(err error) bool {
- _, ok := err.(*ErrDuplicate)
- return ok
-}
-
-func (s *Store) Put(key []byte, newValue []byte) error {
- err := s.Err()
- if err != nil {
- return err
- }
-
- // Get the key in primary storage
- indexKey, err := s.index.Primary.IndexKey(key)
- if err != nil {
- return err
- }
- // TODO: (immaterial) What if we just used the `prevOffset` (which is `types.Block`) and just added the extra field `LastOffset` to it?
- // See if the key already exists and get offset
- prevOffset, found, err := s.index.Get(indexKey)
- if err != nil {
- return err
- }
- // If found, get the key and value stored in primary to see if it is the
- // same (index only stores prefixes).
- var storedKey []byte
- var storedVal []byte
- var cmpKey bool
- if found {
- storedKey, storedVal, err = s.getPrimaryKeyData(prevOffset, indexKey)
- if err != nil {
- return err
- }
- // We need to compare to the resulting indexKey to the storedKey. Two
- // keys may point to same IndexKey (i.e. two CIDS same multihash), and
- // they need to be treated as the same key.
- if storedKey != nil {
- // if we're not accepting updates, this is the point we bail --
- // the identical key is in primary storage, we don't do update operations
-
- if s.immutable {
- return &ErrDuplicate{
- Key: key,
- StoredKey: storedKey,
- Value: newValue,
- StoredValue: storedVal,
- }
- }
- cmpKey = true
- }
- // TODO: the key-value that we got here might be from the cache of primary storage,
- // and this means that it could be outdated if another direct write happened to primary storage.
- if bytes.Equal(newValue, storedVal) {
- // Trying to put the same value in an existing key, so ok to
- // directly return.
- return nil
- }
- if storedKey != nil && bytes.Equal(indexKey, storedKey) {
- // overwrite in primary storage:
- err = s.index.Primary.Overwrite(prevOffset, key, newValue)
- if err != nil {
- return err
- }
- // TODO: remove?
- s.flushTick()
-
- return nil
- }
- }
-
- // We are ready now to start putting/updating the value in the key.
- // Put value in primary storage first. In primary storage we put
- // the key, not the indexKey. The storage knows how to manage the key
- // under the hood while the index is primary storage-agnostic.
- fileOffset, err := s.index.Primary.Put(key, newValue)
- if err != nil {
- return err
- }
-
- // If the key being set is not found, or the stored key is not equal
- // (even if same prefix is shared @index), we put the key without updates
- if !cmpKey {
- if err = s.index.Put(indexKey, fileOffset); err != nil {
- return err
- }
- } else {
- // If the key exists and the one stored is the one we are trying
- // to put this is an update.
- // if found && bytes.Compare(key, storedKey) == 0 {
- if err = s.index.Update(indexKey, fileOffset); err != nil {
- return err
- }
- // Add outdated data in primary storage to freelist
- if err = s.freelist.Put(prevOffset); err != nil {
- return err
- }
- }
-
- s.flushTick()
-
- return nil
-}
-
-func (s *Store) Remove(key []byte) (bool, error) {
- err := s.Err()
- if err != nil {
- return false, err
- }
-
- // Get the key in primary storage
- indexKey, err := s.index.Primary.IndexKey(key)
- if err != nil {
- return false, err
- }
- // See if the key already exists and get offset
- offset, found, err := s.index.Get(indexKey)
- if err != nil {
- return false, err
- }
-
- // If not found it means there's nothing to remove.
- // Return false with no error
- if !found {
- return false, nil
- }
-
- // If found, get the key and value stored in primary to see if it is the
- // same (index only stores prefixes).
- storedKey, _, err := s.getPrimaryKeyData(offset, indexKey)
- if err != nil {
- return false, err
- }
- if storedKey == nil {
- // The indexKey does not exist and there is nothing to remove.
- return false, nil
- }
-
- removed, err := s.index.Remove(storedKey)
- if err != nil {
- return false, err
- }
- if removed {
- // Mark slot in freelist
- err = s.freelist.Put(offset)
- if err != nil {
- return false, err
- }
- }
-
- s.flushTick()
- return removed, nil
-}
-
-func (s *Store) SetFileCacheSize(size int) {
- s.fileCache.SetCacheSize(size)
-}
-
-func (s *Store) getPrimaryKeyData(blk types.Block, indexKey []byte) ([]byte, []byte, error) {
- // Get the key and value stored in primary to see if it is the same (index
- // only stores prefixes).
- storedKey, storedValue, err := s.index.Primary.Get(blk)
- if err != nil {
- // Log the error reading the primary, since no error is returned if the
- // bad index is successfully deleted.
- log.Errorw("Error reading primary, removing bad index", "err", err)
- // The offset returned from the index is not usable, so delete the
- // index entry regardless of which key in indexes. It is not safe to
- // put this offset onto the free list, since it may be an invalid
- // location in the primary.
- if _, err = s.index.Remove(indexKey); err != nil {
- return nil, nil, fmt.Errorf("error removing unusable index: %w", err)
- }
- return nil, nil, nil
- }
-
- // Check that the stored key is the correct type.
- storedKey, err = s.index.Primary.IndexKey(storedKey)
- if err != nil {
- // The key read from the primary is bad. This means that the data
- // stored in the primary is bad or the index has an incorrect location.
- // Either way, the index is unusable, so log the error and delete the
- // index. It is not safe to put this offset onto the free list, since
- // it may be an invalid location in the primary.
- log.Errorw("Bad key stored in primary or bad index, removing index", "err", err)
- if _, err = s.index.Remove(indexKey); err != nil {
- return nil, nil, fmt.Errorf("error removing unusable index: %w", err)
- }
- return nil, nil, nil
- }
-
- // The index stores only prefixes, hence check if the given key fully
- // matches the key that is stored in the primary storage before returning
- // the actual value. If given key and stored key do not match, then some
- // other key that has the same prefix was stored.
- if !bytes.Equal(indexKey, storedKey) {
- return nil, nil, nil
- }
-
- return storedKey, storedValue, nil
-}
-
-func (s *Store) flushTick() {
- s.rateLk.Lock()
- flushRate := s.flushRate
- lastFlush := s.lastFlush
- s.rateLk.Unlock()
-
- if flushRate == 0 {
- // Do not know the flush rate yet.
- return
- }
-
- work := s.index.OutstandingWork() + s.index.Primary.OutstandingWork() + s.freelist.OutstandingWork()
- if work <= s.burstRate {
- // Not enough work to be concerned.
- return
- }
-
- // Calculate inbound rate of data.
- elapsed := time.Since(lastFlush)
- inRate := math.Ceil(float64(work) / elapsed.Seconds())
-
- // If the rate of incoming work exceeds the rate that work can be flushed
- // at, and there is enough work to be concerned about (work > s.burstRate),
- // then trigger an immediate flush and wait for the flush to complete. It
- // is necessary to wait for the flush, otherwise more work could continue
- // to come in and be stored in memory faster that flushes could handle it,
- // leading to memory exhaustion.
- if inRate > flushRate {
- // Get a channel that broadcasts next flush completion.
- s.rateLk.Lock()
- if s.flushNotice == nil {
- s.flushNotice = make(chan struct{})
- }
- flushNotice := s.flushNotice
- s.rateLk.Unlock()
-
- // Trigger flush now, non-blocking.
- select {
- case s.flushNow <- struct{}{}:
- log.Debugw("Work ingress rate exceeded flush rate, waiting for flush", "inRate", inRate, "flushRate", s.flushRate, "elapsed", elapsed, "work", work, "burstRate", s.burstRate)
- default:
- // Already signaled, but flush not yet started. No need to wait
- // since the existing unread signal guarantees the a flush.
- }
-
- // Wait for next flush to complete.
- <-flushNotice
- }
-}
-
-func (s *Store) commit() (types.Work, error) {
- primaryWork, err := s.index.Primary.Flush()
- if err != nil {
- return 0, err
- }
- indexWork, err := s.index.Flush()
- if err != nil {
- return 0, err
- }
- flWork, err := s.freelist.Flush()
- if err != nil {
- return 0, err
- }
- if s.syncOnFlush {
- // finalize disk writes
- if err = s.index.Primary.Sync(); err != nil {
- return 0, err
- }
- if err = s.index.Sync(); err != nil {
- return 0, err
- }
- if err = s.freelist.Sync(); err != nil {
- return 0, err
- }
- }
- return primaryWork + indexWork + flWork, nil
-}
-
-func (s *Store) outstandingWork() bool {
- return s.index.OutstandingWork()+s.index.Primary.OutstandingWork() > 0
-}
-
-// Flush writes outstanding work and buffered data to the primary, index, and
-// freelist files. It then syncs these files to permanent storage.
-func (s *Store) Flush() error {
- lastFlush := time.Now()
-
- s.rateLk.Lock()
- s.lastFlush = lastFlush
- s.rateLk.Unlock()
-
- if !s.outstandingWork() {
- return nil
- }
-
- work, err := s.commit()
- if err != nil {
- return err
- }
-
- var rate float64
- if work > types.Work(s.burstRate) {
- now := time.Now()
- elapsed := now.Sub(lastFlush)
- rate = math.Ceil(float64(work) / elapsed.Seconds())
- }
-
- s.rateLk.Lock()
- if rate != 0 {
- s.flushRate = rate
- }
- if s.flushNotice != nil {
- close(s.flushNotice)
- s.flushNotice = nil
- }
- s.rateLk.Unlock()
-
- return nil
-}
-
-func (s *Store) Has(key []byte) (bool, error) {
- err := s.Err()
- if err != nil {
- return false, err
- }
- indexKey, err := s.index.Primary.IndexKey(key)
- if err != nil {
- return false, err
- }
- blk, found, err := s.index.Get(indexKey)
- if !found || err != nil {
- return false, err
- }
-
- // The index stores only prefixes, hence check if the given key fully matches the
- // key that is stored in the primary storage before returning the actual value.
- // TODO: avoid second lookup
- primaryIndexKey, err := s.index.Primary.GetIndexKey(blk)
- if err != nil {
- return false, err
- }
-
- return bytes.Equal(indexKey, primaryIndexKey), nil
-}
-
-func (s *Store) GetSize(key []byte) (types.Size, bool, error) {
- indexKey, err := s.index.Primary.IndexKey(key)
- if err != nil {
- return 0, false, err
- }
- blk, found, err := s.index.Get(indexKey)
- if err != nil {
- return 0, false, err
- }
- if !found {
- return 0, false, nil
- }
-
- // The index stores only prefixes, hence check if the given key fully matches the
- // key that is stored in the primary storage before returning the actual value.
- // TODO: avoid second lookup
- primaryIndexKey, err := s.index.Primary.GetIndexKey(blk)
- if err != nil {
- return 0, false, err
- }
-
- if !bytes.Equal(indexKey, primaryIndexKey) {
- return 0, false, nil
- }
- return blk.Size - types.Size(len(key)), true, nil
-}
-
-// IndexStorageSize returns the storage used by the index files.
-func (s *Store) IndexStorageSize() (int64, error) {
- return s.index.StorageSize()
-}
-
-// PrimaryStorageSize returns the storage used by the primary storage files.
-func (s *Store) PrimaryStorageSize() (int64, error) {
- return s.index.Primary.StorageSize()
-}
-
-// FreelistStorageSize returns the storage used by the freelist files.
-func (s *Store) FreelistStorageSize() (int64, error) {
- return s.freelist.StorageSize()
-}
-
-// StorageSize returns the storage used by the index, primary, and freelist files.
-func (s *Store) StorageSize() (int64, error) {
- isize, err := s.index.StorageSize()
- if err != nil {
- return 0, err
- }
- psize, err := s.index.Primary.StorageSize()
- if err != nil {
- return 0, err
- }
- fsize, err := s.freelist.StorageSize()
- if err != nil {
- return 0, err
- }
- return isize + psize + fsize, nil
-}
diff --git a/store/store_test.go b/store/store_test.go
deleted file mode 100644
index 9591150b..00000000
--- a/store/store_test.go
+++ /dev/null
@@ -1,222 +0,0 @@
-package store_test
-
-import (
- "context"
- "errors"
- "io"
- "os"
- "path/filepath"
- "testing"
-
- "github.com/davecgh/go-spew/spew"
- store "github.com/rpcpool/yellowstone-faithful/store"
- "github.com/rpcpool/yellowstone-faithful/store/freelist"
- "github.com/rpcpool/yellowstone-faithful/store/testutil"
- "github.com/rpcpool/yellowstone-faithful/store/types"
- "github.com/stretchr/testify/require"
-)
-
-func initStore(t *testing.T, dir string) (*store.Store, error) {
- indexPath := filepath.Join(dir, "storethehash.index")
- dataPath := filepath.Join(dir, "storethehash.data")
- store, err := store.OpenStore(context.Background(), store.GsfaPrimary, dataPath, indexPath)
- if err != nil {
- return nil, err
- }
- t.Cleanup(func() { require.NoError(t, store.Close()) })
- return store, nil
-}
-
-func TestUpdate(t *testing.T) {
- t.Run("when not immutable", func(t *testing.T) {
- tempDir := t.TempDir()
- s, err := initStore(t, tempDir)
- require.NoError(t, err)
- blks := testutil.GenerateEntries(2)
-
- t.Logf("Putting a new block")
- err = s.Put(blks[0].Key.Bytes(), blks[0].Value)
- require.NoError(t, err)
- value, found, err := s.Get(blks[0].Key.Bytes())
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, value, blks[0].Value)
-
- {
- _, err = s.Primary().Flush()
- require.NoError(t, err)
- require.NoError(t, s.Flush())
- require.NoError(t, s.Primary().Sync())
- }
-
- t.Logf("Overwrite same key with different value")
- spew.Dump(blks)
- err = s.Put(blks[0].Key.Bytes(), blks[1].Value)
- require.NoError(t, err)
-
- {
- _, err = s.Primary().Flush()
- require.NoError(t, err)
- require.NoError(t, s.Flush())
- require.NoError(t, s.Primary().Sync())
- }
-
- value, found, err = s.Get(blks[0].Key.Bytes())
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, blks[1].Value[0:8], value[0:8], "value should be overwritten")
- require.Equal(t, blks[1].Value, value, "value should be overwritten")
- require.NotEqual(t, blks[0].Value, value, "value should be overwritten")
- {
- it, err := s.Primary().Iter()
- require.NoError(t, err)
- key, value, err := it.Next()
- require.NoError(t, err)
- require.Equal(t, blks[0].Key.Bytes(), key)
- require.Equal(t, blks[1].Value, value)
- }
-
- t.Logf("Overwrite same key with same value")
- err = s.Put(blks[0].Key.Bytes(), blks[1].Value)
- require.NoError(t, err)
- value, found, err = s.Get(blks[0].Key.Bytes())
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, value, blks[1].Value)
-
- s.Flush()
-
- storeIter := s.NewIterator()
- var count int
- for {
- key, val, err := storeIter.Next()
- if errors.Is(err, io.EOF) {
- break
- }
- require.Zero(t, count)
- require.NoError(t, err)
- require.Equal(t, blks[0].Key.Bytes(), key)
- require.Equal(t, blks[1].Value, val)
- count++
- }
- })
-}
-
-func TestRemove(t *testing.T) {
- tempDir := t.TempDir()
- s, err := initStore(t, tempDir)
- require.NoError(t, err)
- blks := testutil.GenerateEntries(2)
-
- t.Logf("Putting blocks")
- err = s.Put(blks[0].Key.Bytes(), blks[0].Value)
- require.NoError(t, err)
- err = s.Put(blks[1].Key.Bytes(), blks[1].Value)
- require.NoError(t, err)
-
- t.Logf("Removing the first block")
- removed, err := s.Remove(blks[0].Key.Bytes())
- require.NoError(t, err)
- require.True(t, removed)
-
- t.Logf("Checking if the block has been removed successfully")
- value, found, err := s.Get(blks[1].Key.Bytes())
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, value, blks[1].Value)
- _, found, err = s.Get(blks[0].Key.Bytes())
- require.NoError(t, err)
- require.False(t, found)
-
- t.Logf("Trying to remove non-existing key")
- removed, err = s.Remove(blks[0].Key.Bytes())
- require.NoError(t, err)
- require.False(t, removed)
-
- s.Flush()
-
- // Start iterator
- flPath := filepath.Join(tempDir, "storethehash.index.free")
- file, err := os.Open(flPath)
- require.NoError(t, err)
- t.Cleanup(func() { require.NoError(t, file.Close()) })
-
- iter := freelist.NewIterator(file)
- // Check freelist for the only removal. Should be the first position
- blk, err := iter.Next()
- require.Equal(t, blk.Offset, types.Position(0))
- require.NoError(t, err)
- // Check that is the last
- _, err = iter.Next()
- require.EqualError(t, err, io.EOF.Error())
-}
-
-func TestTranslate(t *testing.T) {
- tempDir := t.TempDir()
-
- indexPath := filepath.Join(tempDir, "storethehash.index")
- dataPath := filepath.Join(tempDir, "storethehash.data")
-
- t.Logf("Createing store with 16-bit index")
- s1, err := store.OpenStore(context.Background(), store.GsfaPrimary, dataPath, indexPath, store.IndexBitSize(16))
- require.NoError(t, err)
- t.Cleanup(func() { require.NoError(t, s1.Close()) })
-
- // Store blocks.
- blks := testutil.GenerateEntries(5)
- for i := range blks {
- err = s1.Put(blks[i].Key.Bytes(), blks[i].Value)
- require.NoError(t, err)
- }
- // REmove on block.
- removed, err := s1.Remove(blks[0].Key.Bytes())
- require.NoError(t, err)
- require.True(t, removed)
-
- require.NoError(t, s1.Close())
-
- // Translate to 26 bits
- t.Logf("Translating store index from 16-bit to 24-bit")
- s2, err := store.OpenStore(context.Background(), store.GsfaPrimary, dataPath, indexPath, store.IndexBitSize(24))
- require.NoError(t, err)
- t.Cleanup(func() { require.NoError(t, s2.Close()) })
-
- // Check that blocks still exist.
- for i := 1; i < len(blks); i++ {
- value, found, err := s2.Get(blks[i].Key.Bytes())
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, value, blks[i].Value)
- }
-
- // Check that removed block was not found.
- _, found, err := s2.Get(blks[0].Key.Bytes())
- require.NoError(t, err)
- require.False(t, found)
-
- require.NoError(t, s2.Close())
-
- // Translate back to 24 bits.
- t.Logf("Translating store index from 24-bit to 16-bit")
- s3, err := store.OpenStore(context.Background(), store.GsfaPrimary, dataPath, indexPath, store.IndexBitSize(16))
- require.NoError(t, err)
- t.Cleanup(func() { require.NoError(t, s3.Close()) })
-
- // Check that blocks still exist.
- for i := 1; i < len(blks); i++ {
- value, found, err := s3.Get(blks[i].Key.Bytes())
- require.NoError(t, err)
- require.True(t, found)
- require.Equal(t, value, blks[i].Value)
- }
-
- // Check that removed block was not found.
- _, found, err = s3.Get(blks[0].Key.Bytes())
- require.NoError(t, err)
- require.False(t, found)
-
- require.NoError(t, s3.Close())
-
- // Check that double close of store is ok.
- require.NoError(t, s3.Close())
-}
diff --git a/store/testutil/testutil.go b/store/testutil/testutil.go
deleted file mode 100644
index 92e2102e..00000000
--- a/store/testutil/testutil.go
+++ /dev/null
@@ -1,58 +0,0 @@
-package testutil
-
-// Copyright 2023 rpcpool
-// This file has been modified by github.com/gagliardetto
-//
-// Copyright 2020 IPLD Team and various authors and contributors
-// See LICENSE for details.
-import (
- "crypto/rand"
-
- "github.com/gagliardetto/solana-go"
-)
-
-// RandomBytes returns a byte array of the given size with random values.
-func RandomBytes(n int64) []byte {
- data := make([]byte, n)
- _, err := rand.Read(data)
- if err != nil {
- panic(err)
- }
- return data
-}
-
-type Entry struct {
- Key solana.PublicKey
- Value []byte // 8 bytes
-}
-
-func GenerateEntries(n int) []Entry {
- generatedEntries := make([]Entry, 0, n)
- for i := 0; i < n; i++ {
- key := solana.NewWallet().PublicKey()
- value := RandomBytes(8) // The value is 8 bytes long (uint64 little-endian).
- generatedEntries = append(generatedEntries, Entry{
- Key: key,
- Value: value,
- })
- }
- return generatedEntries
-}
-
-func GenerateEpochs(n int) []Epoch {
- generatedEntries := make([]Epoch, 0, n)
- for i := 0; i < n; i++ {
- key := solana.SignatureFromBytes(RandomBytes(64))
- value := RandomBytes(2) // The value is 2 bytes long (uint16 little-endian).
- generatedEntries = append(generatedEntries, Epoch{
- Key: key,
- Value: value,
- })
- }
- return generatedEntries
-}
-
-type Epoch struct {
- Key solana.Signature // 64 bytes
- Value []byte // 2 bytes
-}
diff --git a/store/types/errors.go b/store/types/errors.go
deleted file mode 100644
index a97635b2..00000000
--- a/store/types/errors.go
+++ /dev/null
@@ -1,37 +0,0 @@
-package types
-
-import "fmt"
-
-type errorType string
-
-func (e errorType) Error() string {
- return string(e)
-}
-
-// ErrOutOfBounds indicates the bucket index was greater than the number of bucks
-const ErrOutOfBounds = errorType("Buckets out of bound error")
-
-// ErrIndexTooLarge indicates the maximum supported bucket size is 32-bits
-const ErrIndexTooLarge = errorType("Index size cannot be more than 32-bits")
-
-const ErrKeyTooShort = errorType("Key must be at least 4 bytes long")
-
-const ErrKeyExists = errorType("key exists")
-
-type ErrIndexWrongBitSize [2]byte
-
-func (e ErrIndexWrongBitSize) Error() string {
- return fmt.Sprintf("Index bit size for buckets is %d, expected %d", e[0], e[1])
-}
-
-type ErrIndexWrongFileSize [2]uint32
-
-func (e ErrIndexWrongFileSize) Error() string {
- return fmt.Sprintf("Index file size limit is %d, expected %d", e[0], e[1])
-}
-
-type ErrPrimaryWrongFileSize [2]uint32
-
-func (e ErrPrimaryWrongFileSize) Error() string {
- return fmt.Sprintf("Primary file size limit is %d, expected %d", e[0], e[1])
-}
diff --git a/store/types/types.go b/store/types/types.go
deleted file mode 100644
index bc84ac4a..00000000
--- a/store/types/types.go
+++ /dev/null
@@ -1,17 +0,0 @@
-package types
-
-// Position indicates a position in a file
-type Position uint64
-
-const OffBytesLen = 8
-
-type Block struct {
- Offset Position
- Size Size
-}
-
-type Size uint32
-
-const SizeBytesLen = 4
-
-type Work uint64
diff --git a/txstatus/src/lib.rs b/txstatus/src/lib.rs
index 0cbd2819..122e4fd2 100644
--- a/txstatus/src/lib.rs
+++ b/txstatus/src/lib.rs
@@ -37,19 +37,19 @@ pub unsafe extern "C" fn parse_instruction(bytes: *const u8, len: usize) -> Resp
};
{
instruction.program_id_index = decoder.read_u8().unwrap();
- let accounts_len = decoder.read_u8().unwrap() as usize;
+ let accounts_len = decoder.read_u16().unwrap() as usize;
for _ in 0..accounts_len {
let account_index = decoder.read_u8().unwrap();
instruction.accounts.push(account_index);
}
- let data_len = decoder.read_u8().unwrap() as usize;
+ let data_len = decoder.read_u16().unwrap() as usize;
for _ in 0..data_len {
let data_byte = decoder.read_u8().unwrap();
instruction.data.push(data_byte);
}
}
- let static_account_keys_len = decoder.read_u8().unwrap() as usize;
+ let static_account_keys_len = decoder.read_u16().unwrap() as usize;
// println!(
// "[rust] static_account_keys_len: {:?}",
// static_account_keys_len
@@ -65,16 +65,38 @@ pub unsafe extern "C" fn parse_instruction(bytes: *const u8, len: usize) -> Resp
let has_dynamic_account_keys = decoder.read_option().unwrap();
let parsed_account_keys: Combined = if has_dynamic_account_keys {
let mut loaded_addresses = LoadedAddresses::default();
- let num_writable_accounts = decoder.read_u8().unwrap() as usize;
+ let num_writable_accounts = decoder.read_u16().unwrap() as usize;
// println!("[rust] num_writable_accounts: {:?}", num_writable_accounts);
// read 32 bytes for each writable account:
- for _ in 0..num_writable_accounts {
- let account_key_bytes = decoder.read_bytes(32).unwrap();
+ for dyn_wri_index in 0..num_writable_accounts {
+ let account_key_bytes = decoder.read_bytes(32);
+ if account_key_bytes.is_err() {
+ // println!("[rust] account_key_bytes error: {:?}", account_key_bytes);
+ let mut response = vec![0; 32];
+ // add error string to response:
+ let error = account_key_bytes.err().unwrap();
+ let error = format!(
+ "account_key_bytes error at index: {:?}: {:?}",
+ dyn_wri_index, error
+ );
+ response.extend_from_slice(error.as_bytes());
+ let data = response.as_mut_ptr();
+ let len = response.len();
+
+ return Response {
+ buf: Buffer {
+ data: unsafe { data.add(32) },
+ len: len - 32,
+ },
+ status: 1,
+ };
+ }
+ let account_key_bytes = account_key_bytes.unwrap();
let account_key = solana_sdk::pubkey::Pubkey::try_from(account_key_bytes)
.expect("invalid account key in writable accounts");
loaded_addresses.writable.push(account_key);
}
- let num_readonly_accounts = decoder.read_u8().unwrap() as usize;
+ let num_readonly_accounts = decoder.read_u16().unwrap() as usize;
// read 32 bytes for each readonly account:
for _ in 0..num_readonly_accounts {
let account_key_bytes = decoder.read_bytes(32).unwrap();
diff --git a/txstatus/src/reader.rs b/txstatus/src/reader.rs
index 465889b4..57cc89e0 100644
--- a/txstatus/src/reader.rs
+++ b/txstatus/src/reader.rs
@@ -126,6 +126,22 @@ impl Decoder {
Ok(out)
}
+ pub fn read_u16(&mut self) -> Result {
+ if self.remaining() < type_size::UINT16 {
+ return Err(Error::InvalidValue {
+ msg: format!(
+ "uint16 requires [{}] bytes, remaining [{}]",
+ type_size::UINT16,
+ self.remaining()
+ ),
+ });
+ }
+ let buf = self.read_bytes(type_size::UINT16)?;
+ let buf: [u8; 2] = buf.try_into().unwrap();
+ let out = u16::from_le_bytes(buf);
+ Ok(out)
+ }
+
pub fn read_u32(&mut self, order: byte_order::ByteOrder) -> Result {
if self.remaining() < type_size::UINT32 {
return Err(Error::InvalidValue {
@@ -212,6 +228,39 @@ mod tests {
assert_eq!(0, d.remaining());
}
+ #[test]
+ fn test_decoder_u16() {
+ // little endian
+ let mut buf = vec![];
+ buf.extend_from_slice(18360u16.to_le_bytes().as_ref());
+ buf.extend_from_slice(28917u16.to_le_bytes().as_ref());
+ buf.extend_from_slice(1023u16.to_le_bytes().as_ref());
+ buf.extend_from_slice(0u16.to_le_bytes().as_ref());
+ buf.extend_from_slice(33u16.to_le_bytes().as_ref());
+
+ let mut d = Decoder::new(buf);
+
+ let n = d.read_u16().unwrap();
+ assert_eq!(18360, n);
+ assert_eq!(8, d.remaining());
+
+ let n = d.read_u16().unwrap();
+ assert_eq!(28917, n);
+ assert_eq!(6, d.remaining());
+
+ let n = d.read_u16().unwrap();
+ assert_eq!(1023, n);
+ assert_eq!(4, d.remaining());
+
+ let n = d.read_u16().unwrap();
+ assert_eq!(0, n);
+ assert_eq!(2, d.remaining());
+
+ let n = d.read_u16().unwrap();
+ assert_eq!(33, n);
+ assert_eq!(0, d.remaining());
+ }
+
#[test]
fn test_decoder_byte() {
let buf = vec![0x00, 0x01];
diff --git a/txstatus/types.go b/txstatus/types.go
index 3d903d7c..43a8859c 100644
--- a/txstatus/types.go
+++ b/txstatus/types.go
@@ -48,7 +48,7 @@ func (inst Parameters) MarshalWithEncoder(encoder *bin.Encoder) error {
type CompiledInstruction struct {
ProgramIDIndex uint8
- Accounts []uint8
+ Accounts solana.Uint8SliceAsNum
Data []byte
}
@@ -61,8 +61,8 @@ func (inst CompiledInstruction) MarshalWithEncoder(encoder *bin.Encoder) error {
}
// .compiled_instruction.accounts:
{
- // len uint8
- err := encoder.WriteUint8(uint8(len(inst.Accounts)))
+ // len uint16
+ err := encoder.WriteUint16(uint16(len(inst.Accounts)), binary.LittleEndian)
if err != nil {
return fmt.Errorf("failed to write len(Accounts): %w", err)
}
@@ -74,8 +74,12 @@ func (inst CompiledInstruction) MarshalWithEncoder(encoder *bin.Encoder) error {
}
// .compiled_instruction.data:
{
- // len uint8
- err := encoder.WriteUint8(uint8(len(inst.Data)))
+ // len uint16
+ dataLen := uint16(len(inst.Data))
+ if int(dataLen) != len(inst.Data) {
+ return fmt.Errorf("encoded len(Data) is larger than 16 bits")
+ }
+ err := encoder.WriteUint16(dataLen, binary.LittleEndian)
if err != nil {
return fmt.Errorf("failed to write len(Data): %w", err)
}
@@ -98,8 +102,8 @@ func (inst AccountKeys) MarshalWithEncoder(encoder *bin.Encoder) error {
{
// account_keys.static_keys:
{
- // len uint8
- err := encoder.WriteUint8(uint8(len(inst.StaticKeys)))
+ // len uint16
+ err := encoder.WriteUint16(uint16(len(inst.StaticKeys)), binary.LittleEndian)
if err != nil {
return fmt.Errorf("failed to write len(StaticKeys): %w", err)
}
@@ -141,8 +145,8 @@ func (inst LoadedAddresses) MarshalWithEncoder(encoder *bin.Encoder) error {
{
// account_keys.dynamic_keys.writable:
{
- // len uint8
- err := encoder.WriteUint8(uint8(len(inst.Writable)))
+ // len uint16
+ err := encoder.WriteUint16(uint16(len(inst.Writable)), binary.LittleEndian)
if err != nil {
return fmt.Errorf("failed to write len(Writable): %w", err)
}
@@ -156,8 +160,8 @@ func (inst LoadedAddresses) MarshalWithEncoder(encoder *bin.Encoder) error {
}
// account_keys.dynamic_keys.readonly:
{
- // len uint8
- err := encoder.WriteUint8(uint8(len(inst.Readonly)))
+ // len uint16
+ err := encoder.WriteUint16(uint16(len(inst.Readonly)), binary.LittleEndian)
if err != nil {
return fmt.Errorf("failed to write len(Readonly): %w", err)
}