Unverified Commit 794c6133 authored by Martin Holst Swende's avatar Martin Holst Swende Committed by GitHub

core/rawdb: freezer batch write (#23462)

This change is a rewrite of the freezer code.

When writing ancient chain data to the freezer, the previous version first encoded each
individual item to a temporary buffer, then wrote the buffer. For small item sizes (for
example, in the block hash freezer table), this strategy causes a lot of system calls for
writing tiny chunks of data. It also allocated a lot of temporary []byte buffers.

In the new version, we instead encode multiple items into a re-useable batch buffer, which
is then written to the file all at once. This avoids performing a system call for every
inserted item.

To make the internal batching work, the ancient database API had to be changed. While
integrating this new API in BlockChain.InsertReceiptChain, additional optimizations were
also added there.
Co-authored-by: 's avatarFelix Lange <fjl@twurst.com>
parent 9a0df80b
This diff is collapsed.
...@@ -670,6 +670,7 @@ func TestFastVsFullChains(t *testing.T) { ...@@ -670,6 +670,7 @@ func TestFastVsFullChains(t *testing.T) {
if n, err := ancient.InsertReceiptChain(blocks, receipts, uint64(len(blocks)/2)); err != nil { if n, err := ancient.InsertReceiptChain(blocks, receipts, uint64(len(blocks)/2)); err != nil {
t.Fatalf("failed to insert receipt %d: %v", n, err) t.Fatalf("failed to insert receipt %d: %v", n, err)
} }
// Iterate over all chain data components, and cross reference // Iterate over all chain data components, and cross reference
for i := 0; i < len(blocks); i++ { for i := 0; i < len(blocks); i++ {
num, hash := blocks[i].NumberU64(), blocks[i].Hash() num, hash := blocks[i].NumberU64(), blocks[i].Hash()
...@@ -693,10 +694,27 @@ func TestFastVsFullChains(t *testing.T) { ...@@ -693,10 +694,27 @@ func TestFastVsFullChains(t *testing.T) {
} else if types.CalcUncleHash(fblock.Uncles()) != types.CalcUncleHash(arblock.Uncles()) || types.CalcUncleHash(anblock.Uncles()) != types.CalcUncleHash(arblock.Uncles()) { } else if types.CalcUncleHash(fblock.Uncles()) != types.CalcUncleHash(arblock.Uncles()) || types.CalcUncleHash(anblock.Uncles()) != types.CalcUncleHash(arblock.Uncles()) {
t.Errorf("block #%d [%x]: uncles mismatch: fastdb %v, ancientdb %v, archivedb %v", num, hash, fblock.Uncles(), anblock, arblock.Uncles()) t.Errorf("block #%d [%x]: uncles mismatch: fastdb %v, ancientdb %v, archivedb %v", num, hash, fblock.Uncles(), anblock, arblock.Uncles())
} }
if freceipts, anreceipts, areceipts := rawdb.ReadReceipts(fastDb, hash, *rawdb.ReadHeaderNumber(fastDb, hash), fast.Config()), rawdb.ReadReceipts(ancientDb, hash, *rawdb.ReadHeaderNumber(ancientDb, hash), fast.Config()), rawdb.ReadReceipts(archiveDb, hash, *rawdb.ReadHeaderNumber(archiveDb, hash), fast.Config()); types.DeriveSha(freceipts, trie.NewStackTrie(nil)) != types.DeriveSha(areceipts, trie.NewStackTrie(nil)) {
// Check receipts.
freceipts := rawdb.ReadReceipts(fastDb, hash, num, fast.Config())
anreceipts := rawdb.ReadReceipts(ancientDb, hash, num, fast.Config())
areceipts := rawdb.ReadReceipts(archiveDb, hash, num, fast.Config())
if types.DeriveSha(freceipts, trie.NewStackTrie(nil)) != types.DeriveSha(areceipts, trie.NewStackTrie(nil)) {
t.Errorf("block #%d [%x]: receipts mismatch: fastdb %v, ancientdb %v, archivedb %v", num, hash, freceipts, anreceipts, areceipts) t.Errorf("block #%d [%x]: receipts mismatch: fastdb %v, ancientdb %v, archivedb %v", num, hash, freceipts, anreceipts, areceipts)
} }
// Check that hash-to-number mappings are present in all databases.
if m := rawdb.ReadHeaderNumber(fastDb, hash); m == nil || *m != num {
t.Errorf("block #%d [%x]: wrong hash-to-number mapping in fastdb: %v", num, hash, m)
}
if m := rawdb.ReadHeaderNumber(ancientDb, hash); m == nil || *m != num {
t.Errorf("block #%d [%x]: wrong hash-to-number mapping in ancientdb: %v", num, hash, m)
}
if m := rawdb.ReadHeaderNumber(archiveDb, hash); m == nil || *m != num {
t.Errorf("block #%d [%x]: wrong hash-to-number mapping in archivedb: %v", num, hash, m)
}
} }
// Check that the canonical chains are the same between the databases // Check that the canonical chains are the same between the databases
for i := 0; i < len(blocks)+1; i++ { for i := 0; i < len(blocks)+1; i++ {
if fhash, ahash := rawdb.ReadCanonicalHash(fastDb, uint64(i)), rawdb.ReadCanonicalHash(archiveDb, uint64(i)); fhash != ahash { if fhash, ahash := rawdb.ReadCanonicalHash(fastDb, uint64(i)), rawdb.ReadCanonicalHash(archiveDb, uint64(i)); fhash != ahash {
...@@ -1639,20 +1657,34 @@ func TestBlockchainRecovery(t *testing.T) { ...@@ -1639,20 +1657,34 @@ func TestBlockchainRecovery(t *testing.T) {
} }
} }
func TestIncompleteAncientReceiptChainInsertion(t *testing.T) { // This test checks that InsertReceiptChain will roll back correctly when attempting to insert a side chain.
// Configure and generate a sample block chain func TestInsertReceiptChainRollback(t *testing.T) {
var ( // Generate forked chain. The returned BlockChain object is used to process the side chain blocks.
gendb = rawdb.NewMemoryDatabase() tmpChain, sideblocks, canonblocks, err := getLongAndShortChains()
key, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") if err != nil {
address = crypto.PubkeyToAddress(key.PublicKey) t.Fatal(err)
funds = big.NewInt(1000000000) }
gspec = &Genesis{Config: params.TestChainConfig, Alloc: GenesisAlloc{address: {Balance: funds}}} defer tmpChain.Stop()
genesis = gspec.MustCommit(gendb) // Get the side chain receipts.
) if _, err := tmpChain.InsertChain(sideblocks); err != nil {
height := uint64(1024) t.Fatal("processing side chain failed:", err)
blocks, receipts := GenerateChain(gspec.Config, genesis, ethash.NewFaker(), gendb, int(height), nil) }
t.Log("sidechain head:", tmpChain.CurrentBlock().Number(), tmpChain.CurrentBlock().Hash())
sidechainReceipts := make([]types.Receipts, len(sideblocks))
for i, block := range sideblocks {
sidechainReceipts[i] = tmpChain.GetReceiptsByHash(block.Hash())
}
// Get the canon chain receipts.
if _, err := tmpChain.InsertChain(canonblocks); err != nil {
t.Fatal("processing canon chain failed:", err)
}
t.Log("canon head:", tmpChain.CurrentBlock().Number(), tmpChain.CurrentBlock().Hash())
canonReceipts := make([]types.Receipts, len(canonblocks))
for i, block := range canonblocks {
canonReceipts[i] = tmpChain.GetReceiptsByHash(block.Hash())
}
// Import the chain as a ancient-first node and ensure all pointers are updated // Set up a BlockChain that uses the ancient store.
frdir, err := ioutil.TempDir("", "") frdir, err := ioutil.TempDir("", "")
if err != nil { if err != nil {
t.Fatalf("failed to create temp freezer dir: %v", err) t.Fatalf("failed to create temp freezer dir: %v", err)
...@@ -1662,38 +1694,43 @@ func TestIncompleteAncientReceiptChainInsertion(t *testing.T) { ...@@ -1662,38 +1694,43 @@ func TestIncompleteAncientReceiptChainInsertion(t *testing.T) {
if err != nil { if err != nil {
t.Fatalf("failed to create temp freezer db: %v", err) t.Fatalf("failed to create temp freezer db: %v", err)
} }
gspec := Genesis{Config: params.AllEthashProtocolChanges}
gspec.MustCommit(ancientDb) gspec.MustCommit(ancientDb)
ancient, _ := NewBlockChain(ancientDb, nil, gspec.Config, ethash.NewFaker(), vm.Config{}, nil, nil) ancientChain, _ := NewBlockChain(ancientDb, nil, gspec.Config, ethash.NewFaker(), vm.Config{}, nil, nil)
defer ancient.Stop() defer ancientChain.Stop()
headers := make([]*types.Header, len(blocks)) // Import the canonical header chain.
for i, block := range blocks { canonHeaders := make([]*types.Header, len(canonblocks))
headers[i] = block.Header() for i, block := range canonblocks {
} canonHeaders[i] = block.Header()
if n, err := ancient.InsertHeaderChain(headers, 1); err != nil {
t.Fatalf("failed to insert header %d: %v", n, err)
} }
// Abort ancient receipt chain insertion deliberately if _, err = ancientChain.InsertHeaderChain(canonHeaders, 1); err != nil {
ancient.terminateInsert = func(hash common.Hash, number uint64) bool { t.Fatal("can't import canon headers:", err)
return number == blocks[len(blocks)/2].NumberU64()
} }
previousFastBlock := ancient.CurrentFastBlock()
if n, err := ancient.InsertReceiptChain(blocks, receipts, uint64(3*len(blocks)/4)); err == nil { // Try to insert blocks/receipts of the side chain.
t.Fatalf("failed to insert receipt %d: %v", n, err) _, err = ancientChain.InsertReceiptChain(sideblocks, sidechainReceipts, uint64(len(sideblocks)))
if err == nil {
t.Fatal("expected error from InsertReceiptChain.")
} }
if ancient.CurrentFastBlock().NumberU64() != previousFastBlock.NumberU64() { if ancientChain.CurrentFastBlock().NumberU64() != 0 {
t.Fatalf("failed to rollback ancient data, want %d, have %d", previousFastBlock.NumberU64(), ancient.CurrentFastBlock().NumberU64()) t.Fatalf("failed to rollback ancient data, want %d, have %d", 0, ancientChain.CurrentFastBlock().NumberU64())
} }
if frozen, err := ancient.db.Ancients(); err != nil || frozen != 1 { if frozen, err := ancientChain.db.Ancients(); err != nil || frozen != 1 {
t.Fatalf("failed to truncate ancient data") t.Fatalf("failed to truncate ancient data, frozen index is %d", frozen)
} }
ancient.terminateInsert = nil
if n, err := ancient.InsertReceiptChain(blocks, receipts, uint64(3*len(blocks)/4)); err != nil { // Insert blocks/receipts of the canonical chain.
t.Fatalf("failed to insert receipt %d: %v", n, err) _, err = ancientChain.InsertReceiptChain(canonblocks, canonReceipts, uint64(len(canonblocks)))
if err != nil {
t.Fatalf("can't import canon chain receipts: %v", err)
} }
if ancient.CurrentFastBlock().NumberU64() != blocks[len(blocks)-1].NumberU64() { if ancientChain.CurrentFastBlock().NumberU64() != canonblocks[len(canonblocks)-1].NumberU64() {
t.Fatalf("failed to insert ancient recept chain after rollback") t.Fatalf("failed to insert ancient recept chain after rollback")
} }
if frozen, _ := ancientChain.db.Ancients(); frozen != uint64(len(canonblocks))+1 {
t.Fatalf("wrong ancients count %d", frozen)
}
} }
// Tests that importing a very large side fork, which is larger than the canon chain, // Tests that importing a very large side fork, which is larger than the canon chain,
...@@ -1958,9 +1995,8 @@ func testInsertKnownChainData(t *testing.T, typ string) { ...@@ -1958,9 +1995,8 @@ func testInsertKnownChainData(t *testing.T, typ string) {
asserter(t, blocks2[len(blocks2)-1]) asserter(t, blocks2[len(blocks2)-1])
} }
// getLongAndShortChains returns two chains, // getLongAndShortChains returns two chains: A is longer, B is heavier.
// A is longer, B is heavier func getLongAndShortChains() (bc *BlockChain, longChain []*types.Block, heavyChain []*types.Block, err error) {
func getLongAndShortChains() (*BlockChain, []*types.Block, []*types.Block, error) {
// Generate a canonical chain to act as the main dataset // Generate a canonical chain to act as the main dataset
engine := ethash.NewFaker() engine := ethash.NewFaker()
db := rawdb.NewMemoryDatabase() db := rawdb.NewMemoryDatabase()
...@@ -1968,7 +2004,7 @@ func getLongAndShortChains() (*BlockChain, []*types.Block, []*types.Block, error ...@@ -1968,7 +2004,7 @@ func getLongAndShortChains() (*BlockChain, []*types.Block, []*types.Block, error
// Generate and import the canonical chain, // Generate and import the canonical chain,
// Offset the time, to keep the difficulty low // Offset the time, to keep the difficulty low
longChain, _ := GenerateChain(params.TestChainConfig, genesis, engine, db, 80, func(i int, b *BlockGen) { longChain, _ = GenerateChain(params.TestChainConfig, genesis, engine, db, 80, func(i int, b *BlockGen) {
b.SetCoinbase(common.Address{1}) b.SetCoinbase(common.Address{1})
}) })
diskdb := rawdb.NewMemoryDatabase() diskdb := rawdb.NewMemoryDatabase()
...@@ -1982,10 +2018,13 @@ func getLongAndShortChains() (*BlockChain, []*types.Block, []*types.Block, error ...@@ -1982,10 +2018,13 @@ func getLongAndShortChains() (*BlockChain, []*types.Block, []*types.Block, error
// Generate fork chain, make it shorter than canon, with common ancestor pretty early // Generate fork chain, make it shorter than canon, with common ancestor pretty early
parentIndex := 3 parentIndex := 3
parent := longChain[parentIndex] parent := longChain[parentIndex]
heavyChain, _ := GenerateChain(params.TestChainConfig, parent, engine, db, 75, func(i int, b *BlockGen) { heavyChainExt, _ := GenerateChain(params.TestChainConfig, parent, engine, db, 75, func(i int, b *BlockGen) {
b.SetCoinbase(common.Address{2}) b.SetCoinbase(common.Address{2})
b.OffsetTime(-9) b.OffsetTime(-9)
}) })
heavyChain = append(heavyChain, longChain[:parentIndex+1]...)
heavyChain = append(heavyChain, heavyChainExt...)
// Verify that the test is sane // Verify that the test is sane
var ( var (
longerTd = new(big.Int) longerTd = new(big.Int)
......
...@@ -31,6 +31,8 @@ var ( ...@@ -31,6 +31,8 @@ var (
// ErrNoGenesis is returned when there is no Genesis Block. // ErrNoGenesis is returned when there is no Genesis Block.
ErrNoGenesis = errors.New("genesis not found in chain") ErrNoGenesis = errors.New("genesis not found in chain")
errSideChainReceipts = errors.New("side blocks can't be accepted as ancient chain data")
) )
// List of evm-call-message pre-checking errors. All state transition messages will // List of evm-call-message pre-checking errors. All state transition messages will
......
...@@ -19,6 +19,7 @@ package rawdb ...@@ -19,6 +19,7 @@ package rawdb
import ( import (
"bytes" "bytes"
"encoding/binary" "encoding/binary"
"fmt"
"math/big" "math/big"
"sort" "sort"
...@@ -81,6 +82,37 @@ func ReadAllHashes(db ethdb.Iteratee, number uint64) []common.Hash { ...@@ -81,6 +82,37 @@ func ReadAllHashes(db ethdb.Iteratee, number uint64) []common.Hash {
return hashes return hashes
} }
type NumberHash struct {
Number uint64
Hash common.Hash
}
// ReadAllHashes retrieves all the hashes assigned to blocks at a certain heights,
// both canonical and reorged forks included.
// This method considers both limits to be _inclusive_.
func ReadAllHashesInRange(db ethdb.Iteratee, first, last uint64) []*NumberHash {
var (
start = encodeBlockNumber(first)
keyLength = len(headerPrefix) + 8 + 32
hashes = make([]*NumberHash, 0, 1+last-first)
it = db.NewIterator(headerPrefix, start)
)
defer it.Release()
for it.Next() {
key := it.Key()
if len(key) != keyLength {
continue
}
num := binary.BigEndian.Uint64(key[len(headerPrefix) : len(headerPrefix)+8])
if num > last {
break
}
hash := common.BytesToHash(key[len(key)-32:])
hashes = append(hashes, &NumberHash{num, hash})
}
return hashes
}
// ReadAllCanonicalHashes retrieves all canonical number and hash mappings at the // ReadAllCanonicalHashes retrieves all canonical number and hash mappings at the
// certain chain range. If the accumulated entries reaches the given threshold, // certain chain range. If the accumulated entries reaches the given threshold,
// abort the iteration and return the semi-finish result. // abort the iteration and return the semi-finish result.
...@@ -656,34 +688,48 @@ func WriteBlock(db ethdb.KeyValueWriter, block *types.Block) { ...@@ -656,34 +688,48 @@ func WriteBlock(db ethdb.KeyValueWriter, block *types.Block) {
} }
// WriteAncientBlock writes entire block data into ancient store and returns the total written size. // WriteAncientBlock writes entire block data into ancient store and returns the total written size.
func WriteAncientBlock(db ethdb.AncientWriter, block *types.Block, receipts types.Receipts, td *big.Int) int { func WriteAncientBlocks(db ethdb.AncientWriter, blocks []*types.Block, receipts []types.Receipts, td *big.Int) (int64, error) {
// Encode all block components to RLP format. var (
headerBlob, err := rlp.EncodeToBytes(block.Header()) tdSum = new(big.Int).Set(td)
if err != nil { stReceipts []*types.ReceiptForStorage
log.Crit("Failed to RLP encode block header", "err", err) )
return db.ModifyAncients(func(op ethdb.AncientWriteOp) error {
for i, block := range blocks {
// Convert receipts to storage format and sum up total difficulty.
stReceipts = stReceipts[:0]
for _, receipt := range receipts[i] {
stReceipts = append(stReceipts, (*types.ReceiptForStorage)(receipt))
} }
bodyBlob, err := rlp.EncodeToBytes(block.Body()) header := block.Header()
if err != nil { if i > 0 {
log.Crit("Failed to RLP encode body", "err", err) tdSum.Add(tdSum, header.Difficulty)
} }
storageReceipts := make([]*types.ReceiptForStorage, len(receipts)) if err := writeAncientBlock(op, block, header, stReceipts, tdSum); err != nil {
for i, receipt := range receipts { return err
storageReceipts[i] = (*types.ReceiptForStorage)(receipt)
} }
receiptBlob, err := rlp.EncodeToBytes(storageReceipts)
if err != nil {
log.Crit("Failed to RLP encode block receipts", "err", err)
} }
tdBlob, err := rlp.EncodeToBytes(td) return nil
if err != nil { })
log.Crit("Failed to RLP encode block total difficulty", "err", err) }
func writeAncientBlock(op ethdb.AncientWriteOp, block *types.Block, header *types.Header, receipts []*types.ReceiptForStorage, td *big.Int) error {
num := block.NumberU64()
if err := op.AppendRaw(freezerHashTable, num, block.Hash().Bytes()); err != nil {
return fmt.Errorf("can't add block %d hash: %v", num, err)
} }
// Write all blob to flatten files. if err := op.Append(freezerHeaderTable, num, header); err != nil {
err = db.AppendAncient(block.NumberU64(), block.Hash().Bytes(), headerBlob, bodyBlob, receiptBlob, tdBlob) return fmt.Errorf("can't append block header %d: %v", num, err)
if err != nil {
log.Crit("Failed to write block data to ancient store", "err", err)
} }
return len(headerBlob) + len(bodyBlob) + len(receiptBlob) + len(tdBlob) + common.HashLength if err := op.Append(freezerBodiesTable, num, block.Body()); err != nil {
return fmt.Errorf("can't append block body %d: %v", num, err)
}
if err := op.Append(freezerReceiptTable, num, receipts); err != nil {
return fmt.Errorf("can't append block %d receipts: %v", num, err)
}
if err := op.Append(freezerDifficultyTable, num, td); err != nil {
return fmt.Errorf("can't append block %d total difficulty: %v", num, err)
}
return nil
} }
// DeleteBlock removes all block data associated with a hash. // DeleteBlock removes all block data associated with a hash.
......
...@@ -29,6 +29,7 @@ import ( ...@@ -29,6 +29,7 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/params"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"golang.org/x/crypto/sha3" "golang.org/x/crypto/sha3"
...@@ -438,7 +439,7 @@ func TestAncientStorage(t *testing.T) { ...@@ -438,7 +439,7 @@ func TestAncientStorage(t *testing.T) {
if err != nil { if err != nil {
t.Fatalf("failed to create temp freezer dir: %v", err) t.Fatalf("failed to create temp freezer dir: %v", err)
} }
defer os.Remove(frdir) defer os.RemoveAll(frdir)
db, err := NewDatabaseWithFreezer(NewMemoryDatabase(), frdir, "", false) db, err := NewDatabaseWithFreezer(NewMemoryDatabase(), frdir, "", false)
if err != nil { if err != nil {
...@@ -467,8 +468,10 @@ func TestAncientStorage(t *testing.T) { ...@@ -467,8 +468,10 @@ func TestAncientStorage(t *testing.T) {
if blob := ReadTdRLP(db, hash, number); len(blob) > 0 { if blob := ReadTdRLP(db, hash, number); len(blob) > 0 {
t.Fatalf("non existent td returned") t.Fatalf("non existent td returned")
} }
// Write and verify the header in the database // Write and verify the header in the database
WriteAncientBlock(db, block, nil, big.NewInt(100)) WriteAncientBlocks(db, []*types.Block{block}, []types.Receipts{nil}, big.NewInt(100))
if blob := ReadHeaderRLP(db, hash, number); len(blob) == 0 { if blob := ReadHeaderRLP(db, hash, number); len(blob) == 0 {
t.Fatalf("no header returned") t.Fatalf("no header returned")
} }
...@@ -481,6 +484,7 @@ func TestAncientStorage(t *testing.T) { ...@@ -481,6 +484,7 @@ func TestAncientStorage(t *testing.T) {
if blob := ReadTdRLP(db, hash, number); len(blob) == 0 { if blob := ReadTdRLP(db, hash, number); len(blob) == 0 {
t.Fatalf("no td returned") t.Fatalf("no td returned")
} }
// Use a fake hash for data retrieval, nothing should be returned. // Use a fake hash for data retrieval, nothing should be returned.
fakeHash := common.BytesToHash([]byte{0x01, 0x02, 0x03}) fakeHash := common.BytesToHash([]byte{0x01, 0x02, 0x03})
if blob := ReadHeaderRLP(db, fakeHash, number); len(blob) != 0 { if blob := ReadHeaderRLP(db, fakeHash, number); len(blob) != 0 {
...@@ -528,3 +532,141 @@ func TestCanonicalHashIteration(t *testing.T) { ...@@ -528,3 +532,141 @@ func TestCanonicalHashIteration(t *testing.T) {
} }
} }
} }
func TestHashesInRange(t *testing.T) {
mkHeader := func(number, seq int) *types.Header {
h := types.Header{
Difficulty: new(big.Int),
Number: big.NewInt(int64(number)),
GasLimit: uint64(seq),
}
return &h
}
db := NewMemoryDatabase()
// For each number, write N versions of that particular number
total := 0
for i := 0; i < 15; i++ {
for ii := 0; ii < i; ii++ {
WriteHeader(db, mkHeader(i, ii))
total++
}
}
if have, want := len(ReadAllHashesInRange(db, 10, 10)), 10; have != want {
t.Fatalf("Wrong number of hashes read, want %d, got %d", want, have)
}
if have, want := len(ReadAllHashesInRange(db, 10, 9)), 0; have != want {
t.Fatalf("Wrong number of hashes read, want %d, got %d", want, have)
}
if have, want := len(ReadAllHashesInRange(db, 0, 100)), total; have != want {
t.Fatalf("Wrong number of hashes read, want %d, got %d", want, have)
}
if have, want := len(ReadAllHashesInRange(db, 9, 10)), 9+10; have != want {
t.Fatalf("Wrong number of hashes read, want %d, got %d", want, have)
}
if have, want := len(ReadAllHashes(db, 10)), 10; have != want {
t.Fatalf("Wrong number of hashes read, want %d, got %d", want, have)
}
if have, want := len(ReadAllHashes(db, 16)), 0; have != want {
t.Fatalf("Wrong number of hashes read, want %d, got %d", want, have)
}
if have, want := len(ReadAllHashes(db, 1)), 1; have != want {
t.Fatalf("Wrong number of hashes read, want %d, got %d", want, have)
}
}
// This measures the write speed of the WriteAncientBlocks operation.
func BenchmarkWriteAncientBlocks(b *testing.B) {
// Open freezer database.
frdir, err := ioutil.TempDir("", "")
if err != nil {
b.Fatalf("failed to create temp freezer dir: %v", err)
}
defer os.RemoveAll(frdir)
db, err := NewDatabaseWithFreezer(NewMemoryDatabase(), frdir, "", false)
if err != nil {
b.Fatalf("failed to create database with ancient backend")
}
// Create the data to insert. The blocks must have consecutive numbers, so we create
// all of them ahead of time. However, there is no need to create receipts
// individually for each block, just make one batch here and reuse it for all writes.
const batchSize = 128
const blockTxs = 20
allBlocks := makeTestBlocks(b.N, blockTxs)
batchReceipts := makeTestReceipts(batchSize, blockTxs)
b.ResetTimer()
// The benchmark loop writes batches of blocks, but note that the total block count is
// b.N. This means the resulting ns/op measurement is the time it takes to write a
// single block and its associated data.
var td = big.NewInt(55)
var totalSize int64
for i := 0; i < b.N; i += batchSize {
length := batchSize
if i+batchSize > b.N {
length = b.N - i
}
blocks := allBlocks[i : i+length]
receipts := batchReceipts[:length]
writeSize, err := WriteAncientBlocks(db, blocks, receipts, td)
if err != nil {
b.Fatal(err)
}
totalSize += writeSize
}
// Enable MB/s reporting.
b.SetBytes(totalSize / int64(b.N))
}
// makeTestBlocks creates fake blocks for the ancient write benchmark.
func makeTestBlocks(nblock int, txsPerBlock int) []*types.Block {
key, _ := crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291")
signer := types.LatestSignerForChainID(big.NewInt(8))
// Create transactions.
txs := make([]*types.Transaction, txsPerBlock)
for i := 0; i < len(txs); i++ {
var err error
to := common.Address{1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1}
txs[i], err = types.SignNewTx(key, signer, &types.LegacyTx{
Nonce: 2,
GasPrice: big.NewInt(30000),
Gas: 0x45454545,
To: &to,
})
if err != nil {
panic(err)
}
}
// Create the blocks.
blocks := make([]*types.Block, nblock)
for i := 0; i < nblock; i++ {
header := &types.Header{
Number: big.NewInt(int64(i)),
Extra: []byte("test block"),
}
blocks[i] = types.NewBlockWithHeader(header).WithBody(txs, nil)
blocks[i].Hash() // pre-cache the block hash
}
return blocks
}
// makeTestReceipts creates fake receipts for the ancient write benchmark.
func makeTestReceipts(n int, nPerBlock int) []types.Receipts {
receipts := make([]*types.Receipt, nPerBlock)
for i := 0; i < len(receipts); i++ {
receipts[i] = &types.Receipt{
Status: types.ReceiptStatusSuccessful,
CumulativeGasUsed: 0x888888888,
Logs: make([]*types.Log, 5),
}
}
allReceipts := make([]types.Receipts, n)
for i := 0; i < n; i++ {
allReceipts[i] = receipts
}
return allReceipts
}
...@@ -104,9 +104,9 @@ func (db *nofreezedb) AncientSize(kind string) (uint64, error) { ...@@ -104,9 +104,9 @@ func (db *nofreezedb) AncientSize(kind string) (uint64, error) {
return 0, errNotSupported return 0, errNotSupported
} }
// AppendAncient returns an error as we don't have a backing chain freezer. // ModifyAncients is not supported.
func (db *nofreezedb) AppendAncient(number uint64, hash, header, body, receipts, td []byte) error { func (db *nofreezedb) ModifyAncients(func(ethdb.AncientWriteOp) error) (int64, error) {
return errNotSupported return 0, errNotSupported
} }
// TruncateAncients returns an error as we don't have a backing chain freezer. // TruncateAncients returns an error as we don't have a backing chain freezer.
...@@ -122,9 +122,7 @@ func (db *nofreezedb) Sync() error { ...@@ -122,9 +122,7 @@ func (db *nofreezedb) Sync() error {
// NewDatabase creates a high level database on top of a given key-value data // NewDatabase creates a high level database on top of a given key-value data
// store without a freezer moving immutable chain segments into cold storage. // store without a freezer moving immutable chain segments into cold storage.
func NewDatabase(db ethdb.KeyValueStore) ethdb.Database { func NewDatabase(db ethdb.KeyValueStore) ethdb.Database {
return &nofreezedb{ return &nofreezedb{KeyValueStore: db}
KeyValueStore: db,
}
} }
// NewDatabaseWithFreezer creates a high level database on top of a given key- // NewDatabaseWithFreezer creates a high level database on top of a given key-
...@@ -132,7 +130,7 @@ func NewDatabase(db ethdb.KeyValueStore) ethdb.Database { ...@@ -132,7 +130,7 @@ func NewDatabase(db ethdb.KeyValueStore) ethdb.Database {
// storage. // storage.
func NewDatabaseWithFreezer(db ethdb.KeyValueStore, freezer string, namespace string, readonly bool) (ethdb.Database, error) { func NewDatabaseWithFreezer(db ethdb.KeyValueStore, freezer string, namespace string, readonly bool) (ethdb.Database, error) {
// Create the idle freezer instance // Create the idle freezer instance
frdb, err := newFreezer(freezer, namespace, readonly) frdb, err := newFreezer(freezer, namespace, readonly, freezerTableSize, FreezerNoSnappy)
if err != nil { if err != nil {
return nil, err return nil, err
} }
......
This diff is collapsed.
// Copyright 2021 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package rawdb
import (
"fmt"
"sync/atomic"
"github.com/ethereum/go-ethereum/common/math"
"github.com/ethereum/go-ethereum/rlp"
"github.com/golang/snappy"
)
// This is the maximum amount of data that will be buffered in memory
// for a single freezer table batch.
const freezerBatchBufferLimit = 2 * 1024 * 1024
// freezerBatch is a write operation of multiple items on a freezer.
type freezerBatch struct {
tables map[string]*freezerTableBatch
}
func newFreezerBatch(f *freezer) *freezerBatch {
batch := &freezerBatch{tables: make(map[string]*freezerTableBatch, len(f.tables))}
for kind, table := range f.tables {
batch.tables[kind] = table.newBatch()
}
return batch
}
// Append adds an RLP-encoded item of the given kind.
func (batch *freezerBatch) Append(kind string, num uint64, item interface{}) error {
return batch.tables[kind].Append(num, item)
}
// AppendRaw adds an item of the given kind.
func (batch *freezerBatch) AppendRaw(kind string, num uint64, item []byte) error {
return batch.tables[kind].AppendRaw(num, item)
}
// reset initializes the batch.
func (batch *freezerBatch) reset() {
for _, tb := range batch.tables {
tb.reset()
}
}
// commit is called at the end of a write operation and
// writes all remaining data to tables.
func (batch *freezerBatch) commit() (item uint64, writeSize int64, err error) {
// Check that count agrees on all batches.
item = uint64(math.MaxUint64)
for name, tb := range batch.tables {
if item < math.MaxUint64 && tb.curItem != item {
return 0, 0, fmt.Errorf("table %s is at item %d, want %d", name, tb.curItem, item)
}
item = tb.curItem
}
// Commit all table batches.
for _, tb := range batch.tables {
if err := tb.commit(); err != nil {
return 0, 0, err
}
writeSize += tb.totalBytes
}
return item, writeSize, nil
}
// freezerTableBatch is a batch for a freezer table.
type freezerTableBatch struct {
t *freezerTable
sb *snappyBuffer
encBuffer writeBuffer
dataBuffer []byte
indexBuffer []byte
curItem uint64 // expected index of next append
totalBytes int64 // counts written bytes since reset
}
// newBatch creates a new batch for the freezer table.
func (t *freezerTable) newBatch() *freezerTableBatch {
batch := &freezerTableBatch{t: t}
if !t.noCompression {
batch.sb = new(snappyBuffer)
}
batch.reset()
return batch
}
// reset clears the batch for reuse.
func (batch *freezerTableBatch) reset() {
batch.dataBuffer = batch.dataBuffer[:0]
batch.indexBuffer = batch.indexBuffer[:0]
batch.curItem = atomic.LoadUint64(&batch.t.items)
batch.totalBytes = 0
}
// Append rlp-encodes and adds data at the end of the freezer table. The item number is a
// precautionary parameter to ensure data correctness, but the table will reject already
// existing data.
func (batch *freezerTableBatch) Append(item uint64, data interface{}) error {
if item != batch.curItem {
return errOutOrderInsertion
}
// Encode the item.
batch.encBuffer.Reset()
if err := rlp.Encode(&batch.encBuffer, data); err != nil {
return err
}
encItem := batch.encBuffer.data
if batch.sb != nil {
encItem = batch.sb.compress(encItem)
}
return batch.appendItem(encItem)
}
// AppendRaw injects a binary blob at the end of the freezer table. The item number is a
// precautionary parameter to ensure data correctness, but the table will reject already
// existing data.
func (batch *freezerTableBatch) AppendRaw(item uint64, blob []byte) error {
if item != batch.curItem {
return errOutOrderInsertion
}
encItem := blob
if batch.sb != nil {
encItem = batch.sb.compress(blob)
}
return batch.appendItem(encItem)
}
func (batch *freezerTableBatch) appendItem(data []byte) error {
// Check if item fits into current data file.
itemSize := int64(len(data))
itemOffset := batch.t.headBytes + int64(len(batch.dataBuffer))
if itemOffset+itemSize > int64(batch.t.maxFileSize) {
// It doesn't fit, go to next file first.
if err := batch.commit(); err != nil {
return err
}
if err := batch.t.advanceHead(); err != nil {
return err
}
itemOffset = 0
}
// Put data to buffer.
batch.dataBuffer = append(batch.dataBuffer, data...)
batch.totalBytes += itemSize
// Put index entry to buffer.
entry := indexEntry{filenum: batch.t.headId, offset: uint32(itemOffset + itemSize)}
batch.indexBuffer = entry.append(batch.indexBuffer)
batch.curItem++
return batch.maybeCommit()
}
// maybeCommit writes the buffered data if the buffer is full enough.
func (batch *freezerTableBatch) maybeCommit() error {
if len(batch.dataBuffer) > freezerBatchBufferLimit {
return batch.commit()
}
return nil
}
// commit writes the batched items to the backing freezerTable.
func (batch *freezerTableBatch) commit() error {
// Write data.
_, err := batch.t.head.Write(batch.dataBuffer)
if err != nil {
return err
}
dataSize := int64(len(batch.dataBuffer))
batch.dataBuffer = batch.dataBuffer[:0]
// Write index.
_, err = batch.t.index.Write(batch.indexBuffer)
if err != nil {
return err
}
indexSize := int64(len(batch.indexBuffer))
batch.indexBuffer = batch.indexBuffer[:0]
// Update headBytes of table.
batch.t.headBytes += dataSize
atomic.StoreUint64(&batch.t.items, batch.curItem)
// Update metrics.
batch.t.sizeGauge.Inc(dataSize + indexSize)
batch.t.writeMeter.Mark(dataSize + indexSize)
return nil
}
// snappyBuffer writes snappy in block format, and can be reused. It is
// reset when WriteTo is called.
type snappyBuffer struct {
dst []byte
}
// compress snappy-compresses the data.
func (s *snappyBuffer) compress(data []byte) []byte {
// The snappy library does not care what the capacity of the buffer is,
// but only checks the length. If the length is too small, it will
// allocate a brand new buffer.
// To avoid that, we check the required size here, and grow the size of the
// buffer to utilize the full capacity.
if n := snappy.MaxEncodedLen(len(data)); len(s.dst) < n {
if cap(s.dst) < n {
s.dst = make([]byte, n)
}
s.dst = s.dst[:n]
}
s.dst = snappy.Encode(s.dst, data)
return s.dst
}
// writeBuffer implements io.Writer for a byte slice.
type writeBuffer struct {
data []byte
}
func (wb *writeBuffer) Write(data []byte) (int, error) {
wb.data = append(wb.data, data...)
return len(data), nil
}
func (wb *writeBuffer) Reset() {
wb.data = wb.data[:0]
}
This diff is collapsed.
This diff is collapsed.
// Copyright 2021 The go-ethereum Authors
// This file is part of the go-ethereum library.
//
// The go-ethereum library is free software: you can redistribute it and/or modify
// it under the terms of the GNU Lesser General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// The go-ethereum library is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU Lesser General Public License for more details.
//
// You should have received a copy of the GNU Lesser General Public License
// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
package rawdb
import (
"bytes"
"errors"
"fmt"
"io/ioutil"
"math/big"
"math/rand"
"os"
"sync"
"testing"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/rlp"
"github.com/stretchr/testify/require"
)
var freezerTestTableDef = map[string]bool{"test": true}
func TestFreezerModify(t *testing.T) {
t.Parallel()
// Create test data.
var valuesRaw [][]byte
var valuesRLP []*big.Int
for x := 0; x < 100; x++ {
v := getChunk(256, x)
valuesRaw = append(valuesRaw, v)
iv := big.NewInt(int64(x))
iv = iv.Exp(iv, iv, nil)
valuesRLP = append(valuesRLP, iv)
}
tables := map[string]bool{"raw": true, "rlp": false}
f, dir := newFreezerForTesting(t, tables)
defer os.RemoveAll(dir)
defer f.Close()
// Commit test data.
_, err := f.ModifyAncients(func(op ethdb.AncientWriteOp) error {
for i := range valuesRaw {
if err := op.AppendRaw("raw", uint64(i), valuesRaw[i]); err != nil {
return err
}
if err := op.Append("rlp", uint64(i), valuesRLP[i]); err != nil {
return err
}
}
return nil
})
if err != nil {
t.Fatal("ModifyAncients failed:", err)
}
// Dump indexes.
for _, table := range f.tables {
t.Log(table.name, "index:", table.dumpIndexString(0, int64(len(valuesRaw))))
}
// Read back test data.
checkAncientCount(t, f, "raw", uint64(len(valuesRaw)))
checkAncientCount(t, f, "rlp", uint64(len(valuesRLP)))
for i := range valuesRaw {
v, _ := f.Ancient("raw", uint64(i))
if !bytes.Equal(v, valuesRaw[i]) {
t.Fatalf("wrong raw value at %d: %x", i, v)
}
ivEnc, _ := f.Ancient("rlp", uint64(i))
want, _ := rlp.EncodeToBytes(valuesRLP[i])
if !bytes.Equal(ivEnc, want) {
t.Fatalf("wrong RLP value at %d: %x", i, ivEnc)
}
}
}
// This checks that ModifyAncients rolls back freezer updates
// when the function passed to it returns an error.
func TestFreezerModifyRollback(t *testing.T) {
t.Parallel()
f, dir := newFreezerForTesting(t, freezerTestTableDef)
defer os.RemoveAll(dir)
theError := errors.New("oops")
_, err := f.ModifyAncients(func(op ethdb.AncientWriteOp) error {
// Append three items. This creates two files immediately,
// because the table size limit of the test freezer is 2048.
require.NoError(t, op.AppendRaw("test", 0, make([]byte, 2048)))
require.NoError(t, op.AppendRaw("test", 1, make([]byte, 2048)))
require.NoError(t, op.AppendRaw("test", 2, make([]byte, 2048)))
return theError
})
if err != theError {
t.Errorf("ModifyAncients returned wrong error %q", err)
}
checkAncientCount(t, f, "test", 0)
f.Close()
// Reopen and check that the rolled-back data doesn't reappear.
tables := map[string]bool{"test": true}
f2, err := newFreezer(dir, "", false, 2049, tables)
if err != nil {
t.Fatalf("can't reopen freezer after failed ModifyAncients: %v", err)
}
defer f2.Close()
checkAncientCount(t, f2, "test", 0)
}
// This test runs ModifyAncients and Ancient concurrently with each other.
func TestFreezerConcurrentModifyRetrieve(t *testing.T) {
t.Parallel()
f, dir := newFreezerForTesting(t, freezerTestTableDef)
defer os.RemoveAll(dir)
defer f.Close()
var (
numReaders = 5
writeBatchSize = uint64(50)
written = make(chan uint64, numReaders*6)
wg sync.WaitGroup
)
wg.Add(numReaders + 1)
// Launch the writer. It appends 10000 items in batches.
go func() {
defer wg.Done()
defer close(written)
for item := uint64(0); item < 10000; item += writeBatchSize {
_, err := f.ModifyAncients(func(op ethdb.AncientWriteOp) error {
for i := uint64(0); i < writeBatchSize; i++ {
item := item + i
value := getChunk(32, int(item))
if err := op.AppendRaw("test", item, value); err != nil {
return err
}
}
return nil
})
if err != nil {
panic(err)
}
for i := 0; i < numReaders; i++ {
written <- item + writeBatchSize
}
}
}()
// Launch the readers. They read random items from the freezer up to the
// current frozen item count.
for i := 0; i < numReaders; i++ {
go func() {
defer wg.Done()
for frozen := range written {
for rc := 0; rc < 80; rc++ {
num := uint64(rand.Intn(int(frozen)))
value, err := f.Ancient("test", num)
if err != nil {
panic(fmt.Errorf("error reading %d (frozen %d): %v", num, frozen, err))
}
if !bytes.Equal(value, getChunk(32, int(num))) {
panic(fmt.Errorf("wrong value at %d", num))
}
}
}
}()
}
wg.Wait()
}
// This test runs ModifyAncients and TruncateAncients concurrently with each other.
func TestFreezerConcurrentModifyTruncate(t *testing.T) {
f, dir := newFreezerForTesting(t, freezerTestTableDef)
defer os.RemoveAll(dir)
defer f.Close()
var item = make([]byte, 256)
for i := 0; i < 1000; i++ {
// First reset and write 100 items.
if err := f.TruncateAncients(0); err != nil {
t.Fatal("truncate failed:", err)
}
_, err := f.ModifyAncients(func(op ethdb.AncientWriteOp) error {
for i := uint64(0); i < 100; i++ {
if err := op.AppendRaw("test", i, item); err != nil {
return err
}
}
return nil
})
if err != nil {
t.Fatal("modify failed:", err)
}
checkAncientCount(t, f, "test", 100)
// Now append 100 more items and truncate concurrently.
var (
wg sync.WaitGroup
truncateErr error
modifyErr error
)
wg.Add(3)
go func() {
_, modifyErr = f.ModifyAncients(func(op ethdb.AncientWriteOp) error {
for i := uint64(100); i < 200; i++ {
if err := op.AppendRaw("test", i, item); err != nil {
return err
}
}
return nil
})
wg.Done()
}()
go func() {
truncateErr = f.TruncateAncients(10)
wg.Done()
}()
go func() {
f.AncientSize("test")
wg.Done()
}()
wg.Wait()
// Now check the outcome. If the truncate operation went through first, the append
// fails, otherwise it succeeds. In either case, the freezer should be positioned
// at 10 after both operations are done.
if truncateErr != nil {
t.Fatal("concurrent truncate failed:", err)
}
if !(modifyErr == nil || modifyErr == errOutOrderInsertion) {
t.Fatal("wrong error from concurrent modify:", modifyErr)
}
checkAncientCount(t, f, "test", 10)
}
}
func newFreezerForTesting(t *testing.T, tables map[string]bool) (*freezer, string) {
t.Helper()
dir, err := ioutil.TempDir("", "freezer")
if err != nil {
t.Fatal(err)
}
// note: using low max table size here to ensure the tests actually
// switch between multiple files.
f, err := newFreezer(dir, "", false, 2049, tables)
if err != nil {
t.Fatal("can't open freezer", err)
}
return f, dir
}
// checkAncientCount verifies that the freezer contains n items.
func checkAncientCount(t *testing.T, f *freezer, kind string, n uint64) {
t.Helper()
if frozen, _ := f.Ancients(); frozen != n {
t.Fatalf("Ancients() returned %d, want %d", frozen, n)
}
// Check at index n-1.
if n > 0 {
index := n - 1
if ok, _ := f.HasAncient(kind, index); !ok {
t.Errorf("HasAncient(%q, %d) returned false unexpectedly", kind, index)
}
if _, err := f.Ancient(kind, index); err != nil {
t.Errorf("Ancient(%q, %d) returned unexpected error %q", kind, index, err)
}
}
// Check at index n.
index := n
if ok, _ := f.HasAncient(kind, index); ok {
t.Errorf("HasAncient(%q, %d) returned true unexpectedly", kind, index)
}
if _, err := f.Ancient(kind, index); err == nil {
t.Errorf("Ancient(%q, %d) didn't return expected error", kind, index)
} else if err != errOutOfBounds {
t.Errorf("Ancient(%q, %d) returned unexpected error %q", kind, index, err)
}
}
...@@ -80,10 +80,9 @@ func (t *table) AncientSize(kind string) (uint64, error) { ...@@ -80,10 +80,9 @@ func (t *table) AncientSize(kind string) (uint64, error) {
return t.db.AncientSize(kind) return t.db.AncientSize(kind)
} }
// AppendAncient is a noop passthrough that just forwards the request to the underlying // ModifyAncients runs an ancient write operation on the underlying database.
// database. func (t *table) ModifyAncients(fn func(ethdb.AncientWriteOp) error) (int64, error) {
func (t *table) AppendAncient(number uint64, hash, header, body, receipts, td []byte) error { return t.db.ModifyAncients(fn)
return t.db.AppendAncient(number, hash, header, body, receipts, td)
} }
// TruncateAncients is a noop passthrough that just forwards the request to the underlying // TruncateAncients is a noop passthrough that just forwards the request to the underlying
......
...@@ -92,9 +92,10 @@ type AncientReader interface { ...@@ -92,9 +92,10 @@ type AncientReader interface {
// AncientWriter contains the methods required to write to immutable ancient data. // AncientWriter contains the methods required to write to immutable ancient data.
type AncientWriter interface { type AncientWriter interface {
// AppendAncient injects all binary blobs belong to block at the end of the // ModifyAncients runs a write operation on the ancient store.
// append-only immutable table files. // If the function returns an error, any changes to the underlying store are reverted.
AppendAncient(number uint64, hash, header, body, receipt, td []byte) error // The integer return value is the total size of the written data.
ModifyAncients(func(AncientWriteOp) error) (int64, error)
// TruncateAncients discards all but the first n ancient data from the ancient store. // TruncateAncients discards all but the first n ancient data from the ancient store.
TruncateAncients(n uint64) error TruncateAncients(n uint64) error
...@@ -103,6 +104,15 @@ type AncientWriter interface { ...@@ -103,6 +104,15 @@ type AncientWriter interface {
Sync() error Sync() error
} }
// AncientWriteOp is given to the function argument of ModifyAncients.
type AncientWriteOp interface {
// Append adds an RLP-encoded item.
Append(kind string, number uint64, item interface{}) error
// AppendRaw adds an item without RLP-encoding it.
AppendRaw(kind string, number uint64, item []byte) error
}
// Reader contains the methods required to read data from both key-value as well as // Reader contains the methods required to read data from both key-value as well as
// immutable ancient data. // immutable ancient data.
type Reader interface { type Reader interface {
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment