-
Notifications
You must be signed in to change notification settings - Fork 1.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Incremental Rollup and Tablet Size Calculation #4972
Merged
Merged
Changes from 8 commits
Commits
Show all changes
25 commits
Select commit
Hold shift + click to select a range
5f07dc9
Bring incremental rollup commit
parasssh 75233b5
Incremental Rollups -- let's track versions of keys.
manishrjain 76d2838
Merge master
manishrjain fe10d20
Build works with master merge
manishrjain 527a082
Fix map ini
manishrjain 50e7544
Some evidence to show it's not just writes, but write in the range of…
manishrjain 700b54b
Some more tries to understand the behavior
manishrjain 53b1ae5
Merge branch 'master' into mrjn/incremental
55de90c
Using Martin's fix.
manishrjain eabfe9a
Make rollup goroutine work with the task tracking
manishrjain 3bbde8c
Cleaned up a bit of the log messages by using strings and converting …
manishrjain 401d6a7
Merge branch 'master' into mrjn/incremental
7f2a9d7
Merge branch 'mrjn/incremental' of https://github.com/dgraph-io/dgrap…
1656b38
Fix an issue where we were only sending the keys for rollup which did…
manishrjain ace03ac
Switch tablet size calculation to use DB.Tables method.
manishrjain 000727f
Ensure that when closing, all registered tasks are properly stopped.
manishrjain 2743576
Revert unnecessary change in list.go
manishrjain ea1081b
Better handle if another task is going on.
manishrjain 280c634
Call rollup even if we find one delta posting
manishrjain 42fb5e8
Add a note about iterating over tables
manishrjain 71d89dd
fix GoLang CI comments
a4a7c59
Merge branch 'mrjn/incremental' of https://github.com/dgraph-io/dgrap…
d4cea37
Revert CI changes done by Paras. Add comment about why.
manishrjain b4a07c0
Remove question, add comment
manishrjain 50e5b72
Merge master
manishrjain File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,15 +19,13 @@ package worker | |
import ( | ||
"bytes" | ||
"context" | ||
"encoding/hex" | ||
"fmt" | ||
"math" | ||
"sort" | ||
"sync" | ||
"sync/atomic" | ||
"time" | ||
|
||
humanize "github.com/dustin/go-humanize" | ||
"github.com/dustin/go-humanize" | ||
"github.com/golang/glog" | ||
"github.com/pkg/errors" | ||
"go.etcd.io/etcd/raft" | ||
|
@@ -38,7 +36,6 @@ import ( | |
"go.opencensus.io/tag" | ||
otrace "go.opencensus.io/trace" | ||
|
||
"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/dgraph/conn" | ||
|
@@ -83,7 +80,7 @@ func (id op) String() string { | |
case opIndexing: | ||
return "opIndexing" | ||
default: | ||
return "unknown" | ||
return "opUnknown" | ||
} | ||
} | ||
|
||
|
@@ -114,7 +111,6 @@ func (n *node) startTask(id op) (*y.Closer, error) { | |
} | ||
|
||
closer := y.NewCloser(1) | ||
|
||
switch id { | ||
case opRollup: | ||
if len(n.ops) > 0 { | ||
|
@@ -123,10 +119,12 @@ func (n *node) startTask(id op) (*y.Closer, error) { | |
go posting.IncrRollup.Process(closer) | ||
|
||
case opSnapshot, opIndexing: | ||
if roCloser, has := n.ops[opRollup]; has { | ||
roCloser.SignalAndWait() | ||
} else if len(n.ops) > 0 { | ||
return nil, errors.Errorf("another operation is already running") | ||
for otherId, otherCloser := range n.ops { | ||
if otherId == opRollup { | ||
otherCloser.SignalAndWait() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shouldn't we pause rollup only when we know for sure there are no other operation. Let's say, |
||
} else { | ||
return nil, errors.Errorf("operation %s is already running", otherId) | ||
} | ||
} | ||
default: | ||
glog.Errorf("Got an unhandled operation %s. Ignoring...", id) | ||
|
@@ -152,6 +150,23 @@ func (n *node) waitForTask(id op) { | |
closer.Wait() | ||
} | ||
|
||
func (n *node) stopAllTasks() { | ||
defer n.closer.Done() // CLOSER:1 | ||
<-n.closer.HasBeenClosed() | ||
|
||
var closers []*y.Closer | ||
n.opsLock.Lock() | ||
for _, closer := range n.ops { | ||
closers = append(closers, closer) | ||
} | ||
n.opsLock.Unlock() | ||
|
||
for _, closer := range closers { | ||
closer.SignalAndWait() | ||
} | ||
glog.Infof("Stopped all ongoing registered tasks.") | ||
} | ||
|
||
// Now that we apply txn updates via Raft, waiting based on Txn timestamps is | ||
// sufficient. We don't need to wait for proposals to be applied. | ||
|
||
|
@@ -174,7 +189,7 @@ func newNode(store *raftwal.DiskStorage, gid uint32, id uint64, myAddr string) * | |
// to maintain quorum health. | ||
applyCh: make(chan []*pb.Proposal, 1000), | ||
elog: trace.NewEventLog("Dgraph", "ApplyCh"), | ||
closer: y.NewCloser(3), // Matches CLOSER:1 | ||
closer: y.NewCloser(4), // Matches CLOSER:1 | ||
ops: make(map[op]*y.Closer), | ||
} | ||
return n | ||
|
@@ -526,19 +541,15 @@ func (n *node) applyCommitted(proposal *pb.Proposal) error { | |
|
||
func (n *node) processTabletSizes() { | ||
defer n.closer.Done() // CLOSER:1 | ||
tick := time.NewTicker(5 * time.Minute) // Rolling up once every 5 minutes seems alright. | ||
tick := time.NewTicker(5 * time.Minute) // Once every 5 minutes seems alright. | ||
defer tick.Stop() | ||
|
||
for { | ||
select { | ||
case <-n.closer.HasBeenClosed(): | ||
return | ||
case <-tick.C: | ||
if err := n.calcTabletSizes(); err != nil { | ||
// If we encounter error here, we don't need to do anything about | ||
// it. Just let the user know. | ||
glog.Errorf("Error while calculating tablet sizes: %v", err) | ||
} | ||
n.calculateTabletSizes() | ||
} | ||
} | ||
} | ||
|
@@ -1144,98 +1155,63 @@ func listWrap(kv *bpb.KV) *bpb.KVList { | |
return &bpb.KVList{Kv: []*bpb.KV{kv}} | ||
} | ||
|
||
// calcTabletSizes updates the tablet sizes for the keys. | ||
func (n *node) calcTabletSizes() error { | ||
// calculateTabletSizes updates the tablet sizes for the keys. | ||
func (n *node) calculateTabletSizes() { | ||
if !n.AmLeader() { | ||
// Only leader needs to calculate the tablet sizes. | ||
return nil | ||
// Only leader sends the tablet size updates to Zero. No one else does. | ||
return | ||
} | ||
var total int64 | ||
tablets := make(map[string]*pb.Tablet) | ||
|
||
m := new(sync.Map) | ||
|
||
addTo := func(key []byte, delta int64) { | ||
pk, err := x.Parse(key) | ||
|
||
// Type keys should not count for tablet size calculations. | ||
if pk.IsType() { | ||
return | ||
} | ||
|
||
tableInfos := pstore.Tables(false) | ||
glog.V(2).Infof("Calculating tablet sizes. Found %d tables\n", len(tableInfos)) | ||
for _, tinfo := range tableInfos { | ||
left, err := x.Parse(tinfo.Left) | ||
if err != nil { | ||
glog.Errorf("Error while parsing key %s: %v", hex.Dump(key), err) | ||
return | ||
glog.V(2).Infof("Unable to parse key: %v", err) | ||
continue | ||
} | ||
val, ok := m.Load(pk.Attr) | ||
if !ok { | ||
sz := new(int64) | ||
val, _ = m.LoadOrStore(pk.Attr, sz) | ||
right, err := x.Parse(tinfo.Right) | ||
if err != nil { | ||
glog.V(2).Infof("Unable to parse key: %v", err) | ||
continue | ||
} | ||
size := val.(*int64) | ||
atomic.AddInt64(size, delta) | ||
} | ||
|
||
// TODO: Remove all this. | ||
stream := pstore.NewStreamAt(math.MaxUint64) | ||
stream.LogPrefix = "Tablet Size Calculation" | ||
stream.Prefix = []byte{x.DefaultPrefix} | ||
stream.ChooseKey = func(item *badger.Item) bool { | ||
switch item.UserMeta() { | ||
case posting.BitSchemaPosting, posting.BitCompletePosting, posting.BitEmptyPosting: | ||
addTo(item.Key(), item.EstimatedSize()) | ||
return false | ||
case x.ByteUnused: | ||
return false | ||
default: | ||
// not doing rollups anymore. | ||
return false | ||
if left.Attr != right.Attr { | ||
// Skip all tables not fully owned by one predicate. | ||
// We could later specifically iterate over these tables to get their estimated sizes. | ||
glog.V(2).Info("Skipping table not owned by one predicate") | ||
continue | ||
} | ||
} | ||
|
||
stream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { | ||
return nil, nil // no-op | ||
} | ||
stream.Send = func(list *bpb.KVList) error { | ||
return nil | ||
} | ||
ctx, cancel := context.WithCancel(context.Background()) | ||
defer cancel() | ||
if err := stream.Orchestrate(ctx); err != nil { | ||
return err | ||
} | ||
|
||
// Only leader sends the tablet size updates to Zero. No one else does. | ||
// doSendMembership is also being concurrently called from another goroutine. | ||
go func() { | ||
tablets := make(map[string]*pb.Tablet) | ||
var total int64 | ||
m.Range(func(key, val interface{}) bool { | ||
pred := key.(string) | ||
size := atomic.LoadInt64(val.(*int64)) | ||
pred := left.Attr | ||
if tablet, ok := tablets[pred]; ok { | ||
tablet.Space += int64(tinfo.EstimatedSz) | ||
} else { | ||
tablets[pred] = &pb.Tablet{ | ||
GroupId: n.gid, | ||
Predicate: pred, | ||
Space: size, | ||
Space: int64(tinfo.EstimatedSz), | ||
} | ||
total += size | ||
return true | ||
}) | ||
// Update Zero with the tablet sizes. If Zero sees a tablet which does not belong to | ||
// this group, it would send instruction to delete that tablet. There's an edge case | ||
// here if the followers are still running Rollup, and happen to read a key before and | ||
// write after the tablet deletion, causing that tablet key to resurface. Then, only the | ||
// follower would have that key, not the leader. | ||
// However, if the follower then becomes the leader, we'd be able to get rid of that | ||
// key then. Alternatively, we could look into cancelling the Rollup if we see a | ||
// predicate deletion. | ||
if err := groups().doSendMembership(tablets); err != nil { | ||
glog.Warningf("While sending membership to Zero. Error: %v", err) | ||
} else { | ||
glog.V(2).Infof("Sent tablet size update to Zero. Total size: %s", | ||
humanize.Bytes(uint64(total))) | ||
} | ||
}() | ||
|
||
return nil | ||
total += int64(tinfo.EstimatedSz) | ||
} | ||
if len(tablets) == 0 { | ||
return | ||
} | ||
// Update Zero with the tablet sizes. If Zero sees a tablet which does not belong to | ||
// this group, it would send instruction to delete that tablet. There's an edge case | ||
// here if the followers are still running Rollup, and happen to read a key before and | ||
// write after the tablet deletion, causing that tablet key to resurface. Then, only the | ||
// follower would have that key, not the leader. | ||
// However, if the follower then becomes the leader, we'd be able to get rid of that | ||
// key then. Alternatively, we could look into cancelling the Rollup if we see a | ||
// predicate deletion. | ||
if err := groups().doSendMembership(tablets); err != nil { | ||
glog.Warningf("While sending membership to Zero. Error: %v", err) | ||
} else { | ||
glog.V(2).Infof("Sent tablet size update to Zero. Total size: %s", | ||
humanize.Bytes(uint64(total))) | ||
} | ||
} | ||
|
||
var errNoConnection = errors.New("No connection exists") | ||
|
@@ -1570,6 +1546,7 @@ func (n *node) InitAndStartNode() { | |
go n.processApplyCh() | ||
go n.BatchAndSendMessages() | ||
x.Check2(n.startTask(opRollup)) | ||
go n.stopAllTasks() | ||
go n.Run() | ||
} | ||
|
||
|
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Didn't we want to do this if deltaCount >= 2?