From 8e4085b13cda2f3ab500ae6c2492d54efef7dc3a Mon Sep 17 00:00:00 2001 From: Ajeet D'Souza <98ajeet@gmail.com> Date: Tue, 10 Nov 2020 14:06:00 +0530 Subject: [PATCH 01/35] fix: Disable CompactL0OnClose by default (#1586) --- db_test.go | 3 ++- merge_test.go | 3 ++- options.go | 7 +++---- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/db_test.go b/db_test.go index 53aeb3423..66caff898 100644 --- a/db_test.go +++ b/db_test.go @@ -342,7 +342,8 @@ func TestForceCompactL0(t *testing.T) { require.NoError(t, err) defer removeDir(dir) - opts := getTestOptions(dir) + // This test relies on CompactL0OnClose + opts := getTestOptions(dir).WithCompactL0OnClose(true) opts.ValueLogFileSize = 15 << 20 opts.managedTxns = true db, err := Open(opts) diff --git a/merge_test.go b/merge_test.go index 23e028081..38c93c065 100644 --- a/merge_test.go +++ b/merge_test.go @@ -115,7 +115,8 @@ func TestGetMergeOperator(t *testing.T) { require.NoError(t, err) defer removeDir(dir) - opts := getTestOptions(dir) + // This test relies on CompactL0OnClose + opts := getTestOptions(dir).WithCompactL0OnClose(true) db, err := Open(opts) require.NoError(t, err) mergeKey := []byte("foo") diff --git a/options.go b/options.go index 45bd5e8ff..33e2c4865 100644 --- a/options.go +++ b/options.go @@ -130,7 +130,7 @@ func DefaultOptions(path string) Options { BlockSize: 4 * 1024, SyncWrites: false, NumVersionsToKeep: 1, - CompactL0OnClose: true, + CompactL0OnClose: false, VerifyValueChecksum: false, Compression: options.Snappy, BlockCacheSize: 256 << 20, @@ -425,10 +425,9 @@ func (opt Options) WithNumCompactors(val int) Options { } // WithCompactL0OnClose determines whether Level 0 should be compacted before closing the DB. This -// ensures that both reads and writes are efficient when the DB is opened later. CompactL0OnClose -// is set to true if KeepL0InMemory is set to true. +// ensures that both reads and writes are efficient when the DB is opened later. // -// The default value of CompactL0OnClose is true. +// The default value of CompactL0OnClose is false. func (opt Options) WithCompactL0OnClose(val bool) Options { opt.CompactL0OnClose = val return opt From 50f07a64b0545ff603391b66864491466be30527 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Thu, 12 Nov 2020 00:00:12 +0530 Subject: [PATCH 02/35] Add smallstep/certificates to projects using badger (#1589) --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 76bc17fc8..0a0b88e8d 100644 --- a/README.md +++ b/README.md @@ -162,6 +162,7 @@ Below is a list of known projects that use Badger: * [emitter](https://github.com/emitter-io/emitter) - Scalable, low latency, distributed pub/sub broker with message storage, uses MQTT, gossip and badger. * [OctoSQL](https://github.com/cube2222/octosql) - Query tool that allows you to join, analyse and transform data from multiple databases using SQL. * [Dkron](https://dkron.io/) - Distributed, fault tolerant job scheduling system. +* [smallstep/certificates](https://github.com/smallstep/certificates) - Step-ca is an online certificate authority for secure, automated certificate management. * [Sandglass](https://github.com/celrenheit/sandglass) - distributed, horizontally scalable, persistent, time sorted message queue. * [TalariaDB](https://github.com/grab/talaria) - Grab's Distributed, low latency time-series database. * [Sloop](https://github.com/salesforce/sloop) - Salesforce's Kubernetes History Visualization Project. From b94b56159c8bbb047d3e435e6c9efabeab7f75ac Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 11 Nov 2020 13:02:30 -0800 Subject: [PATCH 03/35] Use AllocatorPool in Stream and TableBuilder (#1593) We removed the global z.Allocator pool from z package. Instead, we now use a new z.AllocatorPool class in the places which need a pool. In this case, we brought it to TableBuilder and Stream. Fix up a memory leak in Stream. Co-authored-by: Ibrahim Jarif --- badger/main.go | 1 - db.go | 4 ++++ db_test.go | 2 +- go.mod | 2 +- go.sum | 4 ++-- options.go | 1 + stream.go | 58 ++++++++++++++++++++++++++++++------------------ stream_test.go | 52 +++++++++++++++++++++++++++++++++++++++++++ stream_writer.go | 5 +++-- table/builder.go | 37 +++++++++++++++--------------- table/table.go | 6 +++-- test.sh | 48 ++++++++++++++++++++++++++------------- 12 files changed, 157 insertions(+), 63 deletions(-) diff --git a/badger/main.go b/badger/main.go index ed76978fa..c284f5aec 100644 --- a/badger/main.go +++ b/badger/main.go @@ -49,7 +49,6 @@ func main() { z.Free(out) cmd.Execute() - z.Done() fmt.Printf("Num Allocated Bytes at program end: %s\n", humanize.IBytes(uint64(z.NumAllocBytes()))) if z.NumAllocBytes() > 0 { diff --git a/db.go b/db.go index 249e14c77..079efcded 100644 --- a/db.go +++ b/db.go @@ -95,6 +95,7 @@ type DB struct { registry *KeyRegistry blockCache *ristretto.Cache indexCache *ristretto.Cache + allocPool *z.AllocatorPool } const ( @@ -218,6 +219,7 @@ func Open(opt Options) (*DB, error) { valueDirGuard: valueDirLockGuard, orc: newOracle(opt), pub: newPublisher(), + allocPool: z.NewAllocatorPool(8), } // Cleanup all the goroutines started by badger in case of an error. defer func() { @@ -476,6 +478,8 @@ func (db *DB) IsClosed() bool { } func (db *DB) close() (err error) { + defer db.allocPool.Release() + db.opt.Debugf("Closing database") db.opt.Infof("Lifetime L0 stalled for: %s\n", time.Duration(db.lc.l0stallsMs)) diff --git a/db_test.go b/db_test.go index 66caff898..b1bec0774 100644 --- a/db_test.go +++ b/db_test.go @@ -533,7 +533,7 @@ func TestGetMore(t *testing.T) { data := func(i int) []byte { return []byte(fmt.Sprintf("%b", i)) } - n := 500000 + n := 200000 m := 45 // Increasing would cause ErrTxnTooBig for i := 0; i < n; i += m { if (i % 10000) == 0 { diff --git a/go.mod b/go.mod index bdb784454..e8b5c83bf 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,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.4-0.20201105000107-750f5be31aad + github.com/dgraph-io/ristretto v0.0.4-0.20201111190620-1040b7ded521 github.com/dustin/go-humanize v1.0.0 github.com/golang/protobuf v1.3.1 github.com/golang/snappy v0.0.1 diff --git a/go.sum b/go.sum index 657325fbb..f02228044 100644 --- a/go.sum +++ b/go.sum @@ -15,8 +15,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.4-0.20201105000107-750f5be31aad h1:BN2A+lqBwLx57/l1MUv3iKLIY8XddNkZg2zZN/BFM1I= -github.com/dgraph-io/ristretto v0.0.4-0.20201105000107-750f5be31aad/go.mod h1:bDI4cDaalvYSji3vBVDKrn9ouDZrwN974u8ZO/AhYXs= +github.com/dgraph-io/ristretto v0.0.4-0.20201111190620-1040b7ded521 h1:81yY9QfuVfSVccqD7cLA/JohhMX+TQCfSjmGS7jUJCc= +github.com/dgraph-io/ristretto v0.0.4-0.20201111190620-1040b7ded521/go.mod h1:bDI4cDaalvYSji3vBVDKrn9ouDZrwN974u8ZO/AhYXs= 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/options.go b/options.go index 33e2c4865..ede01d2e4 100644 --- a/options.go +++ b/options.go @@ -177,6 +177,7 @@ func buildTableOptions(db *DB) table.Options { ZSTDCompressionLevel: opt.ZSTDCompressionLevel, BlockCache: db.blockCache, IndexCache: db.indexCache, + AllocPool: db.allocPool, DataKey: dk, } } diff --git a/stream.go b/stream.go index d3544662a..972226e4c 100644 --- a/stream.go +++ b/stream.go @@ -92,6 +92,8 @@ type Stream struct { // Use allocators to generate KVs. allocatorsMu sync.RWMutex allocators map[int]*z.Allocator + + allocPool *z.AllocatorPool } func (st *Stream) Allocator(threadId int) *z.Allocator { @@ -182,15 +184,9 @@ func (st *Stream) produceRanges(ctx context.Context) { func (st *Stream) newAllocator(threadId int) *z.Allocator { st.allocatorsMu.Lock() - var a *z.Allocator - if cur, ok := st.allocators[threadId]; ok && cur.Size() == 0 { - a = cur // Reuse. - } else { - // Current allocator has been used already. Create a new one. - a = z.NewAllocator(batchSize) - // a.Tag = fmt.Sprintf("Stream %d: %s", threadId, st.LogPrefix) - st.allocators[threadId] = a - } + a := st.allocPool.Get(batchSize) + a.Tag = "Stream " + st.LogPrefix + st.allocators[threadId] = a st.allocatorsMu.Unlock() return a } @@ -206,6 +202,12 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { } defer txn.Discard() + // produceKVs is running iterate serially. So, we can define the outList here. + outList := new(pb.KVList) + // There would be one last remaining Allocator for this threadId when produceKVs finishes, which + // would be released by Orchestrate. + outList.AllocRef = st.newAllocator(threadId).Ref + iterate := func(kr keyRange) error { iterOpts := DefaultIteratorOptions iterOpts.AllVersions = true @@ -218,9 +220,6 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { // This unique stream id is used to identify all the keys from this iteration. streamId := atomic.AddUint32(&st.nextStreamId, 1) - outList := new(pb.KVList) - outList.AllocRef = st.newAllocator(threadId).Ref - sendIt := func() error { select { case st.kvChan <- outList: @@ -280,10 +279,7 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { StreamDone: true, }) } - if len(outList.Kv) > 0 { - return sendIt() - } - return nil + return sendIt() } for { @@ -309,6 +305,16 @@ func (st *Stream) streamKVs(ctx context.Context) error { defer t.Stop() now := time.Now() + allocs := make(map[uint64]struct{}) + returnAllocs := func() { + for ref := range allocs { + a := z.AllocatorFrom(ref) + st.allocPool.Return(a) + delete(allocs, ref) + } + } + defer returnAllocs() + sendBatch := func(batch *pb.KVList) error { sz := uint64(proto.Size(batch)) bytesSent += sz @@ -320,6 +326,7 @@ func (st *Stream) streamKVs(ctx context.Context) error { st.db.opt.Infof("%s Created batch of size: %s in %s.\n", st.LogPrefix, humanize.Bytes(sz), time.Since(t)) + returnAllocs() return nil } @@ -340,6 +347,7 @@ func (st *Stream) streamKVs(ctx context.Context) error { } y.AssertTrue(kvs != nil) batch.Kv = append(batch.Kv, kvs.Kv...) + allocs[kvs.AllocRef] = struct{}{} default: break loop @@ -372,6 +380,7 @@ outer: } y.AssertTrue(kvs != nil) batch = kvs + allocs[kvs.AllocRef] = struct{}{} // Otherwise, slurp more keys into this batch. if err := slurp(batch); err != nil { @@ -391,6 +400,17 @@ outer: // are serial. In case any of these steps encounter an error, Orchestrate would stop execution and // return that error. Orchestrate can be called multiple times, but in serial order. func (st *Stream) Orchestrate(ctx context.Context) error { + st.allocPool = z.NewAllocatorPool(st.NumGo) + defer func() { + for _, a := range st.allocators { + // Using AllocatorFrom is better because if the allocator is already freed up, it would + // return nil. + a = z.AllocatorFrom(a.Ref) + a.Release() + } + st.allocPool.Release() + }() + st.rangeCh = make(chan keyRange, 3) // Contains keys for posting lists. // kvChan should only have a small capacity to ensure that we don't buffer up too much data if @@ -439,17 +459,13 @@ func (st *Stream) Orchestrate(ctx context.Context) error { // Wait for key streaming to be over. err := <-kvErr - - for _, a := range st.allocators { - a.Release() - } return err } func (db *DB) newStream() *Stream { return &Stream{ db: db, - NumGo: 16, + NumGo: 8, LogPrefix: "Badger.Stream", allocators: make(map[int]*z.Allocator), } diff --git a/stream_test.go b/stream_test.go index d6bd5e568..536966008 100644 --- a/stream_test.go +++ b/stream_test.go @@ -25,6 +25,7 @@ import ( "strings" "testing" + "github.com/dgraph-io/badger/v2/pb" bpb "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" "github.com/golang/protobuf/proto" @@ -265,3 +266,54 @@ func TestBigStream(t *testing.T) { } require.NoError(t, db.Close()) } + +// There was a bug in the stream writer code which would cause allocators to be +// freed up twice if the default keyToList was not used. This test verifies that issue. +func TestStreamCustomKeyToList(t *testing.T) { + dir, err := ioutil.TempDir("", "badger-test") + require.NoError(t, err) + defer removeDir(dir) + + db, err := OpenManaged(DefaultOptions(dir)) + require.NoError(t, err) + + var count int + for _, key := range []string{"p0", "p1", "p2"} { + for i := 1; i <= 100; i++ { + txn := db.NewTransactionAt(math.MaxUint64, true) + require.NoError(t, txn.SetEntry(NewEntry([]byte(key), value(i)))) + count++ + require.NoError(t, txn.CommitAt(uint64(i), nil)) + } + } + + stream := db.NewStreamAt(math.MaxUint64) + stream.LogPrefix = "Testing" + stream.KeyToList = func(key []byte, itr *Iterator) (*pb.KVList, error) { + item := itr.Item() + val, err := item.ValueCopy(nil) + if err != nil { + return nil, err + } + kv := &pb.KV{ + Key: y.Copy(item.Key()), + Value: val, + } + return &pb.KVList{ + Kv: []*pb.KV{kv}, + }, nil + } + res := map[string]struct{}{"p0": {}, "p1": {}, "p2": {}} + stream.Send = func(list *pb.KVList) error { + for _, kv := range list.Kv { + key := string(kv.Key) + if _, ok := res[key]; !ok { + panic(fmt.Sprintf("%s key not found", key)) + } + delete(res, key) + } + return nil + } + require.NoError(t, stream.Orchestrate(ctxb)) + require.Zero(t, len(res)) +} diff --git a/stream_writer.go b/stream_writer.go index af1335920..1b1ef17f8 100644 --- a/stream_writer.go +++ b/stream_writer.go @@ -270,6 +270,7 @@ func (sw *StreamWriter) Cancel() { type sortedWriter struct { db *DB throttle *y.Throttle + opts table.Options builder *table.Builder lastKey []byte @@ -286,6 +287,7 @@ func (sw *StreamWriter) newWriter(streamID uint32) (*sortedWriter, error) { } w := &sortedWriter{ db: sw.db, + opts: bopts, streamID: streamID, throttle: sw.throttle, builder: table.NewTableBuilder(bopts), @@ -379,8 +381,7 @@ func (w *sortedWriter) send(done bool) error { return nil } - bopts := buildTableOptions(w.db) - w.builder = table.NewTableBuilder(bopts) + w.builder = table.NewTableBuilder(w.opts) return nil } diff --git a/table/builder.go b/table/builder.go index 925821a78..5f400385f 100644 --- a/table/builder.go +++ b/table/builder.go @@ -86,7 +86,7 @@ type Builder struct { lenOffsets uint32 estimatedSize uint32 keyHashes []uint32 // Used for building the bloomfilter. - opt *Options + opts *Options maxVersion uint64 onDiskSize uint32 @@ -127,17 +127,18 @@ func NewTableBuilder(opts Options) *Builder { sz = maxAllocatorInitialSz } b := &Builder{ - alloc: z.GetAllocatorFromPool(sz), - opt: &opts, + alloc: opts.AllocPool.Get(sz), + opts: &opts, } + b.alloc.Tag = "Builder" b.curBlock = &bblock{ data: b.alloc.Allocate(opts.BlockSize + padding), } - b.opt.tableCapacity = uint64(float64(b.opt.TableSize) * 0.9) + b.opts.tableCapacity = uint64(float64(b.opts.TableSize) * 0.95) // 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 { + if b.opts.Compression == options.None && b.opts.DataKey == nil { return b } @@ -154,7 +155,7 @@ func NewTableBuilder(opts Options) *Builder { func (b *Builder) handleBlock() { defer b.wg.Done() - doCompress := b.opt.Compression != options.None + doCompress := b.opts.Compression != options.None for item := range b.blockChan { // Extract the block. blockBuf := item.data[:item.end] @@ -185,7 +186,7 @@ func (b *Builder) handleBlock() { // Close closes the TableBuilder. func (b *Builder) Close() { - z.ReturnAllocator(b.alloc) + b.opts.AllocPool.Return(b.alloc) } // Empty returns whether it's empty. @@ -313,7 +314,7 @@ func (b *Builder) shouldFinishBlock(key []byte, value y.ValueStruct) bool { // Integer overflow check for table size. y.AssertTrue(uint64(b.curBlock.end)+uint64(estimatedSize) < math.MaxUint32) - return estimatedSize > uint32(b.opt.BlockSize) + return estimatedSize > uint32(b.opts.BlockSize) } // Add adds a key-value pair to the block. @@ -322,7 +323,7 @@ func (b *Builder) Add(key []byte, value y.ValueStruct, valueLen uint32) { b.finishBlock() // Create a new block and start writing. b.curBlock = &bblock{ - data: b.alloc.Allocate(b.opt.BlockSize + padding), + data: b.alloc.Allocate(b.opts.BlockSize + padding), } } b.addHelper(key, value, valueLen) @@ -338,7 +339,7 @@ func (b *Builder) Add(key []byte, value y.ValueStruct, valueLen uint32) { func (b *Builder) ReachedCapacity() bool { // If encryption/compression is enabled then use the compresssed size. sumBlockSizes := atomic.LoadUint32(&b.compressedSize) - if b.opt.Compression == options.None && b.opt.DataKey == nil { + if b.opts.Compression == options.None && b.opts.DataKey == nil { sumBlockSizes = b.uncompressedSize } blocksSize := sumBlockSizes + // actual length of current buffer @@ -351,7 +352,7 @@ func (b *Builder) ReachedCapacity() bool { 4 + // Index length b.lenOffsets - return uint64(estimateSz) > b.opt.tableCapacity + return uint64(estimateSz) > b.opts.tableCapacity } // Finish finishes the table by appending the index. @@ -412,8 +413,8 @@ func (b *Builder) Done() buildData { } var f y.Filter - if b.opt.BloomFalsePositive > 0 { - bits := y.BloomBitsPerKey(len(b.keyHashes), b.opt.BloomFalsePositive) + if b.opts.BloomFalsePositive > 0 { + bits := y.BloomBitsPerKey(len(b.keyHashes), b.opts.BloomFalsePositive) f = y.NewFilter(b.keyHashes, bits) } index, dataSize := b.buildIndex(f) @@ -455,11 +456,11 @@ func (b *Builder) calculateChecksum(data []byte) []byte { // DataKey returns datakey of the builder. func (b *Builder) DataKey() *pb.DataKey { - return b.opt.DataKey + return b.opts.DataKey } func (b *Builder) Opts() *Options { - return b.opt + return b.opts } // encrypt will encrypt the given data and appends IV to the end of the encrypted data. @@ -483,12 +484,12 @@ func (b *Builder) encrypt(data []byte) ([]byte, error) { // shouldEncrypt tells us whether to encrypt the data or not. // We encrypt only if the data key exist. Otherwise, not. func (b *Builder) shouldEncrypt() bool { - return b.opt.DataKey != nil + return b.opts.DataKey != nil } // compressData compresses the given data. func (b *Builder) compressData(data []byte) ([]byte, error) { - switch b.opt.Compression { + switch b.opts.Compression { case options.None: return data, nil case options.Snappy: @@ -498,7 +499,7 @@ func (b *Builder) compressData(data []byte) ([]byte, error) { case options.ZSTD: sz := y.ZSTDCompressBound(len(data)) dst := b.alloc.Allocate(sz) - return y.ZSTDCompress(dst, data, b.opt.ZSTDCompressionLevel) + return y.ZSTDCompress(dst, data, b.opts.ZSTDCompressionLevel) } return nil, errors.New("Unsupported compression type") } diff --git a/table/table.go b/table/table.go index a28bd33c1..0ff60c22b 100644 --- a/table/table.go +++ b/table/table.go @@ -80,6 +80,8 @@ type Options struct { BlockCache *ristretto.Cache IndexCache *ristretto.Cache + AllocPool *z.AllocatorPool + // ZSTDCompressionLevel is the ZSTD compression level used for compressing blocks. ZSTDCompressionLevel int } @@ -241,12 +243,12 @@ func CreateTable(fname string, builder *Builder) (*Table, error) { written := bd.Copy(mf.Data) y.AssertTrue(written == len(mf.Data)) - if builder.opt.SyncWrites { + if builder.opts.SyncWrites { if err := z.Msync(mf.Data); err != nil { return nil, y.Wrapf(err, "while calling msync on %s", fname) } } - return OpenTable(mf, *builder.opt) + return OpenTable(mf, *builder.opts) } // OpenTable assumes file has only one table and opens it. Takes ownership of fd upon function diff --git a/test.sh b/test.sh index 9077b7e96..9b4b94698 100755 --- a/test.sh +++ b/test.sh @@ -36,17 +36,19 @@ function InstallJemalloc() { popd } +tags="-tags=jemalloc" + # Ensure that we can compile the binary. pushd badger -go build -v . +go build -v $tags . popd -tags="-tags=jemalloc" # tags="" InstallJemalloc # Run the memory intensive tests first. manual() { + timeout="-timeout 2m" echo "==> Running package tests for $packages" set -e for pkg in $packages; do @@ -59,10 +61,10 @@ manual() { # Run the special Truncate test. rm -rf p set -e - go test $tags -run='TestTruncateVlogNoClose$' --manual=true + go test $tags $timeout -run='TestTruncateVlogNoClose$' --manual=true truncate --size=4096 p/000000.vlog - go test $tags -run='TestTruncateVlogNoClose2$' --manual=true - go test $tags -run='TestTruncateVlogNoClose3$' --manual=true + go test $tags $timeout -run='TestTruncateVlogNoClose2$' --manual=true + go test $tags $timeout -run='TestTruncateVlogNoClose3$' --manual=true rm -rf p # TODO(ibrahim): Let's make these tests have Manual prefix. @@ -71,14 +73,14 @@ manual() { # TestValueGCManaged # TestDropPrefix # TestDropAllManaged - go test $tags -run='TestBigKeyValuePairs$' --manual=true - go test $tags -run='TestPushValueLogLimit' --manual=true - go test $tags -run='TestKeyCount' --manual=true - go test $tags -run='TestIteratePrefix' --manual=true - go test $tags -run='TestIterateParallel' --manual=true - go test $tags -run='TestBigStream' --manual=true - go test $tags -run='TestGoroutineLeak' --manual=true - go test $tags -run='TestGetMore' --manual=true + go test $tags $timeout -run='TestBigKeyValuePairs$' --manual=true + go test $tags $timeout -run='TestPushValueLogLimit' --manual=true + go test $tags $timeout -run='TestKeyCount' --manual=true + go test $tags $timeout -run='TestIteratePrefix' --manual=true + go test $tags $timeout -run='TestIterateParallel' --manual=true + go test $tags $timeout -run='TestBigStream' --manual=true + go test $tags $timeout -run='TestGoroutineLeak' --manual=true + go test $tags $timeout -run='TestGetMore' --manual=true echo "==> DONE manual tests" } @@ -89,11 +91,27 @@ root() { echo "==> Running root level tests." set -e - go test $tags -timeout=25m . -race -parallel 16 + go test $tags -timeout=25m . -v -race -parallel 16 echo "==> DONE root level tests" } +stream() { + pushd badger + baseDir=$(mktemp -d -p .) + ./badger benchmark write -s --dir=$baseDir/test | tee $baseDir/log.txt + ./badger stream --dir=$baseDir/test -o "$baseDir/test2" | tee --append $baseDir/log.txt + count=$(cat "$baseDir/log.txt" | grep "at program end: 0 B" | wc -l) + rm -rf $baseDir + if [ $count -ne 2 ]; then + echo "LEAK detected in Badger stream." + return 1 + fi + echo "==> DONE stream test" + return 0 +} + +export -f stream export -f manual export -f root -parallel --halt now,fail=1 --progress --line-buffer ::: manual root +parallel --halt now,fail=1 --progress --line-buffer ::: stream manual root From e002a9d4a500e4661b3afd01c7a33da97e896a8a Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 11 Nov 2020 16:04:50 -0800 Subject: [PATCH 04/35] Bring in Ristretto with allocator fix. --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index e8b5c83bf..67b3eb8e8 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,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.4-0.20201111190620-1040b7ded521 + github.com/dgraph-io/ristretto v0.0.4-0.20201111235953-24ae56eee4ab github.com/dustin/go-humanize v1.0.0 github.com/golang/protobuf v1.3.1 github.com/golang/snappy v0.0.1 diff --git a/go.sum b/go.sum index f02228044..7e99a6fd9 100644 --- a/go.sum +++ b/go.sum @@ -15,8 +15,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.4-0.20201111190620-1040b7ded521 h1:81yY9QfuVfSVccqD7cLA/JohhMX+TQCfSjmGS7jUJCc= -github.com/dgraph-io/ristretto v0.0.4-0.20201111190620-1040b7ded521/go.mod h1:bDI4cDaalvYSji3vBVDKrn9ouDZrwN974u8ZO/AhYXs= +github.com/dgraph-io/ristretto v0.0.4-0.20201111235953-24ae56eee4ab h1:vpmDtF6TzkokYwlM7bORte06limiJiaVjgn6fF1TTDY= +github.com/dgraph-io/ristretto v0.0.4-0.20201111235953-24ae56eee4ab/go.mod h1:bDI4cDaalvYSji3vBVDKrn9ouDZrwN974u8ZO/AhYXs= 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= From 34c61545407226b475c1b3c2de1e0ffe3b32ee93 Mon Sep 17 00:00:00 2001 From: Naman Jain Date: Thu, 12 Nov 2020 14:04:17 +0530 Subject: [PATCH 05/35] fix(readonly): fix the file opening mode (#1592) --- memtable.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/memtable.go b/memtable.go index 724de7c2b..cd57571db 100644 --- a/memtable.go +++ b/memtable.go @@ -81,7 +81,11 @@ func (db *DB) openMemTables(opt Options) error { return fids[i] < fids[j] }) for _, fid := range fids { - mt, err := db.openMemTable(fid) + flags := os.O_RDWR + if db.opt.ReadOnly { + flags = os.O_RDONLY + } + mt, err := db.openMemTable(fid, flags) if err != nil { return y.Wrapf(err, "while opening fid: %d", fid) } @@ -103,7 +107,7 @@ func (db *DB) openMemTables(opt Options) error { const memFileExt string = ".mem" -func (db *DB) openMemTable(fid int) (*memTable, error) { +func (db *DB) openMemTable(fid, flags int) (*memTable, error) { filepath := db.mtFilePath(fid) s := skl.NewSkiplist(arenaSize(db.opt)) mt := &memTable{ @@ -122,7 +126,7 @@ func (db *DB) openMemTable(fid int) (*memTable, error) { registry: db.registry, writeAt: vlogHeaderSize, } - lerr := mt.wal.open(filepath, os.O_RDWR|os.O_CREATE, db.opt) + lerr := mt.wal.open(filepath, flags, db.opt) if lerr != z.NewFile && lerr != nil { return nil, y.Wrapf(lerr, "While opening memtable: %s", filepath) } @@ -145,7 +149,7 @@ func (db *DB) openMemTable(fid int) (*memTable, error) { var errExpectingNewFile = errors.New("Expecting to create a new file, but found an existing file") func (db *DB) newMemTable() (*memTable, error) { - mt, err := db.openMemTable(db.nextMemFid) + mt, err := db.openMemTable(db.nextMemFid, os.O_CREATE|os.O_RDWR) if err == z.NewFile { db.nextMemFid++ return mt, nil @@ -268,10 +272,7 @@ func (lf *logFile) Truncate(end int64) error { } else if fi.Size() == end { return nil } - if lf.opt.ReadOnly { - return y.Wrapf(ErrTruncateNeeded, - "truncate to %d from %d for file: %s", end, lf.size, lf.path) - } + y.AssertTrue(!lf.opt.ReadOnly) lf.size = uint32(end) return lf.MmapFile.Truncate(end) } From 741de05a13307aee6c68201e048c905e8dc429a9 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Thu, 12 Nov 2020 15:05:11 -0800 Subject: [PATCH 06/35] Small improvements (#1598) Decrease the size of DISCARD file and WAL for Memtables. --- db.go | 17 ++++++----------- discard.go | 14 +++++++++----- memtable.go | 20 +++++++++++++++----- value.go | 7 +++++-- value_test.go | 2 +- 5 files changed, 36 insertions(+), 24 deletions(-) diff --git a/db.go b/db.go index 079efcded..befec1957 100644 --- a/db.go +++ b/db.go @@ -925,19 +925,11 @@ func (db *DB) ensureRoomForWrite() error { db.Lock() defer db.Unlock() - var forceFlush bool - // We don't need to force flush the memtable in in-memory mode because the size of the WAL will - // always be zero. - if !forceFlush && !db.opt.InMemory { - // Force flush if memTable WAL is getting filled up. - forceFlush = int64(db.mt.wal.writeAt) > db.opt.ValueLogFileSize - } - - if !forceFlush && db.mt.sl.MemSize() < db.opt.MemTableSize { + y.AssertTrue(db.mt != nil) // A nil mt indicates that DB is being closed. + if !db.mt.isFull() { return nil } - y.AssertTrue(db.mt != nil) // A nil mt indicates that DB is being closed. select { case db.flushChan <- flushTask{mt: db.mt}: db.opt.Debugf("Flushing memtable, mt.size=%d size of flushChan: %d\n", @@ -1637,7 +1629,10 @@ func (db *DB) dropAll() (func(), error) { // - Compact rest of the levels, Li->Li, picking tables which have Kp. // - Resume memtable flushes, compactions and writes. func (db *DB) DropPrefix(prefixes ...[]byte) error { - db.opt.Infof("DropPrefix Called %s", prefixes) + if len(prefixes) == 0 { + return nil + } + db.opt.Infof("DropPrefix called for %s", prefixes) f, err := db.prepareToDrop() if err != nil { return err diff --git a/discard.go b/discard.go index ba15191fd..9d7d3da03 100644 --- a/discard.go +++ b/discard.go @@ -38,14 +38,12 @@ type discardStats struct { } const discardFname string = "DISCARD" -const discardFsize int = 1 << 30 -const maxSlot int = 64 << 20 func initDiscardStats(opt Options) (*discardStats, error) { fname := path.Join(opt.ValueDir, discardFname) // 1GB file can store 67M discard entries. Each entry is 16 bytes. - mf, err := z.OpenMmapFile(fname, os.O_CREATE|os.O_RDWR, discardFsize) + mf, err := z.OpenMmapFile(fname, os.O_CREATE|os.O_RDWR, 1<<20) lf := &discardStats{ MmapFile: mf, opt: opt, @@ -58,7 +56,7 @@ func initDiscardStats(opt Options) (*discardStats, error) { return nil, y.Wrapf(err, "while opening file: %s\n", discardFname) } - for slot := 0; slot < maxSlot; slot++ { + for slot := 0; slot < lf.maxSlot(); slot++ { if lf.get(16*slot) == 0 { lf.nextEmptySlot = slot break @@ -98,6 +96,10 @@ func (lf *discardStats) zeroOut() { lf.set(lf.nextEmptySlot*16+8, 0) } +func (lf *discardStats) maxSlot() int { + return len(lf.Data) / 16 +} + // Update would update the discard stats for the given file id. If discard is // 0, it would return the current value of discard for the file. If discard is // < 0, it would set the current value of discard to zero for the file. @@ -134,7 +136,9 @@ func (lf *discardStats) Update(fidu uint32, discard int64) int64 { // Move to next slot. lf.nextEmptySlot++ - y.AssertTrue(lf.nextEmptySlot < maxSlot) + for lf.nextEmptySlot >= lf.maxSlot() { + y.Check(lf.Truncate(2 * int64(len(lf.Data)))) + } lf.zeroOut() sort.Sort(lf) diff --git a/memtable.go b/memtable.go index cd57571db..a6f33bae6 100644 --- a/memtable.go +++ b/memtable.go @@ -125,8 +125,9 @@ func (db *DB) openMemTable(fid, flags int) (*memTable, error) { path: filepath, registry: db.registry, writeAt: vlogHeaderSize, + opt: db.opt, } - lerr := mt.wal.open(filepath, flags, db.opt) + lerr := mt.wal.open(filepath, flags, 2*db.opt.MemTableSize) if lerr != z.NewFile && lerr != nil { return nil, y.Wrapf(lerr, "While opening memtable: %s", filepath) } @@ -170,6 +171,17 @@ func (mt *memTable) SyncWAL() error { return mt.wal.Sync() } +func (mt *memTable) isFull() bool { + if mt.sl.MemSize() >= mt.opt.MemTableSize { + return true + } + if mt.opt.InMemory { + // InMemory mode doesn't have any WAL. + return false + } + return int64(mt.wal.writeAt) >= mt.opt.MemTableSize +} + func (mt *memTable) Put(key []byte, value y.ValueStruct) error { entry := &Entry{ Key: key, @@ -540,10 +552,8 @@ func (lf *logFile) zeroNextEntry() { z.ZeroOut(lf.Data, int(lf.writeAt), int(lf.writeAt+maxHeaderSize)) } -func (lf *logFile) open(path string, flags int, opt Options) error { - lf.opt = opt - - mf, ferr := z.OpenMmapFile(path, flags, 2*int(opt.ValueLogFileSize)) +func (lf *logFile) open(path string, flags int, fsize int64) error { + mf, ferr := z.OpenMmapFile(path, flags, int(fsize)) lf.MmapFile = mf if ferr == z.NewFile { diff --git a/value.go b/value.go index 115ea319c..bcce9bba7 100644 --- a/value.go +++ b/value.go @@ -508,8 +508,9 @@ func (vlog *valueLog) createVlogFile() (*logFile, error) { path: path, registry: vlog.db.registry, writeAt: vlogHeaderSize, + opt: vlog.opt, } - err := lf.open(path, os.O_RDWR|os.O_CREATE|os.O_EXCL, vlog.opt) + err := lf.open(path, os.O_RDWR|os.O_CREATE|os.O_EXCL, 2*vlog.opt.ValueLogFileSize) if err != z.NewFile && err != nil { return nil, err } @@ -574,7 +575,9 @@ func (vlog *valueLog) open(db *DB) error { y.AssertTrue(ok) // Just open in RDWR mode. This should not create a new log file. - if err := lf.open(vlog.fpath(fid), os.O_RDWR, vlog.opt); err != nil { + lf.opt = vlog.opt + if err := lf.open(vlog.fpath(fid), os.O_RDWR, + 2*vlog.opt.ValueLogFileSize); err != nil { return y.Wrapf(err, "Open existing file: %q", lf.path) } // We shouldn't delete the maxFid file. diff --git a/value_test.go b/value_test.go index 5645a763e..ad52f6845 100644 --- a/value_test.go +++ b/value_test.go @@ -990,7 +990,7 @@ func TestValueLogTruncate(t *testing.T) { require.Equal(t, 4, int(db.mt.wal.fid)) fileStat, err := db.mt.wal.Fd.Stat() require.NoError(t, err) - require.Equal(t, 2*db.opt.ValueLogFileSize, fileStat.Size()) + require.Equal(t, 2*db.opt.MemTableSize, fileStat.Size()) fileCountAfterCorruption := len(db.Tables()) + len(db.imm) + 1 // +1 for db.mt // We should have one memtable and one sst file. From 5a96b2cfcb6fcbbd63c2908c6e492bf3b33e5249 Mon Sep 17 00:00:00 2001 From: Ahsan Barkati Date: Thu, 19 Nov 2020 22:29:41 +0530 Subject: [PATCH 07/35] fix(skiplist): Remove z.Buffer from skiplist (#1600) Remove z.Buffer from skiplist because we are using static sized buffer. --- go.mod | 6 +++--- go.sum | 47 +++++++++++++++++++++++++++++++++++++++++------ skl/arena.go | 41 ++++++++++++++++++++++++++--------------- skl/skl.go | 21 +++++++++++++-------- 4 files changed, 83 insertions(+), 32 deletions(-) diff --git a/go.mod b/go.mod index 67b3eb8e8..6613c798c 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,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.4-0.20201111235953-24ae56eee4ab + github.com/dgraph-io/ristretto v0.0.4-0.20201118204411-eeefcb8bb4ef github.com/dustin/go-humanize v1.0.0 github.com/golang/protobuf v1.3.1 github.com/golang/snappy v0.0.1 @@ -18,7 +18,7 @@ require ( github.com/spf13/cobra v0.0.5 github.com/stretchr/testify v1.4.0 go.opencensus.io v0.22.5 - golang.org/x/net v0.0.0-20190620200207-3b0461eec859 - golang.org/x/sys v0.0.0-20200918174421-af09f7315aff + golang.org/x/net v0.0.0-20201021035429-f5854403a974 + golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 // indirect ) diff --git a/go.sum b/go.sum index 7e99a6fd9..325f83d4a 100644 --- a/go.sum +++ b/go.sum @@ -11,17 +11,20 @@ github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDk github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/cosiner/argv v0.1.0/go.mod h1:EusR6TucWKX+zFgtdUsKT2Cvg45K5rtpCcWz4hK06d8= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= 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.4-0.20201111235953-24ae56eee4ab h1:vpmDtF6TzkokYwlM7bORte06limiJiaVjgn6fF1TTDY= -github.com/dgraph-io/ristretto v0.0.4-0.20201111235953-24ae56eee4ab/go.mod h1:bDI4cDaalvYSji3vBVDKrn9ouDZrwN974u8ZO/AhYXs= +github.com/dgraph-io/ristretto v0.0.4-0.20201118204411-eeefcb8bb4ef h1:+Q2Ow3+Ut07Q1LMDkIvzgyxk17o4u8by1JT7hTb+T1Y= +github.com/dgraph-io/ristretto v0.0.4-0.20201118204411-eeefcb8bb4ef/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= 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= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/go-delve/delve v1.5.0/go.mod h1:c6b3a1Gry6x8a4LGCe/CWzrocrfaHvkUxCj3k4bvSUQ= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6 h1:ZgQEtGgCBiWRM39fZuwSd1LwSqqSW0hOdXCYYDX0R3I= @@ -36,65 +39,89 @@ github.com/google/flatbuffers v1.12.0 h1:/PtAHvnBY4Kqnx/xCQ3OIV9uYcSFGScBsWI3Oog github.com/google/flatbuffers v1.12.0/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= github.com/google/go-cmp v0.3.0 h1:crn/baboCvb5fXaQ0IJ1SGTsTVrWpDsCWC8EGETZijY= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-dap v0.2.0/go.mod h1:5q8aYQFnHOAZEMP+6vmq25HKYAEwE+LF5yh7JKrrhSQ= +github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0 h1:45sCR5RtlFHMR4UwH9sdQ5TC8v0qDQCHnXt+kaKSTVE= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= +github.com/mattn/go-colorable v0.0.0-20170327083344-ded68f7a9561/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= +github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/mmcloughlin/avo v0.0.0-20201105074841-5d2f697d268f/go.mod h1:6aKT4zZIrpGqB3RpFU14ByCSSyKY6LfJz4J/JJChHfI= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= +github.com/peterh/liner v0.0.0-20170317030525-88609521dc4b/go.mod h1:xIteQHvHuaLYG9IFj6mSxM0fCKrs34IrEQUhOYuGPHc= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= +github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= +github.com/spf13/cobra v0.0.0-20170417170307-b6cb39589372/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.5 h1:f0B+LkLX6DtmRH1isoNA9VTtNUK9K8xYd28JNNfOv/s= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= +github.com/spf13/pflag v0.0.0-20170417173400-9e4c21054fa1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.3 h1:zPAT6CGy6wXeQ7NtTnaTerfKOsV6V6F8agHXFiazDkg= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= -github.com/stretchr/objx v0.1.0 h1:4G4v2dO3VZwixGIRoQ5Lfboy6nUhCyYzaqnIAPPhYs4= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/twitchyliquid64/golang-asm v0.15.0/go.mod h1:a1lVb/DtPvCB8fslRZhAngC2+aY1QWCk3Cedj/Gdt08= github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= go.opencensus.io v0.22.5 h1:dntmOdLpSpHlVqbW5Eay97DelsZHe+55D+xC6i0dDS0= go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= +go.starlark.net v0.0.0-20190702223751-32f345186213/go.mod h1:c1/X6cHgvdXj6pUlmWKMkuqRnW4K8x2vwt6JAaaircg= +golang.org/x/arch v0.0.0-20190927153633-4e8777c89be4/go.mod h1:flIaEI6LNU6xOCD5PaJvn9wGP0agmIOqjrtsKGRguv4= +golang.org/x/arch v0.0.0-20201008161808-52c3e6f60cff/go.mod h1:flIaEI6LNU6xOCD5PaJvn9wGP0agmIOqjrtsKGRguv4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190620200207-3b0461eec859 h1:R/3boaszxrf1GEUWTVDzSKVwLmSJpwZ1yqXm8j0v2QI= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20201021035429-f5854403a974 h1:IX6qOQeG5uLjB/hjjwjedwfjND0hgjPMMyO1RoIXQNI= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200918174421-af09f7315aff h1:1CPUrky56AcgSpxz/KfgzQWzfG09u5YOL8MvPYBlrL8= -golang.org/x/sys v0.0.0-20200918174421-af09f7315aff/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f h1:+Nyd8tzPX9R7BWHguqsrbFdRx3WQ/1ib8I44HXV5yTA= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= @@ -102,6 +129,12 @@ golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGm golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191127201027-ecd32218bd7f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20201105001634-bc3cf281b174/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -113,6 +146,8 @@ google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiq gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= diff --git a/skl/arena.go b/skl/arena.go index 96e6b5c76..b4f339ea2 100644 --- a/skl/arena.go +++ b/skl/arena.go @@ -17,10 +17,11 @@ package skl import ( + "log" + "sync/atomic" "unsafe" "github.com/dgraph-io/badger/v2/y" - "github.com/dgraph-io/ristretto/z" ) const ( @@ -35,11 +36,21 @@ const ( // Arena should be lock-free. type Arena struct { - *z.Buffer + data []byte + offset uint32 } func (s *Arena) size() int64 { - return int64(s.LenNoPadding()) + return int64(atomic.LoadUint32(&s.offset)) +} + +func (s *Arena) allocate(sz uint32) uint32 { + offset := atomic.AddUint32(&s.offset, sz) + if offset >= uint32(len(s.data)) { + log.Fatalf("Arena too small, toWrite:%d newTotal:%d limit:%d", + sz, offset, len(s.data)) + } + return offset } // allocateValue encodes valueStruct and put it in the arena buffer. @@ -58,9 +69,10 @@ func (s *Arena) putNode(height int) uint32 { // Pad the allocation with enough bytes to ensure pointer alignment. l := uint32(MaxNodeSize - unusedSize + nodeAlign) - n := s.IncrementOffset(int(l)) + n := s.allocate(l) + // Return the aligned offset. - m := (uint32(n) - l + uint32(nodeAlign)) & ^uint32(nodeAlign) + m := (n - l + uint32(nodeAlign)) & ^uint32(nodeAlign) return m } @@ -70,18 +82,17 @@ func (s *Arena) putNode(height int) uint32 { // decoding will incur some overhead. func (s *Arena) putVal(v y.ValueStruct) uint32 { l := uint32(v.EncodedSize()) - m := s.IncrementOffset(int(l)) - buf := s.Bytes()[uint32(m)-l : m] + offset := s.allocate(l) - l + buf := s.data[offset : offset+l] v.Encode(buf) - return uint32(m) - l + return offset } // putKey puts the key and returns its offset func (s *Arena) putKey(key []byte) uint32 { keySz := uint32(len(key)) - n := s.IncrementOffset(int(keySz)) - offset := uint32(n) - keySz - buf := s.Bytes()[offset : offset+keySz] + offset := s.allocate(keySz) - keySz + buf := s.data[offset : offset+keySz] y.AssertTrue(len(key) == copy(buf, key)) return offset } @@ -92,18 +103,18 @@ func (s *Arena) getNode(offset uint32) *node { if offset == 0 { return nil } - return (*node)(unsafe.Pointer(&s.Bytes()[offset])) + return (*node)(unsafe.Pointer(&s.data[offset])) } // getKey returns byte slice at offset. func (s *Arena) getKey(offset uint32, size uint16) []byte { - return s.Bytes()[offset : offset+uint32(size)] + return s.data[offset : offset+uint32(size)] } // getVal returns byte slice at offset. The given size should be just the value // size and should NOT include the meta bytes. func (s *Arena) getVal(offset uint32, size uint32) (ret y.ValueStruct) { - ret.Decode(s.Bytes()[offset : offset+size]) + ret.Decode(s.data[offset : offset+size]) return } @@ -113,7 +124,7 @@ func (s *Arena) getNodeOffset(nd *node) uint32 { if nd == nil { return 0 } - val := uint32(uintptr(unsafe.Pointer(nd)) - uintptr(unsafe.Pointer(&s.Bytes()[0]))) + val := uint32(uintptr(unsafe.Pointer(nd)) - uintptr(unsafe.Pointer(&s.data[0]))) return val } diff --git a/skl/skl.go b/skl/skl.go index 4e1fcd4a7..4ad8f386c 100644 --- a/skl/skl.go +++ b/skl/skl.go @@ -84,6 +84,7 @@ type Skiplist struct { hasVersions bool comparator comparatorFunc OnClose func() + Release func() } // IncrRef increases the refcount @@ -100,7 +101,9 @@ func (s *Skiplist) DecrRef() { if s.OnClose != nil { s.OnClose() } - s.arena.Release() + if s.Release != nil { + s.Release() + } // Indicate we are closed. Good for testing. Also, lets GC reclaim memory. Race condition // here would suggest we are accessing skiplist when we are supposed to have no reference! s.arena = nil @@ -132,16 +135,18 @@ func decodeValue(value uint64) (valOffset uint32, valSize uint32) { // NewSkiplist makes a new empty skiplist, with a given arena size. func NewSkiplist(arenaSize int64) *Skiplist { - buf, err := z.NewBufferWith(int(arenaSize), int(arenaSize), z.UseCalloc) - y.Check(err) - skl := NewSkiplistWithBuffer(buf, true) - return skl + buf := z.Calloc(int(arenaSize)) + s := NewSkiplistWith(buf, true) + s.Release = func() { + z.Free(s.arena.data) + } + return s } -// NewSkiplistWithBuffer makes a new skiplist, with a given buffer. -func NewSkiplistWithBuffer(buf *z.Buffer, hasVersions bool) *Skiplist { +// NewSkiplistWith makes a new skiplist, with a given byte slice. +func NewSkiplistWith(buf []byte, hasVersions bool) *Skiplist { arena := new(Arena) - arena.Buffer = buf + arena.data = buf offset := arena.allocateValue(y.ValueStruct{}) head := newNode(arena, nil, offset, maxHeight) sl := &Skiplist{ From 5ff9e1dd1b89230b137154bd80ea03925710d7ab Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Mon, 23 Nov 2020 11:14:37 -0800 Subject: [PATCH 08/35] Bug Fix: Fix up how we use z.Allocator Do not use AllocatorPool, because that has shown weird crashes due to Go's interpretation of slices. The new system uses Go memory for z.Allocator and avoids reusing it. --- db.go | 4 ---- go.mod | 2 +- go.sum | 4 ++-- options.go | 1 - stream.go | 8 ++------ table/builder.go | 4 ++-- table/table.go | 2 -- 7 files changed, 7 insertions(+), 18 deletions(-) diff --git a/db.go b/db.go index befec1957..eb8a3b85b 100644 --- a/db.go +++ b/db.go @@ -95,7 +95,6 @@ type DB struct { registry *KeyRegistry blockCache *ristretto.Cache indexCache *ristretto.Cache - allocPool *z.AllocatorPool } const ( @@ -219,7 +218,6 @@ func Open(opt Options) (*DB, error) { valueDirGuard: valueDirLockGuard, orc: newOracle(opt), pub: newPublisher(), - allocPool: z.NewAllocatorPool(8), } // Cleanup all the goroutines started by badger in case of an error. defer func() { @@ -478,8 +476,6 @@ func (db *DB) IsClosed() bool { } func (db *DB) close() (err error) { - defer db.allocPool.Release() - db.opt.Debugf("Closing database") db.opt.Infof("Lifetime L0 stalled for: %s\n", time.Duration(db.lc.l0stallsMs)) diff --git a/go.mod b/go.mod index 6613c798c..0c43ba3ba 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,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.4-0.20201118204411-eeefcb8bb4ef + github.com/dgraph-io/ristretto v0.0.4-0.20201123185045-68b18eb1b695 github.com/dustin/go-humanize v1.0.0 github.com/golang/protobuf v1.3.1 github.com/golang/snappy v0.0.1 diff --git a/go.sum b/go.sum index 325f83d4a..7b01bad1f 100644 --- a/go.sum +++ b/go.sum @@ -17,8 +17,8 @@ github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ3 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.4-0.20201118204411-eeefcb8bb4ef h1:+Q2Ow3+Ut07Q1LMDkIvzgyxk17o4u8by1JT7hTb+T1Y= -github.com/dgraph-io/ristretto v0.0.4-0.20201118204411-eeefcb8bb4ef/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= +github.com/dgraph-io/ristretto v0.0.4-0.20201123185045-68b18eb1b695 h1:UP7ZrWkI7Qnp0T2ejWq7HGJOfiTIUfLE58Jg862a1eQ= +github.com/dgraph-io/ristretto v0.0.4-0.20201123185045-68b18eb1b695/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= 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/options.go b/options.go index ede01d2e4..33e2c4865 100644 --- a/options.go +++ b/options.go @@ -177,7 +177,6 @@ func buildTableOptions(db *DB) table.Options { ZSTDCompressionLevel: opt.ZSTDCompressionLevel, BlockCache: db.blockCache, IndexCache: db.indexCache, - AllocPool: db.allocPool, DataKey: dk, } } diff --git a/stream.go b/stream.go index 972226e4c..63e949166 100644 --- a/stream.go +++ b/stream.go @@ -92,8 +92,6 @@ type Stream struct { // Use allocators to generate KVs. allocatorsMu sync.RWMutex allocators map[int]*z.Allocator - - allocPool *z.AllocatorPool } func (st *Stream) Allocator(threadId int) *z.Allocator { @@ -184,7 +182,7 @@ func (st *Stream) produceRanges(ctx context.Context) { func (st *Stream) newAllocator(threadId int) *z.Allocator { st.allocatorsMu.Lock() - a := st.allocPool.Get(batchSize) + a := z.NewAllocator(batchSize) a.Tag = "Stream " + st.LogPrefix st.allocators[threadId] = a st.allocatorsMu.Unlock() @@ -309,7 +307,7 @@ func (st *Stream) streamKVs(ctx context.Context) error { returnAllocs := func() { for ref := range allocs { a := z.AllocatorFrom(ref) - st.allocPool.Return(a) + a.Release() delete(allocs, ref) } } @@ -400,7 +398,6 @@ outer: // are serial. In case any of these steps encounter an error, Orchestrate would stop execution and // return that error. Orchestrate can be called multiple times, but in serial order. func (st *Stream) Orchestrate(ctx context.Context) error { - st.allocPool = z.NewAllocatorPool(st.NumGo) defer func() { for _, a := range st.allocators { // Using AllocatorFrom is better because if the allocator is already freed up, it would @@ -408,7 +405,6 @@ func (st *Stream) Orchestrate(ctx context.Context) error { a = z.AllocatorFrom(a.Ref) a.Release() } - st.allocPool.Release() }() st.rangeCh = make(chan keyRange, 3) // Contains keys for posting lists. diff --git a/table/builder.go b/table/builder.go index 5f400385f..39bb166c8 100644 --- a/table/builder.go +++ b/table/builder.go @@ -127,7 +127,7 @@ func NewTableBuilder(opts Options) *Builder { sz = maxAllocatorInitialSz } b := &Builder{ - alloc: opts.AllocPool.Get(sz), + alloc: z.NewAllocator(sz), opts: &opts, } b.alloc.Tag = "Builder" @@ -186,7 +186,7 @@ func (b *Builder) handleBlock() { // Close closes the TableBuilder. func (b *Builder) Close() { - b.opts.AllocPool.Return(b.alloc) + b.alloc.Release() } // Empty returns whether it's empty. diff --git a/table/table.go b/table/table.go index 0ff60c22b..17cc31f58 100644 --- a/table/table.go +++ b/table/table.go @@ -80,8 +80,6 @@ type Options struct { BlockCache *ristretto.Cache IndexCache *ristretto.Cache - AllocPool *z.AllocatorPool - // ZSTDCompressionLevel is the ZSTD compression level used for compressing blocks. ZSTDCompressionLevel int } From 340ccfc4a75dcbca3181d68e0a153886b9f52c27 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Mon, 23 Nov 2020 13:21:45 -0800 Subject: [PATCH 09/35] Test: Add a test for encryption. --- y/encrypt_test.go | 64 +++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 64 insertions(+) create mode 100644 y/encrypt_test.go diff --git a/y/encrypt_test.go b/y/encrypt_test.go new file mode 100644 index 000000000..91d50acb1 --- /dev/null +++ b/y/encrypt_test.go @@ -0,0 +1,64 @@ +/* + * 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 y + +import ( + "crypto/aes" + "crypto/rand" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestXORBlock(t *testing.T) { + key := make([]byte, 32) + rand.Read(key) + + var iv []byte + { + b, err := aes.NewCipher(key) + require.NoError(t, err) + iv = make([]byte, b.BlockSize()) + rand.Read(iv) + t.Logf("Using %d size IV\n", len(iv)) + } + + src := make([]byte, 1024) + rand.Read(src) + + dst := make([]byte, 1024) + err := XORBlock(dst, src, key, iv) + require.NoError(t, err) + + act := make([]byte, 1024) + err = XORBlock(act, dst, key, iv) + require.NoError(t, err) + require.Equal(t, src, act) + + // Now check if we can use the same byte slice as src and dst. While this is useful to know that + // we can use src and dst as the same slice, this isn't applicable to Badger because we're + // reading data right off mmap. We should not modify that data, so we have to use a different + // slice for dst anyway. + cp := append([]byte{}, src...) + err = XORBlock(cp, cp, key, iv) + require.NoError(t, err) + require.Equal(t, dst, cp) + + err = XORBlock(cp, cp, key, iv) + require.NoError(t, err) + require.Equal(t, src, cp) +} From f36daf538d71baed9d23c4b1c187c01115e73634 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Wed, 25 Nov 2020 17:31:49 +0530 Subject: [PATCH 10/35] fix(stream): Stop produceKVs on error (#1604) The orchestrate function would get blocked forever if send function returned an error. The produceKv go routines would also get blocked since the size of the error chan was 1. --- stream.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/stream.go b/stream.go index 63e949166..e857324c0 100644 --- a/stream.go +++ b/stream.go @@ -398,6 +398,8 @@ outer: // are serial. In case any of these steps encounter an error, Orchestrate would stop execution and // return that error. Orchestrate can be called multiple times, but in serial order. func (st *Stream) Orchestrate(ctx context.Context) error { + ctx, cancel := context.WithCancel(ctx) + defer cancel() defer func() { for _, a := range st.allocators { // Using AllocatorFrom is better because if the allocator is already freed up, it would @@ -421,7 +423,7 @@ func (st *Stream) Orchestrate(ctx context.Context) error { // Picks up ranges from Badger, and sends them to rangeCh. go st.produceRanges(ctx) - errCh := make(chan error, 1) // Stores error by consumeKeys. + errCh := make(chan error, st.NumGo) // Stores error by consumeKeys. var wg sync.WaitGroup for i := 0; i < st.NumGo; i++ { wg.Add(1) @@ -442,7 +444,11 @@ func (st *Stream) Orchestrate(ctx context.Context) error { kvErr := make(chan error, 1) go func() { // Picks up KV lists from kvChan, and sends them to Output. - kvErr <- st.streamKVs(ctx) + err := st.streamKVs(ctx) + if err != nil { + cancel() // Stop all the go routines. + } + kvErr <- err }() wg.Wait() // Wait for produceKVs to be over. close(st.kvChan) // Now we can close kvChan. From 925e15b8c467f8555dd08b9d4ce3ab9a513a1e1a Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Wed, 25 Nov 2020 17:59:19 +0530 Subject: [PATCH 11/35] Fix race condition in L0StallMs variable (#1605) This fixes two issues - Atomic variable was not being accessed correctly - Atomic variable should be the first member of the struct to ensure proper alignment. Failure to do so will cause a segmentation fault. Fixes DGRAPH-2773 --- db.go | 2 +- levels.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/db.go b/db.go index eb8a3b85b..3065837f4 100644 --- a/db.go +++ b/db.go @@ -477,7 +477,7 @@ func (db *DB) IsClosed() bool { func (db *DB) close() (err error) { db.opt.Debugf("Closing database") - db.opt.Infof("Lifetime L0 stalled for: %s\n", time.Duration(db.lc.l0stallsMs)) + db.opt.Infof("Lifetime L0 stalled for: %s\n", time.Duration(atomic.LoadInt64(&db.lc.l0stallsMs))) atomic.StoreInt32(&db.blockWrites, 1) diff --git a/levels.go b/levels.go index 0b3950817..49f7ccbe8 100644 --- a/levels.go +++ b/levels.go @@ -41,13 +41,13 @@ import ( type levelsController struct { nextFileID uint64 // Atomic + l0stallsMs int64 // Atomic // The following are initialized once and const. levels []*levelHandler kv *DB - cstatus compactStatus - l0stallsMs int64 + cstatus compactStatus } // revertToManifest checks that all necessary table files exist and removes all table files not From feb1f5f3ad2a96f5c7a33f6d34e00851db37f423 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Wed, 25 Nov 2020 18:45:52 +0530 Subject: [PATCH 12/35] chore(iterators): Do not return error on missing vlog (#1602) Add debugging information in yieldItemValue function to find the root cause of missing vlog files error. Note: This commit should be reverted once the issue has been resolved. --- iterator.go | 22 +++++++++++++++++++++- value_test.go | 12 ++++++++---- 2 files changed, 29 insertions(+), 5 deletions(-) diff --git a/iterator.go b/iterator.go index 9eefbacf7..db0578ace 100644 --- a/iterator.go +++ b/iterator.go @@ -170,8 +170,28 @@ func (item *Item) yieldItemValue() ([]byte, func(), error) { if err != nil { db.opt.Logger.Errorf("Unable to read: Key: %v, Version : %v, meta: %v, userMeta: %v"+ " Error: %v", key, item.version, item.meta, item.userMeta, err) + txn := db.NewTransaction(false) + defer txn.Discard() + + iopt := DefaultIteratorOptions + iopt.AllVersions = true + iopt.InternalAccess = true + iopt.PrefetchValues = false + + it := txn.NewKeyIterator(item.Key(), iopt) + defer it.Close() + for it.Rewind(); it.Valid(); it.Next() { + item := it.Item() + var vp valuePointer + if item.meta&bitValuePointer > 0 { + vp.Decode(item.vptr) + } + db.opt.Logger.Errorf("Key: %v, Version : %v, meta: %v, userMeta: %v valuePointer: %+v", + item.Key(), item.version, item.meta, item.userMeta, vp) + } } - return result, cb, err + // Don't return error if we cannot read the value. Just log the error. + return result, cb, nil } func runCallback(cb func()) { diff --git a/value_test.go b/value_test.go index ad52f6845..4b8505633 100644 --- a/value_test.go +++ b/value_test.go @@ -1084,10 +1084,14 @@ func TestValueEntryChecksum(t *testing.T) { entry, err := txn.Get(k) require.NoError(t, err) - x, err := entry.ValueCopy(nil) - require.Error(t, err) - require.Contains(t, err.Error(), "ErrEOF") - require.Nil(t, x) + // TODO(ibrahim): This test is broken since we're not returning errors + // in case we cannot read the values. This is incorrect behavior but + // we're doing this to debug an issue where the values are being read + // from old vlog files. + _, _ = entry.ValueCopy(nil) + // require.Error(t, err) + // require.Contains(t, err.Error(), "ErrEOF") + // require.Nil(t, x) require.NoError(t, db.Close()) }) From ab8b5d9ed1964c203a9c5cf0a46434f49c26995e Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 25 Nov 2020 10:44:31 -0800 Subject: [PATCH 13/35] Opt(stream): Use z.Buffer to stream data (#1606) Stream.Send now sends out z.Buffer instead of pb.KVList. z.Buffer marshals each KV as a separate slice. This significantly reduces the memory requirement by the Stream framework. Stream no longer uses z.Allocator or tries to put pb.KV struct on the Allocator for memory safety reasons. Bring back the z.AllocatorPool for table.Builder. Changes: * Use z.Buffer for stream.Send * Only use 8 streams in write bench * Revert "Bug Fix: Fix up how we use z.Allocator" This reverts commit 5ff9e1dd1b89230b137154bd80ea03925710d7ab. * Bring allocator back. Use z.Buffer for send * Add BufferToKVList function * Print jemalloc while stream * Bring in latest Ristretto * Fix memory leak and benchmark read test Co-authored-by: Ibrahim Jarif --- backup.go | 7 +- badger/cmd/bank.go | 15 ++-- badger/cmd/read_bench.go | 22 ++++-- badger/cmd/write_bench.go | 19 ++--- batch.go | 35 ++++++--- db.go | 8 ++- db2_test.go | 22 +++++- db_test.go | 11 +-- go.mod | 2 +- go.sum | 4 +- options.go | 1 + stream.go | 145 +++++++++++++++++--------------------- stream_test.go | 15 ++-- stream_writer.go | 18 +++-- stream_writer_test.go | 100 +++++++++++++++----------- table/builder.go | 4 +- table/table.go | 2 + test.sh | 7 +- 18 files changed, 259 insertions(+), 178 deletions(-) diff --git a/backup.go b/backup.go index 1c0032401..c729a38b9 100644 --- a/backup.go +++ b/backup.go @@ -25,6 +25,7 @@ import ( "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" + "github.com/dgraph-io/ristretto/z" "github.com/golang/protobuf/proto" ) @@ -115,7 +116,11 @@ func (stream *Stream) Backup(w io.Writer, since uint64) (uint64, error) { } var maxVersion uint64 - stream.Send = func(list *pb.KVList) error { + stream.Send = func(buf *z.Buffer) error { + list, err := BufferToKVList(buf) + if err != nil { + return err + } out := list.Kv[:0] for _, kv := range list.Kv { if maxVersion < kv.Version { diff --git a/badger/cmd/bank.go b/badger/cmd/bank.go index e08a5d1e1..6643256fe 100644 --- a/badger/cmd/bank.go +++ b/badger/cmd/bank.go @@ -33,6 +33,7 @@ import ( "github.com/dgraph-io/badger/v2" "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" + "github.com/dgraph-io/ristretto/z" "github.com/spf13/cobra" ) @@ -359,7 +360,7 @@ func runTest(cmd *cobra.Command, args []string) error { WithNumVersionsToKeep(int(math.MaxInt32)). WithBlockCacheSize(1 << 30). WithIndexCacheSize(1 << 30) - + if verbose { opts = opts.WithLoggingLevel(badger.DEBUG) } @@ -498,13 +499,15 @@ func runTest(cmd *cobra.Command, args []string) error { batch := tmpDb.NewWriteBatch() stream := db.NewStream() - stream.Send = func(list *pb.KVList) error { - for _, kv := range list.Kv { - if err := batch.Set(kv.Key, kv.Value); err != nil { + stream.Send = func(buf *z.Buffer) error { + err := buf.SliceIterate(func(s []byte) error { + var kv pb.KV + if err := kv.Unmarshal(s); err != nil { return err } - } - return nil + return batch.Set(kv.Key, kv.Value) + }) + return err } y.Check(stream.Orchestrate(context.Background())) y.Check(batch.Flush()) diff --git a/badger/cmd/read_bench.go b/badger/cmd/read_bench.go index 0a25c179b..64d6a4582 100644 --- a/badger/cmd/read_bench.go +++ b/badger/cmd/read_bench.go @@ -25,6 +25,7 @@ import ( "time" humanize "github.com/dustin/go-humanize" + "github.com/pkg/errors" "github.com/spf13/cobra" "github.com/dgraph-io/badger/v2" @@ -79,7 +80,7 @@ func init() { // Scan the whole database using the iterators func fullScanDB(db *badger.DB) { - txn := db.NewTransaction(false) + txn := db.NewTransactionAt(math.MaxUint64, false) defer txn.Discard() startTime = time.Now() @@ -111,7 +112,7 @@ func readBench(cmd *cobra.Command, args []string) error { WithBlockCacheSize(blockCacheSize << 20). WithIndexCacheSize(indexCacheSize << 20) fmt.Printf("Opening badger with options = %+v\n", opt) - db, err := badger.Open(opt) + db, err := badger.OpenManaged(opt) if err != nil { return y.Wrapf(err, "unable to open DB") } @@ -205,21 +206,30 @@ func getSampleKeys(db *badger.DB) ([][]byte, error) { return l, nil } + errStop := errors.Errorf("Stop iterating") ctx, cancel := context.WithCancel(context.Background()) defer cancel() - stream.Send = func(l *pb.KVList) error { + stream.Send = func(buf *z.Buffer) error { if count >= sampleSize { return nil } - for _, kv := range l.Kv { + err := buf.SliceIterate(func(s []byte) error { + var kv pb.KV + if err := kv.Unmarshal(s); err != nil { + return err + } keys = append(keys, kv.Key) count++ if count >= sampleSize { cancel() - return nil + return errStop } + return nil + }) + if err == errStop || err == nil { + return nil } - return nil + return err } if err := stream.Orchestrate(ctx); err != nil && err != context.Canceled { diff --git a/badger/cmd/write_bench.go b/badger/cmd/write_bench.go index d6fefe557..418b3b3b3 100644 --- a/badger/cmd/write_bench.go +++ b/badger/cmd/write_bench.go @@ -204,37 +204,38 @@ func writeSorted(db *badger.DB, num uint64) error { } wg := &sync.WaitGroup{} - writeCh := make(chan *pb.KVList, 3) + writeCh := make(chan *z.Buffer, 3) writeRange := func(start, end uint64, streamId uint32) { // end is not included. defer wg.Done() - kvs := &pb.KVList{} + kvBuf := z.NewBuffer(5 << 20) var sz int for i := start; i < end; i++ { key := make([]byte, 8) binary.BigEndian.PutUint64(key, i) - kvs.Kv = append(kvs.Kv, &pb.KV{ + kv := &pb.KV{ Key: key, Value: value, Version: 1, StreamId: streamId, - }) + } + badger.KVToBuffer(kv, kvBuf) sz += es atomic.AddUint64(&entriesWritten, 1) atomic.AddUint64(&sizeWritten, uint64(es)) if sz >= 4<<20 { // 4 MB - writeCh <- kvs - kvs = &pb.KVList{} + writeCh <- kvBuf + kvBuf = z.NewBuffer(1 << 20) sz = 0 } } - writeCh <- kvs + writeCh <- kvBuf } // Let's create some streams. - width := num / 16 + width := num / 4 streamID := uint32(0) for start := uint64(0); start < num; start += width { end := start + width @@ -254,6 +255,7 @@ func writeSorted(db *badger.DB, num uint64) error { if err := writer.Write(kvs); err != nil { panic(err) } + y.Check(kvs.Release()) } log.Println("DONE streaming. Flushing...") return writer.Flush() @@ -395,6 +397,7 @@ func reportStats(c *z.Closer, db *badger.DB) { y.FixedDuration(time.Since(startTime)), humanize.Bytes(sz), humanize.Bytes(bytesRate), entries, entriesRate, humanize.IBytes(uint64(z.NumAllocBytes()))) + if count%10 == 0 { fmt.Printf(db.LevelsToString()) } diff --git a/batch.go b/batch.go index c9bd838b7..79f28b6cd 100644 --- a/batch.go +++ b/batch.go @@ -22,6 +22,7 @@ import ( "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" + "github.com/dgraph-io/ristretto/z" "github.com/pkg/errors" ) @@ -95,17 +96,35 @@ func (wb *WriteBatch) callback(err error) { wb.err.Store(err) } -func (wb *WriteBatch) Write(kvList *pb.KVList) error { +func (wb *WriteBatch) writeKV(kv *pb.KV) error { + e := Entry{Key: kv.Key, Value: kv.Value} + if len(kv.UserMeta) > 0 { + e.UserMeta = kv.UserMeta[0] + } + y.AssertTrue(kv.Version != 0) + e.version = kv.Version + return wb.handleEntry(&e) +} + +func (wb *WriteBatch) Write(buf *z.Buffer) error { wb.Lock() defer wb.Unlock() - for _, kv := range kvList.Kv { - e := Entry{Key: kv.Key, Value: kv.Value} - if len(kv.UserMeta) > 0 { - e.UserMeta = kv.UserMeta[0] + + err := buf.SliceIterate(func(s []byte) error { + kv := &pb.KV{} + if err := kv.Unmarshal(s); err != nil { + return err } - y.AssertTrue(kv.Version != 0) - e.version = kv.Version - if err := wb.handleEntry(&e); err != nil { + return wb.writeKV(kv) + }) + return err +} + +func (wb *WriteBatch) WriteList(kvList *pb.KVList) error { + wb.Lock() + defer wb.Unlock() + for _, kv := range kvList.Kv { + if err := wb.writeKV(kv); err != nil { return err } } diff --git a/db.go b/db.go index 3065837f4..f0bbcb5d2 100644 --- a/db.go +++ b/db.go @@ -95,6 +95,7 @@ type DB struct { registry *KeyRegistry blockCache *ristretto.Cache indexCache *ristretto.Cache + allocPool *z.AllocatorPool } const ( @@ -218,6 +219,7 @@ func Open(opt Options) (*DB, error) { valueDirGuard: valueDirLockGuard, orc: newOracle(opt), pub: newPublisher(), + allocPool: z.NewAllocatorPool(8), } // Cleanup all the goroutines started by badger in case of an error. defer func() { @@ -476,6 +478,8 @@ func (db *DB) IsClosed() bool { } func (db *DB) close() (err error) { + defer db.allocPool.Release() + db.opt.Debugf("Closing database") db.opt.Infof("Lifetime L0 stalled for: %s\n", time.Duration(atomic.LoadInt64(&db.lc.l0stallsMs))) @@ -1779,8 +1783,8 @@ func (db *DB) StreamDB(outOptions Options) error { stream := db.NewStreamAt(math.MaxUint64) stream.LogPrefix = fmt.Sprintf("Streaming DB to new DB at %s", outDir) - stream.Send = func(kvs *pb.KVList) error { - return writer.Write(kvs) + stream.Send = func(buf *z.Buffer) error { + return writer.Write(buf) } if err := stream.Orchestrate(context.Background()); err != nil { return y.Wrapf(err, "cannot stream DB to out DB at %s", outDir) diff --git a/db2_test.go b/db2_test.go index f452a2e16..2d40b1e95 100644 --- a/db2_test.go +++ b/db2_test.go @@ -969,8 +969,20 @@ func TestKeyCount(t *testing.T) { wg.Wait() close(writeCh) }() + + write := func(kvs *pb.KVList) error { + buf := z.NewBuffer(1 << 20) + defer buf.Release() + + for _, kv := range kvs.Kv { + KVToBuffer(kv, buf) + } + writer.Write(buf) + return nil + } + for kvs := range writeCh { - require.NoError(t, writer.Write(kvs)) + require.NoError(t, write(kvs)) } require.NoError(t, writer.Flush()) } @@ -999,8 +1011,11 @@ func TestKeyCount(t *testing.T) { streams := make(map[uint32]int) stream := db2.NewStream() - stream.Send = func(list *pb.KVList) error { - count += len(list.Kv) + stream.Send = func(buf *z.Buffer) error { + list, err := BufferToKVList(buf) + if err != nil { + return err + } for _, kv := range list.Kv { last := streams[kv.StreamId] key := binary.BigEndian.Uint64(kv.Key) @@ -1010,6 +1025,7 @@ func TestKeyCount(t *testing.T) { } streams[kv.StreamId] = int(key) } + count += len(list.Kv) return nil } require.NoError(t, stream.Orchestrate(context.Background())) diff --git a/db_test.go b/db_test.go index b1bec0774..fb4572115 100644 --- a/db_test.go +++ b/db_test.go @@ -38,6 +38,7 @@ import ( "github.com/dgraph-io/badger/v2/options" "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" + "github.com/dgraph-io/ristretto/z" ) // summary is produced when DB is closed. Currently it is used only for testing. @@ -2088,17 +2089,19 @@ func TestVerifyChecksum(t *testing.T) { runBadgerTest(t, &opt, func(t *testing.T, db *DB) { value := make([]byte, 32) y.Check2(rand.Read(value)) - l := &pb.KVList{} st := 0 + + buf := z.NewBuffer(10 << 20) + defer buf.Release() for i := 0; i < 1000; i++ { key := make([]byte, 8) binary.BigEndian.PutUint64(key, uint64(i)) - l.Kv = append(l.Kv, &pb.KV{ + KVToBuffer(&pb.KV{ Key: key, Value: value, StreamId: uint32(st), Version: 1, - }) + }, buf) if i%100 == 0 { st++ } @@ -2106,7 +2109,7 @@ func TestVerifyChecksum(t *testing.T) { sw := db.NewStreamWriter() require.NoError(t, sw.Prepare(), "sw.Prepare() failed") - require.NoError(t, sw.Write(l), "sw.Write() failed") + require.NoError(t, sw.Write(buf), "sw.Write() failed") require.NoError(t, sw.Flush(), "sw.Flush() failed") require.NoError(t, db.VerifyChecksum(), "checksum verification failed for DB") diff --git a/go.mod b/go.mod index 0c43ba3ba..c28fd564c 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,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.4-0.20201123185045-68b18eb1b695 + github.com/dgraph-io/ristretto v0.0.4-0.20201125174811-766bca5e9938 github.com/dustin/go-humanize v1.0.0 github.com/golang/protobuf v1.3.1 github.com/golang/snappy v0.0.1 diff --git a/go.sum b/go.sum index 7b01bad1f..7f46653b3 100644 --- a/go.sum +++ b/go.sum @@ -17,8 +17,8 @@ github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ3 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.4-0.20201123185045-68b18eb1b695 h1:UP7ZrWkI7Qnp0T2ejWq7HGJOfiTIUfLE58Jg862a1eQ= -github.com/dgraph-io/ristretto v0.0.4-0.20201123185045-68b18eb1b695/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= +github.com/dgraph-io/ristretto v0.0.4-0.20201125174811-766bca5e9938 h1:FdSJif9oUVeH+MpsScsrL6OAbdW0pUYvXmkdhDSWWcQ= +github.com/dgraph-io/ristretto v0.0.4-0.20201125174811-766bca5e9938/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= 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/options.go b/options.go index 33e2c4865..ede01d2e4 100644 --- a/options.go +++ b/options.go @@ -177,6 +177,7 @@ func buildTableOptions(db *DB) table.Options { ZSTDCompressionLevel: opt.ZSTDCompressionLevel, BlockCache: db.blockCache, IndexCache: db.indexCache, + AllocPool: db.allocPool, DataKey: dk, } } diff --git a/stream.go b/stream.go index e857324c0..2bb5337a2 100644 --- a/stream.go +++ b/stream.go @@ -28,7 +28,6 @@ import ( "github.com/dgraph-io/badger/v2/y" "github.com/dgraph-io/ristretto/z" humanize "github.com/dustin/go-humanize" - "github.com/golang/protobuf/proto" ) const batchSize = 16 << 20 // 16 MB @@ -80,24 +79,14 @@ type Stream struct { // This is the method where Stream sends the final output. All calls to Send are done by a // single goroutine, i.e. logic within Send method can expect single threaded execution. - Send func(*pb.KVList) error + Send func(buf *z.Buffer) error readTs uint64 db *DB rangeCh chan keyRange - kvChan chan *pb.KVList + kvChan chan *z.Buffer nextStreamId uint32 doneMarkers bool - - // Use allocators to generate KVs. - allocatorsMu sync.RWMutex - allocators map[int]*z.Allocator -} - -func (st *Stream) Allocator(threadId int) *z.Allocator { - st.allocatorsMu.RLock() - defer st.allocatorsMu.RUnlock() - return st.allocators[threadId] } // SendDoneMarkers when true would send out done markers on the stream. False by default. @@ -108,8 +97,7 @@ func (st *Stream) SendDoneMarkers(done bool) { // ToList is a default implementation of KeyToList. It picks up all valid versions of the key, // skipping over deleted or expired keys. func (st *Stream) ToList(key []byte, itr *Iterator) (*pb.KVList, error) { - alloc := st.Allocator(itr.ThreadId) - ka := alloc.Copy(key) + ka := y.Copy(key) list := &pb.KVList{} for ; itr.Valid(); itr.Next() { @@ -122,11 +110,11 @@ func (st *Stream) ToList(key []byte, itr *Iterator) (*pb.KVList, error) { break } - kv := y.NewKV(alloc) + kv := &pb.KV{} kv.Key = ka if err := item.Value(func(val []byte) error { - kv.Value = alloc.Copy(val) + kv.Value = y.Copy(val) return nil }); err != nil { @@ -134,7 +122,7 @@ func (st *Stream) ToList(key []byte, itr *Iterator) (*pb.KVList, error) { } kv.Version = item.Version() kv.ExpiresAt = item.ExpiresAt() - kv.UserMeta = alloc.Copy([]byte{item.UserMeta()}) + kv.UserMeta = y.Copy([]byte{item.UserMeta()}) list.Kv = append(list.Kv, kv) if st.db.opt.NumVersionsToKeep == 1 { @@ -180,18 +168,8 @@ func (st *Stream) produceRanges(ctx context.Context) { close(st.rangeCh) } -func (st *Stream) newAllocator(threadId int) *z.Allocator { - st.allocatorsMu.Lock() - a := z.NewAllocator(batchSize) - a.Tag = "Stream " + st.LogPrefix - st.allocators[threadId] = a - st.allocatorsMu.Unlock() - return a -} - // produceKVs picks up ranges from rangeCh, generates KV lists and sends them to kvChan. func (st *Stream) produceKVs(ctx context.Context, threadId int) error { - var size int var txn *Txn if st.readTs > 0 { txn = st.db.NewTransactionAt(st.readTs, false) @@ -201,10 +179,12 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { defer txn.Discard() // produceKVs is running iterate serially. So, we can define the outList here. - outList := new(pb.KVList) - // There would be one last remaining Allocator for this threadId when produceKVs finishes, which - // would be released by Orchestrate. - outList.AllocRef = st.newAllocator(threadId).Ref + outList := z.NewBuffer(2 * batchSize) + defer func() { + // The outList variable changes. So, we need to evaluate the variable in the defer. DO NOT + // call `defer outList.Release()`. + outList.Release() + }() iterate := func(kr keyRange) error { iterOpts := DefaultIteratorOptions @@ -221,12 +201,10 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { sendIt := func() error { select { case st.kvChan <- outList: + outList = z.NewBuffer(2 * batchSize) case <-ctx.Done(): return ctx.Err() } - outList = new(pb.KVList) - outList.AllocRef = st.newAllocator(threadId).Ref - size = 0 return nil } @@ -258,11 +236,9 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { continue } for _, kv := range list.Kv { - size += proto.Size(kv) kv.StreamId = streamId - outList.Kv = append(outList.Kv, kv) - - if size < batchSize { + KVToBuffer(kv, outList) + if outList.LenNoPadding() < batchSize { continue } if err := sendIt(); err != nil { @@ -272,10 +248,11 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { } // Mark the stream as done. if st.doneMarkers { - outList.Kv = append(outList.Kv, &pb.KV{ + kv := &pb.KV{ StreamId: streamId, StreamDone: true, - }) + } + KVToBuffer(kv, outList) } return sendIt() } @@ -297,45 +274,33 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { } func (st *Stream) streamKVs(ctx context.Context) error { - var count int var bytesSent uint64 t := time.NewTicker(time.Second) defer t.Stop() now := time.Now() - allocs := make(map[uint64]struct{}) - returnAllocs := func() { - for ref := range allocs { - a := z.AllocatorFrom(ref) - a.Release() - delete(allocs, ref) + sendBatch := func(batch *z.Buffer) error { + defer batch.Release() + sz := uint64(batch.LenNoPadding()) + if sz == 0 { + return nil } - } - defer returnAllocs() - - sendBatch := func(batch *pb.KVList) error { - sz := uint64(proto.Size(batch)) bytesSent += sz - count += len(batch.Kv) - t := time.Now() + st.db.opt.Infof("%s Sending batch of size: %s.\n", st.LogPrefix, humanize.Bytes(sz)) if err := st.Send(batch); err != nil { + st.db.opt.Warningf("Error while sending: %v\n", err) return err } - st.db.opt.Infof("%s Created batch of size: %s in %s.\n", - st.LogPrefix, humanize.Bytes(sz), time.Since(t)) - - returnAllocs() return nil } - slurp := func(batch *pb.KVList) error { + slurp := func(batch *z.Buffer) error { loop: for { // Send the batch immediately if it already exceeds the maximum allowed size. // If the size of the batch exceeds maxStreamSize, break from the loop to // avoid creating a batch that is so big that certain limits are reached. - sz := uint64(proto.Size(batch)) - if sz > maxStreamSize { + if batch.LenNoPadding() > int(maxStreamSize) { break loop } select { @@ -344,8 +309,8 @@ func (st *Stream) streamKVs(ctx context.Context) error { break loop } y.AssertTrue(kvs != nil) - batch.Kv = append(batch.Kv, kvs.Kv...) - allocs[kvs.AllocRef] = struct{}{} + y.Check2(batch.Write(kvs.Bytes())) + y.Check(kvs.Release()) default: break loop @@ -356,7 +321,7 @@ func (st *Stream) streamKVs(ctx context.Context) error { outer: for { - var batch *pb.KVList + var batch *z.Buffer select { case <-ctx.Done(): return ctx.Err() @@ -368,9 +333,12 @@ outer: continue } speed := bytesSent / durSec + + var ms z.MemStats + z.ReadMemStats(&ms) st.db.opt.Infof("%s Time elapsed: %s, bytes sent: %s, speed: %s/sec, jemalloc: %s\n", st.LogPrefix, y.FixedDuration(dur), humanize.IBytes(bytesSent), - humanize.IBytes(speed), humanize.IBytes(uint64(z.NumAllocBytes()))) + humanize.IBytes(speed), humanize.IBytes(ms.Active)) case kvs, ok := <-st.kvChan: if !ok { @@ -378,7 +346,6 @@ outer: } y.AssertTrue(kvs != nil) batch = kvs - allocs[kvs.AllocRef] = struct{}{} // Otherwise, slurp more keys into this batch. if err := slurp(batch); err != nil { @@ -387,7 +354,7 @@ outer: } } - st.db.opt.Infof("%s Sent %d keys\n", st.LogPrefix, count) + st.db.opt.Infof("%s Sent data of size %s\n", st.LogPrefix, humanize.IBytes(bytesSent)) return nil } @@ -400,21 +367,12 @@ outer: func (st *Stream) Orchestrate(ctx context.Context) error { ctx, cancel := context.WithCancel(ctx) defer cancel() - defer func() { - for _, a := range st.allocators { - // Using AllocatorFrom is better because if the allocator is already freed up, it would - // return nil. - a = z.AllocatorFrom(a.Ref) - a.Release() - } - }() - st.rangeCh = make(chan keyRange, 3) // Contains keys for posting lists. // kvChan should only have a small capacity to ensure that we don't buffer up too much data if // sending is slow. Page size is set to 4MB, which is used to lazily cap the size of each // KVList. To get 128MB buffer, we can set the channel size to 32. - st.kvChan = make(chan *pb.KVList, 32) + st.kvChan = make(chan *z.Buffer, 32) if st.KeyToList == nil { st.KeyToList = st.ToList @@ -452,6 +410,12 @@ func (st *Stream) Orchestrate(ctx context.Context) error { }() wg.Wait() // Wait for produceKVs to be over. close(st.kvChan) // Now we can close kvChan. + defer func() { + // If due to some error, we have buffers left in kvChan, we should release them. + for buf := range st.kvChan { + buf.Release() + } + }() select { case err := <-errCh: // Check error from produceKVs. @@ -466,10 +430,9 @@ func (st *Stream) Orchestrate(ctx context.Context) error { func (db *DB) newStream() *Stream { return &Stream{ - db: db, - NumGo: 8, - LogPrefix: "Badger.Stream", - allocators: make(map[int]*z.Allocator), + db: db, + NumGo: 8, + LogPrefix: "Badger.Stream", } } @@ -490,3 +453,21 @@ func (db *DB) NewStreamAt(readTs uint64) *Stream { stream.readTs = readTs return stream } + +func BufferToKVList(buf *z.Buffer) (*pb.KVList, error) { + var list pb.KVList + err := buf.SliceIterate(func(s []byte) error { + kv := new(pb.KV) + if err := kv.Unmarshal(s); err != nil { + return err + } + list.Kv = append(list.Kv, kv) + return nil + }) + return &list, err +} + +func KVToBuffer(kv *pb.KV, buf *z.Buffer) { + out := buf.SliceAllocate(kv.Size()) + y.Check2(kv.MarshalToSizedBuffer(out)) +} diff --git a/stream_test.go b/stream_test.go index 536966008..35787c79c 100644 --- a/stream_test.go +++ b/stream_test.go @@ -28,6 +28,7 @@ import ( "github.com/dgraph-io/badger/v2/pb" bpb "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" + "github.com/dgraph-io/ristretto/z" "github.com/golang/protobuf/proto" "github.com/stretchr/testify/require" ) @@ -51,15 +52,19 @@ type collector struct { kv []*bpb.KV } -func (c *collector) Send(list *bpb.KVList) error { +func (c *collector) Send(buf *z.Buffer) error { + list, err := BufferToKVList(buf) + if err != nil { + return err + } for _, kv := range list.Kv { if kv.StreamDone == true { - continue + return nil } cp := proto.Clone(kv).(*bpb.KV) c.kv = append(c.kv, cp) } - return nil + return err } var ctxb = context.Background() @@ -304,7 +309,9 @@ func TestStreamCustomKeyToList(t *testing.T) { }, nil } res := map[string]struct{}{"p0": {}, "p1": {}, "p2": {}} - stream.Send = func(list *pb.KVList) error { + stream.Send = func(buf *z.Buffer) error { + list, err := BufferToKVList(buf) + require.NoError(t, err) for _, kv := range list.Kv { key := string(kv.Key) if _, ok := res[key]; !ok { diff --git a/stream_writer.go b/stream_writer.go index 1b1ef17f8..6302d897f 100644 --- a/stream_writer.go +++ b/stream_writer.go @@ -83,8 +83,8 @@ func (sw *StreamWriter) Prepare() error { // Write writes KVList to DB. Each KV within the list contains the stream id which StreamWriter // would use to demux the writes. Write is thread safe and can be called concurrently by multiple // goroutines. -func (sw *StreamWriter) Write(kvs *pb.KVList) error { - if len(kvs.GetKv()) == 0 { +func (sw *StreamWriter) Write(buf *z.Buffer) error { + if buf.LenNoPadding() == 0 { return nil } @@ -93,10 +93,15 @@ func (sw *StreamWriter) Write(kvs *pb.KVList) error { // the valid kvs. closedStreams := make(map[uint32]struct{}) streamReqs := make(map[uint32]*request) - for _, kv := range kvs.Kv { + + err := buf.SliceIterate(func(s []byte) error { + var kv pb.KV + if err := kv.Unmarshal(s); err != nil { + return err + } if kv.StreamDone { closedStreams[kv.StreamId] = struct{}{} - continue + return nil } // Panic if some kv comes after stream has been marked as closed. @@ -129,7 +134,12 @@ func (sw *StreamWriter) Write(kvs *pb.KVList) error { streamReqs[kv.StreamId] = req } req.Entries = append(req.Entries, e) + return nil + }) + if err != nil { + return err } + all := make([]*request, 0, len(streamReqs)) for _, req := range streamReqs { all = append(all, req) diff --git a/stream_writer_test.go b/stream_writer_test.go index 3cf7990e6..02859ee8a 100644 --- a/stream_writer_test.go +++ b/stream_writer_test.go @@ -30,23 +30,24 @@ import ( "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" + "github.com/dgraph-io/ristretto/z" ) -func getSortedKVList(valueSize, listSize int) *pb.KVList { +func getSortedKVList(valueSize, listSize int) *z.Buffer { value := make([]byte, valueSize) y.Check2(rand.Read(value)) - list := &pb.KVList{} + buf := z.NewBuffer(10 << 20) for i := 0; i < listSize; i++ { key := make([]byte, 8) binary.BigEndian.PutUint64(key, uint64(i)) - list.Kv = append(list.Kv, &pb.KV{ + KVToBuffer(&pb.KV{ Key: key, Value: value, Version: 20, - }) + }, buf) } - return list + return buf } // check if we can read values after writing using stream writer @@ -54,7 +55,7 @@ func TestStreamWriter1(t *testing.T) { test := func(t *testing.T, opts *Options) { runBadgerTest(t, opts, func(t *testing.T, db *DB) { // write entries using stream writer - noOfKeys := 1000 + noOfKeys := 10 valueSize := 128 list := getSortedKVList(valueSize, noOfKeys) sw := db.NewStreamWriter() @@ -173,22 +174,23 @@ func TestStreamWriter3(t *testing.T) { // insert keys which are even value := make([]byte, valueSize) y.Check2(rand.Read(value)) - list := &pb.KVList{} counter := 0 + buf := z.NewBuffer(10 << 20) + defer buf.Release() for i := 0; i < noOfKeys; i++ { key := make([]byte, 8) binary.BigEndian.PutUint64(key, uint64(counter)) - list.Kv = append(list.Kv, &pb.KV{ + KVToBuffer(&pb.KV{ Key: key, Value: value, Version: 20, - }) + }, buf) counter = counter + 2 } sw := db.NewStreamWriter() require.NoError(t, sw.Prepare(), "sw.Prepare() failed") - require.NoError(t, sw.Write(list), "sw.Write() failed") + require.NoError(t, sw.Write(buf), "sw.Write() failed") // get max version of sw, will be used in transactions for managed mode maxVs := sw.maxVersion require.NoError(t, sw.Flush(), "sw.Flush() failed") @@ -270,24 +272,23 @@ func TestStreamWriter4(t *testing.T) { require.NoError(t, err, "error while updating db") } - list := &pb.KVList{} - list.Kv = append(list.Kv, &pb.KV{ + buf := z.NewBuffer(10 << 20) + defer buf.Release() + KVToBuffer(&pb.KV{ Key: []byte("key-1"), Value: []byte("value-1"), Version: 1, - }) + }, buf) sw := db.NewStreamWriter() require.NoError(t, sw.Prepare(), "sw.Prepare() failed") - require.NoError(t, sw.Write(list), "sw.Write() failed") + require.NoError(t, sw.Write(buf), "sw.Write() failed") require.NoError(t, sw.Flush(), "sw.Flush() failed") }) } func TestStreamWriter5(t *testing.T) { runBadgerTest(t, nil, func(t *testing.T, db *DB) { - list := &pb.KVList{} - left := make([]byte, 6) left[0] = 0x00 copy(left[1:], []byte("break")) @@ -296,20 +297,22 @@ func TestStreamWriter5(t *testing.T) { right[0] = 0xff copy(right[1:], []byte("break")) - list.Kv = append(list.Kv, &pb.KV{ + buf := z.NewBuffer(10 << 20) + defer buf.Release() + KVToBuffer(&pb.KV{ Key: left, Value: []byte("val"), Version: 1, - }) - list.Kv = append(list.Kv, &pb.KV{ + }, buf) + KVToBuffer(&pb.KV{ Key: right, Value: []byte("val"), Version: 1, - }) + }, buf) sw := db.NewStreamWriter() require.NoError(t, sw.Prepare(), "sw.Prepare() failed") - require.NoError(t, sw.Write(list), "sw.Write() failed") + require.NoError(t, sw.Write(buf), "sw.Write() failed") require.NoError(t, sw.Flush(), "sw.Flush() failed") require.NoError(t, db.Close()) @@ -325,7 +328,6 @@ func TestStreamWriter5(t *testing.T) { func TestStreamWriter6(t *testing.T) { runBadgerTest(t, nil, func(t *testing.T, db *DB) { - list := &pb.KVList{} str := []string{"a", "b", "c"} ver := uint64(0) // The baseTable size is 32 KB (1<<15) and the max table size for level @@ -333,6 +335,8 @@ func TestStreamWriter6(t *testing.T) { // will be written to level 6, we need to insert at least 1 mb of data. // Setting keycount below 32 would cause this test to fail. keyCount := 40 + buf := z.NewBuffer(10 << 20) + defer buf.Release() for i := range str { for j := 0; j < keyCount; j++ { ver++ @@ -341,7 +345,7 @@ func TestStreamWriter6(t *testing.T) { Value: []byte("val"), Version: uint64(keyCount - j), } - list.Kv = append(list.Kv, kv) + KVToBuffer(kv, buf) } } @@ -349,7 +353,7 @@ func TestStreamWriter6(t *testing.T) { // we would have 6 tables, if keys are not equal. Here we should have 3 tables. sw := db.NewStreamWriter() require.NoError(t, sw.Prepare(), "sw.Prepare() failed") - require.NoError(t, sw.Write(list), "sw.Write() failed") + require.NoError(t, sw.Write(buf), "sw.Write() failed") require.NoError(t, sw.Flush(), "sw.Flush() failed") tables := db.Tables() @@ -368,22 +372,23 @@ func TestStreamWriter6(t *testing.T) { // This test uses a StreamWriter without calling Flush() at the end. func TestStreamWriterCancel(t *testing.T) { runBadgerTest(t, nil, func(t *testing.T, db *DB) { - list := &pb.KVList{} str := []string{"a", "a", "b", "b", "c", "c"} ver := 1 + buf := z.NewBuffer(10 << 20) + defer buf.Release() for i := range str { kv := &pb.KV{ Key: bytes.Repeat([]byte(str[i]), int(db.opt.BaseTableSize)), Value: []byte("val"), Version: uint64(ver), } - list.Kv = append(list.Kv, kv) + KVToBuffer(kv, buf) ver = (ver + 1) % 2 } sw := db.NewStreamWriter() require.NoError(t, sw.Prepare(), "sw.Prepare() failed") - require.NoError(t, sw.Write(list), "sw.Write() failed") + require.NoError(t, sw.Write(buf), "sw.Write() failed") sw.Cancel() // Use the API incorrectly. @@ -403,7 +408,8 @@ func TestStreamDone(t *testing.T) { var val [10]byte rand.Read(val[:]) for i := 0; i < 10; i++ { - list := &pb.KVList{} + buf := z.NewBuffer(10 << 20) + defer buf.Release() kv1 := &pb.KV{ Key: []byte(fmt.Sprintf("%d", i)), Value: val[:], @@ -414,8 +420,9 @@ func TestStreamDone(t *testing.T) { StreamId: uint32(i), StreamDone: true, } - list.Kv = append(list.Kv, kv1, kv2) - require.NoError(t, sw.Write(list), "sw.Write() failed") + KVToBuffer(kv1, buf) + KVToBuffer(kv2, buf) + require.NoError(t, sw.Write(buf), "sw.Write() failed") } require.NoError(t, sw.Flush(), "sw.Flush() failed") require.NoError(t, db.Close()) @@ -442,7 +449,8 @@ func TestSendOnClosedStream(t *testing.T) { var val [10]byte rand.Read(val[:]) - list := &pb.KVList{} + buf := z.NewBuffer(10 << 20) + defer buf.Release() kv1 := &pb.KV{ Key: []byte(fmt.Sprintf("%d", 1)), Value: val[:], @@ -453,8 +461,9 @@ func TestSendOnClosedStream(t *testing.T) { StreamId: uint32(1), StreamDone: true, } - list.Kv = append(list.Kv, kv1, kv2) - require.NoError(t, sw.Write(list), "sw.Write() failed") + KVToBuffer(kv1, buf) + KVToBuffer(kv2, buf) + require.NoError(t, sw.Write(buf), "sw.Write() failed") // Defer for panic. defer func() { @@ -463,15 +472,16 @@ func TestSendOnClosedStream(t *testing.T) { require.NoError(t, db.Close()) }() // Send once stream is closed. - list = &pb.KVList{} + buf1 := z.NewBuffer(10 << 20) + defer buf1.Release() kv1 = &pb.KV{ Key: []byte(fmt.Sprintf("%d", 2)), Value: val[:], Version: 1, StreamId: uint32(1), } - list.Kv = append(list.Kv, kv1) - sw.Write(list) + KVToBuffer(kv1, buf1) + sw.Write(buf1) } func TestSendOnClosedStream2(t *testing.T) { @@ -489,7 +499,8 @@ func TestSendOnClosedStream2(t *testing.T) { var val [10]byte rand.Read(val[:]) - list := &pb.KVList{} + buf := z.NewBuffer(10 << 20) + defer buf.Release() kv1 := &pb.KV{ Key: []byte(fmt.Sprintf("%d", 1)), Value: val[:], @@ -506,7 +517,9 @@ func TestSendOnClosedStream2(t *testing.T) { Version: 1, StreamId: uint32(1), } - list.Kv = append(list.Kv, kv1, kv2, kv3) + KVToBuffer(kv1, buf) + KVToBuffer(kv2, buf) + KVToBuffer(kv3, buf) // Defer for panic. defer func() { @@ -515,7 +528,7 @@ func TestSendOnClosedStream2(t *testing.T) { require.NoError(t, db.Close()) }() - require.NoError(t, sw.Write(list), "sw.Write() failed") + require.NoError(t, sw.Write(buf), "sw.Write() failed") } func TestStreamWriterEncrypted(t *testing.T) { @@ -533,16 +546,17 @@ func TestStreamWriterEncrypted(t *testing.T) { key := []byte("mykey") value := []byte("myvalue") - list := &pb.KVList{} - list.Kv = append(list.Kv, &pb.KV{ + buf := z.NewBuffer(10 << 20) + defer buf.Release() + KVToBuffer(&pb.KV{ Key: key, Value: value, Version: 20, - }) + }, buf) sw := db.NewStreamWriter() require.NoError(t, sw.Prepare(), "Prepare failed") - require.NoError(t, sw.Write(list), "Write failed") + require.NoError(t, sw.Write(buf), "Write failed") require.NoError(t, sw.Flush(), "Flush failed") err = db.View(func(txn *Txn) error { diff --git a/table/builder.go b/table/builder.go index 39bb166c8..5f400385f 100644 --- a/table/builder.go +++ b/table/builder.go @@ -127,7 +127,7 @@ func NewTableBuilder(opts Options) *Builder { sz = maxAllocatorInitialSz } b := &Builder{ - alloc: z.NewAllocator(sz), + alloc: opts.AllocPool.Get(sz), opts: &opts, } b.alloc.Tag = "Builder" @@ -186,7 +186,7 @@ func (b *Builder) handleBlock() { // Close closes the TableBuilder. func (b *Builder) Close() { - b.alloc.Release() + b.opts.AllocPool.Return(b.alloc) } // Empty returns whether it's empty. diff --git a/table/table.go b/table/table.go index 17cc31f58..0ff60c22b 100644 --- a/table/table.go +++ b/table/table.go @@ -80,6 +80,8 @@ type Options struct { BlockCache *ristretto.Cache IndexCache *ristretto.Cache + AllocPool *z.AllocatorPool + // ZSTDCompressionLevel is the ZSTD compression level used for compressing blocks. ZSTDCompressionLevel int } diff --git a/test.sh b/test.sh index 9b4b94698..e4f688afe 100755 --- a/test.sh +++ b/test.sh @@ -1,6 +1,6 @@ #!/bin/bash -set -e +set -eo pipefail go version @@ -96,13 +96,16 @@ root() { } stream() { + set -eo pipefail pushd badger baseDir=$(mktemp -d -p .) ./badger benchmark write -s --dir=$baseDir/test | tee $baseDir/log.txt + ./badger benchmark read --dir=$baseDir/test --full-scan | tee --append $baseDir/log.txt + ./badger benchmark read --dir=$baseDir/test -d=30s | tee --append $baseDir/log.txt ./badger stream --dir=$baseDir/test -o "$baseDir/test2" | tee --append $baseDir/log.txt count=$(cat "$baseDir/log.txt" | grep "at program end: 0 B" | wc -l) rm -rf $baseDir - if [ $count -ne 2 ]; then + if [ $count -ne 4 ]; then echo "LEAK detected in Badger stream." return 1 fi From 3d225d79b515ba0bcd81c929dbb6822f03742526 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 25 Nov 2020 18:19:09 -0800 Subject: [PATCH 14/35] Bug Fix: Create the right txn based on Badger mode --- iterator.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/iterator.go b/iterator.go index db0578ace..8f086c608 100644 --- a/iterator.go +++ b/iterator.go @@ -20,6 +20,7 @@ import ( "bytes" "fmt" "hash/crc32" + "math" "sort" "sync" "sync/atomic" @@ -170,7 +171,12 @@ func (item *Item) yieldItemValue() ([]byte, func(), error) { if err != nil { db.opt.Logger.Errorf("Unable to read: Key: %v, Version : %v, meta: %v, userMeta: %v"+ " Error: %v", key, item.version, item.meta, item.userMeta, err) - txn := db.NewTransaction(false) + var txn *Txn + if db.opt.managedTxns { + txn = db.NewTransactionAt(math.MaxUint64, false) + } else { + txn = db.NewTransaction(false) + } defer txn.Discard() iopt := DefaultIteratorOptions From 2d88aea98099cc021fe39a783629759c90d9e139 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dami=C3=A1n=20Parrino?= Date: Mon, 30 Nov 2020 08:58:37 -0300 Subject: [PATCH 15/35] Remove link from index.md (#1607) Remove `Github issues` links --- docs/content/resources/index.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/content/resources/index.md b/docs/content/resources/index.md index 45e2dba4f..e90d21026 100644 --- a/docs/content/resources/index.md +++ b/docs/content/resources/index.md @@ -12,10 +12,9 @@ Go](https://open.dgraph.io/post/badger/) 4. [Concurrent ACID Transactions in Badger](https://blog.dgraph.io/post/badger-txn/) ## Contact -- Please use [discuss.dgraph.io](https://discuss.dgraph.io) for questions, feature requests and discussions. -- Please use [Github issue tracker](https://github.com/dgraph-io/badger/issues) for filing bugs or feature requests. +- Please use [discuss.dgraph.io](https://discuss.dgraph.io) for questions, bugs, feature requests, and discussions. - Follow us on Twitter [@dgraphlabs](https://twitter.com/dgraphlabs). ## Contributing -If you're interested in contributing to Badger see [CONTRIBUTING.md](https://github.com/dgraph-io/badger/blob/master/CONTRIBUTING.md). \ No newline at end of file +If you're interested in contributing to Badger see [CONTRIBUTING.md](https://github.com/dgraph-io/badger/blob/master/CONTRIBUTING.md). From 6e7f078d11c2a1e209b32726396fefa29de18259 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 2 Dec 2020 18:48:06 -0800 Subject: [PATCH 16/35] Keep the cheaper parts of the index within table struct. (#1608) Even for some integer fields like MaxVersion, KeyCount, OffsetsLength, we end up calling `fetchIndex`, which hits Ristretto cache. Instead, we could just keep these in memory always, because they're so cheap. This PR does that. --- table/table.go | 76 +++++++++++++++++++++++++------------------------- 1 file changed, 38 insertions(+), 38 deletions(-) diff --git a/table/table.go b/table/table.go index 0ff60c22b..e97126d13 100644 --- a/table/table.go +++ b/table/table.go @@ -101,7 +101,8 @@ type Table struct { tableSize int // Initialized in OpenTable, using fd.Stat(). _index *fb.TableIndex // Nil if encryption is enabled. Use fetchIndex to access. - ref int32 // For file garbage collection. Atomic. + _cheap *cheapIndex + ref int32 // For file garbage collection. Atomic. // The following are initialized once and const. smallest, biggest []byte // Smallest and largest keys (with timestamps). @@ -119,10 +120,35 @@ type Table struct { opt *Options } -// MaxVersion returns the maximum version across all keys stored in this table. -func (t *Table) MaxVersion() uint64 { - return t.fetchIndex().MaxVersion() +type cheapIndex struct { + MaxVersion uint64 + KeyCount uint32 + UncompressedSize uint32 + OnDiskSize uint32 + BloomFilterLength int + OffsetsLength int +} + +func (t *Table) cheapIndex() *cheapIndex { + return t._cheap } +func (t *Table) offsetsLength() int { return t.cheapIndex().OffsetsLength } + +// MaxVersion returns the maximum version across all keys stored in this table. +func (t *Table) MaxVersion() uint64 { return t.cheapIndex().MaxVersion } + +// BloomFilterSize returns the size of the bloom filter in bytes stored in memory. +func (t *Table) BloomFilterSize() int { return t.cheapIndex().BloomFilterLength } + +// UncompressedSize is the size uncompressed data stored in this file. +func (t *Table) UncompressedSize() uint32 { return t.cheapIndex().UncompressedSize } + +// KeyCount is the total number of keys in this table. +func (t *Table) KeyCount() uint32 { return t.cheapIndex().KeyCount } + +// OnDiskSize returns the total size of key-values stored in this table (including the +// disk space occupied on the value log). +func (t *Table) OnDiskSize() uint32 { return t.cheapIndex().OnDiskSize } // CompressionType returns the compression algorithm used for block compression. func (t *Table) CompressionType() options.CompressionType { @@ -391,8 +417,15 @@ func (t *Table) initIndex() (*fb.BlockOffset, error) { // If there's no encryption, this points to the mmap'ed buffer. t._index = index } + t._cheap = &cheapIndex{ + MaxVersion: index.MaxVersion(), + KeyCount: index.KeyCount(), + UncompressedSize: index.UncompressedSize(), + OnDiskSize: index.OnDiskSize(), + OffsetsLength: index.OffsetsLength(), + BloomFilterLength: index.BloomFilterLength(), + } - t.onDiskSize = index.OnDiskSize() t.hasBloomFilter = len(index.BloomFilterBytes()) > 0 var bo fb.BlockOffset @@ -444,10 +477,6 @@ func (t *Table) fetchIndex() *fb.TableIndex { return index } -func (t *Table) offsetsLength() int { - return t.fetchIndex().OffsetsLength() -} - func (t *Table) offsets(ko *fb.BlockOffset, i int) bool { return t.fetchIndex().Offsets(ko, i) } @@ -574,30 +603,11 @@ func (t *Table) indexKey() uint64 { return t.id } -// UncompressedSize is the size uncompressed data stored in this file. -func (t *Table) UncompressedSize() uint32 { - return t.fetchIndex().UncompressedSize() -} - -// KeyCount is the total number of keys in this table. -func (t *Table) KeyCount() uint32 { - return t.fetchIndex().KeyCount() -} - // IndexSize is the size of table index in bytes. func (t *Table) IndexSize() int { return t.indexLen } -// BloomFilterSize returns the size of the bloom filter in bytes stored in memory. -func (t *Table) BloomFilterSize() int { - return t.fetchIndex().BloomFilterLength() -} - -// OnDiskSize returns the total size of key-values stored in this table (including the -// disk space occupied on the value log). -func (t *Table) OnDiskSize() uint32 { return t.onDiskSize } - // Size is its file size in bytes func (t *Table) Size() int64 { return int64(t.tableSize) } @@ -625,16 +635,6 @@ func (t *Table) DoesNotHave(hash uint32) bool { return !y.Filter(bf).MayContain(hash) } -// readBloomFilter reads the bloom filter from the SST and returns its length -// along with the bloom filter. -func (t *Table) readBloomFilter() (*z.Bloom, int) { - index := t.fetchIndex() - // Read bloom filter from the index. - bf, err := z.JSONUnmarshal(index.BloomFilterBytes()) - y.Check(err) - return bf, index.BloomFilterLength() -} - // readTableIndex reads table index from the sst and returns its pb format. func (t *Table) readTableIndex() (*fb.TableIndex, error) { data := t.readNoFail(t.indexStart, t.indexLen) From 70088c6e3493e5ffc1c21c6908e2bebb33da9dc3 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 2 Dec 2020 19:30:43 -0800 Subject: [PATCH 17/35] Fix(OOM): Reuse pb.KVs in Stream (#1609) We store a slice of pb.KVs in Iterator, so it can be used by Stream users. --- backup.go | 3 ++- iterator.go | 17 +++++++++++++++++ stream.go | 5 ++++- stream_test.go | 3 ++- 4 files changed, 25 insertions(+), 3 deletions(-) diff --git a/backup.go b/backup.go index c729a38b9..ee3d6d7ef 100644 --- a/backup.go +++ b/backup.go @@ -87,7 +87,8 @@ func (stream *Stream) Backup(w io.Writer, since uint64) (uint64, error) { // clear txn bits meta := item.meta &^ (bitTxn | bitFinTxn) - kv := &pb.KV{ + kv := itr.NewKV() + *kv = pb.KV{ Key: item.KeyCopy(nil), Value: valCopy, UserMeta: []byte{item.UserMeta()}, diff --git a/iterator.go b/iterator.go index 8f086c608..f176fa3e4 100644 --- a/iterator.go +++ b/iterator.go @@ -26,6 +26,7 @@ import ( "sync/atomic" "time" + "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/table" "github.com/dgraph-io/badger/v2/y" @@ -421,6 +422,8 @@ type Iterator struct { // the iterator. It can be used, for example, to uniquely identify each of the // iterators created by the stream interface ThreadId int + + reuse []*pb.KV } // NewIterator returns a new iterator. Depending upon the options, either only keys, or both @@ -480,6 +483,20 @@ func (txn *Txn) NewKeyIterator(key []byte, opt IteratorOptions) *Iterator { return txn.NewIterator(opt) } +// NewKV must be called serially. It is NOT thread-safe. +func (it *Iterator) NewKV() *pb.KV { + if len(it.reuse) == 0 { + return &pb.KV{} + } + kv := it.reuse[len(it.reuse)-1] + it.reuse = it.reuse[:len(it.reuse)-1] + if kv == nil { + kv = &pb.KV{} + } + kv.Reset() + return kv +} + func (it *Iterator) newItem() *Item { item := it.waste.pop() if item == nil { diff --git a/stream.go b/stream.go index 2bb5337a2..8733fe15e 100644 --- a/stream.go +++ b/stream.go @@ -110,7 +110,7 @@ func (st *Stream) ToList(key []byte, itr *Iterator) (*pb.KVList, error) { break } - kv := &pb.KV{} + kv := itr.NewKV() kv.Key = ka if err := item.Value(func(val []byte) error { @@ -245,6 +245,9 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { return err } } + if len(itr.reuse) < 100 { + itr.reuse = append(itr.reuse, list.Kv...) + } } // Mark the stream as done. if st.doneMarkers { diff --git a/stream_test.go b/stream_test.go index 35787c79c..1bc7b97b9 100644 --- a/stream_test.go +++ b/stream_test.go @@ -300,7 +300,8 @@ func TestStreamCustomKeyToList(t *testing.T) { if err != nil { return nil, err } - kv := &pb.KV{ + kv := itr.NewKV() + *kv = pb.KV{ Key: y.Copy(item.Key()), Value: val, } From 74f2e0277ca4830953fa3e43e7cd64bf1bc785b6 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Thu, 3 Dec 2020 15:30:11 -0800 Subject: [PATCH 18/35] Opt(pickTables): Fix an optimization regression NewKeyIterator uses pickTables which was optimized in the past. But, a recent PR: https://github.com/dgraph-io/badger/pull/1546 removed this optimization, which is now making NewKeyIterator quite expensive. This PR brings that optimization back. --- iterator.go | 19 ++++++++++++++++--- table/table.go | 7 ++++++- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/iterator.go b/iterator.go index f176fa3e4..d6af3d854 100644 --- a/iterator.go +++ b/iterator.go @@ -354,7 +354,6 @@ func (opt *IteratorOptions) pickTable(t table.TableInterface) bool { // Bloom filter lookup would only work if opt.Prefix does NOT have the read // timestamp as part of the key. if opt.prefixIsKey && t.DoesNotHave(y.Hash(opt.Prefix)) { - y.NumLSMBloomHits.Add("pickTable", 1) return false } return true @@ -369,6 +368,8 @@ func (opt *IteratorOptions) pickTables(all []*table.Table) []*table.Table { return out } sIdx := sort.Search(len(all), func(i int) bool { + // table.Biggest >= opt.prefix + // if opt.Prefix < table.Biggest, then surely it is not in any of the preceding tables. return opt.compareToPrefix(all[i].Biggest()) >= 0 }) if sIdx == len(all) { @@ -386,11 +387,23 @@ func (opt *IteratorOptions) pickTables(all []*table.Table) []*table.Table { return out } + // opt.prefixIsKey == true. This code is optimizing for opt.prefixIsKey part. var out []*table.Table + hash := y.Hash(opt.Prefix) for _, t := range filtered { - if opt.pickTable(t) { - out = append(out, t) + // When we encounter the first table whose smallest key is higher than opt.Prefix, we can + // stop. This is an IMPORTANT optimization, just considering how often we call + // NewKeyIterator. + if opt.compareToPrefix(t.Smallest()) > 0 { + // if table.Smallest > opt.Prefix, then this and all tables after this can be ignored. + break + } + // opt.Prefix is actually the key. So, we can run bloom filter checks + // as well. + if t.DoesNotHave(hash) { + continue } + out = append(out, t) } return out } diff --git a/table/table.go b/table/table.go index e97126d13..f596d6da5 100644 --- a/table/table.go +++ b/table/table.go @@ -630,9 +630,14 @@ func (t *Table) DoesNotHave(hash uint32) bool { return false } + y.NumLSMBloomHits.Add("DoesNotHave_ALL", 1) index := t.fetchIndex() bf := index.BloomFilterBytes() - return !y.Filter(bf).MayContain(hash) + mayContain := y.Filter(bf).MayContain(hash) + if !mayContain { + y.NumLSMBloomHits.Add("DoesNotHave_HIT", 1) + } + return !mayContain } // readTableIndex reads table index from the sst and returns its pb format. From 3782d885da9640d61ee705781e84f3cff761ff55 Mon Sep 17 00:00:00 2001 From: Daniel Mai Date: Thu, 3 Dec 2020 21:48:55 -0800 Subject: [PATCH 19/35] chore(cmd/stream): Default to snappy for compression flag. (#1610) The DefaultOptions already has snappy set to the default, so the stream CLI tool aligns with that now. --- badger/cmd/stream.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/badger/cmd/stream.go b/badger/cmd/stream.go index da6ee875a..a7d3ba59c 100644 --- a/badger/cmd/stream.go +++ b/badger/cmd/stream.go @@ -51,7 +51,7 @@ func init() { streamCmd.Flags().IntVarP(&numVersions, "num_versions", "", 0, "Option to configure the maximum number of versions per key. "+ "Values <= 0 will be considered to have the max number of versions.") - streamCmd.Flags().Uint32VarP(&compressionType, "compression", "", 0, + streamCmd.Flags().Uint32VarP(&compressionType, "compression", "", 1, "Option to configure the compression type in output DB. "+ "0 to disable, 1 for Snappy, and 2 for ZSTD.") streamCmd.Flags().StringVarP(&keyPath, "encryption-key-file", "e", "", From 0f77c00a34d49d7e4b2e417c5f0899c6341712bb Mon Sep 17 00:00:00 2001 From: Daniel Mai Date: Thu, 3 Dec 2020 21:52:29 -0800 Subject: [PATCH 20/35] chore(cmd/info): Fix printed spacing of summary. (#1559) * chore(cmd/info): Fix printed spacing of summary. Column-aligns the output for the Summary section: Before: [Summary] Level 0 size: 0 B Level 1 size: 2.3 kB Total SST size: 2.3 kB Value log size: 20 B After: [Summary] Level 0 size: 0 B Level 1 size: 2.3 kB Total SST size: 2.3 kB Value log size: 20 B * fix: Set block cache and index cache sizes. This fixes a panic when running badger info panic: BlockCacheSize should be set since compression/encryption are enabled --- badger/cmd/info.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/badger/cmd/info.go b/badger/cmd/info.go index ec4086e71..f7c400235 100644 --- a/badger/cmd/info.go +++ b/badger/cmd/info.go @@ -99,6 +99,8 @@ func handleInfo(cmd *cobra.Command, args []string) error { db, err := badger.Open(badger.DefaultOptions(sstDir). WithValueDir(vlogDir). WithReadOnly(opt.readOnly). + WithBlockCacheSize(100 << 20). + WithIndexCacheSize(200 << 20). WithEncryptionKey([]byte(opt.encryptionKey))) if err != nil { return y.Wrap(err, "failed to open database") @@ -434,7 +436,7 @@ func printInfo(dir, valueDir string) error { totalSSTSize += sz } - fmt.Printf("Total SST size: %8s\n", hbytes(totalSSTSize)) + fmt.Printf("Total SST size: %10s\n", hbytes(totalSSTSize)) fmt.Printf("Value log size: %10s\n", hbytes(valueLogSize)) fmt.Println() totalExtra := numExtra + numValueDirExtra From a75cb55b76c946de4248ce61b52e2608a93908cf Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Fri, 4 Dec 2020 12:59:40 -0800 Subject: [PATCH 21/35] Fix(OOM): Use z.Allocator for Stream (#1611) * Let's use allocator again * Switch to z.NumAllocBytes * Make stream work with both another Badger DB or with a file to backup. * Add a test for Allocator * Use allocator for Backup * Bring in latest Ristretto Co-authored-by: Daniel Mai --- backup.go | 14 +++++++---- badger/cmd/stream.go | 57 +++++++++++++++++++++++++++----------------- go.mod | 2 +- go.sum | 2 ++ iterator.go | 18 ++------------ stream.go | 21 ++++++++-------- stream_test.go | 3 +-- y/y_test.go | 28 ++++++++++++++++++++++ 8 files changed, 89 insertions(+), 56 deletions(-) diff --git a/backup.go b/backup.go index ee3d6d7ef..574a9dfce 100644 --- a/backup.go +++ b/backup.go @@ -62,6 +62,7 @@ func (db *DB) Backup(w io.Writer, since uint64) (uint64, error) { func (stream *Stream) Backup(w io.Writer, since uint64) (uint64, error) { stream.KeyToList = func(key []byte, itr *Iterator) (*pb.KVList, error) { list := &pb.KVList{} + a := itr.alloc for ; itr.Valid(); itr.Next() { item := itr.Item() if !bytes.Equal(item.Key(), key) { @@ -77,7 +78,10 @@ func (stream *Stream) Backup(w io.Writer, since uint64) (uint64, error) { if !item.IsDeletedOrExpired() { // No need to copy value, if item is deleted or expired. var err error - valCopy, err = item.ValueCopy(nil) + err = item.Value(func(val []byte) error { + valCopy = a.Copy(val) + return nil + }) if err != nil { stream.db.opt.Errorf("Key [%x, %d]. Error while fetching value [%v]\n", item.Key(), item.Version(), err) @@ -87,14 +91,14 @@ func (stream *Stream) Backup(w io.Writer, since uint64) (uint64, error) { // clear txn bits meta := item.meta &^ (bitTxn | bitFinTxn) - kv := itr.NewKV() + kv := y.NewKV(a) *kv = pb.KV{ - Key: item.KeyCopy(nil), + Key: a.Copy(item.Key()), Value: valCopy, - UserMeta: []byte{item.UserMeta()}, + UserMeta: a.Copy([]byte{item.UserMeta()}), Version: item.Version(), ExpiresAt: item.ExpiresAt(), - Meta: []byte{meta}, + Meta: a.Copy([]byte{meta}), } list.Kv = append(list.Kv, kv) diff --git a/badger/cmd/stream.go b/badger/cmd/stream.go index a7d3ba59c..9e1d10305 100644 --- a/badger/cmd/stream.go +++ b/badger/cmd/stream.go @@ -39,6 +39,7 @@ This command streams the contents of this DB into another DB with the given opti } var outDir string +var outFile string var compressionType uint32 func init() { @@ -46,6 +47,8 @@ func init() { RootCmd.AddCommand(streamCmd) streamCmd.Flags().StringVarP(&outDir, "out", "o", "", "Path to output DB. The directory should be empty.") + streamCmd.Flags().StringVarP(&outFile, "", "f", "", + "Run a backup to this file.") streamCmd.Flags().BoolVarP(&readOnly, "read_only", "", true, "Option to open input DB in read-only mode") streamCmd.Flags().IntVarP(&numVersions, "num_versions", "", 0, @@ -59,20 +62,6 @@ func init() { } func stream(cmd *cobra.Command, args []string) error { - // Check that outDir doesn't exist or is empty. - if _, err := os.Stat(outDir); err == nil { - f, err := os.Open(outDir) - if err != nil { - return err - } - defer f.Close() - - _, err = f.Readdirnames(1) - if err != io.EOF { - return errors.Errorf("cannot run stream tool on non-empty output directory %s", outDir) - } - } - // Options for input DB. if numVersions <= 0 { numVersions = math.MaxInt32 @@ -94,20 +83,44 @@ func stream(cmd *cobra.Command, args []string) error { return errors.Errorf( "compression value must be one of 0 (disabled), 1 (Snappy), or 2 (ZSTD)") } - outOpt := inOpt. - WithDir(outDir). - WithValueDir(outDir). - WithNumVersionsToKeep(numVersions). - WithCompression(options.CompressionType(compressionType)). - WithReadOnly(false) - inDB, err := badger.OpenManaged(inOpt) if err != nil { return y.Wrapf(err, "cannot open DB at %s", sstDir) } defer inDB.Close() - err = inDB.StreamDB(outOpt) + stream := inDB.NewStreamAt(math.MaxUint64) + + if len(outDir) > 0 { + if _, err := os.Stat(outDir); err == nil { + f, err := os.Open(outDir) + if err != nil { + return err + } + defer f.Close() + + _, err = f.Readdirnames(1) + if err != io.EOF { + return errors.Errorf( + "cannot run stream tool on non-empty output directory %s", outDir) + } + } + + stream.LogPrefix = "DB.Stream" + outOpt := inOpt. + WithDir(outDir). + WithValueDir(outDir). + WithNumVersionsToKeep(numVersions). + WithCompression(options.CompressionType(compressionType)). + WithReadOnly(false) + err = inDB.StreamDB(outOpt) + + } else if len(outFile) > 0 { + stream.LogPrefix = "DB.Backup" + f, err := os.OpenFile(outFile, os.O_RDWR|os.O_CREATE, 0666) + y.Check(err) + _, err = stream.Backup(f, 0) + } fmt.Println("Done.") return err } diff --git a/go.mod b/go.mod index c28fd564c..896a6ce53 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,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.4-0.20201125174811-766bca5e9938 + github.com/dgraph-io/ristretto v0.0.4-0.20201204194510-b7ca2e90f544 github.com/dustin/go-humanize v1.0.0 github.com/golang/protobuf v1.3.1 github.com/golang/snappy v0.0.1 diff --git a/go.sum b/go.sum index 7f46653b3..a74e13871 100644 --- a/go.sum +++ b/go.sum @@ -19,6 +19,8 @@ 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.4-0.20201125174811-766bca5e9938 h1:FdSJif9oUVeH+MpsScsrL6OAbdW0pUYvXmkdhDSWWcQ= github.com/dgraph-io/ristretto v0.0.4-0.20201125174811-766bca5e9938/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= +github.com/dgraph-io/ristretto v0.0.4-0.20201204194510-b7ca2e90f544 h1:6vntPuznvHo+vxTe3KZYzJeorSUt5wkY+1ICtn1GEj0= +github.com/dgraph-io/ristretto v0.0.4-0.20201204194510-b7ca2e90f544/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= 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/iterator.go b/iterator.go index d6af3d854..987227acc 100644 --- a/iterator.go +++ b/iterator.go @@ -26,8 +26,8 @@ import ( "sync/atomic" "time" - "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/table" + "github.com/dgraph-io/ristretto/z" "github.com/dgraph-io/badger/v2/y" ) @@ -436,7 +436,7 @@ type Iterator struct { // iterators created by the stream interface ThreadId int - reuse []*pb.KV + alloc *z.Allocator } // NewIterator returns a new iterator. Depending upon the options, either only keys, or both @@ -496,20 +496,6 @@ func (txn *Txn) NewKeyIterator(key []byte, opt IteratorOptions) *Iterator { return txn.NewIterator(opt) } -// NewKV must be called serially. It is NOT thread-safe. -func (it *Iterator) NewKV() *pb.KV { - if len(it.reuse) == 0 { - return &pb.KV{} - } - kv := it.reuse[len(it.reuse)-1] - it.reuse = it.reuse[:len(it.reuse)-1] - if kv == nil { - kv = &pb.KV{} - } - kv.Reset() - return kv -} - func (it *Iterator) newItem() *Item { item := it.waste.pop() if item == nil { diff --git a/stream.go b/stream.go index 8733fe15e..cc61f7f7e 100644 --- a/stream.go +++ b/stream.go @@ -97,7 +97,8 @@ func (st *Stream) SendDoneMarkers(done bool) { // ToList is a default implementation of KeyToList. It picks up all valid versions of the key, // skipping over deleted or expired keys. func (st *Stream) ToList(key []byte, itr *Iterator) (*pb.KVList, error) { - ka := y.Copy(key) + a := itr.alloc + ka := a.Copy(key) list := &pb.KVList{} for ; itr.Valid(); itr.Next() { @@ -110,11 +111,11 @@ func (st *Stream) ToList(key []byte, itr *Iterator) (*pb.KVList, error) { break } - kv := itr.NewKV() + kv := y.NewKV(a) kv.Key = ka if err := item.Value(func(val []byte) error { - kv.Value = y.Copy(val) + kv.Value = a.Copy(val) return nil }); err != nil { @@ -122,7 +123,7 @@ func (st *Stream) ToList(key []byte, itr *Iterator) (*pb.KVList, error) { } kv.Version = item.Version() kv.ExpiresAt = item.ExpiresAt() - kv.UserMeta = y.Copy([]byte{item.UserMeta()}) + kv.UserMeta = a.Copy([]byte{item.UserMeta()}) list.Kv = append(list.Kv, kv) if st.db.opt.NumVersionsToKeep == 1 { @@ -195,6 +196,10 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { itr.ThreadId = threadId defer itr.Close() + itr.alloc = z.NewAllocator(1 << 20) + itr.alloc.Tag = "Stream.Iterate" + defer itr.alloc.Release() + // This unique stream id is used to identify all the keys from this iteration. streamId := atomic.AddUint32(&st.nextStreamId, 1) @@ -228,6 +233,7 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { } // Now convert to key value. + itr.alloc.Reset() list, err := st.KeyToList(item.KeyCopy(nil), itr) if err != nil { return err @@ -245,9 +251,6 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { return err } } - if len(itr.reuse) < 100 { - itr.reuse = append(itr.reuse, list.Kv...) - } } // Mark the stream as done. if st.doneMarkers { @@ -337,11 +340,9 @@ outer: } speed := bytesSent / durSec - var ms z.MemStats - z.ReadMemStats(&ms) st.db.opt.Infof("%s Time elapsed: %s, bytes sent: %s, speed: %s/sec, jemalloc: %s\n", st.LogPrefix, y.FixedDuration(dur), humanize.IBytes(bytesSent), - humanize.IBytes(speed), humanize.IBytes(ms.Active)) + humanize.IBytes(speed), humanize.IBytes(uint64(z.NumAllocBytes()))) case kvs, ok := <-st.kvChan: if !ok { diff --git a/stream_test.go b/stream_test.go index 1bc7b97b9..35787c79c 100644 --- a/stream_test.go +++ b/stream_test.go @@ -300,8 +300,7 @@ func TestStreamCustomKeyToList(t *testing.T) { if err != nil { return nil, err } - kv := itr.NewKV() - *kv = pb.KV{ + kv := &pb.KV{ Key: y.Copy(item.Key()), Value: val, } diff --git a/y/y_test.go b/y/y_test.go index 151d711c5..3ef0b4191 100644 --- a/y/y_test.go +++ b/y/y_test.go @@ -9,6 +9,8 @@ import ( "testing" "time" + "github.com/dgraph-io/badger/v2/pb" + "github.com/dgraph-io/ristretto/z" "github.com/stretchr/testify/require" ) @@ -276,3 +278,29 @@ func TestEncodedSize(t *testing.T) { require.Equal(t, valBufSize+uint32(2)+expVarintSize, valStruct.EncodedSize()) } + +func TestAllocatorReuse(t *testing.T) { + a := z.NewAllocator(1024) + defer a.Release() + + N := 1024 + buf := make([]byte, 4096) + rand.Read(buf) + + for i := 0; i < N; i++ { + a.Reset() + var list pb.KVList + for j := 0; j < N; j++ { + kv := NewKV(a) + sz := rand.Intn(1024) + kv.Key = a.Copy(buf[:sz]) + kv.Value = a.Copy(buf[:4*sz]) + kv.Meta = a.Copy([]byte{1}) + kv.Version = uint64(sz) + list.Kv = append(list.Kv, kv) + } + _, err := list.Marshal() + require.NoError(t, err) + } + t.Logf("Allocator: %s\n", a) +} From b80c792be789ef9a0c698c866a3bb4df093953b9 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Fri, 4 Dec 2020 13:38:44 -0800 Subject: [PATCH 22/35] Make itr.Alloc public --- backup.go | 2 +- iterator.go | 2 +- stream.go | 10 +++++----- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/backup.go b/backup.go index 574a9dfce..eae06659d 100644 --- a/backup.go +++ b/backup.go @@ -62,7 +62,7 @@ func (db *DB) Backup(w io.Writer, since uint64) (uint64, error) { func (stream *Stream) Backup(w io.Writer, since uint64) (uint64, error) { stream.KeyToList = func(key []byte, itr *Iterator) (*pb.KVList, error) { list := &pb.KVList{} - a := itr.alloc + a := itr.Alloc for ; itr.Valid(); itr.Next() { item := itr.Item() if !bytes.Equal(item.Key(), key) { diff --git a/iterator.go b/iterator.go index 987227acc..3f6adf2c1 100644 --- a/iterator.go +++ b/iterator.go @@ -436,7 +436,7 @@ type Iterator struct { // iterators created by the stream interface ThreadId int - alloc *z.Allocator + Alloc *z.Allocator } // NewIterator returns a new iterator. Depending upon the options, either only keys, or both diff --git a/stream.go b/stream.go index cc61f7f7e..e3e9bae3f 100644 --- a/stream.go +++ b/stream.go @@ -97,7 +97,7 @@ func (st *Stream) SendDoneMarkers(done bool) { // ToList is a default implementation of KeyToList. It picks up all valid versions of the key, // skipping over deleted or expired keys. func (st *Stream) ToList(key []byte, itr *Iterator) (*pb.KVList, error) { - a := itr.alloc + a := itr.Alloc ka := a.Copy(key) list := &pb.KVList{} @@ -196,9 +196,9 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { itr.ThreadId = threadId defer itr.Close() - itr.alloc = z.NewAllocator(1 << 20) - itr.alloc.Tag = "Stream.Iterate" - defer itr.alloc.Release() + itr.Alloc = z.NewAllocator(1 << 20) + itr.Alloc.Tag = "Stream.Iterate" + defer itr.Alloc.Release() // This unique stream id is used to identify all the keys from this iteration. streamId := atomic.AddUint32(&st.nextStreamId, 1) @@ -233,7 +233,7 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { } // Now convert to key value. - itr.alloc.Reset() + itr.Alloc.Reset() list, err := st.KeyToList(item.KeyCopy(nil), itr) if err != nil { return err From 5e418933a0d3f9838c80068b1f3ee80053c5a99b Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Fri, 4 Dec 2020 14:34:07 -0800 Subject: [PATCH 23/35] Fix: Do not stop Stream due to KeyToList error. --- stream.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/stream.go b/stream.go index e3e9bae3f..8dddde1a4 100644 --- a/stream.go +++ b/stream.go @@ -236,7 +236,8 @@ func (st *Stream) produceKVs(ctx context.Context, threadId int) error { itr.Alloc.Reset() list, err := st.KeyToList(item.KeyCopy(nil), itr) if err != nil { - return err + st.db.opt.Warningf("While reading key: %x, got error: %v", item.Key(), err) + continue } if list == nil || len(list.Kv) == 0 { continue From 3a4d8e7399b7301d1a7b41cf6e40f8f121cea4c9 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Fri, 4 Dec 2020 17:41:01 -0800 Subject: [PATCH 24/35] Bring in latest Ristretto --- badger/main.go | 2 +- go.mod | 2 +- go.sum | 2 ++ 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/badger/main.go b/badger/main.go index c284f5aec..a57683d51 100644 --- a/badger/main.go +++ b/badger/main.go @@ -52,6 +52,6 @@ func main() { fmt.Printf("Num Allocated Bytes at program end: %s\n", humanize.IBytes(uint64(z.NumAllocBytes()))) if z.NumAllocBytes() > 0 { - z.PrintLeaks() + fmt.Println(z.Leaks()) } } diff --git a/go.mod b/go.mod index 896a6ce53..7714dbfed 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,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.4-0.20201204194510-b7ca2e90f544 + github.com/dgraph-io/ristretto v0.0.4-0.20201205013540-bafef7527542 github.com/dustin/go-humanize v1.0.0 github.com/golang/protobuf v1.3.1 github.com/golang/snappy v0.0.1 diff --git a/go.sum b/go.sum index a74e13871..bea53bb1d 100644 --- a/go.sum +++ b/go.sum @@ -21,6 +21,8 @@ github.com/dgraph-io/ristretto v0.0.4-0.20201125174811-766bca5e9938 h1:FdSJif9oU github.com/dgraph-io/ristretto v0.0.4-0.20201125174811-766bca5e9938/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= github.com/dgraph-io/ristretto v0.0.4-0.20201204194510-b7ca2e90f544 h1:6vntPuznvHo+vxTe3KZYzJeorSUt5wkY+1ICtn1GEj0= github.com/dgraph-io/ristretto v0.0.4-0.20201204194510-b7ca2e90f544/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= +github.com/dgraph-io/ristretto v0.0.4-0.20201205013540-bafef7527542 h1:Yya3DAZRF8ibfavFQQysegFbPJQ8plpta39uev0lw1I= +github.com/dgraph-io/ristretto v0.0.4-0.20201205013540-bafef7527542/go.mod h1:tv2ec8nA7vRpSYX7/MbP52ihrUMXIHit54CQMq8npXQ= 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= From 63726a849024768772f0c04655512e327ff8724e Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Fri, 4 Dec 2020 21:54:57 -0800 Subject: [PATCH 25/35] Fix(OOM): Avoid too many splits In edge cases, we end up with too many splits during compactions, which make compactions take up too much RAM. Avoid that by limiting splits to max 5. Also, avoid running more compactions when the memory usage is going above 16GB. --- levels.go | 28 ++++++++++++++++++++++------ 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/levels.go b/levels.go index 49f7ccbe8..fa9dbb51f 100644 --- a/levels.go +++ b/levels.go @@ -426,9 +426,6 @@ func (s *levelsController) runCompactor(id int, lc *z.Closer) { return } - ticker := time.NewTicker(50 * time.Millisecond) - defer ticker.Stop() - moveL0toFront := func(prios []compactionPriority) []compactionPriority { idx := -1 for i, p := range prios { @@ -474,10 +471,21 @@ func (s *levelsController) runCompactor(id int, lc *z.Closer) { return false } + ticker := time.NewTicker(50 * time.Millisecond) + defer ticker.Stop() + var backOff int for { select { // Can add a done channel or other stuff. case <-ticker.C: + if z.NumAllocBytes() > 16<<30 { + // Back off. We're already using a lot of memory. + backOff++ + if backOff%1000 == 0 { + s.kv.opt.Infof("Compaction backed off %d times\n", backOff) + } + break + } runOnce() case <-lc.HasBeenClosed(): return @@ -982,8 +990,16 @@ type compactDef struct { func (s *levelsController) addSplits(cd *compactDef) { cd.splits = cd.splits[:0] - // Pick one every 3 tables. - const N = 3 + // Let's say we have 10 tables in cd.bot and min width = 3. Then, we'll pick + // 0, 1, 2 (pick), 3, 4, 5 (pick), 6, 7, 8 (pick), 9 (pick, because last table). + // This gives us 4 picks for 10 tables. + // In an edge case, 142 tables in bottom led to 48 splits. That's too many splits, because it + // then uses up a lot of memory for table builder. + // We should keep it so we have at max 5 splits. + width := int(math.Ceil(float64(len(cd.bot) / 5.0))) + if width < 3 { + width = 3 + } skr := cd.thisRange skr.extend(cd.nextRange) @@ -1000,7 +1016,7 @@ func (s *levelsController) addSplits(cd *compactDef) { addRange([]byte{}) return } - if i%N == N-1 { + if i%width == width-1 { // Right should always have ts=maxUint64 otherwise we'll lose keys // in subcompaction. Consider the following. // Top table is [A1...C3(deleted)] From 6142e81c26a8cddf489144ec9746ff8bafd56e75 Mon Sep 17 00:00:00 2001 From: Daniel Mai Date: Fri, 4 Dec 2020 22:22:00 -0800 Subject: [PATCH 26/35] Fix typo in width calculation. --- levels.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/levels.go b/levels.go index fa9dbb51f..bd1e53fbd 100644 --- a/levels.go +++ b/levels.go @@ -996,7 +996,7 @@ func (s *levelsController) addSplits(cd *compactDef) { // In an edge case, 142 tables in bottom led to 48 splits. That's too many splits, because it // then uses up a lot of memory for table builder. // We should keep it so we have at max 5 splits. - width := int(math.Ceil(float64(len(cd.bot) / 5.0))) + width := int(math.Ceil(float64(len(cd.bot)) / 5.0)) if width < 3 { width = 3 } From f492aa38111d551bda6ffc0c263ceed9ec728117 Mon Sep 17 00:00:00 2001 From: Naman Jain Date: Mon, 7 Dec 2020 13:39:14 +0530 Subject: [PATCH 27/35] fix(compaction): fix table size estimation on compaction (#1613) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit If a table has a mixture of value log pointers and embedded values, badger will carry over the last length from a value log entry into the subsequent embedded entries. Co-authored-by: Raúl Kripalani --- levels.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/levels.go b/levels.go index bd1e53fbd..aeabec232 100644 --- a/levels.go +++ b/levels.go @@ -644,7 +644,6 @@ func (s *levelsController) subcompact(it y.Iterator, kr keyRange, cd compactDef, var lastKey, skipKey []byte var numBuilds, numVersions int - var vp valuePointer addKeys := func(builder *table.Builder) { timeStart := time.Now() @@ -745,6 +744,7 @@ func (s *levelsController) subcompact(it y.Iterator, kr keyRange, cd compactDef, } } numKeys++ + var vp valuePointer if vs.Meta&bitValuePointer > 0 { vp.Decode(vs.Value) } From eaf91a1992b5030d1d46501ade5b1da1bfff92e2 Mon Sep 17 00:00:00 2001 From: Mohit kumar Bajoria Date: Mon, 7 Dec 2020 23:51:15 +0530 Subject: [PATCH 28/35] docs: Set relativeURL for Netlify. (#1612) Since we are in process of moving to Netlify, we need this change for docs to work. This change has no effect on current badger docs --- docs/config.toml | 4 +++- docs/themes/hugo-docs/layouts/404.html | 18 +++++++++--------- .../hugo-docs/layouts/partials/footer.html | 4 ++-- .../hugo-docs/layouts/partials/header.html | 2 +- .../hugo-docs/layouts/partials/topbar.html | 2 +- .../hugo-docs/layouts/shortcodes/load-img.html | 2 +- 6 files changed, 17 insertions(+), 15 deletions(-) diff --git a/docs/config.toml b/docs/config.toml index e2a734cd0..0f65f3fee 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -1,6 +1,8 @@ languageCode = "en-us" theme = "hugo-docs" -canonifyURLs = true +canonifyURLs = false + +relativeURLs = true [markup.goldmark.renderer] unsafe = true diff --git a/docs/themes/hugo-docs/layouts/404.html b/docs/themes/hugo-docs/layouts/404.html index e643355b1..8a9b80e07 100644 --- a/docs/themes/hugo-docs/layouts/404.html +++ b/docs/themes/hugo-docs/layouts/404.html @@ -4,13 +4,13 @@ {{ partial "meta.html" . }} {{ .Title }} - - - - - - - + + + + + + +