From 1291ce80587027410ac763209c9225ff3600796e Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Tue, 11 Aug 2020 09:25:55 -0700 Subject: [PATCH 01/24] Revert "Revert "Compress/Encrypt Blocks in the background (#1227)" (#1409)" This reverts commit 6001230ce640979998e40978f16fb6265a7475bb. --- db2_test.go | 9 +- options.go | 1 + table/builder.go | 270 +++++++++++++++++++++++++++++++----------- table/builder_test.go | 107 +++++++++++------ table/iterator.go | 1 + table/table.go | 5 +- table/table_test.go | 10 +- y/iterator.go | 6 +- 8 files changed, 293 insertions(+), 116 deletions(-) diff --git a/db2_test.go b/db2_test.go index 4c29d6358..d86e5a40a 100644 --- a/db2_test.go +++ b/db2_test.go @@ -670,16 +670,13 @@ func TestL0GCBug(t *testing.T) { // Simulate a crash by not closing db1 but releasing the locks. if db1.dirLockGuard != nil { require.NoError(t, db1.dirLockGuard.release()) + db1.dirLockGuard = nil } if db1.valueDirGuard != nil { require.NoError(t, db1.valueDirGuard.release()) + db1.valueDirGuard = nil } - for _, f := range db1.vlog.filesMap { - require.NoError(t, f.fd.Close()) - } - require.NoError(t, db1.registry.Close()) - require.NoError(t, db1.lc.close()) - require.NoError(t, db1.manifest.close()) + require.NoError(t, db1.Close()) db2, err := Open(opts) require.NoError(t, err) diff --git a/options.go b/options.go index 36d073b78..fa121a4b5 100644 --- a/options.go +++ b/options.go @@ -176,6 +176,7 @@ func DefaultOptions(path string) Options { func buildTableOptions(opt Options) table.Options { return table.Options{ + TableSize: uint64(opt.MaxTableSize), BlockSize: opt.BlockSize, BloomFalsePositive: opt.BloomFalsePositive, LoadBloomsOnOpen: opt.LoadBloomsOnOpen, diff --git a/table/builder.go b/table/builder.go index c5f60ae33..f59d0df2c 100644 --- a/table/builder.go +++ b/table/builder.go @@ -17,9 +17,10 @@ package table import ( - "bytes" "crypto/aes" "math" + "runtime" + "sync" "unsafe" "github.com/dgryski/go-farm" @@ -33,11 +34,14 @@ import ( "github.com/dgraph-io/ristretto/z" ) -func newBuffer(sz int) *bytes.Buffer { - b := new(bytes.Buffer) - b.Grow(sz) - return b -} +const ( + KB = 1024 + MB = KB * 1024 + + // When a block is encrypted, it's length increases. We add 200 bytes of padding to + // handle cases when block size increases. This is an approximate number. + padding = 200 +) type header struct { overlap uint16 // Overlap with base key. @@ -61,10 +65,18 @@ func (h *header) Decode(buf []byte) { copy(((*[headerSize]byte)(unsafe.Pointer(h))[:]), buf[:headerSize]) } +type bblock struct { + data []byte + start uint32 // Points to the starting offset of the block. + end uint32 // Points to the end offset of the block. +} + // Builder is used in building a table. type Builder struct { // Typically tens or hundreds of meg. This is for one single file. - buf *bytes.Buffer + buf []byte + sz uint32 + bufLock sync.Mutex // This lock guards the buf. We acquire lock when we resize the buf. baseKey []byte // Base key for the current block. baseOffset uint32 // Offset for the current block. @@ -72,23 +84,99 @@ type Builder struct { tableIndex *pb.TableIndex keyHashes []uint64 // Used for building the bloomfilter. opt *Options + + // Used to concurrently compress/encrypt blocks. + wg sync.WaitGroup + blockChan chan *bblock + blockList []*bblock } // NewTableBuilder makes a new TableBuilder. func NewTableBuilder(opts Options) *Builder { - return &Builder{ - buf: newBuffer(1 << 20), + b := &Builder{ + // Additional 5 MB to store index (approximate). + // We trim the additional space in table.Finish(). + buf: make([]byte, opts.TableSize+5*MB), tableIndex: &pb.TableIndex{}, keyHashes: make([]uint64, 0, 1024), // Avoid some malloc calls. opt: &opts, } + + // If encryption or compression is not enabled, do not start compression/encryption goroutines + // and write directly to the buffer. + if b.opt.Compression == options.None && b.opt.DataKey == nil { + return b + } + + count := 2 * runtime.NumCPU() + b.blockChan = make(chan *bblock, count*2) + + b.wg.Add(count) + for i := 0; i < count; i++ { + go b.handleBlock() + } + return b +} + +var slicePool = sync.Pool{ + New: func() interface{} { + // Make 4 KB blocks for reuse. + b := make([]byte, 0, 4<<10) + return &b + }, +} + +func (b *Builder) handleBlock() { + defer b.wg.Done() + for item := range b.blockChan { + // Extract the block. + blockBuf := item.data[item.start:item.end] + var dst *[]byte + // Compress the block. + if b.opt.Compression != options.None { + var err error + + dst = slicePool.Get().(*[]byte) + *dst = (*dst)[:0] + + blockBuf, err = b.compressData(*dst, blockBuf) + y.Check(err) + } + if b.shouldEncrypt() { + eBlock, err := b.encrypt(blockBuf) + y.Check(y.Wrapf(err, "Error while encrypting block in table builder.")) + blockBuf = eBlock + } + + // BlockBuf should always less than or equal to allocated space. If the blockBuf is greater + // than allocated space that means the data from this block cannot be stored in its + // existing location and trying to copy it over would mean we would over-write some data + // of the next block. + allocatedSpace := (item.end - item.start) + padding + 1 + y.AssertTruef(uint32(len(blockBuf)) <= allocatedSpace, "newend: %d oldend: %d padding: %d", + item.start+uint32(len(blockBuf)), item.end, padding) + + // Acquire the buflock here. The builder.grow function might change + // the b.buf while this goroutine was running. + b.bufLock.Lock() + // Copy over compressed/encrypted data back to the main buffer. + copy(b.buf[item.start:], blockBuf) + b.bufLock.Unlock() + + // Fix the boundary of the block. + item.end = item.start + uint32(len(blockBuf)) + + if dst != nil { + slicePool.Put(dst) + } + } } // Close closes the TableBuilder. func (b *Builder) Close() {} // Empty returns whether it's empty. -func (b *Builder) Empty() bool { return b.buf.Len() == 0 } +func (b *Builder) Empty() bool { return b.sz == 0 } // keyDiff returns a suffix of newKey that is different from b.baseKey. func (b *Builder) keyDiff(newKey []byte) []byte { @@ -124,20 +212,52 @@ func (b *Builder) addHelper(key []byte, v y.ValueStruct, vpLen uint64) { } // store current entry's offset - y.AssertTrue(uint32(b.buf.Len()) < math.MaxUint32) - b.entryOffsets = append(b.entryOffsets, uint32(b.buf.Len())-b.baseOffset) + y.AssertTrue(b.sz < math.MaxUint32) + b.entryOffsets = append(b.entryOffsets, b.sz-b.baseOffset) // Layout: header, diffKey, value. - b.buf.Write(h.Encode()) - b.buf.Write(diffKey) // We only need to store the key difference. + b.append(h.Encode()) + b.append(diffKey) + + if uint32(len(b.buf)) < b.sz+v.EncodedSize() { + b.grow(v.EncodedSize()) + } + b.sz += v.Encode(b.buf[b.sz:]) - v.EncodeTo(b.buf) // Size of KV on SST. sstSz := uint64(uint32(headerSize) + uint32(len(diffKey)) + v.EncodedSize()) // Total estimated size = size on SST + size on vlog (length of value pointer). b.tableIndex.EstimatedSize += (sstSz + vpLen) } +// grow increases the size of b.buf by atleast 50%. +func (b *Builder) grow(n uint32) { + l := uint32(len(b.buf)) + if n < l/2 { + n = l / 2 + } + b.bufLock.Lock() + newBuf := make([]byte, l+n) + copy(newBuf, b.buf) + b.buf = newBuf + b.bufLock.Unlock() +} +func (b *Builder) append(data []byte) { + // Ensure we have enough space to store new data. + if uint32(len(b.buf)) < b.sz+uint32(len(data)) { + b.grow(uint32(len(data))) + } + copy(b.buf[b.sz:], data) + b.sz += uint32(len(data)) +} + +func (b *Builder) addPadding(sz uint32) { + if uint32(len(b.buf)) < b.sz+sz { + b.grow(sz) + } + b.sz += sz +} + /* Structure of Block. +-------------------+---------------------+--------------------+--------------+------------------+ @@ -151,41 +271,36 @@ Structure of Block. */ // In case the data is encrypted, the "IV" is added to the end of the block. func (b *Builder) finishBlock() { - b.buf.Write(y.U32SliceToBytes(b.entryOffsets)) - b.buf.Write(y.U32ToBytes(uint32(len(b.entryOffsets)))) - - blockBuf := b.buf.Bytes()[b.baseOffset:] // Store checksum for current block. - b.writeChecksum(blockBuf) - - // Compress the block. - if b.opt.Compression != options.None { - var err error - // TODO: Find a way to reuse buffers. Current implementation creates a - // new buffer for each compressData call. - blockBuf, err = b.compressData(b.buf.Bytes()[b.baseOffset:]) - y.Check(err) - // Truncate already written data. - b.buf.Truncate(int(b.baseOffset)) - // Write compressed data. - b.buf.Write(blockBuf) - } - if b.shouldEncrypt() { - block := b.buf.Bytes()[b.baseOffset:] - eBlock, err := b.encrypt(block) - y.Check(y.Wrapf(err, "Error while encrypting block in table builder.")) - // We're rewriting the block, after encrypting. - b.buf.Truncate(int(b.baseOffset)) - b.buf.Write(eBlock) + b.append(y.U32SliceToBytes(b.entryOffsets)) + b.append(y.U32ToBytes(uint32(len(b.entryOffsets)))) + + b.writeChecksum(b.buf[b.baseOffset:b.sz]) + + // If compression/encryption is disabled, no need to send the block to the blockChan. + // There's nothing to be done. + if b.blockChan == nil { + b.addBlockToIndex() + return } - // TODO(Ashish):Add padding: If we want to make block as multiple of OS pages, we can - // implement padding. This might be useful while using direct I/O. + b.addPadding(padding) - // Add key to the block index + // Block end is the actual end of the block ignoring the padding. + block := &bblock{start: b.baseOffset, end: uint32(b.sz - padding), data: b.buf} + b.blockList = append(b.blockList, block) + + b.addBlockToIndex() + // Push to the block handler. + b.blockChan <- block +} + +func (b *Builder) addBlockToIndex() { + blockBuf := b.buf[b.baseOffset:b.sz] + // Add key to the block index. bo := &pb.BlockOffset{ Key: y.Copy(b.baseKey), Offset: b.baseOffset, - Len: uint32(b.buf.Len()) - b.baseOffset, + Len: uint32(len(blockBuf)), } b.tableIndex.Offsets = append(b.tableIndex.Offsets, bo) } @@ -203,7 +318,7 @@ func (b *Builder) shouldFinishBlock(key []byte, value y.ValueStruct) bool { 4 + // size of list 8 + // Sum64 in checksum proto 4) // checksum length - estimatedSize := uint32(b.buf.Len()) - b.baseOffset + uint32(6 /*header size for entry*/) + + estimatedSize := uint32(b.sz) - b.baseOffset + uint32(6 /*header size for entry*/) + uint32(len(key)) + uint32(value.EncodedSize()) + entriesOffsetsSize if b.shouldEncrypt() { @@ -220,8 +335,8 @@ func (b *Builder) Add(key []byte, value y.ValueStruct, valueLen uint32) { b.finishBlock() // Start a new block. Initialize the block. b.baseKey = []byte{} - y.AssertTrue(uint32(b.buf.Len()) < math.MaxUint32) - b.baseOffset = uint32(b.buf.Len()) + y.AssertTrue(uint32(b.sz) < math.MaxUint32) + b.baseOffset = uint32((b.sz)) b.entryOffsets = b.entryOffsets[:0] } b.addHelper(key, value, uint64(valueLen)) @@ -235,14 +350,14 @@ func (b *Builder) Add(key []byte, value y.ValueStruct, valueLen uint32) { // ReachedCapacity returns true if we... roughly (?) reached capacity? func (b *Builder) ReachedCapacity(cap int64) bool { - blocksSize := b.buf.Len() + // length of current buffer - len(b.entryOffsets)*4 + // all entry offsets size + blocksSize := b.sz + // length of current buffer + uint32(len(b.entryOffsets)*4) + // all entry offsets size 4 + // count of all entry offsets 8 + // checksum bytes 4 // checksum length estimateSz := blocksSize + 4 + // Index length - 5*(len(b.tableIndex.Offsets)) // approximate index size + 5*(uint32(len(b.tableIndex.Offsets))) // approximate index size return int64(estimateSz) > cap } @@ -271,6 +386,35 @@ func (b *Builder) Finish() []byte { b.finishBlock() // This will never start a new block. + if b.blockChan != nil { + close(b.blockChan) + } + // Wait for block handler to finish. + b.wg.Wait() + + dst := b.buf + // Fix block boundaries. This includes moving the blocks so that we + // don't have any interleaving space between them. + if len(b.blockList) > 0 { + dstLen := uint32(0) + for i, bl := range b.blockList { + off := b.tableIndex.Offsets[i] + // Length of the block is end minus the start. + off.Len = bl.end - bl.start + // New offset of the block is the point in the main buffer till + // which we have written data. + off.Offset = dstLen + + copy(dst[dstLen:], b.buf[bl.start:bl.end]) + + // New length is the start of the block plus its length. + dstLen = off.Offset + off.Len + } + // Start writing to the buffer from the point until which we have valid data. + // Fix the length because append and writeChecksum also rely on it. + b.sz = dstLen + } + index, err := proto.Marshal(b.tableIndex) y.Check(err) @@ -278,17 +422,12 @@ func (b *Builder) Finish() []byte { index, err = b.encrypt(index) y.Check(err) } - // Write index the file. - n, err := b.buf.Write(index) - y.Check(err) - - y.AssertTrue(uint32(n) < math.MaxUint32) - // Write index size. - _, err = b.buf.Write(y.U32ToBytes(uint32(n))) - y.Check(err) + // Write index the buffer. + b.append(index) + b.append(y.U32ToBytes(uint32(len(index)))) b.writeChecksum(index) - return b.buf.Bytes() + return b.buf[:b.sz] } func (b *Builder) writeChecksum(data []byte) { @@ -309,13 +448,10 @@ func (b *Builder) writeChecksum(data []byte) { // Write checksum to the file. chksum, err := proto.Marshal(&checksum) y.Check(err) - n, err := b.buf.Write(chksum) - y.Check(err) + b.append(chksum) - y.AssertTrue(uint32(n) < math.MaxUint32) // Write checksum size. - _, err = b.buf.Write(y.U32ToBytes(uint32(n))) - y.Check(err) + b.append(y.U32ToBytes(uint32(len(chksum)))) } // DataKey returns datakey of the builder. @@ -345,14 +481,14 @@ func (b *Builder) shouldEncrypt() bool { } // compressData compresses the given data. -func (b *Builder) compressData(data []byte) ([]byte, error) { +func (b *Builder) compressData(dst, data []byte) ([]byte, error) { switch b.opt.Compression { case options.None: return data, nil case options.Snappy: - return snappy.Encode(nil, data), nil + return snappy.Encode(dst, data), nil case options.ZSTD: - return y.ZSTDCompress(nil, data, b.opt.ZSTDCompressionLevel) + return y.ZSTDCompress(dst, data, b.opt.ZSTDCompressionLevel) } return nil, errors.New("Unsupported compression type") } diff --git a/table/builder_test.go b/table/builder_test.go index 05978bc16..11c3c19e4 100644 --- a/table/builder_test.go +++ b/table/builder_test.go @@ -33,30 +33,60 @@ import ( func TestTableIndex(t *testing.T) { rand.Seed(time.Now().Unix()) - keyPrefix := "key" - t.Run("single key", func(t *testing.T) { - opts := Options{Compression: options.ZSTD} - f := buildTestTable(t, keyPrefix, 1, opts) - tbl, err := OpenTable(f, opts) - require.NoError(t, err) - require.Len(t, tbl.blockIndex, 1) - }) + keysCount := 100000 + key := make([]byte, 32) + _, err := rand.Read(key) + require.NoError(t, err) + subTest := []struct { + name string + opts Options + }{ + { + name: "No encyption/compression", + opts: Options{ + BlockSize: 4 * 1024, + BloomFalsePositive: 0.01, + TableSize: 30 << 20, + }, + }, + { + // Encryption mode. + name: "Only encryption", + opts: Options{ + BlockSize: 4 * 1024, + BloomFalsePositive: 0.01, + TableSize: 30 << 20, + DataKey: &pb.DataKey{Data: key}, + }, + }, + { + // Compression mode. + name: "Only compression", + opts: Options{ + BlockSize: 4 * 1024, + BloomFalsePositive: 0.01, + TableSize: 30 << 20, + Compression: options.ZSTD, + ZSTDCompressionLevel: 3, + }, + }, + { + // Compression mode and encryption. + name: "Compression and encryption", + opts: Options{ + BlockSize: 4 * 1024, + BloomFalsePositive: 0.01, + TableSize: 30 << 20, + Compression: options.ZSTD, + ZSTDCompressionLevel: 3, + DataKey: &pb.DataKey{Data: key}, + }, + }, + } - t.Run("multiple keys", func(t *testing.T) { - opts := []Options{} - // Normal mode. - opts = append(opts, Options{BlockSize: 4 * 1024, BloomFalsePositive: 0.01}) - // Encryption mode. - key := make([]byte, 32) - _, err := rand.Read(key) - require.NoError(t, err) - opts = append(opts, Options{BlockSize: 4 * 1024, BloomFalsePositive: 0.01, - DataKey: &pb.DataKey{Data: key}}) - // Compression mode. - opts = append(opts, Options{BlockSize: 4 * 1024, BloomFalsePositive: 0.01, - Compression: options.ZSTD}) - keysCount := 10000 - for _, opt := range opts { + for _, tt := range subTest { + t.Run(tt.name, func(t *testing.T) { + opt := tt.opts builder := NewTableBuilder(opt) filename := fmt.Sprintf("%s%c%d.sst", os.TempDir(), os.PathSeparator, rand.Uint32()) f, err := y.OpenSyncedFile(filename, true) @@ -82,8 +112,9 @@ func TestTableIndex(t *testing.T) { tbl, err := OpenTable(f, opt) require.NoError(t, err, "unable to open table") + if opt.DataKey == nil { - // key id is zero if thre is no datakey. + // key id is zero if there is no datakey. require.Equal(t, tbl.KeyID(), uint64(0)) } @@ -94,8 +125,8 @@ func TestTableIndex(t *testing.T) { } f.Close() require.NoError(t, os.RemoveAll(filename)) - } - }) + }) + } } func TestInvalidCompression(t *testing.T) { @@ -126,18 +157,21 @@ func BenchmarkBuilder(b *testing.B) { keysCount := 1300000 // This number of entries consumes ~64MB of memory. + var keyList [][]byte + for i := 0; i < keysCount; i++ { + keyList = append(keyList, key(i)) + } bench := func(b *testing.B, opt *Options) { - // KeyCount * (keySize + ValSize) b.SetBytes(int64(keysCount) * (32 + 32)) + opt.BlockSize = 4 * 1024 + opt.BloomFalsePositive = 0.01 + opt.TableSize = 5 << 20 + b.ResetTimer() for i := 0; i < b.N; i++ { - opt.BlockSize = 4 * 1024 - opt.BloomFalsePositive = 0.01 builder := NewTableBuilder(*opt) - - for i := 0; i < keysCount; i++ { - builder.Add(key(i), vs, 0) + for j := 0; j < keysCount; j++ { + builder.Add(keyList[j], vs, 0) } - _ = builder.Finish() } } @@ -147,6 +181,13 @@ func BenchmarkBuilder(b *testing.B) { opt.Compression = options.None bench(b, &opt) }) + b.Run("encryption", func(b *testing.B) { + var opt Options + key := make([]byte, 32) + rand.Read(key) + opt.DataKey = &pb.DataKey{Data: key} + bench(b, &opt) + }) b.Run("zstd compression", func(b *testing.B) { var opt Options opt.Compression = options.ZSTD diff --git a/table/iterator.go b/table/iterator.go index 8f46fe1b5..1c11b3341 100644 --- a/table/iterator.go +++ b/table/iterator.go @@ -67,6 +67,7 @@ func (itr *blockIterator) setIdx(i int) { baseHeader.Decode(itr.data) itr.baseKey = itr.data[headerSize : headerSize+baseHeader.diff] } + var endOffset int // idx points to the last entry in the block. if itr.idx+1 == len(itr.entryOffsets) { diff --git a/table/table.go b/table/table.go index e459524f5..c05a4be45 100644 --- a/table/table.go +++ b/table/table.go @@ -56,6 +56,9 @@ const sizeOfOffsetStruct int64 = 3*8 + // key array take 3 words type Options struct { // Options for Opening/Building Table. + // Maximum size of the table. + TableSize uint64 + // ChkMode is the checksum verification mode for Table. ChkMode options.ChecksumVerificationMode @@ -493,7 +496,7 @@ func (t *Table) block(idx int) (*block, error) { // Checksum length greater than block size could happen if the table was compressed and // it was opened with an incorrect compression algorithm (or the data was corrupted). if blk.chkLen > len(blk.data) { - return nil, errors.New("invalid checksum length. Either the data is" + + return nil, errors.New("invalid checksum length. Either the data is " + "corrupted or the table options are incorrectly set") } diff --git a/table/table_test.go b/table/table_test.go index a31ad1ce9..c0fbfe807 100644 --- a/table/table_test.go +++ b/table/table_test.go @@ -37,11 +37,6 @@ import ( "github.com/stretchr/testify/require" ) -const ( - KB = 1024 - MB = KB * 1024 -) - func key(prefix string, i int) string { return prefix + fmt.Sprintf("%04d", i) } @@ -355,7 +350,7 @@ func TestIterateBackAndForth(t *testing.T) { it.seekToFirst() k = it.Key() - require.EqualValues(t, key("key", 0), y.ParseKey(k)) + require.EqualValues(t, key("key", 0), string(y.ParseKey(k))) } func TestUniIterator(t *testing.T) { @@ -700,7 +695,8 @@ func TestTableBigValues(t *testing.T) { require.NoError(t, err, "unable to create file") n := 100 // Insert 100 keys. - opts := Options{Compression: options.ZSTD, BlockSize: 4 * 1024, BloomFalsePositive: 0.01} + opts := Options{Compression: options.ZSTD, BlockSize: 4 * 1024, BloomFalsePositive: 0.01, + TableSize: uint64(n) * 1 << 20} builder := NewTableBuilder(opts) for i := 0; i < n; i++ { key := y.KeyWithTs([]byte(key("", i)), 0) diff --git a/y/iterator.go b/y/iterator.go index 6d0f677c0..7c9b21194 100644 --- a/y/iterator.go +++ b/y/iterator.go @@ -64,11 +64,12 @@ func (v *ValueStruct) Decode(b []byte) { } // Encode expects a slice of length at least v.EncodedSize(). -func (v *ValueStruct) Encode(b []byte) { +func (v *ValueStruct) Encode(b []byte) uint32 { b[0] = v.Meta b[1] = v.UserMeta sz := binary.PutUvarint(b[2:], v.ExpiresAt) - copy(b[2+sz:], v.Value) + n := copy(b[2+sz:], v.Value) + return uint32(2 + sz + n) } // EncodeTo should be kept in sync with the Encode function above. The reason @@ -79,6 +80,7 @@ func (v *ValueStruct) EncodeTo(buf *bytes.Buffer) { buf.WriteByte(v.UserMeta) var enc [binary.MaxVarintLen64]byte sz := binary.PutUvarint(enc[:], v.ExpiresAt) + buf.Write(enc[:sz]) buf.Write(v.Value) } From f2e9b48df034e93074649e31d1e8cde68b8de987 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Tue, 11 Aug 2020 09:25:59 -0700 Subject: [PATCH 02/24] Revert "Revert "Buffer pool for decompression (#1308)" (#1408)" This reverts commit 800305eac593bc8608d5b013f42dfa1b4f5360d1. --- db.go | 3 +++ table/builder.go | 15 ++++++----- table/iterator.go | 13 ++++++++- table/table.go | 67 ++++++++++++++++++++++++++++++++++++++--------- 4 files changed, 78 insertions(+), 20 deletions(-) diff --git a/db.go b/db.go index d5dd2a3c6..a95d1dccc 100644 --- a/db.go +++ b/db.go @@ -312,6 +312,9 @@ func Open(opt Options) (db *DB, err error) { MaxCost: int64(float64(opt.MaxCacheSize) * 0.95), BufferItems: 64, Metrics: true, + OnEvict: func(_, _ uint64, value interface{}, _ int64) { + table.BlockEvictHandler(value) + }, } db.blockCache, err = ristretto.NewCache(&config) if err != nil { diff --git a/table/builder.go b/table/builder.go index f59d0df2c..b0ccfe066 100644 --- a/table/builder.go +++ b/table/builder.go @@ -118,10 +118,13 @@ func NewTableBuilder(opts Options) *Builder { return b } -var slicePool = sync.Pool{ +var blockPool = &sync.Pool{ New: func() interface{} { - // Make 4 KB blocks for reuse. - b := make([]byte, 0, 4<<10) + // Create 5 Kb blocks even when the default size of blocks is 4 KB. The + // ZSTD decompresion library increases the buffer by 2X if it's not big + // enough. Using a 5 KB block instead of a 4 KB one avoids the + // unncessary 2X allocation by the decompression library. + b := make([]byte, 5<<10) return &b }, } @@ -135,9 +138,7 @@ func (b *Builder) handleBlock() { // Compress the block. if b.opt.Compression != options.None { var err error - - dst = slicePool.Get().(*[]byte) - *dst = (*dst)[:0] + dst = blockPool.Get().(*[]byte) blockBuf, err = b.compressData(*dst, blockBuf) y.Check(err) @@ -167,7 +168,7 @@ func (b *Builder) handleBlock() { item.end = item.start + uint32(len(blockBuf)) if dst != nil { - slicePool.Put(dst) + blockPool.Put(dst) } } } diff --git a/table/iterator.go b/table/iterator.go index 1c11b3341..43709b370 100644 --- a/table/iterator.go +++ b/table/iterator.go @@ -33,6 +33,7 @@ type blockIterator struct { key []byte val []byte entryOffsets []uint32 + block *block // prevOverlap stores the overlap of the previous key with the base key. // This avoids unnecessary copy of base key when the overlap is same for multiple keys. @@ -40,6 +41,13 @@ type blockIterator struct { } func (itr *blockIterator) setBlock(b *block) { + // Decrement the ref for the old block. If the old block was compressed, we + // might be able to reuse it. + itr.block.decrRef() + // Increment the ref for the new block. + b.incrRef() + + itr.block = b itr.err = nil itr.idx = 0 itr.baseKey = itr.baseKey[:0] @@ -102,7 +110,9 @@ func (itr *blockIterator) Error() error { return itr.err } -func (itr *blockIterator) Close() {} +func (itr *blockIterator) Close() { + itr.block.decrRef() +} var ( origin = 0 @@ -172,6 +182,7 @@ func (t *Table) NewIterator(reversed bool) *Iterator { // Close closes the iterator (and it must be called). func (itr *Iterator) Close() error { + itr.bi.Close() return itr.t.DecrRef() } diff --git a/table/table.go b/table/table.go index c05a4be45..83e5c24c5 100644 --- a/table/table.go +++ b/table/table.go @@ -185,15 +185,44 @@ func (t *Table) DecrRef() error { return nil } +// BlockEvictHandler is used to reuse the byte slice stored in the block on cache eviction. +func BlockEvictHandler(value interface{}) { + if b, ok := value.(*block); ok { + b.decrRef() + } +} + type block struct { offset int data []byte checksum []byte - entriesIndexStart int // start index of entryOffsets list - entryOffsets []uint32 - chkLen int // checksum length + entriesIndexStart int // start index of entryOffsets list + entryOffsets []uint32 // used to binary search an entry in the block. + chkLen int // checksum length. + isReusable bool // used to determine if the blocked should be reused. + ref int32 } +func (b *block) incrRef() { + atomic.AddInt32(&b.ref, 1) +} +func (b *block) decrRef() { + if b == nil { + return + } + + p := atomic.AddInt32(&b.ref, -1) + // Insert the []byte into pool only if the block is resuable. When a block + // is reusable a new []byte is used for decompression and this []byte can + // be reused. + // In case of an uncompressed block, the []byte is a reference to the + // table.mmap []byte slice. Any attempt to write data to the mmap []byte + // will lead to SEGFAULT. + if p == 0 && b.isReusable { + blockPool.Put(&b.data) + } + y.AssertTrue(p >= 0) +} func (b *block) size() int64 { return int64(3*intSize /* Size of the offset, entriesIndexStart and chkLen */ + cap(b.data) + cap(b.checksum) + cap(b.entryOffsets)*4) @@ -482,8 +511,7 @@ func (t *Table) block(idx int) (*block, error) { } } - blk.data, err = t.decompressData(blk.data) - if err != nil { + if err = t.decompress(blk); err != nil { return nil, errors.Wrapf(err, "failed to decode compressed data in file: %s at offset: %d, len: %d", t.fd.Name(), blk.offset, ko.Len) @@ -525,6 +553,7 @@ func (t *Table) block(idx int) (*block, error) { } if t.opt.Cache != nil && t.opt.KeepBlocksInCache { key := t.blockCacheKey(idx) + blk.incrRef() t.opt.Cache.Set(key, blk, blk.size()) } return blk, nil @@ -647,7 +676,8 @@ func (t *Table) VerifyChecksum() error { return y.Wrapf(err, "checksum validation failed for table: %s, block: %d, offset:%d", t.Filename(), i, os.Offset) } - + b.incrRef() + defer b.decrRef() // OnBlockRead or OnTableAndBlockRead, we don't need to call verify checksum // on block, verification would be done while reading block itself. if !(t.opt.ChkMode == options.OnBlockRead || t.opt.ChkMode == options.OnTableAndBlockRead) { @@ -713,15 +743,28 @@ func NewFilename(id uint64, dir string) string { return filepath.Join(dir, IDToFilename(id)) } -// decompressData decompresses the given data. -func (t *Table) decompressData(data []byte) ([]byte, error) { +// decompress decompresses the data stored in a block. +func (t *Table) decompress(b *block) error { + var err error switch t.opt.Compression { case options.None: - return data, nil + // Nothing to be done here. case options.Snappy: - return snappy.Decode(nil, data) + dst := blockPool.Get().(*[]byte) + b.data, err = snappy.Decode(*dst, b.data) + if err != nil { + return errors.Wrap(err, "failed to decompress") + } + b.isReusable = true case options.ZSTD: - return y.ZSTDDecompress(nil, data) + dst := blockPool.Get().(*[]byte) + b.data, err = y.ZSTDDecompress(*dst, b.data) + if err != nil { + return errors.Wrap(err, "failed to decompress") + } + b.isReusable = true + default: + return errors.New("Unsupported compression type") } - return nil, errors.New("Unsupported compression type") + return nil } From 707323d81ac68a0ede1a57096139c747bb954d18 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Tue, 11 Aug 2020 09:26:01 -0700 Subject: [PATCH 03/24] Revert "Revert "fix: Fix race condition in block.incRef (#1337)" (#1407)" This reverts commit 63d9309cd0ac9ab7f3f1ed63ad75374e5422a9a6. --- badger/cmd/bank.go | 29 +++++++++++++++++++++++--- table/iterator.go | 2 -- table/table.go | 52 +++++++++++++++++++++++++++++++++++++++------- 3 files changed, 70 insertions(+), 13 deletions(-) diff --git a/badger/cmd/bank.go b/badger/cmd/bank.go index 5491eebd6..a312ca9e4 100644 --- a/badger/cmd/bank.go +++ b/badger/cmd/bank.go @@ -125,7 +125,7 @@ func toSlice(bal uint64) []byte { } func getBalance(txn *badger.Txn, account int) (uint64, error) { - item, err := txn.Get(key(account)) + item, err := get(txn, key(account)) if err != nil { return 0, err } @@ -197,6 +197,25 @@ func diff(a, b []account) string { var errFailure = errors.New("test failed due to balance mismatch") +// get function will fetch the value for the key "k" either by using the +// txn.Get API or the iterator.Seek API. +func get(txn *badger.Txn, k []byte) (*badger.Item, error) { + if rand.Int()%2 == 0 { + return txn.Get(k) + } + + iopt := badger.DefaultIteratorOptions + // PrefectValues is expensive. We don't need it here. + iopt.PrefetchValues = false + it := txn.NewIterator(iopt) + defer it.Close() + it.Seek(k) + if it.Valid() { + return it.Item(), nil + } + return nil, badger.ErrKeyNotFound +} + // seekTotal retrives the total of all accounts by seeking for each account key. func seekTotal(txn *badger.Txn) ([]account, error) { expected := uint64(numAccounts) * uint64(initialBal) @@ -204,7 +223,7 @@ func seekTotal(txn *badger.Txn) ([]account, error) { var total uint64 for i := 0; i < numAccounts; i++ { - item, err := txn.Get(key(i)) + item, err := get(txn, key(i)) if err != nil { log.Printf("Error for account: %d. err=%v. key=%q\n", i, err, key(i)) return accounts, err @@ -343,7 +362,11 @@ func runTest(cmd *cobra.Command, args []string) error { WithNumMemtables(2). // Do not GC any versions, because we need them for the disect.. WithNumVersionsToKeep(int(math.MaxInt32)). - WithValueThreshold(1) // Make all values go to value log + WithValueThreshold(1). // Make all values go to value log + WithCompression(options.ZSTD). + WithKeepL0InMemory(false). + WithMaxCacheSize(10 << 20) + if mmap { opts = opts.WithTableLoadingMode(options.MemoryMap) } diff --git a/table/iterator.go b/table/iterator.go index 43709b370..d48e58138 100644 --- a/table/iterator.go +++ b/table/iterator.go @@ -44,8 +44,6 @@ func (itr *blockIterator) setBlock(b *block) { // Decrement the ref for the old block. If the old block was compressed, we // might be able to reuse it. itr.block.decrRef() - // Increment the ref for the new block. - b.incrRef() itr.block = b itr.err = nil diff --git a/table/table.go b/table/table.go index 83e5c24c5..b71499ca3 100644 --- a/table/table.go +++ b/table/table.go @@ -203,25 +203,46 @@ type block struct { ref int32 } -func (b *block) incrRef() { - atomic.AddInt32(&b.ref, 1) +// incrRef increments the ref of a block and return a bool indicating if the +// increment was successful. A true value indicates that the block can be used. +func (b *block) incrRef() bool { + for { + // We can't blindly add 1 to ref. We need to check whether it has + // reached zero first, because if it did, then we should absolutely not + // use this block. + ref := atomic.LoadInt32(&b.ref) + // The ref would not be equal to 0 unless the existing + // block get evicted before this line. If the ref is zero, it means that + // the block is already added the the blockPool and cannot be used + // anymore. The ref of a new block is 1 so the following condition will + // be true only if the block got reused before we could increment its + // ref. + if ref == 0 { + return false + } + // Increment the ref only if it is not zero and has not changed between + // the time we read it and we're updating it. + // + if atomic.CompareAndSwapInt32(&b.ref, ref, ref+1) { + return true + } + } } func (b *block) decrRef() { if b == nil { return } - p := atomic.AddInt32(&b.ref, -1) // Insert the []byte into pool only if the block is resuable. When a block // is reusable a new []byte is used for decompression and this []byte can // be reused. // In case of an uncompressed block, the []byte is a reference to the // table.mmap []byte slice. Any attempt to write data to the mmap []byte // will lead to SEGFAULT. - if p == 0 && b.isReusable { + if atomic.AddInt32(&b.ref, -1) == 0 && b.isReusable { blockPool.Put(&b.data) } - y.AssertTrue(p >= 0) + y.AssertTrue(atomic.LoadInt32(&b.ref) >= 0) } func (b *block) size() int64 { return int64(3*intSize /* Size of the offset, entriesIndexStart and chkLen */ + @@ -480,6 +501,9 @@ func calculateOffsetsSize(offsets []*pb.BlockOffset) int64 { return totalSize + 3*8 } +// block function return a new block. Each block holds a ref and the byte +// slice stored in the block will be reused when the ref becomes zero. The +// caller should release the block by calling block.decrRef() on it. func (t *Table) block(idx int) (*block, error) { y.AssertTruef(idx >= 0, "idx=%d", idx) if idx >= t.noOfBlocks { @@ -489,7 +513,12 @@ func (t *Table) block(idx int) (*block, error) { key := t.blockCacheKey(idx) blk, ok := t.opt.Cache.Get(key) if ok && blk != nil { - return blk.(*block), nil + // Use the block only if the increment was successful. The block + // could get evicted from the cache between the Get() call and the + // incrRef() call. + if b := blk.(*block); b.incrRef() { + return b, nil + } } } @@ -497,6 +526,7 @@ func (t *Table) block(idx int) (*block, error) { ko := t.blockOffsets()[idx] blk := &block{ offset: int(ko.Offset), + ref: 1, } var err error if blk.data, err = t.read(blk.offset, int(ko.Len)); err != nil { @@ -553,8 +583,14 @@ func (t *Table) block(idx int) (*block, error) { } if t.opt.Cache != nil && t.opt.KeepBlocksInCache { key := t.blockCacheKey(idx) - blk.incrRef() - t.opt.Cache.Set(key, blk, blk.size()) + // incrRef should never return false here because we're calling it on a + // new block with ref=1. + y.AssertTrue(blk.incrRef()) + + // Decrement the block ref if we could not insert it in the cache. + if !t.opt.Cache.Set(key, blk, blk.size()) { + blk.decrRef() + } } return blk, nil } From 54aa796f8f62b6478bfa013b8e75aa634bfa8d70 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Tue, 11 Aug 2020 11:43:41 -0700 Subject: [PATCH 04/24] Compression, Decompression and some Encryption is using Calloc. --- key_registry.go | 8 ++--- table/builder.go | 82 +++++++++++++++++++++++++++++++++--------------- table/table.go | 42 +++++++++++++++++++------ value.go | 29 ++++++----------- y/encrypt.go | 27 +++++++++++++++- y/y.go | 14 +++++++++ 6 files changed, 142 insertions(+), 60 deletions(-) diff --git a/key_registry.go b/key_registry.go index db32acd1a..d995c3cc7 100644 --- a/key_registry.go +++ b/key_registry.go @@ -159,7 +159,7 @@ func validRegistry(fp *os.File, encryptionKey []byte) error { } if len(encryptionKey) > 0 { // Decrypting sanity text. - if eSanityText, err = y.XORBlock(eSanityText, encryptionKey, iv); err != nil { + if eSanityText, err = y.XORBlockAllocate(eSanityText, encryptionKey, iv); err != nil { return y.Wrapf(err, "During validRegistry") } } @@ -200,7 +200,7 @@ func (kri *keyRegistryIterator) next() (*pb.DataKey, error) { } if len(kri.encryptionKey) > 0 { // Decrypt the key if the storage key exists. - if dataKey.Data, err = y.XORBlock(dataKey.Data, kri.encryptionKey, dataKey.Iv); err != nil { + if dataKey.Data, err = y.XORBlockAllocate(dataKey.Data, kri.encryptionKey, dataKey.Iv); err != nil { return nil, y.Wrapf(err, "While decrypting datakey in keyRegistryIterator.next") } } @@ -254,7 +254,7 @@ func WriteKeyRegistry(reg *KeyRegistry, opt KeyRegistryOptions) error { eSanity := sanityText if len(opt.EncryptionKey) > 0 { var err error - eSanity, err = y.XORBlock(eSanity, opt.EncryptionKey, iv) + eSanity, err = y.XORBlockAllocate(eSanity, opt.EncryptionKey, iv) if err != nil { return y.Wrapf(err, "Error while encrpting sanity text in WriteKeyRegistry") } @@ -395,7 +395,7 @@ func storeDataKey(buf *bytes.Buffer, storageKey []byte, k *pb.DataKey) error { return nil } var err error - k.Data, err = y.XORBlock(k.Data, storageKey, k.Iv) + k.Data, err = y.XORBlockAllocate(k.Data, storageKey, k.Iv) return err } // In memory datakey will be plain text so encrypting before storing to the disk. diff --git a/table/builder.go b/table/builder.go index b0ccfe066..6fb608c8d 100644 --- a/table/builder.go +++ b/table/builder.go @@ -23,11 +23,13 @@ import ( "sync" "unsafe" + "github.com/DataDog/zstd" "github.com/dgryski/go-farm" "github.com/golang/protobuf/proto" "github.com/golang/snappy" "github.com/pkg/errors" + "github.com/dgraph-io/badger/v2/manual" "github.com/dgraph-io/badger/v2/options" "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" @@ -118,33 +120,32 @@ func NewTableBuilder(opts Options) *Builder { return b } -var blockPool = &sync.Pool{ - New: func() interface{} { - // Create 5 Kb blocks even when the default size of blocks is 4 KB. The - // ZSTD decompresion library increases the buffer by 2X if it's not big - // enough. Using a 5 KB block instead of a 4 KB one avoids the - // unncessary 2X allocation by the decompression library. - b := make([]byte, 5<<10) - return &b - }, -} +// var blockPool = &sync.Pool{ +// New: func() interface{} { +// // Create 5 Kb blocks even when the default size of blocks is 4 KB. The +// // ZSTD decompresion library increases the buffer by 2X if it's not big +// // enough. Using a 5 KB block instead of a 4 KB one avoids the +// // unncessary 2X allocation by the decompression library. +// b := make([]byte, 5<<10) +// return &b +// }, +// } func (b *Builder) handleBlock() { defer b.wg.Done() + + doCompress := b.opt.Compression != options.None for item := range b.blockChan { // Extract the block. blockBuf := item.data[item.start:item.end] - var dst *[]byte // Compress the block. - if b.opt.Compression != options.None { + if doCompress { var err error - dst = blockPool.Get().(*[]byte) - - blockBuf, err = b.compressData(*dst, blockBuf) + blockBuf, err = b.compressData(blockBuf) y.Check(err) } if b.shouldEncrypt() { - eBlock, err := b.encrypt(blockBuf) + eBlock, err := b.encrypt(blockBuf, doCompress) y.Check(y.Wrapf(err, "Error while encrypting block in table builder.")) blockBuf = eBlock } @@ -167,8 +168,8 @@ func (b *Builder) handleBlock() { // Fix the boundary of the block. item.end = item.start + uint32(len(blockBuf)) - if dst != nil { - blockPool.Put(dst) + if doCompress { + manual.Free(blockBuf) } } } @@ -420,7 +421,7 @@ func (b *Builder) Finish() []byte { y.Check(err) if b.shouldEncrypt() { - index, err = b.encrypt(index) + index, err = b.encrypt(index, false) y.Check(err) } // Write index the buffer. @@ -462,17 +463,44 @@ func (b *Builder) DataKey() *pb.DataKey { // encrypt will encrypt the given data and appends IV to the end of the encrypted data. // This should be only called only after checking shouldEncrypt method. -func (b *Builder) encrypt(data []byte) ([]byte, error) { +func (b *Builder) encrypt(data []byte, viaC bool) ([]byte, error) { iv, err := y.GenerateIV() if err != nil { return data, y.Wrapf(err, "Error while generating IV in Builder.encrypt") } - data, err = y.XORBlock(data, b.DataKey().Data, iv) - if err != nil { + needSz := len(data) + len(iv) + var dst []byte + if viaC { + dst = manual.New(needSz) + } else { + dst = make([]byte, needSz) + } + dst = dst[:len(data)] + + if err = y.XORBlock(dst, data, b.DataKey().Data, iv); err != nil { + if viaC { + manual.Free(dst) + } return data, y.Wrapf(err, "Error while encrypting in Builder.encrypt") } - data = append(data, iv...) - return data, nil + if viaC { + manual.Free(data) + } + + y.AssertTrue(cap(dst)-len(dst) >= len(iv)) + dst = append(dst, iv...) + // if !viaC || cap(data)-len(data) >= len(iv) { + // data = append(data, iv...) + // } else { + // // This has to be viaC. + // var buf []byte + // buf = manual.New(len(data) + len(iv)) + // copy(buf, data) + // copy(buf[len(data):], iv) + // manual.Free(data) + // data = buf + // } + return dst, nil } // shouldEncrypt tells us whether to encrypt the data or not. @@ -482,13 +510,17 @@ func (b *Builder) shouldEncrypt() bool { } // compressData compresses the given data. -func (b *Builder) compressData(dst, data []byte) ([]byte, error) { +func (b *Builder) compressData(data []byte) ([]byte, error) { switch b.opt.Compression { case options.None: return data, nil case options.Snappy: + sz := snappy.MaxEncodedLen(len(data)) + dst := manual.New(sz) return snappy.Encode(dst, data), nil case options.ZSTD: + sz := zstd.CompressBound(len(data)) + dst := manual.New(sz) return y.ZSTDCompress(dst, data, b.opt.ZSTDCompressionLevel) } return nil, errors.New("Unsupported compression type") diff --git a/table/table.go b/table/table.go index b71499ca3..b7807b6b3 100644 --- a/table/table.go +++ b/table/table.go @@ -35,6 +35,7 @@ import ( "github.com/golang/snappy" "github.com/pkg/errors" + "github.com/dgraph-io/badger/v2/manual" "github.com/dgraph-io/badger/v2/options" "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" @@ -199,7 +200,7 @@ type block struct { entriesIndexStart int // start index of entryOffsets list entryOffsets []uint32 // used to binary search an entry in the block. chkLen int // checksum length. - isReusable bool // used to determine if the blocked should be reused. + freeMe bool // used to determine if the blocked should be reused. ref int32 } @@ -239,8 +240,9 @@ func (b *block) decrRef() { // In case of an uncompressed block, the []byte is a reference to the // table.mmap []byte slice. Any attempt to write data to the mmap []byte // will lead to SEGFAULT. - if atomic.AddInt32(&b.ref, -1) == 0 && b.isReusable { - blockPool.Put(&b.data) + if atomic.AddInt32(&b.ref, -1) == 0 && b.freeMe { + manual.Free(b.data) + // blockPool.Put(&b.data) } y.AssertTrue(atomic.LoadInt32(&b.ref) >= 0) } @@ -749,7 +751,13 @@ func (t *Table) decrypt(data []byte) ([]byte, error) { iv := data[len(data)-aes.BlockSize:] // Rest all bytes are data. data = data[:len(data)-aes.BlockSize] - return y.XORBlock(data, t.opt.DataKey.Data, iv) + + // TODO: Check if this is done via Calloc. Otherwise, we'll have a memory leak. + dst := make([]byte, len(data)) + if err := y.XORBlock(dst, data, t.opt.DataKey.Data, iv); err != nil { + return nil, errors.Wrapf(err, "while decrypt") + } + return dst, nil } // ParseFileID reads the file id out of a filename. @@ -781,26 +789,40 @@ func NewFilename(id uint64, dir string) string { // decompress decompresses the data stored in a block. func (t *Table) decompress(b *block) error { + var dst []byte + var addr *byte var err error + switch t.opt.Compression { case options.None: // Nothing to be done here. + return nil case options.Snappy: - dst := blockPool.Get().(*[]byte) - b.data, err = snappy.Decode(*dst, b.data) + if sz, err := snappy.DecodedLen(b.data); err == nil { + dst = manual.New(sz) + addr = &dst[0] + } + b.data, err = snappy.Decode(dst, b.data) if err != nil { + manual.Free(dst) return errors.Wrap(err, "failed to decompress") } - b.isReusable = true case options.ZSTD: - dst := blockPool.Get().(*[]byte) - b.data, err = y.ZSTDDecompress(*dst, b.data) + dst = manual.New(len(b.data) * 3) // We have to guess. + addr = &dst[0] + b.data, err = y.ZSTDDecompress(dst, b.data) if err != nil { + manual.Free(dst) return errors.Wrap(err, "failed to decompress") } - b.isReusable = true default: return errors.New("Unsupported compression type") } + + if len(b.data) > 0 && addr != &b.data[0] { + manual.Free(dst) + } else { + b.freeMe = true + } return nil } diff --git a/value.go b/value.go index cef996a79..6e0394f1e 100644 --- a/value.go +++ b/value.go @@ -105,13 +105,14 @@ func (lf *logFile) encodeEntry(e *Entry, buf *bytes.Buffer, offset uint32) (int, userMeta: e.UserMeta, } + hash := crc32.New(y.CastagnoliCrcTable) + + writer := io.MultiWriter(buf, hash) + // encode header. var headerEnc [maxHeaderSize]byte sz := h.Encode(headerEnc[:]) - y.Check2(buf.Write(headerEnc[:sz])) - // write hash. - hash := crc32.New(y.CastagnoliCrcTable) - y.Check2(hash.Write(headerEnc[:sz])) + y.Check2(writer.Write(headerEnc[:sz])) // we'll encrypt only key and value. if lf.encryptionEnabled() { // TODO: no need to allocate the bytes. we can calculate the encrypted buf one by one @@ -120,25 +121,13 @@ func (lf *logFile) encodeEntry(e *Entry, buf *bytes.Buffer, offset uint32) (int, eBuf := make([]byte, 0, len(e.Key)+len(e.Value)) eBuf = append(eBuf, e.Key...) eBuf = append(eBuf, e.Value...) - var err error - eBuf, err = y.XORBlock(eBuf, lf.dataKey.Data, lf.generateIV(offset)) - if err != nil { + if err := y.XORBlockStream(writer, eBuf, lf.dataKey.Data, lf.generateIV(offset)); err != nil { return 0, y.Wrapf(err, "Error while encoding entry for vlog.") } - // write encrypted buf. - y.Check2(buf.Write(eBuf)) - // write the hash. - y.Check2(hash.Write(eBuf)) } else { // Encryption is disabled so writing directly to the buffer. - // write key. - y.Check2(buf.Write(e.Key)) - // write key hash. - y.Check2(hash.Write(e.Key)) - // write value. - y.Check2(buf.Write(e.Value)) - // write value hash. - y.Check2(hash.Write(e.Value)) + y.Check2(writer.Write(e.Key)) + y.Check2(writer.Write(e.Value)) } // write crc32 hash. var crcBuf [crc32.Size]byte @@ -172,7 +161,7 @@ func (lf *logFile) decodeEntry(buf []byte, offset uint32) (*Entry, error) { } func (lf *logFile) decryptKV(buf []byte, offset uint32) ([]byte, error) { - return y.XORBlock(buf, lf.dataKey.Data, lf.generateIV(offset)) + return y.XORBlockAllocate(buf, lf.dataKey.Data, lf.generateIV(offset)) } // KeyID returns datakey's ID. diff --git a/y/encrypt.go b/y/encrypt.go index dbfe019f1..34b4729b6 100644 --- a/y/encrypt.go +++ b/y/encrypt.go @@ -17,15 +17,29 @@ package y import ( + "bytes" "crypto/aes" "crypto/cipher" "crypto/rand" + "io" + + "github.com/pkg/errors" ) // XORBlock encrypts the given data with AES and XOR's with IV. // Can be used for both encryption and decryption. IV is of // AES block size. -func XORBlock(src, key, iv []byte) ([]byte, error) { +func XORBlock(dst, src, key, iv []byte) error { + block, err := aes.NewCipher(key) + if err != nil { + return err + } + stream := cipher.NewCTR(block, iv) + stream.XORKeyStream(dst, src) + return nil +} + +func XORBlockAllocate(src, key, iv []byte) ([]byte, error) { block, err := aes.NewCipher(key) if err != nil { return nil, err @@ -36,6 +50,17 @@ func XORBlock(src, key, iv []byte) ([]byte, error) { return dst, nil } +func XORBlockStream(w io.Writer, src, key, iv []byte) error { + block, err := aes.NewCipher(key) + if err != nil { + return err + } + stream := cipher.NewCTR(block, iv) + sw := cipher.StreamWriter{S: stream, W: w} + _, err = io.Copy(sw, bytes.NewReader(src)) + return errors.Wrapf(err, "XORBlockStream") +} + // GenerateIV generates IV. func GenerateIV() ([]byte, error) { iv := make([]byte, aes.BlockSize) diff --git a/y/y.go b/y/y.go index 93b47c7f7..1b47defa4 100644 --- a/y/y.go +++ b/y/y.go @@ -167,6 +167,7 @@ func SameKey(src, dst []byte) bool { // One problem is with n distinct sizes in random order it'll reallocate log(n) times. type Slice struct { buf []byte + // viaC bool } // Resize reuses the Slice's buffer (or makes a new one) and returns a slice in that buffer of @@ -178,6 +179,19 @@ func (s *Slice) Resize(sz int) []byte { return s.buf[0:sz] } +// func (s *Slice) Append(dst []byte) []byte { +// if !s.viaC || cap(s.buf)-len(s.buf) >= len(dst) { +// s.buf = append(s.buf, dst) +// } else { +// buf := make([]byte, len(dst)+len(s.buf)) +// copy(buf, s.buf) +// copy(buf[len(s.buf):], dst) +// manual.Free(s.buf) +// s.buf = buf +// s.viaC = false +// } +// } + // FixedDuration returns a string representation of the given duration with the // hours, minutes, and seconds. func FixedDuration(d time.Duration) string { From b8e2016dbe1c6d4c9c6a104d03d1add9b1b880b7 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Tue, 11 Aug 2020 11:44:06 -0700 Subject: [PATCH 05/24] Add Calloc functions --- manual/manual.go | 60 ++++++++++++++++++++++++++++++++++++++++++ manual/manual_32bit.go | 12 +++++++++ manual/manual_64bit.go | 12 +++++++++ manual/manual_nocgo.go | 19 +++++++++++++ 4 files changed, 103 insertions(+) create mode 100644 manual/manual.go create mode 100644 manual/manual_32bit.go create mode 100644 manual/manual_64bit.go create mode 100644 manual/manual_nocgo.go diff --git a/manual/manual.go b/manual/manual.go new file mode 100644 index 000000000..640816a04 --- /dev/null +++ b/manual/manual.go @@ -0,0 +1,60 @@ +// Copyright 2020 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package manual + +// #include +import "C" +import "unsafe" + +// The go:linkname directives provides backdoor access to private functions in +// the runtime. Below we're accessing the throw function. + +//go:linkname throw runtime.throw +func throw(s string) + +// TODO(peter): Rather than relying an C malloc/free, we could fork the Go +// runtime page allocator and allocate large chunks of memory using mmap or +// similar. + +// New allocates a slice of size n. The returned slice is from manually managed +// memory and MUST be released by calling Free. Failure to do so will result in +// a memory leak. +func New(n int) []byte { + if n == 0 { + return make([]byte, 0) + } + // We need to be conscious of the Cgo pointer passing rules: + // + // https://golang.org/cmd/cgo/#hdr-Passing_pointers + // + // ... + // Note: the current implementation has a bug. While Go code is permitted + // to write nil or a C pointer (but not a Go pointer) to C memory, the + // current implementation may sometimes cause a runtime error if the + // contents of the C memory appear to be a Go pointer. Therefore, avoid + // passing uninitialized C memory to Go code if the Go code is going to + // store pointer values in it. Zero out the memory in C before passing it + // to Go. + ptr := C.calloc(C.size_t(n), 1) + if ptr == nil { + // NB: throw is like panic, except it guarantees the process will be + // terminated. The call below is exactly what the Go runtime invokes when + // it cannot allocate memory. + throw("out of memory") + } + // Interpret the C pointer as a pointer to a Go array, then slice. + return (*[MaxArrayLen]byte)(unsafe.Pointer(ptr))[:n:n] +} + +// Free frees the specified slice. +func Free(b []byte) { + if cap(b) != 0 { + if len(b) == 0 { + b = b[:cap(b)] + } + ptr := unsafe.Pointer(&b[0]) + C.free(ptr) + } +} diff --git a/manual/manual_32bit.go b/manual/manual_32bit.go new file mode 100644 index 000000000..9d1add490 --- /dev/null +++ b/manual/manual_32bit.go @@ -0,0 +1,12 @@ +// Copyright 2020 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +// +build 386 amd64p32 arm armbe mips mipsle mips64p32 mips64p32le ppc sparc + +package manual + +const ( + // MaxArrayLen is a safe maximum length for slices on this architecture. + MaxArrayLen = 1<<31 - 1 +) diff --git a/manual/manual_64bit.go b/manual/manual_64bit.go new file mode 100644 index 000000000..f03846953 --- /dev/null +++ b/manual/manual_64bit.go @@ -0,0 +1,12 @@ +// Copyright 2020 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +// +build amd64 arm64 arm64be ppc64 ppc64le mips64 mips64le s390x sparc64 + +package manual + +const ( + // MaxArrayLen is a safe maximum length for slices on this architecture. + MaxArrayLen = 1<<50 - 1 +) diff --git a/manual/manual_nocgo.go b/manual/manual_nocgo.go new file mode 100644 index 000000000..466d1ee63 --- /dev/null +++ b/manual/manual_nocgo.go @@ -0,0 +1,19 @@ +// Copyright 2020 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +// +build !cgo + +package manual + +// Provides versions of New and Free when cgo is not available (e.g. cross +// compilation). + +// New allocates a slice of size n. +func New(n int) []byte { + return make([]byte, n) +} + +// Free frees the specified slice. +func Free(b []byte) { +} From f0b712ce020f08782f5f40c144ddf09367cf9606 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Tue, 11 Aug 2020 12:29:59 -0700 Subject: [PATCH 06/24] Rename to Calloc. And also add a debug for number of blocks --- badger/cmd/write_bench.go | 47 +++++++++++++++++++-------------------- manual/manual.go | 2 +- manual/manual_nocgo.go | 2 +- table/builder.go | 6 ++--- table/table.go | 8 +++++-- 5 files changed, 34 insertions(+), 31 deletions(-) diff --git a/badger/cmd/write_bench.go b/badger/cmd/write_bench.go index 7045fcc8b..ac76cb609 100644 --- a/badger/cmd/write_bench.go +++ b/badger/cmd/write_bench.go @@ -17,7 +17,6 @@ package cmd import ( - "bytes" "encoding/binary" "fmt" "log" @@ -309,29 +308,29 @@ func reportStats(c *y.Closer, db *badger.DB) { case <-c.HasBeenClosed(): return case <-t.C: - txn := db.NewTransaction(false) - defer txn.Discard() - - iopt := badger.DefaultIteratorOptions - iopt.AllVersions = true - iopt.InternalAccess = true - - it := txn.NewIterator(iopt) - defer it.Close() - for it.Rewind(); it.Valid(); it.Next() { - i := it.Item() - if bytes.HasPrefix(i.Key(), []byte("!badger!")) { - internalKeyCount++ - } - if bytes.HasPrefix(i.Key(), []byte("!badger!Move")) { - moveKeyCount++ - } - if i.IsDeletedOrExpired() { - invalidKeyCount++ - } else { - validKeyCount++ - } - } + // txn := db.NewTransaction(false) + // defer txn.Discard() + + // iopt := badger.DefaultIteratorOptions + // iopt.AllVersions = true + // iopt.InternalAccess = true + + // it := txn.NewIterator(iopt) + // defer it.Close() + // for it.Rewind(); it.Valid(); it.Next() { + // i := it.Item() + // if bytes.HasPrefix(i.Key(), []byte("!badger!")) { + // internalKeyCount++ + // } + // if bytes.HasPrefix(i.Key(), []byte("!badger!Move")) { + // moveKeyCount++ + // } + // if i.IsDeletedOrExpired() { + // invalidKeyCount++ + // } else { + // validKeyCount++ + // } + // } // fetch directory contents if showDir { diff --git a/manual/manual.go b/manual/manual.go index 640816a04..031b7ed84 100644 --- a/manual/manual.go +++ b/manual/manual.go @@ -21,7 +21,7 @@ func throw(s string) // New allocates a slice of size n. The returned slice is from manually managed // memory and MUST be released by calling Free. Failure to do so will result in // a memory leak. -func New(n int) []byte { +func Calloc(n int) []byte { if n == 0 { return make([]byte, 0) } diff --git a/manual/manual_nocgo.go b/manual/manual_nocgo.go index 466d1ee63..bcd638ea4 100644 --- a/manual/manual_nocgo.go +++ b/manual/manual_nocgo.go @@ -10,7 +10,7 @@ package manual // compilation). // New allocates a slice of size n. -func New(n int) []byte { +func Calloc(n int) []byte { return make([]byte, n) } diff --git a/table/builder.go b/table/builder.go index 6fb608c8d..22661d71a 100644 --- a/table/builder.go +++ b/table/builder.go @@ -471,7 +471,7 @@ func (b *Builder) encrypt(data []byte, viaC bool) ([]byte, error) { needSz := len(data) + len(iv) var dst []byte if viaC { - dst = manual.New(needSz) + dst = manual.Calloc(needSz) } else { dst = make([]byte, needSz) } @@ -516,11 +516,11 @@ func (b *Builder) compressData(data []byte) ([]byte, error) { return data, nil case options.Snappy: sz := snappy.MaxEncodedLen(len(data)) - dst := manual.New(sz) + dst := manual.Calloc(sz) return snappy.Encode(dst, data), nil case options.ZSTD: sz := zstd.CompressBound(len(data)) - dst := manual.New(sz) + dst := manual.Calloc(sz) return y.ZSTDCompress(dst, data, b.opt.ZSTDCompressionLevel) } return nil, errors.New("Unsupported compression type") diff --git a/table/table.go b/table/table.go index b7807b6b3..866213e6d 100644 --- a/table/table.go +++ b/table/table.go @@ -204,6 +204,8 @@ type block struct { ref int32 } +var numBlocks int32 + // incrRef increments the ref of a block and return a bool indicating if the // increment was successful. A true value indicates that the block can be used. func (b *block) incrRef() bool { @@ -242,6 +244,7 @@ func (b *block) decrRef() { // will lead to SEGFAULT. if atomic.AddInt32(&b.ref, -1) == 0 && b.freeMe { manual.Free(b.data) + fmt.Printf("freeing up %p. Num Blocks: %d\n", b.data, atomic.AddInt32(&numBlocks, -1)) // blockPool.Put(&b.data) } y.AssertTrue(atomic.LoadInt32(&b.ref) >= 0) @@ -530,6 +533,7 @@ func (t *Table) block(idx int) (*block, error) { offset: int(ko.Offset), ref: 1, } + atomic.AddInt32(&numBlocks, 1) var err error if blk.data, err = t.read(blk.offset, int(ko.Len)); err != nil { return nil, errors.Wrapf(err, @@ -799,7 +803,7 @@ func (t *Table) decompress(b *block) error { return nil case options.Snappy: if sz, err := snappy.DecodedLen(b.data); err == nil { - dst = manual.New(sz) + dst = manual.Calloc(sz) addr = &dst[0] } b.data, err = snappy.Decode(dst, b.data) @@ -808,7 +812,7 @@ func (t *Table) decompress(b *block) error { return errors.Wrap(err, "failed to decompress") } case options.ZSTD: - dst = manual.New(len(b.data) * 3) // We have to guess. + dst = manual.Calloc(len(b.data) * 3) // We have to guess. addr = &dst[0] b.data, err = y.ZSTDDecompress(dst, b.data) if err != nil { From f70c560091723abcd8c8007029c510deef20205f Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Tue, 11 Aug 2020 13:12:44 -0700 Subject: [PATCH 07/24] The allocations and deallocations match up --- manual/manual.go | 9 ++++++++- table/table.go | 27 +++++++++++++++++---------- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/manual/manual.go b/manual/manual.go index 031b7ed84..8950d4e67 100644 --- a/manual/manual.go +++ b/manual/manual.go @@ -6,7 +6,10 @@ package manual // #include import "C" -import "unsafe" +import ( + "sync/atomic" + "unsafe" +) // The go:linkname directives provides backdoor access to private functions in // the runtime. Below we're accessing the throw function. @@ -18,6 +21,8 @@ func throw(s string) // runtime page allocator and allocate large chunks of memory using mmap or // similar. +var NumAllocs int32 + // New allocates a slice of size n. The returned slice is from manually managed // memory and MUST be released by calling Free. Failure to do so will result in // a memory leak. @@ -44,6 +49,7 @@ func Calloc(n int) []byte { // it cannot allocate memory. throw("out of memory") } + atomic.AddInt32(&NumAllocs, 1) // Interpret the C pointer as a pointer to a Go array, then slice. return (*[MaxArrayLen]byte)(unsafe.Pointer(ptr))[:n:n] } @@ -56,5 +62,6 @@ func Free(b []byte) { } ptr := unsafe.Pointer(&b[0]) C.free(ptr) + atomic.AddInt32(&NumAllocs, -1) } } diff --git a/table/table.go b/table/table.go index 866213e6d..680af00cb 100644 --- a/table/table.go +++ b/table/table.go @@ -242,9 +242,12 @@ func (b *block) decrRef() { // In case of an uncompressed block, the []byte is a reference to the // table.mmap []byte slice. Any attempt to write data to the mmap []byte // will lead to SEGFAULT. - if atomic.AddInt32(&b.ref, -1) == 0 && b.freeMe { - manual.Free(b.data) - fmt.Printf("freeing up %p. Num Blocks: %d\n", b.data, atomic.AddInt32(&numBlocks, -1)) + if atomic.AddInt32(&b.ref, -1) == 0 { + if b.freeMe { + manual.Free(b.data) + } + num := atomic.AddInt32(&numBlocks, -1) + fmt.Printf("Num Blocks: %d. Num Allocs: %d\n", num, atomic.LoadInt32(&manual.NumAllocs)) // blockPool.Put(&b.data) } y.AssertTrue(atomic.LoadInt32(&b.ref) >= 0) @@ -533,7 +536,9 @@ func (t *Table) block(idx int) (*block, error) { offset: int(ko.Offset), ref: 1, } + defer blk.decrRef() // Deal with any errors, where blk would not be returned. atomic.AddInt32(&numBlocks, 1) + var err error if blk.data, err = t.read(blk.offset, int(ko.Len)); err != nil { return nil, errors.Wrapf(err, @@ -593,11 +598,17 @@ func (t *Table) block(idx int) (*block, error) { // new block with ref=1. y.AssertTrue(blk.incrRef()) + // Manish: Set is not guaranteed to actually tell you whether block went into cache or not. + // All it is telling you is that Cache has pushed it to the setBuf channel. The policy can + // still reject the block. So, perhaps change the policy to call eviction handler, if it + // rejects the block. + // Decrement the block ref if we could not insert it in the cache. if !t.opt.Cache.Set(key, blk, blk.size()) { blk.decrRef() } } + blk.incrRef() return blk, nil } @@ -718,7 +729,7 @@ func (t *Table) VerifyChecksum() error { return y.Wrapf(err, "checksum validation failed for table: %s, block: %d, offset:%d", t.Filename(), i, os.Offset) } - b.incrRef() + // We should not call incrRef here, because the block already has one ref when created. defer b.decrRef() // OnBlockRead or OnTableAndBlockRead, we don't need to call verify checksum // on block, verification would be done while reading block itself. @@ -730,7 +741,6 @@ func (t *Table) VerifyChecksum() error { } } } - return nil } @@ -794,7 +804,6 @@ func NewFilename(id uint64, dir string) string { // decompress decompresses the data stored in a block. func (t *Table) decompress(b *block) error { var dst []byte - var addr *byte var err error switch t.opt.Compression { @@ -804,7 +813,6 @@ func (t *Table) decompress(b *block) error { case options.Snappy: if sz, err := snappy.DecodedLen(b.data); err == nil { dst = manual.Calloc(sz) - addr = &dst[0] } b.data, err = snappy.Decode(dst, b.data) if err != nil { @@ -812,8 +820,7 @@ func (t *Table) decompress(b *block) error { return errors.Wrap(err, "failed to decompress") } case options.ZSTD: - dst = manual.Calloc(len(b.data) * 3) // We have to guess. - addr = &dst[0] + dst = manual.Calloc(len(b.data) * 4) // We have to guess. b.data, err = y.ZSTDDecompress(dst, b.data) if err != nil { manual.Free(dst) @@ -823,7 +830,7 @@ func (t *Table) decompress(b *block) error { return errors.New("Unsupported compression type") } - if len(b.data) > 0 && addr != &b.data[0] { + if len(b.data) > 0 && len(dst) > 0 && &dst[0] != &b.data[0] { manual.Free(dst) } else { b.freeMe = true From ea70a96696a0f0f98f9d4302eccc8d35e64c356f Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Tue, 11 Aug 2020 13:38:40 -0700 Subject: [PATCH 08/24] A bit more debugging --- manual/manual.go | 8 ++++---- table/builder.go | 11 ----------- table/table.go | 3 ++- 3 files changed, 6 insertions(+), 16 deletions(-) diff --git a/manual/manual.go b/manual/manual.go index 8950d4e67..e5b277eab 100644 --- a/manual/manual.go +++ b/manual/manual.go @@ -21,7 +21,7 @@ func throw(s string) // runtime page allocator and allocate large chunks of memory using mmap or // similar. -var NumAllocs int32 +var NumAllocs int64 // New allocates a slice of size n. The returned slice is from manually managed // memory and MUST be released by calling Free. Failure to do so will result in @@ -49,19 +49,19 @@ func Calloc(n int) []byte { // it cannot allocate memory. throw("out of memory") } - atomic.AddInt32(&NumAllocs, 1) + atomic.AddInt64(&NumAllocs, int64(n)) // Interpret the C pointer as a pointer to a Go array, then slice. return (*[MaxArrayLen]byte)(unsafe.Pointer(ptr))[:n:n] } // Free frees the specified slice. func Free(b []byte) { - if cap(b) != 0 { + if sz := cap(b); sz != 0 { if len(b) == 0 { b = b[:cap(b)] } ptr := unsafe.Pointer(&b[0]) C.free(ptr) - atomic.AddInt32(&NumAllocs, -1) + atomic.AddInt64(&NumAllocs, -int64(sz)) } } diff --git a/table/builder.go b/table/builder.go index 22661d71a..095a4e3a0 100644 --- a/table/builder.go +++ b/table/builder.go @@ -120,17 +120,6 @@ func NewTableBuilder(opts Options) *Builder { return b } -// var blockPool = &sync.Pool{ -// New: func() interface{} { -// // Create 5 Kb blocks even when the default size of blocks is 4 KB. The -// // ZSTD decompresion library increases the buffer by 2X if it's not big -// // enough. Using a 5 KB block instead of a 4 KB one avoids the -// // unncessary 2X allocation by the decompression library. -// b := make([]byte, 5<<10) -// return &b -// }, -// } - func (b *Builder) handleBlock() { defer b.wg.Done() diff --git a/table/table.go b/table/table.go index 680af00cb..72f81dfe3 100644 --- a/table/table.go +++ b/table/table.go @@ -247,7 +247,8 @@ func (b *block) decrRef() { manual.Free(b.data) } num := atomic.AddInt32(&numBlocks, -1) - fmt.Printf("Num Blocks: %d. Num Allocs: %d\n", num, atomic.LoadInt32(&manual.NumAllocs)) + allocs := float64(atomic.LoadInt64(&manual.NumAllocs)) / float64((1 << 20)) + fmt.Printf("Num Blocks: %d. Num Allocs (MB): %.2f\n", num, allocs) // blockPool.Put(&b.data) } y.AssertTrue(atomic.LoadInt32(&b.ref) >= 0) From 3e660bbfac192a08dfb49486b5b5cb2164da576d Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 12 Aug 2020 05:58:10 -0700 Subject: [PATCH 09/24] Switch table builder to use Calloc. --- db.go | 3 ++- levels.go | 6 +++--- stream_writer.go | 3 ++- table/builder.go | 17 +++++++++++++---- 4 files changed, 20 insertions(+), 9 deletions(-) diff --git a/db.go b/db.go index a95d1dccc..68ee39a1c 100644 --- a/db.go +++ b/db.go @@ -977,6 +977,7 @@ func buildL0Table(ft flushTask, bopts table.Options) []byte { defer iter.Close() b := table.NewTableBuilder(bopts) defer b.Close() + var vp valuePointer for iter.SeekToFirst(); iter.Valid(); iter.Next() { if len(ft.dropPrefixes) > 0 && hasAnyPrefixes(iter.Key(), ft.dropPrefixes) { @@ -988,7 +989,7 @@ func buildL0Table(ft flushTask, bopts table.Options) []byte { } b.Add(iter.Key(), iter.Value(), vp.Len) } - return b.Finish() + return b.Finish(true) } type flushTask struct { diff --git a/levels.go b/levels.go index 78509391c..c57015b56 100644 --- a/levels.go +++ b/levels.go @@ -652,7 +652,7 @@ nextTable: return nil, errors.Wrapf(err, "While opening new table: %d", fileID) } - if _, err := fd.Write(builder.Finish()); err != nil { + if _, err := fd.Write(builder.Finish(false)); err != nil { return nil, errors.Wrapf(err, "Unable to write to file: %d", fileID) } tbl, err := table.OpenTable(fd, bopts) @@ -661,7 +661,7 @@ nextTable: } if builder.Empty() { // Cleanup builder resources: - builder.Finish() + builder.Finish(false) builder.Close() continue } @@ -674,7 +674,7 @@ nextTable: err error ) if s.kv.opt.InMemory { - tbl, err = table.OpenInMemoryTable(builder.Finish(), fileID, &bopts) + tbl, err = table.OpenInMemoryTable(builder.Finish(true), fileID, &bopts) } else { tbl, err = build(fileID) } diff --git a/stream_writer.go b/stream_writer.go index 3ed865c75..4e3793419 100644 --- a/stream_writer.go +++ b/stream_writer.go @@ -377,6 +377,7 @@ func (w *sortedWriter) send(done bool) error { return err } go func(builder *table.Builder) { + defer builder.Close() err := w.createTable(builder) w.throttle.Done(err) }(w.builder) @@ -410,7 +411,7 @@ func (w *sortedWriter) Done() error { } func (w *sortedWriter) createTable(builder *table.Builder) error { - data := builder.Finish() + data := builder.Finish(w.db.opt.InMemory) if len(data) == 0 { return nil } diff --git a/table/builder.go b/table/builder.go index 095a4e3a0..56f580f33 100644 --- a/table/builder.go +++ b/table/builder.go @@ -98,7 +98,7 @@ func NewTableBuilder(opts Options) *Builder { b := &Builder{ // Additional 5 MB to store index (approximate). // We trim the additional space in table.Finish(). - buf: make([]byte, opts.TableSize+5*MB), + buf: manual.Calloc(int(opts.TableSize + 16*MB)), tableIndex: &pb.TableIndex{}, keyHashes: make([]uint64, 0, 1024), // Avoid some malloc calls. opt: &opts, @@ -164,7 +164,9 @@ func (b *Builder) handleBlock() { } // Close closes the TableBuilder. -func (b *Builder) Close() {} +func (b *Builder) Close() { + manual.Free(b.buf) +} // Empty returns whether it's empty. func (b *Builder) Empty() bool { return b.sz == 0 } @@ -227,9 +229,12 @@ func (b *Builder) grow(n uint32) { if n < l/2 { n = l / 2 } + newBuf := manual.Calloc(int(l + n)) + y.AssertTrue(uint32(len(newBuf)) == l+n) + b.bufLock.Lock() - newBuf := make([]byte, l+n) copy(newBuf, b.buf) + manual.Free(b.buf) b.buf = newBuf b.bufLock.Unlock() } @@ -365,7 +370,7 @@ The table structure looks like +---------+------------+-----------+---------------+ */ // In case the data is encrypted, the "IV" is added to the end of the index. -func (b *Builder) Finish() []byte { +func (b *Builder) Finish(allocate bool) []byte { if b.opt.BloomFalsePositive > 0 { bf := z.NewBloomFilter(float64(len(b.keyHashes)), b.opt.BloomFalsePositive) for _, h := range b.keyHashes { @@ -418,6 +423,10 @@ func (b *Builder) Finish() []byte { b.append(y.U32ToBytes(uint32(len(index)))) b.writeChecksum(index) + + if allocate { + return append([]byte{}, b.buf[:b.sz]...) + } return b.buf[:b.sz] } From e254eccfcb322a097fac381bd7ef313ebb2b717b Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 12 Aug 2020 06:42:56 -0700 Subject: [PATCH 10/24] Switch levels.go compaction to only build 5 tables at a time. --- badger/main.go | 2 +- levels.go | 83 +++++++++++++++++++++++--------------------------- 2 files changed, 39 insertions(+), 46 deletions(-) diff --git a/badger/main.go b/badger/main.go index 7542f1e17..1d6bcd65e 100644 --- a/badger/main.go +++ b/badger/main.go @@ -29,7 +29,7 @@ func main() { go func() { for i := 8080; i < 9080; i++ { fmt.Printf("Listening for /debug HTTP requests at port: %d\n", i) - if err := http.ListenAndServe(fmt.Sprintf("localhost:%d", i), nil); err != nil { + if err := http.ListenAndServe(fmt.Sprintf("0.0.0.0:%d", i), nil); err != nil { fmt.Println("Port busy. Trying another one...") continue diff --git a/levels.go b/levels.go index c57015b56..fd7639651 100644 --- a/levels.go +++ b/levels.go @@ -541,15 +541,14 @@ nextTable: // that would affect the snapshot view guarantee provided by transactions. discardTs := s.kv.orc.discardAtOrBelow() - // Start generating new tables. - type newTableResult struct { - table *table.Table - err error - } - resultCh := make(chan newTableResult) var numBuilds, numVersions int var lastKey, skipKey []byte var vp valuePointer + var newTables []*table.Table + mu := new(sync.Mutex) // Guards newTables + + inflightBuilders := y.NewThrottle(5) + y.Check(inflightBuilders.Do()) for it.Valid() { timeStart := time.Now() dk, err := s.kv.registry.latestDataKey() @@ -646,19 +645,6 @@ nextTable: // called Add() at least once, and builder is not Empty(). s.kv.opt.Debugf("LOG Compact. Added %d keys. Skipped %d keys. Iteration took: %v", numKeys, numSkips, time.Since(timeStart)) - build := func(fileID uint64) (*table.Table, error) { - fd, err := y.CreateSyncedFile(table.NewFilename(fileID, s.kv.opt.Dir), true) - if err != nil { - return nil, errors.Wrapf(err, "While opening new table: %d", fileID) - } - - if _, err := fd.Write(builder.Finish(false)); err != nil { - return nil, errors.Wrapf(err, "Unable to write to file: %d", fileID) - } - tbl, err := table.OpenTable(fd, bopts) - // decrRef is added below. - return tbl, errors.Wrapf(err, "Unable to open table: %q", fd.Name()) - } if builder.Empty() { // Cleanup builder resources: builder.Finish(false) @@ -667,49 +653,56 @@ nextTable: } numBuilds++ fileID := s.reserveFileID() + if err := inflightBuilders.Do(); err != nil { + // Can't return from here, until I decrRef all the tables that I built so far. + break + } go func(builder *table.Builder) { defer builder.Close() - var ( - tbl *table.Table - err error - ) + + build := func(fileID uint64) (*table.Table, error) { + fd, err := y.CreateSyncedFile(table.NewFilename(fileID, s.kv.opt.Dir), true) + if err != nil { + return nil, errors.Wrapf(err, "While opening new table: %d", fileID) + } + + if _, err := fd.Write(builder.Finish(false)); err != nil { + return nil, errors.Wrapf(err, "Unable to write to file: %d", fileID) + } + tbl, err := table.OpenTable(fd, bopts) + // decrRef is added below. + return tbl, errors.Wrapf(err, "Unable to open table: %q", fd.Name()) + } + + var tbl *table.Table + var err error if s.kv.opt.InMemory { tbl, err = table.OpenInMemoryTable(builder.Finish(true), fileID, &bopts) } else { tbl, err = build(fileID) } - resultCh <- newTableResult{tbl, err} - }(builder) - } + inflightBuilders.Done(err) - newTables := make([]*table.Table, 0, 20) - // Wait for all table builders to finish. - var firstErr error - for x := 0; x < numBuilds; x++ { - res := <-resultCh - newTables = append(newTables, res.table) - if firstErr == nil { - firstErr = res.err - } + mu.Lock() + newTables = append(newTables, tbl) + mu.Unlock() + }(builder) } - if firstErr == nil { + // Wait for all table builders to finish and also for newTables accumulator to finish. + err := inflightBuilders.Finish() + if err == nil { // Ensure created files' directory entries are visible. We don't mind the extra latency // from not doing this ASAP after all file creation has finished because this is a // background operation. - firstErr = s.kv.syncDir(s.kv.opt.Dir) + err = s.kv.syncDir(s.kv.opt.Dir) } - if firstErr != nil { + if err != nil { // An error happened. Delete all the newly created table files (by calling DecrRef // -- we're the only holders of a ref). - for j := 0; j < numBuilds; j++ { - if newTables[j] != nil { - _ = newTables[j].DecrRef() - } - } - errorReturn := errors.Wrapf(firstErr, "While running compaction for: %+v", cd) - return nil, nil, errorReturn + _ = decrRefs(newTables) + return nil, nil, errors.Wrapf(err, "while running compactions for: %+v", cd) } sort.Slice(newTables, func(i, j int) bool { From f416452ca4c029d9629c84f2cd23218982687fa1 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 12 Aug 2020 07:55:21 -0700 Subject: [PATCH 11/24] Have a dedicated compactor for L0 and L1. --- badger/cmd/write_bench.go | 6 ++--- db.go | 7 ++++-- level_handler.go | 5 ++--- levels.go | 47 ++++++++++++++++++++++++++------------- options.go | 6 ++--- table/table.go | 8 +++---- 6 files changed, 47 insertions(+), 32 deletions(-) diff --git a/badger/cmd/write_bench.go b/badger/cmd/write_bench.go index ac76cb609..dbe244a46 100644 --- a/badger/cmd/write_bench.go +++ b/badger/cmd/write_bench.go @@ -364,9 +364,9 @@ func reportStats(c *y.Closer, db *badger.DB) { fmt.Printf("Time elapsed: %s, bytes written: %s, speed: %s/sec, "+ "entries written: %d, speed: %d/sec, gcSuccess: %d\n", y.FixedDuration(time.Since(startTime)), humanize.Bytes(sz), humanize.Bytes(bytesRate), entries, entriesRate, gcSuccess) - fmt.Printf("Valid Keys Count: %d\nInvalid Keys Count: %d\nMove Keys Count: %d\n"+ - "Internal Keys Count: %d\n", validKeyCount, invalidKeyCount, moveKeyCount, - internalKeyCount) + // fmt.Printf("Valid Keys Count: %d\nInvalid Keys Count: %d\nMove Keys Count: %d\n"+ + // "Internal Keys Count: %d\n", validKeyCount, invalidKeyCount, moveKeyCount, + // internalKeyCount) } } } diff --git a/db.go b/db.go index 68ee39a1c..96de553f2 100644 --- a/db.go +++ b/db.go @@ -192,6 +192,9 @@ func (db *DB) replayFunction() func(Entry, valuePointer) error { // Open returns a new DB object. func Open(opt Options) (db *DB, err error) { + if opt.NumCompactors < 2 { + return nil, errors.New("Cannot have less than 2 compactors") + } if opt.InMemory && (opt.Dir != "" || opt.ValueDir != "") { return nil, errors.New("Cannot use badger in Disk-less mode with Dir or ValueDir set") } @@ -531,7 +534,7 @@ func (db *DB) close() (err error) { // Force Compact L0 // We don't need to care about cstatus since no parallel compaction is running. if db.opt.CompactL0OnClose { - err := db.lc.doCompact(compactionPriority{level: 0, score: 1.73}) + err := db.lc.doCompact(173, compactionPriority{level: 0, score: 1.73}) switch err { case errFillTables: // This error only means that there might be enough tables to do a compaction. So, we @@ -1442,7 +1445,7 @@ func (db *DB) Flatten(workers int) error { errCh := make(chan error, 1) for i := 0; i < workers; i++ { go func() { - errCh <- db.lc.doCompact(cp) + errCh <- db.lc.doCompact(174, cp) }() } var success int diff --git a/level_handler.go b/level_handler.go index 19ba0892b..ce48965fe 100644 --- a/level_handler.go +++ b/level_handler.go @@ -188,9 +188,8 @@ func (s *levelHandler) tryAddLevel0Table(t *table.Table) bool { // Need lock as we may be deleting the first table during a level 0 compaction. s.Lock() defer s.Unlock() - // Return false only if L0 is in memory and number of tables is more than number of - // ZeroTableStall. For on disk L0, we should just add the tables to the level. - if s.db.opt.KeepL0InMemory && len(s.tables) >= s.db.opt.NumLevelZeroTablesStall { + // Stall (by returning false) if we are above the specified stall setting for L0. + if len(s.tables) >= s.db.opt.NumLevelZeroTablesStall { return false } diff --git a/levels.go b/levels.go index fd7639651..61ff2fa54 100644 --- a/levels.go +++ b/levels.go @@ -29,6 +29,7 @@ import ( "golang.org/x/net/trace" + "github.com/dgraph-io/badger/v2/manual" "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/table" "github.com/dgraph-io/badger/v2/y" @@ -306,7 +307,7 @@ func (s *levelsController) dropPrefixes(prefixes [][]byte) error { // function in logs, and forces a compaction. dropPrefixes: prefixes, } - if err := s.doCompact(cp); err != nil { + if err := s.doCompact(175, cp); err != nil { opt.Warningf("While compacting level 0: %v", err) return nil } @@ -366,11 +367,13 @@ func (s *levelsController) startCompact(lc *y.Closer) { n := s.kv.opt.NumCompactors lc.AddRunning(n - 1) for i := 0; i < n; i++ { - go s.runWorker(lc) + // The worker with id=0 is dedicated to L0 and L1. This is not counted + // towards the user specified NumCompactors. + go s.runCompactor(i, lc) } } -func (s *levelsController) runWorker(lc *y.Closer) { +func (s *levelsController) runCompactor(id int, lc *y.Closer) { defer lc.Done() randomDelay := time.NewTimer(time.Duration(rand.Int31n(1000)) * time.Millisecond) @@ -381,7 +384,7 @@ func (s *levelsController) runWorker(lc *y.Closer) { return } - ticker := time.NewTicker(time.Second) + ticker := time.NewTicker(100 * time.Millisecond) defer ticker.Stop() for { @@ -391,7 +394,15 @@ func (s *levelsController) runWorker(lc *y.Closer) { prios := s.pickCompactLevels() loop: for _, p := range prios { - err := s.doCompact(p) + if id == 0 && p.level > 1 { + // If I'm ID zero, I only compact L0 and L1. + continue + } + if id != 0 && p.level <= 1 { + // If I'm ID non-zero, I do NOT compact L0 and L1. + continue + } + err := s.doCompact(id, p) switch err { case nil: break loop @@ -453,10 +464,11 @@ func (s *levelsController) pickCompactLevels() (prios []compactionPriority) { prios = append(prios, pri) } } - // We used to sort compaction priorities based on the score. But, we - // decided to compact based on the level, not the priority. So, upper - // levels (level 0, level 1, etc) always get compacted first, before the - // lower levels -- this allows us to avoid stalls. + // We should continue to sort the compaction priorities by score. Now that we have a dedicated + // compactor for L0 and L1, we don't need to sort by level here. + sort.Slice(prios, func(i, j int) bool { + return prios[i].score > prios[j].score + }) return prios } @@ -548,7 +560,6 @@ nextTable: mu := new(sync.Mutex) // Guards newTables inflightBuilders := y.NewThrottle(5) - y.Check(inflightBuilders.Do()) for it.Valid() { timeStart := time.Now() dk, err := s.kv.registry.latestDataKey() @@ -685,6 +696,9 @@ nextTable: mu.Lock() newTables = append(newTables, tbl) + num := atomic.LoadInt32(&table.NumBlocks) + allocs := float64(atomic.LoadInt64(&manual.NumAllocs)) / float64((1 << 20)) + fmt.Printf("Num Blocks: %d. Num Allocs (MB): %.2f\n", num, allocs) mu.Unlock() }(builder) } @@ -956,7 +970,7 @@ func (s *levelsController) runCompactDef(l int, cd compactDef) (err error) { var errFillTables = errors.New("Unable to fill tables") // doCompact picks some table on level l and compacts it away to the next level. -func (s *levelsController) doCompact(p compactionPriority) error { +func (s *levelsController) doCompact(id int, p compactionPriority) error { l := p.level y.AssertTrue(l+1 < s.kv.opt.MaxLevels) // Sanity check. @@ -969,7 +983,7 @@ func (s *levelsController) doCompact(p compactionPriority) error { cd.elog.SetMaxEvents(100) defer cd.elog.Finish() - s.kv.opt.Infof("Got compaction priority: %+v", p) + s.kv.opt.Debugf("[Compactor: %d] Attempting to run compaction: %+v", id, p) // While picking tables to be compacted, both levels' tables are expected to // remain unchanged. @@ -985,16 +999,17 @@ func (s *levelsController) doCompact(p compactionPriority) error { } defer s.cstatus.delete(cd) // Remove the ranges from compaction status. - s.kv.opt.Infof("Running for level: %d\n", cd.thisLevel.level) + s.kv.opt.Infof("[Compactor: %d] Running compaction: %+v for level: %d\n", + id, p, cd.thisLevel.level) s.cstatus.toLog(cd.elog) if err := s.runCompactDef(l, cd); err != nil { // This compaction couldn't be done successfully. - s.kv.opt.Warningf("LOG Compact FAILED with error: %+v: %+v", err, cd) + s.kv.opt.Warningf("[Compactor: %d] LOG Compact FAILED with error: %+v: %+v", id, err, cd) return err } s.cstatus.toLog(cd.elog) - s.kv.opt.Infof("Compaction for level: %d DONE", cd.thisLevel.level) + s.kv.opt.Infof("[Compactor: %d] Compaction for level: %d DONE", id, cd.thisLevel.level) return nil } @@ -1018,7 +1033,7 @@ func (s *levelsController) addLevel0Table(t *table.Table) error { // Stall. Make sure all levels are healthy before we unstall. var timeStart time.Time { - s.kv.opt.Debugf("STALLED STALLED STALLED: %v\n", time.Since(s.lastUnstalled)) + s.kv.opt.Infof("STALLED STALLED STALLED: %v\n", time.Since(s.lastUnstalled)) s.cstatus.RLock() for i := 0; i < s.kv.opt.MaxLevels; i++ { s.kv.opt.Debugf("level=%d. Status=%s Size=%d\n", diff --git a/options.go b/options.go index fa121a4b5..3f40d0838 100644 --- a/options.go +++ b/options.go @@ -129,9 +129,9 @@ func DefaultOptions(path string) Options { // table.Nothing to not preload the tables. MaxLevels: 7, MaxTableSize: 64 << 20, - NumCompactors: 2, // Compactions can be expensive. Only run 2. + NumCompactors: 2, // Run at least 2 compactors. One is dedicated for L0. NumLevelZeroTables: 5, - NumLevelZeroTablesStall: 10, + NumLevelZeroTablesStall: 15, NumMemtables: 5, BloomFalsePositive: 0.01, BlockSize: 4 * 1024, @@ -461,7 +461,7 @@ func (opt Options) WithValueLogMaxEntries(val uint32) Options { // NumCompactors sets the number of compaction workers to run concurrently. // Setting this to zero stops compactions, which could eventually cause writes to block forever. // -// The default value of NumCompactors is 2. +// The default value of NumCompactors is 2. One is dedicated just for L0. func (opt Options) WithNumCompactors(val int) Options { opt.NumCompactors = val return opt diff --git a/table/table.go b/table/table.go index 72f81dfe3..4518b75f7 100644 --- a/table/table.go +++ b/table/table.go @@ -204,7 +204,7 @@ type block struct { ref int32 } -var numBlocks int32 +var NumBlocks int32 // incrRef increments the ref of a block and return a bool indicating if the // increment was successful. A true value indicates that the block can be used. @@ -246,9 +246,7 @@ func (b *block) decrRef() { if b.freeMe { manual.Free(b.data) } - num := atomic.AddInt32(&numBlocks, -1) - allocs := float64(atomic.LoadInt64(&manual.NumAllocs)) / float64((1 << 20)) - fmt.Printf("Num Blocks: %d. Num Allocs (MB): %.2f\n", num, allocs) + atomic.AddInt32(&NumBlocks, -1) // blockPool.Put(&b.data) } y.AssertTrue(atomic.LoadInt32(&b.ref) >= 0) @@ -538,7 +536,7 @@ func (t *Table) block(idx int) (*block, error) { ref: 1, } defer blk.decrRef() // Deal with any errors, where blk would not be returned. - atomic.AddInt32(&numBlocks, 1) + atomic.AddInt32(&NumBlocks, 1) var err error if blk.data, err = t.read(blk.offset, int(ko.Len)); err != nil { From 878d066e54e7b4e9d5714388115ea3aaffa39138 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Wed, 12 Aug 2020 22:07:33 +0530 Subject: [PATCH 12/24] Add pool/calloc benchmark --- manual/manual_test.go | 74 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 74 insertions(+) create mode 100644 manual/manual_test.go diff --git a/manual/manual_test.go b/manual/manual_test.go new file mode 100644 index 000000000..70b739492 --- /dev/null +++ b/manual/manual_test.go @@ -0,0 +1,74 @@ +/* + * Copyright 2020 Dgraph Labs, Inc. and Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package manual + +import ( + "sync" + "testing" + "time" + + "math/rand" +) + +// $ go test -failfast -run xxx -bench . -benchmem -count 10 > out.txt +// $ benchstat out.txt +// name time/op +// Allocation/Pool-8 200µs ± 5% +// Allocation/Calloc-8 100µs ±11% +// +// name alloc/op +// Allocation/Pool-8 477B ±29% +// Allocation/Calloc-8 4.00B ± 0% +// +// name allocs/op +// Allocation/Pool-8 1.00 ± 0% +// Allocation/Calloc-8 0.00 +func BenchmarkAllocation(b *testing.B) { + b.Run("Pool", func(b *testing.B) { + pool := sync.Pool{ + New: func() interface{} { + return make([]byte, 4<<10) + }, + } + b.RunParallel(func(pb *testing.PB) { + source := rand.NewSource(time.Now().UnixNano()) + r := rand.New(source) + for pb.Next() { + x := pool.Get().([]byte) + sz := r.Intn(100) << 10 + if len(x) < sz { + x = make([]byte, sz) + } + rand.Read(x) + pool.Put(x) + } + }) + }) + + b.Run("Calloc", func(b *testing.B) { + b.RunParallel(func(pb *testing.PB) { + source := rand.NewSource(time.Now().UnixNano()) + r := rand.New(source) + for pb.Next() { + sz := r.Intn(100) << 10 + x := Calloc(sz) + rand.Read(x) + Free(x) + } + }) + }) +} From f698845dbfb740025095aa227394f3cc79acc2f3 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Fri, 14 Aug 2020 15:42:44 +0530 Subject: [PATCH 13/24] Move Calloc and Free to y --- table/builder.go | 25 ++++++++++----------- table/table.go | 13 +++++------ manual/manual.go => y/calloc.go | 2 +- manual/manual_32bit.go => y/calloc_32bit.go | 2 +- manual/manual_64bit.go => y/calloc_64bit.go | 2 +- manual/manual_nocgo.go => y/calloc_nocgo.go | 2 +- manual/manual_test.go => y/calloc_test.go | 2 +- 7 files changed, 23 insertions(+), 25 deletions(-) rename manual/manual.go => y/calloc.go (99%) rename manual/manual_32bit.go => y/calloc_32bit.go (96%) rename manual/manual_64bit.go => y/calloc_64bit.go (96%) rename manual/manual_nocgo.go => y/calloc_nocgo.go (96%) rename manual/manual_test.go => y/calloc_test.go (99%) diff --git a/table/builder.go b/table/builder.go index 56f580f33..dfb681b74 100644 --- a/table/builder.go +++ b/table/builder.go @@ -29,7 +29,6 @@ import ( "github.com/golang/snappy" "github.com/pkg/errors" - "github.com/dgraph-io/badger/v2/manual" "github.com/dgraph-io/badger/v2/options" "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" @@ -98,7 +97,7 @@ func NewTableBuilder(opts Options) *Builder { b := &Builder{ // Additional 5 MB to store index (approximate). // We trim the additional space in table.Finish(). - buf: manual.Calloc(int(opts.TableSize + 16*MB)), + buf: y.Calloc(int(opts.TableSize + 16*MB)), tableIndex: &pb.TableIndex{}, keyHashes: make([]uint64, 0, 1024), // Avoid some malloc calls. opt: &opts, @@ -158,14 +157,14 @@ func (b *Builder) handleBlock() { item.end = item.start + uint32(len(blockBuf)) if doCompress { - manual.Free(blockBuf) + y.Free(blockBuf) } } } // Close closes the TableBuilder. func (b *Builder) Close() { - manual.Free(b.buf) + y.Free(b.buf) } // Empty returns whether it's empty. @@ -229,12 +228,12 @@ func (b *Builder) grow(n uint32) { if n < l/2 { n = l / 2 } - newBuf := manual.Calloc(int(l + n)) + newBuf := y.Calloc(int(l + n)) y.AssertTrue(uint32(len(newBuf)) == l+n) b.bufLock.Lock() copy(newBuf, b.buf) - manual.Free(b.buf) + y.Free(b.buf) b.buf = newBuf b.bufLock.Unlock() } @@ -469,7 +468,7 @@ func (b *Builder) encrypt(data []byte, viaC bool) ([]byte, error) { needSz := len(data) + len(iv) var dst []byte if viaC { - dst = manual.Calloc(needSz) + dst = y.Calloc(needSz) } else { dst = make([]byte, needSz) } @@ -477,12 +476,12 @@ func (b *Builder) encrypt(data []byte, viaC bool) ([]byte, error) { if err = y.XORBlock(dst, data, b.DataKey().Data, iv); err != nil { if viaC { - manual.Free(dst) + y.Free(dst) } return data, y.Wrapf(err, "Error while encrypting in Builder.encrypt") } if viaC { - manual.Free(data) + y.Free(data) } y.AssertTrue(cap(dst)-len(dst) >= len(iv)) @@ -492,10 +491,10 @@ func (b *Builder) encrypt(data []byte, viaC bool) ([]byte, error) { // } else { // // This has to be viaC. // var buf []byte - // buf = manual.New(len(data) + len(iv)) + // buf = y.New(len(data) + len(iv)) // copy(buf, data) // copy(buf[len(data):], iv) - // manual.Free(data) + // y.Free(data) // data = buf // } return dst, nil @@ -514,11 +513,11 @@ func (b *Builder) compressData(data []byte) ([]byte, error) { return data, nil case options.Snappy: sz := snappy.MaxEncodedLen(len(data)) - dst := manual.Calloc(sz) + dst := y.Calloc(sz) return snappy.Encode(dst, data), nil case options.ZSTD: sz := zstd.CompressBound(len(data)) - dst := manual.Calloc(sz) + dst := y.Calloc(sz) return y.ZSTDCompress(dst, data, b.opt.ZSTDCompressionLevel) } return nil, errors.New("Unsupported compression type") diff --git a/table/table.go b/table/table.go index 4518b75f7..ad684d71d 100644 --- a/table/table.go +++ b/table/table.go @@ -35,7 +35,6 @@ import ( "github.com/golang/snappy" "github.com/pkg/errors" - "github.com/dgraph-io/badger/v2/manual" "github.com/dgraph-io/badger/v2/options" "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" @@ -244,7 +243,7 @@ func (b *block) decrRef() { // will lead to SEGFAULT. if atomic.AddInt32(&b.ref, -1) == 0 { if b.freeMe { - manual.Free(b.data) + y.Free(b.data) } atomic.AddInt32(&NumBlocks, -1) // blockPool.Put(&b.data) @@ -811,18 +810,18 @@ func (t *Table) decompress(b *block) error { return nil case options.Snappy: if sz, err := snappy.DecodedLen(b.data); err == nil { - dst = manual.Calloc(sz) + dst = y.Calloc(sz) } b.data, err = snappy.Decode(dst, b.data) if err != nil { - manual.Free(dst) + y.Free(dst) return errors.Wrap(err, "failed to decompress") } case options.ZSTD: - dst = manual.Calloc(len(b.data) * 4) // We have to guess. + dst = y.Calloc(len(b.data) * 4) // We have to guess. b.data, err = y.ZSTDDecompress(dst, b.data) if err != nil { - manual.Free(dst) + y.Free(dst) return errors.Wrap(err, "failed to decompress") } default: @@ -830,7 +829,7 @@ func (t *Table) decompress(b *block) error { } if len(b.data) > 0 && len(dst) > 0 && &dst[0] != &b.data[0] { - manual.Free(dst) + y.Free(dst) } else { b.freeMe = true } diff --git a/manual/manual.go b/y/calloc.go similarity index 99% rename from manual/manual.go rename to y/calloc.go index e5b277eab..d08f97d84 100644 --- a/manual/manual.go +++ b/y/calloc.go @@ -2,7 +2,7 @@ // of this source code is governed by a BSD-style license that can be found in // the LICENSE file. -package manual +package y // #include import "C" diff --git a/manual/manual_32bit.go b/y/calloc_32bit.go similarity index 96% rename from manual/manual_32bit.go rename to y/calloc_32bit.go index 9d1add490..94f17c086 100644 --- a/manual/manual_32bit.go +++ b/y/calloc_32bit.go @@ -4,7 +4,7 @@ // +build 386 amd64p32 arm armbe mips mipsle mips64p32 mips64p32le ppc sparc -package manual +package y const ( // MaxArrayLen is a safe maximum length for slices on this architecture. diff --git a/manual/manual_64bit.go b/y/calloc_64bit.go similarity index 96% rename from manual/manual_64bit.go rename to y/calloc_64bit.go index f03846953..bf8d774fb 100644 --- a/manual/manual_64bit.go +++ b/y/calloc_64bit.go @@ -4,7 +4,7 @@ // +build amd64 arm64 arm64be ppc64 ppc64le mips64 mips64le s390x sparc64 -package manual +package y const ( // MaxArrayLen is a safe maximum length for slices on this architecture. diff --git a/manual/manual_nocgo.go b/y/calloc_nocgo.go similarity index 96% rename from manual/manual_nocgo.go rename to y/calloc_nocgo.go index bcd638ea4..03d518b7c 100644 --- a/manual/manual_nocgo.go +++ b/y/calloc_nocgo.go @@ -4,7 +4,7 @@ // +build !cgo -package manual +package y // Provides versions of New and Free when cgo is not available (e.g. cross // compilation). diff --git a/manual/manual_test.go b/y/calloc_test.go similarity index 99% rename from manual/manual_test.go rename to y/calloc_test.go index 70b739492..544d37414 100644 --- a/manual/manual_test.go +++ b/y/calloc_test.go @@ -14,7 +14,7 @@ * limitations under the License. */ -package manual +package y import ( "sync" From 1c1e17c9bcb7ecf0399afed372733ede5867c9e5 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Fri, 14 Aug 2020 15:48:27 +0530 Subject: [PATCH 14/24] Fix numCompactor error --- db.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/db.go b/db.go index 96de553f2..1254ebe87 100644 --- a/db.go +++ b/db.go @@ -192,8 +192,11 @@ func (db *DB) replayFunction() func(Entry, valuePointer) error { // Open returns a new DB object. func Open(opt Options) (db *DB, err error) { - if opt.NumCompactors < 2 { - return nil, errors.New("Cannot have less than 2 compactors") + // It's okay to have zero compactors which will disable all compactions but + // we cannot have just one compactor otherwise we will end up with all data + // one level 2. + if opt.NumCompactors == 1 { + return nil, errors.New("Cannot have 1 compactor. Need at least 2") } if opt.InMemory && (opt.Dir != "" || opt.ValueDir != "") { return nil, errors.New("Cannot use badger in Disk-less mode with Dir or ValueDir set") From 233be8ee6467c6aad08fce5706a1e672bdc9b97b Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Fri, 14 Aug 2020 15:49:05 +0530 Subject: [PATCH 15/24] Handle nil table in compaction and test fixes --- db.go | 2 +- db2_test.go | 2 +- levels.go | 12 ++++-- levels_test.go | 96 +++++++++++++++++++++--------------------------- manifest_test.go | 2 +- 5 files changed, 52 insertions(+), 62 deletions(-) diff --git a/db.go b/db.go index 1254ebe87..59fbbcce8 100644 --- a/db.go +++ b/db.go @@ -1448,7 +1448,7 @@ func (db *DB) Flatten(workers int) error { errCh := make(chan error, 1) for i := 0; i < workers; i++ { go func() { - errCh <- db.lc.doCompact(174, cp) + errCh <- db.lc.doCompact(175, cp) }() } var success int diff --git a/db2_test.go b/db2_test.go index d86e5a40a..18c6f82ff 100644 --- a/db2_test.go +++ b/db2_test.go @@ -547,7 +547,7 @@ func createTableWithRange(t *testing.T, db *DB, start, end int) *table.Table { fd, err := y.CreateSyncedFile(table.NewFilename(fileID, db.opt.Dir), true) require.NoError(t, err) - _, err = fd.Write(b.Finish()) + _, err = fd.Write(b.Finish(false)) require.NoError(t, err, "unable to write to file") tab, err := table.OpenTable(fd, bopts) diff --git a/levels.go b/levels.go index 61ff2fa54..96b468f78 100644 --- a/levels.go +++ b/levels.go @@ -29,7 +29,6 @@ import ( "golang.org/x/net/trace" - "github.com/dgraph-io/badger/v2/manual" "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/table" "github.com/dgraph-io/badger/v2/y" @@ -307,7 +306,7 @@ func (s *levelsController) dropPrefixes(prefixes [][]byte) error { // function in logs, and forces a compaction. dropPrefixes: prefixes, } - if err := s.doCompact(175, cp); err != nil { + if err := s.doCompact(174, cp); err != nil { opt.Warningf("While compacting level 0: %v", err) return nil } @@ -694,11 +693,16 @@ nextTable: } inflightBuilders.Done(err) + // If we couldn't build the table, return fast. + if err != nil { + return + } + mu.Lock() newTables = append(newTables, tbl) num := atomic.LoadInt32(&table.NumBlocks) - allocs := float64(atomic.LoadInt64(&manual.NumAllocs)) / float64((1 << 20)) - fmt.Printf("Num Blocks: %d. Num Allocs (MB): %.2f\n", num, allocs) + allocs := float64(atomic.LoadInt64(&y.NumAllocs)) / float64((1 << 20)) + s.kv.opt.Logger.Debugf("Num Blocks: %d. Num Allocs (MB): %.2f\n", num, allocs) mu.Unlock() }(builder) } diff --git a/levels_test.go b/levels_test.go index b50217ed3..f8aa65e89 100644 --- a/levels_test.go +++ b/levels_test.go @@ -49,7 +49,7 @@ func createAndOpen(db *DB, td []keyValVersion, level int) { panic(err) } - if _, err = fd.Write(b.Finish()); err != nil { + if _, err = fd.Write(b.Finish(false)); err != nil { panic(err) } tab, err := table.OpenTable(fd, opts) @@ -740,7 +740,7 @@ func createEmptyTable(db *DB) *table.Table { b.Add(y.KeyWithTs([]byte("foo"), 1), y.ValueStruct{}, 0) // Open table in memory to avoid adding changes to manifest file. - tab, err := table.OpenInMemoryTable(b.Finish(), db.lc.reserveFileID(), &opts) + tab, err := table.OpenInMemoryTable(b.Finish(true), db.lc.reserveFileID(), &opts) if err != nil { panic(err) } @@ -749,52 +749,6 @@ func createEmptyTable(db *DB) *table.Table { } func TestL0Stall(t *testing.T) { - test := func(t *testing.T, opt *Options) { - runBadgerTest(t, opt, func(t *testing.T, db *DB) { - db.lc.levels[0].Lock() - // Add NumLevelZeroTableStall+1 number of tables to level 0. This would fill up level - // zero and all new additions are expected to stall if L0 is in memory. - for i := 0; i < opt.NumLevelZeroTablesStall+1; i++ { - db.lc.levels[0].tables = append(db.lc.levels[0].tables, createEmptyTable(db)) - } - db.lc.levels[0].Unlock() - - timeout := time.After(5 * time.Second) - done := make(chan bool) - - go func() { - tab := createEmptyTable(db) - require.NoError(t, db.lc.addLevel0Table(tab)) - tab.DecrRef() - done <- true - }() - // Let it stall for a second. - time.Sleep(time.Second) - - select { - case <-timeout: - if opt.KeepL0InMemory { - t.Log("Timeout triggered") - // Mark this test as successful since L0 is in memory and the - // addition of new table to L0 is supposed to stall. - - // Remove tables from level 0 so that the stalled - // compaction can make progress. This does not have any - // effect on the test. This is done so that the goroutine - // stuck on addLevel0Table can make progress and end. - db.lc.levels[0].Lock() - db.lc.levels[0].tables = nil - db.lc.levels[0].Unlock() - <-done - } else { - t.Fatal("Test didn't finish in time") - } - case <-done: - // The test completed before 5 second timeout. Mark it as successful. - } - }) - } - opt := DefaultOptions("") // Disable all compactions. opt.NumCompactors = 0 @@ -803,13 +757,45 @@ func TestL0Stall(t *testing.T) { // Addition of new tables will stall if there are 4 or more L0 tables. opt.NumLevelZeroTablesStall = 4 - t.Run("with KeepL0InMemory", func(t *testing.T) { - opt.KeepL0InMemory = true - test(t, &opt) - }) - t.Run("with L0 on disk", func(t *testing.T) { - opt.KeepL0InMemory = false - test(t, &opt) + runBadgerTest(t, &opt, func(t *testing.T, db *DB) { + db.lc.levels[0].Lock() + // Add NumLevelZeroTableStall+1 number of tables to level 0. This would fill up level + // zero and all new additions are expected to stall if L0 is in memory. + for i := 0; i < opt.NumLevelZeroTablesStall+1; i++ { + db.lc.levels[0].tables = append(db.lc.levels[0].tables, createEmptyTable(db)) + } + db.lc.levels[0].Unlock() + + timeout := time.After(5 * time.Second) + done := make(chan bool) + + go func() { + tab := createEmptyTable(db) + require.NoError(t, db.lc.addLevel0Table(tab)) + tab.DecrRef() + done <- true + }() + // Let it stall for a second. + time.Sleep(time.Second) + + select { + case <-timeout: + t.Log("Timeout triggered") + // Mark this test as successful since L0 is in memory and the + // addition of new table to L0 is supposed to stall. + + // Remove tables from level 0 so that the stalled + // compaction can make progress. This does not have any + // effect on the test. This is done so that the goroutine + // stuck on addLevel0Table can make progress and end. + db.lc.levels[0].Lock() + db.lc.levels[0].tables = nil + db.lc.levels[0].Unlock() + <-done + case <-done: + // The test completed before 5 second timeout. Mark it as successful. + t.Fatal("Test did not stall") + } }) } diff --git a/manifest_test.go b/manifest_test.go index 5062b3f1b..c52e719d5 100644 --- a/manifest_test.go +++ b/manifest_test.go @@ -155,7 +155,7 @@ func buildTable(t *testing.T, keyValues [][]string, bopts table.Options) *os.Fil UserMeta: 0, }, 0) } - _, err = f.Write(b.Finish()) + _, err = f.Write(b.Finish(false)) require.NoError(t, err, "unable to write to file.") f.Close() f, _ = y.OpenSyncedFile(filename, true) From 378491d2b0b1d78dc902357cd7b9ff6594d24920 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Fri, 14 Aug 2020 15:50:52 +0530 Subject: [PATCH 16/24] Revert "Revert "add assert to check integer overflow for table size (#1402)" (#1406)" This reverts commit e0d058ce7a8931e235f9fb5e37aa761f28017f5d. --- table/builder.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/table/builder.go b/table/builder.go index dfb681b74..112251a0c 100644 --- a/table/builder.go +++ b/table/builder.go @@ -321,6 +321,9 @@ func (b *Builder) shouldFinishBlock(key []byte, value y.ValueStruct) bool { // So, size of IV is added to estimatedSize. estimatedSize += aes.BlockSize } + // Integer overflow check for table size. + y.AssertTrue(uint64(b.sz)+uint64(estimatedSize) < math.MaxUint32) + return estimatedSize > uint32(b.opt.BlockSize) } From 84df81772a9e8f5d27bbd551d05aacf3fd4e2e1d Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Fri, 14 Aug 2020 17:46:58 +0530 Subject: [PATCH 17/24] Make level multiplier 15 --- options.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/options.go b/options.go index 3f40d0838..1f90b7829 100644 --- a/options.go +++ b/options.go @@ -122,7 +122,7 @@ func DefaultOptions(path string) Options { Dir: path, ValueDir: path, LevelOneSize: 256 << 20, - LevelSizeMultiplier: 10, + LevelSizeMultiplier: 15, TableLoadingMode: options.MemoryMap, ValueLogLoadingMode: options.MemoryMap, // table.MemoryMap to mmap() the tables. @@ -336,7 +336,7 @@ func (opt Options) WithMaxTableSize(val int64) Options { // Once a level grows to be larger than this ratio allowed, the compaction process will be // triggered. // -// The default value of LevelSizeMultiplier is 10. +// The default value of LevelSizeMultiplier is 15. func (opt Options) WithLevelSizeMultiplier(val int) Options { opt.LevelSizeMultiplier = val return opt From 915b0a5765760d985f3f5a4d65baccac3a9128fc Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Fri, 14 Aug 2020 07:20:37 -0700 Subject: [PATCH 18/24] Don't use Rand, instead use local rand instance. --- y/calloc_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/y/calloc_test.go b/y/calloc_test.go index 544d37414..171422bd0 100644 --- a/y/calloc_test.go +++ b/y/calloc_test.go @@ -53,7 +53,7 @@ func BenchmarkAllocation(b *testing.B) { if len(x) < sz { x = make([]byte, sz) } - rand.Read(x) + r.Read(x) pool.Put(x) } }) @@ -66,7 +66,7 @@ func BenchmarkAllocation(b *testing.B) { for pb.Next() { sz := r.Intn(100) << 10 x := Calloc(sz) - rand.Read(x) + r.Read(x) Free(x) } }) From d5527e72515c3f04d0ab3bacf932e2abf98563a4 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Fri, 14 Aug 2020 07:33:20 -0700 Subject: [PATCH 19/24] Pre-allocate 1.2x of Block Size for decompression. --- table/table.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/table/table.go b/table/table.go index ad684d71d..8b55206de 100644 --- a/table/table.go +++ b/table/table.go @@ -818,7 +818,8 @@ func (t *Table) decompress(b *block) error { return errors.Wrap(err, "failed to decompress") } case options.ZSTD: - dst = y.Calloc(len(b.data) * 4) // We have to guess. + sz := int(float64(t.opt.BlockSize) * 1.2) + dst = y.Calloc(sz) b.data, err = y.ZSTDDecompress(dst, b.data) if err != nil { y.Free(dst) From 37b8b9e174c32dddf55867655628a0d20672041d Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Mon, 17 Aug 2020 05:55:49 -0700 Subject: [PATCH 20/24] Bring in latest Ristretto. Deal with rejected blocks. --- db.go | 7 +++++-- go.mod | 2 +- go.sum | 4 ++-- table/table.go | 10 ++++------ 4 files changed, 12 insertions(+), 11 deletions(-) diff --git a/db.go b/db.go index 59fbbcce8..d76c7e709 100644 --- a/db.go +++ b/db.go @@ -318,8 +318,11 @@ func Open(opt Options) (db *DB, err error) { MaxCost: int64(float64(opt.MaxCacheSize) * 0.95), BufferItems: 64, Metrics: true, - OnEvict: func(_, _ uint64, value interface{}, _ int64) { - table.BlockEvictHandler(value) + OnEvict: func(i *ristretto.Item) { + table.BlockEvictHandler(i.Value) + }, + OnReject: func(i *ristretto.Item) { + table.BlockEvictHandler(i.Value) }, } db.blockCache, err = ristretto.NewCache(&config) diff --git a/go.mod b/go.mod index 6cb85b77c..ae61b5e4e 100644 --- a/go.mod +++ b/go.mod @@ -5,7 +5,7 @@ go 1.12 require ( github.com/DataDog/zstd v1.4.1 github.com/cespare/xxhash v1.1.0 - github.com/dgraph-io/ristretto v0.0.3-0.20200630154024-f66de99634de + github.com/dgraph-io/ristretto v0.0.4-0.20200817124926-18e279725890 github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 github.com/dustin/go-humanize v1.0.0 github.com/golang/protobuf v1.3.1 diff --git a/go.sum b/go.sum index a4aa207f9..bdc803b74 100644 --- a/go.sum +++ b/go.sum @@ -13,8 +13,8 @@ github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwc github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dgraph-io/ristretto v0.0.3-0.20200630154024-f66de99634de h1:t0UHb5vdojIDUqktM6+xJAfScFBsVpXZmqC9dsgJmeA= -github.com/dgraph-io/ristretto v0.0.3-0.20200630154024-f66de99634de/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= +github.com/dgraph-io/ristretto v0.0.4-0.20200817124926-18e279725890 h1:/6pLcQq2GNdLPOotXztuLDXYRPraTIzZMPiJW8HzAwg= +github.com/dgraph-io/ristretto v0.0.4-0.20200817124926-18e279725890/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 h1:tdlZCpZ/P9DhczCTSixgIKmwPv6+wP5DGjqLYw5SUiA= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= diff --git a/table/table.go b/table/table.go index 8b55206de..5a80b69bf 100644 --- a/table/table.go +++ b/table/table.go @@ -590,23 +590,21 @@ func (t *Table) block(idx int) (*block, error) { return nil, err } } + + blk.incrRef() if t.opt.Cache != nil && t.opt.KeepBlocksInCache { key := t.blockCacheKey(idx) // incrRef should never return false here because we're calling it on a // new block with ref=1. y.AssertTrue(blk.incrRef()) - // Manish: Set is not guaranteed to actually tell you whether block went into cache or not. - // All it is telling you is that Cache has pushed it to the setBuf channel. The policy can - // still reject the block. So, perhaps change the policy to call eviction handler, if it - // rejects the block. - // Decrement the block ref if we could not insert it in the cache. if !t.opt.Cache.Set(key, blk, blk.size()) { blk.decrRef() } + // We have added an OnReject func in our cache, which gets called in case the block is not + // admitted to the cache. So, every block would be accounted for. } - blk.incrRef() return blk, nil } From b73c7a420ac910db403ce76231590389b7a5fd19 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Mon, 17 Aug 2020 06:01:45 -0700 Subject: [PATCH 21/24] Deal with when Logger is nil --- levels.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/levels.go b/levels.go index 96b468f78..ea0d03b46 100644 --- a/levels.go +++ b/levels.go @@ -702,7 +702,7 @@ nextTable: newTables = append(newTables, tbl) num := atomic.LoadInt32(&table.NumBlocks) allocs := float64(atomic.LoadInt64(&y.NumAllocs)) / float64((1 << 20)) - s.kv.opt.Logger.Debugf("Num Blocks: %d. Num Allocs (MB): %.2f\n", num, allocs) + s.kv.opt.Debugf("Num Blocks: %d. Num Allocs (MB): %.2f\n", num, allocs) mu.Unlock() }(builder) } From 396547051fd59d87d4edc9c1f832ba3aac270e90 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Mon, 17 Aug 2020 08:29:29 -0700 Subject: [PATCH 22/24] Avoid setting cache during compactions. --- db.go | 26 +++++++++++++++++++++ go.mod | 2 ++ level_handler.go | 10 +++++--- levels.go | 12 +++++----- table/iterator.go | 59 +++++++++++++++++++++++++++-------------------- table/table.go | 8 +++---- 6 files changed, 79 insertions(+), 38 deletions(-) diff --git a/db.go b/db.go index 5c004baaa..2f262d9a4 100644 --- a/db.go +++ b/db.go @@ -21,6 +21,7 @@ import ( "context" "encoding/binary" "expvar" + "fmt" "math" "os" "path/filepath" @@ -385,6 +386,28 @@ func Open(opt Options) (db *DB, err error) { headKey := y.KeyWithTs(head, math.MaxUint64) // Need to pass with timestamp, lsm get removes the last 8 bytes and compares key + { + txn := db.newTransaction(false, true) + defer txn.Discard() + iopt := DefaultIteratorOptions + iopt.AllVersions = true + iopt.InternalAccess = true + it := txn.NewKeyIterator(head, iopt) + defer it.Close() + + var vptr valuePointer + for it.Rewind(); it.Valid(); it.Next() { + item := it.Item() + err := item.Value(func(val []byte) error { + vptr.Decode(val) + fmt.Printf("========================== HEAD ===> %+v\n", vptr) + return nil + }) + if err != nil { + return db, errors.Wrap(err, "while getting head") + } + } + } vs, err := db.get(headKey) if err != nil { return db, errors.Wrap(err, "Retrieving head") @@ -393,6 +416,9 @@ func Open(opt Options) (db *DB, err error) { var vptr valuePointer if len(vs.Value) > 0 { vptr.Decode(vs.Value) + fmt.Printf("========================== GET HEAD ===> %+v\n", vptr) + } else { + fmt.Printf("=====================================> GET HEAD IS NIL\n") } replayCloser := y.NewCloser(1) diff --git a/go.mod b/go.mod index ae61b5e4e..aec02f5b5 100644 --- a/go.mod +++ b/go.mod @@ -2,6 +2,8 @@ module github.com/dgraph-io/badger/v2 go 1.12 +replace github.com/dgraph-io/ristretto => /home/mrjn/go/src/github.com/dgraph-io/ristretto + require ( github.com/DataDog/zstd v1.4.1 github.com/cespare/xxhash v1.1.0 diff --git a/level_handler.go b/level_handler.go index ce48965fe..dfae862e9 100644 --- a/level_handler.go +++ b/level_handler.go @@ -267,7 +267,7 @@ func (s *levelHandler) get(key []byte) (y.ValueStruct, error) { continue } - it := th.NewIterator(false) + it := th.NewIterator(0) defer it.Close() y.NumLSMGets.Add(s.strLevel, 1) @@ -291,6 +291,10 @@ func (s *levelHandler) appendIterators(iters []y.Iterator, opt *IteratorOptions) s.RLock() defer s.RUnlock() + var topt int + if opt.Reverse { + topt = table.REVERSED + } if s.level == 0 { // Remember to add in reverse order! // The newer table at the end of s.tables should be added first as it takes precedence. @@ -301,14 +305,14 @@ func (s *levelHandler) appendIterators(iters []y.Iterator, opt *IteratorOptions) out = append(out, t) } } - return appendIteratorsReversed(iters, out, opt.Reverse) + return appendIteratorsReversed(iters, out, topt) } tables := opt.pickTables(s.tables) if len(tables) == 0 { return iters } - return append(iters, table.NewConcatIterator(tables, opt.Reverse)) + return append(iters, table.NewConcatIterator(tables, topt)) } type levelHandlerRLocked struct{} diff --git a/levels.go b/levels.go index ea0d03b46..c13e505f4 100644 --- a/levels.go +++ b/levels.go @@ -517,10 +517,10 @@ func (s *levelsController) compactBuildTables( var iters []y.Iterator switch { case lev == 0: - iters = appendIteratorsReversed(iters, topTables, false) + iters = appendIteratorsReversed(iters, topTables, table.NOCACHE) case len(topTables) > 0: y.AssertTrue(len(topTables) == 1) - iters = []y.Iterator{topTables[0].NewIterator(false)} + iters = []y.Iterator{topTables[0].NewIterator(table.NOCACHE)} } // Next level has level>=1 and we can use ConcatIterator as key ranges do not overlap. @@ -541,7 +541,7 @@ nextTable: } valid = append(valid, table) } - iters = append(iters, table.NewConcatIterator(valid, false)) + iters = append(iters, table.NewConcatIterator(valid, table.NOCACHE)) it := table.NewMergeIterator(iters, false) defer it.Close() // Important to close the iterator to do ref counting. @@ -1110,10 +1110,10 @@ func (s *levelsController) get(key []byte, maxVs *y.ValueStruct, startLevel int) return y.ValueStruct{}, nil } -func appendIteratorsReversed(out []y.Iterator, th []*table.Table, reversed bool) []y.Iterator { +func appendIteratorsReversed(out []y.Iterator, th []*table.Table, opt int) []y.Iterator { for i := len(th) - 1; i >= 0; i-- { // This will increment the reference of the table handler. - out = append(out, th[i].NewIterator(reversed)) + out = append(out, th[i].NewIterator(opt)) } return out } @@ -1146,7 +1146,7 @@ func (s *levelsController) getTableInfo(withKeysCount bool) (result []TableInfo) for _, t := range l.tables { var count uint64 if withKeysCount { - it := t.NewIterator(false) + it := t.NewIterator(table.NOCACHE) // Only used for debugging. for it.Rewind(); it.Valid(); it.Next() { count++ } diff --git a/table/iterator.go b/table/iterator.go index d48e58138..41f9a37bd 100644 --- a/table/iterator.go +++ b/table/iterator.go @@ -167,13 +167,13 @@ type Iterator struct { // Internally, Iterator is bidirectional. However, we only expose the // unidirectional functionality for now. - reversed bool + opt int } // NewIterator returns a new iterator of the Table -func (t *Table) NewIterator(reversed bool) *Iterator { +func (t *Table) NewIterator(opt int) *Iterator { t.IncrRef() // Important. - ti := &Iterator{t: t, reversed: reversed} + ti := &Iterator{t: t, opt: opt} ti.next() return ti } @@ -194,6 +194,10 @@ func (itr *Iterator) Valid() bool { return itr.err == nil } +func (itr *Iterator) useCache() bool { + return itr.opt&NOCACHE == 0 +} + func (itr *Iterator) seekToFirst() { numBlocks := itr.t.noOfBlocks if numBlocks == 0 { @@ -201,7 +205,7 @@ func (itr *Iterator) seekToFirst() { return } itr.bpos = 0 - block, err := itr.t.block(itr.bpos) + block, err := itr.t.block(itr.bpos, itr.useCache()) if err != nil { itr.err = err return @@ -218,7 +222,7 @@ func (itr *Iterator) seekToLast() { return } itr.bpos = numBlocks - 1 - block, err := itr.t.block(itr.bpos) + block, err := itr.t.block(itr.bpos, itr.useCache()) if err != nil { itr.err = err return @@ -230,7 +234,7 @@ func (itr *Iterator) seekToLast() { func (itr *Iterator) seekHelper(blockIdx int, key []byte) { itr.bpos = blockIdx - block, err := itr.t.block(blockIdx) + block, err := itr.t.block(blockIdx, itr.useCache()) if err != nil { itr.err = err return @@ -303,7 +307,7 @@ func (itr *Iterator) next() { } if len(itr.bi.data) == 0 { - block, err := itr.t.block(itr.bpos) + block, err := itr.t.block(itr.bpos, itr.useCache()) if err != nil { itr.err = err return @@ -331,7 +335,7 @@ func (itr *Iterator) prev() { } if len(itr.bi.data) == 0 { - block, err := itr.t.block(itr.bpos) + block, err := itr.t.block(itr.bpos, itr.useCache()) if err != nil { itr.err = err return @@ -373,7 +377,7 @@ func (itr *Iterator) ValueCopy() (ret y.ValueStruct) { // Next follows the y.Iterator interface func (itr *Iterator) Next() { - if !itr.reversed { + if itr.opt&REVERSED > 0 { itr.next() } else { itr.prev() @@ -382,7 +386,7 @@ func (itr *Iterator) Next() { // Rewind follows the y.Iterator interface func (itr *Iterator) Rewind() { - if !itr.reversed { + if itr.opt&REVERSED > 0 { itr.seekToFirst() } else { itr.seekToLast() @@ -391,25 +395,30 @@ func (itr *Iterator) Rewind() { // Seek follows the y.Iterator interface func (itr *Iterator) Seek(key []byte) { - if !itr.reversed { + if itr.opt&REVERSED > 0 { itr.seek(key) } else { itr.seekForPrev(key) } } +var ( + REVERSED int = 2 + NOCACHE int = 4 +) + // ConcatIterator concatenates the sequences defined by several iterators. (It only works with // TableIterators, probably just because it's faster to not be so generic.) type ConcatIterator struct { - idx int // Which iterator is active now. - cur *Iterator - iters []*Iterator // Corresponds to tables. - tables []*Table // Disregarding reversed, this is in ascending order. - reversed bool + idx int // Which iterator is active now. + cur *Iterator + iters []*Iterator // Corresponds to tables. + tables []*Table // Disregarding reversed, this is in ascending order. + options int } // NewConcatIterator creates a new concatenated iterator -func NewConcatIterator(tbls []*Table, reversed bool) *ConcatIterator { +func NewConcatIterator(tbls []*Table, opt int) *ConcatIterator { iters := make([]*Iterator, len(tbls)) for i := 0; i < len(tbls); i++ { // Increment the reference count. Since, we're not creating the iterator right now. @@ -420,10 +429,10 @@ func NewConcatIterator(tbls []*Table, reversed bool) *ConcatIterator { // iters[i] = tbls[i].NewIterator(reversed) } return &ConcatIterator{ - reversed: reversed, - iters: iters, - tables: tbls, - idx: -1, // Not really necessary because s.it.Valid()=false, but good to have. + options: opt, + iters: iters, + tables: tbls, + idx: -1, // Not really necessary because s.it.Valid()=false, but good to have. } } @@ -434,7 +443,7 @@ func (s *ConcatIterator) setIdx(idx int) { return } if s.iters[idx] == nil { - s.iters[idx] = s.tables[idx].NewIterator(s.reversed) + s.iters[idx] = s.tables[idx].NewIterator(s.options) } s.cur = s.iters[s.idx] } @@ -444,7 +453,7 @@ func (s *ConcatIterator) Rewind() { if len(s.iters) == 0 { return } - if !s.reversed { + if s.options&REVERSED == 0 { s.setIdx(0) } else { s.setIdx(len(s.iters) - 1) @@ -470,7 +479,7 @@ func (s *ConcatIterator) Value() y.ValueStruct { // Seek brings us to element >= key if reversed is false. Otherwise, <= key. func (s *ConcatIterator) Seek(key []byte) { var idx int - if !s.reversed { + if s.options&REVERSED == 0 { idx = sort.Search(len(s.tables), func(i int) bool { return y.CompareKeys(s.tables[i].Biggest(), key) >= 0 }) @@ -498,7 +507,7 @@ func (s *ConcatIterator) Next() { return } for { // In case there are empty tables. - if !s.reversed { + if s.options&REVERSED == 0 { s.setIdx(s.idx + 1) } else { s.setIdx(s.idx - 1) diff --git a/table/table.go b/table/table.go index 5a80b69bf..8fadd5bab 100644 --- a/table/table.go +++ b/table/table.go @@ -362,7 +362,7 @@ func (t *Table) initBiggestAndSmallest() error { t.smallest = ko.Key - it2 := t.NewIterator(true) + it2 := t.NewIterator(REVERSED | NOCACHE) defer it2.Close() it2.Rewind() if !it2.Valid() { @@ -510,7 +510,7 @@ func calculateOffsetsSize(offsets []*pb.BlockOffset) int64 { // block function return a new block. Each block holds a ref and the byte // slice stored in the block will be reused when the ref becomes zero. The // caller should release the block by calling block.decrRef() on it. -func (t *Table) block(idx int) (*block, error) { +func (t *Table) block(idx int, useCache bool) (*block, error) { y.AssertTruef(idx >= 0, "idx=%d", idx) if idx >= t.noOfBlocks { return nil, errors.New("block out of index") @@ -592,7 +592,7 @@ func (t *Table) block(idx int) (*block, error) { } blk.incrRef() - if t.opt.Cache != nil && t.opt.KeepBlocksInCache { + if useCache && t.opt.Cache != nil && t.opt.KeepBlocksInCache { key := t.blockCacheKey(idx) // incrRef should never return false here because we're calling it on a // new block with ref=1. @@ -720,7 +720,7 @@ func (t *Table) readTableIndex() *pb.TableIndex { // OpenTable() function. This function is also called inside levelsController.VerifyChecksum(). func (t *Table) VerifyChecksum() error { for i, os := range t.blockOffsets() { - b, err := t.block(i) + b, err := t.block(i, true) if err != nil { return y.Wrapf(err, "checksum validation failed for table: %s, block: %d, offset:%d", t.Filename(), i, os.Offset) From f1dd358f8b6dcc9483dfcb1dc5fda6fb5bdbc9b2 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Thu, 20 Aug 2020 16:03:36 +0530 Subject: [PATCH 23/24] Fix test and cleanup --- db.go | 25 -------------------- go.mod | 2 -- table/builder_test.go | 4 ++-- table/iterator.go | 6 ++--- table/table_test.go | 54 +++++++++++++++++++++---------------------- 5 files changed, 32 insertions(+), 59 deletions(-) diff --git a/db.go b/db.go index 40bf939b5..5a1628f0c 100644 --- a/db.go +++ b/db.go @@ -21,7 +21,6 @@ import ( "context" "encoding/binary" "expvar" - "fmt" "math" "os" "path/filepath" @@ -392,28 +391,6 @@ func Open(opt Options) (db *DB, err error) { headKey := y.KeyWithTs(head, math.MaxUint64) // Need to pass with timestamp, lsm get removes the last 8 bytes and compares key - { - txn := db.newTransaction(false, true) - defer txn.Discard() - iopt := DefaultIteratorOptions - iopt.AllVersions = true - iopt.InternalAccess = true - it := txn.NewKeyIterator(head, iopt) - defer it.Close() - - var vptr valuePointer - for it.Rewind(); it.Valid(); it.Next() { - item := it.Item() - err := item.Value(func(val []byte) error { - vptr.Decode(val) - fmt.Printf("========================== HEAD ===> %+v\n", vptr) - return nil - }) - if err != nil { - return db, errors.Wrap(err, "while getting head") - } - } - } vs, err := db.get(headKey) if err != nil { return db, errors.Wrap(err, "Retrieving head") @@ -422,9 +399,7 @@ func Open(opt Options) (db *DB, err error) { var vptr valuePointer if len(vs.Value) > 0 { vptr.Decode(vs.Value) - fmt.Printf("========================== GET HEAD ===> %+v\n", vptr) } else { - fmt.Printf("=====================================> GET HEAD IS NIL\n") } replayCloser := y.NewCloser(1) diff --git a/go.mod b/go.mod index aec02f5b5..ae61b5e4e 100644 --- a/go.mod +++ b/go.mod @@ -2,8 +2,6 @@ module github.com/dgraph-io/badger/v2 go 1.12 -replace github.com/dgraph-io/ristretto => /home/mrjn/go/src/github.com/dgraph-io/ristretto - require ( github.com/DataDog/zstd v1.4.1 github.com/cespare/xxhash v1.1.0 diff --git a/table/builder_test.go b/table/builder_test.go index 6b80587d8..522e02e64 100644 --- a/table/builder_test.go +++ b/table/builder_test.go @@ -226,7 +226,7 @@ func TestBloomfilter(t *testing.T) { require.NoError(t, err) require.Equal(t, withBlooms, tab.hasBloomFilter) // Forward iteration - it := tab.NewIterator(false) + it := tab.NewIterator(0) c := 0 for it.Rewind(); it.Valid(); it.Next() { c++ @@ -236,7 +236,7 @@ func TestBloomfilter(t *testing.T) { require.Equal(t, keyCount, c) // Backward iteration - it = tab.NewIterator(true) + it = tab.NewIterator(REVERSED) c = 0 for it.Rewind(); it.Valid(); it.Next() { c++ diff --git a/table/iterator.go b/table/iterator.go index 41f9a37bd..c7bf3a258 100644 --- a/table/iterator.go +++ b/table/iterator.go @@ -377,7 +377,7 @@ func (itr *Iterator) ValueCopy() (ret y.ValueStruct) { // Next follows the y.Iterator interface func (itr *Iterator) Next() { - if itr.opt&REVERSED > 0 { + if itr.opt&REVERSED == 0 { itr.next() } else { itr.prev() @@ -386,7 +386,7 @@ func (itr *Iterator) Next() { // Rewind follows the y.Iterator interface func (itr *Iterator) Rewind() { - if itr.opt&REVERSED > 0 { + if itr.opt&REVERSED == 0 { itr.seekToFirst() } else { itr.seekToLast() @@ -395,7 +395,7 @@ func (itr *Iterator) Rewind() { // Seek follows the y.Iterator interface func (itr *Iterator) Seek(key []byte) { - if itr.opt&REVERSED > 0 { + if itr.opt&REVERSED == 0 { itr.seek(key) } else { itr.seekForPrev(key) diff --git a/table/table_test.go b/table/table_test.go index 4b93332fd..99d3f1e43 100644 --- a/table/table_test.go +++ b/table/table_test.go @@ -97,7 +97,7 @@ func TestTableIterator(t *testing.T) { table, err := OpenTable(f, opts) require.NoError(t, err) defer table.DecrRef() - it := table.NewIterator(false) + it := table.NewIterator(0) defer it.Close() count := 0 for it.Rewind(); it.Valid(); it.Next() { @@ -120,7 +120,7 @@ func TestSeekToFirst(t *testing.T) { table, err := OpenTable(f, opts) require.NoError(t, err) defer table.DecrRef() - it := table.NewIterator(false) + it := table.NewIterator(0) defer it.Close() it.seekToFirst() require.True(t, it.Valid()) @@ -139,7 +139,7 @@ func TestSeekToLast(t *testing.T) { table, err := OpenTable(f, opts) require.NoError(t, err) defer table.DecrRef() - it := table.NewIterator(false) + it := table.NewIterator(0) defer it.Close() it.seekToLast() require.True(t, it.Valid()) @@ -162,7 +162,7 @@ func TestSeek(t *testing.T) { require.NoError(t, err) defer table.DecrRef() - it := table.NewIterator(false) + it := table.NewIterator(0) defer it.Close() var data = []struct { @@ -198,7 +198,7 @@ func TestSeekForPrev(t *testing.T) { require.NoError(t, err) defer table.DecrRef() - it := table.NewIterator(false) + it := table.NewIterator(0) defer it.Close() var data = []struct { @@ -236,7 +236,7 @@ func TestIterateFromStart(t *testing.T) { table, err := OpenTable(f, opts) require.NoError(t, err) defer table.DecrRef() - ti := table.NewIterator(false) + ti := table.NewIterator(0) defer ti.Close() ti.reset() ti.seekToFirst() @@ -264,7 +264,7 @@ func TestIterateFromEnd(t *testing.T) { table, err := OpenTable(f, opts) require.NoError(t, err) defer table.DecrRef() - ti := table.NewIterator(false) + ti := table.NewIterator(0) defer ti.Close() ti.reset() ti.seek(y.KeyWithTs([]byte("zzzzzz"), 0)) // Seek to end, an invalid element. @@ -289,7 +289,7 @@ func TestTable(t *testing.T) { table, err := OpenTable(f, opts) require.NoError(t, err) defer table.DecrRef() - ti := table.NewIterator(false) + ti := table.NewIterator(0) defer ti.Close() kid := 1010 seek := y.KeyWithTs([]byte(key("key", kid)), 0) @@ -319,7 +319,7 @@ func TestIterateBackAndForth(t *testing.T) { defer table.DecrRef() seek := y.KeyWithTs([]byte(key("key", 1010)), 0) - it := table.NewIterator(false) + it := table.NewIterator(0) defer it.Close() it.seek(seek) require.True(t, it.Valid()) @@ -360,7 +360,7 @@ func TestUniIterator(t *testing.T) { require.NoError(t, err) defer table.DecrRef() { - it := table.NewIterator(false) + it := table.NewIterator(0) defer it.Close() var count int for it.Rewind(); it.Valid(); it.Next() { @@ -372,7 +372,7 @@ func TestUniIterator(t *testing.T) { require.EqualValues(t, 10000, count) } { - it := table.NewIterator(true) + it := table.NewIterator(REVERSED) defer it.Close() var count int for it.Rewind(); it.Valid(); it.Next() { @@ -397,7 +397,7 @@ func TestConcatIteratorOneTable(t *testing.T) { require.NoError(t, err) defer tbl.DecrRef() - it := NewConcatIterator([]*Table{tbl}, false) + it := NewConcatIterator([]*Table{tbl}, 0) defer it.Close() it.Rewind() @@ -425,7 +425,7 @@ func TestConcatIterator(t *testing.T) { defer tbl3.DecrRef() { - it := NewConcatIterator([]*Table{tbl, tbl2, tbl3}, false) + it := NewConcatIterator([]*Table{tbl, tbl2, tbl3}, 0) defer it.Close() it.Rewind() require.True(t, it.Valid()) @@ -457,7 +457,7 @@ func TestConcatIterator(t *testing.T) { require.False(t, it.Valid()) } { - it := NewConcatIterator([]*Table{tbl, tbl2, tbl3}, true) + it := NewConcatIterator([]*Table{tbl, tbl2, tbl3}, REVERSED) defer it.Close() it.Rewind() require.True(t, it.Valid()) @@ -519,8 +519,8 @@ func TestMergingIterator(t *testing.T) { tbl2, err := OpenTable(f2, opts) require.NoError(t, err) defer tbl2.DecrRef() - it1 := tbl1.NewIterator(false) - it2 := NewConcatIterator([]*Table{tbl2}, false) + it1 := tbl1.NewIterator(0) + it2 := NewConcatIterator([]*Table{tbl2}, 0) it := NewMergeIterator([]y.Iterator{it1, it2}, false) defer it.Close() @@ -568,8 +568,8 @@ func TestMergingIteratorReversed(t *testing.T) { tbl2, err := OpenTable(f2, opts) require.NoError(t, err) defer tbl2.DecrRef() - it1 := tbl1.NewIterator(true) - it2 := NewConcatIterator([]*Table{tbl2}, true) + it1 := tbl1.NewIterator(REVERSED) + it2 := NewConcatIterator([]*Table{tbl2}, REVERSED) it := NewMergeIterator([]y.Iterator{it1, it2}, true) defer it.Close() @@ -603,8 +603,8 @@ func TestMergingIteratorTakeOne(t *testing.T) { require.NoError(t, err) defer t2.DecrRef() - it1 := NewConcatIterator([]*Table{t1}, false) - it2 := NewConcatIterator([]*Table{t2}, false) + it1 := NewConcatIterator([]*Table{t1}, 0) + it2 := NewConcatIterator([]*Table{t2}, 0) it := NewMergeIterator([]y.Iterator{it1, it2}, false) defer it.Close() @@ -651,8 +651,8 @@ func TestMergingIteratorTakeTwo(t *testing.T) { require.NoError(t, err) defer t2.DecrRef() - it1 := NewConcatIterator([]*Table{t1}, false) - it2 := NewConcatIterator([]*Table{t2}, false) + it1 := NewConcatIterator([]*Table{t1}, 0) + it2 := NewConcatIterator([]*Table{t2}, 0) it := NewMergeIterator([]y.Iterator{it1, it2}, false) defer it.Close() @@ -710,7 +710,7 @@ func TestTableBigValues(t *testing.T) { require.NoError(t, err, "unable to open table") defer tbl.DecrRef() - itr := tbl.NewIterator(false) + itr := tbl.NewIterator(0) require.True(t, itr.Valid()) count := 0 @@ -761,7 +761,7 @@ func BenchmarkRead(b *testing.B) { // Iterate b.N times over the entire table. for i := 0; i < b.N; i++ { func() { - it := tbl.NewIterator(false) + it := tbl.NewIterator(0) defer it.Close() for it.seekToFirst(); it.Valid(); it.next() { } @@ -783,7 +783,7 @@ func BenchmarkReadAndBuild(b *testing.B) { opts := Options{Compression: options.ZSTD, BlockSize: 4 * 0124, BloomFalsePositive: 0.01} opts.Cache = cache newBuilder := NewTableBuilder(opts) - it := tbl.NewIterator(false) + it := tbl.NewIterator(0) defer it.Close() for it.seekToFirst(); it.Valid(); it.next() { vs := it.Value() @@ -832,7 +832,7 @@ func BenchmarkReadMerged(b *testing.B) { func() { var iters []y.Iterator for _, tbl := range tables { - iters = append(iters, tbl.NewIterator(false)) + iters = append(iters, tbl.NewIterator(0)) } it := NewMergeIterator(iters, false) defer it.Close() @@ -873,7 +873,7 @@ func BenchmarkRandomRead(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - itr := tbl.NewIterator(false) + itr := tbl.NewIterator(0) no := r.Intn(n) k := []byte(fmt.Sprintf("%016x", no)) v := []byte(fmt.Sprintf("%d", no)) From 5314fed6192139ddeca430a05be3fa59637b20a0 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Thu, 20 Aug 2020 16:45:57 +0530 Subject: [PATCH 24/24] cleanup --- db.go | 3 +-- levels.go | 2 +- table/iterator.go | 4 ++-- y/y.go | 14 -------------- 4 files changed, 4 insertions(+), 19 deletions(-) diff --git a/db.go b/db.go index 5a1628f0c..710eb8527 100644 --- a/db.go +++ b/db.go @@ -194,7 +194,7 @@ func (db *DB) replayFunction() func(Entry, valuePointer) error { func Open(opt Options) (db *DB, err error) { // It's okay to have zero compactors which will disable all compactions but // we cannot have just one compactor otherwise we will end up with all data - // one level 2. + // on level 2. if opt.NumCompactors == 1 { return nil, errors.New("Cannot have 1 compactor. Need at least 2") } @@ -399,7 +399,6 @@ func Open(opt Options) (db *DB, err error) { var vptr valuePointer if len(vs.Value) > 0 { vptr.Decode(vs.Value) - } else { } replayCloser := y.NewCloser(1) diff --git a/levels.go b/levels.go index c896da7b4..5de4dd2b5 100644 --- a/levels.go +++ b/levels.go @@ -1146,7 +1146,7 @@ func (s *levelsController) getTableInfo(withKeysCount bool) (result []TableInfo) for _, t := range l.tables { var count uint64 if withKeysCount { - it := t.NewIterator(table.NOCACHE) // Only used for debugging. + it := t.NewIterator(table.NOCACHE) for it.Rewind(); it.Valid(); it.Next() { count++ } diff --git a/table/iterator.go b/table/iterator.go index c7bf3a258..815449983 100644 --- a/table/iterator.go +++ b/table/iterator.go @@ -167,7 +167,7 @@ type Iterator struct { // Internally, Iterator is bidirectional. However, we only expose the // unidirectional functionality for now. - opt int + opt int // Valid options are REVERSED and NOCACHE. } // NewIterator returns a new iterator of the Table @@ -414,7 +414,7 @@ type ConcatIterator struct { cur *Iterator iters []*Iterator // Corresponds to tables. tables []*Table // Disregarding reversed, this is in ascending order. - options int + options int // Valid options are REVERSED and NOCACHE. } // NewConcatIterator creates a new concatenated iterator diff --git a/y/y.go b/y/y.go index 1b47defa4..93b47c7f7 100644 --- a/y/y.go +++ b/y/y.go @@ -167,7 +167,6 @@ func SameKey(src, dst []byte) bool { // One problem is with n distinct sizes in random order it'll reallocate log(n) times. type Slice struct { buf []byte - // viaC bool } // Resize reuses the Slice's buffer (or makes a new one) and returns a slice in that buffer of @@ -179,19 +178,6 @@ func (s *Slice) Resize(sz int) []byte { return s.buf[0:sz] } -// func (s *Slice) Append(dst []byte) []byte { -// if !s.viaC || cap(s.buf)-len(s.buf) >= len(dst) { -// s.buf = append(s.buf, dst) -// } else { -// buf := make([]byte, len(dst)+len(s.buf)) -// copy(buf, s.buf) -// copy(buf[len(s.buf):], dst) -// manual.Free(s.buf) -// s.buf = buf -// s.viaC = false -// } -// } - // FixedDuration returns a string representation of the given duration with the // hours, minutes, and seconds. func FixedDuration(d time.Duration) string {