From 666bc77bfa26b4c06ca1877275ff7074c3e71bf9 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 23 Mar 2020 15:52:32 +0530 Subject: [PATCH 01/50] Added batchWrite in index.go --- posting/index.go | 23 +++++++++++++++++------ worker/draft.go | 1 + 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/posting/index.go b/posting/index.go index bdc6850dc6c..d3c40f1eecd 100644 --- a/posting/index.go +++ b/posting/index.go @@ -553,7 +553,7 @@ func (r *rebuilder) Run(ctx context.Context) error { WithLogRotatesToFlush(10). WithMaxCacheSize(50) // TODO(Aman): Disable cache altogether - tmpDB, err := badger.OpenManaged(dbOpts) + tmpDB, err := badger.OpenManaged(dbOpts) //Again in managed mode. if err != nil { return errors.Wrap(err, "error opening temp badger for reindexing") } @@ -570,8 +570,9 @@ func (r *rebuilder) Run(ctx context.Context) error { // TODO(Aman): Replace TxnWriter with WriteBatch. While we do that we should ensure that // WriteBatch has a mechanism for throttling. Also, find other places where TxnWriter // could be replaced with WriteBatch in the code - tmpWriter := NewTxnWriter(tmpDB) - stream := pstore.NewStreamAt(r.startTs) + //tmpWriter := NewTxnWriter(tmpDB) + tmpWriter2 := tmpDB.NewWriteBatchAt(r.startTs) + stream := pstore.NewStreamAt(r.startTs) //pstore badge is opene in managed mode. Note the "AT" stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix stream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { @@ -619,8 +620,18 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - if err := tmpWriter.Write(kvList); err != nil { - return errors.Wrap(err, "error setting entries in temp badger") + // if err := tmpWriter.Write(kvList); err != nil { + // return errors.Wrap(err, "error setting entries in temp badger") + // } + // Is it a good idea to support WriteAPI in WriteBatch which takes a list of KVs.? + for _, kv := range kvList.Kv { + //var meta byte + //if len(kv.UserMeta) > 0 { + // meta = kv.UserMeta[0] + //} + if err := tmpWriter2.Set(kv.Key, kv.Value); err != nil { + return err + } } return nil @@ -630,7 +641,7 @@ func (r *rebuilder) Run(ctx context.Context) error { if err := stream.Orchestrate(ctx); err != nil { return err } - if err := tmpWriter.Flush(); err != nil { + if err := tmpWriter2.Flush(); err != nil { return err } glog.V(1).Infof("Rebuilding index for predicate %s: building temp index took: %v\n", diff --git a/worker/draft.go b/worker/draft.go index 391cef04fef..3747665ccbe 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -622,6 +622,7 @@ func (n *node) processApplyCh() { } } +// TODO(Anurag): Are we using pkey? Remove if redundant. func (n *node) commitOrAbort(pkey string, delta *pb.OracleDelta) error { // First let's commit all mutations to disk. writer := posting.NewTxnWriter(pstore) From 55cfbad3a0d1ea174ae120907a7bd0ee65e16db1 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 23 Mar 2020 22:19:03 +0530 Subject: [PATCH 02/50] Add batchWriter where the data is published to pStore and removed the batchWriter for tempDB --- posting/index.go | 35 +++++++++++++++-------------------- 1 file changed, 15 insertions(+), 20 deletions(-) diff --git a/posting/index.go b/posting/index.go index d3c40f1eecd..f5193732d92 100644 --- a/posting/index.go +++ b/posting/index.go @@ -570,8 +570,9 @@ func (r *rebuilder) Run(ctx context.Context) error { // TODO(Aman): Replace TxnWriter with WriteBatch. While we do that we should ensure that // WriteBatch has a mechanism for throttling. Also, find other places where TxnWriter // could be replaced with WriteBatch in the code - //tmpWriter := NewTxnWriter(tmpDB) - tmpWriter2 := tmpDB.NewWriteBatchAt(r.startTs) + // WriteBatch can not be used here because it doesn't have an API to allow multiple versions. + // We wish to store same keys with diff version/timestamp to identify when doing roll-up + tmpWriter := NewTxnWriter(tmpDB) stream := pstore.NewStreamAt(r.startTs) //pstore badge is opene in managed mode. Note the "AT" stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix @@ -620,18 +621,8 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - // if err := tmpWriter.Write(kvList); err != nil { - // return errors.Wrap(err, "error setting entries in temp badger") - // } - // Is it a good idea to support WriteAPI in WriteBatch which takes a list of KVs.? - for _, kv := range kvList.Kv { - //var meta byte - //if len(kv.UserMeta) > 0 { - // meta = kv.UserMeta[0] - //} - if err := tmpWriter2.Set(kv.Key, kv.Value); err != nil { - return err - } + if err := tmpWriter.Write(kvList); err != nil { + return errors.Wrap(err, "error setting entries in temp badger") } return nil @@ -641,7 +632,7 @@ func (r *rebuilder) Run(ctx context.Context) error { if err := stream.Orchestrate(ctx); err != nil { return err } - if err := tmpWriter2.Flush(); err != nil { + if err := tmpWriter.Flush(); err != nil { return err } glog.V(1).Infof("Rebuilding index for predicate %s: building temp index took: %v\n", @@ -655,18 +646,19 @@ func (r *rebuilder) Run(ctx context.Context) error { r.attr, time.Since(start)) }() - writer := NewTxnWriter(pstore) + // writer := NewTxnWriter(pstore) + batchWriter := pstore.NewWriteBatchAt(r.startTs) tmpStream := tmpDB.NewStreamAt(counter) tmpStream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (2/2):", r.attr) tmpStream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { - l, err := ReadPostingList(key, itr) + l, err := ReadPostingList(key, itr) //Reads all the versions for a key if err != nil { return nil, errors.Wrap(err, "error in reading posting list from pstore") } // No need to write a loop after ReadPostingList to skip unread entries // for a given key because we only wrote BitDeltaPosting to temp badger. - kvs, err := l.Rollup() + kvs, err := l.Rollup() //Merges all the posting list into one posting list. if err != nil { return nil, err } @@ -681,9 +673,12 @@ func (r *rebuilder) Run(ctx context.Context) error { // We choose to write the PL at r.startTs, so it won't be read by txns, // which occurred before this schema mutation. - if err := writer.SetAt(kv.Key, kv.Value, BitCompletePosting, r.startTs); err != nil { + e := &badger.Entry{Key: kv.Key, Value: kv.Value, UserMeta: BitCompletePosting} + if err := batchWriter.SetEntry((e).WithDiscard()); err != nil { return errors.Wrap(err, "error in writing index to pstore") } + // if err := writer.SetAt(kv.Key, kv.Value, BitCompletePosting, r.startTs); err != nil { + // } } return nil @@ -693,7 +688,7 @@ func (r *rebuilder) Run(ctx context.Context) error { return err } glog.V(1).Infof("Rebuilding index for predicate %s: Flushing all writes.\n", r.attr) - return writer.Flush() + return batchWriter.Flush() } // IndexRebuild holds the info needed to initiate a rebuilt of the indices. From 8eaecac0205f1f0b33f919b0c9671e511b87491c Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 24 Mar 2020 14:30:02 +0530 Subject: [PATCH 03/50] Code clean-up --- posting/index.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/posting/index.go b/posting/index.go index f5193732d92..59a082a98ee 100644 --- a/posting/index.go +++ b/posting/index.go @@ -567,9 +567,6 @@ func (r *rebuilder) Run(ctx context.Context) error { // We set it to 1 in case there are no keys found and NewStreamAt is called with ts=0. var counter uint64 = 1 - // TODO(Aman): Replace TxnWriter with WriteBatch. While we do that we should ensure that - // WriteBatch has a mechanism for throttling. Also, find other places where TxnWriter - // could be replaced with WriteBatch in the code // WriteBatch can not be used here because it doesn't have an API to allow multiple versions. // We wish to store same keys with diff version/timestamp to identify when doing roll-up tmpWriter := NewTxnWriter(tmpDB) @@ -646,7 +643,6 @@ func (r *rebuilder) Run(ctx context.Context) error { r.attr, time.Since(start)) }() - // writer := NewTxnWriter(pstore) batchWriter := pstore.NewWriteBatchAt(r.startTs) tmpStream := tmpDB.NewStreamAt(counter) tmpStream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (2/2):", r.attr) @@ -677,8 +673,6 @@ func (r *rebuilder) Run(ctx context.Context) error { if err := batchWriter.SetEntry((e).WithDiscard()); err != nil { return errors.Wrap(err, "error in writing index to pstore") } - // if err := writer.SetAt(kv.Key, kv.Value, BitCompletePosting, r.startTs); err != nil { - // } } return nil From a07f87d9efc52768d2780515e8e861a8b8994cfa Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 24 Mar 2020 18:42:09 +0530 Subject: [PATCH 04/50] Edited comments on changes --- posting/index.go | 16 +++++++++------- worker/draft.go | 2 +- worker/snapshot.go | 1 - 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/posting/index.go b/posting/index.go index 59a082a98ee..aa645f751d5 100644 --- a/posting/index.go +++ b/posting/index.go @@ -553,7 +553,7 @@ func (r *rebuilder) Run(ctx context.Context) error { WithLogRotatesToFlush(10). WithMaxCacheSize(50) // TODO(Aman): Disable cache altogether - tmpDB, err := badger.OpenManaged(dbOpts) //Again in managed mode. + tmpDB, err := badger.OpenManaged(dbOpts) if err != nil { return errors.Wrap(err, "error opening temp badger for reindexing") } @@ -567,10 +567,12 @@ func (r *rebuilder) Run(ctx context.Context) error { // We set it to 1 in case there are no keys found and NewStreamAt is called with ts=0. var counter uint64 = 1 - // WriteBatch can not be used here because it doesn't have an API to allow multiple versions. - // We wish to store same keys with diff version/timestamp to identify when doing roll-up + // WriteBatch can not be used here because it doesn't have an API to allow writing multiple versions. + // We wish to store same keys with diff version/timestamp to ensure that we get all of them back + // when doing roll-up. WriteBatch can only be used when we want to write all txns at the same + // timestamp. tmpWriter := NewTxnWriter(tmpDB) - stream := pstore.NewStreamAt(r.startTs) //pstore badge is opene in managed mode. Note the "AT" + stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix stream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { @@ -647,14 +649,14 @@ func (r *rebuilder) Run(ctx context.Context) error { tmpStream := tmpDB.NewStreamAt(counter) tmpStream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (2/2):", r.attr) tmpStream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { - l, err := ReadPostingList(key, itr) //Reads all the versions for a key + l, err := ReadPostingList(key, itr) if err != nil { return nil, errors.Wrap(err, "error in reading posting list from pstore") } // No need to write a loop after ReadPostingList to skip unread entries // for a given key because we only wrote BitDeltaPosting to temp badger. - kvs, err := l.Rollup() //Merges all the posting list into one posting list. + kvs, err := l.Rollup() if err != nil { return nil, err } @@ -670,7 +672,7 @@ func (r *rebuilder) Run(ctx context.Context) error { // We choose to write the PL at r.startTs, so it won't be read by txns, // which occurred before this schema mutation. e := &badger.Entry{Key: kv.Key, Value: kv.Value, UserMeta: BitCompletePosting} - if err := batchWriter.SetEntry((e).WithDiscard()); err != nil { + if err := batchWriter.SetEntry(e); err != nil { return errors.Wrap(err, "error in writing index to pstore") } } diff --git a/worker/draft.go b/worker/draft.go index 6077712db84..94eccca6889 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -639,7 +639,7 @@ func (n *node) processApplyCh() { } } -// TODO(Anurag): Are we using pkey? Remove if redundant. +// TODO(Anurag): Are we using pkey? Remove if unused. func (n *node) commitOrAbort(pkey string, delta *pb.OracleDelta) error { // First let's commit all mutations to disk. writer := posting.NewTxnWriter(pstore) diff --git a/worker/snapshot.go b/worker/snapshot.go index 7cd78948414..dde07a8ed17 100644 --- a/worker/snapshot.go +++ b/worker/snapshot.go @@ -62,7 +62,6 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { if err := sw.Prepare(); err != nil { return 0, err } - writer = sw } else { writer = posting.NewTxnWriter(pstore) From 5a6985c7a0782e80a0f024ff77b30948d8882181 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 24 Mar 2020 18:52:15 +0530 Subject: [PATCH 05/50] Addressed golangci comments --- posting/index.go | 8 ++++---- worker/snapshot.go | 1 + 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/posting/index.go b/posting/index.go index aa645f751d5..1ec2ca0cf07 100644 --- a/posting/index.go +++ b/posting/index.go @@ -567,10 +567,10 @@ func (r *rebuilder) Run(ctx context.Context) error { // We set it to 1 in case there are no keys found and NewStreamAt is called with ts=0. var counter uint64 = 1 - // WriteBatch can not be used here because it doesn't have an API to allow writing multiple versions. - // We wish to store same keys with diff version/timestamp to ensure that we get all of them back - // when doing roll-up. WriteBatch can only be used when we want to write all txns at the same - // timestamp. + // WriteBatch can not be used here because it doesn't have an API to allow writing + // multiple versions. We wish to store same keys with diff version/timestamp to + // ensure that we get all of them back when doing roll-up. WriteBatch can only be + // used when we want to write all txns at the same timestamp. tmpWriter := NewTxnWriter(tmpDB) stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) diff --git a/worker/snapshot.go b/worker/snapshot.go index dde07a8ed17..7cd78948414 100644 --- a/worker/snapshot.go +++ b/worker/snapshot.go @@ -62,6 +62,7 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { if err := sw.Prepare(); err != nil { return 0, err } + writer = sw } else { writer = posting.NewTxnWriter(pstore) From 541945d49b28789cbaed316b4806a10ad7ff42f7 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 30 Mar 2020 15:59:35 +0530 Subject: [PATCH 06/50] Added benchmark file --- posting/writer_test.go | 52 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 52 insertions(+) create mode 100644 posting/writer_test.go diff --git a/posting/writer_test.go b/posting/writer_test.go new file mode 100644 index 00000000000..8211ed42404 --- /dev/null +++ b/posting/writer_test.go @@ -0,0 +1,52 @@ +/* + * Copyright 2019 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 posting + +import ( + "testing" + + "github.com/dgraph-io/badger/v2" + "github.com/dgraph-io/dgraph/x" +) + +type kv struct { + key []byte + value []byte +} + +var KVList = []kv{} + +func BenchmarkTxnWriter(b *testing.B) { + + for i := 0; i < 50; i++ { + n := kv{key: []byte(string(i)), value: []byte("Check Value")} + KVList = append(KVList, n) + } + + var db *badger.DB + w := NewTxnWriter(db) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + for _, typ := range KVList { + k := typ.key + v := typ.value + x.Check(w.SetAt(k, v, BitSchemaPosting, 1)) + } + } + +} From 5fb9809863739fd53f9450b635c2bf32ec44ab58 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 30 Mar 2020 18:28:13 +0530 Subject: [PATCH 07/50] Added function to time BatchWrite --- posting/writer_test.go | 39 ++++++++++++++++++++++++++++++++++++--- 1 file changed, 36 insertions(+), 3 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 8211ed42404..1221589c66f 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -17,9 +17,12 @@ package posting import ( + "io/ioutil" + "math" "testing" "github.com/dgraph-io/badger/v2" + "github.com/dgraph-io/badger/v2/options" "github.com/dgraph-io/dgraph/x" ) @@ -30,14 +33,25 @@ type kv struct { var KVList = []kv{} -func BenchmarkTxnWriter(b *testing.B) { +var tmpIndexDir, err = ioutil.TempDir("", "dgraph_index_") + +var dbOpts = badger.DefaultOptions(tmpIndexDir). + WithSyncWrites(false). + WithNumVersionsToKeep(math.MaxInt64). + WithLogger(&x.ToGlog{}). + WithCompression(options.None). + WithEventLogging(false). + WithLogRotatesToFlush(10). + WithMaxCacheSize(50) // TODO(Aman): Disable cache altogether + +var db, err2 = badger.OpenManaged(dbOpts) - for i := 0; i < 50; i++ { +func BenchmarkTxnWriter(b *testing.B) { + for i := 0; i < 50000; i++ { n := kv{key: []byte(string(i)), value: []byte("Check Value")} KVList = append(KVList, n) } - var db *badger.DB w := NewTxnWriter(db) b.ResetTimer() @@ -50,3 +64,22 @@ func BenchmarkTxnWriter(b *testing.B) { } } + +func BenchmarkWriteBatch(b *testing.B) { + for i := 0; i < 50000; i++ { + n := kv{key: []byte(string(i)), value: []byte("Check Value")} + KVList = append(KVList, n) + } + + batch := db.NewWriteBatchAt(1) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + for _, typ := range KVList { + k := typ.key + v := typ.value + x.Check(batch.Set(k, v)) + } + } + +} From 325204685a7b7b8f8a0f0cd4b48bd1ee67c3e7f6 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 30 Mar 2020 21:45:24 +0530 Subject: [PATCH 08/50] Code cleanup --- posting/writer_test.go | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 1221589c66f..6502878edc1 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -17,6 +17,7 @@ package posting import ( + "fmt" "io/ioutil" "math" "testing" @@ -31,8 +32,6 @@ type kv struct { value []byte } -var KVList = []kv{} - var tmpIndexDir, err = ioutil.TempDir("", "dgraph_index_") var dbOpts = badger.DefaultOptions(tmpIndexDir). @@ -46,14 +45,18 @@ var dbOpts = badger.DefaultOptions(tmpIndexDir). var db, err2 = badger.OpenManaged(dbOpts) -func BenchmarkTxnWriter(b *testing.B) { +func createKVList() []kv { + var KVList = []kv{} for i := 0; i < 50000; i++ { n := kv{key: []byte(string(i)), value: []byte("Check Value")} KVList = append(KVList, n) } + return KVList +} +func BenchmarkTxnWriter(b *testing.B) { + KVList := createKVList() w := NewTxnWriter(db) - b.ResetTimer() for i := 0; i < b.N; i++ { for _, typ := range KVList { @@ -61,18 +64,16 @@ func BenchmarkTxnWriter(b *testing.B) { v := typ.value x.Check(w.SetAt(k, v, BitSchemaPosting, 1)) } + if err := w.Flush(); err != nil { + fmt.Printf("Got error while flushing txnwriter: %v\n", err) + } } } func BenchmarkWriteBatch(b *testing.B) { - for i := 0; i < 50000; i++ { - n := kv{key: []byte(string(i)), value: []byte("Check Value")} - KVList = append(KVList, n) - } - + KVList := createKVList() batch := db.NewWriteBatchAt(1) - b.ResetTimer() for i := 0; i < b.N; i++ { for _, typ := range KVList { @@ -80,6 +81,9 @@ func BenchmarkWriteBatch(b *testing.B) { v := typ.value x.Check(batch.Set(k, v)) } + if err := batch.Flush(); err != nil { + fmt.Printf("Got error while flushing batch: %v\n", err) + } } } From 528eca06c200023a4f46c8e62dfa15dde55fbbc2 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 30 Mar 2020 22:05:46 +0530 Subject: [PATCH 09/50] Bug Fix --- posting/writer_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 6502878edc1..2e4a56228b0 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -56,9 +56,10 @@ func createKVList() []kv { func BenchmarkTxnWriter(b *testing.B) { KVList := createKVList() - w := NewTxnWriter(db) + b.ResetTimer() for i := 0; i < b.N; i++ { + w := NewTxnWriter(db) for _, typ := range KVList { k := typ.key v := typ.value @@ -73,9 +74,9 @@ func BenchmarkTxnWriter(b *testing.B) { func BenchmarkWriteBatch(b *testing.B) { KVList := createKVList() - batch := db.NewWriteBatchAt(1) b.ResetTimer() for i := 0; i < b.N; i++ { + batch := db.NewWriteBatchAt(1) for _, typ := range KVList { k := typ.key v := typ.value From 258173b3097e77e141565a841aab55027ddf6e11 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 30 Mar 2020 22:19:17 +0530 Subject: [PATCH 10/50] Remove glog initialization --- posting/writer_test.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 2e4a56228b0..759c924a8b8 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -17,6 +17,7 @@ package posting import ( + "flag" "fmt" "io/ioutil" "math" @@ -33,11 +34,10 @@ type kv struct { } var tmpIndexDir, err = ioutil.TempDir("", "dgraph_index_") - var dbOpts = badger.DefaultOptions(tmpIndexDir). WithSyncWrites(false). WithNumVersionsToKeep(math.MaxInt64). - WithLogger(&x.ToGlog{}). + //WithLogger(&x.ToGlog{}). WithCompression(options.None). WithEventLogging(false). WithLogRotatesToFlush(10). @@ -55,6 +55,7 @@ func createKVList() []kv { } func BenchmarkTxnWriter(b *testing.B) { + flag.Parse() KVList := createKVList() b.ResetTimer() @@ -73,6 +74,7 @@ func BenchmarkTxnWriter(b *testing.B) { } func BenchmarkWriteBatch(b *testing.B) { + flag.Parse() KVList := createKVList() b.ResetTimer() for i := 0; i < b.N; i++ { From 8bef4643769facc170d009e578d9775ea3ac6839 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 31 Mar 2020 11:07:02 +0530 Subject: [PATCH 11/50] Removed unused flags --- posting/writer_test.go | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 759c924a8b8..715ae2d09ec 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -17,7 +17,6 @@ package posting import ( - "flag" "fmt" "io/ioutil" "math" @@ -34,30 +33,25 @@ type kv struct { } var tmpIndexDir, err = ioutil.TempDir("", "dgraph_index_") + var dbOpts = badger.DefaultOptions(tmpIndexDir). WithSyncWrites(false). WithNumVersionsToKeep(math.MaxInt64). - //WithLogger(&x.ToGlog{}). - WithCompression(options.None). - WithEventLogging(false). - WithLogRotatesToFlush(10). - WithMaxCacheSize(50) // TODO(Aman): Disable cache altogether + WithCompression(options.None) var db, err2 = badger.OpenManaged(dbOpts) func createKVList() []kv { var KVList = []kv{} for i := 0; i < 50000; i++ { - n := kv{key: []byte(string(i)), value: []byte("Check Value")} + n := kv{key: []byte(string(i)), value: []byte(string(i))} KVList = append(KVList, n) } return KVList } func BenchmarkTxnWriter(b *testing.B) { - flag.Parse() KVList := createKVList() - b.ResetTimer() for i := 0; i < b.N; i++ { w := NewTxnWriter(db) @@ -74,7 +68,6 @@ func BenchmarkTxnWriter(b *testing.B) { } func BenchmarkWriteBatch(b *testing.B) { - flag.Parse() KVList := createKVList() b.ResetTimer() for i := 0; i < b.N; i++ { From 465a1b855335dcc802da5a97921af851c10778b6 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 31 Mar 2020 17:13:03 +0530 Subject: [PATCH 12/50] Merged benchamrk functions in one --- posting/writer_test.go | 116 +++++++++++++++++++++++++---------------- 1 file changed, 71 insertions(+), 45 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 715ae2d09ec..5996ee01d50 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -1,5 +1,5 @@ /* - * Copyright 2019 Dgraph Labs, Inc. and Contributors + * 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. @@ -17,14 +17,14 @@ package posting import ( - "fmt" "io/ioutil" "math" + "os" "testing" "github.com/dgraph-io/badger/v2" "github.com/dgraph-io/badger/v2/options" - "github.com/dgraph-io/dgraph/x" + "github.com/stretchr/testify/require" ) type kv struct { @@ -32,54 +32,80 @@ type kv struct { value []byte } -var tmpIndexDir, err = ioutil.TempDir("", "dgraph_index_") +func BenchmarkWriter(b *testing.B) { + createKVList := func() []kv { + var KVList = []kv{} + for i := 0; i < 5000000; i++ { + n := kv{key: []byte(string(i)), value: []byte(string(i))} + KVList = append(KVList, n) + } + return KVList + } -var dbOpts = badger.DefaultOptions(tmpIndexDir). - WithSyncWrites(false). - WithNumVersionsToKeep(math.MaxInt64). - WithCompression(options.None) + dbOpts := badger.DefaultOptions(""). + WithLogger(nil). + WithSyncWrites(false). + WithNumVersionsToKeep(math.MaxInt64). + WithCompression(options.None) -var db, err2 = badger.OpenManaged(dbOpts) + KVList := createKVList() + validate := false -func createKVList() []kv { - var KVList = []kv{} - for i := 0; i < 50000; i++ { - n := kv{key: []byte(string(i)), value: []byte(string(i))} - KVList = append(KVList, n) - } - return KVList -} + b.Run("TxnWriter", func(b *testing.B) { + tmpIndexDir, err := ioutil.TempDir("", "dgraph") + require.NoError(b, err) + defer os.RemoveAll(tmpIndexDir) -func BenchmarkTxnWriter(b *testing.B) { - KVList := createKVList() - b.ResetTimer() - for i := 0; i < b.N; i++ { - w := NewTxnWriter(db) - for _, typ := range KVList { - k := typ.key - v := typ.value - x.Check(w.SetAt(k, v, BitSchemaPosting, 1)) - } - if err := w.Flush(); err != nil { - fmt.Printf("Got error while flushing txnwriter: %v\n", err) + dbOpts.Dir = tmpIndexDir + dbOpts.ValueDir = tmpIndexDir + var db, err2 = badger.OpenManaged(dbOpts) + require.NoError(b, err2) + defer db.Close() + + b.ResetTimer() + for i := 0; i < b.N; i++ { + w := NewTxnWriter(db) + for _, typ := range KVList { + k := typ.key + v := typ.value + err = w.SetAt(k, v, BitSchemaPosting, 1) + if validate { + require.NoError(b, err) + } + } + err := w.Flush() + if validate { + require.NoError(b, err) + } } - } + }) + b.Run("Write batch", func(b *testing.B) { + tmpIndexDir, err := ioutil.TempDir("", "dgraph") + require.NoError(b, err) + defer os.RemoveAll(tmpIndexDir) -} + dbOpts.Dir = tmpIndexDir + dbOpts.ValueDir = tmpIndexDir -func BenchmarkWriteBatch(b *testing.B) { - KVList := createKVList() - b.ResetTimer() - for i := 0; i < b.N; i++ { - batch := db.NewWriteBatchAt(1) - for _, typ := range KVList { - k := typ.key - v := typ.value - x.Check(batch.Set(k, v)) - } - if err := batch.Flush(); err != nil { - fmt.Printf("Got error while flushing batch: %v\n", err) - } - } + var db, err2 = badger.OpenManaged(dbOpts) + require.NoError(b, err2) + defer db.Close() + b.ResetTimer() + for i := 0; i < b.N; i++ { + batch := db.NewWriteBatchAt(1) + for _, typ := range KVList { + k := typ.key + v := typ.value + err = batch.Set(k, v) + if validate { + require.NoError(b, err) + } + } + err := batch.Flush() + if validate { + require.NoError(b, err) + } + } + }) } From f289f58e7e178e5d3e7fc573b6be8c20d8300167 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 31 Mar 2020 18:00:05 +0530 Subject: [PATCH 13/50] Minor edits --- posting/writer_test.go | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 5996ee01d50..a271c6fa3ca 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -73,13 +73,11 @@ func BenchmarkWriter(b *testing.B) { require.NoError(b, err) } } - err := w.Flush() - if validate { - require.NoError(b, err) - } + require.NoError(b, w.Flush()) + } }) - b.Run("Write batch", func(b *testing.B) { + b.Run("WriteBatch", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) defer os.RemoveAll(tmpIndexDir) @@ -102,10 +100,7 @@ func BenchmarkWriter(b *testing.B) { require.NoError(b, err) } } - err := batch.Flush() - if validate { - require.NoError(b, err) - } + require.NoError(b, batch.Flush()) } }) } From 2355bfbecf52d87819a8cf5f37481aa49c0f02bc Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 31 Mar 2020 18:12:24 +0530 Subject: [PATCH 14/50] Minor edits again --- posting/writer_test.go | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index a271c6fa3ca..9079062c883 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -49,7 +49,6 @@ func BenchmarkWriter(b *testing.B) { WithCompression(options.None) KVList := createKVList() - validate := false b.Run("TxnWriter", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") @@ -68,10 +67,7 @@ func BenchmarkWriter(b *testing.B) { for _, typ := range KVList { k := typ.key v := typ.value - err = w.SetAt(k, v, BitSchemaPosting, 1) - if validate { - require.NoError(b, err) - } + w.SetAt(k, v, BitSchemaPosting, 1) } require.NoError(b, w.Flush()) @@ -95,10 +91,7 @@ func BenchmarkWriter(b *testing.B) { for _, typ := range KVList { k := typ.key v := typ.value - err = batch.Set(k, v) - if validate { - require.NoError(b, err) - } + batch.Set(k, v) } require.NoError(b, batch.Flush()) } From 11783f44b4a0c7dfd2e21155ffbeb0e0ca4e0c0e Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 6 Apr 2020 17:30:49 +0530 Subject: [PATCH 15/50] Changed go.mod file to point to local bager --- go.mod | 2 ++ 1 file changed, 2 insertions(+) diff --git a/go.mod b/go.mod index 87a8eb6b003..17234f54fa7 100644 --- a/go.mod +++ b/go.mod @@ -65,3 +65,5 @@ require ( gopkg.in/ini.v1 v1.48.0 // indirect gopkg.in/yaml.v2 v2.2.4 ) + +replace github.com/dgraph-io/badger/v2 => /home/alvis/go/src/github.com/dgraph-io/badger From 70b7c5943b0dc852b039cb68048d2cb5f1e3ff0f Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 6 Apr 2020 19:55:08 +0530 Subject: [PATCH 16/50] Replace txnWriter with WriteBatch which takes different versions --- posting/index.go | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index 1ec2ca0cf07..00f2db0b417 100644 --- a/posting/index.go +++ b/posting/index.go @@ -572,6 +572,7 @@ func (r *rebuilder) Run(ctx context.Context) error { // ensure that we get all of them back when doing roll-up. WriteBatch can only be // used when we want to write all txns at the same timestamp. tmpWriter := NewTxnWriter(tmpDB) + tmpBatchWriter := pstore.NewWriteBatchAt(r.startTs) stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix @@ -620,7 +621,20 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - if err := tmpWriter.Write(kvList); err != nil { + //if err := tmpWriter.Write(kvList); err != nil { + // return errors.Wrap(err, "error setting entries in temp badger") + //} + // for _, kv := range kvList.Kv { + // var meta byte + // if len(kv.UserMeta) > 0 { + // meta = kv.UserMeta[0] + // } + // e := &Entry{Key: kv.Key, Value: kv.Value, UserMeta: meta, Version: kv.Version} + // if err := tmpBatchWriter.SetEntry(e); err != nil { + // return err + // } + // } + if err := tmpBatchWriter.Write(kvList); err != nil { return errors.Wrap(err, "error setting entries in temp badger") } From 7379274f19111cb83fcc01fd517e9f87886723df Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 6 Apr 2020 20:39:20 +0530 Subject: [PATCH 17/50] Temporary changes --- posting/index.go | 14 +++++++------- posting/index_test.go | 2 ++ 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/posting/index.go b/posting/index.go index 00f2db0b417..51b10edc3ec 100644 --- a/posting/index.go +++ b/posting/index.go @@ -572,7 +572,7 @@ func (r *rebuilder) Run(ctx context.Context) error { // ensure that we get all of them back when doing roll-up. WriteBatch can only be // used when we want to write all txns at the same timestamp. tmpWriter := NewTxnWriter(tmpDB) - tmpBatchWriter := pstore.NewWriteBatchAt(r.startTs) + //tmpBatchWriter := pstore.NewWriteBatchAt(r.startTs) stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix @@ -621,9 +621,9 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - //if err := tmpWriter.Write(kvList); err != nil { - // return errors.Wrap(err, "error setting entries in temp badger") - //} + if err := tmpWriter.Write(kvList); err != nil { + return errors.Wrap(err, "error setting entries in temp badger") + } // for _, kv := range kvList.Kv { // var meta byte // if len(kv.UserMeta) > 0 { @@ -634,9 +634,9 @@ func (r *rebuilder) Run(ctx context.Context) error { // return err // } // } - if err := tmpBatchWriter.Write(kvList); err != nil { - return errors.Wrap(err, "error setting entries in temp badger") - } + //if err := tmpBatchWriter.Write(kvList); err != nil { + // return errors.Wrap(err, "error setting entries in temp badger") + //} return nil } diff --git a/posting/index_test.go b/posting/index_test.go index 3ee12949a78..125862cedb5 100644 --- a/posting/index_test.go +++ b/posting/index_test.go @@ -19,6 +19,7 @@ package posting import ( "bytes" "context" + "fmt" "math" "testing" "time" @@ -394,6 +395,7 @@ func TestRebuildReverseEdges(t *testing.T) { var revVals []*List var prevKey []byte it.Seek(prefix) + fmt.Println("Check", prefix) for it.ValidForPrefix(prefix) { item := it.Item() key := item.Key() From 503d77ac2d23b7100ae073c77cacfb857bfa1fc4 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 6 Apr 2020 23:44:33 +0530 Subject: [PATCH 18/50] Test Write Batch --- posting/index.go | 18 +++++++++--------- posting/index_test.go | 7 ++++++- 2 files changed, 15 insertions(+), 10 deletions(-) diff --git a/posting/index.go b/posting/index.go index 51b10edc3ec..2b8c36536a6 100644 --- a/posting/index.go +++ b/posting/index.go @@ -571,8 +571,8 @@ func (r *rebuilder) Run(ctx context.Context) error { // multiple versions. We wish to store same keys with diff version/timestamp to // ensure that we get all of them back when doing roll-up. WriteBatch can only be // used when we want to write all txns at the same timestamp. - tmpWriter := NewTxnWriter(tmpDB) - //tmpBatchWriter := pstore.NewWriteBatchAt(r.startTs) + //tmpWriter := NewTxnWriter(tmpDB) + tmpBatchWriter := pstore.NewWriteBatchAt(r.startTs) stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix @@ -621,9 +621,9 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - if err := tmpWriter.Write(kvList); err != nil { - return errors.Wrap(err, "error setting entries in temp badger") - } + // if err := tmpWriter.Write(kvList); err != nil { + // return errors.Wrap(err, "error setting entries in temp badger") + // } // for _, kv := range kvList.Kv { // var meta byte // if len(kv.UserMeta) > 0 { @@ -634,9 +634,9 @@ func (r *rebuilder) Run(ctx context.Context) error { // return err // } // } - //if err := tmpBatchWriter.Write(kvList); err != nil { - // return errors.Wrap(err, "error setting entries in temp badger") - //} + if err := tmpBatchWriter.Write(kvList); err != nil { + return errors.Wrap(err, "error setting entries in temp badger") + } return nil } @@ -645,7 +645,7 @@ func (r *rebuilder) Run(ctx context.Context) error { if err := stream.Orchestrate(ctx); err != nil { return err } - if err := tmpWriter.Flush(); err != nil { + if err := tmpBatchWriter.Flush(); err != nil { return err } glog.V(1).Infof("Rebuilding index for predicate %s: building temp index took: %v\n", diff --git a/posting/index_test.go b/posting/index_test.go index 125862cedb5..ea8ac0c3caa 100644 --- a/posting/index_test.go +++ b/posting/index_test.go @@ -379,6 +379,7 @@ func TestRebuildReverseEdges(t *testing.T) { OldSchema: nil, CurrentSchema: ¤tSchema, } + fmt.Println("Current Schema: ", currentSchema) // TODO: Remove after fixing sync marks. require.NoError(t, rebuildReverseEdges(context.Background(), &rb)) @@ -395,8 +396,12 @@ func TestRebuildReverseEdges(t *testing.T) { var revVals []*List var prevKey []byte it.Seek(prefix) - fmt.Println("Check", prefix) + fmt.Println("Prefix: ", prefix) + count := 1 for it.ValidForPrefix(prefix) { + + fmt.Println("Count: ", count) + count = count + 1 item := it.Item() key := item.Key() if bytes.Equal(key, prevKey) { From 4a91d0bd8f5db50b5d154be0f7a46edd9ce0e0c7 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 7 Apr 2020 18:34:42 +0530 Subject: [PATCH 19/50] Add Write Batch at one instance --- posting/index.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/posting/index.go b/posting/index.go index 2b8c36536a6..4e34c460f48 100644 --- a/posting/index.go +++ b/posting/index.go @@ -571,8 +571,8 @@ func (r *rebuilder) Run(ctx context.Context) error { // multiple versions. We wish to store same keys with diff version/timestamp to // ensure that we get all of them back when doing roll-up. WriteBatch can only be // used when we want to write all txns at the same timestamp. - //tmpWriter := NewTxnWriter(tmpDB) - tmpBatchWriter := pstore.NewWriteBatchAt(r.startTs) + // tmpWriter := NewTxnWriter(tmpDB) + tmpBatchWriter := pstore.NewWriteBatchAt(counter) stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix From fce4ca103199c5cac908f1aeee0fcd343136fb8c Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Wed, 8 Apr 2020 15:07:25 +0530 Subject: [PATCH 20/50] Use batchset instead of txnwriter --- posting/index.go | 76 ++++++++++++++++++++++++++++++++++++++---------- 1 file changed, 60 insertions(+), 16 deletions(-) diff --git a/posting/index.go b/posting/index.go index 5ebfb2bf16c..6e9bd387982 100644 --- a/posting/index.go +++ b/posting/index.go @@ -24,6 +24,7 @@ import ( "io/ioutil" "math" "os" + "sync" "sync/atomic" "time" @@ -35,6 +36,7 @@ import ( "github.com/dgraph-io/badger/v2" "github.com/dgraph-io/badger/v2/options" bpb "github.com/dgraph-io/badger/v2/pb" + "github.com/dgraph-io/badger/v2/y" "github.com/dgraph-io/dgraph/ee/enc" "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/schema" @@ -558,6 +560,7 @@ func (r *rebuilder) Run(ctx context.Context) error { WithEventLogging(false). WithLogRotatesToFlush(10). WithEncryptionKey(enc.ReadEncryptionKeyFile(x.WorkerConfig.BadgerKeyFile)) + tmpDB, err := badger.OpenManaged(dbOpts) if err != nil { return errors.Wrap(err, "error opening temp badger for reindexing") @@ -572,10 +575,6 @@ func (r *rebuilder) Run(ctx context.Context) error { // We set it to 1 in case there are no keys found and NewStreamAt is called with ts=0. var counter uint64 = 1 - // TODO(Aman): Replace TxnWriter with WriteBatch. While we do that we should ensure that - // WriteBatch has a mechanism for throttling. Also, find other places where TxnWriter - // could be replaced with WriteBatch in the code - tmpWriter := NewTxnWriter(tmpDB) stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix @@ -623,11 +622,44 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } + var wg sync.WaitGroup stream.Send = func(kvList *bpb.KVList) error { - if err := tmpWriter.Write(kvList); err != nil { - return errors.Wrap(err, "error setting entries in temp badger") + var entries []*badger.Entry + for _, kv := range kvList.Kv { + var meta byte + var entry *badger.Entry + if len(kv.UserMeta) > 0 { + meta = kv.UserMeta[0] + } + switch meta { + case BitCompletePosting, BitEmptyPosting: + entry = (&badger.Entry{ + Key: kv.Key, + Value: kv.Value, + UserMeta: meta, + }).WithDiscard() + default: + entry = &badger.Entry{ + Key: kv.Key, + Value: kv.Value, + UserMeta: meta, + } + } + entry.Key = y.KeyWithTs(entry.Key, kv.Version) + entries = append(entries, entry) } + wg.Add(1) + if err := tmpDB.BatchSetAsync(entries, func(err error) { + defer wg.Done() + if err != nil { + // TODO: Handle error + glog.Error(err) + return + } + }); err != nil { + return err + } return nil } @@ -635,9 +667,8 @@ func (r *rebuilder) Run(ctx context.Context) error { if err := stream.Orchestrate(ctx); err != nil { return err } - if err := tmpWriter.Flush(); err != nil { - return err - } + + wg.Wait() glog.V(1).Infof("Rebuilding index for predicate %s: building temp index took: %v\n", r.attr, time.Since(start)) @@ -649,7 +680,6 @@ func (r *rebuilder) Run(ctx context.Context) error { r.attr, time.Since(start)) }() - writer := NewTxnWriter(pstore) tmpStream := tmpDB.NewStreamAt(counter) tmpStream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (2/2):", r.attr) tmpStream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { @@ -668,26 +698,40 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } tmpStream.Send = func(kvList *bpb.KVList) error { + var entries []*badger.Entry + var entry *badger.Entry for _, kv := range kvList.Kv { if len(kv.Value) == 0 { continue } + entry = (&badger.Entry{ + Key: kv.Key, + Value: kv.Value, + UserMeta: BitCompletePosting, + }).WithDiscard() + entry.Key = y.KeyWithTs(entry.Key, r.startTs) + entries = append(entries, entry) + } - // We choose to write the PL at r.startTs, so it won't be read by txns, - // which occurred before this schema mutation. - if err := writer.SetAt(kv.Key, kv.Value, BitCompletePosting, r.startTs); err != nil { - return errors.Wrap(err, "error in writing index to pstore") + wg.Add(1) + if err := pstore.BatchSetAsync(entries, func(err error) { + defer wg.Done() + // TODO: Handle error + if err != nil { + glog.Error(err) } + }); err != nil { + return err } - return nil } if err := tmpStream.Orchestrate(ctx); err != nil { return err } + wg.Wait() glog.V(1).Infof("Rebuilding index for predicate %s: Flushing all writes.\n", r.attr) - return writer.Flush() + return nil } // IndexRebuild holds the info needed to initiate a rebuilt of the indices. From ea3582a2e79160d33eeb666cc96ddc627f2ef892 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Wed, 8 Apr 2020 15:10:28 +0530 Subject: [PATCH 21/50] Fix go.mod --- go.mod | 14 +++++++------- go.sum | 23 +++++++++++++++-------- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/go.mod b/go.mod index a0ebed5009e..8a3c6bd8721 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,7 @@ require ( github.com/99designs/gqlgen v0.11.0 github.com/DataDog/datadog-go v0.0.0-20190425163447-40bafcb5f6c1 // indirect github.com/DataDog/opencensus-go-exporter-datadog v0.0.0-20190503082300-0f32ad59ab08 - github.com/DataDog/zstd v1.4.4 // indirect + github.com/DataDog/zstd v1.4.5 // indirect github.com/OneOfOne/xxhash v1.2.5 // indirect github.com/beorn7/perks v1.0.0 // indirect github.com/blevesearch/bleve v0.0.0-20181114232033-e1f5e6cdcd76 @@ -16,11 +16,11 @@ require ( github.com/blevesearch/segment v0.0.0-20160915185041-762005e7a34f // indirect github.com/blevesearch/snowballstem v0.0.0-20180110192139-26b06a2c243d // indirect github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd - github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200316175624-91c31ebe8c22 + github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200408093502-e82473ba28e7 github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 - github.com/dgraph-io/ristretto v0.0.2-0.20200115201040-8f368f2f2ab3 + github.com/dgraph-io/ristretto v0.0.2 github.com/dgrijalva/jwt-go v3.2.0+incompatible - github.com/dgryski/go-farm v0.0.0-20191112170834-c2139c5d712b + github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 github.com/dgryski/go-groupvarint v0.0.0-20190318181831-5ce5df8ca4e1 github.com/dustin/go-humanize v1.0.0 github.com/getsentry/sentry-go v0.5.1 @@ -38,7 +38,7 @@ require ( github.com/mitchellh/panicwrap v1.0.0 github.com/paulmach/go.geojson v0.0.0-20170327170536-40612a87147b github.com/philhofer/fwd v1.0.0 // indirect - github.com/pkg/errors v0.8.1 + github.com/pkg/errors v0.9.1 github.com/pkg/profile v1.2.1 github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829 github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90 // indirect @@ -56,8 +56,8 @@ require ( go.etcd.io/etcd v0.0.0-20190228193606-a943ad0ee4c9 go.opencensus.io v0.21.0 golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550 - golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553 - golang.org/x/sys v0.0.0-20200116001909-b77594299b42 + golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e + golang.org/x/sys v0.0.0-20200408040146-ea54a3c99b9b golang.org/x/text v0.3.2 google.golang.org/genproto v0.0.0-20190516172635-bb713bdc0e52 // indirect google.golang.org/grpc v1.23.0 diff --git a/go.sum b/go.sum index ada338d2ed4..377cc220f78 100644 --- a/go.sum +++ b/go.sum @@ -18,8 +18,8 @@ github.com/DataDog/datadog-go v0.0.0-20190425163447-40bafcb5f6c1/go.mod h1:LButx github.com/DataDog/opencensus-go-exporter-datadog v0.0.0-20190503082300-0f32ad59ab08 h1:5btKvK+N+FpW0EEgvxq7LWcUEwIRLsL4IwIo0u+Qlhs= github.com/DataDog/opencensus-go-exporter-datadog v0.0.0-20190503082300-0f32ad59ab08/go.mod h1:gMGUEe16aZh0QN941HgDjwrdjU4iTthPoz2/AtDRADE= github.com/DataDog/zstd v1.4.1/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= -github.com/DataDog/zstd v1.4.4 h1:+IawcoXhCBylN7ccwdwf8LOH2jKq7NavGpEPanrlTzE= -github.com/DataDog/zstd v1.4.4/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= +github.com/DataDog/zstd v1.4.5 h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ= +github.com/DataDog/zstd v1.4.5/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5/go.mod h1:lmUJ/7eu/Q8D7ML55dXQrVaamCz2vxCfdQBasLZfHKk= @@ -76,17 +76,19 @@ 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/badger v1.6.0 h1:DshxFxZWXUcO0xX476VJC07Xsr6ZCBVRHKZ93Oh7Evo= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200316175624-91c31ebe8c22 h1:X2g/STOldw/iNkdP9BHI/8zHe4N/Lkp8Z4jLyywtnCI= -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200316175624-91c31ebe8c22/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200408093502-e82473ba28e7 h1:h3gMqZKj0z6A9oRuALJnXBbFIlaQHzOky7u82jJmQB0= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200408093502-e82473ba28e7/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 h1:DTgOrw91nMIukDm/WEvdobPLl0LgeDd/JE66+24jBks= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453/go.mod h1:Co+FwJrnndSrPORO8Gdn20dR7FPTfmXr0W/su0Ve/Ig= github.com/dgraph-io/ristretto v0.0.2-0.20200115201040-8f368f2f2ab3 h1:MQLRM35Pp0yAyBYksjbj1nZI/w6eyRY/mWoM1sFf4kU= github.com/dgraph-io/ristretto v0.0.2-0.20200115201040-8f368f2f2ab3/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= +github.com/dgraph-io/ristretto v0.0.2 h1:a5WaUrDa0qm0YrAAS1tUykT5El3kt62KNZZeMxQn3po= +github.com/dgraph-io/ristretto v0.0.2/go.mod h1:KPxhHT9ZxKefz+PCeOGsrHpl1qZ7i70dGTu2u+Ahh6E= github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= -github.com/dgryski/go-farm v0.0.0-20191112170834-c2139c5d712b h1:SeiGBzKrEtuDddnBABHkp4kq9sBGE9nuYmk6FPTg0zg= -github.com/dgryski/go-farm v0.0.0-20191112170834-c2139c5d712b/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= +github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-groupvarint v0.0.0-20190318181831-5ce5df8ca4e1 h1:RSnSk6/ViWmqng/mNcPztSPgr6/4EVDxMmBH/a0w/6I= github.com/dgryski/go-groupvarint v0.0.0-20190318181831-5ce5df8ca4e1/go.mod h1:MlkUQveSLEDbIgq2r1e++tSf0zfzU9mQpa9Qkczl+9Y= github.com/dgryski/trifles v0.0.0-20190318185328-a8d75aae118c h1:TUuUh0Xgj97tLMNtWtNvI9mIV6isjEb9lBMNv+77IGM= @@ -273,6 +275,8 @@ github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTw github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/profile v1.2.1 h1:F++O52m40owAmADcojzM+9gyjmMOY/T4oYJkgFDH8RE= github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= @@ -410,8 +414,8 @@ golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/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-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553 h1:efeOvDhwQ29Dj3SdAV/MJf8oukgn+8D8WgaCaRMchF8= -golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e h1:3G+cUijn7XD+S4eJFddp53Pv7+slrESplyjG25HgL+k= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -434,6 +438,9 @@ golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42 h1:vEOn+mP2zCOVzKckCZy6YsCtDblrpj/w7B9nxGNELpg= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200408040146-ea54a3c99b9b h1:h03Ur1RlPrGTjua4koYdpGl8W0eYo8p1uI9w7RPlkdk= +golang.org/x/sys v0.0.0-20200408040146-ea54a3c99b9b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= From 20cda1823a90fc29f3c1bf056c0327e4c4000a5e Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Thu, 9 Apr 2020 18:50:08 +0530 Subject: [PATCH 22/50] Merged Batchset with tests and benchmarks --- go.mod | 2 +- posting/index.go | 2 +- posting/writer_test.go | 59 +++++++++++++++++++++++++++++++++++++----- 3 files changed, 55 insertions(+), 8 deletions(-) diff --git a/go.mod b/go.mod index cafa5c72434..95ad1b74ec0 100644 --- a/go.mod +++ b/go.mod @@ -66,4 +66,4 @@ require ( gopkg.in/yaml.v2 v2.2.4 ) -replace github.com/dgraph-io/badger/v2 => /home/alvis/go/src/github.com/dgraph-io/badger +//replace github.com/dgraph-io/badger/v2 => /home/alvis/go/src/github.com/dgraph-io/badger diff --git a/posting/index.go b/posting/index.go index 7a8e77b1d3c..cdf0f0262a7 100644 --- a/posting/index.go +++ b/posting/index.go @@ -576,7 +576,7 @@ func (r *rebuilder) Run(ctx context.Context) error { var counter uint64 = 1 // tmpWriter := NewTxnWriter(tmpDB) - tmpBatchWriter := pstore.NewWriteBatchAt(counter) + //tmpBatchWriter := pstore.NewWriteBatchAt(counter) stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix diff --git a/posting/writer_test.go b/posting/writer_test.go index 9079062c883..7a4155b7e90 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -20,10 +20,13 @@ import ( "io/ioutil" "math" "os" + "sync" "testing" "github.com/dgraph-io/badger/v2" "github.com/dgraph-io/badger/v2/options" + "github.com/dgraph-io/badger/v2/y" + "github.com/golang/glog" "github.com/stretchr/testify/require" ) @@ -86,14 +89,58 @@ func BenchmarkWriter(b *testing.B) { defer db.Close() b.ResetTimer() + for i := 0; i < b.N; i++ { - batch := db.NewWriteBatchAt(1) - for _, typ := range KVList { - k := typ.key - v := typ.value - batch.Set(k, v) + var wg sync.WaitGroup + var entries []*badger.Entry + // batch := db.NewWriteBatchAt(1) + // for _, typ := range KVList { + // k := typ.key + // v := typ.value + // batch.Set(k, v) + // } + // require.NoError(b, batch.Flush()) + for _, kv := range KVList { + // var meta byte + // var entry *badger.Entry + // if len(kv.UserMeta) > 0 { + // meta = kv.UserMeta[0] + // } + // switch meta { + // case BitCompletePosting, BitEmptyPosting: + // entry = (&badger.Entry{ + // Key: kv.Key, + // Value: kv.Value, + // UserMeta: meta, + // }).WithDiscard() + // default: + k := kv.key + v := kv.value + var meta byte + var entry *badger.Entry + + entry = &badger.Entry{ + Key: k, + Value: v, + UserMeta: meta, + } + // UserMeta: meta, + // } + entry.Key = y.KeyWithTs(entry.Key, 1) //Same version for each key + entries = append(entries, entry) } - require.NoError(b, batch.Flush()) + + wg.Add(1) + err := db.BatchSetAsync(entries, func(err error) { + defer wg.Done() + if err != nil { + // TODO: Handle error + glog.Error(err) + return + } + }) + require.NoError(b, err) + } }) } From 7f4112bfd92fe5c1c6920852b287841be9d6fa07 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Thu, 9 Apr 2020 21:33:32 +0530 Subject: [PATCH 23/50] Reduce the numbers of kv pairs --- posting/writer_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 7a4155b7e90..d664f49f349 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -38,7 +38,7 @@ type kv struct { func BenchmarkWriter(b *testing.B) { createKVList := func() []kv { var KVList = []kv{} - for i := 0; i < 5000000; i++ { + for i := 0; i < 50000; i++ { n := kv{key: []byte(string(i)), value: []byte(string(i))} KVList = append(KVList, n) } From 85724cd7aed7366a19164296eef9bab8e398d84b Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 13 Apr 2020 11:31:48 +0530 Subject: [PATCH 24/50] Add wg Wait --- posting/writer_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/posting/writer_test.go b/posting/writer_test.go index d664f49f349..eb72b110b29 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -140,6 +140,7 @@ func BenchmarkWriter(b *testing.B) { } }) require.NoError(b, err) + wg.Wait() } }) From a976e0addf3e496c66097b965b0c348c08fbe6ad Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Thu, 23 Apr 2020 16:47:07 +0530 Subject: [PATCH 25/50] resolve merge conflicts --- go.mod | 4 ---- go.sum | 10 ---------- 2 files changed, 14 deletions(-) diff --git a/go.mod b/go.mod index 1dc36c6c9d7..00cd6b56e7e 100644 --- a/go.mod +++ b/go.mod @@ -63,12 +63,8 @@ require ( go.opencensus.io v0.21.0 golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550 golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e -<<<<<<< HEAD - golang.org/x/sys v0.0.0-20200408040146-ea54a3c99b9b -======= golang.org/x/sync v0.0.0-20190423024810-112230192c58 golang.org/x/sys v0.0.0-20200420163511-1957bb5e6d1f ->>>>>>> master golang.org/x/text v0.3.2 google.golang.org/genproto v0.0.0-20190516172635-bb713bdc0e52 // indirect google.golang.org/grpc v1.23.0 diff --git a/go.sum b/go.sum index 4790aad85e0..b5825910f39 100644 --- a/go.sum +++ b/go.sum @@ -76,13 +76,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/badger v1.6.0 h1:DshxFxZWXUcO0xX476VJC07Xsr6ZCBVRHKZ93Oh7Evo= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -<<<<<<< HEAD -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200408093502-e82473ba28e7 h1:h3gMqZKj0z6A9oRuALJnXBbFIlaQHzOky7u82jJmQB0= -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200408093502-e82473ba28e7/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= -======= github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200421062606-cddf7c03451c h1:IXsBFBQ0g5JlPfu+3HotLmkej2xgyrkKceWmFlXSYIQ= github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200421062606-cddf7c03451c/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= ->>>>>>> master github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 h1:DTgOrw91nMIukDm/WEvdobPLl0LgeDd/JE66+24jBks= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453/go.mod h1:Co+FwJrnndSrPORO8Gdn20dR7FPTfmXr0W/su0Ve/Ig= github.com/dgraph-io/ristretto v0.0.2-0.20200115201040-8f368f2f2ab3 h1:MQLRM35Pp0yAyBYksjbj1nZI/w6eyRY/mWoM1sFf4kU= @@ -454,13 +449,8 @@ golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200116001909-b77594299b42 h1:vEOn+mP2zCOVzKckCZy6YsCtDblrpj/w7B9nxGNELpg= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -<<<<<<< HEAD -golang.org/x/sys v0.0.0-20200408040146-ea54a3c99b9b h1:h03Ur1RlPrGTjua4koYdpGl8W0eYo8p1uI9w7RPlkdk= -golang.org/x/sys v0.0.0-20200408040146-ea54a3c99b9b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -======= golang.org/x/sys v0.0.0-20200420163511-1957bb5e6d1f h1:gWF768j/LaZugp8dyS4UwsslYCYz9XgFxvlgsn0n9H8= golang.org/x/sys v0.0.0-20200420163511-1957bb5e6d1f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= ->>>>>>> master golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= From 022a9cb8b39b66ee6cb4c4c3b26c389fcd28aef6 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Thu, 23 Apr 2020 16:56:45 +0530 Subject: [PATCH 26/50] resolve merge conflicts again --- go.mod | 4 ---- 1 file changed, 4 deletions(-) diff --git a/go.mod b/go.mod index 00cd6b56e7e..f1d21ccc641 100644 --- a/go.mod +++ b/go.mod @@ -16,11 +16,7 @@ require ( github.com/blevesearch/segment v0.0.0-20160915185041-762005e7a34f // indirect github.com/blevesearch/snowballstem v0.0.0-20180110192139-26b06a2c243d // indirect github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd -<<<<<<< HEAD - github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200408093502-e82473ba28e7 -======= github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200421062606-cddf7c03451c ->>>>>>> master github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 github.com/dgraph-io/ristretto v0.0.2 github.com/dgrijalva/jwt-go v3.2.0+incompatible From d72ba0623da9f6a60a5424c39cd53d12f3110bbd Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Thu, 23 Apr 2020 20:09:55 +0530 Subject: [PATCH 27/50] revert back to master for index.go --- posting/index.go | 78 ++++++++++-------------------------------------- 1 file changed, 16 insertions(+), 62 deletions(-) diff --git a/posting/index.go b/posting/index.go index c21d82b7b5e..41f04339da2 100644 --- a/posting/index.go +++ b/posting/index.go @@ -24,7 +24,6 @@ import ( "io/ioutil" "math" "os" - "sync" "sync/atomic" "time" @@ -36,7 +35,6 @@ import ( "github.com/dgraph-io/badger/v2" "github.com/dgraph-io/badger/v2/options" bpb "github.com/dgraph-io/badger/v2/pb" - "github.com/dgraph-io/badger/v2/y" "github.com/dgraph-io/dgraph/ee/enc" "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/schema" @@ -561,7 +559,6 @@ func (r *rebuilder) Run(ctx context.Context) error { WithCompression(options.None). WithLogRotatesToFlush(10). WithEncryptionKey(enc.ReadEncryptionKeyFile(x.WorkerConfig.BadgerKeyFile)) - tmpDB, err := badger.OpenManaged(dbOpts) if err != nil { return errors.Wrap(err, "error opening temp badger for reindexing") @@ -576,8 +573,10 @@ func (r *rebuilder) Run(ctx context.Context) error { // We set it to 1 in case there are no keys found and NewStreamAt is called with ts=0. var counter uint64 = 1 - // tmpWriter := NewTxnWriter(tmpDB) - //tmpBatchWriter := pstore.NewWriteBatchAt(counter) + // TODO(Aman): Replace TxnWriter with WriteBatch. While we do that we should ensure that + // WriteBatch has a mechanism for throttling. Also, find other places where TxnWriter + // could be replaced with WriteBatch in the code + tmpWriter := NewTxnWriter(tmpDB) stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix @@ -625,44 +624,11 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } - var wg sync.WaitGroup stream.Send = func(kvList *bpb.KVList) error { - var entries []*badger.Entry - for _, kv := range kvList.Kv { - var meta byte - var entry *badger.Entry - if len(kv.UserMeta) > 0 { - meta = kv.UserMeta[0] - } - switch meta { - case BitCompletePosting, BitEmptyPosting: - entry = (&badger.Entry{ - Key: kv.Key, - Value: kv.Value, - UserMeta: meta, - }).WithDiscard() - default: - entry = &badger.Entry{ - Key: kv.Key, - Value: kv.Value, - UserMeta: meta, - } - } - entry.Key = y.KeyWithTs(entry.Key, kv.Version) - entries = append(entries, entry) + if err := tmpWriter.Write(kvList); err != nil { + return errors.Wrap(err, "error setting entries in temp badger") } - wg.Add(1) - if err := tmpDB.BatchSetAsync(entries, func(err error) { - defer wg.Done() - if err != nil { - // TODO: Handle error - glog.Error(err) - return - } - }); err != nil { - return err - } return nil } @@ -670,8 +636,9 @@ func (r *rebuilder) Run(ctx context.Context) error { if err := stream.Orchestrate(ctx); err != nil { return err } - - wg.Wait() + if err := tmpWriter.Flush(); err != nil { + return err + } glog.V(1).Infof("Rebuilding index for predicate %s: building temp index took: %v\n", r.attr, time.Since(start)) @@ -683,6 +650,7 @@ func (r *rebuilder) Run(ctx context.Context) error { r.attr, time.Since(start)) }() + writer := NewTxnWriter(pstore) tmpStream := tmpDB.NewStreamAt(counter) tmpStream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (2/2):", r.attr) tmpStream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { @@ -701,40 +669,26 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } tmpStream.Send = func(kvList *bpb.KVList) error { - var entries []*badger.Entry - var entry *badger.Entry for _, kv := range kvList.Kv { if len(kv.Value) == 0 { continue } - entry = (&badger.Entry{ - Key: kv.Key, - Value: kv.Value, - UserMeta: BitCompletePosting, - }).WithDiscard() - entry.Key = y.KeyWithTs(entry.Key, r.startTs) - entries = append(entries, entry) - } - wg.Add(1) - if err := pstore.BatchSetAsync(entries, func(err error) { - defer wg.Done() - // TODO: Handle error - if err != nil { - glog.Error(err) + // We choose to write the PL at r.startTs, so it won't be read by txns, + // which occurred before this schema mutation. + if err := writer.SetAt(kv.Key, kv.Value, BitCompletePosting, r.startTs); err != nil { + return errors.Wrap(err, "error in writing index to pstore") } - }); err != nil { - return err } + return nil } if err := tmpStream.Orchestrate(ctx); err != nil { return err } - wg.Wait() glog.V(1).Infof("Rebuilding index for predicate %s: Flushing all writes.\n", r.attr) - return nil + return writer.Flush() } // IndexRebuild holds the info needed to initiate a rebuilt of the indices. From f5fd3782190cd43bbf5bf43f5780c0cba9024d12 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Thu, 23 Apr 2020 20:12:15 +0530 Subject: [PATCH 28/50] revert index_test.go to master --- posting/index_test.go | 7 ------- 1 file changed, 7 deletions(-) diff --git a/posting/index_test.go b/posting/index_test.go index cde1e5191e8..9bec4609224 100644 --- a/posting/index_test.go +++ b/posting/index_test.go @@ -19,7 +19,6 @@ package posting import ( "bytes" "context" - "fmt" "math" "testing" "time" @@ -379,7 +378,6 @@ func TestRebuildReverseEdges(t *testing.T) { OldSchema: nil, CurrentSchema: ¤tSchema, } - fmt.Println("Current Schema: ", currentSchema) // TODO: Remove after fixing sync marks. require.NoError(t, rebuildReverseEdges(context.Background(), &rb)) @@ -396,12 +394,7 @@ func TestRebuildReverseEdges(t *testing.T) { var revVals []*List var prevKey []byte it.Seek(prefix) - fmt.Println("Prefix: ", prefix) - count := 1 for it.ValidForPrefix(prefix) { - - fmt.Println("Count: ", count) - count = count + 1 item := it.Item() key := item.Key() if bytes.Equal(key, prevKey) { From 074bcb34ad996f92b3f822cc13aa451a256a086e Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Fri, 24 Apr 2020 12:19:37 +0530 Subject: [PATCH 29/50] Use new batch writer api --- posting/index.go | 19 +++++--- posting/writer_test.go | 99 ++++++++++++++++++++---------------------- 2 files changed, 62 insertions(+), 56 deletions(-) diff --git a/posting/index.go b/posting/index.go index 41f04339da2..278c52eb7ea 100644 --- a/posting/index.go +++ b/posting/index.go @@ -576,7 +576,8 @@ func (r *rebuilder) Run(ctx context.Context) error { // TODO(Aman): Replace TxnWriter with WriteBatch. While we do that we should ensure that // WriteBatch has a mechanism for throttling. Also, find other places where TxnWriter // could be replaced with WriteBatch in the code - tmpWriter := NewTxnWriter(tmpDB) + //tmpWriter := NewTxnWriter(tmpDB) + tmpWriter := tmpDB.NewManagedWriteBatch() stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) stream.Prefix = r.prefix @@ -625,9 +626,14 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - if err := tmpWriter.Write(kvList); err != nil { - return errors.Wrap(err, "error setting entries in temp badger") + for _, kv := range kvList.Kv { + if err := tmpWriter.SetEntryAt((&badger.Entry{Key: kv.Key, Value: kv.Value, UserMeta: kv.UserMeta[0]}).WithDiscard(), kv.Version); err != nil { + return errors.Wrap(err, "error setting entries in temp badger") + } } + // if err := tmpWriter.Write(kvList); err != nil { + // return errors.Wrap(err, "error setting entries in temp badger") + // } return nil } @@ -650,7 +656,8 @@ func (r *rebuilder) Run(ctx context.Context) error { r.attr, time.Since(start)) }() - writer := NewTxnWriter(pstore) + //writer := NewTxnWriter(pstore) + writer := pstore.NewManagedWriteBatch() tmpStream := tmpDB.NewStreamAt(counter) tmpStream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (2/2):", r.attr) tmpStream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { @@ -676,7 +683,9 @@ func (r *rebuilder) Run(ctx context.Context) error { // We choose to write the PL at r.startTs, so it won't be read by txns, // which occurred before this schema mutation. - if err := writer.SetAt(kv.Key, kv.Value, BitCompletePosting, r.startTs); err != nil { + //err := writer.SetAt(kv.Key, kv.Value, BitCompletePosting, r.startTs) + + if err := writer.SetEntryAt((&badger.Entry{Key: kv.Key, Value: kv.Value, UserMeta: BitCompletePosting}).WithDiscard(), r.startTs); err != nil { return errors.Wrap(err, "error in writing index to pstore") } } diff --git a/posting/writer_test.go b/posting/writer_test.go index eb72b110b29..1c7e2b0dda2 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -20,13 +20,10 @@ import ( "io/ioutil" "math" "os" - "sync" "testing" "github.com/dgraph-io/badger/v2" "github.com/dgraph-io/badger/v2/options" - "github.com/dgraph-io/badger/v2/y" - "github.com/golang/glog" "github.com/stretchr/testify/require" ) @@ -91,56 +88,56 @@ func BenchmarkWriter(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - var wg sync.WaitGroup - var entries []*badger.Entry - // batch := db.NewWriteBatchAt(1) - // for _, typ := range KVList { - // k := typ.key - // v := typ.value - // batch.Set(k, v) + // var wg sync.WaitGroup + // var entries []*badger.Entry + // // batch := db.NewWriteBatchAt(1) + // // for _, typ := range KVList { + // // k := typ.key + // // v := typ.value + // // batch.Set(k, v) + // // } + // // require.NoError(b, batch.Flush()) + // for _, kv := range KVList { + // // var meta byte + // // var entry *badger.Entry + // // if len(kv.UserMeta) > 0 { + // // meta = kv.UserMeta[0] + // // } + // // switch meta { + // // case BitCompletePosting, BitEmptyPosting: + // // entry = (&badger.Entry{ + // // Key: kv.Key, + // // Value: kv.Value, + // // UserMeta: meta, + // // }).WithDiscard() + // // default: + // k := kv.key + // v := kv.value + // var meta byte + // var entry *badger.Entry + + // entry = &badger.Entry{ + // Key: k, + // Value: v, + // UserMeta: meta, // } - // require.NoError(b, batch.Flush()) - for _, kv := range KVList { - // var meta byte - // var entry *badger.Entry - // if len(kv.UserMeta) > 0 { - // meta = kv.UserMeta[0] - // } - // switch meta { - // case BitCompletePosting, BitEmptyPosting: - // entry = (&badger.Entry{ - // Key: kv.Key, - // Value: kv.Value, - // UserMeta: meta, - // }).WithDiscard() - // default: - k := kv.key - v := kv.value - var meta byte - var entry *badger.Entry - - entry = &badger.Entry{ - Key: k, - Value: v, - UserMeta: meta, - } - // UserMeta: meta, - // } - entry.Key = y.KeyWithTs(entry.Key, 1) //Same version for each key - entries = append(entries, entry) - } - - wg.Add(1) - err := db.BatchSetAsync(entries, func(err error) { - defer wg.Done() - if err != nil { - // TODO: Handle error - glog.Error(err) - return - } - }) + // // UserMeta: meta, + // // } + // entry.Key = y.KeyWithTs(entry.Key, 1) //Same version for each key + // entries = append(entries, entry) + // } + + // wg.Add(1) + // err := db.BatchSetAsync(entries, func(err error) { + // defer wg.Done() + // if err != nil { + // // TODO: Handle error + // glog.Error(err) + // return + // } + // }) require.NoError(b, err) - wg.Wait() + // wg.Wait() } }) From 46074c5bffbad539e459aad9db639e0e1e9f362e Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Tue, 28 Apr 2020 17:52:40 +0530 Subject: [PATCH 30/50] Fix minor issue --- posting/index.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index 278c52eb7ea..8b6044efa62 100644 --- a/posting/index.go +++ b/posting/index.go @@ -627,7 +627,11 @@ func (r *rebuilder) Run(ctx context.Context) error { } stream.Send = func(kvList *bpb.KVList) error { for _, kv := range kvList.Kv { - if err := tmpWriter.SetEntryAt((&badger.Entry{Key: kv.Key, Value: kv.Value, UserMeta: kv.UserMeta[0]}).WithDiscard(), kv.Version); err != nil { + e := &badger.Entry{Key: kv.Key, Value: kv.Value} + if len(kv.UserMeta) > 0 { + e.UserMeta = kv.UserMeta[0] + } + if err := tmpWriter.SetEntryAt(e, kv.Version); err != nil { return errors.Wrap(err, "error setting entries in temp badger") } } From 8ee34e6d36250dafd4ecccd7881e0da57ad57689 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Tue, 28 Apr 2020 17:53:59 +0530 Subject: [PATCH 31/50] Update badger --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index f1d21ccc641..62322f56866 100644 --- a/go.mod +++ b/go.mod @@ -16,7 +16,7 @@ require ( github.com/blevesearch/segment v0.0.0-20160915185041-762005e7a34f // indirect github.com/blevesearch/snowballstem v0.0.0-20180110192139-26b06a2c243d // indirect github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd - github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200421062606-cddf7c03451c + github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200427150514-72a4b44363dc github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 github.com/dgraph-io/ristretto v0.0.2 github.com/dgrijalva/jwt-go v3.2.0+incompatible diff --git a/go.sum b/go.sum index b5825910f39..c045c359f62 100644 --- a/go.sum +++ b/go.sum @@ -76,8 +76,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/badger v1.6.0 h1:DshxFxZWXUcO0xX476VJC07Xsr6ZCBVRHKZ93Oh7Evo= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200421062606-cddf7c03451c h1:IXsBFBQ0g5JlPfu+3HotLmkej2xgyrkKceWmFlXSYIQ= -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200421062606-cddf7c03451c/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200427150514-72a4b44363dc h1:R8/a+ptLEDK4gqEFXaPMzNwvQgS9qjsvaxnJlwRp3hI= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200427150514-72a4b44363dc/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 h1:DTgOrw91nMIukDm/WEvdobPLl0LgeDd/JE66+24jBks= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453/go.mod h1:Co+FwJrnndSrPORO8Gdn20dR7FPTfmXr0W/su0Ve/Ig= github.com/dgraph-io/ristretto v0.0.2-0.20200115201040-8f368f2f2ab3 h1:MQLRM35Pp0yAyBYksjbj1nZI/w6eyRY/mWoM1sFf4kU= From 0b7e39ac913cdb980572f1d1d999576525ed6936 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Thu, 30 Apr 2020 12:56:02 +0530 Subject: [PATCH 32/50] Update badger --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index f72668b2edd..d0d95eb831e 100644 --- a/go.mod +++ b/go.mod @@ -16,7 +16,7 @@ require ( github.com/blevesearch/segment v0.0.0-20160915185041-762005e7a34f // indirect github.com/blevesearch/snowballstem v0.0.0-20180110192139-26b06a2c243d // indirect github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd - github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200427150514-72a4b44363dc + github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430072010-59cb8e59de19 github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 github.com/dgraph-io/ristretto v0.0.2 github.com/dgrijalva/jwt-go v3.2.0+incompatible diff --git a/go.sum b/go.sum index e0a29643738..8e261785413 100644 --- a/go.sum +++ b/go.sum @@ -76,8 +76,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/badger v1.6.0 h1:DshxFxZWXUcO0xX476VJC07Xsr6ZCBVRHKZ93Oh7Evo= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200427150514-72a4b44363dc h1:R8/a+ptLEDK4gqEFXaPMzNwvQgS9qjsvaxnJlwRp3hI= -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200427150514-72a4b44363dc/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430072010-59cb8e59de19 h1:dwCsQopAdyHjI8AfZqQsvg43GFllyl5J2Bk8/YbyIeM= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430072010-59cb8e59de19/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 h1:DTgOrw91nMIukDm/WEvdobPLl0LgeDd/JE66+24jBks= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453/go.mod h1:Co+FwJrnndSrPORO8Gdn20dR7FPTfmXr0W/su0Ve/Ig= github.com/dgraph-io/ristretto v0.0.2-0.20200115201040-8f368f2f2ab3 h1:MQLRM35Pp0yAyBYksjbj1nZI/w6eyRY/mWoM1sFf4kU= From 2818574c64040f05c929eb066c942244ea166250 Mon Sep 17 00:00:00 2001 From: Ibrahim Jarif Date: Thu, 30 Apr 2020 17:47:47 +0530 Subject: [PATCH 33/50] Update badger --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index d0d95eb831e..7535b69d302 100644 --- a/go.mod +++ b/go.mod @@ -16,7 +16,7 @@ require ( github.com/blevesearch/segment v0.0.0-20160915185041-762005e7a34f // indirect github.com/blevesearch/snowballstem v0.0.0-20180110192139-26b06a2c243d // indirect github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd - github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430072010-59cb8e59de19 + github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430101140-5d19cc727d87 github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 github.com/dgraph-io/ristretto v0.0.2 github.com/dgrijalva/jwt-go v3.2.0+incompatible diff --git a/go.sum b/go.sum index 8e261785413..275b757169e 100644 --- a/go.sum +++ b/go.sum @@ -76,8 +76,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/badger v1.6.0 h1:DshxFxZWXUcO0xX476VJC07Xsr6ZCBVRHKZ93Oh7Evo= github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430072010-59cb8e59de19 h1:dwCsQopAdyHjI8AfZqQsvg43GFllyl5J2Bk8/YbyIeM= -github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430072010-59cb8e59de19/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430101140-5d19cc727d87 h1:FsCl1Yg3KVeYEzE7QlvpYg9WnySjLA5vbS9TlmEeUP8= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430101140-5d19cc727d87/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 h1:DTgOrw91nMIukDm/WEvdobPLl0LgeDd/JE66+24jBks= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453/go.mod h1:Co+FwJrnndSrPORO8Gdn20dR7FPTfmXr0W/su0Ve/Ig= github.com/dgraph-io/ristretto v0.0.2-0.20200115201040-8f368f2f2ab3 h1:MQLRM35Pp0yAyBYksjbj1nZI/w6eyRY/mWoM1sFf4kU= From 90e0b2222228b16d2c758a8e4f79199cccb89541 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Sun, 3 May 2020 17:06:35 +0530 Subject: [PATCH 34/50] Add batchwriter inplace of txn writer --- posting/index.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index 8b6044efa62..3c2b0692a73 100644 --- a/posting/index.go +++ b/posting/index.go @@ -689,7 +689,10 @@ func (r *rebuilder) Run(ctx context.Context) error { // which occurred before this schema mutation. //err := writer.SetAt(kv.Key, kv.Value, BitCompletePosting, r.startTs) - if err := writer.SetEntryAt((&badger.Entry{Key: kv.Key, Value: kv.Value, UserMeta: BitCompletePosting}).WithDiscard(), r.startTs); err != nil { + if err := writer.SetEntryAt( + (&badger.Entry{Key: kv.Key, + Value: kv.Value, + UserMeta: BitCompletePosting}).WithDiscard(), r.startTs); err != nil { return errors.Wrap(err, "error in writing index to pstore") } } From 9b39c31bad1d429153450dcd6a6caecb3139fbdf Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 11:54:59 +0530 Subject: [PATCH 35/50] Add write batch in snapshot.go --- worker/snapshot.go | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/worker/snapshot.go b/worker/snapshot.go index 7cd78948414..2a02391cf79 100644 --- a/worker/snapshot.go +++ b/worker/snapshot.go @@ -65,7 +65,8 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { writer = sw } else { - writer = posting.NewTxnWriter(pstore) + //writer = posting.NewTxnWriter(pstore) + writer = pstore.NewManagedWriteBatch() } // We can use count to check the number of posting lists returned in tests. @@ -86,9 +87,19 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { } glog.V(1).Infof("Received a batch of %d keys. Total so far: %d\n", len(kvs.Kv), count) - if err := writer.Write(&bpb.KVList{Kv: kvs.Kv}); err != nil { - return 0, err + for _, kv := range kvs.Kv { + e := &badger.Entry{Key: kv.Key, Value: kv.Value} + if len(kv.UserMeta) > 0 { + e.UserMeta = kv.UserMeta[0] + } + if err := writer.SetEntryAt(e, kv.Version); err != nil { + return 0, err + } } + // if err := writer.Write(&bpb.KVList{Kv: kvs.Kv}); err != nil { + // return 0, err + // } + count += len(kvs.Kv) } if err := writer.Flush(); err != nil { From b9c3aa89dc714b5f8ddd69237dbd73e8f4ca4c55 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 13:41:17 +0530 Subject: [PATCH 36/50] Defined new struct to handle interface definition --- worker/snapshot.go | 41 +++++++++++++++++++++++++++-------------- 1 file changed, 27 insertions(+), 14 deletions(-) diff --git a/worker/snapshot.go b/worker/snapshot.go index 2a02391cf79..2c1f50333b2 100644 --- a/worker/snapshot.go +++ b/worker/snapshot.go @@ -38,6 +38,30 @@ type badgerWriter interface { Write(kvs *bpb.KVList) error Flush() error } +type newwriteBatch struct { + wb *badger.WriteBatch +} + +func newWriteBatchWriter(db *badger.DB) *newwriteBatch { + return &newwriteBatch{wb: db.NewManagedWriteBatch()} +} + +func (nwb *newwriteBatch) Write(kvs *bpb.KVList) error { + for _, kv := range kvs.Kv { + e := &badger.Entry{Key: kv.Key, Value: kv.Value} + if len(kv.UserMeta) > 0 { + e.UserMeta = kv.UserMeta[0] + } + if err := nwb.wb.SetEntryAt(e, kv.Version); err != nil { + return err + } + } + return nil +} + +func (nwb *newwriteBatch) Flush() error { + return nwb.wb.Flush() +} // populateSnapshot gets data for a shard from the leader and writes it to BadgerDB on the follower. func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { @@ -62,11 +86,9 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { if err := sw.Prepare(); err != nil { return 0, err } - writer = sw } else { - //writer = posting.NewTxnWriter(pstore) - writer = pstore.NewManagedWriteBatch() + writer = newWriteBatchWriter(pstore) } // We can use count to check the number of posting lists returned in tests. @@ -87,18 +109,9 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { } glog.V(1).Infof("Received a batch of %d keys. Total so far: %d\n", len(kvs.Kv), count) - for _, kv := range kvs.Kv { - e := &badger.Entry{Key: kv.Key, Value: kv.Value} - if len(kv.UserMeta) > 0 { - e.UserMeta = kv.UserMeta[0] - } - if err := writer.SetEntryAt(e, kv.Version); err != nil { - return 0, err - } + if err := writer.Write(&bpb.KVList{Kv: kvs.Kv}); err != nil { + return 0, err } - // if err := writer.Write(&bpb.KVList{Kv: kvs.Kv}); err != nil { - // return 0, err - // } count += len(kvs.Kv) } From baf5eaf7c72359b6103e166a1689e14a2ab407ad Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 13:57:33 +0530 Subject: [PATCH 37/50] Modify writer_test.go to account for new API --- posting/writer_test.go | 57 +++++------------------------------------- 1 file changed, 6 insertions(+), 51 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 1c7e2b0dda2..c8199fff43d 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -88,57 +88,12 @@ func BenchmarkWriter(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - // var wg sync.WaitGroup - // var entries []*badger.Entry - // // batch := db.NewWriteBatchAt(1) - // // for _, typ := range KVList { - // // k := typ.key - // // v := typ.value - // // batch.Set(k, v) - // // } - // // require.NoError(b, batch.Flush()) - // for _, kv := range KVList { - // // var meta byte - // // var entry *badger.Entry - // // if len(kv.UserMeta) > 0 { - // // meta = kv.UserMeta[0] - // // } - // // switch meta { - // // case BitCompletePosting, BitEmptyPosting: - // // entry = (&badger.Entry{ - // // Key: kv.Key, - // // Value: kv.Value, - // // UserMeta: meta, - // // }).WithDiscard() - // // default: - // k := kv.key - // v := kv.value - // var meta byte - // var entry *badger.Entry - - // entry = &badger.Entry{ - // Key: k, - // Value: v, - // UserMeta: meta, - // } - // // UserMeta: meta, - // // } - // entry.Key = y.KeyWithTs(entry.Key, 1) //Same version for each key - // entries = append(entries, entry) - // } - - // wg.Add(1) - // err := db.BatchSetAsync(entries, func(err error) { - // defer wg.Done() - // if err != nil { - // // TODO: Handle error - // glog.Error(err) - // return - // } - // }) - require.NoError(b, err) - // wg.Wait() - + wb := db.NewManagedWriteBatch() + for _, typ := range KVList { + e := &badger.Entry{Key: typ.key, Value: typ.value} + wb.SetEntryAt(e, 1) + } + require.NoError(b, wb.Flush()) } }) } From af791219896d50fc2d87c8adfeacb0a4c94fa713 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 15:58:52 +0530 Subject: [PATCH 38/50] Code clean-up --- go.mod | 4 +--- posting/index.go | 9 --------- worker/draft.go | 2 +- 3 files changed, 2 insertions(+), 13 deletions(-) diff --git a/go.mod b/go.mod index 7535b69d302..4d943b37222 100644 --- a/go.mod +++ b/go.mod @@ -67,6 +67,4 @@ require ( gopkg.in/DataDog/dd-trace-go.v1 v1.13.1 // indirect gopkg.in/ini.v1 v1.48.0 // indirect gopkg.in/yaml.v2 v2.2.4 -) - -//replace github.com/dgraph-io/badger/v2 => /home/alvis/go/src/github.com/dgraph-io/badger +) \ No newline at end of file diff --git a/posting/index.go b/posting/index.go index 3c2b0692a73..796f62db02b 100644 --- a/posting/index.go +++ b/posting/index.go @@ -573,10 +573,6 @@ func (r *rebuilder) Run(ctx context.Context) error { // We set it to 1 in case there are no keys found and NewStreamAt is called with ts=0. var counter uint64 = 1 - // TODO(Aman): Replace TxnWriter with WriteBatch. While we do that we should ensure that - // WriteBatch has a mechanism for throttling. Also, find other places where TxnWriter - // could be replaced with WriteBatch in the code - //tmpWriter := NewTxnWriter(tmpDB) tmpWriter := tmpDB.NewManagedWriteBatch() stream := pstore.NewStreamAt(r.startTs) stream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (1/2):", r.attr) @@ -635,9 +631,6 @@ func (r *rebuilder) Run(ctx context.Context) error { return errors.Wrap(err, "error setting entries in temp badger") } } - // if err := tmpWriter.Write(kvList); err != nil { - // return errors.Wrap(err, "error setting entries in temp badger") - // } return nil } @@ -660,7 +653,6 @@ func (r *rebuilder) Run(ctx context.Context) error { r.attr, time.Since(start)) }() - //writer := NewTxnWriter(pstore) writer := pstore.NewManagedWriteBatch() tmpStream := tmpDB.NewStreamAt(counter) tmpStream.LogPrefix = fmt.Sprintf("Rebuilding index for predicate %s (2/2):", r.attr) @@ -687,7 +679,6 @@ func (r *rebuilder) Run(ctx context.Context) error { // We choose to write the PL at r.startTs, so it won't be read by txns, // which occurred before this schema mutation. - //err := writer.SetAt(kv.Key, kv.Value, BitCompletePosting, r.startTs) if err := writer.SetEntryAt( (&badger.Entry{Key: kv.Key, diff --git a/worker/draft.go b/worker/draft.go index d364142d101..4398ca98550 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -711,7 +711,7 @@ func (n *node) processApplyCh() { } } -// TODO(Anurag): Are we using pkey? Remove if unused. +// TODO(Anurag - 4 May 2020): Are we using pkey? Remove if unused. func (n *node) commitOrAbort(pkey string, delta *pb.OracleDelta) error { // First let's commit all mutations to disk. writer := posting.NewTxnWriter(pstore) From 59f29d156b645cd07173d9631e00fb0a557e50f3 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 16:24:00 +0530 Subject: [PATCH 39/50] Add Write API for batch writer inside a function which accepts KVs list --- posting/index.go | 10 ++-------- worker/snapshot.go | 16 +++++----------- x/x.go | 17 ++++++++++++++++- 3 files changed, 23 insertions(+), 20 deletions(-) diff --git a/posting/index.go b/posting/index.go index 796f62db02b..a64d6bd63cb 100644 --- a/posting/index.go +++ b/posting/index.go @@ -622,14 +622,8 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - for _, kv := range kvList.Kv { - e := &badger.Entry{Key: kv.Key, Value: kv.Value} - if len(kv.UserMeta) > 0 { - e.UserMeta = kv.UserMeta[0] - } - if err := tmpWriter.SetEntryAt(e, kv.Version); err != nil { - return errors.Wrap(err, "error setting entries in temp badger") - } + if err := WriteBatchWriter(tmpWriter, kvList); err != nil { + return errors.Wrap(err, "error setting entries in temp badger") } return nil diff --git a/worker/snapshot.go b/worker/snapshot.go index 2c1f50333b2..0a2b87dc8a4 100644 --- a/worker/snapshot.go +++ b/worker/snapshot.go @@ -38,7 +38,7 @@ type badgerWriter interface { Write(kvs *bpb.KVList) error Flush() error } -type newwriteBatch struct { +type newWriteBatch struct { wb *badger.WriteBatch } @@ -46,20 +46,14 @@ func newWriteBatchWriter(db *badger.DB) *newwriteBatch { return &newwriteBatch{wb: db.NewManagedWriteBatch()} } -func (nwb *newwriteBatch) Write(kvs *bpb.KVList) error { - for _, kv := range kvs.Kv { - e := &badger.Entry{Key: kv.Key, Value: kv.Value} - if len(kv.UserMeta) > 0 { - e.UserMeta = kv.UserMeta[0] - } - if err := nwb.wb.SetEntryAt(e, kv.Version); err != nil { - return err - } +func (nwb *newWriteBatch) Write(kvs *bpb.KVList) error { + if err := WriteBatchWriter(nwb.wb, kvs); err != nil { + return err } return nil } -func (nwb *newwriteBatch) Flush() error { +func (nwb *newWriteBatch) Flush() error { return nwb.wb.Flush() } diff --git a/x/x.go b/x/x.go index 95c1c04474e..eee19827b29 100644 --- a/x/x.go +++ b/x/x.go @@ -38,10 +38,10 @@ import ( "time" "github.com/dgraph-io/badger/v2" + bpb "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" "github.com/dgraph-io/dgo/v200" "github.com/dgraph-io/dgo/v200/protos/api" - "github.com/golang/glog" "github.com/pkg/errors" "github.com/spf13/viper" @@ -850,6 +850,21 @@ func AskUserPassword(userid string, pwdType string, times int) (string, error) { return password, nil } +//WriteBatchWriter exposes Write API batch writer. +func WriteBatchWriter(writer *badger.WriteBatch, kvList *bpb.KVList) error { + for _, kv := range kvList.Kv { + e := &badger.Entry{Key: kv.Key, Value: kv.Value} + if len(kv.UserMeta) > 0 { + e.UserMeta = kv.UserMeta[0] + } + if err := writer.SetEntryAt(e, kv.Version); err != nil { + return err + } + } + return nil + +} + // GetPassAndLogin uses the given credentials and client to perform the login operation. func GetPassAndLogin(dg *dgo.Dgraph, opt *CredOpt) error { password := opt.Conf.GetString(opt.PasswordOpt) From c5ee1e6391b10a152900ad039408ed0c2c3ca0e2 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 17:08:02 +0530 Subject: [PATCH 40/50] bugfixes --- posting/index.go | 3 +-- worker/snapshot.go | 7 ++++--- x/x.go | 1 + 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/posting/index.go b/posting/index.go index a64d6bd63cb..03a0851c0d8 100644 --- a/posting/index.go +++ b/posting/index.go @@ -622,7 +622,7 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - if err := WriteBatchWriter(tmpWriter, kvList); err != nil { + if err := x.WriteBatchWriter(tmpWriter, kvList); err != nil { return errors.Wrap(err, "error setting entries in temp badger") } @@ -673,7 +673,6 @@ func (r *rebuilder) Run(ctx context.Context) error { // We choose to write the PL at r.startTs, so it won't be read by txns, // which occurred before this schema mutation. - if err := writer.SetEntryAt( (&badger.Entry{Key: kv.Key, Value: kv.Value, diff --git a/worker/snapshot.go b/worker/snapshot.go index 0a2b87dc8a4..ac1fdfa03ae 100644 --- a/worker/snapshot.go +++ b/worker/snapshot.go @@ -27,6 +27,7 @@ import ( "github.com/dgraph-io/dgraph/conn" "github.com/dgraph-io/dgraph/posting" "github.com/dgraph-io/dgraph/protos/pb" + "github.com/dgraph-io/dgraph/x" ) const ( @@ -42,12 +43,12 @@ type newWriteBatch struct { wb *badger.WriteBatch } -func newWriteBatchWriter(db *badger.DB) *newwriteBatch { +func newWriteBatchWriter(db *badger.DB) *newWriteBatch { return &newwriteBatch{wb: db.NewManagedWriteBatch()} } func (nwb *newWriteBatch) Write(kvs *bpb.KVList) error { - if err := WriteBatchWriter(nwb.wb, kvs); err != nil { + if err := x.WriteBatchWriter(nwb.wb, kvs); err != nil { return err } return nil @@ -80,6 +81,7 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { if err := sw.Prepare(); err != nil { return 0, err } + writer = sw } else { writer = newWriteBatchWriter(pstore) @@ -106,7 +108,6 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { if err := writer.Write(&bpb.KVList{Kv: kvs.Kv}); err != nil { return 0, err } - count += len(kvs.Kv) } if err := writer.Flush(); err != nil { diff --git a/x/x.go b/x/x.go index eee19827b29..0cbf824a9f9 100644 --- a/x/x.go +++ b/x/x.go @@ -854,6 +854,7 @@ func AskUserPassword(userid string, pwdType string, times int) (string, error) { func WriteBatchWriter(writer *badger.WriteBatch, kvList *bpb.KVList) error { for _, kv := range kvList.Kv { e := &badger.Entry{Key: kv.Key, Value: kv.Value} + // if len(kv.UserMeta) > 0 { e.UserMeta = kv.UserMeta[0] } From 0778ce7f988eccc757135aa0402473c9ce9740f9 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 18:29:24 +0530 Subject: [PATCH 41/50] Add multi-threaded version of batchwrite for benchmarking --- posting/writer_test.go | 81 +++++++++++++++++++++++++++++++++++++++++- 1 file changed, 80 insertions(+), 1 deletion(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index c8199fff43d..91e6283ea42 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -20,6 +20,7 @@ import ( "io/ioutil" "math" "os" + "sync" "testing" "github.com/dgraph-io/badger/v2" @@ -42,6 +43,27 @@ func BenchmarkWriter(b *testing.B) { return KVList } + writeInBagder := func(db *badger.DB, KVList []kv, wg *sync.WaitGroup) { + defer wg.Done() + wb := db.NewManagedWriteBatch() + for _, typ := range KVList { + e := &badger.Entry{Key: typ.key, Value: typ.value} + wb.SetEntryAt(e, 1) + } + require.NoError(b, wb.Flush()) + + } + + writeInBagder2 := func(wb *badger.WriteBatch, KVList []kv, wg *sync.WaitGroup) { + defer wg.Done() + + for _, typ := range KVList { + e := &badger.Entry{Key: typ.key, Value: typ.value} + wb.SetEntryAt(e, 1) + } + + } + dbOpts := badger.DefaultOptions(""). WithLogger(nil). WithSyncWrites(false). @@ -50,6 +72,7 @@ func BenchmarkWriter(b *testing.B) { KVList := createKVList() + //Vanilla TxnWriter b.Run("TxnWriter", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) @@ -73,7 +96,8 @@ func BenchmarkWriter(b *testing.B) { } }) - b.Run("WriteBatch", func(b *testing.B) { + //Single threaded BatchWriter + b.Run("WriteBatch1", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) defer os.RemoveAll(tmpIndexDir) @@ -96,4 +120,59 @@ func BenchmarkWriter(b *testing.B) { require.NoError(b, wb.Flush()) } }) + //Multi threaded Batchwriter with thread contention in WriteBatch + b.Run("WriteBatchMultThread1", func(b *testing.B) { + tmpIndexDir, err := ioutil.TempDir("", "dgraph") + require.NoError(b, err) + defer os.RemoveAll(tmpIndexDir) + + dbOpts.Dir = tmpIndexDir + dbOpts.ValueDir = tmpIndexDir + + var db, err2 = badger.OpenManaged(dbOpts) + require.NoError(b, err2) + defer db.Close() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + var wg sync.WaitGroup + wg.Add(5) + go writeInBagder(db, KVList[:10000], &wg) + go writeInBagder(db, KVList[10001:20000], &wg) + go writeInBagder(db, KVList[20001:30000], &wg) + go writeInBagder(db, KVList[30001:40000], &wg) + go writeInBagder(db, KVList[40001:], &wg) + wg.Wait() + + } + }) + //Multi threaded Batchwriter with thread contention in SetEntry + b.Run("WriteBatchMultThread2", func(b *testing.B) { + tmpIndexDir, err := ioutil.TempDir("", "dgraph") + require.NoError(b, err) + defer os.RemoveAll(tmpIndexDir) + + dbOpts.Dir = tmpIndexDir + dbOpts.ValueDir = tmpIndexDir + + var db, err2 = badger.OpenManaged(dbOpts) + require.NoError(b, err2) + defer db.Close() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + var wg sync.WaitGroup + wg.Add(5) + wb := db.NewManagedWriteBatch() + go writeInBagder2(wb, KVList[:10000], &wg) + go writeInBagder2(wb, KVList[10001:20000], &wg) + go writeInBagder2(wb, KVList[20001:30000], &wg) + go writeInBagder2(wb, KVList[30001:40000], &wg) + go writeInBagder2(wb, KVList[40001:], &wg) + wg.Wait() + require.NoError(b, wb.Flush()) + } + }) } From 014e069689ce0d1fb5c9cae3f944f73ea225f9ca Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 20:57:10 +0530 Subject: [PATCH 42/50] Address minor comments on PR --- go.sum | 1 + posting/index.go | 2 +- posting/writer_test.go | 33 ++++++++++++++++++--------------- worker/snapshot.go | 14 +++++++------- x/x.go | 4 ++-- 5 files changed, 29 insertions(+), 25 deletions(-) diff --git a/go.sum b/go.sum index 275b757169e..7124d3ab3fa 100644 --- a/go.sum +++ b/go.sum @@ -342,6 +342,7 @@ github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnIn github.com/spf13/viper v1.3.2 h1:VUFqw5KcqRf7i70GOzW7N+Q7+gxVBkSSqiXB12+JQ4M= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.1/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= diff --git a/posting/index.go b/posting/index.go index 03a0851c0d8..21a3754433d 100644 --- a/posting/index.go +++ b/posting/index.go @@ -622,7 +622,7 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - if err := x.WriteBatchWriter(tmpWriter, kvList); err != nil { + if err := x.BulkWriteKVsBatchWriter(tmpWriter, kvList); err != nil { return errors.Wrap(err, "error setting entries in temp badger") } diff --git a/posting/writer_test.go b/posting/writer_test.go index 91e6283ea42..1611b94dd5d 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -43,18 +43,19 @@ func BenchmarkWriter(b *testing.B) { return KVList } - writeInBagder := func(db *badger.DB, KVList []kv, wg *sync.WaitGroup) { + writeInBadger := func(db *badger.DB, KVList []kv, wg *sync.WaitGroup) { defer wg.Done() wb := db.NewManagedWriteBatch() for _, typ := range KVList { e := &badger.Entry{Key: typ.key, Value: typ.value} - wb.SetEntryAt(e, 1) + err := wb.SetEntryAt(e, 1) + require.NoError(b, err) } require.NoError(b, wb.Flush()) } - writeInBagder2 := func(wb *badger.WriteBatch, KVList []kv, wg *sync.WaitGroup) { + writeInBadger2 := func(wb *badger.WriteBatch, KVList []kv, wg *sync.WaitGroup) { defer wg.Done() for _, typ := range KVList { @@ -90,7 +91,8 @@ func BenchmarkWriter(b *testing.B) { for _, typ := range KVList { k := typ.key v := typ.value - w.SetAt(k, v, BitSchemaPosting, 1) + err := w.SetAt(k, v, BitSchemaPosting, 1) + require.NoError(b, err) } require.NoError(b, w.Flush()) @@ -115,7 +117,8 @@ func BenchmarkWriter(b *testing.B) { wb := db.NewManagedWriteBatch() for _, typ := range KVList { e := &badger.Entry{Key: typ.key, Value: typ.value} - wb.SetEntryAt(e, 1) + err := wb.SetEntryAt(e, 1) + require.NoError(b, err) } require.NoError(b, wb.Flush()) } @@ -138,11 +141,11 @@ func BenchmarkWriter(b *testing.B) { for i := 0; i < b.N; i++ { var wg sync.WaitGroup wg.Add(5) - go writeInBagder(db, KVList[:10000], &wg) - go writeInBagder(db, KVList[10001:20000], &wg) - go writeInBagder(db, KVList[20001:30000], &wg) - go writeInBagder(db, KVList[30001:40000], &wg) - go writeInBagder(db, KVList[40001:], &wg) + go writeInBadger(db, KVList[:10000], &wg) + go writeInBadger(db, KVList[10001:20000], &wg) + go writeInBadger(db, KVList[20001:30000], &wg) + go writeInBadger(db, KVList[30001:40000], &wg) + go writeInBadger(db, KVList[40001:], &wg) wg.Wait() } @@ -166,11 +169,11 @@ func BenchmarkWriter(b *testing.B) { var wg sync.WaitGroup wg.Add(5) wb := db.NewManagedWriteBatch() - go writeInBagder2(wb, KVList[:10000], &wg) - go writeInBagder2(wb, KVList[10001:20000], &wg) - go writeInBagder2(wb, KVList[20001:30000], &wg) - go writeInBagder2(wb, KVList[30001:40000], &wg) - go writeInBagder2(wb, KVList[40001:], &wg) + go writeInBadger2(wb, KVList[:10000], &wg) + go writeInBadger2(wb, KVList[10001:20000], &wg) + go writeInBadger2(wb, KVList[20001:30000], &wg) + go writeInBadger2(wb, KVList[30001:40000], &wg) + go writeInBadger2(wb, KVList[40001:], &wg) wg.Wait() require.NoError(b, wb.Flush()) } diff --git a/worker/snapshot.go b/worker/snapshot.go index ac1fdfa03ae..548942775cb 100644 --- a/worker/snapshot.go +++ b/worker/snapshot.go @@ -39,22 +39,22 @@ type badgerWriter interface { Write(kvs *bpb.KVList) error Flush() error } -type newWriteBatch struct { +type newBatchWriter struct { wb *badger.WriteBatch } -func newWriteBatchWriter(db *badger.DB) *newWriteBatch { - return &newwriteBatch{wb: db.NewManagedWriteBatch()} +func writeBatchWriter(db *badger.DB) *newBatchWriter { + return &newBatchWriter{wb: db.NewManagedWriteBatch()} } -func (nwb *newWriteBatch) Write(kvs *bpb.KVList) error { - if err := x.WriteBatchWriter(nwb.wb, kvs); err != nil { +func (nwb *newBatchWriter) Write(kvs *bpb.KVList) error { + if err := x.BulkWriteKVsBatchWriter(nwb.wb, kvs); err != nil { return err } return nil } -func (nwb *newWriteBatch) Flush() error { +func (nwb *newBatchWriter) Flush() error { return nwb.wb.Flush() } @@ -84,7 +84,7 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { writer = sw } else { - writer = newWriteBatchWriter(pstore) + writer = writeBatchWriter(pstore) } // We can use count to check the number of posting lists returned in tests. diff --git a/x/x.go b/x/x.go index 0cbf824a9f9..dc235f5e43f 100644 --- a/x/x.go +++ b/x/x.go @@ -850,8 +850,8 @@ func AskUserPassword(userid string, pwdType string, times int) (string, error) { return password, nil } -//WriteBatchWriter exposes Write API batch writer. -func WriteBatchWriter(writer *badger.WriteBatch, kvList *bpb.KVList) error { +//BulkWriteKVsBatchWriter exposes Write API batch writer. +func BulkWriteKVsBatchWriter(writer *badger.WriteBatch, kvList *bpb.KVList) error { for _, kv := range kvList.Kv { e := &badger.Entry{Key: kv.Key, Value: kv.Value} // From cb8564f6b81f9798bfc89a6c0f1b1584642a12ba Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 22:14:00 +0530 Subject: [PATCH 43/50] Split entry in two lines --- posting/index.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/posting/index.go b/posting/index.go index 21a3754433d..d790a7ffe4d 100644 --- a/posting/index.go +++ b/posting/index.go @@ -673,10 +673,8 @@ func (r *rebuilder) Run(ctx context.Context) error { // We choose to write the PL at r.startTs, so it won't be read by txns, // which occurred before this schema mutation. - if err := writer.SetEntryAt( - (&badger.Entry{Key: kv.Key, - Value: kv.Value, - UserMeta: BitCompletePosting}).WithDiscard(), r.startTs); err != nil { + e := &badger.Entry{Key: kv.Key, Value: kv.Value, UserMeta: BitCompletePosting} + if err := writer.SetEntryAt(e.WithDiscard(), r.startTs); err != nil { return errors.Wrap(err, "error in writing index to pstore") } } From 12393eed47acb8c378aa2a10ce466c8208c6abaf Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Mon, 4 May 2020 23:11:37 +0530 Subject: [PATCH 44/50] Use Write API from badger --- posting/index.go | 2 +- worker/snapshot.go | 21 +-------------------- x/x.go | 17 ----------------- 3 files changed, 2 insertions(+), 38 deletions(-) diff --git a/posting/index.go b/posting/index.go index d790a7ffe4d..7da3844611b 100644 --- a/posting/index.go +++ b/posting/index.go @@ -622,7 +622,7 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } stream.Send = func(kvList *bpb.KVList) error { - if err := x.BulkWriteKVsBatchWriter(tmpWriter, kvList); err != nil { + if err := tmpWriter.Write(kvList); err != nil { return errors.Wrap(err, "error setting entries in temp badger") } diff --git a/worker/snapshot.go b/worker/snapshot.go index 548942775cb..518f8382abd 100644 --- a/worker/snapshot.go +++ b/worker/snapshot.go @@ -27,7 +27,6 @@ import ( "github.com/dgraph-io/dgraph/conn" "github.com/dgraph-io/dgraph/posting" "github.com/dgraph-io/dgraph/protos/pb" - "github.com/dgraph-io/dgraph/x" ) const ( @@ -39,24 +38,6 @@ type badgerWriter interface { Write(kvs *bpb.KVList) error Flush() error } -type newBatchWriter struct { - wb *badger.WriteBatch -} - -func writeBatchWriter(db *badger.DB) *newBatchWriter { - return &newBatchWriter{wb: db.NewManagedWriteBatch()} -} - -func (nwb *newBatchWriter) Write(kvs *bpb.KVList) error { - if err := x.BulkWriteKVsBatchWriter(nwb.wb, kvs); err != nil { - return err - } - return nil -} - -func (nwb *newBatchWriter) Flush() error { - return nwb.wb.Flush() -} // populateSnapshot gets data for a shard from the leader and writes it to BadgerDB on the follower. func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { @@ -84,7 +65,7 @@ func (n *node) populateSnapshot(snap pb.Snapshot, pl *conn.Pool) (int, error) { writer = sw } else { - writer = writeBatchWriter(pstore) + writer = pstore.NewManagedWriteBatch() } // We can use count to check the number of posting lists returned in tests. diff --git a/x/x.go b/x/x.go index dc235f5e43f..b2bd9f1f770 100644 --- a/x/x.go +++ b/x/x.go @@ -38,7 +38,6 @@ import ( "time" "github.com/dgraph-io/badger/v2" - bpb "github.com/dgraph-io/badger/v2/pb" "github.com/dgraph-io/badger/v2/y" "github.com/dgraph-io/dgo/v200" "github.com/dgraph-io/dgo/v200/protos/api" @@ -850,22 +849,6 @@ func AskUserPassword(userid string, pwdType string, times int) (string, error) { return password, nil } -//BulkWriteKVsBatchWriter exposes Write API batch writer. -func BulkWriteKVsBatchWriter(writer *badger.WriteBatch, kvList *bpb.KVList) error { - for _, kv := range kvList.Kv { - e := &badger.Entry{Key: kv.Key, Value: kv.Value} - // - if len(kv.UserMeta) > 0 { - e.UserMeta = kv.UserMeta[0] - } - if err := writer.SetEntryAt(e, kv.Version); err != nil { - return err - } - } - return nil - -} - // GetPassAndLogin uses the given credentials and client to perform the login operation. func GetPassAndLogin(dg *dgo.Dgraph, opt *CredOpt) error { password := opt.Conf.GetString(opt.PasswordOpt) From d21e7cfa7ecb4fd9fb8c55f7247932959c877a1f Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 5 May 2020 00:03:35 +0530 Subject: [PATCH 45/50] Temp changes to run benchmarks --- go.mod | 3 ++- posting/index.go | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 4d943b37222..4b82f021860 100644 --- a/go.mod +++ b/go.mod @@ -67,4 +67,5 @@ require ( gopkg.in/DataDog/dd-trace-go.v1 v1.13.1 // indirect gopkg.in/ini.v1 v1.48.0 // indirect gopkg.in/yaml.v2 v2.2.4 -) \ No newline at end of file +) +replace github.com/dgraph-io/badger/v2 => /home/alvis/go/src/github.com/dgraph-io/badger \ No newline at end of file diff --git a/posting/index.go b/posting/index.go index 7da3844611b..225e6a4505a 100644 --- a/posting/index.go +++ b/posting/index.go @@ -638,7 +638,8 @@ func (r *rebuilder) Run(ctx context.Context) error { } glog.V(1).Infof("Rebuilding index for predicate %s: building temp index took: %v\n", r.attr, time.Since(start)) - + fmt.Printf("Anurag: Rebuilding index for predicate %s: building temp index took: %v\n", + r.attr, time.Since(start)) // Now we write all the created posting lists to disk. glog.V(1).Infof("Rebuilding index for predicate %s: writing index to badger", r.attr) start = time.Now() From 4a3e5fb5f8165244907c996d4df5e71535a418d3 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 5 May 2020 13:28:06 +0530 Subject: [PATCH 46/50] Add a TODO and update writer_test.go --- posting/index.go | 2 ++ posting/writer_test.go | 63 +++++++++++++++++++----------------------- 2 files changed, 30 insertions(+), 35 deletions(-) diff --git a/posting/index.go b/posting/index.go index 225e6a4505a..ab7040b44c1 100644 --- a/posting/index.go +++ b/posting/index.go @@ -667,6 +667,8 @@ func (r *rebuilder) Run(ctx context.Context) error { return &bpb.KVList{Kv: kvs}, nil } tmpStream.Send = func(kvList *bpb.KVList) error { + // TODO (Anurag): Instead of calling SetEntryAt everytime, we can filter KVList and call Write only once. + // SetEntryAt requries lock for every entry, whereas Write reduces lock contention. for _, kv := range kvList.Kv { if len(kv.Value) == 0 { continue diff --git a/posting/writer_test.go b/posting/writer_test.go index 1611b94dd5d..17e734c6cbb 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -25,42 +25,35 @@ import ( "github.com/dgraph-io/badger/v2" "github.com/dgraph-io/badger/v2/options" + bpb "github.com/dgraph-io/badger/v2/pb" "github.com/stretchr/testify/require" ) -type kv struct { - key []byte - value []byte -} - func BenchmarkWriter(b *testing.B) { - createKVList := func() []kv { - var KVList = []kv{} + createKVList := func() bpb.KVList { + var KVList bpb.KVList for i := 0; i < 50000; i++ { - n := kv{key: []byte(string(i)), value: []byte(string(i))} - KVList = append(KVList, n) + n := &bpb.KV{Key: []byte(string(i)), Value: []byte(string(i)), Version: 5} + KVList.Kv = append(KVList.Kv, n) } return KVList } - writeInBadger := func(db *badger.DB, KVList []kv, wg *sync.WaitGroup) { + writeInBadger := func(db *badger.DB, KVList *bpb.KVList, wg *sync.WaitGroup) { defer wg.Done() wb := db.NewManagedWriteBatch() - for _, typ := range KVList { - e := &badger.Entry{Key: typ.key, Value: typ.value} - err := wb.SetEntryAt(e, 1) - require.NoError(b, err) + if err := wb.Write(KVList); err != nil { + panic(err) } require.NoError(b, wb.Flush()) } - writeInBadger2 := func(wb *badger.WriteBatch, KVList []kv, wg *sync.WaitGroup) { + writeInBadger2 := func(wb *badger.WriteBatch, KVList *bpb.KVList, wg *sync.WaitGroup) { defer wg.Done() - for _, typ := range KVList { - e := &badger.Entry{Key: typ.key, Value: typ.value} - wb.SetEntryAt(e, 1) + if err := wb.Write(KVList); err != nil { + panic(err) } } @@ -88,9 +81,9 @@ func BenchmarkWriter(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { w := NewTxnWriter(db) - for _, typ := range KVList { - k := typ.key - v := typ.value + for _, typ := range KVList.Kv { + k := typ.Key + v := typ.Value err := w.SetAt(k, v, BitSchemaPosting, 1) require.NoError(b, err) } @@ -115,10 +108,8 @@ func BenchmarkWriter(b *testing.B) { for i := 0; i < b.N; i++ { wb := db.NewManagedWriteBatch() - for _, typ := range KVList { - e := &badger.Entry{Key: typ.key, Value: typ.value} - err := wb.SetEntryAt(e, 1) - require.NoError(b, err) + if err := wb.Write(&KVList); err != nil { + panic(err) } require.NoError(b, wb.Flush()) } @@ -141,11 +132,12 @@ func BenchmarkWriter(b *testing.B) { for i := 0; i < b.N; i++ { var wg sync.WaitGroup wg.Add(5) - go writeInBadger(db, KVList[:10000], &wg) - go writeInBadger(db, KVList[10001:20000], &wg) - go writeInBadger(db, KVList[20001:30000], &wg) - go writeInBadger(db, KVList[30001:40000], &wg) - go writeInBadger(db, KVList[40001:], &wg) + + go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[:10000]}, &wg) + go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[10001:20000]}, &wg) + go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[20001:30000]}, &wg) + go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[30001:40000]}, &wg) + go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[40001:]}, &wg) wg.Wait() } @@ -169,11 +161,12 @@ func BenchmarkWriter(b *testing.B) { var wg sync.WaitGroup wg.Add(5) wb := db.NewManagedWriteBatch() - go writeInBadger2(wb, KVList[:10000], &wg) - go writeInBadger2(wb, KVList[10001:20000], &wg) - go writeInBadger2(wb, KVList[20001:30000], &wg) - go writeInBadger2(wb, KVList[30001:40000], &wg) - go writeInBadger2(wb, KVList[40001:], &wg) + go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[:10000]}, &wg) + go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[10001:20000]}, &wg) + go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[20001:30000]}, &wg) + go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[30001:40000]}, &wg) + go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[40001:]}, &wg) + wg.Wait() require.NoError(b, wb.Flush()) } From 74136f093c9fa8b6f410ef3d80a6f9df35686680 Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 5 May 2020 21:07:11 +0530 Subject: [PATCH 47/50] Add single threaded version to tests to emulate stream behaviour --- posting/writer_test.go | 101 +++++++++++++++++++++++++++++++++++------ 1 file changed, 87 insertions(+), 14 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 17e734c6cbb..0475e6ef284 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -39,7 +39,8 @@ func BenchmarkWriter(b *testing.B) { return KVList } - writeInBadger := func(db *badger.DB, KVList *bpb.KVList, wg *sync.WaitGroup) { + // Creates separate writer for each thread + writeInBadgerMThreadsB := func(db *badger.DB, KVList *bpb.KVList, wg *sync.WaitGroup) { defer wg.Done() wb := db.NewManagedWriteBatch() if err := wb.Write(KVList); err != nil { @@ -49,13 +50,29 @@ func BenchmarkWriter(b *testing.B) { } - writeInBadger2 := func(wb *badger.WriteBatch, KVList *bpb.KVList, wg *sync.WaitGroup) { + // Resuses one writer for all threads + writeInBadgerMThreadsW := func(wb *badger.WriteBatch, KVList *bpb.KVList, wg *sync.WaitGroup) { defer wg.Done() if err := wb.Write(KVList); err != nil { panic(err) } + } + // Creates separate writer for each thread + writeInBadgerSingleThreadB := func(db *badger.DB, KVList *bpb.KVList) { + wb := db.NewManagedWriteBatch() + if err := wb.Write(KVList); err != nil { + panic(err) + } + + } + // Resuses one writer for all threads + writeInBadgerSingleThreadW := func(wb *badger.WriteBatch, KVList *bpb.KVList) { + if err := wb.Write(KVList); err != nil { + panic(err) + } + } dbOpts := badger.DefaultOptions(""). @@ -115,7 +132,7 @@ func BenchmarkWriter(b *testing.B) { } }) //Multi threaded Batchwriter with thread contention in WriteBatch - b.Run("WriteBatchMultThread1", func(b *testing.B) { + b.Run("WriteBatchMultThreadB", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) defer os.RemoveAll(tmpIndexDir) @@ -133,17 +150,73 @@ func BenchmarkWriter(b *testing.B) { var wg sync.WaitGroup wg.Add(5) - go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[:10000]}, &wg) - go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[10001:20000]}, &wg) - go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[20001:30000]}, &wg) - go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[30001:40000]}, &wg) - go writeInBadger(db, &bpb.KVList{Kv: KVList.Kv[40001:]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[:10000]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[10001:20000]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[20001:30000]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[30001:40000]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[40001:]}, &wg) wg.Wait() } }) //Multi threaded Batchwriter with thread contention in SetEntry - b.Run("WriteBatchMultThread2", func(b *testing.B) { + b.Run("WriteBatchMultThreadW", func(b *testing.B) { + tmpIndexDir, err := ioutil.TempDir("", "dgraph") + require.NoError(b, err) + defer os.RemoveAll(tmpIndexDir) + + dbOpts.Dir = tmpIndexDir + dbOpts.ValueDir = tmpIndexDir + + var db, err2 = badger.OpenManaged(dbOpts) + require.NoError(b, err2) + defer db.Close() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + var wg sync.WaitGroup + wg.Add(5) + wb := db.NewManagedWriteBatch() + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[:10000]}, &wg) + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[10001:20000]}, &wg) + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[20001:30000]}, &wg) + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[30001:40000]}, &wg) + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[40001:]}, &wg) + + wg.Wait() + require.NoError(b, wb.Flush()) + } + }) + b.Run("WriteBatchSingleThreadB", func(b *testing.B) { + tmpIndexDir, err := ioutil.TempDir("", "dgraph") + require.NoError(b, err) + defer os.RemoveAll(tmpIndexDir) + + dbOpts.Dir = tmpIndexDir + dbOpts.ValueDir = tmpIndexDir + + var db, err2 = badger.OpenManaged(dbOpts) + require.NoError(b, err2) + defer db.Close() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + var wg sync.WaitGroup + wg.Add(5) + wb := db.NewManagedWriteBatch() + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[:10000]}) + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[10001:20000]}) + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[20001:30000]}) + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[30001:40000]}) + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[40001:]}) + + wg.Wait() + require.NoError(b, wb.Flush()) + } + }) + b.Run("WriteBatchSingleThreadW", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) defer os.RemoveAll(tmpIndexDir) @@ -161,11 +234,11 @@ func BenchmarkWriter(b *testing.B) { var wg sync.WaitGroup wg.Add(5) wb := db.NewManagedWriteBatch() - go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[:10000]}, &wg) - go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[10001:20000]}, &wg) - go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[20001:30000]}, &wg) - go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[30001:40000]}, &wg) - go writeInBadger2(wb, &bpb.KVList{Kv: KVList.Kv[40001:]}, &wg) + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[:10000]}) + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[10001:20000]}) + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[20001:30000]}) + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[30001:40000]}) + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[40001:]}) wg.Wait() require.NoError(b, wb.Flush()) From bbf99d116830c4940e78671ccba63428e134e38d Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Tue, 5 May 2020 21:12:10 +0530 Subject: [PATCH 48/50] Remove wait groups from dingle threaded version --- posting/writer_test.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/posting/writer_test.go b/posting/writer_test.go index 0475e6ef284..2e263264402 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -203,8 +203,6 @@ func BenchmarkWriter(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - var wg sync.WaitGroup - wg.Add(5) wb := db.NewManagedWriteBatch() writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[:10000]}) writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[10001:20000]}) @@ -212,7 +210,6 @@ func BenchmarkWriter(b *testing.B) { writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[30001:40000]}) writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[40001:]}) - wg.Wait() require.NoError(b, wb.Flush()) } }) @@ -231,8 +228,6 @@ func BenchmarkWriter(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - var wg sync.WaitGroup - wg.Add(5) wb := db.NewManagedWriteBatch() writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[:10000]}) writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[10001:20000]}) @@ -240,7 +235,6 @@ func BenchmarkWriter(b *testing.B) { writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[30001:40000]}) writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[40001:]}) - wg.Wait() require.NoError(b, wb.Flush()) } }) From 722db3cbfbc45f339a22f7d254359a95e9ed473b Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Wed, 6 May 2020 11:51:21 +0530 Subject: [PATCH 49/50] Minor fixes --- go.mod | 3 +-- posting/index.go | 3 +-- posting/writer_test.go | 59 +++++++++++++++++++++--------------------- x/x.go | 1 + 4 files changed, 32 insertions(+), 34 deletions(-) diff --git a/go.mod b/go.mod index 4b82f021860..4d943b37222 100644 --- a/go.mod +++ b/go.mod @@ -67,5 +67,4 @@ require ( gopkg.in/DataDog/dd-trace-go.v1 v1.13.1 // indirect gopkg.in/ini.v1 v1.48.0 // indirect gopkg.in/yaml.v2 v2.2.4 -) -replace github.com/dgraph-io/badger/v2 => /home/alvis/go/src/github.com/dgraph-io/badger \ No newline at end of file +) \ No newline at end of file diff --git a/posting/index.go b/posting/index.go index ab7040b44c1..1d08760013e 100644 --- a/posting/index.go +++ b/posting/index.go @@ -638,8 +638,7 @@ func (r *rebuilder) Run(ctx context.Context) error { } glog.V(1).Infof("Rebuilding index for predicate %s: building temp index took: %v\n", r.attr, time.Since(start)) - fmt.Printf("Anurag: Rebuilding index for predicate %s: building temp index took: %v\n", - r.attr, time.Since(start)) + // Now we write all the created posting lists to disk. glog.V(1).Infof("Rebuilding index for predicate %s: writing index to badger", r.attr) start = time.Now() diff --git a/posting/writer_test.go b/posting/writer_test.go index 2e263264402..a331bdfc677 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -29,11 +29,13 @@ import ( "github.com/stretchr/testify/require" ) +var val = make([]byte, 128) + func BenchmarkWriter(b *testing.B) { createKVList := func() bpb.KVList { var KVList bpb.KVList - for i := 0; i < 50000; i++ { - n := &bpb.KV{Key: []byte(string(i)), Value: []byte(string(i)), Version: 5} + for i := 0; i < 5000000; i++ { + n := &bpb.KV{Key: []byte(string(i)), Value: val, Version: 5} KVList.Kv = append(KVList.Kv, n) } return KVList @@ -65,6 +67,7 @@ func BenchmarkWriter(b *testing.B) { if err := wb.Write(KVList); err != nil { panic(err) } + require.NoError(b, wb.Flush()) } // Resuses one writer for all threads @@ -132,7 +135,7 @@ func BenchmarkWriter(b *testing.B) { } }) //Multi threaded Batchwriter with thread contention in WriteBatch - b.Run("WriteBatchMultThreadB", func(b *testing.B) { + b.Run("WriteBatchMultThreadDiffWB", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) defer os.RemoveAll(tmpIndexDir) @@ -150,17 +153,17 @@ func BenchmarkWriter(b *testing.B) { var wg sync.WaitGroup wg.Add(5) - go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[:10000]}, &wg) - go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[10001:20000]}, &wg) - go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[20001:30000]}, &wg) - go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[30001:40000]}, &wg) - go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[40001:]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[:1000000]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[1000001:2000000]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[2000001:3000000]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[3000001:4000000]}, &wg) + go writeInBadgerMThreadsB(db, &bpb.KVList{Kv: KVList.Kv[4000001:]}, &wg) wg.Wait() } }) //Multi threaded Batchwriter with thread contention in SetEntry - b.Run("WriteBatchMultThreadW", func(b *testing.B) { + b.Run("WriteBatchMultThreadSameWB", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) defer os.RemoveAll(tmpIndexDir) @@ -178,17 +181,17 @@ func BenchmarkWriter(b *testing.B) { var wg sync.WaitGroup wg.Add(5) wb := db.NewManagedWriteBatch() - go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[:10000]}, &wg) - go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[10001:20000]}, &wg) - go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[20001:30000]}, &wg) - go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[30001:40000]}, &wg) - go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[40001:]}, &wg) + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[:1000000]}, &wg) + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[1000001:2000000]}, &wg) + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[2000001:3000000]}, &wg) + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[3000001:4000000]}, &wg) + go writeInBadgerMThreadsW(wb, &bpb.KVList{Kv: KVList.Kv[4000001:]}, &wg) wg.Wait() require.NoError(b, wb.Flush()) } }) - b.Run("WriteBatchSingleThreadB", func(b *testing.B) { + b.Run("WriteBatchSingleThreadDiffWB", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) defer os.RemoveAll(tmpIndexDir) @@ -203,17 +206,14 @@ func BenchmarkWriter(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - wb := db.NewManagedWriteBatch() - writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[:10000]}) - writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[10001:20000]}) - writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[20001:30000]}) - writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[30001:40000]}) - writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[40001:]}) - - require.NoError(b, wb.Flush()) + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[:1000000]}) + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[1000001:2000000]}) + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[2000001:3000000]}) + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[3000001:4000000]}) + writeInBadgerSingleThreadB(db, &bpb.KVList{Kv: KVList.Kv[4000001:]}) } }) - b.Run("WriteBatchSingleThreadW", func(b *testing.B) { + b.Run("WriteBatchSingleThreadSameWB", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) defer os.RemoveAll(tmpIndexDir) @@ -229,12 +229,11 @@ func BenchmarkWriter(b *testing.B) { for i := 0; i < b.N; i++ { wb := db.NewManagedWriteBatch() - writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[:10000]}) - writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[10001:20000]}) - writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[20001:30000]}) - writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[30001:40000]}) - writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[40001:]}) - + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[:1000000]}) + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[1000001:2000000]}) + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[2000001:3000000]}) + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[3000001:4000000]}) + writeInBadgerSingleThreadW(wb, &bpb.KVList{Kv: KVList.Kv[4000001:]}) require.NoError(b, wb.Flush()) } }) diff --git a/x/x.go b/x/x.go index b2bd9f1f770..95c1c04474e 100644 --- a/x/x.go +++ b/x/x.go @@ -41,6 +41,7 @@ import ( "github.com/dgraph-io/badger/v2/y" "github.com/dgraph-io/dgo/v200" "github.com/dgraph-io/dgo/v200/protos/api" + "github.com/golang/glog" "github.com/pkg/errors" "github.com/spf13/viper" From 78deec022c674e02a786a998eafad887f323487e Mon Sep 17 00:00:00 2001 From: Anurag Sharma Date: Wed, 6 May 2020 14:31:32 +0530 Subject: [PATCH 50/50] Update badger and edit comments in test file --- go.mod | 4 ++-- go.sum | 2 ++ posting/writer_test.go | 8 ++++---- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 4d943b37222..3b3f565ff4a 100644 --- a/go.mod +++ b/go.mod @@ -16,7 +16,7 @@ require ( github.com/blevesearch/segment v0.0.0-20160915185041-762005e7a34f // indirect github.com/blevesearch/snowballstem v0.0.0-20180110192139-26b06a2c243d // indirect github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd - github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430101140-5d19cc727d87 + github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200506081535-536fed1846d0 github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 github.com/dgraph-io/ristretto v0.0.2 github.com/dgrijalva/jwt-go v3.2.0+incompatible @@ -67,4 +67,4 @@ require ( gopkg.in/DataDog/dd-trace-go.v1 v1.13.1 // indirect gopkg.in/ini.v1 v1.48.0 // indirect gopkg.in/yaml.v2 v2.2.4 -) \ No newline at end of file +) diff --git a/go.sum b/go.sum index 7124d3ab3fa..b69ad0f8b7c 100644 --- a/go.sum +++ b/go.sum @@ -78,6 +78,8 @@ github.com/dgraph-io/badger v1.6.0 h1:DshxFxZWXUcO0xX476VJC07Xsr6ZCBVRHKZ93Oh7Ev github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430101140-5d19cc727d87 h1:FsCl1Yg3KVeYEzE7QlvpYg9WnySjLA5vbS9TlmEeUP8= github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200430101140-5d19cc727d87/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200506081535-536fed1846d0 h1:4VBIyLibX6qFfz6wSbEvp4RBfoKETvHfIln18ROLiHI= +github.com/dgraph-io/badger/v2 v2.0.1-rc1.0.20200506081535-536fed1846d0/go.mod h1:3KY8+bsP8wI0OEnQJAKpd4wIJW/Mm32yw2j/9FUVnIM= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453 h1:DTgOrw91nMIukDm/WEvdobPLl0LgeDd/JE66+24jBks= github.com/dgraph-io/dgo/v200 v200.0.0-20200401175452-e463f9234453/go.mod h1:Co+FwJrnndSrPORO8Gdn20dR7FPTfmXr0W/su0Ve/Ig= github.com/dgraph-io/ristretto v0.0.2-0.20200115201040-8f368f2f2ab3 h1:MQLRM35Pp0yAyBYksjbj1nZI/w6eyRY/mWoM1sFf4kU= diff --git a/posting/writer_test.go b/posting/writer_test.go index a331bdfc677..3518943379d 100644 --- a/posting/writer_test.go +++ b/posting/writer_test.go @@ -86,7 +86,7 @@ func BenchmarkWriter(b *testing.B) { KVList := createKVList() - //Vanilla TxnWriter + // Vanilla TxnWriter b.Run("TxnWriter", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) @@ -111,7 +111,7 @@ func BenchmarkWriter(b *testing.B) { } }) - //Single threaded BatchWriter + // Single threaded BatchWriter b.Run("WriteBatch1", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) @@ -134,7 +134,7 @@ func BenchmarkWriter(b *testing.B) { require.NoError(b, wb.Flush()) } }) - //Multi threaded Batchwriter with thread contention in WriteBatch + // Multi threaded Batchwriter with thread contention in WriteBatch b.Run("WriteBatchMultThreadDiffWB", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err) @@ -162,7 +162,7 @@ func BenchmarkWriter(b *testing.B) { } }) - //Multi threaded Batchwriter with thread contention in SetEntry + // Multi threaded Batchwriter with thread contention in SetEntry b.Run("WriteBatchMultThreadSameWB", func(b *testing.B) { tmpIndexDir, err := ioutil.TempDir("", "dgraph") require.NoError(b, err)