From b569841677e5d395aec826c7c1f5cb2796b68703 Mon Sep 17 00:00:00 2001 From: gagliardetto Date: Wed, 30 Oct 2024 12:14:42 +0100 Subject: [PATCH] Tx meta fixes (#175) * Cleanup gsfa handling of tx metadata * Add `find-missing-tx-metadata` command for finding transactions with missing metadata. * Cleanup * Cleanup * Cleanup * Undefined kind is -1 * Cleanup * More checking * Cleanup * Cleanup * More options * More errors * Move solana errors to dedicated dir * Solanaerrors * make gen-proto * Stub multireader * Handle InsufficientFundsForRent --- accum/block.go | 43 +- accum/tx.go | 245 +++++++++++ adapters.go | 3 +- carreader/reader.go | 2 +- cmd-car-split.go | 21 +- cmd-dump-car.go | 13 +- cmd-find-missing-tx-meta.go | 295 +++++++++++++ cmd-x-index-gsfa.go | 123 +----- cmd-x-traverse.go | 7 +- err.go | 3 +- getSignaturesForAddress.go | 93 ---- go.mod | 4 +- grpc-server.go | 5 +- gsfa/linkedlog/compress.go | 37 -- gsfa/linkedlog/linked-log.go | 5 +- iplddecoders/decoders.go | 5 +- main.go | 1 + multiepoch-getBlock.go | 5 +- readahead/readahead.go | 9 +- readasonecar/read-as-one-car.go | 158 +++++++ solana-errors.go | 202 --------- solana-errors/from-json-to-protobuf.go | 161 +++++++ solana-errors/from-json-to-protobuf_test.go | 78 ++++ solana-errors/solana-errors.go | 308 +++++++++++++ solana-tx-meta-parsers/parsers.go | 66 +++ storage.go | 63 +-- .../confirmed_block/confirmed_block.pb.go | 408 ++++-------------- .../transaction_by_addr.pb.go | 402 +++++++---------- .../transaction_by_addr.proto | 2 + tooling/compress.go | 38 ++ tooling/data-frames.go | 58 +++ tooling/files.go | 36 ++ 32 files changed, 1759 insertions(+), 1140 deletions(-) create mode 100644 accum/tx.go create mode 100644 cmd-find-missing-tx-meta.go create mode 100644 readasonecar/read-as-one-car.go create mode 100644 solana-errors/from-json-to-protobuf.go create mode 100644 solana-errors/from-json-to-protobuf_test.go create mode 100644 solana-errors/solana-errors.go create mode 100644 tooling/compress.go create mode 100644 tooling/data-frames.go create mode 100644 tooling/files.go diff --git a/accum/block.go b/accum/block.go index 9c56c827..3c4e9e19 100644 --- a/accum/block.go +++ b/accum/block.go @@ -13,6 +13,7 @@ import ( ) type ObjectAccumulator struct { + skipNodes uint64 flushOnKind iplddecoders.Kind reader *carreader.CarReader ignoreKinds iplddecoders.KindSlice @@ -42,6 +43,11 @@ func NewObjectAccumulator( } } +// SetSkip(n) +func (oa *ObjectAccumulator) SetSkip(n uint64) { + oa.skipNodes = n +} + var flushBufferPool = sync.Pool{ New: func() interface{} { return &flushBuffer{} @@ -58,14 +64,14 @@ func putFlushBuffer(fb *flushBuffer) { } type flushBuffer struct { - head *ObjectWithMetadata - other []ObjectWithMetadata + parent *ObjectWithMetadata + children []ObjectWithMetadata } // Reset resets the flushBuffer. func (fb *flushBuffer) Reset() { - fb.head = nil - clear(fb.other) + fb.parent = nil + fb.children = fb.children[:0] } type ObjectWithMetadata struct { @@ -84,7 +90,7 @@ func (oa *ObjectAccumulator) startFlusher(ctx context.Context) { if fb == nil { return } - if err := oa.flush(fb.head, fb.other); err != nil { + if err := oa.flush(fb.parent, fb.children); err != nil { if isStop(err) { return } @@ -99,8 +105,8 @@ func (oa *ObjectAccumulator) startFlusher(ctx context.Context) { func (oa *ObjectAccumulator) sendToFlusher(head *ObjectWithMetadata, other []ObjectWithMetadata) { oa.flushWg.Add(1) fb := getFlushBuffer() - fb.head = head - fb.other = clone(other) + fb.parent = head + fb.children = other oa.flushQueue <- fb } @@ -118,19 +124,20 @@ func (oa *ObjectAccumulator) Run(ctx context.Context) error { totalOffset += size } } + numSkipped := uint64(0) objectCap := 5000 buffersLoop: for { - objects := make([]ObjectWithMetadata, 0, objectCap) + children := make([]ObjectWithMetadata, 0, objectCap) currentBufferLoop: for { if ctx.Err() != nil { return ctx.Err() } - c, sectionLength, data, err := oa.reader.NextNodeBytes() + cid_, sectionLength, data, err := oa.reader.NextNodeBytes() if err != nil { if errors.Is(err, io.EOF) { - oa.sendToFlusher(nil, objects) + oa.sendToFlusher(nil, children) break buffersLoop } return err @@ -138,13 +145,18 @@ buffersLoop: currentOffset := totalOffset totalOffset += sectionLength + if numSkipped < oa.skipNodes { + numSkipped++ + continue + } + if data == nil { - oa.sendToFlusher(nil, objects) + oa.sendToFlusher(nil, children) break buffersLoop } - objm := ObjectWithMetadata{ - Cid: c, + element := ObjectWithMetadata{ + Cid: cid_, Offset: currentOffset, SectionLength: sectionLength, ObjectData: data, @@ -152,13 +164,14 @@ buffersLoop: kind := iplddecoders.Kind(data[1]) if kind == oa.flushOnKind { - oa.sendToFlusher(&objm, (objects)) + // element is parent + oa.sendToFlusher(&element, children) break currentBufferLoop } else { if len(oa.ignoreKinds) > 0 && oa.ignoreKinds.Has(kind) { continue } - objects = append(objects, objm) + children = append(children, element) } } } diff --git a/accum/tx.go b/accum/tx.go new file mode 100644 index 00000000..a60d6604 --- /dev/null +++ b/accum/tx.go @@ -0,0 +1,245 @@ +package accum + +import ( + "context" + "fmt" + "sync" + + "github.com/gagliardetto/solana-go" + "github.com/ipfs/go-cid" + "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode" + "github.com/rpcpool/yellowstone-faithful/iplddecoders" + solanatxmetaparsers "github.com/rpcpool/yellowstone-faithful/solana-tx-meta-parsers" + "github.com/rpcpool/yellowstone-faithful/tooling" +) + +type TransactionWithSlot struct { + Offset uint64 + Length uint64 + Slot uint64 + Blocktime uint64 + Error error + Transaction solana.Transaction + Metadata *solanatxmetaparsers.TransactionStatusMetaContainer +} + +// IsMetaNotFound returns true if the error is a not found error. +func (obj TransactionWithSlot) IsMetaNotFound() bool { + e, ok := obj.Error.(txMetaError) + if !ok { + return false + } + return e.IsMetaNotFound() +} + +// IsMetaParseError returns true if the error is a parsing error. +func (obj TransactionWithSlot) IsMetaParseError() bool { + e, ok := obj.Error.(txMetaError) + if !ok { + return false + } + return e.IsMetaParseError() +} + +// Ok returns true if the error is nil. +func (obj TransactionWithSlot) Ok() bool { + return obj.Error == nil +} + +type txMetaError struct { + Sig solana.Signature + Err error + isNotFound bool + isParseError bool +} + +func (obj txMetaError) Error() string { + switch { + case obj.isNotFound: + return fmt.Sprintf("not found: %s", obj.Err) + case obj.isParseError: + return fmt.Sprintf("parse error: %s", obj.Err) + default: + return fmt.Sprintf("error: %s", obj.Err) + } +} + +func (obj txMetaError) Is(target error) bool { + if _, ok := target.(txMetaError); ok { + return true + } + return false +} + +func (obj txMetaError) Unwrap() error { + return obj.Err +} + +func (obj txMetaError) IsMetaNotFound() bool { + return obj.isNotFound +} + +func (obj txMetaError) IsMetaParseError() bool { + return obj.isParseError +} + +func (obj txMetaError) Ok() bool { + return obj.Err == nil +} + +func newTxMetaErrorNotFound(sig solana.Signature, err error) txMetaError { + return txMetaError{ + Sig: sig, + Err: err, + isNotFound: true, + } +} + +func newTxMetaErrorParseError(sig solana.Signature, err error) txMetaError { + return txMetaError{ + Sig: sig, + Err: err, + isParseError: true, + } +} + +var poolOfTransactionWithSlotSlices = sync.Pool{ + New: func() interface{} { + return make([]*TransactionWithSlot, 0, 1000) + }, +} + +func getTransactionWithSlotSlice() []*TransactionWithSlot { + return poolOfTransactionWithSlotSlices.Get().([]*TransactionWithSlot) +} + +func PutTransactionWithSlotSlice(slice []*TransactionWithSlot) { + slice = slice[:0] + poolOfTransactionWithSlotSlices.Put(slice) +} + +var poolDataBlocksMap = sync.Pool{ + New: func() interface{} { + return make(map[string]ObjectWithMetadata, 0) + }, +} + +func clearDataBlocksMap(m map[string]ObjectWithMetadata) { + for k := range m { + delete(m, k) + } +} + +func getDatablocksMap() map[string]ObjectWithMetadata { + return poolDataBlocksMap.Get().(map[string]ObjectWithMetadata) +} + +func putDataBlocksMap(m map[string]ObjectWithMetadata) { + clearDataBlocksMap(m) + poolDataBlocksMap.Put(m) +} + +func ObjectsToTransactionsAndMetadata( + block *ipldbindcode.Block, + objects []ObjectWithMetadata, +) ([]*TransactionWithSlot, error) { + transactions := getTransactionWithSlotSlice() + dataBlocksMap := getDatablocksMap() + defer putDataBlocksMap(dataBlocksMap) + for objI := range objects { + object := objects[objI] + // check if the object is a transaction: + kind := iplddecoders.Kind(object.ObjectData[1]) + if kind == iplddecoders.KindDataFrame { + dataBlocksMap[object.Cid.String()] = object + continue + } + if kind != iplddecoders.KindTransaction { + continue + } + decodedTxObj, err := iplddecoders.DecodeTransaction(object.ObjectData) + if err != nil { + return nil, fmt.Errorf("error while decoding transaction from nodex %s: %w", object.Cid, err) + } + tws := &TransactionWithSlot{ + Offset: object.Offset, + Length: object.SectionLength, + Slot: uint64(decodedTxObj.Slot), + Blocktime: uint64(block.Meta.Blocktime), + } + tx, err := decodedTxObj.GetSolanaTransaction() + if err != nil { + return nil, fmt.Errorf("error while getting solana transaction from object %s: %w", object.Cid, err) + } + tws.Transaction = *tx + sigs := tx.Signatures + if len(sigs) == 0 { + return nil, fmt.Errorf("transaction has no signatures: %s", object.Cid) + } + sig := sigs[0] + + if total, ok := decodedTxObj.Metadata.GetTotal(); !ok || total == 1 { + // metadata fit into the transaction object: + completeBuffer := decodedTxObj.Metadata.Bytes() + if ha, ok := decodedTxObj.Metadata.GetHash(); ok { + err := ipldbindcode.VerifyHash(completeBuffer, ha) + if err != nil { + return nil, fmt.Errorf("failed to verify metadata hash: %w", err) + } + } + if len(completeBuffer) > 0 { + uncompressedMeta, err := tooling.DecompressZstd(completeBuffer) + if err != nil { + return nil, fmt.Errorf("failed to decompress metadata: %w", err) + } + status, err := solanatxmetaparsers.ParseTransactionStatusMetaContainer(uncompressedMeta) + if err == nil { + tws.Metadata = status + } else { + tws.Error = newTxMetaErrorParseError(sig, err) + } + } else { + tws.Error = newTxMetaErrorNotFound(sig, fmt.Errorf("metadata is empty")) + } + clearDataBlocksMap(dataBlocksMap) + } else { + // metadata didn't fit into the transaction object, and was split into multiple dataframes: + metaBuffer, err := tooling.LoadDataFromDataFrames( + &decodedTxObj.Metadata, + func(ctx context.Context, wantedCid cid.Cid) (*ipldbindcode.DataFrame, error) { + if dataBlock, ok := dataBlocksMap[wantedCid.String()]; ok { + df, err := iplddecoders.DecodeDataFrame(dataBlock.ObjectData) + if err != nil { + return nil, err + } + return df, nil + } + return nil, fmt.Errorf("dataframe not found") + }) + if err != nil { + return nil, fmt.Errorf("failed to load metadata: %w", err) + } + // clear dataBlocksMap so it can accumulate dataframes for the next transaction: + clearDataBlocksMap(dataBlocksMap) + + // if we have a metadata buffer, try to decompress it: + if len(metaBuffer) > 0 { + uncompressedMeta, err := tooling.DecompressZstd(metaBuffer) + if err != nil { + return nil, fmt.Errorf("failed to decompress metadata: %w", err) + } + status, err := solanatxmetaparsers.ParseTransactionStatusMetaContainer(uncompressedMeta) + if err == nil { + tws.Metadata = status + } else { + tws.Error = newTxMetaErrorParseError(sig, err) + } + } else { + tws.Error = newTxMetaErrorNotFound(sig, fmt.Errorf("metadata is empty")) + } + } + + transactions = append(transactions, tws) + } + return transactions, nil +} diff --git a/adapters.go b/adapters.go index 20e3669c..c91d09a6 100644 --- a/adapters.go +++ b/adapters.go @@ -4,6 +4,7 @@ import ( "encoding/base64" "github.com/mr-tron/base58" + solanaerrors "github.com/rpcpool/yellowstone-faithful/solana-errors" ) func ptrToUint64(v uint64) *uint64 { @@ -34,7 +35,7 @@ func adaptTransactionMetaToExpectedOutput(m map[string]any) map[string]any { } { if _, ok := meta["err"]; ok { - meta["err"], _ = parseTransactionError(meta["err"]) + meta["err"], _ = solanaerrors.ParseTransactionError(meta["err"]) } else { meta["err"] = nil } diff --git a/carreader/reader.go b/carreader/reader.go index 1336ae80..a68b321e 100644 --- a/carreader/reader.go +++ b/carreader/reader.go @@ -19,8 +19,8 @@ import ( type CarReader struct { headerSize *uint64 - br *bufio.Reader Header *carv1.CarHeader + br *bufio.Reader } func alignValueToPageSize(value int) int { diff --git a/cmd-car-split.go b/cmd-car-split.go index da539bba..ca3a31bb 100644 --- a/cmd-car-split.go +++ b/cmd-car-split.go @@ -171,7 +171,6 @@ func newCmd_SplitCar() *cli.Command { cp := new(commp.Calc) createNewFile := func() error { - if currentFile != nil { sl, err := writeSubsetNode(currentSubsetInfo, writer) if err != nil { @@ -269,16 +268,16 @@ func newCmd_SplitCar() *cli.Command { accum := accum.NewObjectAccumulator( rd, iplddecoders.KindBlock, - func(owm1 *accum.ObjectWithMetadata, owm2 []accum.ObjectWithMetadata) error { - if owm1 == nil { + func(parent *accum.ObjectWithMetadata, children []accum.ObjectWithMetadata) error { + if parent == nil { return nil } - owms := append(owm2, *owm1) + family := append(children, *parent) dagSize := 0 - for _, owm := range owms { - dagSize += owm.RawSectionSize() + for _, member := range family { + dagSize += member.RawSectionSize() } if currentFile == nil || currentFileSize+uint64(dagSize) > maxFileSize || len(currentSubsetInfo.blockLinks) > maxLinks { @@ -290,7 +289,7 @@ func newCmd_SplitCar() *cli.Command { } // owm1 is necessarily a Block - block, err := iplddecoders.DecodeBlock(owm1.ObjectData) + block, err := iplddecoders.DecodeBlock(parent.ObjectData) if err != nil { return fmt.Errorf("failed to decode block: %w", err) } @@ -302,9 +301,9 @@ func newCmd_SplitCar() *cli.Command { currentSubsetInfo.lastSlot = block.Slot } - currentSubsetInfo.blockLinks = append(currentSubsetInfo.blockLinks, cidlink.Link{Cid: owm1.Cid}) + currentSubsetInfo.blockLinks = append(currentSubsetInfo.blockLinks, cidlink.Link{Cid: parent.Cid}) - err = writeBlockDag(owms) + err = writeBlockDag(family) if err != nil { return fmt.Errorf("failed to write block dag to file: %w", err) } @@ -414,7 +413,6 @@ func newCmd_SplitCar() *cli.Command { } return nil - }, } } @@ -481,7 +479,6 @@ func writeNode(node datamodel.Node, w io.Writer) (cid.Cid, error) { if _, err := w.Write(data); err != nil { return cid.Cid{}, err } - } } return cd, nil @@ -491,7 +488,7 @@ func writeMetadata(metadata *splitcarfetcher.Metadata, epoch int) error { metadataFileName := fmt.Sprintf("epoch-%d-metadata.yaml", epoch) // Open file in append mode - metadataFile, err := os.OpenFile(metadataFileName, os.O_APPEND|os.O_CREATE|os.O_WRONLY, 0644) + metadataFile, err := os.OpenFile(metadataFileName, os.O_APPEND|os.O_CREATE|os.O_WRONLY, 0o644) if err != nil { return fmt.Errorf("failed to open metadata file: %w", err) } diff --git a/cmd-dump-car.go b/cmd-dump-car.go index 699c0bb0..4d1b6460 100644 --- a/cmd-dump-car.go +++ b/cmd-dump-car.go @@ -10,8 +10,6 @@ import ( "strings" "time" - "github.com/klauspost/compress/zstd" - "github.com/davecgh/go-spew/spew" bin "github.com/gagliardetto/binary" "github.com/gagliardetto/solana-go" @@ -21,6 +19,7 @@ import ( "github.com/rpcpool/yellowstone-faithful/readahead" solanablockrewards "github.com/rpcpool/yellowstone-faithful/solana-block-rewards" solanatxmetaparsers "github.com/rpcpool/yellowstone-faithful/solana-tx-meta-parsers" + "github.com/rpcpool/yellowstone-faithful/tooling" "github.com/urfave/cli/v2" "k8s.io/klog/v2" ) @@ -246,7 +245,7 @@ func newCmd_DumpCar() *cli.Command { } } if len(completeBuffer) > 0 { - uncompressedMeta, err := decompressZstd(completeBuffer) + uncompressedMeta, err := tooling.DecompressZstd(completeBuffer) if err != nil { panic(err) } @@ -318,7 +317,7 @@ func newCmd_DumpCar() *cli.Command { } } if len(completeBuffer) > 0 { - uncompressedRewards, err := decompressZstd(completeBuffer) + uncompressedRewards, err := tooling.DecompressZstd(completeBuffer) if err != nil { panic(err) } @@ -365,9 +364,3 @@ func (s intSlice) has(v int) bool { func (s intSlice) empty() bool { return len(s) == 0 } - -var decoder, _ = zstd.NewReader(nil) - -func decompressZstd(data []byte) ([]byte, error) { - return decoder.DecodeAll(data, nil) -} diff --git a/cmd-find-missing-tx-meta.go b/cmd-find-missing-tx-meta.go new file mode 100644 index 00000000..bea7704f --- /dev/null +++ b/cmd-find-missing-tx-meta.go @@ -0,0 +1,295 @@ +package main + +import ( + "context" + "fmt" + "io/fs" + "os" + "path/filepath" + "sync/atomic" + "time" + + "github.com/davecgh/go-spew/spew" + "github.com/dustin/go-humanize" + "github.com/gagliardetto/solana-go" + "github.com/rpcpool/yellowstone-faithful/accum" + "github.com/rpcpool/yellowstone-faithful/carreader" + "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode" + "github.com/rpcpool/yellowstone-faithful/iplddecoders" + "github.com/rpcpool/yellowstone-faithful/tooling" + "github.com/urfave/cli/v2" + "k8s.io/klog/v2" +) + +func newCmd_find_missing_tx_metadata() *cli.Command { + return &cli.Command{ + Name: "find-missing-tx-metadata", + Description: "Find missing transaction metadata in a CAR file.", + ArgsUsage: "", + Before: func(c *cli.Context) error { + return nil + }, + Flags: []cli.Flag{ + &cli.BoolFlag{ + Name: "silent", + Usage: "Do not print progress", + Value: false, + }, + &cli.StringSliceFlag{ + Name: "watch", + Usage: "Watch for these transactions; provide a base58-encoded signature; can be repeated; will print the transaction if found", + }, + // destination dir for the output files + &cli.StringFlag{ + Name: "output-dir", + Usage: "Destination directory for the output files", + Value: "", + }, + // skip is the number of objects (any kind) to skip before starting to process. + &cli.Uint64Flag{ + Name: "skip", + Usage: "Number of objects to skip before starting to process", + Value: 0, + }, + // dont' save missing metadata + &cli.BoolFlag{ + Name: "no-save-missing-metadata", + Usage: "Do not save the signatures of transactions that are missing metadata", + Value: false, + }, + // dont' save metadata parsing errors + &cli.BoolFlag{ + Name: "no-save-meta-parsing-errors", + Usage: "Do not save the errors that occurred while parsing the metadata", + Value: false, + }, + }, + Action: func(c *cli.Context) error { + carPath := c.Args().First() + var file fs.File + var err error + if carPath == "-" { + file = os.Stdin + } else { + file, err = os.Open(carPath) + if err != nil { + klog.Exit(err.Error()) + } + defer file.Close() + } + + silent := c.Bool("silent") + + if silent { + klog.Infoln("Silent mode is ON: will not print progress") + } + + noSaveMissingMetadata := c.Bool("no-save-missing-metadata") + if noSaveMissingMetadata { + klog.Infoln("Will not save missing metadata") + } + + noSaveMetaParsingErrors := c.Bool("no-save-meta-parsing-errors") + if noSaveMetaParsingErrors { + klog.Infoln("Will not save metadata parsing errors") + } + + rd, err := carreader.New(file) + if err != nil { + klog.Exitf("Failed to open CAR: %s", err) + } + + outputDir := c.String("output-dir") + if outputDir == "" { + outputDir = filepath.Dir(carPath) + } else { + if err := os.MkdirAll(outputDir, os.ModePerm); err != nil { + klog.Exitf("Failed to create output directory: %s", err) + } + } + + // In the same directory as the CAR file, create a file where we will write the signatures of the transactions that are missing metadata. + missingTxPath := filepath.Join(outputDir, filepath.Base(carPath)+".missing-tx-metadata.txt") + fileMissingMetadata, err := tooling.NewBufferedWritableFile(missingTxPath) + if err != nil { + klog.Exitf("Failed to create file for missing metadata: %s", err) + } + + // In the same directory as the CAR file, create a file where we will write the errors that occurred while parsing the metadata. + txMetaParseErrorPath := filepath.Join(outputDir, filepath.Base(carPath)+".tx-meta-parsing-error.txt") + fileTxMetaParsingError, err := tooling.NewBufferedWritableFile(txMetaParseErrorPath) + if err != nil { + klog.Exitf("Failed to create file for tx meta parsing error: %s", err) + } + + numProcessedTransactions := new(atomic.Int64) + startedAt := time.Now() + + slotCounter := uint64(0) + numMaxObjects := uint64(0) + + lastPrintedAt := time.Now() + lastTimeDid1kSlots := time.Now() + var eta time.Duration + etaSampleSlots := uint64(2_000) + var tookToDo1kSlots time.Duration + + var firstSlot *uint64 + var epochStart, epochEnd uint64 + + watch := []solana.Signature{} + for _, sigStr := range c.StringSlice("watch") { + sig, err := solana.SignatureFromBase58(sigStr) + if err != nil { + klog.Exitf("Invalid signature: %s", sigStr) + } + watch = append(watch, sig) + } + if len(watch) > 0 { + klog.Infof("Watching for %d transactions", len(watch)) + } + numTransactionsWithMissingMetadata := new(atomic.Uint64) + numTransactionsWithMetaParsingError := new(atomic.Uint64) + accum := accum.NewObjectAccumulator( + rd, + iplddecoders.KindBlock, + func(parent *accum.ObjectWithMetadata, children []accum.ObjectWithMetadata) error { + slotCounter++ + numObjects := len(children) + 1 + if numObjects > int(numMaxObjects) { + numMaxObjects = uint64(numObjects) + } + + if parent == nil { + transactions, err := accum.ObjectsToTransactionsAndMetadata(&ipldbindcode.Block{ + Meta: ipldbindcode.SlotMeta{ + Blocktime: 0, + }, + }, children) + if err != nil { + return fmt.Errorf("error while converting objects to transactions: %w", err) + } + if len(transactions) == 0 { + return nil + } + spew.Dump(parent, transactions, len(children)) + } + + // decode the block: + block, err := iplddecoders.DecodeBlock(parent.ObjectData) + if err != nil { + return fmt.Errorf("error while decoding block: %w", err) + } + if slotCounter%etaSampleSlots == 0 { + tookToDo1kSlots = time.Since(lastTimeDid1kSlots) + lastTimeDid1kSlots = time.Now() + } + if firstSlot == nil { + slot := uint64(block.Slot) + firstSlot = &slot + // determine epoch: + epoch := CalcEpochForSlot(slot) + epochStart, epochEnd = CalcEpochLimits(epoch) + } + if tookToDo1kSlots > 0 { + remainingSlots := epochEnd - uint64(block.Slot) + if epochEnd < uint64(block.Slot) { + remainingSlots = 0 + } + eta = time.Duration(float64(tookToDo1kSlots) / float64(etaSampleSlots) * float64(remainingSlots)) + } + transactions, err := accum.ObjectsToTransactionsAndMetadata(block, children) + if err != nil { + return fmt.Errorf("error while converting objects to transactions: %w", err) + } + for ii := range transactions { + txWithInfo := transactions[ii] + numProcessedTransactions.Add(1) + + if len(watch) > 0 { + for _, watchSig := range watch { + if watchSig == txWithInfo.Transaction.Signatures[0] { + spew.Dump(txWithInfo) + } + } + } + + if txWithInfo.Metadata == nil { + if txWithInfo.IsMetaNotFound() { + numTransactionsWithMissingMetadata.Add(1) + if !noSaveMissingMetadata { + err := fileMissingMetadata.WriteString(txWithInfo.Transaction.Signatures[0].String() + "\n") + if err != nil { + return fmt.Errorf("error while writing to file: %w", err) + } + } + } + if txWithInfo.IsMetaParseError() { + numTransactionsWithMetaParsingError.Add(1) + quotedError := fmt.Sprintf("%q", txWithInfo.Error) + if !noSaveMetaParsingErrors { + err := fileTxMetaParsingError.WriteString(quotedError + "\n") + if err != nil { + return fmt.Errorf("error while writing to file: %w", err) + } + } + } + } + + if time.Since(lastPrintedAt) > time.Millisecond*500 { + percentDone := float64(txWithInfo.Slot-epochStart) / float64(epochEnd-epochStart) * 100 + // clear line, then print progress + msg := fmt.Sprintf( + "\rChecking tx meta - %s missing, %s parse err - %s | %s | %.2f%% | slot %s | tx %s", + humanize.Comma(int64(numTransactionsWithMissingMetadata.Load())), + humanize.Comma(int64(numTransactionsWithMetaParsingError.Load())), + 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)) + } + if !silent { + fmt.Print(msg) + } + lastPrintedAt = time.Now() + } + } + return nil + }, + // Ignore these kinds in the accumulator: + iplddecoders.KindEntry, + iplddecoders.KindRewards, + ) + + if skip := c.Uint64("skip"); skip > 0 { + klog.Infof("Skipping %s objects", humanize.Comma(int64(skip))) + accum.SetSkip(skip) + } + + if err := accum.Run(context.Background()); err != nil { + return fmt.Errorf("error while accumulating objects: %w", err) + } + + fileMissingMetadata.Close() + fileTxMetaParsingError.Close() + + klog.Infof("Checked %s transactions", humanize.Comma(int64(numProcessedTransactions.Load()))) + klog.Infof("Finished in %s", time.Since(startedAt)) + + klog.Infof("Transactions with missing metadata: %d", numTransactionsWithMissingMetadata.Load()) + klog.Infof("Transactions with metadata parsing error: %d", numTransactionsWithMetaParsingError.Load()) + + // NOTE: if there are parsing errors, THEY WILL BE IGNORED. + if numTransactionsWithMissingMetadata.Load() > 0 { + file.Close() + os.Exit(1) + } + os.Exit(0) + return nil + }, + } +} diff --git a/cmd-x-index-gsfa.go b/cmd-x-index-gsfa.go index 26f0556a..65db5400 100644 --- a/cmd-x-index-gsfa.go +++ b/cmd-x-index-gsfa.go @@ -12,7 +12,6 @@ import ( "github.com/davecgh/go-spew/spew" "github.com/dustin/go-humanize" - "github.com/gagliardetto/solana-go" "github.com/ipfs/go-cid" "github.com/rpcpool/yellowstone-faithful/accum" "github.com/rpcpool/yellowstone-faithful/carreader" @@ -21,8 +20,6 @@ import ( "github.com/rpcpool/yellowstone-faithful/indexmeta" "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode" "github.com/rpcpool/yellowstone-faithful/iplddecoders" - solanatxmetaparsers "github.com/rpcpool/yellowstone-faithful/solana-tx-meta-parsers" - "github.com/rpcpool/yellowstone-faithful/third_party/solana_proto/confirmed_block" "github.com/urfave/cli/v2" "k8s.io/klog/v2" ) @@ -183,30 +180,30 @@ func newCmd_Index_gsfa() *cli.Command { accum := accum.NewObjectAccumulator( rd, iplddecoders.KindBlock, - func(owm1 *accum.ObjectWithMetadata, owm2 []accum.ObjectWithMetadata) error { + func(parent *accum.ObjectWithMetadata, children []accum.ObjectWithMetadata) error { numSlots++ - numObjects := len(owm2) + 1 + numObjects := len(children) + 1 if numObjects > int(numMaxObjects) { numMaxObjects = uint64(numObjects) } - if owm1 == nil { - transactions, err := objectsToTransactions(&ipldbindcode.Block{ + if parent == nil { + transactions, err := accum.ObjectsToTransactionsAndMetadata(&ipldbindcode.Block{ Meta: ipldbindcode.SlotMeta{ Blocktime: 0, }, - }, owm2) + }, children) if err != nil { return fmt.Errorf("error while converting objects to transactions: %w", err) } if len(transactions) == 0 { return nil } - spew.Dump(owm1, transactions, len(owm2)) + spew.Dump(parent, transactions, len(children)) } // decode the block: - block, err := iplddecoders.DecodeBlock(owm1.ObjectData) + block, err := iplddecoders.DecodeBlock(parent.ObjectData) if err != nil { return fmt.Errorf("error while decoding block: %w", err) } @@ -217,17 +214,20 @@ func newCmd_Index_gsfa() *cli.Command { if tookToDo1kSlots > 0 { eta = time.Duration(float64(tookToDo1kSlots) / float64(etaSampleSlots) * float64(epochEnd-epochStart-numSlots)) } - transactions, err := objectsToTransactions(block, owm2) + transactions, err := accum.ObjectsToTransactionsAndMetadata(block, children) if err != nil { return fmt.Errorf("error while converting objects to transactions: %w", err) } + defer accum.PutTransactionWithSlotSlice(transactions) + for ii := range transactions { txWithInfo := transactions[ii] numProcessedTransactions.Add(1) accountKeys := txWithInfo.Transaction.Message.AccountKeys - if txWithInfo.Metadata != nil { - accountKeys = append(accountKeys, byteSlicesToKeySlice(txWithInfo.Metadata.LoadedReadonlyAddresses)...) - accountKeys = append(accountKeys, byteSlicesToKeySlice(txWithInfo.Metadata.LoadedWritableAddresses)...) + if txWithInfo.Metadata != nil && txWithInfo.Metadata.IsProtobuf() { + meta := txWithInfo.Metadata.GetProtobuf() + accountKeys = append(accountKeys, byteSlicesToKeySlice(meta.LoadedReadonlyAddresses)...) + accountKeys = append(accountKeys, byteSlicesToKeySlice(meta.LoadedWritableAddresses)...) } err = indexW.Push( txWithInfo.Offset, @@ -261,10 +261,9 @@ func newCmd_Index_gsfa() *cli.Command { } return nil }, - // Ignore these kinds in the accumulator (only need transactions): + // Ignore these kinds in the accumulator (only need Transactions and DataFrames): iplddecoders.KindEntry, iplddecoders.KindRewards, - iplddecoders.KindDataFrame, ) if err := accum.Run(context.Background()); err != nil { @@ -276,89 +275,6 @@ func newCmd_Index_gsfa() *cli.Command { } } -func objectsToTransactions( - block *ipldbindcode.Block, - objects []accum.ObjectWithMetadata, -) ([]*TransactionWithSlot, error) { - transactions := make([]*TransactionWithSlot, 0, len(objects)) - dataBlocks := make([]accum.ObjectWithMetadata, 0) - for _, object := range objects { - // check if the object is a transaction: - kind := iplddecoders.Kind(object.ObjectData[1]) - if kind == iplddecoders.KindDataFrame { - dataBlocks = append(dataBlocks, object) - continue - } - 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) - } - tws := &TransactionWithSlot{ - Offset: object.Offset, - Length: object.SectionLength, - Slot: uint64(decoded.Slot), - Blocktime: uint64(block.Meta.Blocktime), - } - if total, ok := decoded.Metadata.GetTotal(); !ok || total == 1 { - completeBuffer := decoded.Metadata.Bytes() - if ha, ok := decoded.Metadata.GetHash(); ok { - err := ipldbindcode.VerifyHash(completeBuffer, ha) - if err != nil { - return nil, fmt.Errorf("failed to verify metadata hash: %w", err) - } - } - if len(completeBuffer) > 0 { - uncompressedMeta, err := decompressZstd(completeBuffer) - if err != nil { - return nil, fmt.Errorf("failed to decompress metadata: %w", err) - } - status, err := solanatxmetaparsers.ParseTransactionStatusMeta(uncompressedMeta) - if err == nil { - tws.Metadata = status - } - } - } else { - metaBuffer, err := loadDataFromDataFrames(&decoded.Metadata, func(ctx context.Context, wantedCid cid.Cid) (*ipldbindcode.DataFrame, error) { - for _, dataBlock := range dataBlocks { - if dataBlock.Cid == wantedCid { - df, err := iplddecoders.DecodeDataFrame(dataBlock.ObjectData) - if err != nil { - return nil, err - } - return df, nil - } - } - return nil, fmt.Errorf("dataframe not found") - }) - if err != nil { - return nil, fmt.Errorf("failed to load metadata: %w", err) - } - // reset dataBlocks: - dataBlocks = dataBlocks[:0] - if len(metaBuffer) > 0 { - uncompressedMeta, err := decompressZstd(metaBuffer) - if err != nil { - return nil, fmt.Errorf("failed to decompress metadata: %w", err) - } - status, err := solanatxmetaparsers.ParseTransactionStatusMeta(uncompressedMeta) - if err == nil { - tws.Metadata = status - } - } - } - tx, err := decoded.GetSolanaTransaction() - if err != nil { - return nil, fmt.Errorf("error while getting solana transaction from object %s: %w", object.Cid, err) - } - tws.Transaction = *tx - transactions = append(transactions, tws) - } - return transactions, nil -} - func formatIndexDirname_gsfa(epoch uint64, rootCid cid.Cid, network indexes.Network) string { return fmt.Sprintf( "epoch-%d-%s-%s-%s", @@ -368,12 +284,3 @@ func formatIndexDirname_gsfa(epoch uint64, rootCid cid.Cid, network indexes.Netw "gsfa.indexdir", ) } - -type TransactionWithSlot struct { - Offset uint64 - Length uint64 - Slot uint64 - Blocktime uint64 - Transaction solana.Transaction - Metadata *confirmed_block.TransactionStatusMeta -} diff --git a/cmd-x-traverse.go b/cmd-x-traverse.go index b26b465f..cc776dcb 100644 --- a/cmd-x-traverse.go +++ b/cmd-x-traverse.go @@ -12,6 +12,7 @@ import ( cidlink "github.com/ipld/go-ipld-prime/linking/cid" "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode" solanatxmetaparsers "github.com/rpcpool/yellowstone-faithful/solana-tx-meta-parsers" + "github.com/rpcpool/yellowstone-faithful/tooling" "github.com/urfave/cli/v2" "k8s.io/klog/v2" ) @@ -97,7 +98,7 @@ func newCmd_XTraverse() *cli.Command { { var transaction solana.Transaction { - txBuffer, err := loadDataFromDataFrames(&tx.Data, simpleIter.GetDataFrame) + txBuffer, err := tooling.LoadDataFromDataFrames(&tx.Data, simpleIter.GetDataFrame) if err != nil { panic(err) } @@ -112,12 +113,12 @@ func newCmd_XTraverse() *cli.Command { fmt.Println(transaction.String()) } { - metaBuffer, err := loadDataFromDataFrames(&tx.Metadata, simpleIter.GetDataFrame) + metaBuffer, err := tooling.LoadDataFromDataFrames(&tx.Metadata, simpleIter.GetDataFrame) if err != nil { panic(err) } if len(metaBuffer) > 0 { - uncompressedMeta, err := decompressZstd(metaBuffer) + uncompressedMeta, err := tooling.DecompressZstd(metaBuffer) if err != nil { panic(err) } diff --git a/err.go b/err.go index 1ccdf958..d6b22aef 100644 --- a/err.go +++ b/err.go @@ -3,13 +3,14 @@ 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" + solanaerrors "github.com/rpcpool/yellowstone-faithful/solana-errors" "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) + out, _ := solanaerrors.ParseTransactionError(metaValue.Err) return out case *metalatest.TransactionStatusMeta: switch status := metaValue.Status.(type) { diff --git a/getSignaturesForAddress.go b/getSignaturesForAddress.go index 855cbf44..f9b52544 100644 --- a/getSignaturesForAddress.go +++ b/getSignaturesForAddress.go @@ -1,11 +1,9 @@ package main import ( - "encoding/base64" "encoding/json" "fmt" - bin "github.com/gagliardetto/binary" "github.com/gagliardetto/solana-go" ) @@ -90,94 +88,3 @@ var ( memoProgramIDV1 = solana.MPK("Memo1UhkJRfHyvLMcVucJwxXeuD728EqVDDwQDxFMNo") memoProgramIDV2 = solana.MPK("MemoSq4gqABAXKb96qnH8TysNcWxMyWCqXgDLGmfcHr") ) - -func parseTransactionError(v any) (map[string]any, error) { - // TODO: if any of the following fails, return the original value. - // marshal to json - b, err := fasterJson.Marshal(v) - if err != nil { - return nil, err - } - // then unmarshal to map - var m map[string]any - err = fasterJson.Unmarshal(b, &m) - if err != nil { - return nil, err - } - // get the "err" field - errValue, ok := m["err"] - if !ok { - return nil, nil - } - // try to parse base64 - errValueStr, ok := errValue.(string) - if !ok { - return nil, nil - } - b, err = base64.StdEncoding.DecodeString(errValueStr) - if err != nil { - return nil, err - } - /// - { - dec := bin.NewBinDecoder(b) - transactionErrorType, err := dec.ReadUint32(bin.LE) - if err != nil { - return nil, err - } - // TODO: is this uint8 or uvarint or something else? - errorCode, err := dec.ReadUint8() - if err != nil { - return nil, err - } - transactionErrorTypeName, ok := TransactionErrorType_name[int32(transactionErrorType)] - if !ok { - return nil, fmt.Errorf("unknown transaction error type: %d", transactionErrorType) - } - transactionErrorTypeName = bin.ToPascalCase(transactionErrorTypeName) - - switch TransactionErrorType(transactionErrorType) { - case TransactionErrorType_INSTRUCTION_ERROR: - - instructionErrorType, err := dec.ReadUint32(bin.LE) - if err != nil { - return nil, err - } - - instructionErrorTypeName, ok := InstructionErrorType_name[int32(instructionErrorType)] - if !ok { - return nil, fmt.Errorf("unknown instruction error type: %d", instructionErrorType) - } - instructionErrorTypeName = bin.ToPascalCase(instructionErrorTypeName) - - switch InstructionErrorType(instructionErrorType) { - case InstructionErrorType_CUSTOM: - customErrorType, err := dec.ReadUint32(bin.LE) - if err != nil { - return nil, err - } - return map[string]any{ - transactionErrorTypeName: []any{ - errorCode, - map[string]any{ - instructionErrorTypeName: customErrorType, - }, - }, - }, nil - } - - return map[string]any{ - transactionErrorTypeName: []any{ - errorCode, - instructionErrorTypeName, - }, - }, nil - default: - return map[string]any{ - transactionErrorTypeName: []any{ - errorCode, - }, - }, nil - } - } -} diff --git a/go.mod b/go.mod index 66a5aba1..d5d59da6 100644 --- a/go.mod +++ b/go.mod @@ -63,6 +63,8 @@ require ( github.com/allegro/bigcache/v3 v3.1.0 github.com/anjor/carlet v0.0.0-00010101000000-000000000000 github.com/filecoin-project/go-address v1.1.0 + github.com/filecoin-project/go-fil-commcid v0.1.0 + github.com/filecoin-project/go-fil-commp-hashhash v0.2.0 github.com/filecoin-project/go-leb128 v0.0.0-20190212224330-8d79a5489543 github.com/fsnotify/fsnotify v1.5.4 github.com/goware/urlx v0.3.2 @@ -109,8 +111,6 @@ require ( github.com/filecoin-project/go-amt-ipld/v4 v4.1.0 // indirect github.com/filecoin-project/go-cbor-util v0.0.1 // indirect github.com/filecoin-project/go-ds-versioning v0.1.2 // indirect - github.com/filecoin-project/go-fil-commcid v0.1.0 // indirect - github.com/filecoin-project/go-fil-commp-hashhash v0.2.0 // indirect github.com/filecoin-project/go-hamt-ipld/v3 v3.2.0 // indirect github.com/filecoin-project/go-retrieval-types v1.2.0 // indirect github.com/filecoin-project/go-statemachine v1.0.3 // indirect diff --git a/grpc-server.go b/grpc-server.go index c5155a6f..dcad5b1b 100644 --- a/grpc-server.go +++ b/grpc-server.go @@ -20,6 +20,7 @@ import ( "github.com/rpcpool/yellowstone-faithful/compactindexsized" "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode" old_faithful_grpc "github.com/rpcpool/yellowstone-faithful/old-faithful-proto/old-faithful-grpc" + "github.com/rpcpool/yellowstone-faithful/tooling" "golang.org/x/sync/errgroup" "google.golang.org/grpc" "google.golang.org/grpc/codes" @@ -266,12 +267,12 @@ func (multi *MultiEpoch) GetBlock(ctx context.Context, params *old_faithful_grpc if err != nil { return nil, status.Errorf(codes.Internal, "Failed to get Rewards: %v", err) } - rewardsBuf, err := loadDataFromDataFrames(&rewardsNode.Data, epochHandler.GetDataFrameByCid) + rewardsBuf, err := tooling.LoadDataFromDataFrames(&rewardsNode.Data, epochHandler.GetDataFrameByCid) if err != nil { return nil, status.Errorf(codes.Internal, "Failed to load Rewards dataFrames: %v", err) } - uncompressedRawRewards, err := decompressZstd(rewardsBuf) + uncompressedRawRewards, err := tooling.DecompressZstd(rewardsBuf) if err != nil { return nil, status.Errorf(codes.Internal, "Failed to decompress Rewards: %v", err) } diff --git a/gsfa/linkedlog/compress.go b/gsfa/linkedlog/compress.go index b89ceb46..0ea6bcb2 100644 --- a/gsfa/linkedlog/compress.go +++ b/gsfa/linkedlog/compress.go @@ -1,38 +1 @@ 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 4a1cbfbd..be7f2c6d 100644 --- a/gsfa/linkedlog/linked-log.go +++ b/gsfa/linkedlog/linked-log.go @@ -14,6 +14,7 @@ import ( "github.com/gagliardetto/solana-go" "github.com/rpcpool/yellowstone-faithful/indexes" + "github.com/rpcpool/yellowstone-faithful/tooling" ) type LinkedLog struct { @@ -158,7 +159,7 @@ func (s *LinkedLog) ReadWithSize(offset uint64, size uint64) ([]OffsetAndSizeAnd } func decompressIndexes(data []byte) ([]OffsetAndSizeAndBlocktime, error) { - decompressed, err := decompressZSTD(data) + decompressed, err := tooling.DecompressZstd(data) if err != nil { return nil, fmt.Errorf("error while decompressing data: %w", err) } @@ -250,7 +251,7 @@ func createIndexesPayload(indexes []*OffsetAndSizeAndBlocktime) ([]byte, error) buf = append(buf, index.Bytes()...) } buf = slices.Clip(buf) - return (compressZSTD(buf)) + return tooling.CompressZstd(buf) } func encodeUvarint(n uint64) []byte { diff --git a/iplddecoders/decoders.go b/iplddecoders/decoders.go index d363e5fe..ef0f8e6d 100644 --- a/iplddecoders/decoders.go +++ b/iplddecoders/decoders.go @@ -174,7 +174,10 @@ func DecodeAny(anyRaw []byte) (any, error) { func GetKind(anyRaw []byte) (Kind, error) { if len(anyRaw) == 0 { - return Kind(0), fmt.Errorf("empty bytes") + return Kind(-1), fmt.Errorf("empty bytes") + } + if len(anyRaw) < 2 { + return Kind(-1), fmt.Errorf("not enough bytes") } kind := Kind(anyRaw[1]) return kind, nil diff --git a/main.go b/main.go index 2d1bcd05..997ebbd9 100644 --- a/main.go +++ b/main.go @@ -59,6 +59,7 @@ func main() { newCmd_check_deals(), newCmd_MergeCars(), newCmd_SplitCar(), + newCmd_find_missing_tx_metadata(), }, } diff --git a/multiepoch-getBlock.go b/multiepoch-getBlock.go index 6f1e5dbd..11c96412 100644 --- a/multiepoch-getBlock.go +++ b/multiepoch-getBlock.go @@ -19,6 +19,7 @@ import ( "github.com/rpcpool/yellowstone-faithful/compactindexsized" "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode" solanablockrewards "github.com/rpcpool/yellowstone-faithful/solana-block-rewards" + "github.com/rpcpool/yellowstone-faithful/tooling" "github.com/sourcegraph/jsonrpc2" "golang.org/x/sync/errgroup" "k8s.io/klog/v2" @@ -281,7 +282,7 @@ func (multi *MultiEpoch) handleGetBlock(ctx context.Context, conn *requestContex Message: "Internal error", }, fmt.Errorf("failed to decode Rewards: %v", err) } - rewardsBuf, err := loadDataFromDataFrames(&rewardsNode.Data, epochHandler.GetDataFrameByCid) + rewardsBuf, err := tooling.LoadDataFromDataFrames(&rewardsNode.Data, epochHandler.GetDataFrameByCid) if err != nil { return &jsonrpc2.Error{ Code: jsonrpc2.CodeInternalError, @@ -289,7 +290,7 @@ func (multi *MultiEpoch) handleGetBlock(ctx context.Context, conn *requestContex }, fmt.Errorf("failed to load Rewards dataFrames: %v", err) } - uncompressedRewards, err := decompressZstd(rewardsBuf) + uncompressedRewards, err := tooling.DecompressZstd(rewardsBuf) if err != nil { return &jsonrpc2.Error{ Code: jsonrpc2.CodeInternalError, diff --git a/readahead/readahead.go b/readahead/readahead.go index ca777b0a..03fd1dd2 100644 --- a/readahead/readahead.go +++ b/readahead/readahead.go @@ -12,7 +12,7 @@ const ( MiB = 1024 * KiB ) -const DefaultChunkSize = 12 * MiB +const DefaultChunkSize = 4 * KiB type CachingReader struct { file io.ReadCloser @@ -28,7 +28,6 @@ func NewCachingReader(filePath string, chunkSize int) (*CachingReader, error) { if chunkSize <= 0 { chunkSize = DefaultChunkSize } - chunkSize = alignValueToPageSize(chunkSize) file, err := os.Open(filePath) if err != nil { return nil, err @@ -40,15 +39,9 @@ func NewCachingReaderFromReader(file io.ReadCloser, chunkSize int) (*CachingRead if chunkSize <= 0 { chunkSize = DefaultChunkSize } - chunkSize = alignValueToPageSize(chunkSize) return &CachingReader{file: file, buffer: bufio.NewReaderSize(file, chunkSize), chunkSize: chunkSize}, nil } -func alignValueToPageSize(value int) int { - pageSize := os.Getpagesize() - return (value + pageSize - 1) &^ (pageSize - 1) -} - func (cr *CachingReader) Read(p []byte) (int, error) { if cr.file == nil { return 0, fmt.Errorf("file not open") diff --git a/readasonecar/read-as-one-car.go b/readasonecar/read-as-one-car.go new file mode 100644 index 00000000..329e7a39 --- /dev/null +++ b/readasonecar/read-as-one-car.go @@ -0,0 +1,158 @@ +package readasonecar + +import ( + "fmt" + "io" + "os" + + "github.com/ipfs/go-cid" + "github.com/ipfs/go-libipfs/blocks" + "github.com/rpcpool/yellowstone-faithful/carreader" +) + +type MultiReader struct { + currentIndex int + files []string + onClose []func() error + readers []*carreader.CarReader +} + +type CarReader interface { + NextInfo() (cid.Cid, uint64, error) + NextNode() (cid.Cid, uint64, *blocks.BasicBlock, error) + NextNodeBytes() (cid.Cid, uint64, []byte, error) + + HeadeSizeUntilNow() (uint64, error) + Close() error +} + +func NewMultiReader(files ...string) (*MultiReader, error) { + if len(files) == 0 { + return nil, fmt.Errorf("no files provided") + } + // check that each file exists + for _, file := range files { + if _, err := os.Stat(file); err != nil { + return nil, err + } + } + readers := make([]*carreader.CarReader, len(files)) + onClose := make([]func() error, len(files)) + for i, file := range files { + f, err := os.Open(file) + if err != nil { + return nil, fmt.Errorf("failed to open car file %s: %w", file, err) + } + onClose[i] = f.Close + r, err := carreader.New(f) + if err != nil { + return nil, fmt.Errorf("failed to create car reader for file %s: %w", file, err) + } + readers[i] = r + } + return &MultiReader{files: files}, nil +} + +func (mr *MultiReader) NextInfo() (cid.Cid, uint64, error) { + if mr.currentIndex >= len(mr.files) { + return cid.Cid{}, 0, io.EOF + } + r := mr.readers[mr.currentIndex] + cid, offset, err := r.NextInfo() + if err == io.EOF { + mr.currentIndex++ + return mr.NextInfo() + } + return cid, offset, err +} + +func (mr *MultiReader) NextNode() (cid.Cid, uint64, *blocks.BasicBlock, error) { + if mr.currentIndex >= len(mr.files) { + return cid.Cid{}, 0, nil, io.EOF + } + r := mr.readers[mr.currentIndex] + cid, offset, block, err := r.NextNode() + if err == io.EOF { + mr.currentIndex++ + return mr.NextNode() + } + return cid, offset, block, err +} + +func (mr *MultiReader) NextNodeBytes() (cid.Cid, uint64, []byte, error) { + if mr.currentIndex >= len(mr.files) { + return cid.Cid{}, 0, nil, io.EOF + } + r := mr.readers[mr.currentIndex] + cid, offset, block, err := r.NextNodeBytes() + if err == io.EOF { + mr.currentIndex++ + return mr.NextNodeBytes() + } + return cid, offset, block, err +} + +func (mr *MultiReader) Close() error { + var err error + for _, f := range mr.onClose { + if e := f(); e != nil { + err = e + } + } + return err +} + +func (mr *MultiReader) Files() []string { + return mr.files +} + +func (mr *MultiReader) CurrentIndex() int { + if mr.currentIndex >= len(mr.files) { + return -1 + } + return mr.currentIndex +} + +func (mr *MultiReader) CurrentReader() *carreader.CarReader { + if mr.currentIndex >= len(mr.files) { + return nil + } + return mr.readers[mr.currentIndex] +} + +func (mr *MultiReader) Readers() []*carreader.CarReader { + return mr.readers +} + +// HeaderSize returns the size of the header of the CAR file at the given index. +func (mr *MultiReader) HeaderSize(index int) (uint64, error) { + if index >= len(mr.files) { + return 0, fmt.Errorf("index %d out of bounds", index) + } + return mr.readers[index].HeaderSize() +} + +// HeadeSizeUntilNow returns the size of the headers of all the CAR files read so far (including the current one). +// E.g. if the current index is 2, this will return the sum of the header sizes of the CAR files at indices 0, 1, and 2. +func (mr *MultiReader) HeadeSizeUntilNow() (uint64, error) { + if mr.currentIndex >= len(mr.files) { + cumulativeSize := uint64(0) + for i := 0; i < len(mr.files); i++ { + size, err := mr.readers[i].HeaderSize() + if err != nil { + return 0, err + } + cumulativeSize += size + } + return cumulativeSize, nil + } + cumulativeSize := uint64(0) + for i := 0; i <= mr.currentIndex; i++ { + size, err := mr.readers[i].HeaderSize() + if err != nil { + return 0, err + } + cumulativeSize += size + } + return cumulativeSize, nil +} diff --git a/solana-errors.go b/solana-errors.go index 835c1c3c..06ab7d0f 100644 --- a/solana-errors.go +++ b/solana-errors.go @@ -1,203 +1 @@ package main - -type TransactionErrorType int32 - -const ( - TransactionErrorType_ACCOUNT_IN_USE TransactionErrorType = 0 - TransactionErrorType_ACCOUNT_LOADED_TWICE TransactionErrorType = 1 - TransactionErrorType_ACCOUNT_NOT_FOUND TransactionErrorType = 2 - TransactionErrorType_PROGRAM_ACCOUNT_NOT_FOUND TransactionErrorType = 3 - TransactionErrorType_INSUFFICIENT_FUNDS_FOR_FEE TransactionErrorType = 4 - TransactionErrorType_INVALID_ACCOUNT_FOR_FEE TransactionErrorType = 5 - TransactionErrorType_ALREADY_PROCESSED TransactionErrorType = 6 - TransactionErrorType_BLOCKHASH_NOT_FOUND TransactionErrorType = 7 - TransactionErrorType_INSTRUCTION_ERROR TransactionErrorType = 8 - TransactionErrorType_CALL_CHAIN_TOO_DEEP TransactionErrorType = 9 - TransactionErrorType_MISSING_SIGNATURE_FOR_FEE TransactionErrorType = 10 - TransactionErrorType_INVALID_ACCOUNT_INDEX TransactionErrorType = 11 - TransactionErrorType_SIGNATURE_FAILURE TransactionErrorType = 12 - TransactionErrorType_INVALID_PROGRAM_FOR_EXECUTION TransactionErrorType = 13 - TransactionErrorType_SANITIZE_FAILURE TransactionErrorType = 14 - TransactionErrorType_CLUSTER_MAINTENANCE TransactionErrorType = 15 - TransactionErrorType_ACCOUNT_BORROW_OUTSTANDING_TX TransactionErrorType = 16 - TransactionErrorType_WOULD_EXCEED_MAX_BLOCK_COST_LIMIT TransactionErrorType = 17 - TransactionErrorType_UNSUPPORTED_VERSION TransactionErrorType = 18 - TransactionErrorType_INVALID_WRITABLE_ACCOUNT TransactionErrorType = 19 - TransactionErrorType_WOULD_EXCEED_MAX_ACCOUNT_COST_LIMIT TransactionErrorType = 20 - TransactionErrorType_WOULD_EXCEED_ACCOUNT_DATA_BLOCK_LIMIT TransactionErrorType = 21 - TransactionErrorType_TOO_MANY_ACCOUNT_LOCKS TransactionErrorType = 22 - TransactionErrorType_ADDRESS_LOOKUP_TABLE_NOT_FOUND TransactionErrorType = 23 - TransactionErrorType_INVALID_ADDRESS_LOOKUP_TABLE_OWNER TransactionErrorType = 24 - TransactionErrorType_INVALID_ADDRESS_LOOKUP_TABLE_DATA TransactionErrorType = 25 - TransactionErrorType_INVALID_ADDRESS_LOOKUP_TABLE_INDEX TransactionErrorType = 26 - TransactionErrorType_INVALID_RENT_PAYING_ACCOUNT TransactionErrorType = 27 - TransactionErrorType_WOULD_EXCEED_MAX_VOTE_COST_LIMIT TransactionErrorType = 28 - TransactionErrorType_WOULD_EXCEED_ACCOUNT_DATA_TOTAL_LIMIT TransactionErrorType = 29 - TransactionErrorType_DUPLICATE_INSTRUCTION TransactionErrorType = 30 - TransactionErrorType_INSUFFICIENT_FUNDS_FOR_RENT TransactionErrorType = 31 - TransactionErrorType_MAX_LOADED_ACCOUNTS_DATA_SIZE_EXCEEDED TransactionErrorType = 32 - TransactionErrorType_INVALID_LOADED_ACCOUNTS_DATA_SIZE_LIMIT TransactionErrorType = 33 - TransactionErrorType_RESANITIZATION_NEEDED TransactionErrorType = 34 - TransactionErrorType_PROGRAM_EXECUTION_TEMPORARILY_RESTRICTED TransactionErrorType = 35 -) - -// Enum value maps for TransactionErrorType. -var ( - TransactionErrorType_name = map[int32]string{ - 0: "ACCOUNT_IN_USE", - 1: "ACCOUNT_LOADED_TWICE", - 2: "ACCOUNT_NOT_FOUND", - 3: "PROGRAM_ACCOUNT_NOT_FOUND", - 4: "INSUFFICIENT_FUNDS_FOR_FEE", - 5: "INVALID_ACCOUNT_FOR_FEE", - 6: "ALREADY_PROCESSED", - 7: "BLOCKHASH_NOT_FOUND", - 8: "INSTRUCTION_ERROR", - 9: "CALL_CHAIN_TOO_DEEP", - 10: "MISSING_SIGNATURE_FOR_FEE", - 11: "INVALID_ACCOUNT_INDEX", - 12: "SIGNATURE_FAILURE", - 13: "INVALID_PROGRAM_FOR_EXECUTION", - 14: "SANITIZE_FAILURE", - 15: "CLUSTER_MAINTENANCE", - 16: "ACCOUNT_BORROW_OUTSTANDING_TX", - 17: "WOULD_EXCEED_MAX_BLOCK_COST_LIMIT", - 18: "UNSUPPORTED_VERSION", - 19: "INVALID_WRITABLE_ACCOUNT", - 20: "WOULD_EXCEED_MAX_ACCOUNT_COST_LIMIT", - 21: "WOULD_EXCEED_ACCOUNT_DATA_BLOCK_LIMIT", - 22: "TOO_MANY_ACCOUNT_LOCKS", - 23: "ADDRESS_LOOKUP_TABLE_NOT_FOUND", - 24: "INVALID_ADDRESS_LOOKUP_TABLE_OWNER", - 25: "INVALID_ADDRESS_LOOKUP_TABLE_DATA", - 26: "INVALID_ADDRESS_LOOKUP_TABLE_INDEX", - 27: "INVALID_RENT_PAYING_ACCOUNT", - 28: "WOULD_EXCEED_MAX_VOTE_COST_LIMIT", - 29: "WOULD_EXCEED_ACCOUNT_DATA_TOTAL_LIMIT", - 30: "DUPLICATE_INSTRUCTION", - 31: "INSUFFICIENT_FUNDS_FOR_RENT", - 32: "MAX_LOADED_ACCOUNTS_DATA_SIZE_EXCEEDED", - 33: "INVALID_LOADED_ACCOUNTS_DATA_SIZE_LIMIT", - 34: "RESANITIZATION_NEEDED", - 35: "PROGRAM_EXECUTION_TEMPORARILY_RESTRICTED", - } -) - -// Enum value maps for InstructionErrorType. -var ( - InstructionErrorType_name = map[int32]string{ - 0: "GENERIC_ERROR", - 1: "INVALID_ARGUMENT", - 2: "INVALID_INSTRUCTION_DATA", - 3: "INVALID_ACCOUNT_DATA", - 4: "ACCOUNT_DATA_TOO_SMALL", - 5: "INSUFFICIENT_FUNDS", - 6: "INCORRECT_PROGRAM_ID", - 7: "MISSING_REQUIRED_SIGNATURE", - 8: "ACCOUNT_ALREADY_INITIALIZED", - 9: "UNINITIALIZED_ACCOUNT", - 10: "UNBALANCED_INSTRUCTION", - 11: "MODIFIED_PROGRAM_ID", - 12: "EXTERNAL_ACCOUNT_LAMPORT_SPEND", - 13: "EXTERNAL_ACCOUNT_DATA_MODIFIED", - 14: "READONLY_LAMPORT_CHANGE", - 15: "READONLY_DATA_MODIFIED", - 16: "DUPLICATE_ACCOUNT_INDEX", - 17: "EXECUTABLE_MODIFIED", - 18: "RENT_EPOCH_MODIFIED", - 19: "NOT_ENOUGH_ACCOUNT_KEYS", - 20: "ACCOUNT_DATA_SIZE_CHANGED", - 21: "ACCOUNT_NOT_EXECUTABLE", - 22: "ACCOUNT_BORROW_FAILED", - 23: "ACCOUNT_BORROW_OUTSTANDING", - 24: "DUPLICATE_ACCOUNT_OUT_OF_SYNC", - 25: "CUSTOM", - 26: "INVALID_ERROR", - 27: "EXECUTABLE_DATA_MODIFIED", - 28: "EXECUTABLE_LAMPORT_CHANGE", - 29: "EXECUTABLE_ACCOUNT_NOT_RENT_EXEMPT", - 30: "UNSUPPORTED_PROGRAM_ID", - 31: "CALL_DEPTH", - 32: "MISSING_ACCOUNT", - 33: "REENTRANCY_NOT_ALLOWED", - 34: "MAX_SEED_LENGTH_EXCEEDED", - 35: "INVALID_SEEDS", - 36: "INVALID_REALLOC", - 37: "COMPUTATIONAL_BUDGET_EXCEEDED", - 38: "PRIVILEGE_ESCALATION", - 39: "PROGRAM_ENVIRONMENT_SETUP_FAILURE", - 40: "PROGRAM_FAILED_TO_COMPLETE", - 41: "PROGRAM_FAILED_TO_COMPILE", - 42: "IMMUTABLE", - 43: "INCORRECT_AUTHORITY", - 44: "BORSH_IO_ERROR", - 45: "ACCOUNT_NOT_RENT_EXEMPT", - 46: "INVALID_ACCOUNT_OWNER", - 47: "ARITHMETIC_OVERFLOW", - 48: "UNSUPPORTED_SYSVAR", - 49: "ILLEGAL_OWNER", - 50: "MAX_ACCOUNTS_DATA_ALLOCATIONS_EXCEEDED", - 51: "MAX_ACCOUNTS_EXCEEDED", - 52: "MAX_INSTRUCTION_TRACE_LENGTH_EXCEEDED", - 53: "BUILTIN_PROGRAMS_MUST_CONSUME_COMPUTE_UNITS", - } -) - -type InstructionErrorType int32 - -const ( - InstructionErrorType_GENERIC_ERROR InstructionErrorType = 0 - InstructionErrorType_INVALID_ARGUMENT InstructionErrorType = 1 - InstructionErrorType_INVALID_INSTRUCTION_DATA InstructionErrorType = 2 - InstructionErrorType_INVALID_ACCOUNT_DATA InstructionErrorType = 3 - InstructionErrorType_ACCOUNT_DATA_TOO_SMALL InstructionErrorType = 4 - InstructionErrorType_INSUFFICIENT_FUNDS InstructionErrorType = 5 - InstructionErrorType_INCORRECT_PROGRAM_ID InstructionErrorType = 6 - InstructionErrorType_MISSING_REQUIRED_SIGNATURE InstructionErrorType = 7 - InstructionErrorType_ACCOUNT_ALREADY_INITIALIZED InstructionErrorType = 8 - InstructionErrorType_UNINITIALIZED_ACCOUNT InstructionErrorType = 9 - InstructionErrorType_UNBALANCED_INSTRUCTION InstructionErrorType = 10 - InstructionErrorType_MODIFIED_PROGRAM_ID InstructionErrorType = 11 - InstructionErrorType_EXTERNAL_ACCOUNT_LAMPORT_SPEND InstructionErrorType = 12 - InstructionErrorType_EXTERNAL_ACCOUNT_DATA_MODIFIED InstructionErrorType = 13 - InstructionErrorType_READONLY_LAMPORT_CHANGE InstructionErrorType = 14 - InstructionErrorType_READONLY_DATA_MODIFIED InstructionErrorType = 15 - InstructionErrorType_DUPLICATE_ACCOUNT_INDEX InstructionErrorType = 16 - InstructionErrorType_EXECUTABLE_MODIFIED InstructionErrorType = 17 - InstructionErrorType_RENT_EPOCH_MODIFIED InstructionErrorType = 18 - InstructionErrorType_NOT_ENOUGH_ACCOUNT_KEYS InstructionErrorType = 19 - InstructionErrorType_ACCOUNT_DATA_SIZE_CHANGED InstructionErrorType = 20 - InstructionErrorType_ACCOUNT_NOT_EXECUTABLE InstructionErrorType = 21 - InstructionErrorType_ACCOUNT_BORROW_FAILED InstructionErrorType = 22 - InstructionErrorType_ACCOUNT_BORROW_OUTSTANDING InstructionErrorType = 23 - InstructionErrorType_DUPLICATE_ACCOUNT_OUT_OF_SYNC InstructionErrorType = 24 - InstructionErrorType_CUSTOM InstructionErrorType = 25 - InstructionErrorType_INVALID_ERROR InstructionErrorType = 26 - InstructionErrorType_EXECUTABLE_DATA_MODIFIED InstructionErrorType = 27 - InstructionErrorType_EXECUTABLE_LAMPORT_CHANGE InstructionErrorType = 28 - InstructionErrorType_EXECUTABLE_ACCOUNT_NOT_RENT_EXEMPT InstructionErrorType = 29 - InstructionErrorType_UNSUPPORTED_PROGRAM_ID InstructionErrorType = 30 - InstructionErrorType_CALL_DEPTH InstructionErrorType = 31 - InstructionErrorType_MISSING_ACCOUNT InstructionErrorType = 32 - InstructionErrorType_REENTRANCY_NOT_ALLOWED InstructionErrorType = 33 - InstructionErrorType_MAX_SEED_LENGTH_EXCEEDED InstructionErrorType = 34 - InstructionErrorType_INVALID_SEEDS InstructionErrorType = 35 - InstructionErrorType_INVALID_REALLOC InstructionErrorType = 36 - InstructionErrorType_COMPUTATIONAL_BUDGET_EXCEEDED InstructionErrorType = 37 - InstructionErrorType_PRIVILEGE_ESCALATION InstructionErrorType = 38 - InstructionErrorType_PROGRAM_ENVIRONMENT_SETUP_FAILURE InstructionErrorType = 39 - InstructionErrorType_PROGRAM_FAILED_TO_COMPLETE InstructionErrorType = 40 - InstructionErrorType_PROGRAM_FAILED_TO_COMPILE InstructionErrorType = 41 - InstructionErrorType_IMMUTABLE InstructionErrorType = 42 - InstructionErrorType_INCORRECT_AUTHORITY InstructionErrorType = 43 - InstructionErrorType_BORSH_IO_ERROR InstructionErrorType = 44 - InstructionErrorType_ACCOUNT_NOT_RENT_EXEMPT InstructionErrorType = 45 - InstructionErrorType_INVALID_ACCOUNT_OWNER InstructionErrorType = 46 - InstructionErrorType_ARITHMETIC_OVERFLOW InstructionErrorType = 47 - InstructionErrorType_UNSUPPORTED_SYSVAR InstructionErrorType = 48 - InstructionErrorType_ILLEGAL_OWNER InstructionErrorType = 49 - InstructionErrorType_MAX_ACCOUNTS_DATA_ALLOCATIONS_EXCEEDED InstructionErrorType = 50 - InstructionErrorType_MAX_ACCOUNTS_EXCEEDED InstructionErrorType = 51 - InstructionErrorType_MAX_INSTRUCTION_TRACE_LENGTH_EXCEEDED InstructionErrorType = 52 - InstructionErrorType_BUILTIN_PROGRAMS_MUST_CONSUME_COMPUTE_UNITS InstructionErrorType = 53 -) diff --git a/solana-errors/from-json-to-protobuf.go b/solana-errors/from-json-to-protobuf.go new file mode 100644 index 00000000..b271da45 --- /dev/null +++ b/solana-errors/from-json-to-protobuf.go @@ -0,0 +1,161 @@ +package solanaerrors + +import ( + "bytes" + "encoding/json" + "fmt" + + bin "github.com/gagliardetto/binary" +) + +func FromJSONToProtobuf(j map[string]interface{}) ([]byte, error) { + // get first key + firstKey := getFirstKey(j) + if firstKey == "" { + return nil, fmt.Errorf("no keys found in map") + } + buf := new(bytes.Buffer) + wr := bin.NewBinEncoder(buf) + doer := &ChainOps{} + switch firstKey { + case InstructionError: + doer.Do("write transactionErrorType", func() error { + return wr.WriteUint32(uint32(TransactionErrorType_INSTRUCTION_ERROR), bin.LE) + }) + + { + // read instructionErrorType + arr, ok := j[InstructionError].([]interface{}) + if !ok { + return nil, fmt.Errorf("expected an array") + } + if len(arr) != 2 { + return nil, fmt.Errorf("expected an array of length 2") + } + instructionErrorCodeFloat, ok := arr[0].(float64) + if !ok { + return nil, fmt.Errorf("expected a float64, got %T", arr[0]) + } + + instructionErrorCode := uint8(instructionErrorCodeFloat) + doer.Do("write errorCode", func() error { + return wr.WriteUint8(instructionErrorCode) + }) + + { + switch as := arr[1].(type) { + case string: + { + // if string, then map instructionErrorTypeName to code + ixLoop: + for k, v := range InstructionErrorType_name { + // TODO: the conversion to PascalCase might be wrong and break things. + if bin.ToPascalCase(v) == as { + doer.Do("write instructionErrorType", func() error { + return wr.WriteUint32(uint32(k), bin.LE) + }) + break ixLoop + } + } + } + case map[string]interface{}: + { + // if object, then it's custom + firstKey := getFirstKey(as) + if firstKey == "" { + return nil, fmt.Errorf("no keys found in map") + } + if firstKey != "Custom" { + return nil, fmt.Errorf("expected a Custom key") + } + doer.Do("write customErrorType", func() error { + return wr.WriteUint32(uint32(InstructionErrorType_CUSTOM), bin.LE) + }) + customErrorTypeFloat, ok := as[firstKey].(float64) + if !ok { + return nil, fmt.Errorf("expected a float64") + } + customErrorType := uint32(customErrorTypeFloat) + doer.Do("write customErrorType", func() error { + return wr.WriteUint32(customErrorType, bin.LE) + }) + } + } + } + + } + + err := doer.Err() + if err != nil { + return nil, err + } + + return buf.Bytes(), nil + case InsufficientFundsForRent: + doer.Do("write transactionErrorType", func() error { + return wr.WriteUint32(uint32(TransactionErrorType_INSUFFICIENT_FUNDS_FOR_RENT), bin.LE) + }) + // write the accountIndex + { + // "{\"InsufficientFundsForRent\":{\"account_index\":2}}" + // read accountIndex + object, ok := j[InsufficientFundsForRent].(map[string]any) + if !ok { + return nil, fmt.Errorf("expected an object") + } + accountIndexFloat, ok := object["account_index"].(float64) + if !ok { + return nil, fmt.Errorf("expected a float64") + } + accountIndex := uint8(accountIndexFloat) + doer.Do("write accountIndex", func() error { + return wr.WriteUint8(accountIndex) + }) + + if err := doer.Err(); err != nil { + return nil, err + } + } + + return buf.Bytes(), nil + + default: + return nil, fmt.Errorf("unhandled error type: %s from %q", firstKey, toJsonString(j)) + } +} + +func toJsonString(v interface{}) string { + j, err := json.Marshal(v) + if err != nil { + return fmt.Sprintf("%v", v) + } + return string(j) +} + +func getFirstKey(m map[string]interface{}) string { + for k := range m { + return k + } + return "" +} + +const ( + InstructionError = "InstructionError" + InsufficientFundsForRent = "InsufficientFundsForRent" +) + +type ChainOps struct { + e error +} + +func (c *ChainOps) Do(name string, f func() error) *ChainOps { + if c.e != nil { + return c + } + c.e = f() + return c +} + +func (c *ChainOps) Err() error { + return c.e +} diff --git a/solana-errors/from-json-to-protobuf_test.go b/solana-errors/from-json-to-protobuf_test.go new file mode 100644 index 00000000..27ec1964 --- /dev/null +++ b/solana-errors/from-json-to-protobuf_test.go @@ -0,0 +1,78 @@ +package solanaerrors + +import ( + "encoding/base64" + "encoding/binary" + "encoding/json" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestFromJSONToProtobuf(t *testing.T) { + { + candidate := map[string]any{ + "InstructionError": []any{ + 2.0, + map[string]any{ + "Custom": 6001.0, + }, + }, + } + buf, err := FromJSONToProtobuf( + candidate, + ) + require.NoError(t, err) + require.NotNil(t, buf) + require.Equal(t, + []byte{ + 0x8, 0x0, 0x0, 0x0, + 0x2, + 0x19, 0x0, 0x0, 0x0, + 0x71, 0x17, 0x0, 0x0, + }, + buf, + ) + require.Equal(t, + concat( + uint32tobytes(uint32(TransactionErrorType_INSTRUCTION_ERROR)), + []byte{0x2}, + uint32tobytes(uint32(InstructionErrorType_CUSTOM)), + uint32tobytes(6001), + ), + buf, + ) + { + candidateAsBase64 := base64.StdEncoding.EncodeToString(buf) + wrapped := map[string]any{ + "err": candidateAsBase64, + } + got, err := ParseTransactionError(wrapped) + require.NoError(t, err) + require.NotNil(t, got) + + require.JSONEq(t, + toJson(t, candidate), + toJson(t, got), + ) + } + } +} + +func uint32tobytes(v uint32) []byte { + return binary.LittleEndian.AppendUint32(nil, v) +} + +func concat(bs ...[]byte) []byte { + b := make([]byte, 0) + for _, v := range bs { + b = append(b, v...) + } + return b +} + +func toJson(t *testing.T, v interface{}) string { + b, err := json.Marshal(v) + require.NoError(t, err) + return string(b) +} diff --git a/solana-errors/solana-errors.go b/solana-errors/solana-errors.go new file mode 100644 index 00000000..d1009e5c --- /dev/null +++ b/solana-errors/solana-errors.go @@ -0,0 +1,308 @@ +package solanaerrors + +import ( + "encoding/base64" + "fmt" + + bin "github.com/gagliardetto/binary" + jsoniter "github.com/json-iterator/go" +) + +type TransactionErrorType int32 + +const ( + TransactionErrorType_ACCOUNT_IN_USE TransactionErrorType = 0 + TransactionErrorType_ACCOUNT_LOADED_TWICE TransactionErrorType = 1 + TransactionErrorType_ACCOUNT_NOT_FOUND TransactionErrorType = 2 + TransactionErrorType_PROGRAM_ACCOUNT_NOT_FOUND TransactionErrorType = 3 + TransactionErrorType_INSUFFICIENT_FUNDS_FOR_FEE TransactionErrorType = 4 + TransactionErrorType_INVALID_ACCOUNT_FOR_FEE TransactionErrorType = 5 + TransactionErrorType_ALREADY_PROCESSED TransactionErrorType = 6 + TransactionErrorType_BLOCKHASH_NOT_FOUND TransactionErrorType = 7 + TransactionErrorType_INSTRUCTION_ERROR TransactionErrorType = 8 + TransactionErrorType_CALL_CHAIN_TOO_DEEP TransactionErrorType = 9 + TransactionErrorType_MISSING_SIGNATURE_FOR_FEE TransactionErrorType = 10 + TransactionErrorType_INVALID_ACCOUNT_INDEX TransactionErrorType = 11 + TransactionErrorType_SIGNATURE_FAILURE TransactionErrorType = 12 + TransactionErrorType_INVALID_PROGRAM_FOR_EXECUTION TransactionErrorType = 13 + TransactionErrorType_SANITIZE_FAILURE TransactionErrorType = 14 + TransactionErrorType_CLUSTER_MAINTENANCE TransactionErrorType = 15 + TransactionErrorType_ACCOUNT_BORROW_OUTSTANDING_TX TransactionErrorType = 16 + TransactionErrorType_WOULD_EXCEED_MAX_BLOCK_COST_LIMIT TransactionErrorType = 17 + TransactionErrorType_UNSUPPORTED_VERSION TransactionErrorType = 18 + TransactionErrorType_INVALID_WRITABLE_ACCOUNT TransactionErrorType = 19 + TransactionErrorType_WOULD_EXCEED_MAX_ACCOUNT_COST_LIMIT TransactionErrorType = 20 + TransactionErrorType_WOULD_EXCEED_ACCOUNT_DATA_BLOCK_LIMIT TransactionErrorType = 21 + TransactionErrorType_TOO_MANY_ACCOUNT_LOCKS TransactionErrorType = 22 + TransactionErrorType_ADDRESS_LOOKUP_TABLE_NOT_FOUND TransactionErrorType = 23 + TransactionErrorType_INVALID_ADDRESS_LOOKUP_TABLE_OWNER TransactionErrorType = 24 + TransactionErrorType_INVALID_ADDRESS_LOOKUP_TABLE_DATA TransactionErrorType = 25 + TransactionErrorType_INVALID_ADDRESS_LOOKUP_TABLE_INDEX TransactionErrorType = 26 + TransactionErrorType_INVALID_RENT_PAYING_ACCOUNT TransactionErrorType = 27 + TransactionErrorType_WOULD_EXCEED_MAX_VOTE_COST_LIMIT TransactionErrorType = 28 + TransactionErrorType_WOULD_EXCEED_ACCOUNT_DATA_TOTAL_LIMIT TransactionErrorType = 29 + TransactionErrorType_DUPLICATE_INSTRUCTION TransactionErrorType = 30 + TransactionErrorType_INSUFFICIENT_FUNDS_FOR_RENT TransactionErrorType = 31 + TransactionErrorType_MAX_LOADED_ACCOUNTS_DATA_SIZE_EXCEEDED TransactionErrorType = 32 + TransactionErrorType_INVALID_LOADED_ACCOUNTS_DATA_SIZE_LIMIT TransactionErrorType = 33 + TransactionErrorType_RESANITIZATION_NEEDED TransactionErrorType = 34 + TransactionErrorType_PROGRAM_EXECUTION_TEMPORARILY_RESTRICTED TransactionErrorType = 35 + TransactionErrorType_UNBALANCED_TRANSACTION TransactionErrorType = 36 + TransactionErrorType_PROGRAM_CACHE_HIT_MAX_LIMIT TransactionErrorType = 37 +) + +// Enum value maps for TransactionErrorType. +var ( + TransactionErrorType_name = map[int32]string{ + 0: "ACCOUNT_IN_USE", + 1: "ACCOUNT_LOADED_TWICE", + 2: "ACCOUNT_NOT_FOUND", + 3: "PROGRAM_ACCOUNT_NOT_FOUND", + 4: "INSUFFICIENT_FUNDS_FOR_FEE", + 5: "INVALID_ACCOUNT_FOR_FEE", + 6: "ALREADY_PROCESSED", + 7: "BLOCKHASH_NOT_FOUND", + 8: "INSTRUCTION_ERROR", + 9: "CALL_CHAIN_TOO_DEEP", + 10: "MISSING_SIGNATURE_FOR_FEE", + 11: "INVALID_ACCOUNT_INDEX", + 12: "SIGNATURE_FAILURE", + 13: "INVALID_PROGRAM_FOR_EXECUTION", + 14: "SANITIZE_FAILURE", + 15: "CLUSTER_MAINTENANCE", + 16: "ACCOUNT_BORROW_OUTSTANDING_TX", + 17: "WOULD_EXCEED_MAX_BLOCK_COST_LIMIT", + 18: "UNSUPPORTED_VERSION", + 19: "INVALID_WRITABLE_ACCOUNT", + 20: "WOULD_EXCEED_MAX_ACCOUNT_COST_LIMIT", + 21: "WOULD_EXCEED_ACCOUNT_DATA_BLOCK_LIMIT", + 22: "TOO_MANY_ACCOUNT_LOCKS", + 23: "ADDRESS_LOOKUP_TABLE_NOT_FOUND", + 24: "INVALID_ADDRESS_LOOKUP_TABLE_OWNER", + 25: "INVALID_ADDRESS_LOOKUP_TABLE_DATA", + 26: "INVALID_ADDRESS_LOOKUP_TABLE_INDEX", + 27: "INVALID_RENT_PAYING_ACCOUNT", + 28: "WOULD_EXCEED_MAX_VOTE_COST_LIMIT", + 29: "WOULD_EXCEED_ACCOUNT_DATA_TOTAL_LIMIT", + 30: "DUPLICATE_INSTRUCTION", + 31: "INSUFFICIENT_FUNDS_FOR_RENT", + 32: "MAX_LOADED_ACCOUNTS_DATA_SIZE_EXCEEDED", + 33: "INVALID_LOADED_ACCOUNTS_DATA_SIZE_LIMIT", + 34: "RESANITIZATION_NEEDED", + 35: "PROGRAM_EXECUTION_TEMPORARILY_RESTRICTED", + 36: "UNBALANCED_TRANSACTION", + 37: "PROGRAM_CACHE_HIT_MAX_LIMIT", + } +) + +// Enum value maps for InstructionErrorType. +var ( + InstructionErrorType_name = map[int32]string{ + 0: "GENERIC_ERROR", + 1: "INVALID_ARGUMENT", + 2: "INVALID_INSTRUCTION_DATA", + 3: "INVALID_ACCOUNT_DATA", + 4: "ACCOUNT_DATA_TOO_SMALL", + 5: "INSUFFICIENT_FUNDS", + 6: "INCORRECT_PROGRAM_ID", + 7: "MISSING_REQUIRED_SIGNATURE", + 8: "ACCOUNT_ALREADY_INITIALIZED", + 9: "UNINITIALIZED_ACCOUNT", + 10: "UNBALANCED_INSTRUCTION", + 11: "MODIFIED_PROGRAM_ID", + 12: "EXTERNAL_ACCOUNT_LAMPORT_SPEND", + 13: "EXTERNAL_ACCOUNT_DATA_MODIFIED", + 14: "READONLY_LAMPORT_CHANGE", + 15: "READONLY_DATA_MODIFIED", + 16: "DUPLICATE_ACCOUNT_INDEX", + 17: "EXECUTABLE_MODIFIED", + 18: "RENT_EPOCH_MODIFIED", + 19: "NOT_ENOUGH_ACCOUNT_KEYS", + 20: "ACCOUNT_DATA_SIZE_CHANGED", + 21: "ACCOUNT_NOT_EXECUTABLE", + 22: "ACCOUNT_BORROW_FAILED", + 23: "ACCOUNT_BORROW_OUTSTANDING", + 24: "DUPLICATE_ACCOUNT_OUT_OF_SYNC", + 25: "CUSTOM", + 26: "INVALID_ERROR", + 27: "EXECUTABLE_DATA_MODIFIED", + 28: "EXECUTABLE_LAMPORT_CHANGE", + 29: "EXECUTABLE_ACCOUNT_NOT_RENT_EXEMPT", + 30: "UNSUPPORTED_PROGRAM_ID", + 31: "CALL_DEPTH", + 32: "MISSING_ACCOUNT", + 33: "REENTRANCY_NOT_ALLOWED", + 34: "MAX_SEED_LENGTH_EXCEEDED", + 35: "INVALID_SEEDS", + 36: "INVALID_REALLOC", + 37: "COMPUTATIONAL_BUDGET_EXCEEDED", + 38: "PRIVILEGE_ESCALATION", + 39: "PROGRAM_ENVIRONMENT_SETUP_FAILURE", + 40: "PROGRAM_FAILED_TO_COMPLETE", + 41: "PROGRAM_FAILED_TO_COMPILE", + 42: "IMMUTABLE", + 43: "INCORRECT_AUTHORITY", + 44: "BORSH_IO_ERROR", + 45: "ACCOUNT_NOT_RENT_EXEMPT", + 46: "INVALID_ACCOUNT_OWNER", + 47: "ARITHMETIC_OVERFLOW", + 48: "UNSUPPORTED_SYSVAR", + 49: "ILLEGAL_OWNER", + 50: "MAX_ACCOUNTS_DATA_ALLOCATIONS_EXCEEDED", + 51: "MAX_ACCOUNTS_EXCEEDED", + 52: "MAX_INSTRUCTION_TRACE_LENGTH_EXCEEDED", + 53: "BUILTIN_PROGRAMS_MUST_CONSUME_COMPUTE_UNITS", + } +) + +type InstructionErrorType int32 + +const ( + InstructionErrorType_GENERIC_ERROR InstructionErrorType = 0 + InstructionErrorType_INVALID_ARGUMENT InstructionErrorType = 1 + InstructionErrorType_INVALID_INSTRUCTION_DATA InstructionErrorType = 2 + InstructionErrorType_INVALID_ACCOUNT_DATA InstructionErrorType = 3 + InstructionErrorType_ACCOUNT_DATA_TOO_SMALL InstructionErrorType = 4 + InstructionErrorType_INSUFFICIENT_FUNDS InstructionErrorType = 5 + InstructionErrorType_INCORRECT_PROGRAM_ID InstructionErrorType = 6 + InstructionErrorType_MISSING_REQUIRED_SIGNATURE InstructionErrorType = 7 + InstructionErrorType_ACCOUNT_ALREADY_INITIALIZED InstructionErrorType = 8 + InstructionErrorType_UNINITIALIZED_ACCOUNT InstructionErrorType = 9 + InstructionErrorType_UNBALANCED_INSTRUCTION InstructionErrorType = 10 + InstructionErrorType_MODIFIED_PROGRAM_ID InstructionErrorType = 11 + InstructionErrorType_EXTERNAL_ACCOUNT_LAMPORT_SPEND InstructionErrorType = 12 + InstructionErrorType_EXTERNAL_ACCOUNT_DATA_MODIFIED InstructionErrorType = 13 + InstructionErrorType_READONLY_LAMPORT_CHANGE InstructionErrorType = 14 + InstructionErrorType_READONLY_DATA_MODIFIED InstructionErrorType = 15 + InstructionErrorType_DUPLICATE_ACCOUNT_INDEX InstructionErrorType = 16 + InstructionErrorType_EXECUTABLE_MODIFIED InstructionErrorType = 17 + InstructionErrorType_RENT_EPOCH_MODIFIED InstructionErrorType = 18 + InstructionErrorType_NOT_ENOUGH_ACCOUNT_KEYS InstructionErrorType = 19 + InstructionErrorType_ACCOUNT_DATA_SIZE_CHANGED InstructionErrorType = 20 + InstructionErrorType_ACCOUNT_NOT_EXECUTABLE InstructionErrorType = 21 + InstructionErrorType_ACCOUNT_BORROW_FAILED InstructionErrorType = 22 + InstructionErrorType_ACCOUNT_BORROW_OUTSTANDING InstructionErrorType = 23 + InstructionErrorType_DUPLICATE_ACCOUNT_OUT_OF_SYNC InstructionErrorType = 24 + InstructionErrorType_CUSTOM InstructionErrorType = 25 + InstructionErrorType_INVALID_ERROR InstructionErrorType = 26 + InstructionErrorType_EXECUTABLE_DATA_MODIFIED InstructionErrorType = 27 + InstructionErrorType_EXECUTABLE_LAMPORT_CHANGE InstructionErrorType = 28 + InstructionErrorType_EXECUTABLE_ACCOUNT_NOT_RENT_EXEMPT InstructionErrorType = 29 + InstructionErrorType_UNSUPPORTED_PROGRAM_ID InstructionErrorType = 30 + InstructionErrorType_CALL_DEPTH InstructionErrorType = 31 + InstructionErrorType_MISSING_ACCOUNT InstructionErrorType = 32 + InstructionErrorType_REENTRANCY_NOT_ALLOWED InstructionErrorType = 33 + InstructionErrorType_MAX_SEED_LENGTH_EXCEEDED InstructionErrorType = 34 + InstructionErrorType_INVALID_SEEDS InstructionErrorType = 35 + InstructionErrorType_INVALID_REALLOC InstructionErrorType = 36 + InstructionErrorType_COMPUTATIONAL_BUDGET_EXCEEDED InstructionErrorType = 37 + InstructionErrorType_PRIVILEGE_ESCALATION InstructionErrorType = 38 + InstructionErrorType_PROGRAM_ENVIRONMENT_SETUP_FAILURE InstructionErrorType = 39 + InstructionErrorType_PROGRAM_FAILED_TO_COMPLETE InstructionErrorType = 40 + InstructionErrorType_PROGRAM_FAILED_TO_COMPILE InstructionErrorType = 41 + InstructionErrorType_IMMUTABLE InstructionErrorType = 42 + InstructionErrorType_INCORRECT_AUTHORITY InstructionErrorType = 43 + InstructionErrorType_BORSH_IO_ERROR InstructionErrorType = 44 + InstructionErrorType_ACCOUNT_NOT_RENT_EXEMPT InstructionErrorType = 45 + InstructionErrorType_INVALID_ACCOUNT_OWNER InstructionErrorType = 46 + InstructionErrorType_ARITHMETIC_OVERFLOW InstructionErrorType = 47 + InstructionErrorType_UNSUPPORTED_SYSVAR InstructionErrorType = 48 + InstructionErrorType_ILLEGAL_OWNER InstructionErrorType = 49 + InstructionErrorType_MAX_ACCOUNTS_DATA_ALLOCATIONS_EXCEEDED InstructionErrorType = 50 + InstructionErrorType_MAX_ACCOUNTS_EXCEEDED InstructionErrorType = 51 + InstructionErrorType_MAX_INSTRUCTION_TRACE_LENGTH_EXCEEDED InstructionErrorType = 52 + InstructionErrorType_BUILTIN_PROGRAMS_MUST_CONSUME_COMPUTE_UNITS InstructionErrorType = 53 +) + +var fasterJson = jsoniter.ConfigCompatibleWithStandardLibrary + +func ParseTransactionError(v any) (map[string]any, error) { + // TODO: if any of the following fails, return the original value. + // marshal to json + b, err := fasterJson.Marshal(v) + if err != nil { + return nil, err + } + // then unmarshal to map + var m map[string]any + err = fasterJson.Unmarshal(b, &m) + if err != nil { + return nil, err + } + // get the "err" field + errValue, ok := m["err"] + if !ok { + return nil, nil + } + // try to parse base64 + errValueStr, ok := errValue.(string) + if !ok { + return nil, nil + } + b, err = base64.StdEncoding.DecodeString(errValueStr) + if err != nil { + return nil, err + } + /// + { + dec := bin.NewBinDecoder(b) + transactionErrorType, err := dec.ReadUint32(bin.LE) + if err != nil { + return nil, err + } + // TODO: is this uint8 or uvarint or something else? + errorCode, err := dec.ReadUint8() + if err != nil { + return nil, err + } + transactionErrorTypeName, ok := TransactionErrorType_name[int32(transactionErrorType)] + if !ok { + return nil, fmt.Errorf("unknown transaction error type: %d", transactionErrorType) + } + transactionErrorTypeName = bin.ToPascalCase(transactionErrorTypeName) + + switch TransactionErrorType(transactionErrorType) { + case TransactionErrorType_INSTRUCTION_ERROR: + + instructionErrorType, err := dec.ReadUint32(bin.LE) + if err != nil { + return nil, err + } + + instructionErrorTypeName, ok := InstructionErrorType_name[int32(instructionErrorType)] + if !ok { + return nil, fmt.Errorf("unknown instruction error type: %d", instructionErrorType) + } + instructionErrorTypeName = bin.ToPascalCase(instructionErrorTypeName) + + switch InstructionErrorType(instructionErrorType) { + case InstructionErrorType_CUSTOM: + customErrorType, err := dec.ReadUint32(bin.LE) + if err != nil { + return nil, err + } + return map[string]any{ + transactionErrorTypeName: []any{ + errorCode, + map[string]any{ + instructionErrorTypeName: customErrorType, + }, + }, + }, nil + } + + return map[string]any{ + transactionErrorTypeName: []any{ + errorCode, + instructionErrorTypeName, + }, + }, nil + default: + return map[string]any{ + transactionErrorTypeName: []any{ + errorCode, + }, + }, nil + } + } +} diff --git a/solana-tx-meta-parsers/parsers.go b/solana-tx-meta-parsers/parsers.go index 7efd8875..6ed35266 100644 --- a/solana-tx-meta-parsers/parsers.go +++ b/solana-tx-meta-parsers/parsers.go @@ -9,6 +9,52 @@ import ( "google.golang.org/protobuf/proto" ) +type TransactionStatusMetaContainer struct { + vProtobuf *confirmed_block.TransactionStatusMeta + vSerdeLatest *metalatest.TransactionStatusMeta + vSerdeOldest *metaoldest.TransactionStatusMeta +} + +// Ok returns true if the container holds a value. +func (c *TransactionStatusMetaContainer) Ok() bool { + return c.vProtobuf != nil || c.vSerdeLatest != nil || c.vSerdeOldest != nil +} + +// IsEmpty returns true if the container holds no value. +func (c *TransactionStatusMetaContainer) IsEmpty() bool { + return !c.Ok() +} + +// IsProtobuf returns true if the contained value is a protobuf. +func (c *TransactionStatusMetaContainer) IsProtobuf() bool { + return c.vProtobuf != nil +} + +// IsSerdeLatest returns true if the contained value is the latest serde format. +func (c *TransactionStatusMetaContainer) IsSerdeLatest() bool { + return c.vSerdeLatest != nil +} + +// IsSerdeOldest returns true if the contained value is the oldest serde format. +func (c *TransactionStatusMetaContainer) IsSerdeOldest() bool { + return c.vSerdeOldest != nil +} + +// GetProtobuf returns the contained protobuf value. +func (c *TransactionStatusMetaContainer) GetProtobuf() *confirmed_block.TransactionStatusMeta { + return c.vProtobuf +} + +// GetSerdeLatest returns the contained latest serde format value. +func (c *TransactionStatusMetaContainer) GetSerdeLatest() *metalatest.TransactionStatusMeta { + return c.vSerdeLatest +} + +// GetSerdeOldest returns the contained oldest serde format value. +func (c *TransactionStatusMetaContainer) GetSerdeOldest() *metaoldest.TransactionStatusMeta { + return c.vSerdeOldest +} + func ParseTransactionStatusMeta(buf []byte) (*confirmed_block.TransactionStatusMeta, error) { var status confirmed_block.TransactionStatusMeta err := proto.Unmarshal(buf, &status) @@ -54,3 +100,23 @@ func ParseAnyTransactionStatusMeta(buf []byte) (any, error) { } return nil, fmt.Errorf("failed to parse tx meta: %w", err) } + +// ParseTransactionStatusMetaContainer parses the transaction status meta from the given bytes. +// It tries to parse the bytes as the latest protobuf format, then the latest serde format, and finally the oldest serde format. +// It returns a container that holds the parsed value. +func ParseTransactionStatusMetaContainer(buf []byte) (*TransactionStatusMetaContainer, error) { + any, err := ParseAnyTransactionStatusMeta(buf) + if err != nil { + return nil, err + } + container := &TransactionStatusMetaContainer{} + switch v := any.(type) { + case *confirmed_block.TransactionStatusMeta: + container.vProtobuf = v + case *metalatest.TransactionStatusMeta: + container.vSerdeLatest = v + case *metaoldest.TransactionStatusMeta: + container.vSerdeOldest = v + } + return container, nil +} diff --git a/storage.go b/storage.go index 1587580b..f706eb7f 100644 --- a/storage.go +++ b/storage.go @@ -1,7 +1,6 @@ package main import ( - "bytes" "context" "fmt" "strings" @@ -10,10 +9,10 @@ import ( "github.com/gagliardetto/solana-go" "github.com/ipfs/go-cid" carv2 "github.com/ipld/go-car/v2" - cidlink "github.com/ipld/go-ipld-prime/linking/cid" "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode" solanatxmetaparsers "github.com/rpcpool/yellowstone-faithful/solana-tx-meta-parsers" splitcarfetcher "github.com/rpcpool/yellowstone-faithful/split-car-fetcher" + "github.com/rpcpool/yellowstone-faithful/tooling" "golang.org/x/exp/mmap" "k8s.io/klog/v2" ) @@ -123,60 +122,12 @@ type GetTransactionResponse struct { Signatures []solana.Signature `json:"-"` // TODO: enable this } -func loadDataFromDataFrames( - firstDataFrame *ipldbindcode.DataFrame, - dataFrameGetter func(ctx context.Context, wantedCid cid.Cid) (*ipldbindcode.DataFrame, error), -) ([]byte, error) { - dataBuffer := new(bytes.Buffer) - allFrames, err := getAllFramesFromDataFrame(firstDataFrame, dataFrameGetter) - if err != nil { - return nil, err - } - for _, frame := range allFrames { - dataBuffer.Write(frame.Bytes()) - } - // verify the data hash (if present) - bufHash, ok := firstDataFrame.GetHash() - if !ok { - return dataBuffer.Bytes(), nil - } - err = ipldbindcode.VerifyHash(dataBuffer.Bytes(), bufHash) - if err != nil { - return nil, err - } - return dataBuffer.Bytes(), nil -} - -func getAllFramesFromDataFrame( - firstDataFrame *ipldbindcode.DataFrame, - dataFrameGetter func(ctx context.Context, wantedCid cid.Cid) (*ipldbindcode.DataFrame, error), -) ([]*ipldbindcode.DataFrame, error) { - frames := []*ipldbindcode.DataFrame{firstDataFrame} - // get the next data frames - next, ok := firstDataFrame.GetNext() - if !ok || len(next) == 0 { - return frames, nil - } - for _, cid := range next { - nextDataFrame, err := dataFrameGetter(context.Background(), cid.(cidlink.Link).Cid) - if err != nil { - return nil, err - } - nextFrames, err := getAllFramesFromDataFrame(nextDataFrame, dataFrameGetter) - if err != nil { - return nil, err - } - frames = append(frames, nextFrames...) - } - return frames, nil -} - func parseTransactionAndMetaFromNode( transactionNode *ipldbindcode.Transaction, dataFrameGetter func(ctx context.Context, wantedCid cid.Cid) (*ipldbindcode.DataFrame, error), ) (tx solana.Transaction, meta any, _ error) { { - transactionBuffer, err := loadDataFromDataFrames(&transactionNode.Data, dataFrameGetter) + transactionBuffer, err := tooling.LoadDataFromDataFrames(&transactionNode.Data, dataFrameGetter) if err != nil { return solana.Transaction{}, nil, err } @@ -190,12 +141,12 @@ func parseTransactionAndMetaFromNode( } { - metaBuffer, err := loadDataFromDataFrames(&transactionNode.Metadata, dataFrameGetter) + metaBuffer, err := tooling.LoadDataFromDataFrames(&transactionNode.Metadata, dataFrameGetter) if err != nil { return solana.Transaction{}, nil, err } if len(metaBuffer) > 0 { - uncompressedMeta, err := decompressZstd(metaBuffer) + uncompressedMeta, err := tooling.DecompressZstd(metaBuffer) if err != nil { klog.Errorf("failed to decompress metadata: %v", err) return @@ -215,17 +166,17 @@ func getTransactionAndMetaFromNode( transactionNode *ipldbindcode.Transaction, dataFrameGetter func(ctx context.Context, wantedCid cid.Cid) (*ipldbindcode.DataFrame, error), ) ([]byte, []byte, error) { - transactionBuffer, err := loadDataFromDataFrames(&transactionNode.Data, dataFrameGetter) + transactionBuffer, err := tooling.LoadDataFromDataFrames(&transactionNode.Data, dataFrameGetter) if err != nil { return nil, nil, fmt.Errorf("failed to load transaction: %w", err) } - metaBuffer, err := loadDataFromDataFrames(&transactionNode.Metadata, dataFrameGetter) + metaBuffer, err := tooling.LoadDataFromDataFrames(&transactionNode.Metadata, dataFrameGetter) if err != nil { return nil, nil, fmt.Errorf("failed to load metadata: %w", err) } if len(metaBuffer) > 0 { - uncompressedMeta, err := decompressZstd(metaBuffer) + uncompressedMeta, err := tooling.DecompressZstd(metaBuffer) if err != nil { return nil, nil, fmt.Errorf("failed to decompress metadata: %w", err) } diff --git a/third_party/solana_proto/confirmed_block/confirmed_block.pb.go b/third_party/solana_proto/confirmed_block/confirmed_block.pb.go index 1e947a8c..dbef5788 100644 --- a/third_party/solana_proto/confirmed_block/confirmed_block.pb.go +++ b/third_party/solana_proto/confirmed_block/confirmed_block.pb.go @@ -5,8 +5,8 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.31.0 -// protoc v4.23.1 +// protoc-gen-go v1.35.1 +// protoc v5.27.1 // source: confirmed_block.proto package confirmed_block @@ -96,11 +96,9 @@ type ConfirmedBlock struct { func (x *ConfirmedBlock) Reset() { *x = ConfirmedBlock{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ConfirmedBlock) String() string { @@ -111,7 +109,7 @@ func (*ConfirmedBlock) ProtoMessage() {} func (x *ConfirmedBlock) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -186,11 +184,9 @@ type ConfirmedTransaction struct { func (x *ConfirmedTransaction) Reset() { *x = ConfirmedTransaction{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ConfirmedTransaction) String() string { @@ -201,7 +197,7 @@ func (*ConfirmedTransaction) ProtoMessage() {} func (x *ConfirmedTransaction) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -241,11 +237,9 @@ type Transaction struct { func (x *Transaction) Reset() { *x = Transaction{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Transaction) String() string { @@ -256,7 +250,7 @@ func (*Transaction) ProtoMessage() {} func (x *Transaction) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -300,11 +294,9 @@ type Message struct { func (x *Message) Reset() { *x = Message{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Message) String() string { @@ -315,7 +307,7 @@ func (*Message) ProtoMessage() {} func (x *Message) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -384,11 +376,9 @@ type MessageHeader struct { func (x *MessageHeader) Reset() { *x = MessageHeader{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MessageHeader) String() string { @@ -399,7 +389,7 @@ func (*MessageHeader) ProtoMessage() {} func (x *MessageHeader) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -447,11 +437,9 @@ type MessageAddressTableLookup struct { func (x *MessageAddressTableLookup) Reset() { *x = MessageAddressTableLookup{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *MessageAddressTableLookup) String() string { @@ -462,7 +450,7 @@ func (*MessageAddressTableLookup) ProtoMessage() {} func (x *MessageAddressTableLookup) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -526,11 +514,9 @@ type TransactionStatusMeta struct { func (x *TransactionStatusMeta) Reset() { *x = TransactionStatusMeta{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TransactionStatusMeta) String() string { @@ -541,7 +527,7 @@ func (*TransactionStatusMeta) ProtoMessage() {} func (x *TransactionStatusMeta) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -678,11 +664,9 @@ type TransactionError struct { func (x *TransactionError) Reset() { *x = TransactionError{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TransactionError) String() string { @@ -693,7 +677,7 @@ func (*TransactionError) ProtoMessage() {} func (x *TransactionError) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -726,11 +710,9 @@ type InnerInstructions struct { func (x *InnerInstructions) Reset() { *x = InnerInstructions{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[8] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *InnerInstructions) String() string { @@ -741,7 +723,7 @@ func (*InnerInstructions) ProtoMessage() {} func (x *InnerInstructions) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[8] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -786,11 +768,9 @@ type InnerInstruction struct { func (x *InnerInstruction) Reset() { *x = InnerInstruction{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[9] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *InnerInstruction) String() string { @@ -801,7 +781,7 @@ func (*InnerInstruction) ProtoMessage() {} func (x *InnerInstruction) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[9] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -856,11 +836,9 @@ type CompiledInstruction struct { func (x *CompiledInstruction) Reset() { *x = CompiledInstruction{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[10] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *CompiledInstruction) String() string { @@ -871,7 +849,7 @@ func (*CompiledInstruction) ProtoMessage() {} func (x *CompiledInstruction) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[10] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -921,11 +899,9 @@ type TokenBalance struct { func (x *TokenBalance) Reset() { *x = TokenBalance{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[11] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TokenBalance) String() string { @@ -936,7 +912,7 @@ func (*TokenBalance) ProtoMessage() {} func (x *TokenBalance) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[11] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -999,11 +975,9 @@ type UiTokenAmount struct { func (x *UiTokenAmount) Reset() { *x = UiTokenAmount{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[12] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *UiTokenAmount) String() string { @@ -1014,7 +988,7 @@ func (*UiTokenAmount) ProtoMessage() {} func (x *UiTokenAmount) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[12] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1068,11 +1042,9 @@ type ReturnData struct { func (x *ReturnData) Reset() { *x = ReturnData{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[13] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *ReturnData) String() string { @@ -1083,7 +1055,7 @@ func (*ReturnData) ProtoMessage() {} func (x *ReturnData) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[13] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1126,11 +1098,9 @@ type Reward struct { func (x *Reward) Reset() { *x = Reward{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[14] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Reward) String() string { @@ -1141,7 +1111,7 @@ func (*Reward) ProtoMessage() {} func (x *Reward) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[14] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1201,11 +1171,9 @@ type Rewards struct { func (x *Rewards) Reset() { *x = Rewards{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[15] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Rewards) String() string { @@ -1216,7 +1184,7 @@ func (*Rewards) ProtoMessage() {} func (x *Rewards) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[15] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1248,11 +1216,9 @@ type UnixTimestamp struct { func (x *UnixTimestamp) Reset() { *x = UnixTimestamp{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[16] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *UnixTimestamp) String() string { @@ -1263,7 +1229,7 @@ func (*UnixTimestamp) ProtoMessage() {} func (x *UnixTimestamp) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[16] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1295,11 +1261,9 @@ type BlockHeight struct { func (x *BlockHeight) Reset() { *x = BlockHeight{} - if protoimpl.UnsafeEnabled { - mi := &file_confirmed_block_proto_msgTypes[17] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_confirmed_block_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *BlockHeight) String() string { @@ -1310,7 +1274,7 @@ func (*BlockHeight) ProtoMessage() {} func (x *BlockHeight) ProtoReflect() protoreflect.Message { mi := &file_confirmed_block_proto_msgTypes[17] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -1597,7 +1561,7 @@ func file_confirmed_block_proto_rawDescGZIP() []byte { var file_confirmed_block_proto_enumTypes = make([]protoimpl.EnumInfo, 1) var file_confirmed_block_proto_msgTypes = make([]protoimpl.MessageInfo, 18) -var file_confirmed_block_proto_goTypes = []interface{}{ +var file_confirmed_block_proto_goTypes = []any{ (RewardType)(0), // 0: solana.storage.ConfirmedBlock.RewardType (*ConfirmedBlock)(nil), // 1: solana.storage.ConfirmedBlock.ConfirmedBlock (*ConfirmedTransaction)(nil), // 2: solana.storage.ConfirmedBlock.ConfirmedTransaction @@ -1651,226 +1615,8 @@ func file_confirmed_block_proto_init() { if File_confirmed_block_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_confirmed_block_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConfirmedBlock); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConfirmedTransaction); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Transaction); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Message); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MessageHeader); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MessageAddressTableLookup); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TransactionStatusMeta); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TransactionError); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*InnerInstructions); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*InnerInstruction); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CompiledInstruction); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TokenBalance); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UiTokenAmount); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReturnData); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Reward); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Rewards); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UnixTimestamp); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_confirmed_block_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BlockHeight); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } - file_confirmed_block_proto_msgTypes[6].OneofWrappers = []interface{}{} - file_confirmed_block_proto_msgTypes[9].OneofWrappers = []interface{}{} + file_confirmed_block_proto_msgTypes[6].OneofWrappers = []any{} + file_confirmed_block_proto_msgTypes[9].OneofWrappers = []any{} type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/third_party/solana_proto/transaction_by_addr/transaction_by_addr.pb.go b/third_party/solana_proto/transaction_by_addr/transaction_by_addr.pb.go index cb8154ce..b4537eab 100644 --- a/third_party/solana_proto/transaction_by_addr/transaction_by_addr.pb.go +++ b/third_party/solana_proto/transaction_by_addr/transaction_by_addr.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.31.0 -// protoc v4.23.1 +// protoc-gen-go v1.35.1 +// protoc v5.27.1 // source: transaction_by_addr.proto // https://github.com/solana-labs/solana/blob/63baab57fc324937dc1d72b288ba55279c8e793e/storage-proto/proto/transaction_by_addr.proto @@ -61,6 +61,8 @@ const ( TransactionErrorType_INVALID_LOADED_ACCOUNTS_DATA_SIZE_LIMIT TransactionErrorType = 33 TransactionErrorType_RESANITIZATION_NEEDED TransactionErrorType = 34 TransactionErrorType_PROGRAM_EXECUTION_TEMPORARILY_RESTRICTED TransactionErrorType = 35 + TransactionErrorType_UNBALANCED_TRANSACTION TransactionErrorType = 36 + TransactionErrorType_PROGRAM_CACHE_HIT_MAX_LIMIT TransactionErrorType = 37 ) // Enum value maps for TransactionErrorType. @@ -102,6 +104,8 @@ var ( 33: "INVALID_LOADED_ACCOUNTS_DATA_SIZE_LIMIT", 34: "RESANITIZATION_NEEDED", 35: "PROGRAM_EXECUTION_TEMPORARILY_RESTRICTED", + 36: "UNBALANCED_TRANSACTION", + 37: "PROGRAM_CACHE_HIT_MAX_LIMIT", } TransactionErrorType_value = map[string]int32{ "ACCOUNT_IN_USE": 0, @@ -140,6 +144,8 @@ var ( "INVALID_LOADED_ACCOUNTS_DATA_SIZE_LIMIT": 33, "RESANITIZATION_NEEDED": 34, "PROGRAM_EXECUTION_TEMPORARILY_RESTRICTED": 35, + "UNBALANCED_TRANSACTION": 36, + "PROGRAM_CACHE_HIT_MAX_LIMIT": 37, } ) @@ -382,11 +388,9 @@ type TransactionByAddr struct { func (x *TransactionByAddr) Reset() { *x = TransactionByAddr{} - if protoimpl.UnsafeEnabled { - mi := &file_transaction_by_addr_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_transaction_by_addr_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TransactionByAddr) String() string { @@ -397,7 +401,7 @@ func (*TransactionByAddr) ProtoMessage() {} func (x *TransactionByAddr) ProtoReflect() protoreflect.Message { mi := &file_transaction_by_addr_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -433,11 +437,9 @@ type TransactionByAddrInfo struct { func (x *TransactionByAddrInfo) Reset() { *x = TransactionByAddrInfo{} - if protoimpl.UnsafeEnabled { - mi := &file_transaction_by_addr_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_transaction_by_addr_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TransactionByAddrInfo) String() string { @@ -448,7 +450,7 @@ func (*TransactionByAddrInfo) ProtoMessage() {} func (x *TransactionByAddrInfo) ProtoReflect() protoreflect.Message { mi := &file_transaction_by_addr_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -508,11 +510,9 @@ type Memo struct { func (x *Memo) Reset() { *x = Memo{} - if protoimpl.UnsafeEnabled { - mi := &file_transaction_by_addr_proto_msgTypes[2] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_transaction_by_addr_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *Memo) String() string { @@ -523,7 +523,7 @@ func (*Memo) ProtoMessage() {} func (x *Memo) ProtoReflect() protoreflect.Message { mi := &file_transaction_by_addr_proto_msgTypes[2] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -557,11 +557,9 @@ type TransactionError struct { func (x *TransactionError) Reset() { *x = TransactionError{} - if protoimpl.UnsafeEnabled { - mi := &file_transaction_by_addr_proto_msgTypes[3] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_transaction_by_addr_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TransactionError) String() string { @@ -572,7 +570,7 @@ func (*TransactionError) ProtoMessage() {} func (x *TransactionError) ProtoReflect() protoreflect.Message { mi := &file_transaction_by_addr_proto_msgTypes[3] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -620,11 +618,9 @@ type InstructionError struct { func (x *InstructionError) Reset() { *x = InstructionError{} - if protoimpl.UnsafeEnabled { - mi := &file_transaction_by_addr_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_transaction_by_addr_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *InstructionError) String() string { @@ -635,7 +631,7 @@ func (*InstructionError) ProtoMessage() {} func (x *InstructionError) ProtoReflect() protoreflect.Message { mi := &file_transaction_by_addr_proto_msgTypes[4] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -681,11 +677,9 @@ type TransactionDetails struct { func (x *TransactionDetails) Reset() { *x = TransactionDetails{} - if protoimpl.UnsafeEnabled { - mi := &file_transaction_by_addr_proto_msgTypes[5] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_transaction_by_addr_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *TransactionDetails) String() string { @@ -696,7 +690,7 @@ func (*TransactionDetails) ProtoMessage() {} func (x *TransactionDetails) ProtoReflect() protoreflect.Message { mi := &file_transaction_by_addr_proto_msgTypes[5] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -728,11 +722,9 @@ type UnixTimestamp struct { func (x *UnixTimestamp) Reset() { *x = UnixTimestamp{} - if protoimpl.UnsafeEnabled { - mi := &file_transaction_by_addr_proto_msgTypes[6] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_transaction_by_addr_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *UnixTimestamp) String() string { @@ -743,7 +735,7 @@ func (*UnixTimestamp) ProtoMessage() {} func (x *UnixTimestamp) ProtoReflect() protoreflect.Message { mi := &file_transaction_by_addr_proto_msgTypes[6] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -775,11 +767,9 @@ type CustomError struct { func (x *CustomError) Reset() { *x = CustomError{} - if protoimpl.UnsafeEnabled { - mi := &file_transaction_by_addr_proto_msgTypes[7] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } + mi := &file_transaction_by_addr_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) } func (x *CustomError) String() string { @@ -790,7 +780,7 @@ func (*CustomError) ProtoMessage() {} func (x *CustomError) ProtoReflect() protoreflect.Message { mi := &file_transaction_by_addr_proto_msgTypes[7] - if protoimpl.UnsafeEnabled && x != nil { + if x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { ms.StoreMessageInfo(mi) @@ -885,7 +875,7 @@ var file_transaction_by_addr_proto_rawDesc = []byte{ 0x61, 0x6d, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x25, 0x0a, 0x0b, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x06, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x2a, 0x96, 0x09, 0x0a, 0x14, + 0x01, 0x28, 0x0d, 0x52, 0x06, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x2a, 0xd3, 0x09, 0x0a, 0x14, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, 0x0e, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x55, 0x53, 0x45, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x41, 0x43, 0x43, 0x4f, @@ -959,109 +949,113 @@ var file_transaction_by_addr_proto_rawDesc = []byte{ 0x45, 0x44, 0x45, 0x44, 0x10, 0x22, 0x12, 0x2c, 0x0a, 0x28, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x45, 0x4d, 0x50, 0x4f, 0x52, 0x41, 0x52, 0x49, 0x4c, 0x59, 0x5f, 0x52, 0x45, 0x53, 0x54, 0x52, 0x49, 0x43, 0x54, - 0x45, 0x44, 0x10, 0x23, 0x2a, 0x81, 0x0c, 0x0a, 0x14, 0x49, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, - 0x0d, 0x47, 0x45, 0x4e, 0x45, 0x52, 0x49, 0x43, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x00, - 0x12, 0x14, 0x0a, 0x10, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x41, 0x52, 0x47, 0x55, - 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x01, 0x12, 0x1c, 0x0a, 0x18, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, - 0x44, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x52, 0x55, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x41, - 0x54, 0x41, 0x10, 0x02, 0x12, 0x18, 0x0a, 0x14, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, - 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x10, 0x03, 0x12, 0x1a, - 0x0a, 0x16, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x54, - 0x4f, 0x4f, 0x5f, 0x53, 0x4d, 0x41, 0x4c, 0x4c, 0x10, 0x04, 0x12, 0x16, 0x0a, 0x12, 0x49, 0x4e, - 0x53, 0x55, 0x46, 0x46, 0x49, 0x43, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x46, 0x55, 0x4e, 0x44, 0x53, - 0x10, 0x05, 0x12, 0x18, 0x0a, 0x14, 0x49, 0x4e, 0x43, 0x4f, 0x52, 0x52, 0x45, 0x43, 0x54, 0x5f, - 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x49, 0x44, 0x10, 0x06, 0x12, 0x1e, 0x0a, 0x1a, - 0x4d, 0x49, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, - 0x5f, 0x53, 0x49, 0x47, 0x4e, 0x41, 0x54, 0x55, 0x52, 0x45, 0x10, 0x07, 0x12, 0x1f, 0x0a, 0x1b, - 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x41, 0x4c, 0x52, 0x45, 0x41, 0x44, 0x59, 0x5f, - 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x10, 0x08, 0x12, 0x19, 0x0a, - 0x15, 0x55, 0x4e, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x41, - 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x09, 0x12, 0x1a, 0x0a, 0x16, 0x55, 0x4e, 0x42, 0x41, - 0x4c, 0x41, 0x4e, 0x43, 0x45, 0x44, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x52, 0x55, 0x43, 0x54, 0x49, - 0x4f, 0x4e, 0x10, 0x0a, 0x12, 0x17, 0x0a, 0x13, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, - 0x5f, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x49, 0x44, 0x10, 0x0b, 0x12, 0x22, 0x0a, - 0x1e, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, - 0x54, 0x5f, 0x4c, 0x41, 0x4d, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x4e, 0x44, 0x10, - 0x0c, 0x12, 0x22, 0x0a, 0x1e, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x41, 0x43, - 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, - 0x49, 0x45, 0x44, 0x10, 0x0d, 0x12, 0x1b, 0x0a, 0x17, 0x52, 0x45, 0x41, 0x44, 0x4f, 0x4e, 0x4c, - 0x59, 0x5f, 0x4c, 0x41, 0x4d, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, - 0x10, 0x0e, 0x12, 0x1a, 0x0a, 0x16, 0x52, 0x45, 0x41, 0x44, 0x4f, 0x4e, 0x4c, 0x59, 0x5f, 0x44, - 0x41, 0x54, 0x41, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x0f, 0x12, 0x1b, - 0x0a, 0x17, 0x44, 0x55, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x43, 0x43, 0x4f, - 0x55, 0x4e, 0x54, 0x5f, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, 0x10, 0x12, 0x17, 0x0a, 0x13, 0x45, - 0x58, 0x45, 0x43, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, - 0x45, 0x44, 0x10, 0x11, 0x12, 0x17, 0x0a, 0x13, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x45, 0x50, 0x4f, - 0x43, 0x48, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x12, 0x12, 0x1b, 0x0a, - 0x17, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x4e, 0x4f, 0x55, 0x47, 0x48, 0x5f, 0x41, 0x43, 0x43, 0x4f, - 0x55, 0x4e, 0x54, 0x5f, 0x4b, 0x45, 0x59, 0x53, 0x10, 0x13, 0x12, 0x1d, 0x0a, 0x19, 0x41, 0x43, - 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x5f, - 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x44, 0x10, 0x14, 0x12, 0x1a, 0x0a, 0x16, 0x41, 0x43, 0x43, - 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x41, - 0x42, 0x4c, 0x45, 0x10, 0x15, 0x12, 0x19, 0x0a, 0x15, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, - 0x5f, 0x42, 0x4f, 0x52, 0x52, 0x4f, 0x57, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x16, - 0x12, 0x1e, 0x0a, 0x1a, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x42, 0x4f, 0x52, 0x52, - 0x4f, 0x57, 0x5f, 0x4f, 0x55, 0x54, 0x53, 0x54, 0x41, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x17, - 0x12, 0x21, 0x0a, 0x1d, 0x44, 0x55, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x43, - 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x4f, 0x55, 0x54, 0x5f, 0x4f, 0x46, 0x5f, 0x53, 0x59, 0x4e, - 0x43, 0x10, 0x18, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x55, 0x53, 0x54, 0x4f, 0x4d, 0x10, 0x19, 0x12, - 0x11, 0x0a, 0x0d, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, - 0x10, 0x1a, 0x12, 0x1c, 0x0a, 0x18, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, - 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x1b, - 0x12, 0x1d, 0x0a, 0x19, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4c, - 0x41, 0x4d, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, 0x1c, 0x12, - 0x26, 0x0a, 0x22, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x41, 0x43, - 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x45, - 0x58, 0x45, 0x4d, 0x50, 0x54, 0x10, 0x1d, 0x12, 0x1a, 0x0a, 0x16, 0x55, 0x4e, 0x53, 0x55, 0x50, - 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x49, - 0x44, 0x10, 0x1e, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x41, 0x4c, 0x4c, 0x5f, 0x44, 0x45, 0x50, 0x54, - 0x48, 0x10, 0x1f, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x49, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x5f, 0x41, - 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x20, 0x12, 0x1a, 0x0a, 0x16, 0x52, 0x45, 0x45, 0x4e, - 0x54, 0x52, 0x41, 0x4e, 0x43, 0x59, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x41, 0x4c, 0x4c, 0x4f, 0x57, - 0x45, 0x44, 0x10, 0x21, 0x12, 0x1c, 0x0a, 0x18, 0x4d, 0x41, 0x58, 0x5f, 0x53, 0x45, 0x45, 0x44, - 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, - 0x10, 0x22, 0x12, 0x11, 0x0a, 0x0d, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x53, 0x45, - 0x45, 0x44, 0x53, 0x10, 0x23, 0x12, 0x13, 0x0a, 0x0f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, - 0x5f, 0x52, 0x45, 0x41, 0x4c, 0x4c, 0x4f, 0x43, 0x10, 0x24, 0x12, 0x21, 0x0a, 0x1d, 0x43, 0x4f, - 0x4d, 0x50, 0x55, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x41, 0x4c, 0x5f, 0x42, 0x55, 0x44, 0x47, - 0x45, 0x54, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x25, 0x12, 0x18, 0x0a, - 0x14, 0x50, 0x52, 0x49, 0x56, 0x49, 0x4c, 0x45, 0x47, 0x45, 0x5f, 0x45, 0x53, 0x43, 0x41, 0x4c, - 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x26, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x52, 0x4f, 0x47, 0x52, - 0x41, 0x4d, 0x5f, 0x45, 0x4e, 0x56, 0x49, 0x52, 0x4f, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, - 0x45, 0x54, 0x55, 0x50, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x55, 0x52, 0x45, 0x10, 0x27, 0x12, 0x1e, - 0x0a, 0x1a, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, - 0x5f, 0x54, 0x4f, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x28, 0x12, 0x1d, - 0x0a, 0x19, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, - 0x5f, 0x54, 0x4f, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, 0x45, 0x10, 0x29, 0x12, 0x0d, 0x0a, - 0x09, 0x49, 0x4d, 0x4d, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x2a, 0x12, 0x17, 0x0a, 0x13, - 0x49, 0x4e, 0x43, 0x4f, 0x52, 0x52, 0x45, 0x43, 0x54, 0x5f, 0x41, 0x55, 0x54, 0x48, 0x4f, 0x52, - 0x49, 0x54, 0x59, 0x10, 0x2b, 0x12, 0x12, 0x0a, 0x0e, 0x42, 0x4f, 0x52, 0x53, 0x48, 0x5f, 0x49, - 0x4f, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x2c, 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x43, 0x43, - 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x45, 0x58, - 0x45, 0x4d, 0x50, 0x54, 0x10, 0x2d, 0x12, 0x19, 0x0a, 0x15, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, - 0x44, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x4f, 0x57, 0x4e, 0x45, 0x52, 0x10, - 0x2e, 0x12, 0x17, 0x0a, 0x13, 0x41, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x45, 0x54, 0x49, 0x43, 0x5f, - 0x4f, 0x56, 0x45, 0x52, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x2f, 0x12, 0x16, 0x0a, 0x12, 0x55, 0x4e, - 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x53, 0x59, 0x53, 0x56, 0x41, 0x52, - 0x10, 0x30, 0x12, 0x11, 0x0a, 0x0d, 0x49, 0x4c, 0x4c, 0x45, 0x47, 0x41, 0x4c, 0x5f, 0x4f, 0x57, - 0x4e, 0x45, 0x52, 0x10, 0x31, 0x12, 0x2a, 0x0a, 0x26, 0x4d, 0x41, 0x58, 0x5f, 0x41, 0x43, 0x43, - 0x4f, 0x55, 0x4e, 0x54, 0x53, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x41, 0x4c, 0x4c, 0x4f, 0x43, - 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, - 0x32, 0x12, 0x19, 0x0a, 0x15, 0x4d, 0x41, 0x58, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, - 0x53, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x33, 0x12, 0x29, 0x0a, 0x25, - 0x4d, 0x41, 0x58, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x52, 0x55, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, - 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x5f, 0x45, 0x58, 0x43, - 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x34, 0x12, 0x2f, 0x0a, 0x2b, 0x42, 0x55, 0x49, 0x4c, 0x54, - 0x49, 0x4e, 0x5f, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x53, 0x5f, 0x4d, 0x55, 0x53, 0x54, - 0x5f, 0x43, 0x4f, 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x55, 0x54, 0x45, - 0x5f, 0x55, 0x4e, 0x49, 0x54, 0x53, 0x10, 0x35, 0x42, 0x56, 0x5a, 0x54, 0x67, 0x69, 0x74, 0x68, - 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, 0x70, 0x63, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x79, - 0x65, 0x6c, 0x6c, 0x6f, 0x77, 0x73, 0x74, 0x6f, 0x6e, 0x65, 0x2d, 0x66, 0x61, 0x69, 0x74, 0x68, - 0x66, 0x75, 0x6c, 0x2f, 0x74, 0x68, 0x69, 0x72, 0x64, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x79, 0x2f, - 0x73, 0x6f, 0x6c, 0x61, 0x6e, 0x61, 0x5f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x3b, 0x74, 0x72, 0x61, - 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x62, 0x79, 0x5f, 0x61, 0x64, 0x64, 0x72, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x45, 0x44, 0x10, 0x23, 0x12, 0x1a, 0x0a, 0x16, 0x55, 0x4e, 0x42, 0x41, 0x4c, 0x41, 0x4e, 0x43, + 0x45, 0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x24, + 0x12, 0x1f, 0x0a, 0x1b, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x43, 0x41, 0x43, 0x48, + 0x45, 0x5f, 0x48, 0x49, 0x54, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x4c, 0x49, 0x4d, 0x49, 0x54, 0x10, + 0x25, 0x2a, 0x81, 0x0c, 0x0a, 0x14, 0x49, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x47, 0x45, + 0x4e, 0x45, 0x52, 0x49, 0x43, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x00, 0x12, 0x14, 0x0a, + 0x10, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x41, 0x52, 0x47, 0x55, 0x4d, 0x45, 0x4e, + 0x54, 0x10, 0x01, 0x12, 0x1c, 0x0a, 0x18, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x49, + 0x4e, 0x53, 0x54, 0x52, 0x55, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x10, + 0x02, 0x12, 0x18, 0x0a, 0x14, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x41, 0x43, 0x43, + 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x10, 0x03, 0x12, 0x1a, 0x0a, 0x16, 0x41, + 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x54, 0x4f, 0x4f, 0x5f, + 0x53, 0x4d, 0x41, 0x4c, 0x4c, 0x10, 0x04, 0x12, 0x16, 0x0a, 0x12, 0x49, 0x4e, 0x53, 0x55, 0x46, + 0x46, 0x49, 0x43, 0x49, 0x45, 0x4e, 0x54, 0x5f, 0x46, 0x55, 0x4e, 0x44, 0x53, 0x10, 0x05, 0x12, + 0x18, 0x0a, 0x14, 0x49, 0x4e, 0x43, 0x4f, 0x52, 0x52, 0x45, 0x43, 0x54, 0x5f, 0x50, 0x52, 0x4f, + 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x49, 0x44, 0x10, 0x06, 0x12, 0x1e, 0x0a, 0x1a, 0x4d, 0x49, 0x53, + 0x53, 0x49, 0x4e, 0x47, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, 0x5f, 0x53, 0x49, + 0x47, 0x4e, 0x41, 0x54, 0x55, 0x52, 0x45, 0x10, 0x07, 0x12, 0x1f, 0x0a, 0x1b, 0x41, 0x43, 0x43, + 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x41, 0x4c, 0x52, 0x45, 0x41, 0x44, 0x59, 0x5f, 0x49, 0x4e, 0x49, + 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x10, 0x08, 0x12, 0x19, 0x0a, 0x15, 0x55, 0x4e, + 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x41, 0x43, 0x43, 0x4f, + 0x55, 0x4e, 0x54, 0x10, 0x09, 0x12, 0x1a, 0x0a, 0x16, 0x55, 0x4e, 0x42, 0x41, 0x4c, 0x41, 0x4e, + 0x43, 0x45, 0x44, 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x52, 0x55, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, + 0x0a, 0x12, 0x17, 0x0a, 0x13, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x5f, 0x50, 0x52, + 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x49, 0x44, 0x10, 0x0b, 0x12, 0x22, 0x0a, 0x1e, 0x45, 0x58, + 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x4c, + 0x41, 0x4d, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x0c, 0x12, 0x22, + 0x0a, 0x1e, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, + 0x4e, 0x54, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, + 0x10, 0x0d, 0x12, 0x1b, 0x0a, 0x17, 0x52, 0x45, 0x41, 0x44, 0x4f, 0x4e, 0x4c, 0x59, 0x5f, 0x4c, + 0x41, 0x4d, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, 0x0e, 0x12, + 0x1a, 0x0a, 0x16, 0x52, 0x45, 0x41, 0x44, 0x4f, 0x4e, 0x4c, 0x59, 0x5f, 0x44, 0x41, 0x54, 0x41, + 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x0f, 0x12, 0x1b, 0x0a, 0x17, 0x44, + 0x55, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, + 0x5f, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, 0x10, 0x12, 0x17, 0x0a, 0x13, 0x45, 0x58, 0x45, 0x43, + 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, + 0x11, 0x12, 0x17, 0x0a, 0x13, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x45, 0x50, 0x4f, 0x43, 0x48, 0x5f, + 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x12, 0x12, 0x1b, 0x0a, 0x17, 0x4e, 0x4f, + 0x54, 0x5f, 0x45, 0x4e, 0x4f, 0x55, 0x47, 0x48, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, + 0x5f, 0x4b, 0x45, 0x59, 0x53, 0x10, 0x13, 0x12, 0x1d, 0x0a, 0x19, 0x41, 0x43, 0x43, 0x4f, 0x55, + 0x4e, 0x54, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x5f, 0x43, 0x48, 0x41, + 0x4e, 0x47, 0x45, 0x44, 0x10, 0x14, 0x12, 0x1a, 0x0a, 0x16, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, + 0x54, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, + 0x10, 0x15, 0x12, 0x19, 0x0a, 0x15, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x42, 0x4f, + 0x52, 0x52, 0x4f, 0x57, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x16, 0x12, 0x1e, 0x0a, + 0x1a, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x42, 0x4f, 0x52, 0x52, 0x4f, 0x57, 0x5f, + 0x4f, 0x55, 0x54, 0x53, 0x54, 0x41, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x17, 0x12, 0x21, 0x0a, + 0x1d, 0x44, 0x55, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, + 0x4e, 0x54, 0x5f, 0x4f, 0x55, 0x54, 0x5f, 0x4f, 0x46, 0x5f, 0x53, 0x59, 0x4e, 0x43, 0x10, 0x18, + 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x55, 0x53, 0x54, 0x4f, 0x4d, 0x10, 0x19, 0x12, 0x11, 0x0a, 0x0d, + 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x1a, 0x12, + 0x1c, 0x0a, 0x18, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x44, 0x41, + 0x54, 0x41, 0x5f, 0x4d, 0x4f, 0x44, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x1b, 0x12, 0x1d, 0x0a, + 0x19, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x4c, 0x41, 0x4d, 0x50, + 0x4f, 0x52, 0x54, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, 0x1c, 0x12, 0x26, 0x0a, 0x22, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, + 0x4e, 0x54, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x45, 0x58, 0x45, 0x4d, + 0x50, 0x54, 0x10, 0x1d, 0x12, 0x1a, 0x0a, 0x16, 0x55, 0x4e, 0x53, 0x55, 0x50, 0x50, 0x4f, 0x52, + 0x54, 0x45, 0x44, 0x5f, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x49, 0x44, 0x10, 0x1e, + 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x41, 0x4c, 0x4c, 0x5f, 0x44, 0x45, 0x50, 0x54, 0x48, 0x10, 0x1f, + 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x49, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x5f, 0x41, 0x43, 0x43, 0x4f, + 0x55, 0x4e, 0x54, 0x10, 0x20, 0x12, 0x1a, 0x0a, 0x16, 0x52, 0x45, 0x45, 0x4e, 0x54, 0x52, 0x41, + 0x4e, 0x43, 0x59, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x41, 0x4c, 0x4c, 0x4f, 0x57, 0x45, 0x44, 0x10, + 0x21, 0x12, 0x1c, 0x0a, 0x18, 0x4d, 0x41, 0x58, 0x5f, 0x53, 0x45, 0x45, 0x44, 0x5f, 0x4c, 0x45, + 0x4e, 0x47, 0x54, 0x48, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x22, 0x12, + 0x11, 0x0a, 0x0d, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x53, 0x45, 0x45, 0x44, 0x53, + 0x10, 0x23, 0x12, 0x13, 0x0a, 0x0f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x52, 0x45, + 0x41, 0x4c, 0x4c, 0x4f, 0x43, 0x10, 0x24, 0x12, 0x21, 0x0a, 0x1d, 0x43, 0x4f, 0x4d, 0x50, 0x55, + 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x41, 0x4c, 0x5f, 0x42, 0x55, 0x44, 0x47, 0x45, 0x54, 0x5f, + 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x25, 0x12, 0x18, 0x0a, 0x14, 0x50, 0x52, + 0x49, 0x56, 0x49, 0x4c, 0x45, 0x47, 0x45, 0x5f, 0x45, 0x53, 0x43, 0x41, 0x4c, 0x41, 0x54, 0x49, + 0x4f, 0x4e, 0x10, 0x26, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, + 0x45, 0x4e, 0x56, 0x49, 0x52, 0x4f, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x45, 0x54, 0x55, + 0x50, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x55, 0x52, 0x45, 0x10, 0x27, 0x12, 0x1e, 0x0a, 0x1a, 0x50, + 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x54, 0x4f, + 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x28, 0x12, 0x1d, 0x0a, 0x19, 0x50, + 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x5f, 0x54, 0x4f, + 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x49, 0x4c, 0x45, 0x10, 0x29, 0x12, 0x0d, 0x0a, 0x09, 0x49, 0x4d, + 0x4d, 0x55, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x2a, 0x12, 0x17, 0x0a, 0x13, 0x49, 0x4e, 0x43, + 0x4f, 0x52, 0x52, 0x45, 0x43, 0x54, 0x5f, 0x41, 0x55, 0x54, 0x48, 0x4f, 0x52, 0x49, 0x54, 0x59, + 0x10, 0x2b, 0x12, 0x12, 0x0a, 0x0e, 0x42, 0x4f, 0x52, 0x53, 0x48, 0x5f, 0x49, 0x4f, 0x5f, 0x45, + 0x52, 0x52, 0x4f, 0x52, 0x10, 0x2c, 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, + 0x54, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x52, 0x45, 0x4e, 0x54, 0x5f, 0x45, 0x58, 0x45, 0x4d, 0x50, + 0x54, 0x10, 0x2d, 0x12, 0x19, 0x0a, 0x15, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x5f, 0x41, + 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x5f, 0x4f, 0x57, 0x4e, 0x45, 0x52, 0x10, 0x2e, 0x12, 0x17, + 0x0a, 0x13, 0x41, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x45, 0x54, 0x49, 0x43, 0x5f, 0x4f, 0x56, 0x45, + 0x52, 0x46, 0x4c, 0x4f, 0x57, 0x10, 0x2f, 0x12, 0x16, 0x0a, 0x12, 0x55, 0x4e, 0x53, 0x55, 0x50, + 0x50, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f, 0x53, 0x59, 0x53, 0x56, 0x41, 0x52, 0x10, 0x30, 0x12, + 0x11, 0x0a, 0x0d, 0x49, 0x4c, 0x4c, 0x45, 0x47, 0x41, 0x4c, 0x5f, 0x4f, 0x57, 0x4e, 0x45, 0x52, + 0x10, 0x31, 0x12, 0x2a, 0x0a, 0x26, 0x4d, 0x41, 0x58, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, + 0x54, 0x53, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x41, 0x4c, 0x4c, 0x4f, 0x43, 0x41, 0x54, 0x49, + 0x4f, 0x4e, 0x53, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x32, 0x12, 0x19, + 0x0a, 0x15, 0x4d, 0x41, 0x58, 0x5f, 0x41, 0x43, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x53, 0x5f, 0x45, + 0x58, 0x43, 0x45, 0x45, 0x44, 0x45, 0x44, 0x10, 0x33, 0x12, 0x29, 0x0a, 0x25, 0x4d, 0x41, 0x58, + 0x5f, 0x49, 0x4e, 0x53, 0x54, 0x52, 0x55, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, + 0x43, 0x45, 0x5f, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x5f, 0x45, 0x58, 0x43, 0x45, 0x45, 0x44, + 0x45, 0x44, 0x10, 0x34, 0x12, 0x2f, 0x0a, 0x2b, 0x42, 0x55, 0x49, 0x4c, 0x54, 0x49, 0x4e, 0x5f, + 0x50, 0x52, 0x4f, 0x47, 0x52, 0x41, 0x4d, 0x53, 0x5f, 0x4d, 0x55, 0x53, 0x54, 0x5f, 0x43, 0x4f, + 0x4e, 0x53, 0x55, 0x4d, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x55, 0x54, 0x45, 0x5f, 0x55, 0x4e, + 0x49, 0x54, 0x53, 0x10, 0x35, 0x42, 0x56, 0x5a, 0x54, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, + 0x63, 0x6f, 0x6d, 0x2f, 0x72, 0x70, 0x63, 0x70, 0x6f, 0x6f, 0x6c, 0x2f, 0x79, 0x65, 0x6c, 0x6c, + 0x6f, 0x77, 0x73, 0x74, 0x6f, 0x6e, 0x65, 0x2d, 0x66, 0x61, 0x69, 0x74, 0x68, 0x66, 0x75, 0x6c, + 0x2f, 0x74, 0x68, 0x69, 0x72, 0x64, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x79, 0x2f, 0x73, 0x6f, 0x6c, + 0x61, 0x6e, 0x61, 0x5f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x3b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x62, 0x79, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1078,7 +1072,7 @@ func file_transaction_by_addr_proto_rawDescGZIP() []byte { var file_transaction_by_addr_proto_enumTypes = make([]protoimpl.EnumInfo, 2) var file_transaction_by_addr_proto_msgTypes = make([]protoimpl.MessageInfo, 8) -var file_transaction_by_addr_proto_goTypes = []interface{}{ +var file_transaction_by_addr_proto_goTypes = []any{ (TransactionErrorType)(0), // 0: solana.storage.TransactionByAddr.TransactionErrorType (InstructionErrorType)(0), // 1: solana.storage.TransactionByAddr.InstructionErrorType (*TransactionByAddr)(nil), // 2: solana.storage.TransactionByAddr.TransactionByAddr @@ -1112,104 +1106,6 @@ func file_transaction_by_addr_proto_init() { if File_transaction_by_addr_proto != nil { return } - if !protoimpl.UnsafeEnabled { - file_transaction_by_addr_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TransactionByAddr); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_transaction_by_addr_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TransactionByAddrInfo); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_transaction_by_addr_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Memo); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_transaction_by_addr_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TransactionError); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_transaction_by_addr_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*InstructionError); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_transaction_by_addr_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TransactionDetails); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_transaction_by_addr_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UnixTimestamp); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_transaction_by_addr_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CustomError); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/third_party/solana_proto/transaction_by_addr/transaction_by_addr.proto b/third_party/solana_proto/transaction_by_addr/transaction_by_addr.proto index 01718457..21634794 100644 --- a/third_party/solana_proto/transaction_by_addr/transaction_by_addr.proto +++ b/third_party/solana_proto/transaction_by_addr/transaction_by_addr.proto @@ -65,6 +65,8 @@ enum TransactionErrorType { INVALID_LOADED_ACCOUNTS_DATA_SIZE_LIMIT = 33; RESANITIZATION_NEEDED = 34; PROGRAM_EXECUTION_TEMPORARILY_RESTRICTED = 35; + UNBALANCED_TRANSACTION = 36; + PROGRAM_CACHE_HIT_MAX_LIMIT = 37; } message InstructionError { diff --git a/tooling/compress.go b/tooling/compress.go new file mode 100644 index 00000000..b0402ef3 --- /dev/null +++ b/tooling/compress.go @@ -0,0 +1,38 @@ +package tooling + +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/tooling/data-frames.go b/tooling/data-frames.go new file mode 100644 index 00000000..9cbc6e9b --- /dev/null +++ b/tooling/data-frames.go @@ -0,0 +1,58 @@ +package tooling + +import ( + "bytes" + "context" + + "github.com/ipfs/go-cid" + cidlink "github.com/ipld/go-ipld-prime/linking/cid" + "github.com/rpcpool/yellowstone-faithful/ipld/ipldbindcode" +) + +func LoadDataFromDataFrames( + firstDataFrame *ipldbindcode.DataFrame, + dataFrameGetter func(ctx context.Context, wantedCid cid.Cid) (*ipldbindcode.DataFrame, error), +) ([]byte, error) { + dataBuffer := new(bytes.Buffer) + allFrames, err := getAllFramesFromDataFrame(firstDataFrame, dataFrameGetter) + if err != nil { + return nil, err + } + for _, frame := range allFrames { + dataBuffer.Write(frame.Bytes()) + } + // verify the data hash (if present) + bufHash, ok := firstDataFrame.GetHash() + if !ok { + return dataBuffer.Bytes(), nil + } + err = ipldbindcode.VerifyHash(dataBuffer.Bytes(), bufHash) + if err != nil { + return nil, err + } + return dataBuffer.Bytes(), nil +} + +func getAllFramesFromDataFrame( + firstDataFrame *ipldbindcode.DataFrame, + dataFrameGetter func(ctx context.Context, wantedCid cid.Cid) (*ipldbindcode.DataFrame, error), +) ([]*ipldbindcode.DataFrame, error) { + frames := []*ipldbindcode.DataFrame{firstDataFrame} + // get the next data frames + next, ok := firstDataFrame.GetNext() + if !ok || len(next) == 0 { + return frames, nil + } + for _, cid := range next { + nextDataFrame, err := dataFrameGetter(context.Background(), cid.(cidlink.Link).Cid) + if err != nil { + return nil, err + } + nextFrames, err := getAllFramesFromDataFrame(nextDataFrame, dataFrameGetter) + if err != nil { + return nil, err + } + frames = append(frames, nextFrames...) + } + return frames, nil +} diff --git a/tooling/files.go b/tooling/files.go new file mode 100644 index 00000000..6b504567 --- /dev/null +++ b/tooling/files.go @@ -0,0 +1,36 @@ +package tooling + +import ( + "bufio" + "os" +) + +type BufferedWritableFile struct { + file *os.File + buf *bufio.Writer +} + +// NewBufferedWritableFile creates a new file for writing, with a buffer. +// The file is created at the given path; if the file already exists, it will be overwritten. +func NewBufferedWritableFile(path string) (*BufferedWritableFile, error) { + file, err := os.Create(path) + if err != nil { + return nil, err + } + return &BufferedWritableFile{ + file: file, + buf: bufio.NewWriterSize(file, 1024*1024), + }, nil +} + +func (bwf *BufferedWritableFile) WriteString(s string) error { + _, err := bwf.buf.WriteString(s) + return err +} + +func (bwf *BufferedWritableFile) Close() error { + if err := bwf.buf.Flush(); err != nil { + return err + } + return bwf.file.Close() +}