diff --git a/x/blockdb/README.md b/x/blockdb/README.md index d57896b50ed8..9f62ffde7b60 100644 --- a/x/blockdb/README.md +++ b/x/blockdb/README.md @@ -10,6 +10,7 @@ BlockDB is a specialized database optimized for blockchain blocks. - **Configurable Durability**: Optional `syncToDisk` mode guarantees immediate recoverability - **Automatic Recovery**: Detects and recovers unindexed blocks after unclean shutdowns - **Block Compression**: zstd compression for block data +- **In-Memory Cache**: LRU cache for recently accessed blocks ## Design @@ -167,7 +168,6 @@ if err != nil { ## TODO -- Implement a block cache for recently accessed blocks - Use a buffered pool to avoid allocations on reads and writes - Add performance benchmarks - Consider supporting missing data files (currently we error if any data files are missing) diff --git a/x/blockdb/cache_db.go b/x/blockdb/cache_db.go new file mode 100644 index 000000000000..12e4edbc14d3 --- /dev/null +++ b/x/blockdb/cache_db.go @@ -0,0 +1,101 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package blockdb + +import ( + "slices" + "sync" + + "go.uber.org/zap" + + "github.com/ava-labs/avalanchego/cache/lru" + "github.com/ava-labs/avalanchego/database" +) + +var _ database.HeightIndex = (*cacheDB)(nil) + +// cacheDB caches data from the underlying [Database]. +// +// Operations (Get, Has, Put) are not atomic with the underlying database. +// Concurrent writes to the same height can result in cache inconsistencies where +// the cache and database contain different values. This limitation is acceptable +// because concurrent writes to the same height are not an intended use case. +type cacheDB struct { + db *Database + cache *lru.Cache[BlockHeight, BlockData] + + closeMu sync.RWMutex + closed bool +} + +func newCacheDB(db *Database, size uint16) *cacheDB { + return &cacheDB{ + db: db, + cache: lru.NewCache[BlockHeight, BlockData](int(size)), + } +} + +func (c *cacheDB) Get(height BlockHeight) (BlockData, error) { + c.closeMu.RLock() + defer c.closeMu.RUnlock() + + if c.closed { + c.db.log.Error("Failed Get: database closed", zap.Uint64("height", height)) + return nil, database.ErrClosed + } + + if cached, ok := c.cache.Get(height); ok { + return slices.Clone(cached), nil + } + data, err := c.db.Get(height) + if err != nil { + return nil, err + } + c.cache.Put(height, slices.Clone(data)) + return data, nil +} + +func (c *cacheDB) Put(height BlockHeight, data BlockData) error { + c.closeMu.RLock() + defer c.closeMu.RUnlock() + + if c.closed { + c.db.log.Error("Failed Put: database closed", zap.Uint64("height", height)) + return database.ErrClosed + } + + if err := c.db.Put(height, data); err != nil { + return err + } + + c.cache.Put(height, slices.Clone(data)) + return nil +} + +func (c *cacheDB) Has(height BlockHeight) (bool, error) { + c.closeMu.RLock() + defer c.closeMu.RUnlock() + + if c.closed { + c.db.log.Error("Failed Has: database closed", zap.Uint64("height", height)) + return false, database.ErrClosed + } + + if _, ok := c.cache.Get(height); ok { + return true, nil + } + return c.db.Has(height) +} + +func (c *cacheDB) Close() error { + c.closeMu.Lock() + defer c.closeMu.Unlock() + + if c.closed { + return database.ErrClosed + } + c.closed = true + c.cache.Flush() + return c.db.Close() +} diff --git a/x/blockdb/cache_db_test.go b/x/blockdb/cache_db_test.go new file mode 100644 index 000000000000..257d7a1ca8dc --- /dev/null +++ b/x/blockdb/cache_db_test.go @@ -0,0 +1,144 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package blockdb + +import ( + "slices" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/ava-labs/avalanchego/database" +) + +func TestCacheOnMiss(t *testing.T) { + db := newCacheDatabase(t, DefaultConfig()) + height := uint64(20) + block := randomBlock(t) + require.NoError(t, db.Put(height, block)) + + // Evict the entry from cache to simulate a cache miss + db.cache.Evict(height) + + // Read the block - should populate the cache on cache miss + _, err := db.Get(height) + require.NoError(t, err) + + _, ok := db.cache.Get(height) + require.True(t, ok) +} + +func TestCacheGet(t *testing.T) { + db := newCacheDatabase(t, DefaultConfig()) + height := uint64(30) + block := randomBlock(t) + + // Populate cache directly without writing to database + db.cache.Put(height, block) + + // Get should return the block from cache + data, err := db.Get(height) + require.NoError(t, err) + require.Equal(t, block, data) +} + +func TestCacheHas(t *testing.T) { + db := newCacheDatabase(t, DefaultConfig()) + height := uint64(40) + block := randomBlock(t) + + // Populate cache directly without writing to database + db.cache.Put(height, block) + + // Has should return true from cache even though block is not in database + has, err := db.Has(height) + require.NoError(t, err) + require.True(t, has) +} + +func TestCachePutStoresClone(t *testing.T) { + db := newCacheDatabase(t, DefaultConfig()) + height := uint64(40) + block := randomBlock(t) + clone := slices.Clone(block) + require.NoError(t, db.Put(height, clone)) + + // Modify the original block after Put + clone[0] = 99 + + // Cache should have the original unmodified data + cached, ok := db.cache.Get(height) + require.True(t, ok) + require.Equal(t, block, cached) +} + +func TestCacheGetReturnsClone(t *testing.T) { + db := newCacheDatabase(t, DefaultConfig()) + height := uint64(50) + block := randomBlock(t) + require.NoError(t, db.Put(height, block)) + + // Get the block and modify the returned data + data, err := db.Get(height) + require.NoError(t, err) + data[0] = 99 + + // Cache should still have the original unmodified data + cached, ok := db.cache.Get(height) + require.True(t, ok) + require.Equal(t, block, cached) + + // Second Get should also return original data + data, err = db.Get(height) + require.NoError(t, err) + require.Equal(t, block, data) +} + +func TestCachePutOverridesSameHeight(t *testing.T) { + db := newCacheDatabase(t, DefaultConfig()) + height := uint64(60) + b1 := randomBlock(t) + require.NoError(t, db.Put(height, b1)) + + // Verify first block is in cache + cached, ok := db.cache.Get(height) + require.True(t, ok) + require.Equal(t, b1, cached) + + // Put second block at same height and verify it overrides the first one + b2 := randomBlock(t) + require.NoError(t, db.Put(height, b2)) + cached, ok = db.cache.Get(height) + require.True(t, ok) + require.Equal(t, b2, cached) + + // Get should also return the new block + data, err := db.Get(height) + require.NoError(t, err) + require.Equal(t, b2, data) +} + +func TestCacheClose(t *testing.T) { + db := newCacheDatabase(t, DefaultConfig()) + height := uint64(70) + block := randomBlock(t) + require.NoError(t, db.Put(height, block)) + + _, ok := db.cache.Get(height) + require.True(t, ok) + require.NoError(t, db.Close()) + + // cache is flushed + require.Zero(t, db.cache.Len()) + + // db operations now fails + _, err := db.Get(height) + require.ErrorIs(t, err, database.ErrClosed) + _, err = db.Has(height) + require.ErrorIs(t, err, database.ErrClosed) + err = db.Put(height+1, block) + require.ErrorIs(t, err, database.ErrClosed) + err = db.Close() + require.ErrorIs(t, err, database.ErrClosed) +} diff --git a/x/blockdb/config.go b/x/blockdb/config.go index c587b8483d22..252c11a918f5 100644 --- a/x/blockdb/config.go +++ b/x/blockdb/config.go @@ -11,6 +11,9 @@ const DefaultMaxDataFileSize = 500 * 1024 * 1024 * 1024 // DefaultMaxDataFiles is the default maximum number of data files descriptors cached. const DefaultMaxDataFiles = 10 +// DefaultBlockCacheSize is the default size of the block cache. +const DefaultBlockCacheSize uint16 = 256 + // DatabaseConfig contains configuration parameters for BlockDB. type DatabaseConfig struct { // IndexDir is the directory where the index file is stored. @@ -28,6 +31,9 @@ type DatabaseConfig struct { // MaxDataFiles is the maximum number of data files descriptors cached. MaxDataFiles int + // BlockCacheSize is the size of the block cache (default: 256). + BlockCacheSize uint16 + // CheckpointInterval defines how frequently (in blocks) the index file header is updated (default: 1024). CheckpointInterval uint64 @@ -43,6 +49,7 @@ func DefaultConfig() DatabaseConfig { MinimumHeight: 0, MaxDataFileSize: DefaultMaxDataFileSize, MaxDataFiles: DefaultMaxDataFiles, + BlockCacheSize: DefaultBlockCacheSize, CheckpointInterval: 1024, SyncToDisk: true, } @@ -91,6 +98,12 @@ func (c DatabaseConfig) WithMaxDataFiles(maxFiles int) DatabaseConfig { return c } +// WithBlockCacheSize returns a copy of the config with BlockCacheSize set to the given value. +func (c DatabaseConfig) WithBlockCacheSize(size uint16) DatabaseConfig { + c.BlockCacheSize = size + return c +} + // WithCheckpointInterval returns a copy of the config with CheckpointInterval set to the given value. func (c DatabaseConfig) WithCheckpointInterval(interval uint64) DatabaseConfig { c.CheckpointInterval = interval diff --git a/x/blockdb/database.go b/x/blockdb/database.go index 14d6c70a9c4d..44a4dabc5faa 100644 --- a/x/blockdb/database.go +++ b/x/blockdb/database.go @@ -197,7 +197,7 @@ type Database struct { // Parameters: // - config: Configuration parameters // - log: Logger instance for structured logging -func New(config DatabaseConfig, log logging.Logger) (*Database, error) { +func New(config DatabaseConfig, log logging.Logger) (database.HeightIndex, error) { if err := config.Validate(); err != nil { return nil, err } @@ -231,6 +231,7 @@ func New(config DatabaseConfig, log logging.Logger) (*Database, error) { zap.String("dataDir", config.DataDir), zap.Uint64("maxDataFileSize", config.MaxDataFileSize), zap.Int("maxDataFiles", config.MaxDataFiles), + zap.Uint16("blockCacheSize", config.BlockCacheSize), ) if err := s.openAndInitializeIndex(); err != nil { @@ -256,6 +257,9 @@ func New(config DatabaseConfig, log logging.Logger) (*Database, error) { zap.Uint64("maxBlockHeight", maxHeight), ) + if config.BlockCacheSize > 0 { + return newCacheDB(s, config.BlockCacheSize), nil + } return s, nil } @@ -286,9 +290,7 @@ func (s *Database) Put(height BlockHeight, block BlockData) error { defer s.closeMu.RUnlock() if s.closed { - s.log.Error("Failed to write block: database is closed", - zap.Uint64("height", height), - ) + s.log.Error("Failed Put: database closed", zap.Uint64("height", height)) return database.ErrClosed } @@ -385,12 +387,6 @@ func (s *Database) Put(height BlockHeight, block BlockData) error { // It returns database.ErrNotFound if the block does not exist. func (s *Database) readBlockIndex(height BlockHeight) (indexEntry, error) { var entry indexEntry - if s.closed { - s.log.Error("Failed to read block index: database is closed", - zap.Uint64("height", height), - ) - return entry, database.ErrClosed - } // Skip the index entry read if we know the block is past the max height. maxHeight := s.maxBlockHeight.Load() @@ -436,6 +432,11 @@ func (s *Database) Get(height BlockHeight) (BlockData, error) { s.closeMu.RLock() defer s.closeMu.RUnlock() + if s.closed { + s.log.Error("Failed Get: database closed", zap.Uint64("height", height)) + return nil, database.ErrClosed + } + indexEntry, err := s.readBlockIndex(height) if err != nil { return nil, err @@ -494,6 +495,15 @@ func (s *Database) Has(height BlockHeight) (bool, error) { s.closeMu.RLock() defer s.closeMu.RUnlock() + if s.closed { + s.log.Error("Failed Has: database closed", zap.Uint64("height", height)) + return false, database.ErrClosed + } + + return s.hasWithoutLock(height) +} + +func (s *Database) hasWithoutLock(height BlockHeight) (bool, error) { _, err := s.readBlockIndex(height) if err != nil { if errors.Is(err, database.ErrNotFound) || errors.Is(err, ErrInvalidBlockHeight) { diff --git a/x/blockdb/database_test.go b/x/blockdb/database_test.go index b4af2837cf2a..a85cd253b202 100644 --- a/x/blockdb/database_test.go +++ b/x/blockdb/database_test.go @@ -90,7 +90,7 @@ func TestNew_Params(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - db, err := New(tt.config, nil) + hdb, err := New(tt.config, nil) if tt.wantErr != nil { require.Equal(t, tt.wantErr.Error(), err.Error()) @@ -98,14 +98,17 @@ func TestNew_Params(t *testing.T) { } require.NoError(t, err) - require.NotNil(t, db) + require.NotNil(t, hdb) + db, ok := hdb.(*cacheDB) + require.True(t, ok) + config := db.db.config // Verify the database was created with correct configuration - require.Equal(t, tt.config.MinimumHeight, db.config.MinimumHeight) - require.Equal(t, tt.config.MaxDataFileSize, db.config.MaxDataFileSize) - require.Equal(t, tt.config.MaxDataFiles, db.config.MaxDataFiles) - require.Equal(t, tt.config.CheckpointInterval, db.config.CheckpointInterval) - require.Equal(t, tt.config.SyncToDisk, db.config.SyncToDisk) + require.Equal(t, tt.config.MinimumHeight, config.MinimumHeight) + require.Equal(t, tt.config.MaxDataFileSize, config.MaxDataFileSize) + require.Equal(t, tt.config.MaxDataFiles, config.MaxDataFiles) + require.Equal(t, tt.config.CheckpointInterval, config.CheckpointInterval) + require.Equal(t, tt.config.SyncToDisk, config.SyncToDisk) indexPath := filepath.Join(tt.config.IndexDir, indexFileName) require.FileExists(t, indexPath) @@ -263,9 +266,8 @@ func TestFileCache_Eviction(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - store, cleanup := newTestDatabase(t, tt.config.WithMaxDataFileSize(1024*1.5)) + store := newDatabase(t, tt.config.WithMaxDataFileSize(1024*1.5)) store.compressor = compression.NewNoCompressor() - defer cleanup() // Override the file cache with specified size evictionCount := atomic.Int32{} @@ -346,8 +348,7 @@ func TestMaxDataFiles_CacheLimit(t *testing.T) { WithMaxDataFiles(2). // Only allow 2 files in cache WithMaxDataFileSize(1024) // Small file size to force multiple files - store, cleanup := newTestDatabase(t, config) - defer cleanup() + db := newDatabase(t, config) // Create blocks that will span multiple data files // Each block is ~512 bytes, so 2 blocks per file @@ -355,12 +356,12 @@ func TestMaxDataFiles_CacheLimit(t *testing.T) { // Write blocks to force multiple data files for i := range numBlocks { block := fixedSizeBlock(t, 512, uint64(i)) - require.NoError(t, store.Put(uint64(i), block)) + require.NoError(t, db.Put(uint64(i), block)) } // Verify all blocks are still readable despite evictions for i := range numBlocks { - block, err := store.Get(uint64(i)) + block, err := db.Get(uint64(i)) require.NoError(t, err, "failed to read block at height %d after eviction", i) require.Len(t, block, 512, "block size mismatch at height %d", i) } diff --git a/x/blockdb/datasplit_test.go b/x/blockdb/datasplit_test.go index d978fb714299..e005595e1a9d 100644 --- a/x/blockdb/datasplit_test.go +++ b/x/blockdb/datasplit_test.go @@ -17,8 +17,7 @@ import ( func TestDataSplitting(t *testing.T) { // Each data file should have enough space for 2 blocks config := DefaultConfig().WithMaxDataFileSize(1024 * 2.5) - store, cleanup := newTestDatabase(t, config) - defer cleanup() + store := newDatabase(t, config) // Override the compressor so we can have fixed size blocks store.compressor = compression.NewNoCompressor() @@ -54,13 +53,11 @@ func TestDataSplitting(t *testing.T) { // reopen and verify all blocks are readable require.NoError(t, store.Close()) - config = config.WithDataDir(store.config.DataDir).WithIndexDir(store.config.IndexDir) - store, err = New(config, store.log) - require.NoError(t, err) - store.compressor = compression.NewNoCompressor() - defer store.Close() + dir := store.config.DataDir + db := newDatabase(t, config.WithIndexDir(dir).WithDataDir(dir)) + db.compressor = compression.NewNoCompressor() for i := range numBlocks { - readBlock, err := store.Get(uint64(i)) + readBlock, err := db.Get(uint64(i)) require.NoError(t, err) require.Equal(t, blocks[i], readBlock) } @@ -68,8 +65,7 @@ func TestDataSplitting(t *testing.T) { func TestDataSplitting_DeletedFile(t *testing.T) { config := DefaultConfig().WithMaxDataFileSize(1024 * 2.5) - store, cleanup := newTestDatabase(t, config) - defer cleanup() + store := newDatabase(t, config) // create 5 blocks, 1kb each numBlocks := 5 diff --git a/x/blockdb/helpers_test.go b/x/blockdb/helpers_test.go index 578469ed4f5f..32e1f49e9380 100644 --- a/x/blockdb/helpers_test.go +++ b/x/blockdb/helpers_test.go @@ -11,13 +11,22 @@ import ( "github.com/stretchr/testify/require" + "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/utils/logging" ) -func newTestDatabase(t *testing.T, opts DatabaseConfig) (*Database, func()) { +func newDatabase(t *testing.T, config DatabaseConfig) *Database { t.Helper() + + db := newHeightIndexDatabase(t, config.WithBlockCacheSize(0)) + require.IsType(t, &Database{}, db) + return db.(*Database) +} + +func newHeightIndexDatabase(t *testing.T, config DatabaseConfig) database.HeightIndex { + t.Helper() + dir := t.TempDir() - config := opts if config.IndexDir == "" { config = config.WithIndexDir(dir) } @@ -25,12 +34,16 @@ func newTestDatabase(t *testing.T, opts DatabaseConfig) (*Database, func()) { config = config.WithDataDir(dir) } db, err := New(config, logging.NoLog{}) - require.NoError(t, err, "failed to create database") + require.NoError(t, err) + return db +} - cleanup := func() { - db.Close() - } - return db, cleanup +func newCacheDatabase(t *testing.T, config DatabaseConfig) *cacheDB { + t.Helper() + + db := newHeightIndexDatabase(t, config) + require.IsType(t, &cacheDB{}, db) + return db.(*cacheDB) } // randomBlock generates a random block of size 1KB-50KB. diff --git a/x/blockdb/readblock_test.go b/x/blockdb/readblock_test.go index 3e3fc1339bd2..43dabf9159e9 100644 --- a/x/blockdb/readblock_test.go +++ b/x/blockdb/readblock_test.go @@ -51,6 +51,7 @@ func TestReadOperations(t *testing.T) { MaxDataFileSize: DefaultMaxDataFileSize, CheckpointInterval: 1024, MaxDataFiles: DefaultMaxDataFileSize, + BlockCacheSize: DefaultBlockCacheSize, }, }, { @@ -69,6 +70,7 @@ func TestReadOperations(t *testing.T) { MaxDataFileSize: DefaultMaxDataFileSize, CheckpointInterval: 1024, MaxDataFiles: DefaultMaxDataFileSize, + BlockCacheSize: DefaultBlockCacheSize, }, wantErr: ErrInvalidBlockHeight, }, @@ -92,8 +94,7 @@ func TestReadOperations(t *testing.T) { config = &defaultConfig } - store, cleanup := newTestDatabase(t, *config) - defer cleanup() + store := newDatabase(t, *config) // Seed database with blocks based on config (unless skipSeed is true) seededBlocks := make(map[uint64][]byte) @@ -137,8 +138,7 @@ func TestReadOperations(t *testing.T) { } func TestReadOperations_Concurrency(t *testing.T) { - store, cleanup := newTestDatabase(t, DefaultConfig()) - defer cleanup() + store := newDatabase(t, DefaultConfig()) // Pre-generate blocks and write them numBlocks := 50 @@ -269,8 +269,7 @@ func TestHasBlock(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - store, cleanup := newTestDatabase(t, DefaultConfig().WithMinimumHeight(minHeight)) - defer cleanup() + store := newDatabase(t, DefaultConfig().WithMinimumHeight(minHeight)) for i := minHeight; i <= minHeight+blocksCount; i++ { if i == gapHeight { diff --git a/x/blockdb/recovery_test.go b/x/blockdb/recovery_test.go index 723aca2358ab..0925d5ad2270 100644 --- a/x/blockdb/recovery_test.go +++ b/x/blockdb/recovery_test.go @@ -197,7 +197,7 @@ func TestRecovery_Success(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - store, _ := newTestDatabase(t, config) + store := newDatabase(t, config) blockHeights := []uint64{0, 1, 3, 6, 2, 8, 4} blocks := make(map[uint64][]byte) @@ -217,17 +217,16 @@ func TestRecovery_Success(t *testing.T) { require.NoError(t, tt.corruptIndex(indexPath, blocks)) // Reopen the database and test recovery - recoveredStore, err := New(config.WithIndexDir(store.config.IndexDir).WithDataDir(store.config.DataDir), store.log) - require.NoError(t, err) - defer recoveredStore.Close() + dir := store.config.DataDir + recoveredDB := newDatabase(t, config.WithIndexDir(dir).WithDataDir(dir)) // Verify blocks are readable for _, height := range blockHeights { - readBlock, err := recoveredStore.Get(height) + readBlock, err := recoveredDB.Get(height) require.NoError(t, err) require.Equal(t, blocks[height], readBlock, "block %d should be the same", height) } - checkDatabaseState(t, recoveredStore, 8) + checkDatabaseState(t, recoveredDB, 8) }) } } @@ -518,11 +517,10 @@ func TestRecovery_CorruptionDetection(t *testing.T) { config = config.WithMaxDataFileSize(*tt.maxDataFileSize) } - store, cleanup := newTestDatabase(t, config) + store := newDatabase(t, config) if tt.disableCompression { store.compressor = compression.NewNoCompressor() } - defer cleanup() // Setup blocks blocks := make([][]byte, len(tt.blockHeights)) diff --git a/x/blockdb/writeblock_test.go b/x/blockdb/writeblock_test.go index 00d5d221ad6a..9002506cb225 100644 --- a/x/blockdb/writeblock_test.go +++ b/x/blockdb/writeblock_test.go @@ -4,6 +4,7 @@ package blockdb import ( + "bytes" "math" "os" "strings" @@ -38,18 +39,17 @@ func TestPutGet(t *testing.T) { { name: "nil block", block: nil, - want: []byte{}, + want: nil, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - db, cleanup := newTestDatabase(t, DefaultConfig()) - defer cleanup() + db := newDatabase(t, DefaultConfig()) require.NoError(t, db.Put(0, tt.block)) got, err := db.Get(0) require.NoError(t, err) - require.Equal(t, tt.want, got) + require.True(t, bytes.Equal(tt.want, got)) }) } } @@ -150,9 +150,7 @@ func TestPut_MaxHeight(t *testing.T) { if config.CheckpointInterval == 0 { config = DefaultConfig() } - - store, cleanup := newTestDatabase(t, config) - defer cleanup() + store := newDatabase(t, config) blocksWritten := make(map[uint64][]byte) for _, h := range tt.blockHeights { @@ -169,8 +167,7 @@ func TestPut_MaxHeight(t *testing.T) { } func TestWriteBlock_Concurrency(t *testing.T) { - store, cleanup := newTestDatabase(t, DefaultConfig()) - defer cleanup() + store := newDatabase(t, DefaultConfig()) var wg sync.WaitGroup var errors atomic.Int32 @@ -302,11 +299,10 @@ func TestWriteBlock_Errors(t *testing.T) { config = DefaultConfig() } - store, cleanup := newTestDatabase(t, config) + store := newDatabase(t, config) if tt.disableCompression { store.compressor = compression.NewNoCompressor() } - defer cleanup() if tt.setup != nil { tt.setup(store)